From eb8ba4eaa9883a8b20d89f1dc6e7dce760b145c8 Mon Sep 17 00:00:00 2001 From: DarkWanderer Date: Sat, 17 Apr 2021 20:02:27 +0300 Subject: [PATCH 0001/1260] Adding integration test for ClickHouse.Client --- docker/images.json | 4 + .../test/integration/dotnet_client/Dockerfile | 8 ++ .../test/integration/dotnet_client/Program.cs | 73 +++++++++++++++++++ .../dotnet_client/clickhouse.test.csproj | 13 ++++ .../compose/docker_compose_dotnet_client.yml | 6 ++ .../integration/runner/dockerd-entrypoint.sh | 1 + tests/integration/runner | 2 + 7 files changed, 107 insertions(+) create mode 100644 docker/test/integration/dotnet_client/Dockerfile create mode 100644 docker/test/integration/dotnet_client/Program.cs create mode 100644 docker/test/integration/dotnet_client/clickhouse.test.csproj create mode 100644 docker/test/integration/runner/compose/docker_compose_dotnet_client.yml diff --git a/docker/images.json b/docker/images.json index e2e22468596..32517698437 100644 --- a/docker/images.json +++ b/docker/images.json @@ -111,6 +111,10 @@ "name": "yandex/clickhouse-mysql-golang-client", "dependent": [] }, + "docker/test/integration/dotnet_client": { + "name": "yandex/clickhouse-dotnet-client", + "dependent": [] + }, "docker/test/integration/mysql_java_client": { "name": "yandex/clickhouse-mysql-java-client", "dependent": [] diff --git a/docker/test/integration/dotnet_client/Dockerfile b/docker/test/integration/dotnet_client/Dockerfile new file mode 100644 index 00000000000..f29df349f2f --- /dev/null +++ b/docker/test/integration/dotnet_client/Dockerfile @@ -0,0 +1,8 @@ +FROM mcr.microsoft.com/dotnet/sdk:3.1 + +WORKDIR /client +COPY *.cs *.csproj /client + +ARG VERSION=3.1.0.379 +RUN dotnet add package ClickHouse.Client -v ${VERSION} +RUN dotnet run diff --git a/docker/test/integration/dotnet_client/Program.cs b/docker/test/integration/dotnet_client/Program.cs new file mode 100644 index 00000000000..f8db8637aad --- /dev/null +++ b/docker/test/integration/dotnet_client/Program.cs @@ -0,0 +1,73 @@ +using System; +using System.Threading.Tasks; +using ClickHouse.Client.ADO; +using ClickHouse.Client.Utility; + +namespace clickhouse.test +{ + class Program + { + static async Task Main(string[] args) + { + try + { + using var connection = new ClickHouseConnection(GetConnectionString(args)); + + await connection.ExecuteStatementAsync("CREATE DATABASE IF NOT EXISTS test"); + await connection.ExecuteStatementAsync("TRUNCATE TABLE IF EXISTS test.dotnet_test"); + await connection.ExecuteStatementAsync("CREATE TABLE IF NOT EXISTS test.dotnet_test (`age` Int32, `name` String) Engine = Memory"); + + using var command = connection.CreateCommand(); + command.AddParameter("name", "Linus Torvalds"); + command.AddParameter("age", 51); + command.CommandText = "INSERT INTO test.dotnet_test VALUES({age:Int32}, {name:String})"; + await command.ExecuteNonQueryAsync(); + + using var result = await connection.ExecuteReaderAsync("SELECT * FROM test.dotnet_test"); + while (result.Read()) + { + var values = new object[result.FieldCount]; + result.GetValues(values); + } + } + catch (Exception e) + { + Console.Error.WriteLine(e); + Environment.ExitCode = 1; + } + } + + private static string GetConnectionString(string[] args) + { + var builder = new ClickHouseConnectionStringBuilder(); + int i = 0; + while (i < args.Length) + { + switch (args[i]) + { + case "--host": + builder.Host = args[++i]; + break; + case "--port": + builder.Port = UInt16.Parse(args[++i]); + break; + case "--user": + builder.Username = args[++i]; + break; + case "--password": + builder.Password = args[++i]; + break; + case "--database": + builder.Database = args[++i]; + break; + default: + i++; + break; + } + } + return builder.ToString(); + } + + private static string selectSql = @"SELECT NULL, toInt8(-8), toUInt8(8), toInt16(-16), toUInt16(16), toInt16(-32), toUInt16(32), toInt64(-64), toUInt64(64), toFloat32(32e6), toFloat32(-32e6), toFloat64(64e6), toFloat64(-64e6), 'TestString', toFixedString('ASD',3), toFixedString('ASD',5), toUUID('00000000-0000-0000-0000-000000000000'), toUUID('61f0c404-5cb3-11e7-907b-a6006ad3dba0'), toIPv4('1.2.3.4'), toIPv4('255.255.255.255'), CAST('a', 'Enum(\'a\' = 1, \'b\' = 2)'), CAST('a', 'Enum8(\'a\' = -1, \'b\' = 127)'), CAST('a', 'Enum16(\'a\' = -32768, \'b\' = 32767)'), array(1, 2, 3), array('a', 'b', 'c'), array(1, 2, NULL), toInt32OrNull('123'), toInt32OrNull(NULL), CAST(NULL AS Nullable(DateTime)), CAST(NULL AS LowCardinality(Nullable(String))), toLowCardinality('lowcardinality'), tuple(1, 'a', 8), tuple(123, tuple(5, 'a', 7)), toDateOrNull('1999-11-12'), toDateTime('1988-08-28 11:22:33'), toDateTime64('2043-03-01 18:34:04.4444444', 9), toDecimal32(123.45, 3), toDecimal32(-123.45, 3), toDecimal64(1.2345, 7), toDecimal64(-1.2345, 7), toDecimal128(12.34, 9), toDecimal128(-12.34, 9), toIPv6('2001:0db8:85a3:0000:0000:8a2e:0370:7334')"; + } +} diff --git a/docker/test/integration/dotnet_client/clickhouse.test.csproj b/docker/test/integration/dotnet_client/clickhouse.test.csproj new file mode 100644 index 00000000000..2ff2340ca4c --- /dev/null +++ b/docker/test/integration/dotnet_client/clickhouse.test.csproj @@ -0,0 +1,13 @@ + + + + Exe + netcoreapp3.1 + + + + + + + + diff --git a/docker/test/integration/runner/compose/docker_compose_dotnet_client.yml b/docker/test/integration/runner/compose/docker_compose_dotnet_client.yml new file mode 100644 index 00000000000..4cc494d094a --- /dev/null +++ b/docker/test/integration/runner/compose/docker_compose_dotnet_client.yml @@ -0,0 +1,6 @@ +version: '2.3' +services: + dotnet1: + image: yandex/clickhouse-dotnet-client:${DOCKER_DOTNET_CLIENT_TAG:-latest} + # to keep container running + command: sleep infinity diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index bda6f5a719d..315c50cac7b 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -24,6 +24,7 @@ export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge export CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH=/clickhouse-library-bridge export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} +export DOCKER_DOTNET_CLIENT_TAG=${DOCKER_DOTNET_CLIENT_TAG:=latest} export DOCKER_MYSQL_JAVA_CLIENT_TAG=${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest} export DOCKER_MYSQL_JS_CLIENT_TAG=${DOCKER_MYSQL_JS_CLIENT_TAG:=latest} export DOCKER_MYSQL_PHP_CLIENT_TAG=${DOCKER_MYSQL_PHP_CLIENT_TAG:=latest} diff --git a/tests/integration/runner b/tests/integration/runner index e89e10fbc21..ce1f361a95d 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -170,6 +170,8 @@ if __name__ == "__main__": [image, tag] = img_tag.split(":") if image == "yandex/clickhouse-mysql-golang-client": env_tags += "-e {}={} ".format("DOCKER_MYSQL_GOLANG_CLIENT_TAG", tag) + elif image == "yandex/clickhouse-dotnet-client": + env_tags += "-e {}={} ".format("DOCKER_DOTNET_CLIENT_TAG", tag) elif image == "yandex/clickhouse-mysql-java-client": env_tags += "-e {}={} ".format("DOCKER_MYSQL_JAVA_CLIENT_TAG", tag) elif image == "yandex/clickhouse-mysql-js-client": From 86117e32ce85d6e2f02ccba97054daf1836a13ad Mon Sep 17 00:00:00 2001 From: DarkWanderer Date: Sun, 18 Apr 2021 09:04:12 +0300 Subject: [PATCH 0002/1260] Added pytest entry --- tests/integration/test_mysql_protocol/test.py | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 43daeebeaf5..efff3b7c186 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -65,6 +65,12 @@ def golang_container(): ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.from_env().containers.get(cluster.project_name + '_golang1_1') +@pytest.fixture(scope='module') +def dotnet_container(): + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_dotnet_client.yml') + run_and_check( + ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) + yield docker.from_env().containers.get(cluster.project_name + '_dotnet1_1') @pytest.fixture(scope='module') def php_container(): @@ -416,6 +422,15 @@ def test_golang_client(server_address, golang_container): assert stdout == reference +def test_dotnet_client(server_address, dotnet_container): + code, (stdout, stderr) = dotnet_container.exec_run( + 'dotnet run --host {host} --port {port} --user default --password 123 --database default' + .format(host=server_address, port=server_port), demux=True) + + assert code == 0 + assert stdout == reference + + def test_php_client(server_address, php_container): # type: (str, Container) -> None code, (stdout, stderr) = php_container.exec_run( From fae887e6b603d6ed8cfaf8c5d469e559c31c3cf0 Mon Sep 17 00:00:00 2001 From: DarkWanderer Date: Wed, 19 May 2021 15:59:12 +0300 Subject: [PATCH 0003/1260] Fixed issue (docker/podman different behavior), added .gitignore --- docker/test/integration/dotnet_client/.gitignore | 2 ++ docker/test/integration/dotnet_client/Dockerfile | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) create mode 100644 docker/test/integration/dotnet_client/.gitignore diff --git a/docker/test/integration/dotnet_client/.gitignore b/docker/test/integration/dotnet_client/.gitignore new file mode 100644 index 00000000000..cd42ee34e87 --- /dev/null +++ b/docker/test/integration/dotnet_client/.gitignore @@ -0,0 +1,2 @@ +bin/ +obj/ diff --git a/docker/test/integration/dotnet_client/Dockerfile b/docker/test/integration/dotnet_client/Dockerfile index f29df349f2f..bbca5ec58e5 100644 --- a/docker/test/integration/dotnet_client/Dockerfile +++ b/docker/test/integration/dotnet_client/Dockerfile @@ -1,7 +1,7 @@ FROM mcr.microsoft.com/dotnet/sdk:3.1 WORKDIR /client -COPY *.cs *.csproj /client +COPY *.cs *.csproj /client/ ARG VERSION=3.1.0.379 RUN dotnet add package ClickHouse.Client -v ${VERSION} From 40ce278af5a5e50a37fb358e8b27832df2f548a4 Mon Sep 17 00:00:00 2001 From: antikvist Date: Thu, 10 Jun 2021 00:14:36 +0300 Subject: [PATCH 0004/1260] add new aggregate functions --- ...ggregateFunctionContingencyCoefficient.cpp | 54 +++++ .../AggregateFunctionCramersV.cpp | 27 +++ .../AggregateFunctionCramersV.h | 198 ++++++++++++++++++ ...ggregateFunctionCramersVBiasCorrection.cpp | 59 ++++++ .../AggregateFunctionTheilsU.cpp | 61 ++++++ .../registerAggregateFunctions.cpp | 8 + 6 files changed, 407 insertions(+) create mode 100644 src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionCramersV.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionCramersV.h create mode 100644 src/AggregateFunctions/AggregateFunctionCramersVBiasCorrection.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionTheilsU.cpp diff --git a/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp b/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp new file mode 100644 index 00000000000..269c961d6c5 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp @@ -0,0 +1,54 @@ +#include +#include +#include +#include +#include "registerAggregateFunctions.h" +#include + + +namespace DB +{ +namespace +{ + + +struct ContingencyData : public AggregateFunctionCramersVData +{ + Float64 get_result() const + { + Float64 phi = 0.0; + for (const auto & cell : pairs) { + UInt128 hash_pair = cell.getKey(); + UInt64 count_of_pair_tmp = cell.getMapped(); + Float64 count_of_pair = Float64(count_of_pair_tmp); + UInt64 hash1 = (hash_pair << 64 >> 64); + UInt64 hash2 = (hash_pair >> 64); + + UInt64 count1_tmp = n_i.find(hash1)->getMapped(); + UInt64 count2_tmp = n_j.find(hash2)->getMapped(); + Float64 count1 = static_cast(count1_tmp); + Float64 count2 = Float64(count2_tmp); + + phi += ((count_of_pair * count_of_pair / (count1 * count2) * cur_size) + - 2 * count_of_pair + (count1 * count2 / cur_size)); + } + phi /= cur_size; + return sqrt(phi / (phi + cur_size)); + } +}; + + +AggregateFunctionPtr createAggregateFunctionContingencyCoefficient(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) +{ + assertNoParameters(name, parameters); + return std::make_shared>(argument_types); +} + +} + +void registerAggregateFunctionContingencyCoefficient(AggregateFunctionFactory & factory) +{ + factory.registerFunction("ContingencyCoefficient", createAggregateFunctionContingencyCoefficient); +} + +} diff --git a/src/AggregateFunctions/AggregateFunctionCramersV.cpp b/src/AggregateFunctions/AggregateFunctionCramersV.cpp new file mode 100644 index 00000000000..b04c6a37056 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionCramersV.cpp @@ -0,0 +1,27 @@ +#include +#include +#include +#include +#include "registerAggregateFunctions.h" +#include + +namespace DB +{ +namespace +{ + +AggregateFunctionPtr createAggregateFunctionCramersV(const std::string & name, const DataTypes & argument_types, + const Array & parameters, const Settings *) +{ + assertNoParameters(name, parameters); + return std::make_shared>(argument_types); +} + +} + +void registerAggregateFunctionCramersV(AggregateFunctionFactory & factory) +{ + factory.registerFunction("CramersV", createAggregateFunctionCramersV); +} + +} diff --git a/src/AggregateFunctions/AggregateFunctionCramersV.h b/src/AggregateFunctions/AggregateFunctionCramersV.h new file mode 100644 index 00000000000..ed41737fa2b --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionCramersV.h @@ -0,0 +1,198 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + + + struct AggregateFunctionCramersVData + { + size_t cur_size = 0; + HashMap n_i; + HashMap n_j; + HashMap pairs; + + + void add(UInt64 hash1, UInt64 hash2) + { + cur_size += 1; + n_i[hash1] += 1; + n_j[hash2] += 1; + + UInt128 hash_pair = hash1 | (static_cast(hash2) << 64); + pairs[hash_pair] += 1; + + } + + void merge(const AggregateFunctionCramersVData &other) + { + cur_size += other.cur_size; + + for (const auto& pair : other.n_i) { + UInt64 hash1 = pair.getKey(); + UInt64 count = pair.getMapped(); + n_i[hash1] += count; + } + for (const auto& pair : other.n_j) { + UInt64 hash1 = pair.getKey(); + UInt64 count = pair.getMapped(); + n_j[hash1] += count; + } + for (const auto& pair : other.pairs) { + UInt128 hash1 = pair.getKey(); + UInt64 count = pair.getMapped(); + pairs[hash1] += count; + } + } + + void serialize(WriteBuffer &buf) const + { + writeBinary(cur_size, buf); + n_i.write(buf); + n_j.write(buf); + pairs.write(buf); + } + + void deserialize(ReadBuffer &buf) + { + readBinary(cur_size, buf); + n_i.read(buf); + n_j.read(buf); + pairs.read(buf); + } + + Float64 get_result() const + { + Float64 phi = 0.0; + for (const auto & cell : pairs) { + UInt128 hash_pair = cell.getKey(); + UInt64 count_of_pair_tmp = cell.getMapped(); + Float64 count_of_pair = Float64(count_of_pair_tmp); + UInt64 hash1 = (hash_pair << 64 >> 64); + UInt64 hash2 = (hash_pair >> 64); + + UInt64 count1_tmp = n_i.find(hash1)->getMapped(); + UInt64 count2_tmp = n_j.find(hash2)->getMapped(); + Float64 count1 = static_cast(count1_tmp); + Float64 count2 = Float64(count2_tmp); + + phi += ((count_of_pair * count_of_pair / (count1 * count2) * cur_size) + - 2 * count_of_pair + (count1 * count2 / cur_size)); + } + phi /= cur_size; + + UInt64 q = std::min(n_i.size(), n_j.size()); + phi /= (q - 1); + return sqrt(phi); + + } + }; + + template + + class AggregateFunctionCramersV : public + IAggregateFunctionDataHelper< + Data, + AggregateFunctionCramersV + > + { + + public: + AggregateFunctionCramersV( + const DataTypes & arguments + ): + IAggregateFunctionDataHelper< + Data, + AggregateFunctionCramersV + > ({arguments}, {}) + { + // notice: arguments has been in factory + } + + String getName() const override + { + return "CramersV"; + } + + bool allocatesMemoryInArena() const override { return false; } + + DataTypePtr getReturnType() const override + { + return std::make_shared>(); + } + + void add( + AggregateDataPtr __restrict place, + const IColumn ** columns, + size_t row_num, + Arena * + ) const override + { + UInt64 hash1 = UniqVariadicHash::apply(1, columns, row_num); + UInt64 hash2 = UniqVariadicHash::apply(1, columns + 1, row_num); + + this->data(place).add(hash1, hash2); + } + + void merge( + AggregateDataPtr __restrict place, + ConstAggregateDataPtr rhs, Arena * + ) const override + { + this->data(place).merge(this->data(rhs)); + } + + void serialize( + ConstAggregateDataPtr __restrict place, + WriteBuffer & buf + ) const override + { + this->data(place).serialize(buf); + } + + void deserialize( + AggregateDataPtr __restrict place, + ReadBuffer & buf, Arena * + ) const override + { + this->data(place).deserialize(buf); + } + + void insertResultInto( + AggregateDataPtr __restrict place, + IColumn & to, + Arena * + ) const override + { + Float64 result = this->data(place).get_result(); +// std::cerr << "cur_size" << this->data(place).cur_size << '\n'; +// std::cerr << "n_i size" << this->data(place).n_i.size() << '\n'; +// std::cerr << "n_j size" << this->data(place).n_j.size() << '\n'; +// std::cerr << "pair size " << this->data(place).pairs.size() << '\n'; +// std::cerr << "result " << result << '\n'; + + auto & column = static_cast &>(to); + column.getData().push_back(result); + } + + }; + +} diff --git a/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrection.cpp b/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrection.cpp new file mode 100644 index 00000000000..4aa9ba0a8fd --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrection.cpp @@ -0,0 +1,59 @@ +#include +#include +#include +#include +#include "registerAggregateFunctions.h" +#include + + +namespace DB +{ +namespace +{ + + +struct BiasCorrectionData : public AggregateFunctionCramersVData +{ + Float64 get_result() const + { + Float64 phi = 0.0; + for (const auto & cell : pairs) { + UInt128 hash_pair = cell.getKey(); + UInt64 count_of_pair_tmp = cell.getMapped(); + Float64 count_of_pair = Float64(count_of_pair_tmp); + UInt64 hash1 = (hash_pair << 64 >> 64); + UInt64 hash2 = (hash_pair >> 64); + + UInt64 count1_tmp = n_i.find(hash1)->getMapped(); + UInt64 count2_tmp = n_j.find(hash2)->getMapped(); + Float64 count1 = static_cast(count1_tmp); + Float64 count2 = Float64(count2_tmp); + + phi += ((count_of_pair * count_of_pair / (count1 * count2) * cur_size) + - 2 * count_of_pair + (count1 * count2 / cur_size)); + } + phi /= cur_size; + Float64 answ = std::max(0.0, phi - ((static_cast(n_i.size()) - 1) * (static_cast(n_j.size()) - 1) / (cur_size - 1))); + Float64 k = n_i.size() - (static_cast(n_i.size()) - 1) * (static_cast(n_i.size()) - 1) / (cur_size - 1); + Float64 r = n_j.size() - (static_cast(n_j.size()) - 1) * (static_cast(n_j.size()) - 1) / (cur_size - 1); + Float64 q = std::min(k, r); + answ /= (q - 1); + return sqrt(answ); + } +}; + + +AggregateFunctionPtr createAggregateFunctionCramersVBiasCorrection(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) +{ + assertNoParameters(name, parameters); + return std::make_shared>(argument_types); +} + +} + +void registerAggregateFunctionCramersVBiasCorrection(AggregateFunctionFactory & factory) +{ + factory.registerFunction("CramersVBiasCorrection", createAggregateFunctionCramersVBiasCorrection); +} + +} diff --git a/src/AggregateFunctions/AggregateFunctionTheilsU.cpp b/src/AggregateFunctions/AggregateFunctionTheilsU.cpp new file mode 100644 index 00000000000..954029bb3d6 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionTheilsU.cpp @@ -0,0 +1,61 @@ +#include +#include +#include +#include +#include "registerAggregateFunctions.h" +#include + + +namespace DB +{ +namespace +{ + + + struct TheilsUData : public AggregateFunctionCramersVData +{ + Float64 get_result() const + { + Float64 h_x = 0.0; + for (const auto & cell : n_i) { + UInt64 count_x_tmp = cell.getMapped(); + Float64 count_x = Float64(count_x_tmp); + h_x += (count_x / cur_size) * (log(count_x / cur_size)); + } + + + Float64 dep = 0.0; + for (const auto & cell : pairs) { + UInt128 hash_pair = cell.getKey(); + UInt64 count_of_pair_tmp = cell.getMapped(); + Float64 count_of_pair = Float64(count_of_pair_tmp); + + UInt64 hash2 = (hash_pair >> 64); + + UInt64 count2_tmp = n_j.find(hash2)->getMapped(); + Float64 count2 = Float64 (count2_tmp); + + dep += (count_of_pair / cur_size) * log(count_of_pair / count2); + } + + dep -= h_x; + dep /= h_x; + return dep; + } +}; + + +AggregateFunctionPtr createAggregateFunctionTheilsU(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) +{ + assertNoParameters(name, parameters); + return std::make_shared>(argument_types); +} + +} + +void registerAggregateFunctionTheilsU(AggregateFunctionFactory & factory) +{ + factory.registerFunction("TheilsU", createAggregateFunctionTheilsU); +} + +} diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index db6d8eb75bc..1b391eafa7e 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -48,6 +48,10 @@ void registerAggregateFunctionRankCorrelation(AggregateFunctionFactory &); void registerAggregateFunctionMannWhitney(AggregateFunctionFactory &); void registerAggregateFunctionWelchTTest(AggregateFunctionFactory &); void registerAggregateFunctionStudentTTest(AggregateFunctionFactory &); +void registerAggregateFunctionCramersV(AggregateFunctionFactory &); +void registerAggregateFunctionTheilsU(AggregateFunctionFactory &); +void registerAggregateFunctionContingencyCoefficient(AggregateFunctionFactory &); +void registerAggregateFunctionCramersVBiasCorrection(AggregateFunctionFactory &); class AggregateFunctionCombinatorFactory; void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &); @@ -95,6 +99,10 @@ void registerAggregateFunctions() registerAggregateFunctionUniqUpTo(factory); registerAggregateFunctionTopK(factory); registerAggregateFunctionsBitwise(factory); + registerAggregateFunctionCramersV(factory); + registerAggregateFunctionTheilsU(factory); + registerAggregateFunctionContingencyCoefficient(factory); + registerAggregateFunctionCramersVBiasCorrection(factory); #if !defined(ARCADIA_BUILD) registerAggregateFunctionsBitmap(factory); #endif From 2f54ed2c19344c3feae5555565ec6945060278a5 Mon Sep 17 00:00:00 2001 From: antikvist Date: Fri, 11 Jun 2021 01:11:41 +0300 Subject: [PATCH 0005/1260] fixes --- .../AggregateFunctionContingencyCoefficient.cpp | 7 +++++++ src/AggregateFunctions/AggregateFunctionCramersV.h | 9 +++++++++ .../AggregateFunctionCramersVBiasCorrection.cpp | 8 ++++++++ src/AggregateFunctions/AggregateFunctionTheilsU.cpp | 7 +++++++ 4 files changed, 31 insertions(+) diff --git a/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp b/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp index 269c961d6c5..0391fe3c8ee 100644 --- a/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp +++ b/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp @@ -5,6 +5,10 @@ #include "registerAggregateFunctions.h" #include +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} namespace DB { @@ -16,6 +20,9 @@ struct ContingencyData : public AggregateFunctionCramersVData { Float64 get_result() const { + if (cur_size < 2){ + throw Exception("Aggregate function contingency coefficient requires at least 2 values in columns", ErrorCodes::BAD_ARGUMENTS); + } Float64 phi = 0.0; for (const auto & cell : pairs) { UInt128 hash_pair = cell.getKey(); diff --git a/src/AggregateFunctions/AggregateFunctionCramersV.h b/src/AggregateFunctions/AggregateFunctionCramersV.h index ed41737fa2b..383647f8aa9 100644 --- a/src/AggregateFunctions/AggregateFunctionCramersV.h +++ b/src/AggregateFunctions/AggregateFunctionCramersV.h @@ -19,6 +19,12 @@ #include + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} + namespace DB { @@ -81,6 +87,9 @@ namespace DB Float64 get_result() const { + if (cur_size < 2){ + throw Exception("Aggregate function cramer's v requires et least 2 values in columns", ErrorCodes::BAD_ARGUMENTS); + } Float64 phi = 0.0; for (const auto & cell : pairs) { UInt128 hash_pair = cell.getKey(); diff --git a/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrection.cpp b/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrection.cpp index 4aa9ba0a8fd..c58ca8a59da 100644 --- a/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrection.cpp +++ b/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrection.cpp @@ -6,6 +6,11 @@ #include +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + namespace DB { namespace @@ -16,6 +21,9 @@ struct BiasCorrectionData : public AggregateFunctionCramersVData { Float64 get_result() const { + if (cur_size < 2){ + throw Exception("Aggregate function cramer's v bias corrected at least 2 values in columns", ErrorCodes::BAD_ARGUMENTS); + } Float64 phi = 0.0; for (const auto & cell : pairs) { UInt128 hash_pair = cell.getKey(); diff --git a/src/AggregateFunctions/AggregateFunctionTheilsU.cpp b/src/AggregateFunctions/AggregateFunctionTheilsU.cpp index 954029bb3d6..b2eeff3d7c9 100644 --- a/src/AggregateFunctions/AggregateFunctionTheilsU.cpp +++ b/src/AggregateFunctions/AggregateFunctionTheilsU.cpp @@ -5,6 +5,10 @@ #include "registerAggregateFunctions.h" #include +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} namespace DB { @@ -16,6 +20,9 @@ namespace { Float64 get_result() const { + if (cur_size < 2){ + throw Exception("Aggregate function theil's u requires at least 2 values in columns", ErrorCodes::BAD_ARGUMENTS); + } Float64 h_x = 0.0; for (const auto & cell : n_i) { UInt64 count_x_tmp = cell.getMapped(); From 2c7eeea55c6158f2855244b42a686a4848752da5 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Thu, 8 Jul 2021 17:12:46 +0800 Subject: [PATCH 0006/1260] Fix bug when using null-AggregateFunction --- .../AggregateFunctionNothing.cpp | 27 +++++++++++++++++++ .../registerAggregateFunctions.cpp | 2 ++ .../SerializationAggregateFunction.cpp | 3 ++- .../01922_sum_null_for_remote.reference | 1 + .../0_stateless/01922_sum_null_for_remote.sql | 1 + 5 files changed, 33 insertions(+), 1 deletion(-) create mode 100644 src/AggregateFunctions/AggregateFunctionNothing.cpp create mode 100644 tests/queries/0_stateless/01922_sum_null_for_remote.reference create mode 100644 tests/queries/0_stateless/01922_sum_null_for_remote.sql diff --git a/src/AggregateFunctions/AggregateFunctionNothing.cpp b/src/AggregateFunctions/AggregateFunctionNothing.cpp new file mode 100644 index 00000000000..32bc105bba8 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionNothing.cpp @@ -0,0 +1,27 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +struct Settings; + +AggregateFunctionPtr createAggregateFunctionNothing(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) +{ + assertNoParameters(name, parameters); + assertArityAtMost<1>(name, argument_types); + + return std::make_shared(argument_types, parameters); +} + +void registerAggregateFunctionNothing(AggregateFunctionFactory & factory) +{ + AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true }; + factory.registerFunction("nothing", { createAggregateFunctionNothing, properties }); +} + +} diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 383f10ac24b..42a892e7a21 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -49,6 +49,7 @@ void registerAggregateFunctionMannWhitney(AggregateFunctionFactory &); void registerAggregateFunctionWelchTTest(AggregateFunctionFactory &); void registerAggregateFunctionStudentTTest(AggregateFunctionFactory &); void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory &); +void registerAggregateFunctionNothing(AggregateFunctionFactory &); class AggregateFunctionCombinatorFactory; void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &); @@ -113,6 +114,7 @@ void registerAggregateFunctions() registerAggregateFunctionSequenceNextNode(factory); registerAggregateFunctionWelchTTest(factory); registerAggregateFunctionStudentTTest(factory); + registerAggregateFunctionNothing(factory); registerWindowFunctions(factory); diff --git a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp index 925ba0b9e74..de67c384a34 100644 --- a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp +++ b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp @@ -88,7 +88,8 @@ void SerializationAggregateFunction::deserializeBinaryBulk(IColumn & column, Rea for (size_t i = 0; i < limit; ++i) { - if (istr.eof()) + //AggregateFunctionNothing has no data, so we shouldn't try to get more data. + if (function->getName() != "nothing" && istr.eof()) break; AggregateDataPtr place = arena.alignedAlloc(size_of_state, align_of_state); diff --git a/tests/queries/0_stateless/01922_sum_null_for_remote.reference b/tests/queries/0_stateless/01922_sum_null_for_remote.reference new file mode 100644 index 00000000000..dec7d2fabd2 --- /dev/null +++ b/tests/queries/0_stateless/01922_sum_null_for_remote.reference @@ -0,0 +1 @@ +\N diff --git a/tests/queries/0_stateless/01922_sum_null_for_remote.sql b/tests/queries/0_stateless/01922_sum_null_for_remote.sql new file mode 100644 index 00000000000..a19740364a1 --- /dev/null +++ b/tests/queries/0_stateless/01922_sum_null_for_remote.sql @@ -0,0 +1 @@ +select sum(null) from remote('127.0.0.{1,2}', 'system', 'one') From 1b3fc319041cfe6ee5b3c440e58b932b30333902 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Fri, 16 Jul 2021 10:35:34 +0800 Subject: [PATCH 0007/1260] fix error --- src/AggregateFunctions/AggregateFunctionNothing.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionNothing.cpp b/src/AggregateFunctions/AggregateFunctionNothing.cpp index 32bc105bba8..d1c25026929 100644 --- a/src/AggregateFunctions/AggregateFunctionNothing.cpp +++ b/src/AggregateFunctions/AggregateFunctionNothing.cpp @@ -13,7 +13,7 @@ struct Settings; AggregateFunctionPtr createAggregateFunctionNothing(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { assertNoParameters(name, parameters); - assertArityAtMost<1>(name, argument_types); + assertUnary(name, argument_types); return std::make_shared(argument_types, parameters); } From 276fec96d18283b934edf34f5821832dc3efbe8b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 20 Jul 2021 13:06:50 +0000 Subject: [PATCH 0008/1260] Better code --- src/AggregateFunctions/AggregateFunctionNothing.cpp | 3 +-- src/AggregateFunctions/registerAggregateFunctions.cpp | 2 +- .../Serializations/SerializationAggregateFunction.cpp | 3 +-- 3 files changed, 3 insertions(+), 5 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionNothing.cpp b/src/AggregateFunctions/AggregateFunctionNothing.cpp index d1c25026929..4781ea66a73 100644 --- a/src/AggregateFunctions/AggregateFunctionNothing.cpp +++ b/src/AggregateFunctions/AggregateFunctionNothing.cpp @@ -13,14 +13,13 @@ struct Settings; AggregateFunctionPtr createAggregateFunctionNothing(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { assertNoParameters(name, parameters); - assertUnary(name, argument_types); return std::make_shared(argument_types, parameters); } void registerAggregateFunctionNothing(AggregateFunctionFactory & factory) { - AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true }; + AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = false }; factory.registerFunction("nothing", { createAggregateFunctionNothing, properties }); } diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 42a892e7a21..bbfa7813fa0 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -114,7 +114,7 @@ void registerAggregateFunctions() registerAggregateFunctionSequenceNextNode(factory); registerAggregateFunctionWelchTTest(factory); registerAggregateFunctionStudentTTest(factory); - registerAggregateFunctionNothing(factory); + registerAggregateFunctionNothing(factory); registerWindowFunctions(factory); diff --git a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp index de67c384a34..925ba0b9e74 100644 --- a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp +++ b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp @@ -88,8 +88,7 @@ void SerializationAggregateFunction::deserializeBinaryBulk(IColumn & column, Rea for (size_t i = 0; i < limit; ++i) { - //AggregateFunctionNothing has no data, so we shouldn't try to get more data. - if (function->getName() != "nothing" && istr.eof()) + if (istr.eof()) break; AggregateDataPtr place = arena.alignedAlloc(size_of_state, align_of_state); From 3a00a7f56e0c4b7390956d36ec9e30ea4766109f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 20 Jul 2021 13:08:01 +0000 Subject: [PATCH 0009/1260] Style --- src/AggregateFunctions/registerAggregateFunctions.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index bbfa7813fa0..3a6dd0a1cb7 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -114,7 +114,7 @@ void registerAggregateFunctions() registerAggregateFunctionSequenceNextNode(factory); registerAggregateFunctionWelchTTest(factory); registerAggregateFunctionStudentTTest(factory); - registerAggregateFunctionNothing(factory); + registerAggregateFunctionNothing(factory); registerWindowFunctions(factory); From 2f67f2d387a9dbeca149d332d2c1cac3a93df67b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 3 Aug 2021 15:01:06 +0000 Subject: [PATCH 0010/1260] Allow deserialization for AggregateFunctionNothing --- .../Serializations/SerializationAggregateFunction.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp index 925ba0b9e74..2a077a26085 100644 --- a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp +++ b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp @@ -88,7 +88,8 @@ void SerializationAggregateFunction::deserializeBinaryBulk(IColumn & column, Rea for (size_t i = 0; i < limit; ++i) { - if (istr.eof()) + /// AggregateFunctionNothing does nothing, so we can allow it do anything with istr + if (function->getName() != "nothing" && istr.eof()) break; AggregateDataPtr place = arena.alignedAlloc(size_of_state, align_of_state); From 8920f98dd4bd52ea5149e3bcaaa4e6841da8bec4 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 4 Aug 2021 17:33:37 +0000 Subject: [PATCH 0011/1260] Added a state to AggregateFunctionNothing --- src/AggregateFunctions/AggregateFunctionNothing.h | 12 +++++++++--- src/Compression/CompressedReadBufferBase.cpp | 1 - .../SerializationAggregateFunction.cpp | 3 +-- 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionNothing.h b/src/AggregateFunctions/AggregateFunctionNothing.h index 1f5b95c5cf3..fce13b05979 100644 --- a/src/AggregateFunctions/AggregateFunctionNothing.h +++ b/src/AggregateFunctions/AggregateFunctionNothing.h @@ -4,6 +4,8 @@ #include #include #include +#include +#include namespace DB @@ -46,7 +48,7 @@ public: size_t sizeOfData() const override { - return 0; + return 1; } size_t alignOfData() const override @@ -62,12 +64,16 @@ public: { } - void serialize(ConstAggregateDataPtr, WriteBuffer &) const override + void serialize(ConstAggregateDataPtr, WriteBuffer & buf) const override { + writeChar('\0', buf); } - void deserialize(AggregateDataPtr, ReadBuffer &, Arena *) const override + void deserialize(AggregateDataPtr, ReadBuffer & buf, Arena *) const override { + [[maybe_unused]] char symbol; + readChar(symbol, buf); + assert(symbol == '\0'); } void insertResultInto(AggregateDataPtr, IColumn & to, Arena *) const override diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 749f174677c..006b3fab2d8 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -253,4 +253,3 @@ CompressedReadBufferBase::~CompressedReadBufferBase() = default; /// Proper d } - diff --git a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp index 2a077a26085..925ba0b9e74 100644 --- a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp +++ b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp @@ -88,8 +88,7 @@ void SerializationAggregateFunction::deserializeBinaryBulk(IColumn & column, Rea for (size_t i = 0; i < limit; ++i) { - /// AggregateFunctionNothing does nothing, so we can allow it do anything with istr - if (function->getName() != "nothing" && istr.eof()) + if (istr.eof()) break; AggregateDataPtr place = arena.alignedAlloc(size_of_state, align_of_state); From 36ca0b296b12a0b4c081ea5e86e0c25c23c95c71 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 5 Nov 2021 19:55:30 +0800 Subject: [PATCH 0012/1260] implement hive table engine --- contrib/CMakeLists.txt | 1 + .../table-engines/integrations/hive.md | 157 +++++ .../table-engines/integrations/index.md | 1 + programs/server/Server.cpp | 4 + src/CMakeLists.txt | 8 + src/Common/ErrorCodes.cpp | 2 + src/Common/config.h.in | 1 + src/Common/examples/CMakeLists.txt | 3 + src/Common/examples/hive_metastore_client.cpp | 43 ++ src/Formats/FormatSettings.h | 2 + src/IO/CompressionMethod.cpp | 16 +- src/IO/CompressionMethod.h | 3 +- src/IO/HadoopSnappyReadBuffer.cpp | 218 +++++++ src/IO/HadoopSnappyReadBuffer.h | 115 ++++ src/IO/ReadHelpers.h | 26 + src/IO/RemoteReadBufferCache.cpp | 596 ++++++++++++++++++ src/IO/RemoteReadBufferCache.h | 211 +++++++ src/IO/SnappyReadBuffer.cpp | 76 +++ src/IO/SnappyReadBuffer.h | 35 + src/IO/SnappyWriteBuffer.cpp | 92 +++ src/IO/SnappyWriteBuffer.h | 41 ++ src/IO/examples/CMakeLists.txt | 7 + src/IO/examples/hadoop_snappy_read_buffer.cpp | 43 ++ src/IO/examples/snappy_read_buffer.cpp | 35 + src/IO/examples/test.snappy | Bin 0 -> 5590 bytes src/Interpreters/Context.cpp | 63 +- src/Interpreters/Context.h | 12 +- .../Formats/Impl/ArrowBufferedStreams.cpp | 12 +- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 15 +- .../Formats/Impl/CSVRowInputFormat.cpp | 80 +++ .../Formats/Impl/CSVRowInputFormat.h | 4 + .../Formats/RowInputFormatWithNamesAndTypes.h | 19 +- .../Transforms/AggregatingTransform.h | 3 + src/Processors/examples/CMakeLists.txt | 2 + .../examples/comma_separated_streams.cpp | 121 ++++ src/Processors/examples/test_in | 8 + src/Storages/HDFS/HDFSCommon.cpp | 15 + src/Storages/HDFS/HDFSCommon.h | 3 + src/Storages/MergeTree/MergeList.h | 2 + src/Storages/registerStorages.cpp | 2 + 40 files changed, 2077 insertions(+), 20 deletions(-) create mode 100644 docs/en/engines/table-engines/integrations/hive.md create mode 100644 src/Common/examples/hive_metastore_client.cpp create mode 100644 src/IO/HadoopSnappyReadBuffer.cpp create mode 100644 src/IO/HadoopSnappyReadBuffer.h create mode 100644 src/IO/RemoteReadBufferCache.cpp create mode 100644 src/IO/RemoteReadBufferCache.h create mode 100644 src/IO/SnappyReadBuffer.cpp create mode 100644 src/IO/SnappyReadBuffer.h create mode 100644 src/IO/SnappyWriteBuffer.cpp create mode 100644 src/IO/SnappyWriteBuffer.h create mode 100644 src/IO/examples/hadoop_snappy_read_buffer.cpp create mode 100644 src/IO/examples/snappy_read_buffer.cpp create mode 100644 src/IO/examples/test.snappy create mode 100644 src/Processors/examples/comma_separated_streams.cpp create mode 100644 src/Processors/examples/test_in diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 676654452de..ca537f4d4f7 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -37,6 +37,7 @@ add_subdirectory (murmurhash) add_subdirectory (replxx-cmake) add_subdirectory (unixodbc-cmake) add_subdirectory (nanodbc-cmake) +add_subdirectory (hive-metastore) if (USE_INTERNAL_CAPNP_LIBRARY AND NOT MISSING_INTERNAL_CAPNP_LIBRARY) add_subdirectory(capnproto-cmake) diff --git a/docs/en/engines/table-engines/integrations/hive.md b/docs/en/engines/table-engines/integrations/hive.md new file mode 100644 index 00000000000..ce3828555e7 --- /dev/null +++ b/docs/en/engines/table-engines/integrations/hive.md @@ -0,0 +1,157 @@ +--- +toc_priority: 4 +toc_title: Hive +--- + +# Hive {#hive} + +The Hive engine allows you to perform `SELECT` quries on HDFS Hive table. Currently it supports input formats as below: + +- Text: only supports simple scalar column types except `binary` + +- ORC: support simple scalar columns types except `char`; only support complex types like `array` + +- Parquet: support all simple scalar columns types; only support complex types like `array` + +## Creating a Table {#creating-a-table} + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [ALIAS expr1], + name2 [type2] [ALIAS expr2], + ... +) ENGINE = Hive('thrift://host:port', 'database', 'table'); +PARTITION BY expr +``` +See a detailed description of the [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query) query. + +The table structure can differ from the original Hive table structure: +- Column names should be the same as in the original Hive table, but you can use just some of these columns and in any order, also you can use some alias columns calculated from other columns. +- Column types should be the same from those in the original Hive table. +- Partition by expression should be consistent with the original Hive table, and columns in partition by expression should be in the table structure. + +**Engine Parameters** + +- `thrift://host:port` — Hive Metastore address + +- `database` — Remote database name. + +- `table` — Remote table name. + +## Usage Example {#usage-example} + +Table in Hive: + +``` text +hive > CREATE TABLE `test`.`test_orc`( + `f_tinyint` tinyint, + `f_smallint` smallint, + `f_int` int, + `f_integer` int, + `f_bigint` bigint, + `f_float` float, + `f_double` double, + `f_decimal` decimal(10,0), + `f_timestamp` timestamp, + `f_date` date, + `f_string` string, + `f_varchar` varchar(100), + `f_bool` boolean, + `f_binary` binary, + `f_array_int` array, + `f_array_string` array, + `f_array_float` array, + `f_array_array_int` array>, + `f_array_array_string` array>, + `f_array_array_float` array>) +PARTITIONED BY ( + `day` string) +ROW FORMAT SERDE + 'org.apache.hadoop.hive.ql.io.orc.OrcSerde' +STORED AS INPUTFORMAT + 'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat' +LOCATION + 'hdfs://testcluster/data/hive/test.db/test_orc' +TBLPROPERTIES ( + 'transient_lastDdlTime'='1631944221') +OK +Time taken: 0.51 seconds + +hive > insert into test.test_orc partition(day='2021-09-18') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, current_timestamp(), current_date(), 'hello world', 'hello world', 'hello world', true, 'hello world', array(1, 2, 3), array('hello world', 'hello world'), array(float(1.1), float(1.2)), array(array(1, 2), array(3, 4)), array(array('a', 'b'), array('c', 'd')), array(array(float(1.11), float(2.22)), array(float(3.33), float(4.44))); +OK +Time taken: 36.025 seconds + +hive > select * from test.test_orc; +OK +1 2 3 4 5 6.11 7.22 8 2021-11-05 12:38:16.314 2021-11-05 hello world hello world hello world true hello world [1,2,3] ["hello world","hello world"] [1.1,1.2] [[1,2],[3,4]] [["a","b"],["c","d"]] [[1.11,2.22],[3.33,4.44]] 2021-09-18 +Time taken: 0.295 seconds, Fetched: 1 row(s) +``` + +Table in ClickHouse, retrieving data from the Hive table created above: +``` sql +CREATE TABLE `test`.`test_orc`( + `f_tinyint` Int8, + `f_smallint` Int16, + `f_int` Int32, + `f_integer` Int32, + `f_bigint` Int64, + `f_float` Float32, + `f_double` Float64, + `f_decimal` Float64, + `f_timestamp` DateTime, + `f_date` Date, + `f_string` String, + `f_varchar` String, + `f_bool` UInt8, + `f_binary` String, + `f_array_int` Array(Int32), + `f_array_string` Array(String), + `f_array_float` Array(Float32), + `f_array_array_int` Array(Array(Int32)), + `f_array_array_string` Array(Array(String)), + `f_array_array_float` Array(Array(Float32)), + day String +) +ENGINE = Hive('thrift://localhost:9083', 'test', 'test_orc') +PARTITION BY day +``` + +``` sql +SELECT * FROM test.test_orc \G +``` + +``` text +SELECT * +FROM test_orc + +Query id: c3eaffdc-78ab-43cd-96a4-4acc5b480658 + +Row 1: +────── +f_tinyint: 1 +f_smallint: 2 +f_int: 3 +f_integer: 4 +f_bigint: 5 +f_float: 6.11 +f_double: 7.22 +f_decimal: 8 +f_timestamp: 2021-11-05 20:38:16 +f_date: 2021-11-05 +f_string: hello world +f_varchar: hello world +f_bool: 1 +f_binary: hello world +f_array_int: [1,2,3] +f_array_string: ['hello world','hello world'] +f_array_float: [1.1,1.2] +f_array_array_int: [[1,2],[3,4]] +f_array_array_string: [['a','b'],['c','d']] +f_array_array_float: [[1.11,2.22],[3.33,4.44]] +day: 2021-09-18 + +1 rows in set. Elapsed: 0.078 sec. +``` diff --git a/docs/en/engines/table-engines/integrations/index.md b/docs/en/engines/table-engines/integrations/index.md index 743d25ad616..a06b4c78394 100644 --- a/docs/en/engines/table-engines/integrations/index.md +++ b/docs/en/engines/table-engines/integrations/index.md @@ -20,3 +20,4 @@ List of supported integrations: - [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md) - [PostgreSQL](../../../engines/table-engines/integrations/postgresql.md) - [SQLite](../../../engines/table-engines/integrations/sqlite.md) +- [Hive](../../../engines/table-engines/integrations/hive.md) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index af1686bfa70..00967390667 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -43,6 +43,7 @@ #include #include #include +#include #include #include #include @@ -505,6 +506,9 @@ int Server::main(const std::vector & /*args*/) if (ThreadFuzzer::instance().isEffective()) global_context->addWarningMessage("ThreadFuzzer is enabled. Application will run slowly and unstable."); +if (config().has("local_cache_dir") && config().has("local_cache_quota")) + RemoteReadBufferCache::instance().initOnce(config().getString("local_cache_dir"), config().getUInt64("local_cache_quota")); + #if defined(SANITIZER) global_context->addWarningMessage("Server was built with sanitizer. It will work slowly."); #endif diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 0a7c3123f9d..3696b91108b 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -107,6 +107,7 @@ endif() if (USE_HDFS) add_headers_and_sources(dbms Storages/HDFS) + add_headers_and_sources(dbms Storages/Hive) add_headers_and_sources(dbms Disks/HDFS) endif() @@ -443,6 +444,8 @@ endif() if (USE_HDFS) dbms_target_link_libraries(PRIVATE ${HDFS3_LIBRARY}) dbms_target_include_directories (SYSTEM BEFORE PUBLIC ${HDFS3_INCLUDE_DIR}) + + dbms_target_link_libraries(PRIVATE hivemetastore ${THRIFT_LIBRARY}) endif() if (USE_AWS_S3) @@ -461,6 +464,11 @@ if (USE_BROTLI) target_include_directories (clickhouse_common_io SYSTEM BEFORE PRIVATE ${BROTLI_INCLUDE_DIR}) endif() +if (USE_SNAPPY) + target_link_libraries (clickhouse_common_io PRIVATE ${SNAPPY_LIBRARY}) + target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${SNAPPY_INCLUDE_DIR}) +endif() + if (USE_AMQPCPP) dbms_target_link_libraries(PUBLIC ${AMQPCPP_LIBRARY}) dbms_target_include_directories (SYSTEM BEFORE PUBLIC ${AMQPCPP_INCLUDE_DIR}) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index ec2bd025627..0f5f07fbfea 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -594,6 +594,8 @@ M(624, BAD_FILE_TYPE) \ M(625, IO_SETUP_ERROR) \ M(626, CANNOT_SKIP_UNKNOWN_FIELD) \ + M(627, SNAPPY_UNCOMPRESS_FAILED) \ + M(628, NO_HIVEMETASTORE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Common/config.h.in b/src/Common/config.h.in index bf118e2507f..1b14794fb27 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -20,3 +20,4 @@ #cmakedefine01 USE_YAML_CPP #cmakedefine01 CLICKHOUSE_SPLIT_BINARY #cmakedefine01 USE_BZIP2 +#cmakedefine01 USE_SNAPPY diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index 020f3cc4446..be91101ef40 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -80,3 +80,6 @@ target_link_libraries (shell_command_inout PRIVATE clickhouse_common_io) add_executable (executable_udf executable_udf.cpp) target_link_libraries (executable_udf PRIVATE dbms) + +add_executable(hive_metastore_client hive_metastore_client.cpp) +target_link_libraries (hive_metastore_client PUBLIC hivemetastore ${THRIFT_LIBRARY}) diff --git a/src/Common/examples/hive_metastore_client.cpp b/src/Common/examples/hive_metastore_client.cpp new file mode 100644 index 00000000000..5a48f2829bf --- /dev/null +++ b/src/Common/examples/hive_metastore_client.cpp @@ -0,0 +1,43 @@ +#include + +#include +#include +#include +#include + + +using namespace std; +using namespace apache::thrift; +using namespace apache::thrift::protocol; +using namespace apache::thrift::transport; +using namespace Apache::Hadoop::Hive; + +int main() +{ + std::shared_ptr socket(new TSocket("localhost", 9083)); + std::shared_ptr transport(new TBufferedTransport(socket)); + std::shared_ptr protocol(new TBinaryProtocol(transport)); + ThriftHiveMetastoreClient client(protocol); + + try + { + transport->open(); + + Table table; + client.get_table(table, "default", "persons"); + table.printTo(std::cout); + + vector partitions; + client.get_partitions(partitions, "default", "persons", 0); + for (const auto & part : partitions) + { + part.printTo(std::cout); + } + + transport->close(); + } + catch (TException & tx) + { + cout << "ERROR: " << tx.what() << endl; + } +} diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index b4f1550f0bd..d7e26ed239c 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -1,5 +1,6 @@ #pragma once +#include #include @@ -78,6 +79,7 @@ struct FormatSettings bool crlf_end_of_line = false; bool input_format_enum_as_number = false; bool input_format_arrays_as_nested_csv = false; + Names input_field_names; String null_representation = "\\N"; } csv; diff --git a/src/IO/CompressionMethod.cpp b/src/IO/CompressionMethod.cpp index c61d091cd09..eaab7560e6a 100644 --- a/src/IO/CompressionMethod.cpp +++ b/src/IO/CompressionMethod.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include @@ -46,6 +47,8 @@ std::string toContentEncodingName(CompressionMethod method) return "lz4"; case CompressionMethod::Bzip2: return "bz2"; + case CompressionMethod::Snappy: + return "snappy"; case CompressionMethod::None: return ""; } @@ -79,11 +82,13 @@ CompressionMethod chooseCompressionMethod(const std::string & path, const std::s return CompressionMethod::Lz4; if (method_str == "bz2") return CompressionMethod::Bzip2; + if (method_str == "snappy") + return CompressionMethod::Snappy; if (hint.empty() || hint == "auto" || hint == "none") return CompressionMethod::None; throw Exception( - "Unknown compression method " + hint + ". Only 'auto', 'none', 'gzip', 'deflate', 'br', 'xz', 'zstd', 'lz4', 'bz2' are supported as compression methods", + "Unknown compression method " + hint + ". Only 'auto', 'none', 'gzip', 'deflate', 'br', 'xz', 'zstd', 'lz4', 'bz2', 'snappy' are supported as compression methods", ErrorCodes::NOT_IMPLEMENTED); } @@ -107,6 +112,11 @@ std::unique_ptr wrapReadBufferWithCompressionMethod( if (method == CompressionMethod::Bzip2) return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); #endif +#if USE_SNAPPY + if (method == CompressionMethod::Snappy) + return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); +#endif + if (method == CompressionMethod::None) return nested; @@ -136,6 +146,10 @@ std::unique_ptr wrapWriteBufferWithCompressionMethod( #if USE_BZIP2 if (method == CompressionMethod::Bzip2) return std::make_unique(std::move(nested), level, buf_size, existing_memory, alignment); +#endif +#if USE_SNAPPY + if (method == CompressionMethod::Snappy) + throw Exception("Unsupported compression method", ErrorCodes::NOT_IMPLEMENTED); #endif if (method == CompressionMethod::None) return nested; diff --git a/src/IO/CompressionMethod.h b/src/IO/CompressionMethod.h index df6400c55a3..3953ba9d212 100644 --- a/src/IO/CompressionMethod.h +++ b/src/IO/CompressionMethod.h @@ -33,7 +33,8 @@ enum class CompressionMethod Zstd, Brotli, Lz4, - Bzip2 + Bzip2, + Snappy, }; /// How the compression method is named in HTTP. diff --git a/src/IO/HadoopSnappyReadBuffer.cpp b/src/IO/HadoopSnappyReadBuffer.cpp new file mode 100644 index 00000000000..1adb911d595 --- /dev/null +++ b/src/IO/HadoopSnappyReadBuffer.cpp @@ -0,0 +1,218 @@ +#include + +#if USE_SNAPPY +#include +#include +#include +#include +#include + +#include + +#include "HadoopSnappyReadBuffer.h" + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int SNAPPY_UNCOMPRESS_FAILED; +} + + +inline bool HadoopSnappyDecoder::checkBufferLength(int max) const +{ + return buffer_length >= 0 && buffer_length < max; +} + +inline bool HadoopSnappyDecoder::checkAvailIn(size_t avail_in, int min) +{ + return avail_in >= size_t(min); +} + +inline void HadoopSnappyDecoder::copyToBuffer(size_t * avail_in, const char ** next_in) +{ + assert(*avail_in <= sizeof(buffer)); + + memcpy(buffer, *next_in, *avail_in); + + buffer_length = *avail_in; + *next_in += *avail_in; + *avail_in = 0; +} + + +inline uint32_t HadoopSnappyDecoder::readLength(const char * in) +{ + uint32_t b1 = *(reinterpret_cast(in)); + uint32_t b2 = *(reinterpret_cast(in + 1)); + uint32_t b3 = *(reinterpret_cast(in + 2)); + uint32_t b4 = *(reinterpret_cast(in + 3)); + uint32_t res = ((b1 << 24) + (b2 << 16) + (b3 << 8) + (b4 << 0)); + return res; +} + + +inline HadoopSnappyDecoder::Status HadoopSnappyDecoder::readLength(size_t * avail_in, const char ** next_in, int * length) +{ + char tmp[4] = {0}; + + if (!checkBufferLength(4)) + return Status::INVALID_INPUT; + memcpy(tmp, buffer, buffer_length); + + if (!checkAvailIn(*avail_in, 4 - buffer_length)) + { + copyToBuffer(avail_in, next_in); + return Status::NEEDS_MORE_INPUT; + } + memcpy(tmp + buffer_length, *next_in, 4 - buffer_length); + + *avail_in -= 4 - buffer_length; + *next_in += 4 - buffer_length; + buffer_length = 0; + *length = readLength(tmp); + return Status::OK; +} + +inline HadoopSnappyDecoder::Status HadoopSnappyDecoder::readBlockLength(size_t * avail_in, const char ** next_in) +{ + if (block_length < 0) + return readLength(avail_in, next_in, &block_length); + return Status::OK; +} + +inline HadoopSnappyDecoder::Status HadoopSnappyDecoder::readCompressedLength(size_t * avail_in, const char ** next_in) +{ + if (compressed_length < 0) + return readLength(avail_in, next_in, &compressed_length); + return Status::OK; +} + +inline HadoopSnappyDecoder::Status +HadoopSnappyDecoder::readCompressedData(size_t * avail_in, const char ** next_in, size_t * avail_out, char ** next_out) +{ + if (!checkBufferLength(compressed_length)) + return Status::INVALID_INPUT; + + if (!checkAvailIn(*avail_in, compressed_length - buffer_length)) + { + copyToBuffer(avail_in, next_in); + return Status::NEEDS_MORE_INPUT; + } + + const char * compressed = nullptr; + if (buffer_length > 0) + { + compressed = buffer; + memcpy(buffer + buffer_length, *next_in, compressed_length - buffer_length); + } + else + { + compressed = const_cast(*next_in); + } + + size_t uncompressed_length = *avail_out; + auto status = snappy_uncompress(compressed, compressed_length, *next_out, &uncompressed_length); + if (status != SNAPPY_OK) + { + return Status(status); + } + + *avail_in -= compressed_length - buffer_length; + *next_in += compressed_length - buffer_length; + *avail_out -= uncompressed_length; + *next_out += uncompressed_length; + + total_uncompressed_length += uncompressed_length; + compressed_length = -1; + buffer_length = 0; + return Status::OK; +} + +HadoopSnappyDecoder::Status HadoopSnappyDecoder::readBlock(size_t * avail_in, const char ** next_in, size_t * avail_out, char ** next_out) +{ + if (*avail_in == 0) + { + if (buffer_length == 0 && block_length < 0 && compressed_length < 0) + return Status::OK; + return Status::NEEDS_MORE_INPUT; + } + + HadoopSnappyDecoder::Status status = readBlockLength(avail_in, next_in); + if (status != Status::OK) + return status; + + while (total_uncompressed_length < block_length) + { + status = readCompressedLength(avail_in, next_in); + if (status != Status::OK) + return status; + + status = readCompressedData(avail_in, next_in, avail_out, next_out); + if (status != Status::OK) + return status; + } + if (total_uncompressed_length != block_length) + return Status::INVALID_INPUT; + return Status::OK; +} + +HadoopSnappyReadBuffer::HadoopSnappyReadBuffer(std::unique_ptr in_, size_t buf_size, char * existing_memory, size_t alignment) + : BufferWithOwnMemory(buf_size, existing_memory, alignment) + , in(std::move(in_)) + , decoder(std::make_unique()) + , in_available(0) + , in_data(nullptr) + , out_capacity(0) + , out_data(nullptr) + , eof(false) +{ +} + +HadoopSnappyReadBuffer::~HadoopSnappyReadBuffer() = default; + +bool HadoopSnappyReadBuffer::nextImpl() +{ + if (eof) + return false; + + if (!in_available) + { + in->nextIfAtEnd(); + in_available = in->buffer().end() - in->position(); + in_data = in->position(); + } + + if (decoder->result == Status::NEEDS_MORE_INPUT && (!in_available || in->eof())) + { + throw Exception(String("hadoop snappy decode error:") + statusToString(decoder->result), ErrorCodes::SNAPPY_UNCOMPRESS_FAILED); + } + + out_capacity = internal_buffer.size(); + out_data = internal_buffer.begin(); + decoder->result = decoder->readBlock(&in_available, &in_data, &out_capacity, &out_data); + + in->position() = in->buffer().end() - in_available; + working_buffer.resize(internal_buffer.size() - out_capacity); + + if (decoder->result == Status::OK) + { + decoder->reset(); + if (in->eof()) + { + eof = true; + return !working_buffer.empty(); + } + return true; + } + else if (decoder->result == Status::INVALID_INPUT || decoder->result == Status::BUFFER_TOO_SMALL) + { + throw Exception(String("hadoop snappy decode error:") + statusToString(decoder->result), ErrorCodes::SNAPPY_UNCOMPRESS_FAILED); + } + return true; +} + +} + +#endif diff --git a/src/IO/HadoopSnappyReadBuffer.h b/src/IO/HadoopSnappyReadBuffer.h new file mode 100644 index 00000000000..c2427ab6bb8 --- /dev/null +++ b/src/IO/HadoopSnappyReadBuffer.h @@ -0,0 +1,115 @@ +#pragma once + +#include + +#if USE_SNAPPY + +#include +#include +#include + +namespace DB +{ + + +/* + * Hadoop-snappy format is one of the compression formats base on Snappy used in Hadoop. It uses its own framing format as follows: + * 1. A compressed file consists of one or more blocks. + * 2. A block consists of uncompressed length (big endian 4 byte integer) and one or more subblocks. + * 3. A subblock consists of compressed length (big endian 4 byte integer) and raw compressed data. + * + * HadoopSnappyDecoder implements the decompression of data compressed with hadoop-snappy format. + */ +class HadoopSnappyDecoder +{ +public: + enum class Status : int + { + OK = 0, + INVALID_INPUT = 1, + BUFFER_TOO_SMALL = 2, + NEEDS_MORE_INPUT = 3, + }; + + HadoopSnappyDecoder() = default; + ~HadoopSnappyDecoder() = default; + + Status readBlock(size_t * avail_in, const char ** next_in, size_t * avail_out, char ** next_out); + + inline void reset() + { + buffer_length = 0; + block_length = -1; + compressed_length = -1; + total_uncompressed_length = 0; + } + + Status result; + +private: + inline bool checkBufferLength(int max) const; + inline static bool checkAvailIn(size_t avail_in, int min); + + inline void copyToBuffer(size_t * avail_in, const char ** next_in); + + inline static uint32_t readLength(const char * in); + inline Status readLength(size_t * avail_in, const char ** next_in, int * length); + inline Status readBlockLength(size_t * avail_in, const char ** next_in); + inline Status readCompressedLength(size_t * avail_in, const char ** next_in); + inline Status readCompressedData(size_t * avail_in, const char ** next_in, size_t * avail_out, char ** next_out); + + char buffer[DBMS_DEFAULT_BUFFER_SIZE] = {0}; + int buffer_length = 0; + + int block_length = -1; + int compressed_length = -1; + int total_uncompressed_length = 0; +}; + +/// HadoopSnappyReadBuffer implements read buffer for data compressed with hadoop-snappy format. +class HadoopSnappyReadBuffer : public BufferWithOwnMemory +{ +public: + using Status = HadoopSnappyDecoder::Status; + + inline static String statusToString(Status status) + { + switch (status) + { + case Status::OK: + return "OK"; + case Status::INVALID_INPUT: + return "INVALID_INPUT"; + case Status::BUFFER_TOO_SMALL: + return "BUFFER_TOO_SMALL"; + case Status::NEEDS_MORE_INPUT: + return "NEEDS_MORE_INPUT"; + } + __builtin_unreachable(); + } + + explicit HadoopSnappyReadBuffer( + std::unique_ptr in_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0); + + ~HadoopSnappyReadBuffer() override; + +private: + bool nextImpl() override; + + std::unique_ptr in; + std::unique_ptr decoder; + + size_t in_available; + const char * in_data; + + size_t out_capacity; + char * out_data; + + bool eof; +}; + +} +#endif diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 81973bcd8a4..64e62ee009f 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1054,6 +1054,32 @@ inline void readCSVSimple(T & x, ReadBuffer & buf) assertChar(maybe_quote, buf); } +// Enable read "t" and "f" as UInt8 value in Hive TEXT File. +inline void readCSVSimple(UInt8 & x, ReadBuffer & buf) +{ + if (buf.eof()) + throwReadAfterEOF(); + + char maybe_quote = *buf.position(); + + if (maybe_quote == '\'' || maybe_quote == '\"') + ++buf.position(); + + if (*buf.position() == 't' || *buf.position() == 'f') + { + bool tmp = false; + readBoolTextWord(tmp, buf); + x = tmp; + } + else + { + readText(x, buf); + } + + if (maybe_quote == '\'' || maybe_quote == '\"') + assertChar(maybe_quote, buf); +} + template inline std::enable_if_t, void> readCSV(T & x, ReadBuffer & buf) { readCSVSimple(x, buf); } diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp new file mode 100644 index 00000000000..9ad8c680065 --- /dev/null +++ b/src/IO/RemoteReadBufferCache.cpp @@ -0,0 +1,596 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "Common/Exception.h" +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int BAD_GET; + extern const int LOGICAL_ERROR; +} + +std::shared_ptr +RemoteCacheController::recover(const std::string & local_path_, std::function const & finish_callback) +{ + Poco::File dir_handle(local_path_); + Poco::File data_file(local_path_ + "/data.bin"); + Poco::File meta_file(local_path_ + "/meta.txt"); + if (!dir_handle.exists() || !data_file.exists() || !meta_file.exists()) + { + LOG_ERROR(&Poco::Logger::get("RemoteCacheController"), "not exists direcotry " + local_path_); + return nullptr; + } + + std::ifstream meta_fs(local_path_ + "/meta.txt"); + Poco::JSON::Parser meta_parser; + auto meta_jobj = meta_parser.parse(meta_fs).extract(); + auto remote_path = meta_jobj->get("remote_path").convert(); + auto schema = meta_jobj->get("schema").convert(); + auto cluster = meta_jobj->get("cluster").convert(); + auto downloaded = meta_jobj->get("downloaded").convert(); + auto mod_ts = meta_jobj->get("last_mod_ts").convert(); + if (downloaded == "false") + { + LOG_ERROR(&Poco::Logger::get("RemoteCacheController"), "not a downloaded file " + local_path_); + return nullptr; + } + auto size = data_file.getSize(); + + auto cntrl = std::make_shared(schema, cluster, remote_path, mod_ts, local_path_, nullptr, finish_callback); + cntrl->download_finished = true; + cntrl->current_offset = size; + meta_fs.close(); + + finish_callback(cntrl.get()); + return cntrl; +} + +RemoteCacheController::RemoteCacheController( + const std::string & schema_, + const std::string & cluster_, + const std::string & path_, + UInt64 mod_ts_, + const std::string & local_path_, + std::shared_ptr readbuffer_, + std::function const & finish_callback) +{ + download_thread = nullptr; + schema = schema_; + cluster = cluster_; + local_path = local_path_; + remote_path = path_; + last_mod_ts = mod_ts_; + valid = true; + if (readbuffer_ != nullptr) + { + download_finished = false; + current_offset = 0; + remote_readbuffer = readbuffer_; + // setup local files + out_file = new Poco::FileOutputStream(local_path_ + "/data.bin", std::ios::out | std::ios::binary); + out_file->flush(); + + Poco::JSON::Object jobj; + jobj.set("schema", schema_); + jobj.set("cluster", cluster_); + jobj.set("remote_path", path_); + jobj.set("downloaded", "false"); + jobj.set("last_mod_ts", mod_ts_); + std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + jobj.stringify(buf); + Poco::FileOutputStream meta_file(local_path_ + "/meta.txt", std::ios::out); + meta_file.write(buf.str().c_str(), buf.str().size()); + meta_file.close(); + + backgroupDownload(finish_callback); + } +} + +int RemoteCacheController::waitMoreData(size_t start_offset_, size_t end_offset_) +{ + std::unique_lock lock{mutex}; + if (download_finished) + { + if (download_thread != nullptr) + { + download_thread->wait(); + LOG_TRACE(&Poco::Logger::get("RemoteCacheController"), "try to release down thread"); + delete download_thread; + download_thread = nullptr; + } + // finish reading + if (start_offset_ >= current_offset) + { + lock.unlock(); + return -1; + } + } + else // block until more data is ready + { + if (current_offset >= end_offset_) + { + lock.unlock(); + return 0; + } + else + more_data_signal.wait(lock, [this, end_offset_] { return this->download_finished || this->current_offset >= end_offset_; }); + } + LOG_TRACE(&Poco::Logger::get("RemoteCacheController"), "get more data to read"); + lock.unlock(); + return 0; +} + +void RemoteCacheController::backgroupDownload(std::function const & finish_callback) +{ + download_thread = new ThreadPool(1); + auto task = [this, finish_callback]() + { + size_t n = 0; + size_t total_bytes = 0; + while (!remote_readbuffer->eof()) + { + size_t bytes = remote_readbuffer->buffer().end() - remote_readbuffer->position(); + out_file->write(remote_readbuffer->position(), bytes); + remote_readbuffer->position() += bytes; + total_bytes += bytes; + if (n++ % 10 == 0) + { + std::unique_lock lock(mutex); + current_offset += total_bytes; + total_bytes = 0; + flush(); + lock.unlock(); + more_data_signal.notify_all(); + } + } + std::unique_lock lock(mutex); + current_offset += total_bytes; + download_finished = true; + flush(); + out_file->close(); + delete out_file; + out_file = nullptr; + remote_readbuffer = nullptr; + lock.unlock(); + more_data_signal.notify_all(); + finish_callback(this); + LOG_TRACE( + &Poco::Logger::get("RemoteCacheController"), "finish download.{} into {}. size:{} ", remote_path, local_path, current_offset); + }; + download_thread->scheduleOrThrow(task); +} + +void RemoteCacheController::flush() +{ + if (out_file != nullptr) + { + out_file->flush(); + } + Poco::JSON::Object jobj; + jobj.set("schema", schema); + jobj.set("cluster", cluster); + jobj.set("remote_path", remote_path); + jobj.set("downloaded", download_finished ? "true" : "false"); + jobj.set("last_mod_ts", last_mod_ts); + std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + jobj.stringify(buf); + + std::ofstream meta_file(local_path + "/meta.txt", std::ios::out); + meta_file << buf.str(); + meta_file.close(); +} + +RemoteCacheController::~RemoteCacheController() +{ + delete out_file; + + if (download_thread != nullptr) + { + download_thread->wait(); + delete download_thread; + } +} + +void RemoteCacheController::close() +{ + // delete the directory + LOG_TRACE(&Poco::Logger::get("RemoteCacheController"), "release local resource " + remote_path + ", " + local_path); + Poco::File file(local_path); + file.remove(true); +} + +FILE * RemoteCacheController::allocFile(std::string * local_path_) +{ + if (download_finished) + *local_path_ = local_path + "/data.bin"; + FILE * fs = fopen((local_path + "/data.bin").c_str(), "r"); + if (fs == nullptr) + return fs; + std::lock_guard lock{mutex}; + opened_file_streams.insert(fs); + return fs; +} + +void RemoteCacheController::deallocFile(FILE * fs) +{ + { + std::lock_guard lock{mutex}; + auto it = opened_file_streams.find(fs); + if (it == opened_file_streams.end()) + { + std::string err = "try to close an invalid file " + remote_path; + throw Exception(err, ErrorCodes::BAD_ARGUMENTS); + } + opened_file_streams.erase(it); + } + fclose(fs); +} + +LocalCachedFileReader::LocalCachedFileReader(RemoteCacheController * cntrl_, size_t size_) + : offset(0), file_size(size_), fs(nullptr), controller(cntrl_) +{ + fs = controller->allocFile(&local_path); + if (fs == nullptr) + throw Exception("alloc file failed.", ErrorCodes::BAD_GET); +} +LocalCachedFileReader::~LocalCachedFileReader() +{ + controller->deallocFile(fs); +} + +size_t LocalCachedFileReader::read(char * buf, size_t size) +{ + auto wret = controller->waitMoreData(offset, offset + size); + if (wret < 0) + return 0; + std::lock_guard lock(mutex); + auto ret_size = fread(buf, 1, size, fs); + offset += ret_size; + return ret_size; +} + +off_t LocalCachedFileReader::seek(off_t off) +{ + controller->waitMoreData(off, 1); + std::lock_guard lock(mutex); + auto ret = fseek(fs, off, SEEK_SET); + offset = off; + if (ret != 0) + { + return -1; + } + return off; +} +size_t LocalCachedFileReader::size() +{ + if (file_size != 0) + return file_size; + if (local_path.empty()) + { + LOG_TRACE(&Poco::Logger::get("LocalCachedFileReader"), "empty local_path"); + return 0; + } + Poco::File file_handle(local_path); + auto ret = file_handle.getSize(); + file_size = ret; + return ret; +} + +// the size need be equal to the original buffer +RemoteReadBuffer::RemoteReadBuffer(size_t buff_size) : BufferWithOwnMemory(buff_size) +{ +} + +RemoteReadBuffer::~RemoteReadBuffer() = default; + +std::unique_ptr RemoteReadBuffer::create( + const std::string & schema_, + const std::string & cluster_, + const std::string & remote_path_, + UInt64 mod_ts_, + size_t file_size_, + std::unique_ptr readbuffer) +{ + size_t buff_size = DBMS_DEFAULT_BUFFER_SIZE; + if (readbuffer != nullptr) + buff_size = readbuffer->internalBuffer().size(); + /* + * in the new implement of ReadBufferFromHDFS, buffer size is 0. + * + * in the common case, we don't read bytes from readbuffer directly, so set buff_size = DBMS_DEFAULT_BUFFER_SIZE + * is OK. + * + * we need be careful with the case without local file reader. + */ + if (buff_size == 0) + buff_size = DBMS_DEFAULT_BUFFER_SIZE; + + auto rrb = std::make_unique(buff_size); + auto * raw_rbp = readbuffer.release(); + std::shared_ptr srb(raw_rbp); + RemoteReadBufferCache::CreateReaderError error; + int retry = 0; + do + { + if (retry > 0) + usleep(20 * retry); + + std::tie(rrb->file_reader, error) + = RemoteReadBufferCache::instance().createReader(schema_, cluster_, remote_path_, mod_ts_, file_size_, srb); + retry++; + } while (error == RemoteReadBufferCache::CreateReaderError::FILE_INVALID && retry < 10); + if (rrb->file_reader == nullptr) + { + LOG_ERROR(&Poco::Logger::get("RemoteReadBuffer"), "allocate local file failed for " + remote_path_ + "@" + std::to_string(error)); + rrb->original_readbuffer = srb; + } + return rrb; +} + +bool RemoteReadBuffer::nextImpl() +{ + if (file_reader != nullptr) + { + int bytes_read = file_reader->read(internal_buffer.begin(), internal_buffer.size()); + if (bytes_read) + working_buffer.resize(bytes_read); + else + { + return false; + } + } + else // in the case we cannot use local cache, read from the original readbuffer directly + { + if (original_readbuffer == nullptr) + throw Exception("original readbuffer should not be null", ErrorCodes::LOGICAL_ERROR); + auto status = original_readbuffer->next(); + // we don't need to worry about the memory buffer allocated in RemoteReadBuffer, since it is owned by + // BufferWithOwnMemory, BufferWithOwnMemory would release it. + //LOG_TRACE(&Poco::Logger::get("RemoteReadBuffer"), "from original rb {} {}", original_readbuffer->buffer().size(), original_readbuffer->offset()); + if (status) + BufferBase::set(original_readbuffer->buffer().begin(), original_readbuffer->buffer().size(), original_readbuffer->offset()); + return status; + } + return true; +} + +off_t RemoteReadBuffer::seek(off_t offset, int whence) +{ + off_t pos_in_file = file_reader->getOffset(); + off_t new_pos; + if (whence == SEEK_SET) + new_pos = offset; + else if (whence == SEEK_CUR) + new_pos = pos_in_file - (working_buffer.end() - pos) + offset; + else + throw Exception("expects SEEK_SET or SEEK_CUR as whence", ErrorCodes::BAD_ARGUMENTS); + + /// Position is unchanged. + if (new_pos + (working_buffer.end() - pos) == pos_in_file) + return new_pos; + + if (new_pos <= pos_in_file && new_pos >= pos_in_file - static_cast(working_buffer.size())) + { + /// Position is still inside buffer. + pos = working_buffer.begin() + (new_pos - (pos_in_file - working_buffer.size())); + return new_pos; + } + + pos = working_buffer.end(); + auto ret_off = file_reader->seek(new_pos); + if (ret_off == -1) + throw Exception( + "seek file failed. " + std::to_string(pos_in_file) + "->" + std::to_string(new_pos) + "@" + std::to_string(file_reader->size()) + + "," + std::to_string(whence) + "," + file_reader->getPath(), + ErrorCodes::BAD_ARGUMENTS); + return ret_off; +} + +off_t RemoteReadBuffer::getPosition() +{ + return file_reader->getOffset() - (working_buffer.end() - pos); +} + +RemoteReadBufferCache::RemoteReadBufferCache() = default; + +RemoteReadBufferCache::~RemoteReadBufferCache() = default; + +RemoteReadBufferCache & RemoteReadBufferCache::instance() +{ + static RemoteReadBufferCache instance; + return instance; +} + +void RemoteReadBufferCache::initOnce(const std::string & dir, size_t limit_size_) +{ + LOG_TRACE(log, "init local cache. path: {}, limit {}", dir, limit_size_); + std::lock_guard lock(mutex); + local_path_prefix = dir; + limit_size = limit_size_; + + // scan local disk dir and recover the cache metas + Poco::File root_dir(local_path_prefix); + if (!root_dir.exists()) + return; + auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; + + // four level dir. ///`` + std::vector schema_dirs; + root_dir.list(schema_dirs); + for (auto & schema_file : schema_dirs) + { + std::vector cluster_dirs; + schema_file.list(cluster_dirs); + for (auto & cluster_file : cluster_dirs) + { + std::vector file_dir1; + cluster_file.list(file_dir1); + for (auto & file1 : file_dir1) + { + std::vector file_dir2; + file1.list(file_dir2); + for (auto & file2 : file_dir2) + { + if (caches.find(file2.path()) != caches.end()) + { + LOG_ERROR(log, "ridiculous!! "); + continue; + } + auto cache_cntrl = RemoteCacheController::recover(file2.path(), callback); + if (cache_cntrl == nullptr) + continue; + CacheCell cell; + cell.cache_controller = cache_cntrl; + cell.key_iterator = keys.insert(keys.end(), file2.path()); + caches[file2.path()] = cell; + } + } + } + } + inited = true; +} + +std::string +RemoteReadBufferCache::calculateLocalPath(const std::string & schema_, const std::string & cluster_, const std::string & remote_path_) +{ + std::string local_path = local_path_prefix + "/" + schema_ + "/" + cluster_; + + Poco::MD5Engine md5; + Poco::DigestOutputStream outstr(md5); + outstr << remote_path_; + outstr.flush(); //to pass everything to the digest engine + const Poco::DigestEngine::Digest & digest = md5.digest(); + std::string md5string = Poco::DigestEngine::digestToHex(digest); + + local_path += "/" + md5string.substr(0, 3) + "/" + md5string; + + return local_path; +} + +std::tuple, RemoteReadBufferCache::CreateReaderError> RemoteReadBufferCache::createReader( + const std::string & schema, + const std::string & cluster, + const std::string & remote_path, + UInt64 mod_ts, + size_t file_size, + std::shared_ptr & readbuffer) +{ + if (!hasInitialized()) + { + LOG_ERROR(log, "RemoteReadBufferCache not init"); + return {nullptr, CreateReaderError::NOT_INIT}; + } + auto local_path = calculateLocalPath(schema, cluster, remote_path); + std::lock_guard lock(mutex); + auto citer = caches.find(local_path); + if (citer != caches.end()) + { + // if the file has been update on remote side, we need to redownload it + if (citer->second.cache_controller->getLastModTS() != mod_ts) + { + LOG_TRACE(log, + "remote file has been updated. " + remote_path + ":" + std::to_string(citer->second.cache_controller->getLastModTS()) + "->" + + std::to_string(mod_ts)); + citer->second.cache_controller->markInvalid(); + } + else + { + // move the key to the list end + keys.splice(keys.end(), keys, citer->second.key_iterator); + return {std::make_shared(citer->second.cache_controller.get(), file_size), CreateReaderError::OK}; + } + } + + auto clear_ret = clearLocalCache(); + citer = caches.find(local_path); + if (citer != caches.end()) + { + if (citer->second.cache_controller->isValid()) + { + // move the key to the list end, this case should not happend? + keys.splice(keys.end(), keys, citer->second.key_iterator); + return {std::make_shared(citer->second.cache_controller.get(), file_size), CreateReaderError::OK}; + } + else + { + // maybe someone is holding this file + return {nullptr, CreateReaderError::FILE_INVALID}; + } + } + + // reach the disk capacity limit + if (!clear_ret) + { + LOG_ERROR(log, "local cache is full, return nullptr"); + return {nullptr, CreateReaderError::DISK_FULL}; + } + + Poco::File file(local_path); + file.createDirectories(); + + auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; + auto cache_cntrl = std::make_shared(schema, cluster, remote_path, mod_ts, local_path, readbuffer, callback); + CacheCell cc; + cc.cache_controller = cache_cntrl; + cc.key_iterator = keys.insert(keys.end(), local_path); + caches[local_path] = cc; + + return {std::make_shared(cache_cntrl.get(), file_size), CreateReaderError::OK}; +} + +bool RemoteReadBufferCache::clearLocalCache() +{ + for (auto it = keys.begin(); it != keys.end();) + { + auto cit = caches.find(*it); + auto cntrl = cit->second.cache_controller; + if (!cntrl->isValid() && cntrl->closable()) + { + LOG_TRACE(log, "clear invalid cache: " + *it); + total_size = total_size > cit->second.cache_controller->size() ? total_size - cit->second.cache_controller->size() : 0; + cntrl->close(); + it = keys.erase(it); + caches.erase(cit); + } + else + it++; + } + // clear closable cache from the list head + for (auto it = keys.begin(); it != keys.end();) + { + if (total_size < limit_size) + break; + auto cit = caches.find(*it); + if (cit == caches.end()) + { + throw Exception("file not found in cache?" + *it, ErrorCodes::LOGICAL_ERROR); + } + if (cit->second.cache_controller->closable()) + { + total_size = total_size > cit->second.cache_controller->size() ? total_size - cit->second.cache_controller->size() : 0; + cit->second.cache_controller->close(); + caches.erase(cit); + it = keys.erase(it); + LOG_TRACE(log, "clear local file {} for {}. key size:{}. next{}", cit->second.cache_controller->getLocalPath(), + cit->second.cache_controller->getRemotePath(), keys.size(), *it); + } + else + break; + } + LOG_TRACE(log, "keys size:{}, total_size:{}, limit size:{}", keys.size(), total_size, limit_size); + return total_size < limit_size * 1.5; +} + +} diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h new file mode 100644 index 00000000000..f79f093150b --- /dev/null +++ b/src/IO/RemoteReadBufferCache.h @@ -0,0 +1,211 @@ +#pragma once +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +/** + * + */ +class RemoteCacheController +{ +public: + RemoteCacheController( + const std::string & schema_, + const std::string & cluster_, + const std::string & path_, + UInt64 mod_ts, + const std::string & local_path_, + std::shared_ptr readbuffer_, + std::function const & finish_callback); + ~RemoteCacheController(); // the local files will be deleted in descontructor + + // recover from local disk + static std::shared_ptr + recover(const std::string & local_path, std::function const & finish_callback); + + /** + * called by LocalCachedFileReader, must be used in pair + * local_path will be empty if the file has not been downloaded + */ + FILE * allocFile(std::string * local_path); + void deallocFile(FILE * fs_); + + /** + * when allocFile be called, count++. deallocFile be called, count--. + * the local file could be deleted only count==0 + */ + inline bool closable() + { + std::lock_guard lock{mutex}; + return opened_file_streams.empty() && remote_readbuffer == nullptr; + } + void close(); + + /** + * called in LocalCachedFileReader read(), the reading process would be blocked until + * enough data be downloaded. + * If the file has finished download, the process would be block + * returns + * - 0: has more data to read + * - -1: has reach eof + */ + int waitMoreData(size_t start_offset_, size_t end_offset_); + + inline size_t size() const { return current_offset; } + + inline const std::string & getLocalPath() { return local_path; } + inline const std::string & getRemotePath() { return remote_path; } + + inline UInt64 getLastModTS() const { return last_mod_ts; } + inline void markInvalid() + { + std::lock_guard lock(mutex); + valid = false; + } + inline bool isValid() + { + std::lock_guard lock(mutex); + return valid; + } + +private: + // flush file and meta info into disk + void flush(); + + void backgroupDownload(std::function const & finish_callback); + + std::mutex mutex; + std::condition_variable more_data_signal; + ThreadPool * download_thread; + + std::set opened_file_streams; + + // meta info + bool download_finished; + bool valid; + size_t current_offset; + UInt64 last_mod_ts; + std::string local_path; + std::string remote_path; + std::string schema; + std::string cluster; + + std::shared_ptr remote_readbuffer; + Poco::FileOutputStream * out_file; +}; + +/** + * access local cached files by RemoteCacheController, and be used in RemoteReadBuffer + */ +class LocalCachedFileReader +{ +public: + LocalCachedFileReader(RemoteCacheController * cntrl_, size_t size_); + ~LocalCachedFileReader(); + + // expect to read size bytes into buf, return is the real bytes read + size_t read(char * buf, size_t size); + inline off_t getOffset() const { return static_cast(offset); } + size_t size(); + off_t seek(off_t offset); + inline std::string getPath() { return local_path; } + +private: + std::mutex mutex; + size_t offset; + size_t file_size; + FILE * fs; + std::string local_path; + RemoteCacheController * controller; +}; + +/* + * FIXME:RemoteReadBuffer derive from SeekableReadBuffer may case some risks, since it's not seekable in some cases + * But SeekableReadBuffer is not a interface which make it hard to fixup. + */ +class RemoteReadBuffer : public BufferWithOwnMemory +{ +public: + explicit RemoteReadBuffer(size_t buff_size); + ~RemoteReadBuffer() override; + static std::unique_ptr create( + const std::string & schema_, + const std::string & cluster_, + const std::string & remote_path_, + UInt64 mod_ts_, + size_t file_size_, + std::unique_ptr readbuffer); + + bool nextImpl() override; + inline bool seekable() { return file_reader != nullptr && file_reader->size() > 0; } + off_t seek(off_t off, int whence) override; + off_t getPosition() override; + inline size_t size() { return file_reader->size(); } + +private: + std::shared_ptr file_reader; + std::shared_ptr original_readbuffer; +}; + + +class RemoteReadBufferCache +{ +protected: + RemoteReadBufferCache(); + +public: + ~RemoteReadBufferCache(); + // global instance + static RemoteReadBufferCache & instance(); + + void initOnce(const std::string & dir, size_t limit_size); + inline bool hasInitialized() const { return inited; } + + enum CreateReaderError + { + OK = 0, + NOT_INIT = -1, + DISK_FULL = -2, + FILE_INVALID = -3 + }; + std::tuple, CreateReaderError> createReader( + const std::string & schema, + const std::string & cluster, + const std::string & remote_path, + UInt64 mod_ts, + size_t file_size, + std::shared_ptr & readbuffer); + +private: + std::string local_path_prefix; + + std::atomic inited = false; + std::mutex mutex; + size_t limit_size; + std::atomic total_size; + Poco::Logger * log = &Poco::Logger::get("RemoteReadBufferCache"); + + struct CacheCell + { + std::list::iterator key_iterator; + std::shared_ptr cache_controller; + }; + std::list keys; + std::map caches; + + std::string calculateLocalPath(const std::string & schema_, const std::string & cluster_, const std::string & remote_path_); + bool clearLocalCache(); +}; + +} diff --git a/src/IO/SnappyReadBuffer.cpp b/src/IO/SnappyReadBuffer.cpp new file mode 100644 index 00000000000..c75aee9dc3a --- /dev/null +++ b/src/IO/SnappyReadBuffer.cpp @@ -0,0 +1,76 @@ +#include + +#if USE_SNAPPY +#include +#include +#include + +#include + +#include +#include +#include + +#include "SnappyReadBuffer.h" + +namespace DB +{ +namespace ErrorCodes +{ + extern const int SNAPPY_UNCOMPRESS_FAILED; + extern const int SEEK_POSITION_OUT_OF_BOUND; +} + + +SnappyReadBuffer::SnappyReadBuffer(std::unique_ptr in_, size_t buf_size, char * existing_memory, size_t alignment) + : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)) +{ +} + +bool SnappyReadBuffer::nextImpl() +{ + if (compress_buffer.empty() && uncompress_buffer.empty()) + { + WriteBufferFromString wb(compress_buffer); + copyData(*in, wb); + + bool success = snappy::Uncompress(compress_buffer.data(), wb.count(), &uncompress_buffer); + if (!success) + { + throw Exception("snappy uncomress failed: ", ErrorCodes::SNAPPY_UNCOMPRESS_FAILED); + } + BufferBase::set(const_cast(uncompress_buffer.data()), uncompress_buffer.size(), 0); + return true; + } + return false; +} + +SnappyReadBuffer::~SnappyReadBuffer() = default; + +off_t SnappyReadBuffer::seek(off_t off, int whence) +{ + off_t new_pos; + if (whence == SEEK_SET) + new_pos = off; + else if (whence == SEEK_CUR) + new_pos = count() + off; + else + throw Exception("Only SEEK_SET and SEEK_CUR seek modes allowed.", ErrorCodes::SEEK_POSITION_OUT_OF_BOUND); + + working_buffer = internal_buffer; + if (new_pos < 0 || new_pos > off_t(working_buffer.size())) + throw Exception( + String("Cannot seek through buffer") + " because seek position (" + toString(new_pos) + ") is out of bounds [0, " + + toString(working_buffer.size()) + "]", + ErrorCodes::SEEK_POSITION_OUT_OF_BOUND); + position() = working_buffer.begin() + new_pos; + return new_pos; +} + +off_t SnappyReadBuffer::getPosition() +{ + return count(); +} + +} +#endif diff --git a/src/IO/SnappyReadBuffer.h b/src/IO/SnappyReadBuffer.h new file mode 100644 index 00000000000..e440f2d3003 --- /dev/null +++ b/src/IO/SnappyReadBuffer.h @@ -0,0 +1,35 @@ +#pragma once + +#include + +#if USE_SNAPPY + +#include +#include +#include + +namespace DB +{ +class SnappyReadBuffer : public BufferWithOwnMemory +{ +public: + explicit SnappyReadBuffer( + std::unique_ptr in_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0); + + ~SnappyReadBuffer() override; + + bool nextImpl() override; + off_t seek(off_t off, int whence) override; + off_t getPosition() override; + +private: + std::unique_ptr in; + String compress_buffer; + String uncompress_buffer; +}; + +} +#endif diff --git a/src/IO/SnappyWriteBuffer.cpp b/src/IO/SnappyWriteBuffer.cpp new file mode 100644 index 00000000000..9cbdaab330f --- /dev/null +++ b/src/IO/SnappyWriteBuffer.cpp @@ -0,0 +1,92 @@ +#include + +#if USE_SNAPPY +#include + +#include + +#include +#include "SnappyWriteBuffer.h" + +namespace DB +{ +namespace ErrorCodes +{ + extern const int SNAPPY_COMPRESS_FAILED; +} + +SnappyWriteBuffer::SnappyWriteBuffer(std::unique_ptr out_, size_t buf_size, char * existing_memory, size_t alignment) + : BufferWithOwnMemory(buf_size, existing_memory, alignment), out(std::move(out_)) +{ +} + +SnappyWriteBuffer::~SnappyWriteBuffer() +{ + finish(); +} + +void SnappyWriteBuffer::nextImpl() +{ + if (!offset()) + { + return; + } + + const char * in_data = reinterpret_cast(working_buffer.begin()); + size_t in_available = offset(); + uncompress_buffer.append(in_data, in_available); +} + +void SnappyWriteBuffer::finish() +{ + if (finished) + return; + + try + { + finishImpl(); + out->finalize(); + finished = true; + } + catch (...) + { + /// Do not try to flush next time after exception. + out->position() = out->buffer().begin(); + finished = true; + throw; + } +} + +void SnappyWriteBuffer::finishImpl() +{ + next(); + + bool success = snappy::Compress(uncompress_buffer.data(), uncompress_buffer.size(), &compress_buffer); + if (!success) + { + throw Exception("snappy compress failed: ", ErrorCodes::SNAPPY_COMPRESS_FAILED); + } + + char * in_data = compress_buffer.data(); + size_t in_available = compress_buffer.size(); + char * out_data = nullptr; + size_t out_capacity = 0; + size_t len = 0; + while (in_available > 0) + { + out->nextIfAtEnd(); + out_data = out->position(); + out_capacity = out->buffer().end() - out->position(); + len = in_available > out_capacity ? out_capacity : in_available; + + memcpy(out_data, in_data, len); + in_data += len; + in_available -= len; + out->position() += len; + } +} + +} + +#endif + diff --git a/src/IO/SnappyWriteBuffer.h b/src/IO/SnappyWriteBuffer.h new file mode 100644 index 00000000000..0d3a6f5904f --- /dev/null +++ b/src/IO/SnappyWriteBuffer.h @@ -0,0 +1,41 @@ +#pragma once + +#include + +#if USE_SNAPPY +#include +#include + +namespace DB +{ +/// Performs compression using snappy library and write compressed data to the underlying buffer. +class SnappyWriteBuffer : public BufferWithOwnMemory +{ +public: + explicit SnappyWriteBuffer( + std::unique_ptr out_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0); + + ~SnappyWriteBuffer() override; + + void finalize() override { finish(); } + +private: + void nextImpl() override; + + void finishImpl(); + void finish(); + + std::unique_ptr out; + bool finished = false; + + String uncompress_buffer; + String compress_buffer; +}; + +} + +#endif + diff --git a/src/IO/examples/CMakeLists.txt b/src/IO/examples/CMakeLists.txt index d5907bf67ad..0d335d66d27 100644 --- a/src/IO/examples/CMakeLists.txt +++ b/src/IO/examples/CMakeLists.txt @@ -72,3 +72,10 @@ target_link_libraries (dragonbox_test PRIVATE dragonbox_to_chars) add_executable (zstd_buffers zstd_buffers.cpp) target_link_libraries (zstd_buffers PRIVATE clickhouse_common_io) + +add_executable (snappy_read_buffer snappy_read_buffer.cpp) +target_link_libraries (snappy_read_buffer PRIVATE clickhouse_common_io) + +add_executable (hadoop_snappy_read_buffer hadoop_snappy_read_buffer.cpp) +target_link_libraries (hadoop_snappy_read_buffer PRIVATE clickhouse_common_io) + diff --git a/src/IO/examples/hadoop_snappy_read_buffer.cpp b/src/IO/examples/hadoop_snappy_read_buffer.cpp new file mode 100644 index 00000000000..9cb01e6d697 --- /dev/null +++ b/src/IO/examples/hadoop_snappy_read_buffer.cpp @@ -0,0 +1,43 @@ +#include +#include + +#include + +#include +#include +#include +#include +#include +#include + +std::string uncompress(size_t buf_size) +{ + using namespace DB; + + String path = "test.snappy"; + std::unique_ptr in1 = std::make_unique(path, buf_size); + HadoopSnappyReadBuffer in2(std::move(in1)); + + String output; + WriteBufferFromString out(output); + copyData(in2, out); + + output.resize(out.count()); + return output; +} + +int main() +{ + auto output = uncompress(1024 * 1024); + for (size_t i = 1; i < 1024; ++i) + { + size_t buf_size = 1024 * i; + if (uncompress(buf_size) != output) + { + std::cout << "test hadoop snappy read buffer failed, buf_size:" << buf_size << std::endl; + return 1; + } + } + std::cout << "test hadoop snappy read buffer success" << std::endl; + return 0; +} diff --git a/src/IO/examples/snappy_read_buffer.cpp b/src/IO/examples/snappy_read_buffer.cpp new file mode 100644 index 00000000000..ccf2f70f1c4 --- /dev/null +++ b/src/IO/examples/snappy_read_buffer.cpp @@ -0,0 +1,35 @@ +#include +#include + +#include + +#include +#include +#include +#include +#include +#include + + +int main() +{ + using namespace DB; + String str = "this is a snappy example.\n"; + String input; + for (size_t i = 0; i < 5; i++) + input += str; + + String input1; + snappy::Compress(input.data(), input.size(), &input1); + + std::unique_ptr in1 = std::make_unique(input1); + SnappyReadBuffer in2(std::move(in1)); + + String output; + WriteBufferFromString out(output); + copyData(in2, out); + + output.resize(out.count()); + assert(input == output); + return 0; +} diff --git a/src/IO/examples/test.snappy b/src/IO/examples/test.snappy new file mode 100644 index 0000000000000000000000000000000000000000..1036e7e3a4d66ebdae86cf4d12a5fa9893b9dd90 GIT binary patch literal 5590 zcmc(j%Z}Vc7=_(RWeBZ6+5#jZvG5`b76`fPVwbHbnk5SsJj9GN3rj8aW?q3ufN&?P z@Bs4^yaJB^Z!m&$s=HmbDPJl~hRu_p^SO5Q4B}tYf$tt}$vsaf*v&*h``{ts~ zwrlw$Nn`<06iqRdf>Kf}rGkiUUT>26nP#TikeIfEj!tp8lO#~Cpg{mg7nOu&oB@Zz{(j6AicsaqjlhP95GQnaS_7k%lqvx>d8D8A#|cJaf72Uaf2r@`Lrrp2;jv7>=Tz) zWL389@1@5Wf#C5-<@2u8*sLl)D18|PVzeVFrntfX6%;u! zql|Ffj6*{SQ83+h|BQTCA%k2t-~siu3=$!P7)?IYZ7SLAlO*~3B}_r>rFEN1cKdf1B@>TJ0F8NEM4{^6=aSG{ zY#cDT@&R3M^=G@r-Y=kxN;R$@`M^Ix!5p)_C z8qrB$#07=>8|nXp%O}dk(q;!s*0AJsa4VFHrOobX8V;sor1a77khmugz|yA>3e#28SoMMNm@$BOuXI`GpMT8aO530(mThNYP$vp z3(C*K0nSC1NkC$F5eDbx)x8xkS$Rk=J7kyDN3h~}#Zi{0^GlivJ0ClJatYBe-9}cd zVQA{kP_-03p0;P z6y`93rD^wi!$`{9B%m!P5dd217gqlTX6f`t>o|gm{>2+myV^02Q|| t;q~o<^Ui_3Dw%m?L>P6*(1igRqd)t|h)UIe^vV`TiWe}0TRL37`~&nyg8cvh literal 0 HcmV?d00001 diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index f2d449c04d7..8b2a6144708 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -26,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -84,8 +86,12 @@ #include #include #include +#include #include #include +#include +#include +#include #include @@ -155,6 +161,10 @@ struct ContextSharedPart mutable std::mutex storage_policies_mutex; /// Separate mutex for re-initialization of zookeeper session. This operation could take a long time and must not interfere with another operations. mutable std::mutex zookeeper_mutex; + /// Separate mutex for re-initialization of hive metastore client. This operation could take a long time and must not interfere with another operations. + mutable std::mutex hive_metastore_mutex; + /// Separate mutex for re-initialization of hdfs file system. This operation could take a long time and must not interfere with another operations. + mutable std::mutex hdfs_filesystem_mutex; mutable zkutil::ZooKeeperPtr zookeeper; /// Client for ZooKeeper. ConfigurationPtr zookeeper_config; /// Stores zookeeper configs @@ -167,6 +177,9 @@ struct ContextSharedPart mutable std::map auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients. ConfigurationPtr auxiliary_zookeepers_config; /// Stores auxiliary zookeepers configs + mutable std::map hive_metastore_clients; /// Map for hive metastore clients + // mutable std::map hdfs_filesystems; /// Map for hdfs file systems. + String interserver_io_host; /// The host name by which this server is available for other servers. UInt16 interserver_io_port = 0; /// and port. String interserver_scheme; /// http or https @@ -1808,6 +1821,54 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const return shared->zookeeper; } +HMSClientPtr Context::getHMSClient(const String & name) const +{ + using namespace apache::thrift; + using namespace apache::thrift::protocol; + using namespace apache::thrift::transport; + using namespace Apache::Hadoop::Hive; + + std::lock_guard lock(shared->hive_metastore_mutex); + auto it = shared->hive_metastore_clients.find(name); + if (it == shared->hive_metastore_clients.end() || it->second->isExpired()) + { + // connect to hive metastore + Poco::URI hms_url(name); + const auto& host = hms_url.getHost(); + auto port = hms_url.getPort(); + + std::shared_ptr socket = std::make_shared(host, port); + socket->setKeepAlive(true); + socket->setConnTimeout(60000); + socket->setRecvTimeout(60000); + socket->setSendTimeout(60000); + std::shared_ptr transport(new TBufferedTransport(socket)); + std::shared_ptr protocol(new TBinaryProtocol(transport)); + std::shared_ptr client = std::make_shared(protocol); + try + { + transport->open(); + } + catch (TException & tx) + { + throw Exception("connect to hive metastore:" + name + " failed." + tx.what(), ErrorCodes::BAD_ARGUMENTS); + } + + if (it == shared->hive_metastore_clients.end()) + { + HMSClientPtr hms_client = std::make_shared(std::move(client), shared_from_this()); + shared->hive_metastore_clients[name] = hms_client; + return hms_client; + } + else + { + it->second->setClient(std::move(client)); + return it->second; + } + } + return it->second; +} + namespace { @@ -1827,10 +1888,8 @@ bool checkZooKeeperConfigIsLocal(const Poco::Util::AbstractConfiguration & confi } return false; } - } - bool Context::tryCheckClientConnectionToMyKeeperCluster() const { try diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 7d31a8375d8..448efecc295 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -27,7 +27,8 @@ namespace Poco::Net { class IPAddress; } namespace zkutil { class ZooKeeper; } - +namespace hdfs { class FileSystem; } +// namespace Apache { namespace Hadoop { namespace Hive { class ThriftHiveMetastoreClient; }}} namespace DB { @@ -147,6 +148,9 @@ using InputBlocksReader = std::function; /// Used in distributed task processing using ReadTaskCallback = std::function; +// class HDFSFSPtr; +// using HDFSFileSystemPtr = std::shared_ptr; + /// An empty interface for an arbitrary object that may be attached by a shared pointer /// to query context, when using ClickHouse as a library. struct IHostContext @@ -174,6 +178,9 @@ private: std::unique_ptr shared; }; +class HMSClient; +using HMSClientPtr = std::shared_ptr; + /** A set of known objects that can be used in the query. * Consists of a shared part (always common to all sessions and queries) * and copied part (which can be its own for each session or query). @@ -688,6 +695,9 @@ public: // Reload Zookeeper void reloadZooKeeperIfChanged(const ConfigurationPtr & config) const; + HMSClientPtr getHMSClient(const String & name) const; + // HDFSFileSystemPtr getHDFSFileSystem(const String & name) const; + void setSystemZooKeeperLogAfterInitializationIfNeeded(); /// Create a cache of uncompressed blocks of specified size. This can be done only once. diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp index f98e45a4e49..9e781307a36 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp @@ -5,13 +5,17 @@ #include "ArrowBufferedStreams.h" #if USE_ARROW || USE_ORC || USE_PARQUET - +#include #include #include #include +#include #include #include #include +#include +#include +#include #include @@ -132,6 +136,12 @@ std::shared_ptr asArrowFile(ReadBuffer & in) return std::make_shared(*fd_in, stat.st_size); } + if (auto * fd_in = dynamic_cast(&in)) + { + if (fd_in->seekable()) + return std::make_shared(*fd_in, fd_in->size()); + } + // fallback to loading the entire file in memory std::string file_data; { diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 3dada0d62be..2f875ee3f31 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -61,7 +61,6 @@ namespace ErrorCodes { extern const int UNKNOWN_TYPE; extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; - extern const int THERE_IS_NO_COLUMN; extern const int BAD_ARGUMENTS; extern const int UNKNOWN_EXCEPTION; } @@ -519,7 +518,7 @@ ArrowColumnToCHColumn::ArrowColumnToCHColumn( void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table) { Columns columns_list; - UInt64 num_rows = 0; + UInt64 num_rows = table->num_rows(); columns_list.reserve(header.rows()); @@ -555,10 +554,15 @@ void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptrhas(header_column.name); } - - // TODO: What if some columns were not presented? Insert NULLs? What if a column is not nullable? if (!read_from_nested) - throw Exception{ErrorCodes::THERE_IS_NO_COLUMN, "Column '{}' is not presented in input data.", header_column.name}; + { + ColumnWithTypeAndName column; + column.name = header_column.name; + column.type = header_column.type; + column.column = header_column.column->cloneResized(num_rows); + columns_list.push_back(std::move(column.column)); + continue; + } } std::shared_ptr arrow_column = name_to_column_ptr[header_column.name]; @@ -581,7 +585,6 @@ void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptrsize(); columns_list.push_back(std::move(column.column)); } diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 9de2b908b1e..dbb31fc579e 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -36,6 +36,86 @@ CSVRowInputFormat::CSVRowInputFormat( ErrorCodes::BAD_ARGUMENTS); } +void CSVRowInputFormat::readPrefix() +{ + if (with_names || with_types || data_types.at(0)->textCanContainOnlyValidUTF8()) + { + /// We assume that column name or type cannot contain BOM, so, if format has header, + /// then BOM at beginning of stream cannot be confused with name or type of field, and it is safe to skip it. + skipBOMIfExists(*in); + } + + /// This is a bit of abstraction leakage, but we need it in parallel parsing: + /// we check if this InputFormat is working with the "real" beginning of the data. + if (with_names && getCurrentUnitNumber() == 0) + { + if (format_settings.with_names_use_header) + { + std::vector read_columns(data_types.size(), false); + + if (format_settings.csv.input_field_names.empty()) + { + auto column_names = readNames(); + for (const auto & name : column_names) + addInputColumn(name, read_columns); + } + else + { + /// For Hive Text file, read the first row to get exact number of columns. + char * old_pos = in->position(); + auto values = readHeaderRow(); + in->position() = old_pos; + + input_field_names = format_settings.csv.input_field_names; + input_field_names.resize(values.size()); + for (const auto & column_name : input_field_names) + addInputColumn(column_name, read_columns); + } + + for (size_t i = 0; i != read_columns.size(); ++i) + { + if (!read_columns[i]) + column_mapping->not_presented_columns.push_back(i); + } + } + else + { + setupAllColumnsByTableSchema(); + skipNames(); + } + } + else if (!column_mapping->is_set) + setupAllColumnsByTableSchema(); + + if (with_types && getCurrentUnitNumber() == 0) + { + if (format_settings.with_types_use_header) + { + auto types = readTypes(); + if (types.size() != column_mapping->column_indexes_for_input_fields.size()) + throw Exception( + ErrorCodes::INCORRECT_DATA, + "The number of data types differs from the number of column names in input data"); + + /// Check that types from input matches types from header. + for (size_t i = 0; i < types.size(); ++i) + { + if (column_mapping->column_indexes_for_input_fields[i] && + data_types[*column_mapping->column_indexes_for_input_fields[i]]->getName() != types[i]) + { + throw Exception( + ErrorCodes::INCORRECT_DATA, + "Type of '{}' must be {}, not {}", + getPort().getHeader().getByPosition(*column_mapping->column_indexes_for_input_fields[i]).name, + data_types[*column_mapping->column_indexes_for_input_fields[i]]->getName(), types[i]); + } + } + } + else + skipTypes(); + } +} + static void skipEndOfLine(ReadBuffer & in) { /// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index f239464485a..1d98c22a569 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -25,6 +25,8 @@ public: String getName() const override { return "CSVRowInputFormat"; } + void readPrefix() override; + bool allowSyncAfterError() const override { return true; } void syncAfterError() override; @@ -52,6 +54,8 @@ private: std::vector readTypes() override { return readHeaderRow(); } String readFieldIntoString(); + + std::vector input_field_names; }; } diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index 0fd83238f5f..545a395f48a 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -30,8 +30,8 @@ public: const Params & params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_); - bool readRow(MutableColumns & columns, RowReadExtension & ext) override; - void readPrefix() override; + virtual bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + virtual void readPrefix() override; void resetParser() override; protected: @@ -58,25 +58,28 @@ protected: virtual bool parseRowEndWithDiagnosticInfo(WriteBuffer &) { return true;} bool isGarbageAfterField(size_t, ReadBuffer::Position) override {return false; } + virtual void insertDefaultsForNotSeenColumns(MutableColumns & columns, RowReadExtension & ext); + /// Read row with names and return the list of them. virtual std::vector readNames() = 0; /// Read row with types and return the list of them. virtual std::vector readTypes() = 0; + virtual void addInputColumn(const String & column_name, std::vector & read_columns); + virtual void setupAllColumnsByTableSchema(); + const FormatSettings format_settings; DataTypes data_types; + bool with_names; + bool with_types; + std::unordered_map column_indexes_by_names; + private: bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override; void tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) override; - void setupAllColumnsByTableSchema(); - void addInputColumn(const String & column_name, std::vector & read_columns); - void insertDefaultsForNotSeenColumns(MutableColumns & columns, RowReadExtension & ext); - bool with_names; - bool with_types; - std::unordered_map column_indexes_by_names; }; void registerFileSegmentationEngineForFormatWithNamesAndTypes( diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 2a515fdf3be..01df264005b 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -27,6 +27,9 @@ public: using AggregatorList = std::list; using AggregatorListPtr = std::shared_ptr; +using AggregatorList = std::list; +using AggregatorListPtr = std::shared_ptr; + struct AggregatingTransformParams { Aggregator::Params params; diff --git a/src/Processors/examples/CMakeLists.txt b/src/Processors/examples/CMakeLists.txt index e69de29bb2d..6f78d611f45 100644 --- a/src/Processors/examples/CMakeLists.txt +++ b/src/Processors/examples/CMakeLists.txt @@ -0,0 +1,2 @@ +add_executable (comma_separated_streams comma_separated_streams.cpp) +target_link_libraries (comma_separated_streams PRIVATE dbms) \ No newline at end of file diff --git a/src/Processors/examples/comma_separated_streams.cpp b/src/Processors/examples/comma_separated_streams.cpp new file mode 100644 index 00000000000..a615957561c --- /dev/null +++ b/src/Processors/examples/comma_separated_streams.cpp @@ -0,0 +1,121 @@ +#include + +#include +#include + +#include +#include + +#include +#include + +#include +#include +#include +#include +#include + +using namespace DB; + +int main(int, char **) +try +{ + Block sample; + { + // a + ColumnWithTypeAndName col; + col.name = "a"; + col.type = std::make_shared(); + sample.insert(std::move(col)); + } + { + // b + ColumnWithTypeAndName col; + col.name = "b"; + col.type = std::make_shared(); + sample.insert(std::move(col)); + } + { + // c + ColumnWithTypeAndName col; + col.name = "c"; + col.type = std::make_shared(); + sample.insert(std::move(col)); + } + { + // d + ColumnWithTypeAndName col; + col.name = "d"; + col.type = std::make_shared(); + sample.insert(std::move(col)); + } + { + // e + ColumnWithTypeAndName col; + col.name = "e"; + col.type = std::make_shared(); + sample.insert(std::move(col)); + } + { + // f + ColumnWithTypeAndName col; + col.name = "f"; + col.type = std::make_shared(); + sample.insert(std::move(col)); + } + { + // g + ColumnWithTypeAndName col; + col.name = "g"; + col.type = std::make_shared(); + sample.insert(std::move(col)); + } + { + // h + ColumnWithTypeAndName col; + col.name = "h"; + col.type = std::make_shared(); + sample.insert(std::move(col)); + } + + + ReadBufferFromFile in_buf("test_in"); + WriteBufferFromFile out_buf("test_out"); + + FormatSettings format_settings; + format_settings.csv.input_field_names = + { + "d", + "e", + "f", + "a", + "b", + "c", + "g", + "h", + "i", + "j", + }; + format_settings.csv.delimiter = '\x01'; + format_settings.with_names_use_header = true; + + RowInputFormatParams in_params{DEFAULT_INSERT_BLOCK_SIZE}; + InputFormatPtr input_format = std::make_shared(sample, in_buf, in_params, true, false, format_settings); + auto pipeline = QueryPipeline(std::move(input_format)); + auto reader = std::make_unique(pipeline); + + RowOutputFormatParams out_params; + OutputFormatPtr output_format = std::make_shared(out_buf, sample, true, true, out_params, format_settings); + Block res; + while (reader->pull(res)) + { + output_format->write(res); + } + output_format->flush(); + return 0; +} +catch (...) +{ + std::cerr << getCurrentExceptionMessage(true) << '\n'; + return 1; +} diff --git a/src/Processors/examples/test_in b/src/Processors/examples/test_in new file mode 100644 index 00000000000..c7df97a26a6 --- /dev/null +++ b/src/Processors/examples/test_in @@ -0,0 +1,8 @@ +2021-09-14JPall20.0200 +2021-09-14CIall20.0100 +2021-09-14JMall40.25411 +2021-09-14MMall310.19354838709677422766 +2021-09-14TZAndroid30.3333333333333333311 +2021-09-14SGall80.25412 +2021-09-14PYall11.0001 +2021-09-14MXall10.0100 diff --git a/src/Storages/HDFS/HDFSCommon.cpp b/src/Storages/HDFS/HDFSCommon.cpp index 25c454e7500..f16c7c75c9b 100644 --- a/src/Storages/HDFS/HDFSCommon.cpp +++ b/src/Storages/HDFS/HDFSCommon.cpp @@ -197,6 +197,21 @@ HDFSFSPtr createHDFSFS(hdfsBuilder * builder) return fs; } +String getNameNodeUrl(const String & hdfs_url) +{ + const size_t pos = hdfs_url.find('/', hdfs_url.find("//") + 2); + String namenode_url = hdfs_url.substr(0, pos) + "/"; + return namenode_url; +} + +String getNameNodeCluster(const String &hdfs_url) +{ + auto pos1 = hdfs_url.find("//") + 2; + auto pos2 = hdfs_url.find('/', pos1); + + return hdfs_url.substr(pos1, pos2 - pos1); +} + } #endif diff --git a/src/Storages/HDFS/HDFSCommon.h b/src/Storages/HDFS/HDFSCommon.h index 877bf7dd74f..a977aa4f21a 100644 --- a/src/Storages/HDFS/HDFSCommon.h +++ b/src/Storages/HDFS/HDFSCommon.h @@ -98,5 +98,8 @@ using HDFSFSPtr = std::unique_ptr, detail::HDFSFsD HDFSBuilderWrapper createHDFSBuilder(const String & uri_str, const Poco::Util::AbstractConfiguration &); HDFSFSPtr createHDFSFS(hdfsBuilder * builder); +String getNameNodeUrl(const String & hdfs_url); +String getNameNodeCluster(const String & hdfs_url); + } #endif diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 4139e9599aa..2df32df7f92 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -136,6 +136,8 @@ struct MergeListElement : boost::noncopyable MergeListElement * ptr() { return this; } ~MergeListElement(); + + MergeListElement & ref() { return *this; } }; /** Maintains a list of currently running merges. diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index adbb51a3732..1dd8d8bb772 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -33,6 +33,7 @@ void registerStorageCOS(StorageFactory & factory); #if USE_HDFS void registerStorageHDFS(StorageFactory & factory); +void registerStorageHive(StorageFactory & factory); #endif void registerStorageODBC(StorageFactory & factory); @@ -104,6 +105,7 @@ void registerStorages() #if USE_HDFS registerStorageHDFS(factory); + registerStorageHive(factory); #endif registerStorageODBC(factory); From 3a566eeeeb96473c9c32f65f0fa3008f3cfc6737 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 5 Nov 2021 20:27:37 +0800 Subject: [PATCH 0013/1260] fix typo --- contrib/hive-metastore/CMakeLists.txt | 18 + contrib/hive-metastore/hive_metastore.thrift | 1807 ++++++++++++++++++ src/IO/RemoteReadBufferCache.cpp | 4 +- 3 files changed, 1827 insertions(+), 2 deletions(-) create mode 100644 contrib/hive-metastore/CMakeLists.txt create mode 100644 contrib/hive-metastore/hive_metastore.thrift diff --git a/contrib/hive-metastore/CMakeLists.txt b/contrib/hive-metastore/CMakeLists.txt new file mode 100644 index 00000000000..6e6a74758eb --- /dev/null +++ b/contrib/hive-metastore/CMakeLists.txt @@ -0,0 +1,18 @@ +set (SRCS + ${CMAKE_CURRENT_BINARY_DIR}/hive_metastore_constants.cpp + ${CMAKE_CURRENT_BINARY_DIR}/hive_metastore_types.cpp + ${CMAKE_CURRENT_BINARY_DIR}/ThriftHiveMetastore.cpp +) + +add_custom_command( + OUTPUT ${SRCS} + COMMAND thrift + ARGS -r --gen cpp -out ${CMAKE_CURRENT_BINARY_DIR} ${CMAKE_CURRENT_LIST_DIR}/hive_metastore.thrift + COMMENT "Running cpp thrift compiler for hive_metastore.thrift" + VERBATIM) + +add_library(hivemetastore ${SRCS}) +add_dependencies(hivemetastore ${THRIFT_LIBRARY}) +target_link_libraries(hivemetastore PUBLIC ${THRIFT_LIBRARY}) +target_include_directories(hivemetastore SYSTEM PUBLIC ${CMAKE_CURRENT_BINARY_DIR}) + diff --git a/contrib/hive-metastore/hive_metastore.thrift b/contrib/hive-metastore/hive_metastore.thrift new file mode 100644 index 00000000000..60d91b56492 --- /dev/null +++ b/contrib/hive-metastore/hive_metastore.thrift @@ -0,0 +1,1807 @@ +#!/usr/local/bin/thrift -java + +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +# +# Thrift Service that the MetaStore is built on +# + +namespace java org.apache.hadoop.hive.metastore.api +namespace php metastore +namespace cpp Apache.Hadoop.Hive + +const string DDL_TIME = "transient_lastDdlTime" + +struct Version { + 1: string version, + 2: string comments +} + +struct FieldSchema { + 1: string name, // name of the field + 2: string type, // type of the field. primitive types defined above, specify list, map for lists & maps + 3: string comment +} + +struct SQLPrimaryKey { + 1: string table_db, // table schema + 2: string table_name, // table name + 3: string column_name, // column name + 4: i32 key_seq, // sequence number within primary key + 5: string pk_name, // primary key name + 6: bool enable_cstr, // Enable/Disable + 7: bool validate_cstr, // Validate/No validate + 8: bool rely_cstr // Rely/No Rely +} + +struct SQLForeignKey { + 1: string pktable_db, // primary key table schema + 2: string pktable_name, // primary key table name + 3: string pkcolumn_name, // primary key column name + 4: string fktable_db, // foreign key table schema + 5: string fktable_name, // foreign key table name + 6: string fkcolumn_name, // foreign key column name + 7: i32 key_seq, // sequence within foreign key + 8: i32 update_rule, // what happens to foreign key when parent key is updated + 9: i32 delete_rule, // what happens to foreign key when parent key is deleted + 10: string fk_name, // foreign key name + 11: string pk_name, // primary key name + 12: bool enable_cstr, // Enable/Disable + 13: bool validate_cstr, // Validate/No validate + 14: bool rely_cstr // Rely/No Rely +} + +struct SQLUniqueConstraint { + 1: string table_db, // table schema + 2: string table_name, // table name + 3: string column_name, // column name + 4: i32 key_seq, // sequence number within unique constraint + 5: string uk_name, // unique key name + 6: bool enable_cstr, // Enable/Disable + 7: bool validate_cstr, // Validate/No validate + 8: bool rely_cstr // Rely/No Rely +} + +struct SQLNotNullConstraint { + 1: string table_db, // table schema + 2: string table_name, // table name + 3: string column_name, // column name + 4: string nn_name, // not null name + 5: bool enable_cstr, // Enable/Disable + 6: bool validate_cstr, // Validate/No validate + 7: bool rely_cstr // Rely/No Rely +} + +struct Type { + 1: string name, // one of the types in PrimitiveTypes or CollectionTypes or User defined types + 2: optional string type1, // object type if the name is 'list' (LIST_TYPE), key type if the name is 'map' (MAP_TYPE) + 3: optional string type2, // val type if the name is 'map' (MAP_TYPE) + 4: optional list fields // if the name is one of the user defined types +} + +enum HiveObjectType { + GLOBAL = 1, + DATABASE = 2, + TABLE = 3, + PARTITION = 4, + COLUMN = 5, +} + +enum PrincipalType { + USER = 1, + ROLE = 2, + GROUP = 3, +} + +const string HIVE_FILTER_FIELD_OWNER = "hive_filter_field_owner__" +const string HIVE_FILTER_FIELD_PARAMS = "hive_filter_field_params__" +const string HIVE_FILTER_FIELD_LAST_ACCESS = "hive_filter_field_last_access__" + +enum PartitionEventType { + LOAD_DONE = 1, +} + +// Enums for transaction and lock management +enum TxnState { + COMMITTED = 1, + ABORTED = 2, + OPEN = 3, +} + +enum LockLevel { + DB = 1, + TABLE = 2, + PARTITION = 3, +} + +enum LockState { + ACQUIRED = 1, // requester has the lock + WAITING = 2, // requester is waiting for the lock and should call checklock at a later point to see if the lock has been obtained. + ABORT = 3, // the lock has been aborted, most likely due to timeout + NOT_ACQUIRED = 4, // returned only with lockNoWait, indicates the lock was not available and was not acquired +} + +enum LockType { + SHARED_READ = 1, + SHARED_WRITE = 2, + EXCLUSIVE = 3, +} + +enum CompactionType { + MINOR = 1, + MAJOR = 2, +} + +enum GrantRevokeType { + GRANT = 1, + REVOKE = 2, +} + +enum DataOperationType { + SELECT = 1, + INSERT = 2 + UPDATE = 3, + DELETE = 4, + UNSET = 5,//this is the default to distinguish from NULL from old clients + NO_TXN = 6,//drop table, insert overwrite, etc - something non-transactional +} + +// Types of events the client can request that the metastore fire. For now just support DML operations, as the metastore knows +// about DDL operations and there's no reason for the client to request such an event. +enum EventRequestType { + INSERT = 1, + UPDATE = 2, + DELETE = 3, +} + +struct HiveObjectRef{ + 1: HiveObjectType objectType, + 2: string dbName, + 3: string objectName, + 4: list partValues, + 5: string columnName, +} + +struct PrivilegeGrantInfo { + 1: string privilege, + 2: i32 createTime, + 3: string grantor, + 4: PrincipalType grantorType, + 5: bool grantOption, +} + +struct HiveObjectPrivilege { + 1: HiveObjectRef hiveObject, + 2: string principalName, + 3: PrincipalType principalType, + 4: PrivilegeGrantInfo grantInfo, +} + +struct PrivilegeBag { + 1: list privileges, +} + +struct PrincipalPrivilegeSet { + 1: map> userPrivileges, // user name -> privilege grant info + 2: map> groupPrivileges, // group name -> privilege grant info + 3: map> rolePrivileges, //role name -> privilege grant info +} + +struct GrantRevokePrivilegeRequest { + 1: GrantRevokeType requestType; + 2: PrivilegeBag privileges; + 3: optional bool revokeGrantOption; // Only for revoke request +} + +struct GrantRevokePrivilegeResponse { + 1: optional bool success; +} + +struct Role { + 1: string roleName, + 2: i32 createTime, + 3: string ownerName, +} + +// Representation of a grant for a principal to a role +struct RolePrincipalGrant { + 1: string roleName, + 2: string principalName, + 3: PrincipalType principalType, + 4: bool grantOption, + 5: i32 grantTime, + 6: string grantorName, + 7: PrincipalType grantorPrincipalType +} + +struct GetRoleGrantsForPrincipalRequest { + 1: required string principal_name, + 2: required PrincipalType principal_type +} + +struct GetRoleGrantsForPrincipalResponse { + 1: required list principalGrants; +} + +struct GetPrincipalsInRoleRequest { + 1: required string roleName; +} + +struct GetPrincipalsInRoleResponse { + 1: required list principalGrants; +} + +struct GrantRevokeRoleRequest { + 1: GrantRevokeType requestType; + 2: string roleName; + 3: string principalName; + 4: PrincipalType principalType; + 5: optional string grantor; // Needed for grant + 6: optional PrincipalType grantorType; // Needed for grant + 7: optional bool grantOption; +} + +struct GrantRevokeRoleResponse { + 1: optional bool success; +} + +// namespace for tables +struct Database { + 1: string name, + 2: string description, + 3: string locationUri, + 4: map parameters, // properties associated with the database + 5: optional PrincipalPrivilegeSet privileges, + 6: optional string ownerName, + 7: optional PrincipalType ownerType +} + +// This object holds the information needed by SerDes +struct SerDeInfo { + 1: string name, // name of the serde, table name by default + 2: string serializationLib, // usually the class that implements the extractor & loader + 3: map parameters // initialization parameters +} + +// sort order of a column (column name along with asc(1)/desc(0)) +struct Order { + 1: string col, // sort column name + 2: i32 order // asc(1) or desc(0) +} + +// this object holds all the information about skewed table +struct SkewedInfo { + 1: list skewedColNames, // skewed column names + 2: list> skewedColValues, //skewed values + 3: map skewedColValueLocationMaps, //skewed value to location mappings +} + +// this object holds all the information about physical storage of the data belonging to a table +struct StorageDescriptor { + 1: list cols, // required (refer to types defined above) + 2: string location, // defaults to //tablename + 3: string inputFormat, // SequenceFileInputFormat (binary) or TextInputFormat` or custom format + 4: string outputFormat, // SequenceFileOutputFormat (binary) or IgnoreKeyTextOutputFormat or custom format + 5: bool compressed, // compressed or not + 6: i32 numBuckets, // this must be specified if there are any dimension columns + 7: SerDeInfo serdeInfo, // serialization and deserialization information + 8: list bucketCols, // reducer grouping columns and clustering columns and bucketing columns` + 9: list sortCols, // sort order of the data in each bucket + 10: map parameters, // any user supplied key value hash + 11: optional SkewedInfo skewedInfo, // skewed information + 12: optional bool storedAsSubDirectories // stored as subdirectories or not +} + +// table information +struct Table { + 1: string tableName, // name of the table + 2: string dbName, // database name ('default') + 3: string owner, // owner of this table + 4: i32 createTime, // creation time of the table + 5: i32 lastAccessTime, // last access time (usually this will be filled from HDFS and shouldn't be relied on) + 6: i32 retention, // retention time + 7: StorageDescriptor sd, // storage descriptor of the table + 8: list partitionKeys, // partition keys of the table. only primitive types are supported + 9: map parameters, // to store comments or any other user level parameters + 10: string viewOriginalText, // original view text, null for non-view + 11: string viewExpandedText, // expanded view text, null for non-view + 12: string tableType, // table type enum, e.g. EXTERNAL_TABLE + 13: optional PrincipalPrivilegeSet privileges, + 14: optional bool temporary=false, + 15: optional bool rewriteEnabled, // rewrite enabled or not +} + +struct Partition { + 1: list values // string value is converted to appropriate partition key type + 2: string dbName, + 3: string tableName, + 4: i32 createTime, + 5: i32 lastAccessTime, + 6: StorageDescriptor sd, + 7: map parameters, + 8: optional PrincipalPrivilegeSet privileges +} + +struct PartitionWithoutSD { + 1: list values // string value is converted to appropriate partition key type + 2: i32 createTime, + 3: i32 lastAccessTime, + 4: string relativePath, + 5: map parameters, + 6: optional PrincipalPrivilegeSet privileges +} + +struct PartitionSpecWithSharedSD { + 1: list partitions, + 2: StorageDescriptor sd, +} + +struct PartitionListComposingSpec { + 1: list partitions +} + +struct PartitionSpec { + 1: string dbName, + 2: string tableName, + 3: string rootPath, + 4: optional PartitionSpecWithSharedSD sharedSDPartitionSpec, + 5: optional PartitionListComposingSpec partitionList +} + +struct Index { + 1: string indexName, // unique with in the whole database namespace + 2: string indexHandlerClass, // reserved + 3: string dbName, + 4: string origTableName, + 5: i32 createTime, + 6: i32 lastAccessTime, + 7: string indexTableName, + 8: StorageDescriptor sd, + 9: map parameters, + 10: bool deferredRebuild +} + +// column statistics +struct BooleanColumnStatsData { +1: required i64 numTrues, +2: required i64 numFalses, +3: required i64 numNulls, +4: optional binary bitVectors +} + +struct DoubleColumnStatsData { +1: optional double lowValue, +2: optional double highValue, +3: required i64 numNulls, +4: required i64 numDVs, +5: optional binary bitVectors +} + +struct LongColumnStatsData { +1: optional i64 lowValue, +2: optional i64 highValue, +3: required i64 numNulls, +4: required i64 numDVs, +5: optional binary bitVectors +} + +struct StringColumnStatsData { +1: required i64 maxColLen, +2: required double avgColLen, +3: required i64 numNulls, +4: required i64 numDVs, +5: optional binary bitVectors +} + +struct BinaryColumnStatsData { +1: required i64 maxColLen, +2: required double avgColLen, +3: required i64 numNulls, +4: optional binary bitVectors +} + + +struct Decimal { +1: required binary unscaled, +3: required i16 scale +} + +struct DecimalColumnStatsData { +1: optional Decimal lowValue, +2: optional Decimal highValue, +3: required i64 numNulls, +4: required i64 numDVs, +5: optional binary bitVectors +} + +struct Date { +1: required i64 daysSinceEpoch +} + +struct DateColumnStatsData { +1: optional Date lowValue, +2: optional Date highValue, +3: required i64 numNulls, +4: required i64 numDVs, +5: optional binary bitVectors +} + +union ColumnStatisticsData { +1: BooleanColumnStatsData booleanStats, +2: LongColumnStatsData longStats, +3: DoubleColumnStatsData doubleStats, +4: StringColumnStatsData stringStats, +5: BinaryColumnStatsData binaryStats, +6: DecimalColumnStatsData decimalStats, +7: DateColumnStatsData dateStats +} + +struct ColumnStatisticsObj { +1: required string colName, +2: required string colType, +3: required ColumnStatisticsData statsData +} + +struct ColumnStatisticsDesc { +1: required bool isTblLevel, +2: required string dbName, +3: required string tableName, +4: optional string partName, +5: optional i64 lastAnalyzed +} + +struct ColumnStatistics { +1: required ColumnStatisticsDesc statsDesc, +2: required list statsObj; +} + +struct AggrStats { +1: required list colStats, +2: required i64 partsFound // number of partitions for which stats were found +} + +struct SetPartitionsStatsRequest { +1: required list colStats, +2: optional bool needMerge //stats need to be merged with the existing stats +} + +// schema of the table/query results etc. +struct Schema { + // column names, types, comments + 1: list fieldSchemas, // delimiters etc + 2: map properties +} + +// Key-value store to be used with selected +// Metastore APIs (create, alter methods). +// The client can pass environment properties / configs that can be +// accessed in hooks. +struct EnvironmentContext { + 1: map properties +} + +struct PrimaryKeysRequest { + 1: required string db_name, + 2: required string tbl_name +} + +struct PrimaryKeysResponse { + 1: required list primaryKeys +} + +struct ForeignKeysRequest { + 1: string parent_db_name, + 2: string parent_tbl_name, + 3: string foreign_db_name, + 4: string foreign_tbl_name +} + +struct ForeignKeysResponse { + 1: required list foreignKeys +} + +struct UniqueConstraintsRequest { + 1: required string db_name, + 2: required string tbl_name +} + +struct UniqueConstraintsResponse { + 1: required list uniqueConstraints +} + +struct NotNullConstraintsRequest { + 1: required string db_name, + 2: required string tbl_name +} + +struct NotNullConstraintsResponse { + 1: required list notNullConstraints +} + +struct DropConstraintRequest { + 1: required string dbname, + 2: required string tablename, + 3: required string constraintname +} + +struct AddPrimaryKeyRequest { + 1: required list primaryKeyCols +} + +struct AddForeignKeyRequest { + 1: required list foreignKeyCols +} + +struct AddUniqueConstraintRequest { + 1: required list uniqueConstraintCols +} + +struct AddNotNullConstraintRequest { + 1: required list notNullConstraintCols +} + +// Return type for get_partitions_by_expr +struct PartitionsByExprResult { + 1: required list partitions, + // Whether the results has any (currently, all) partitions which may or may not match + 2: required bool hasUnknownPartitions +} + +struct PartitionsByExprRequest { + 1: required string dbName, + 2: required string tblName, + 3: required binary expr, + 4: optional string defaultPartitionName, + 5: optional i16 maxParts=-1 +} + +struct TableStatsResult { + 1: required list tableStats +} + +struct PartitionsStatsResult { + 1: required map> partStats +} + +struct TableStatsRequest { + 1: required string dbName, + 2: required string tblName, + 3: required list colNames +} + +struct PartitionsStatsRequest { + 1: required string dbName, + 2: required string tblName, + 3: required list colNames, + 4: required list partNames +} + +// Return type for add_partitions_req +struct AddPartitionsResult { + 1: optional list partitions, +} + +// Request type for add_partitions_req +struct AddPartitionsRequest { + 1: required string dbName, + 2: required string tblName, + 3: required list parts, + 4: required bool ifNotExists, + 5: optional bool needResult=true +} + +// Return type for drop_partitions_req +struct DropPartitionsResult { + 1: optional list partitions, +} + +struct DropPartitionsExpr { + 1: required binary expr; + 2: optional i32 partArchiveLevel; +} + +union RequestPartsSpec { + 1: list names; + 2: list exprs; +} + +// Request type for drop_partitions_req +// TODO: we might want to add "bestEffort" flag; where a subset can fail +struct DropPartitionsRequest { + 1: required string dbName, + 2: required string tblName, + 3: required RequestPartsSpec parts, + 4: optional bool deleteData, + 5: optional bool ifExists=true, // currently verified on client + 6: optional bool ignoreProtection, + 7: optional EnvironmentContext environmentContext, + 8: optional bool needResult=true +} + +struct PartitionValuesRequest { + 1: required string dbName, + 2: required string tblName, + 3: required list partitionKeys; + 4: optional bool applyDistinct = true; + 5: optional string filter; + 6: optional list partitionOrder; + 7: optional bool ascending = true; + 8: optional i64 maxParts = -1; +} + +struct PartitionValuesRow { + 1: required list row; +} + +struct PartitionValuesResponse { + 1: required list partitionValues; +} + +enum FunctionType { + JAVA = 1, +} + +enum ResourceType { + JAR = 1, + FILE = 2, + ARCHIVE = 3, +} + +struct ResourceUri { + 1: ResourceType resourceType, + 2: string uri, +} + +// User-defined function +struct Function { + 1: string functionName, + 2: string dbName, + 3: string className, + 4: string ownerName, + 5: PrincipalType ownerType, + 6: i32 createTime, + 7: FunctionType functionType, + 8: list resourceUris, +} + +// Structs for transaction and locks +struct TxnInfo { + 1: required i64 id, + 2: required TxnState state, + 3: required string user, // used in 'show transactions' to help admins find who has open transactions + 4: required string hostname, // used in 'show transactions' to help admins find who has open transactions + 5: optional string agentInfo = "Unknown", + 6: optional i32 heartbeatCount=0, + 7: optional string metaInfo, + 8: optional i64 startedTime, + 9: optional i64 lastHeartbeatTime, +} + +struct GetOpenTxnsInfoResponse { + 1: required i64 txn_high_water_mark, + 2: required list open_txns, +} + +struct GetOpenTxnsResponse { + 1: required i64 txn_high_water_mark, + 2: required list open_txns, // set changed to list since 3.0 + 3: optional i64 min_open_txn, //since 1.3,2.2 + 4: required binary abortedBits, // since 3.0 +} + +struct OpenTxnRequest { + 1: required i32 num_txns, + 2: required string user, + 3: required string hostname, + 4: optional string agentInfo = "Unknown", +} + +struct OpenTxnsResponse { + 1: required list txn_ids, +} + +struct AbortTxnRequest { + 1: required i64 txnid, +} + +struct AbortTxnsRequest { + 1: required list txn_ids, +} + +struct CommitTxnRequest { + 1: required i64 txnid, +} + +struct LockComponent { + 1: required LockType type, + 2: required LockLevel level, + 3: required string dbname, + 4: optional string tablename, + 5: optional string partitionname, + 6: optional DataOperationType operationType = DataOperationType.UNSET, + 7: optional bool isAcid = false, + 8: optional bool isDynamicPartitionWrite = false +} + +struct LockRequest { + 1: required list component, + 2: optional i64 txnid, + 3: required string user, // used in 'show locks' to help admins find who has open locks + 4: required string hostname, // used in 'show locks' to help admins find who has open locks + 5: optional string agentInfo = "Unknown", +} + +struct LockResponse { + 1: required i64 lockid, + 2: required LockState state, +} + +struct CheckLockRequest { + 1: required i64 lockid, + 2: optional i64 txnid, + 3: optional i64 elapsed_ms, +} + +struct UnlockRequest { + 1: required i64 lockid, +} + +struct ShowLocksRequest { + 1: optional string dbname, + 2: optional string tablename, + 3: optional string partname, + 4: optional bool isExtended=false, +} + +struct ShowLocksResponseElement { + 1: required i64 lockid, + 2: required string dbname, + 3: optional string tablename, + 4: optional string partname, + 5: required LockState state, + 6: required LockType type, + 7: optional i64 txnid, + 8: required i64 lastheartbeat, + 9: optional i64 acquiredat, + 10: required string user, + 11: required string hostname, + 12: optional i32 heartbeatCount = 0, + 13: optional string agentInfo, + 14: optional i64 blockedByExtId, + 15: optional i64 blockedByIntId, + 16: optional i64 lockIdInternal, +} + +struct ShowLocksResponse { + 1: list locks, +} + +struct HeartbeatRequest { + 1: optional i64 lockid, + 2: optional i64 txnid +} + +struct HeartbeatTxnRangeRequest { + 1: required i64 min, + 2: required i64 max +} + +struct HeartbeatTxnRangeResponse { + 1: required set aborted, + 2: required set nosuch +} + +struct CompactionRequest { + 1: required string dbname, + 2: required string tablename, + 3: optional string partitionname, + 4: required CompactionType type, + 5: optional string runas, + 6: optional map properties +} + +struct CompactionResponse { + 1: required i64 id, + 2: required string state, + 3: required bool accepted +} + +struct ShowCompactRequest { +} + +struct ShowCompactResponseElement { + 1: required string dbname, + 2: required string tablename, + 3: optional string partitionname, + 4: required CompactionType type, + 5: required string state, + 6: optional string workerid, + 7: optional i64 start, + 8: optional string runAs, + 9: optional i64 hightestTxnId, // Highest Txn ID handled by this compaction + 10: optional string metaInfo, + 11: optional i64 endTime, + 12: optional string hadoopJobId = "None", + 13: optional i64 id, +} + +struct ShowCompactResponse { + 1: required list compacts, +} + +struct AddDynamicPartitions { + 1: required i64 txnid, + 2: required string dbname, + 3: required string tablename, + 4: required list partitionnames, + 5: optional DataOperationType operationType = DataOperationType.UNSET +} + +struct NotificationEventRequest { + 1: required i64 lastEvent, + 2: optional i32 maxEvents, +} + +struct NotificationEvent { + 1: required i64 eventId, + 2: required i32 eventTime, + 3: required string eventType, + 4: optional string dbName, + 5: optional string tableName, + 6: required string message, + 7: optional string messageFormat, +} + +struct NotificationEventResponse { + 1: required list events, +} + +struct CurrentNotificationEventId { + 1: required i64 eventId, +} + +struct NotificationEventsCountRequest { + 1: required i64 fromEventId, + 2: required string dbName, +} + +struct NotificationEventsCountResponse { + 1: required i64 eventsCount, +} + +struct InsertEventRequestData { + 1: optional bool replace, + 2: required list filesAdded, + // Checksum of files (hex string of checksum byte payload) + 3: optional list filesAddedChecksum, +} + +union FireEventRequestData { + 1: InsertEventRequestData insertData +} + +struct FireEventRequest { + 1: required bool successful, + 2: required FireEventRequestData data + // dbname, tablename, and partition vals are included as optional in the top level event rather than placed in each type of + // subevent as I assume they'll be used across most event types. + 3: optional string dbName, + 4: optional string tableName, + 5: optional list partitionVals, +} + +struct FireEventResponse { + // NOP for now, this is just a place holder for future responses +} + +struct MetadataPpdResult { + 1: optional binary metadata, + 2: optional binary includeBitset +} + +// Return type for get_file_metadata_by_expr +struct GetFileMetadataByExprResult { + 1: required map metadata, + 2: required bool isSupported +} + +enum FileMetadataExprType { + ORC_SARG = 1 +} + + +// Request type for get_file_metadata_by_expr +struct GetFileMetadataByExprRequest { + 1: required list fileIds, + 2: required binary expr, + 3: optional bool doGetFooters, + 4: optional FileMetadataExprType type +} + +// Return type for get_file_metadata +struct GetFileMetadataResult { + 1: required map metadata, + 2: required bool isSupported +} + +// Request type for get_file_metadata +struct GetFileMetadataRequest { + 1: required list fileIds +} + +// Return type for put_file_metadata +struct PutFileMetadataResult { +} + +// Request type for put_file_metadata +struct PutFileMetadataRequest { + 1: required list fileIds, + 2: required list metadata, + 3: optional FileMetadataExprType type +} + +// Return type for clear_file_metadata +struct ClearFileMetadataResult { +} + +// Request type for clear_file_metadata +struct ClearFileMetadataRequest { + 1: required list fileIds +} + +// Return type for cache_file_metadata +struct CacheFileMetadataResult { + 1: required bool isSupported +} + +// Request type for cache_file_metadata +struct CacheFileMetadataRequest { + 1: required string dbName, + 2: required string tblName, + 3: optional string partName, + 4: optional bool isAllParts +} + +struct GetAllFunctionsResponse { + 1: optional list functions +} + +enum ClientCapability { + TEST_CAPABILITY = 1, + INSERT_ONLY_TABLES = 2 +} + + +struct ClientCapabilities { + 1: required list values +} + +struct GetTableRequest { + 1: required string dbName, + 2: required string tblName, + 3: optional ClientCapabilities capabilities +} + +struct GetTableResult { + 1: required Table table +} + +struct GetTablesRequest { + 1: required string dbName, + 2: optional list tblNames, + 3: optional ClientCapabilities capabilities +} + +struct GetTablesResult { + 1: required list tables +} + +// Request type for cm_recycle +struct CmRecycleRequest { + 1: required string dataPath, + 2: required bool purge +} + +// Response type for cm_recycle +struct CmRecycleResponse { +} + +struct TableMeta { + 1: required string dbName; + 2: required string tableName; + 3: required string tableType; + 4: optional string comments; +} + +// Data types for workload management. + +enum WMResourcePlanStatus { + ACTIVE = 1, + ENABLED = 2, + DISABLED = 3 +} + +struct WMResourcePlan { + 1: required string name; + 2: optional WMResourcePlanStatus status; + 3: optional i32 queryParallelism; + 4: optional string defaultPoolPath; +} + +struct WMPool { + 1: required string resourcePlanName; + 2: required string poolPath; + 3: optional double allocFraction; + 4: optional i32 queryParallelism; + 5: optional string schedulingPolicy; +} + +struct WMTrigger { + 1: required string resourcePlanName; + 2: required string triggerName; + 3: optional string triggerExpression; + 4: optional string actionExpression; +} + +struct WMMapping { + 1: required string resourcePlanName; + 2: required string entityType; + 3: required string entityName; + 4: optional string poolName; + 5: optional i32 ordering; +} + +struct WMPoolTrigger { + 1: required string pool; + 2: required string trigger; +} + +struct WMFullResourcePlan { + 1: required WMResourcePlan plan; + 2: required list pools; + 3: optional list mappings; + 4: optional list triggers; + 5: optional list poolTriggers; +} + +// Request response for workload management API's. + +struct WMCreateResourcePlanRequest { + 1: optional WMResourcePlan resourcePlan; +} + +struct WMCreateResourcePlanResponse { +} + +struct WMGetActiveResourcePlanRequest { +} + +struct WMGetActiveResourcePlanResponse { + 1: optional WMFullResourcePlan resourcePlan; +} + +struct WMGetResourcePlanRequest { + 1: optional string resourcePlanName; +} + +struct WMGetResourcePlanResponse { + 1: optional WMResourcePlan resourcePlan; +} + +struct WMGetAllResourcePlanRequest { +} + +struct WMGetAllResourcePlanResponse { + 1: optional list resourcePlans; +} + +struct WMAlterResourcePlanRequest { + 1: optional string resourcePlanName; + 2: optional WMResourcePlan resourcePlan; + 3: optional bool isEnableAndActivate; +} + +struct WMAlterResourcePlanResponse { + 1: optional WMFullResourcePlan fullResourcePlan; +} + +struct WMValidateResourcePlanRequest { + 1: optional string resourcePlanName; +} + +struct WMValidateResourcePlanResponse { + 1: optional bool isValid; +} + +struct WMDropResourcePlanRequest { + 1: optional string resourcePlanName; +} + +struct WMDropResourcePlanResponse { +} + +struct WMCreateTriggerRequest { + 1: optional WMTrigger trigger; +} + +struct WMCreateTriggerResponse { +} + +struct WMAlterTriggerRequest { + 1: optional WMTrigger trigger; +} + +struct WMAlterTriggerResponse { +} + +struct WMDropTriggerRequest { + 1: optional string resourcePlanName; + 2: optional string triggerName; +} + +struct WMDropTriggerResponse { +} + +struct WMGetTriggersForResourePlanRequest { + 1: optional string resourcePlanName; +} + +struct WMGetTriggersForResourePlanResponse { + 1: optional list triggers; +} + + +// Exceptions. + +exception MetaException { + 1: string message +} + +exception UnknownTableException { + 1: string message +} + +exception UnknownDBException { + 1: string message +} + +exception AlreadyExistsException { + 1: string message +} + +exception InvalidPartitionException { + 1: string message +} + +exception UnknownPartitionException { + 1: string message +} + +exception InvalidObjectException { + 1: string message +} + +exception NoSuchObjectException { + 1: string message +} + +exception IndexAlreadyExistsException { + 1: string message +} + +exception InvalidOperationException { + 1: string message +} + +exception ConfigValSecurityException { + 1: string message +} + +exception InvalidInputException { + 1: string message +} + +// Transaction and lock exceptions +exception NoSuchTxnException { + 1: string message +} + +exception TxnAbortedException { + 1: string message +} + +exception TxnOpenException { + 1: string message +} + +exception NoSuchLockException { + 1: string message +} + +/** +* This interface is live. +*/ +service ThriftHiveMetastore +{ + string getMetaConf(1:string key) throws(1:MetaException o1) + void setMetaConf(1:string key, 2:string value) throws(1:MetaException o1) + + void create_database(1:Database database) throws(1:AlreadyExistsException o1, 2:InvalidObjectException o2, 3:MetaException o3) + Database get_database(1:string name) throws(1:NoSuchObjectException o1, 2:MetaException o2) + void drop_database(1:string name, 2:bool deleteData, 3:bool cascade) throws(1:NoSuchObjectException o1, 2:InvalidOperationException o2, 3:MetaException o3) + list get_databases(1:string pattern) throws(1:MetaException o1) + list get_all_databases() throws(1:MetaException o1) + void alter_database(1:string dbname, 2:Database db) throws(1:MetaException o1, 2:NoSuchObjectException o2) + + // returns the type with given name (make seperate calls for the dependent types if needed) + Type get_type(1:string name) throws(1:MetaException o1, 2:NoSuchObjectException o2) + bool create_type(1:Type type) throws(1:AlreadyExistsException o1, 2:InvalidObjectException o2, 3:MetaException o3) + bool drop_type(1:string type) throws(1:MetaException o1, 2:NoSuchObjectException o2) + map get_type_all(1:string name) + throws(1:MetaException o2) + + // Gets a list of FieldSchemas describing the columns of a particular table + list get_fields(1: string db_name, 2: string table_name) throws (1: MetaException o1, 2: UnknownTableException o2, 3: UnknownDBException o3), + list get_fields_with_environment_context(1: string db_name, 2: string table_name, 3:EnvironmentContext environment_context) throws (1: MetaException o1, 2: UnknownTableException o2, 3: UnknownDBException o3) + + // Gets a list of FieldSchemas describing both the columns and the partition keys of a particular table + list get_schema(1: string db_name, 2: string table_name) throws (1: MetaException o1, 2: UnknownTableException o2, 3: UnknownDBException o3) + list get_schema_with_environment_context(1: string db_name, 2: string table_name, 3:EnvironmentContext environment_context) throws (1: MetaException o1, 2: UnknownTableException o2, 3: UnknownDBException o3) + + // create a Hive table. Following fields must be set + // tableName + // database (only 'default' for now until Hive QL supports databases) + // owner (not needed, but good to have for tracking purposes) + // sd.cols (list of field schemas) + // sd.inputFormat (SequenceFileInputFormat (binary like falcon tables or u_full) or TextInputFormat) + // sd.outputFormat (SequenceFileInputFormat (binary) or TextInputFormat) + // sd.serdeInfo.serializationLib (SerDe class name eg org.apache.hadoop.hive.serde.simple_meta.MetadataTypedColumnsetSerDe + // * See notes on DDL_TIME + void create_table(1:Table tbl) throws(1:AlreadyExistsException o1, 2:InvalidObjectException o2, 3:MetaException o3, 4:NoSuchObjectException o4) + void create_table_with_environment_context(1:Table tbl, + 2:EnvironmentContext environment_context) + throws (1:AlreadyExistsException o1, + 2:InvalidObjectException o2, 3:MetaException o3, + 4:NoSuchObjectException o4) + void create_table_with_constraints(1:Table tbl, 2: list primaryKeys, 3: list foreignKeys, + 4: list uniqueConstraints, 5: list notNullConstraints) + throws (1:AlreadyExistsException o1, + 2:InvalidObjectException o2, 3:MetaException o3, + 4:NoSuchObjectException o4) + void drop_constraint(1:DropConstraintRequest req) + throws(1:NoSuchObjectException o1, 2:MetaException o3) + void add_primary_key(1:AddPrimaryKeyRequest req) + throws(1:NoSuchObjectException o1, 2:MetaException o2) + void add_foreign_key(1:AddForeignKeyRequest req) + throws(1:NoSuchObjectException o1, 2:MetaException o2) + void add_unique_constraint(1:AddUniqueConstraintRequest req) + throws(1:NoSuchObjectException o1, 2:MetaException o2) + void add_not_null_constraint(1:AddNotNullConstraintRequest req) + throws(1:NoSuchObjectException o1, 2:MetaException o2) + + // drops the table and all the partitions associated with it if the table has partitions + // delete data (including partitions) if deleteData is set to true + void drop_table(1:string dbname, 2:string name, 3:bool deleteData) + throws(1:NoSuchObjectException o1, 2:MetaException o3) + void drop_table_with_environment_context(1:string dbname, 2:string name, 3:bool deleteData, + 4:EnvironmentContext environment_context) + throws(1:NoSuchObjectException o1, 2:MetaException o3) + void truncate_table(1:string dbName, 2:string tableName, 3:list partNames) + throws(1:MetaException o1) + list get_tables(1: string db_name, 2: string pattern) throws (1: MetaException o1) + list get_tables_by_type(1: string db_name, 2: string pattern, 3: string tableType) throws (1: MetaException o1) + list get_table_meta(1: string db_patterns, 2: string tbl_patterns, 3: list tbl_types) + throws (1: MetaException o1) + list get_all_tables(1: string db_name) throws (1: MetaException o1) + + Table get_table(1:string dbname, 2:string tbl_name) + throws (1:MetaException o1, 2:NoSuchObjectException o2) + list
get_table_objects_by_name(1:string dbname, 2:list tbl_names) + GetTableResult get_table_req(1:GetTableRequest req) throws (1:MetaException o1, 2:NoSuchObjectException o2) + GetTablesResult get_table_objects_by_name_req(1:GetTablesRequest req) + throws (1:MetaException o1, 2:InvalidOperationException o2, 3:UnknownDBException o3) + + // Get a list of table names that match a filter. + // The filter operators are LIKE, <, <=, >, >=, =, <> + // + // In the filter statement, values interpreted as strings must be enclosed in quotes, + // while values interpreted as integers should not be. Strings and integers are the only + // supported value types. + // + // The currently supported key names in the filter are: + // Constants.HIVE_FILTER_FIELD_OWNER, which filters on the tables' owner's name + // and supports all filter operators + // Constants.HIVE_FILTER_FIELD_LAST_ACCESS, which filters on the last access times + // and supports all filter operators except LIKE + // Constants.HIVE_FILTER_FIELD_PARAMS, which filters on the tables' parameter keys and values + // and only supports the filter operators = and <>. + // Append the parameter key name to HIVE_FILTER_FIELD_PARAMS in the filter statement. + // For example, to filter on parameter keys called "retention", the key name in the filter + // statement should be Constants.HIVE_FILTER_FIELD_PARAMS + "retention" + // Also, = and <> only work for keys that exist + // in the tables. E.g., if you are looking for tables where key1 <> value, it will only + // look at tables that have a value for the parameter key1. + // Some example filter statements include: + // filter = Constants.HIVE_FILTER_FIELD_OWNER + " like \".*test.*\" and " + + // Constants.HIVE_FILTER_FIELD_LAST_ACCESS + " = 0"; + // filter = Constants.HIVE_FILTER_FIELD_PARAMS + "retention = \"30\" or " + + // Constants.HIVE_FILTER_FIELD_PARAMS + "retention = \"90\"" + // @param dbName + // The name of the database from which you will retrieve the table names + // @param filterType + // The type of filter + // @param filter + // The filter string + // @param max_tables + // The maximum number of tables returned + // @return A list of table names that match the desired filter + list get_table_names_by_filter(1:string dbname, 2:string filter, 3:i16 max_tables=-1) + throws (1:MetaException o1, 2:InvalidOperationException o2, 3:UnknownDBException o3) + + // alter table applies to only future partitions not for existing partitions + // * See notes on DDL_TIME + void alter_table(1:string dbname, 2:string tbl_name, 3:Table new_tbl) + throws (1:InvalidOperationException o1, 2:MetaException o2) + void alter_table_with_environment_context(1:string dbname, 2:string tbl_name, + 3:Table new_tbl, 4:EnvironmentContext environment_context) + throws (1:InvalidOperationException o1, 2:MetaException o2) + // alter table not only applies to future partitions but also cascade to existing partitions + void alter_table_with_cascade(1:string dbname, 2:string tbl_name, 3:Table new_tbl, 4:bool cascade) + throws (1:InvalidOperationException o1, 2:MetaException o2) + // the following applies to only tables that have partitions + // * See notes on DDL_TIME + Partition add_partition(1:Partition new_part) + throws(1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) + Partition add_partition_with_environment_context(1:Partition new_part, + 2:EnvironmentContext environment_context) + throws (1:InvalidObjectException o1, 2:AlreadyExistsException o2, + 3:MetaException o3) + i32 add_partitions(1:list new_parts) + throws(1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) + i32 add_partitions_pspec(1:list new_parts) + throws(1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) + Partition append_partition(1:string db_name, 2:string tbl_name, 3:list part_vals) + throws (1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) + AddPartitionsResult add_partitions_req(1:AddPartitionsRequest request) + throws(1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) + Partition append_partition_with_environment_context(1:string db_name, 2:string tbl_name, + 3:list part_vals, 4:EnvironmentContext environment_context) + throws (1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) + Partition append_partition_by_name(1:string db_name, 2:string tbl_name, 3:string part_name) + throws (1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) + Partition append_partition_by_name_with_environment_context(1:string db_name, 2:string tbl_name, + 3:string part_name, 4:EnvironmentContext environment_context) + throws (1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) + bool drop_partition(1:string db_name, 2:string tbl_name, 3:list part_vals, 4:bool deleteData) + throws(1:NoSuchObjectException o1, 2:MetaException o2) + bool drop_partition_with_environment_context(1:string db_name, 2:string tbl_name, + 3:list part_vals, 4:bool deleteData, 5:EnvironmentContext environment_context) + throws(1:NoSuchObjectException o1, 2:MetaException o2) + bool drop_partition_by_name(1:string db_name, 2:string tbl_name, 3:string part_name, 4:bool deleteData) + throws(1:NoSuchObjectException o1, 2:MetaException o2) + bool drop_partition_by_name_with_environment_context(1:string db_name, 2:string tbl_name, + 3:string part_name, 4:bool deleteData, 5:EnvironmentContext environment_context) + throws(1:NoSuchObjectException o1, 2:MetaException o2) + DropPartitionsResult drop_partitions_req(1: DropPartitionsRequest req) + throws(1:NoSuchObjectException o1, 2:MetaException o2) + + Partition get_partition(1:string db_name, 2:string tbl_name, 3:list part_vals) + throws(1:MetaException o1, 2:NoSuchObjectException o2) + Partition exchange_partition(1:map partitionSpecs, 2:string source_db, + 3:string source_table_name, 4:string dest_db, 5:string dest_table_name) + throws(1:MetaException o1, 2:NoSuchObjectException o2, 3:InvalidObjectException o3, + 4:InvalidInputException o4) + + list exchange_partitions(1:map partitionSpecs, 2:string source_db, + 3:string source_table_name, 4:string dest_db, 5:string dest_table_name) + throws(1:MetaException o1, 2:NoSuchObjectException o2, 3:InvalidObjectException o3, + 4:InvalidInputException o4) + + Partition get_partition_with_auth(1:string db_name, 2:string tbl_name, 3:list part_vals, + 4: string user_name, 5: list group_names) throws(1:MetaException o1, 2:NoSuchObjectException o2) + + Partition get_partition_by_name(1:string db_name 2:string tbl_name, 3:string part_name) + throws(1:MetaException o1, 2:NoSuchObjectException o2) + + // returns all the partitions for this table in reverse chronological order. + // If max parts is given then it will return only that many. + list get_partitions(1:string db_name, 2:string tbl_name, 3:i16 max_parts=-1) + throws(1:NoSuchObjectException o1, 2:MetaException o2) + list get_partitions_with_auth(1:string db_name, 2:string tbl_name, 3:i16 max_parts=-1, + 4: string user_name, 5: list group_names) throws(1:NoSuchObjectException o1, 2:MetaException o2) + + list get_partitions_pspec(1:string db_name, 2:string tbl_name, 3:i32 max_parts=-1) + throws(1:NoSuchObjectException o1, 2:MetaException o2) + + list get_partition_names(1:string db_name, 2:string tbl_name, 3:i16 max_parts=-1) + throws(1:NoSuchObjectException o1, 2:MetaException o2) + + PartitionValuesResponse get_partition_values(1:PartitionValuesRequest request) + throws(1:MetaException o1, 2:NoSuchObjectException o2); + + // get_partition*_ps methods allow filtering by a partial partition specification, + // as needed for dynamic partitions. The values that are not restricted should + // be empty strings. Nulls were considered (instead of "") but caused errors in + // generated Python code. The size of part_vals may be smaller than the + // number of partition columns - the unspecified values are considered the same + // as "". + list get_partitions_ps(1:string db_name 2:string tbl_name + 3:list part_vals, 4:i16 max_parts=-1) + throws(1:MetaException o1, 2:NoSuchObjectException o2) + list get_partitions_ps_with_auth(1:string db_name, 2:string tbl_name, 3:list part_vals, 4:i16 max_parts=-1, + 5: string user_name, 6: list group_names) throws(1:NoSuchObjectException o1, 2:MetaException o2) + + list get_partition_names_ps(1:string db_name, + 2:string tbl_name, 3:list part_vals, 4:i16 max_parts=-1) + throws(1:MetaException o1, 2:NoSuchObjectException o2) + + // get the partitions matching the given partition filter + list get_partitions_by_filter(1:string db_name 2:string tbl_name + 3:string filter, 4:i16 max_parts=-1) + throws(1:MetaException o1, 2:NoSuchObjectException o2) + + // List partitions as PartitionSpec instances. + list get_part_specs_by_filter(1:string db_name 2:string tbl_name + 3:string filter, 4:i32 max_parts=-1) + throws(1:MetaException o1, 2:NoSuchObjectException o2) + + // get the partitions matching the given partition filter + // unlike get_partitions_by_filter, takes serialized hive expression, and with that can work + // with any filter (get_partitions_by_filter only works if the filter can be pushed down to JDOQL. + PartitionsByExprResult get_partitions_by_expr(1:PartitionsByExprRequest req) + throws(1:MetaException o1, 2:NoSuchObjectException o2) + + // get the partitions matching the given partition filter + i32 get_num_partitions_by_filter(1:string db_name 2:string tbl_name 3:string filter) + throws(1:MetaException o1, 2:NoSuchObjectException o2) + + // get partitions give a list of partition names + list get_partitions_by_names(1:string db_name 2:string tbl_name 3:list names) + throws(1:MetaException o1, 2:NoSuchObjectException o2) + + // changes the partition to the new partition object. partition is identified from the part values + // in the new_part + // * See notes on DDL_TIME + void alter_partition(1:string db_name, 2:string tbl_name, 3:Partition new_part) + throws (1:InvalidOperationException o1, 2:MetaException o2) + + // change a list of partitions. All partitions are altered atomically and all + // prehooks are fired together followed by all post hooks + void alter_partitions(1:string db_name, 2:string tbl_name, 3:list new_parts) + throws (1:InvalidOperationException o1, 2:MetaException o2) + void alter_partitions_with_environment_context(1:string db_name, 2:string tbl_name, 3:list new_parts, 4:EnvironmentContext environment_context) throws (1:InvalidOperationException o1, 2:MetaException o2) + + void alter_partition_with_environment_context(1:string db_name, + 2:string tbl_name, 3:Partition new_part, + 4:EnvironmentContext environment_context) + throws (1:InvalidOperationException o1, 2:MetaException o2) + + // rename the old partition to the new partition object by changing old part values to the part values + // in the new_part. old partition is identified from part_vals. + // partition keys in new_part should be the same as those in old partition. + void rename_partition(1:string db_name, 2:string tbl_name, 3:list part_vals, 4:Partition new_part) + throws (1:InvalidOperationException o1, 2:MetaException o2) + + // returns whether or not the partition name is valid based on the value of the config + // hive.metastore.partition.name.whitelist.pattern + bool partition_name_has_valid_characters(1:list part_vals, 2:bool throw_exception) + throws(1: MetaException o1) + + // gets the value of the configuration key in the metastore server. returns + // defaultValue if the key does not exist. if the configuration key does not + // begin with "hive", "mapred", or "hdfs", a ConfigValSecurityException is + // thrown. + string get_config_value(1:string name, 2:string defaultValue) + throws(1:ConfigValSecurityException o1) + + // converts a partition name into a partition values array + list partition_name_to_vals(1: string part_name) + throws(1: MetaException o1) + // converts a partition name into a partition specification (a mapping from + // the partition cols to the values) + map partition_name_to_spec(1: string part_name) + throws(1: MetaException o1) + + void markPartitionForEvent(1:string db_name, 2:string tbl_name, 3:map part_vals, + 4:PartitionEventType eventType) throws (1: MetaException o1, 2: NoSuchObjectException o2, + 3: UnknownDBException o3, 4: UnknownTableException o4, 5: UnknownPartitionException o5, + 6: InvalidPartitionException o6) + bool isPartitionMarkedForEvent(1:string db_name, 2:string tbl_name, 3:map part_vals, + 4: PartitionEventType eventType) throws (1: MetaException o1, 2:NoSuchObjectException o2, + 3: UnknownDBException o3, 4: UnknownTableException o4, 5: UnknownPartitionException o5, + 6: InvalidPartitionException o6) + + //index + Index add_index(1:Index new_index, 2: Table index_table) + throws(1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) + void alter_index(1:string dbname, 2:string base_tbl_name, 3:string idx_name, 4:Index new_idx) + throws (1:InvalidOperationException o1, 2:MetaException o2) + bool drop_index_by_name(1:string db_name, 2:string tbl_name, 3:string index_name, 4:bool deleteData) + throws(1:NoSuchObjectException o1, 2:MetaException o2) + Index get_index_by_name(1:string db_name 2:string tbl_name, 3:string index_name) + throws(1:MetaException o1, 2:NoSuchObjectException o2) + + list get_indexes(1:string db_name, 2:string tbl_name, 3:i16 max_indexes=-1) + throws(1:NoSuchObjectException o1, 2:MetaException o2) + list get_index_names(1:string db_name, 2:string tbl_name, 3:i16 max_indexes=-1) + throws(1:MetaException o2) + + //primary keys and foreign keys + PrimaryKeysResponse get_primary_keys(1:PrimaryKeysRequest request) + throws(1:MetaException o1, 2:NoSuchObjectException o2) + ForeignKeysResponse get_foreign_keys(1:ForeignKeysRequest request) + throws(1:MetaException o1, 2:NoSuchObjectException o2) + // other constraints + UniqueConstraintsResponse get_unique_constraints(1:UniqueConstraintsRequest request) + throws(1:MetaException o1, 2:NoSuchObjectException o2) + NotNullConstraintsResponse get_not_null_constraints(1:NotNullConstraintsRequest request) + throws(1:MetaException o1, 2:NoSuchObjectException o2) + + // column statistics interfaces + + // update APIs persist the column statistics object(s) that are passed in. If statistics already + // exists for one or more columns, the existing statistics will be overwritten. The update APIs + // validate that the dbName, tableName, partName, colName[] passed in as part of the ColumnStatistics + // struct are valid, throws InvalidInputException/NoSuchObjectException if found to be invalid + bool update_table_column_statistics(1:ColumnStatistics stats_obj) throws (1:NoSuchObjectException o1, + 2:InvalidObjectException o2, 3:MetaException o3, 4:InvalidInputException o4) + bool update_partition_column_statistics(1:ColumnStatistics stats_obj) throws (1:NoSuchObjectException o1, + 2:InvalidObjectException o2, 3:MetaException o3, 4:InvalidInputException o4) + + // get APIs return the column statistics corresponding to db_name, tbl_name, [part_name], col_name if + // such statistics exists. If the required statistics doesn't exist, get APIs throw NoSuchObjectException + // For instance, if get_table_column_statistics is called on a partitioned table for which only + // partition level column stats exist, get_table_column_statistics will throw NoSuchObjectException + ColumnStatistics get_table_column_statistics(1:string db_name, 2:string tbl_name, 3:string col_name) throws + (1:NoSuchObjectException o1, 2:MetaException o2, 3:InvalidInputException o3, 4:InvalidObjectException o4) + ColumnStatistics get_partition_column_statistics(1:string db_name, 2:string tbl_name, 3:string part_name, + 4:string col_name) throws (1:NoSuchObjectException o1, 2:MetaException o2, + 3:InvalidInputException o3, 4:InvalidObjectException o4) + TableStatsResult get_table_statistics_req(1:TableStatsRequest request) throws + (1:NoSuchObjectException o1, 2:MetaException o2) + PartitionsStatsResult get_partitions_statistics_req(1:PartitionsStatsRequest request) throws + (1:NoSuchObjectException o1, 2:MetaException o2) + AggrStats get_aggr_stats_for(1:PartitionsStatsRequest request) throws + (1:NoSuchObjectException o1, 2:MetaException o2) + bool set_aggr_stats_for(1:SetPartitionsStatsRequest request) throws + (1:NoSuchObjectException o1, 2:InvalidObjectException o2, 3:MetaException o3, 4:InvalidInputException o4) + + + // delete APIs attempt to delete column statistics, if found, associated with a given db_name, tbl_name, [part_name] + // and col_name. If the delete API doesn't find the statistics record in the metastore, throws NoSuchObjectException + // Delete API validates the input and if the input is invalid throws InvalidInputException/InvalidObjectException. + bool delete_partition_column_statistics(1:string db_name, 2:string tbl_name, 3:string part_name, 4:string col_name) throws + (1:NoSuchObjectException o1, 2:MetaException o2, 3:InvalidObjectException o3, + 4:InvalidInputException o4) + bool delete_table_column_statistics(1:string db_name, 2:string tbl_name, 3:string col_name) throws + (1:NoSuchObjectException o1, 2:MetaException o2, 3:InvalidObjectException o3, + 4:InvalidInputException o4) + + // + // user-defined functions + // + + void create_function(1:Function func) + throws (1:AlreadyExistsException o1, + 2:InvalidObjectException o2, + 3:MetaException o3, + 4:NoSuchObjectException o4) + + void drop_function(1:string dbName, 2:string funcName) + throws (1:NoSuchObjectException o1, 2:MetaException o3) + + void alter_function(1:string dbName, 2:string funcName, 3:Function newFunc) + throws (1:InvalidOperationException o1, 2:MetaException o2) + + list get_functions(1:string dbName, 2:string pattern) + throws (1:MetaException o1) + Function get_function(1:string dbName, 2:string funcName) + throws (1:MetaException o1, 2:NoSuchObjectException o2) + + GetAllFunctionsResponse get_all_functions() throws (1:MetaException o1) + + //authorization privileges + + bool create_role(1:Role role) throws(1:MetaException o1) + bool drop_role(1:string role_name) throws(1:MetaException o1) + list get_role_names() throws(1:MetaException o1) + // Deprecated, use grant_revoke_role() + bool grant_role(1:string role_name, 2:string principal_name, 3:PrincipalType principal_type, + 4:string grantor, 5:PrincipalType grantorType, 6:bool grant_option) throws(1:MetaException o1) + // Deprecated, use grant_revoke_role() + bool revoke_role(1:string role_name, 2:string principal_name, 3:PrincipalType principal_type) + throws(1:MetaException o1) + list list_roles(1:string principal_name, 2:PrincipalType principal_type) throws(1:MetaException o1) + GrantRevokeRoleResponse grant_revoke_role(1:GrantRevokeRoleRequest request) throws(1:MetaException o1) + + // get all role-grants for users/roles that have been granted the given role + // Note that in the returned list of RolePrincipalGrants, the roleName is + // redundant as it would match the role_name argument of this function + GetPrincipalsInRoleResponse get_principals_in_role(1: GetPrincipalsInRoleRequest request) throws(1:MetaException o1) + + // get grant information of all roles granted to the given principal + // Note that in the returned list of RolePrincipalGrants, the principal name,type is + // redundant as it would match the principal name,type arguments of this function + GetRoleGrantsForPrincipalResponse get_role_grants_for_principal(1: GetRoleGrantsForPrincipalRequest request) throws(1:MetaException o1) + + PrincipalPrivilegeSet get_privilege_set(1:HiveObjectRef hiveObject, 2:string user_name, + 3: list group_names) throws(1:MetaException o1) + list list_privileges(1:string principal_name, 2:PrincipalType principal_type, + 3: HiveObjectRef hiveObject) throws(1:MetaException o1) + + // Deprecated, use grant_revoke_privileges() + bool grant_privileges(1:PrivilegeBag privileges) throws(1:MetaException o1) + // Deprecated, use grant_revoke_privileges() + bool revoke_privileges(1:PrivilegeBag privileges) throws(1:MetaException o1) + GrantRevokePrivilegeResponse grant_revoke_privileges(1:GrantRevokePrivilegeRequest request) throws(1:MetaException o1); + + // this is used by metastore client to send UGI information to metastore server immediately + // after setting up a connection. + list set_ugi(1:string user_name, 2:list group_names) throws (1:MetaException o1) + + //Authentication (delegation token) interfaces + + // get metastore server delegation token for use from the map/reduce tasks to authenticate + // to metastore server + string get_delegation_token(1:string token_owner, 2:string renewer_kerberos_principal_name) + throws (1:MetaException o1) + + // method to renew delegation token obtained from metastore server + i64 renew_delegation_token(1:string token_str_form) throws (1:MetaException o1) + + // method to cancel delegation token obtained from metastore server + void cancel_delegation_token(1:string token_str_form) throws (1:MetaException o1) + + // add a delegation token + bool add_token(1:string token_identifier, 2:string delegation_token) + + // remove a delegation token + bool remove_token(1:string token_identifier) + + // get a delegation token by identifier + string get_token(1:string token_identifier) + + // get all delegation token identifiers + list get_all_token_identifiers() + + // add master key + i32 add_master_key(1:string key) throws (1:MetaException o1) + + // update master key + void update_master_key(1:i32 seq_number, 2:string key) throws (1:NoSuchObjectException o1, 2:MetaException o2) + + // remove master key + bool remove_master_key(1:i32 key_seq) + + // get master keys + list get_master_keys() + + // Transaction and lock management calls + // Get just list of open transactions + GetOpenTxnsResponse get_open_txns() + // Get list of open transactions with state (open, aborted) + GetOpenTxnsInfoResponse get_open_txns_info() + OpenTxnsResponse open_txns(1:OpenTxnRequest rqst) + void abort_txn(1:AbortTxnRequest rqst) throws (1:NoSuchTxnException o1) + void abort_txns(1:AbortTxnsRequest rqst) throws (1:NoSuchTxnException o1) + void commit_txn(1:CommitTxnRequest rqst) throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2) + LockResponse lock(1:LockRequest rqst) throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2) + LockResponse check_lock(1:CheckLockRequest rqst) + throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2, 3:NoSuchLockException o3) + void unlock(1:UnlockRequest rqst) throws (1:NoSuchLockException o1, 2:TxnOpenException o2) + ShowLocksResponse show_locks(1:ShowLocksRequest rqst) + void heartbeat(1:HeartbeatRequest ids) throws (1:NoSuchLockException o1, 2:NoSuchTxnException o2, 3:TxnAbortedException o3) + HeartbeatTxnRangeResponse heartbeat_txn_range(1:HeartbeatTxnRangeRequest txns) + void compact(1:CompactionRequest rqst) + CompactionResponse compact2(1:CompactionRequest rqst) + ShowCompactResponse show_compact(1:ShowCompactRequest rqst) + void add_dynamic_partitions(1:AddDynamicPartitions rqst) throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2) + + // Notification logging calls + NotificationEventResponse get_next_notification(1:NotificationEventRequest rqst) + CurrentNotificationEventId get_current_notificationEventId() + NotificationEventsCountResponse get_notification_events_count(NotificationEventsCountRequest rqst) + FireEventResponse fire_listener_event(1:FireEventRequest rqst) + void flushCache() + + // Repl Change Management api + CmRecycleResponse cm_recycle(1:CmRecycleRequest request) throws(1:MetaException o1) + + GetFileMetadataByExprResult get_file_metadata_by_expr(1:GetFileMetadataByExprRequest req) + GetFileMetadataResult get_file_metadata(1:GetFileMetadataRequest req) + PutFileMetadataResult put_file_metadata(1:PutFileMetadataRequest req) + ClearFileMetadataResult clear_file_metadata(1:ClearFileMetadataRequest req) + CacheFileMetadataResult cache_file_metadata(1:CacheFileMetadataRequest req) + + // Metastore DB properties + string get_metastore_db_uuid() throws (1:MetaException o1) + + // Workload management API's + WMCreateResourcePlanResponse create_resource_plan(1:WMCreateResourcePlanRequest request) + throws(1:AlreadyExistsException o1, 2:InvalidObjectException o2, 3:MetaException o3) + + WMGetResourcePlanResponse get_resource_plan(1:WMGetResourcePlanRequest request) + throws(1:NoSuchObjectException o1, 2:MetaException o2) + + WMGetActiveResourcePlanResponse get_active_resource_plan(1:WMGetActiveResourcePlanRequest request) + throws(1:MetaException o2) + + WMGetAllResourcePlanResponse get_all_resource_plans(1:WMGetAllResourcePlanRequest request) + throws(1:MetaException o1) + + WMAlterResourcePlanResponse alter_resource_plan(1:WMAlterResourcePlanRequest request) + throws(1:NoSuchObjectException o1, 2:InvalidOperationException o2, 3:MetaException o3) + + WMValidateResourcePlanResponse validate_resource_plan(1:WMValidateResourcePlanRequest request) + throws(1:NoSuchObjectException o1, 2:MetaException o2) + + WMDropResourcePlanResponse drop_resource_plan(1:WMDropResourcePlanRequest request) + throws(1:NoSuchObjectException o1, 2:InvalidOperationException o2, 3:MetaException o3) + + WMCreateTriggerResponse create_wm_trigger(1:WMCreateTriggerRequest request) + throws(1:AlreadyExistsException o1, 2:NoSuchObjectException o2, 3:InvalidObjectException o3, 4:MetaException o4) + + WMAlterTriggerResponse alter_wm_trigger(1:WMAlterTriggerRequest request) + throws(1:NoSuchObjectException o1, 2:InvalidObjectException o2, 3:MetaException o3) + + WMDropTriggerResponse drop_wm_trigger(1:WMDropTriggerRequest request) + throws(1:NoSuchObjectException o1, 2:InvalidOperationException o2, 3:MetaException o3) + + WMGetTriggersForResourePlanResponse get_triggers_for_resourceplan(1:WMGetTriggersForResourePlanRequest request) + throws(1:NoSuchObjectException o1, 2:MetaException o2) +} + +// * Note about the DDL_TIME: When creating or altering a table or a partition, +// if the DDL_TIME is not set, the current time will be used. + +// For storing info about archived partitions in parameters + +// Whether the partition is archived +const string IS_ARCHIVED = "is_archived", +// The original location of the partition, before archiving. After archiving, +// this directory will contain the archive. When the partition +// is dropped, this directory will be deleted +const string ORIGINAL_LOCATION = "original_location", + +// Whether or not the table is considered immutable - immutable tables can only be +// overwritten or created if unpartitioned, or if partitioned, partitions inside them +// can only be overwritten or created. Immutability supports write-once and replace +// semantics, but not append. +const string IS_IMMUTABLE = "immutable", + +// these should be needed only for backward compatibility with filestore +const string META_TABLE_COLUMNS = "columns", +const string META_TABLE_COLUMN_TYPES = "columns.types", +const string BUCKET_FIELD_NAME = "bucket_field_name", +const string BUCKET_COUNT = "bucket_count", +const string FIELD_TO_DIMENSION = "field_to_dimension", +const string META_TABLE_NAME = "name", +const string META_TABLE_DB = "db", +const string META_TABLE_LOCATION = "location", +const string META_TABLE_SERDE = "serde", +const string META_TABLE_PARTITION_COLUMNS = "partition_columns", +const string META_TABLE_PARTITION_COLUMN_TYPES = "partition_columns.types", +const string FILE_INPUT_FORMAT = "file.inputformat", +const string FILE_OUTPUT_FORMAT = "file.outputformat", +const string META_TABLE_STORAGE = "storage_handler", +const string TABLE_IS_TRANSACTIONAL = "transactional", +const string TABLE_NO_AUTO_COMPACT = "no_auto_compaction", +const string TABLE_TRANSACTIONAL_PROPERTIES = "transactional_properties", + + diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 9ad8c680065..f540f9da985 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -29,7 +29,7 @@ RemoteCacheController::recover(const std::string & local_path_, std::function, RemoteReadBufferCache::Create { if (citer->second.cache_controller->isValid()) { - // move the key to the list end, this case should not happend? + // move the key to the list end, this case should not happen? keys.splice(keys.end(), keys, citer->second.key_iterator); return {std::make_shared(citer->second.cache_controller.get(), file_size), CreateReaderError::OK}; } From b6ac5442e2593806bbebab6008a33cdeafb58020 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 5 Nov 2021 20:32:17 +0800 Subject: [PATCH 0014/1260] fix duplicated header --- src/Processors/Formats/Impl/ArrowBufferedStreams.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp index 9e781307a36..e708e826690 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include From 7b5d7a0f02f51a884afc1518977906fca7e2d97c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 6 Nov 2021 10:31:15 +0800 Subject: [PATCH 0015/1260] add missing source files --- contrib/arrow-cmake/CMakeLists.txt | 17 + src/Storages/Hive/HiveCommon.cpp | 190 +++++++++ src/Storages/Hive/HiveCommon.h | 97 +++++ src/Storages/Hive/HiveFile.cpp | 169 ++++++++ src/Storages/Hive/HiveFile.h | 230 +++++++++++ src/Storages/Hive/HiveSettings.cpp | 64 +++ src/Storages/Hive/HiveSettings.h | 40 ++ src/Storages/Hive/StorageHive.cpp | 636 +++++++++++++++++++++++++++++ src/Storages/Hive/StorageHive.h | 152 +++++++ 9 files changed, 1595 insertions(+) create mode 100644 src/Storages/Hive/HiveCommon.cpp create mode 100644 src/Storages/Hive/HiveCommon.h create mode 100644 src/Storages/Hive/HiveFile.cpp create mode 100644 src/Storages/Hive/HiveFile.h create mode 100644 src/Storages/Hive/HiveSettings.cpp create mode 100644 src/Storages/Hive/HiveSettings.h create mode 100644 src/Storages/Hive/StorageHive.cpp create mode 100644 src/Storages/Hive/StorageHive.h diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 841c280d192..9b4428afb18 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -48,6 +48,23 @@ set_target_properties(${THRIFT_LIBRARY} PROPERTIES CXX_STANDARD 14) # REMOVE aft target_include_directories(${THRIFT_LIBRARY} SYSTEM PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src") target_link_libraries (${THRIFT_LIBRARY} PRIVATE boost::headers_only) +set(COMPILER_DIR "${ClickHouse_SOURCE_DIR}/contrib/thrift/compiler/cpp") +configure_file(${COMPILER_DIR}/src/thrift/version.h.in ${COMPILER_DIR}/src/thrift/version.h) +set(thrift-compiler_SOURCES + ${COMPILER_DIR}/src/thrift/main.cc + ${COMPILER_DIR}/src/thrift/audit/t_audit.cpp + ${COMPILER_DIR}/src/thrift/generate/t_cpp_generator.cpp + ${COMPILER_DIR}/src/thrift/common.cc + ${COMPILER_DIR}/src/thrift/generate/t_generator.cc + ${COMPILER_DIR}/src/thrift/parse/t_typedef.cc + ${COMPILER_DIR}/src/thrift/parse/parse.cc + ${COMPILER_DIR}/src/thrift/version.h + ${COMPILER_DIR}/src/thrift/thrifty.cc + ${COMPILER_DIR}/src/thrift/thriftl.cc + ${COMPILER_DIR}/src/thrift/thrifty.hh +) +add_executable(thrift-compiler ${thrift-compiler_SOURCES}) +include_directories(thrift-compiler SYSTEM ${COMPILER_DIR}/src) # === orc diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp new file mode 100644 index 00000000000..9be89f5c78d --- /dev/null +++ b/src/Storages/Hive/HiveCommon.cpp @@ -0,0 +1,190 @@ +#include + + +#if USE_HDFS + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NO_HIVEMETASTORE; + extern const int BAD_ARGUMENTS; +} + +std::shared_ptr HMSClient::getTableMeta(const std::string & db_name, const std::string & table_name) +{ + LOG_TRACE(log, "get table meta:" + db_name + ":" + table_name); + std::lock_guard lock{mutex}; + + auto table = std::make_shared(); + std::vector partitions; + try + { + client->get_table(*table, db_name, table_name); + + /** + * query the lastest partition info to check new change + */ + client->get_partitions(partitions, db_name, table_name, -1); + } + catch (apache::thrift::transport::TTransportException & e) + { + setExpired(); + throw Exception("Hive Metastore expired because " + String(e.what()), ErrorCodes::NO_HIVEMETASTORE); + } + + std::string cache_key = db_name + "." + table_name; + std::shared_ptr result = table_meta_cache.get(cache_key); + bool update_cache = false; + std::map old_partition_infos; + std::map partition_infos; + if (result) + { + old_partition_infos = result->getPartitionInfos(); + if (old_partition_infos.size() != partitions.size()) + update_cache = true; + } + else + { + update_cache = true; + } + + for (const auto & partition : partitions) + { + auto & pinfo = partition_infos[partition.sd.location]; + pinfo.partition = partition; + + // query files under the patition by hdfs api is costly, we reuse the files in case the partion has no change + if (result) + { + auto it = old_partition_infos.find(partition.sd.location); + if (it != old_partition_infos.end() && it->second.equal(partition)) + pinfo.files = it->second.files; + else + update_cache = true; + } + } + + /** + * FIXME: force to update. + * we have found some cases under which the partition's meta don't update if the table changed + */ + //invalid_meta = true; + + if (update_cache) + { + LOG_INFO(log, "reload hive partition meta info:" + db_name + ":" + table_name); + result = std::make_shared(db_name, table_name, table, std::move(partition_infos), getContext()); + table_meta_cache.set(cache_key, result); + } + return result; +} + +void HMSClient::clearTableMeta(const std::string & db_name, const std::string & table_name) +{ + std::lock_guard lock{mutex}; + std::string cache_key = db_name + "." + table_name; + std::shared_ptr meta = table_meta_cache.get(cache_key); + if (meta) + table_meta_cache.set(cache_key, nullptr); +} + +void HMSClient::setClient(std::shared_ptr c) +{ + std::lock_guard lock{mutex}; + client = c; + clearExpired(); +} + +bool HMSClient::PartitionInfo::equal(const Apache::Hadoop::Hive::Partition & other) +{ + // parameters include keys:numRows,numFiles,rawDataSize,totalSize,transient_lastDdlTime + auto it1 = partition.parameters.begin(); + auto it2 = other.parameters.begin(); + for (; it1 != partition.parameters.end() && it2 != other.parameters.end(); ++it1, ++it2) + { + if (it1->first != it2->first || it1->second != it2->second) + return false; + } + return (it1 == partition.parameters.end() && it2 == other.parameters.end()); +} + +std::vector HMSClient::HiveTableMeta::getPartitions() +{ + std::vector result; + + std::lock_guard lock{mutex}; + for (const auto & partition : partition_infos) + result.emplace_back(partition.second.partition); + return result; +} + +std::vector HMSClient::HiveTableMeta::getLocationFiles(const std::string & location) +{ + std::map::const_iterator it; + if (!empty_partition_keys) + { + std::lock_guard lock{mutex}; + it = partition_infos.find(location); + if (it == partition_infos.end()) + throw Exception("invalid location " + location, ErrorCodes::BAD_ARGUMENTS); + if (it->second.files != nullptr) + return *(it->second.files); + } + + auto fs_builder = createHDFSBuilder(getNameNodeUrl(table->sd.location), getContext()->getGlobalContext()->getConfigRef()); + auto fs = createHDFSFS(fs_builder.get()); + Poco::URI uri(location); + HDFSFileInfo ls; + ls.file_info = hdfsListDirectory(fs.get(), uri.getPath().c_str(), &ls.length); + auto result = std::make_shared>(); + for (int i = 0; i < ls.length; ++i) + { + auto & finfo = ls.file_info[i]; + if (finfo.mKind != 'D' && finfo.mSize > 0) + result->emplace_back(String(finfo.mName), finfo.mLastMod, finfo.mSize); + } + + if (!empty_partition_keys) + { + std::lock_guard lock{mutex}; + partition_infos[location].files = result; + } + return *result; +} + +std::vector HMSClient::HiveTableMeta::getLocationFiles(const HDFSFSPtr & fs, const std::string & location) +{ + std::map::const_iterator it; + if (!empty_partition_keys) + { + std::lock_guard lock{mutex}; + it = partition_infos.find(location); + if (it == partition_infos.end()) + throw Exception("invalid location " + location, ErrorCodes::BAD_ARGUMENTS); + if (it->second.files != nullptr) + return *(it->second.files); + } + + Poco::URI location_uri(location); + HDFSFileInfo ls; + ls.file_info = hdfsListDirectory(fs.get(), location_uri.getPath().c_str(), &ls.length); + auto result = std::make_shared>(); + for (int i = 0; i < ls.length; ++i) + { + auto & finfo = ls.file_info[i]; + if (finfo.mKind != 'D' && finfo.mSize > 0) + result->emplace_back(String(finfo.mName), finfo.mLastMod, finfo.mSize); + } + + if (!empty_partition_keys) + { + std::lock_guard lock{mutex}; + partition_infos[location].files = result; + } + return *result; +} + + +} +#endif diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h new file mode 100644 index 00000000000..647bbde8f89 --- /dev/null +++ b/src/Storages/Hive/HiveCommon.h @@ -0,0 +1,97 @@ +#pragma once + +#if !defined(ARCADIA_BUILD) +#include +#endif + +#if USE_HDFS +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +class HMSClient : public WithContext +{ +public: + struct FileInfo + { + FileInfo() = default; + FileInfo(const std::string & path_, UInt64 ts_, size_t size_) : path(path_), last_mod_ts(ts_), size(size_) { } + std::string path; + UInt64 last_mod_ts; // in ms + size_t size; + }; + + struct PartitionInfo + { + Apache::Hadoop::Hive::Partition partition; + std::shared_ptr> files; + bool equal(const Apache::Hadoop::Hive::Partition & other); + }; + + // use for speeding up query metadata + struct HiveTableMeta : public WithContext + { + public: + HiveTableMeta( + const std::string & db_name_, + const std::string & table_name_, + std::shared_ptr table_, + std::map && partition_infos_, + ContextPtr context_) + : WithContext(context_) + , db_name(db_name_) + , table_name(table_name_) + , table(table_) + , partition_infos(partition_infos_) + , empty_partition_keys(table->partitionKeys.empty()) + { + } + std::vector getPartitions(); + inline std::map getPartitionInfos() + { + std::lock_guard lock{mutex}; + return partition_infos; + } + std::vector getLocationFiles(const std::string & location); + std::vector getLocationFiles(const HDFSFSPtr & fs, const std::string & location); + inline std::shared_ptr getTable() { return table; } + + private: + std::string db_name; + std::string table_name; + + std::mutex mutex; + std::shared_ptr table; + std::map partition_infos; + const bool empty_partition_keys; + }; + + explicit HMSClient(std::shared_ptr client_, ContextPtr context_) + : WithContext(context_), client(client_), table_meta_cache(1000) + { + } + + std::shared_ptr getTableMeta(const std::string & db_name, const std::string & table_name); + void clearTableMeta(const std::string & db_name, const std::string & table_name); + void setClient(std::shared_ptr c); + inline bool isExpired() const { return expired; } + inline void setExpired() { expired = true; } + inline void clearExpired() { expired = false; } + +private: + std::shared_ptr client; + LRUCache table_meta_cache; + mutable std::mutex mutex; + std::atomic expired{false}; + + Poco::Logger * log = &Poco::Logger::get("HMSClient"); +}; +} + +#endif diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp new file mode 100644 index 00000000000..b9ea7c85758 --- /dev/null +++ b/src/Storages/Hive/HiveFile.cpp @@ -0,0 +1,169 @@ +#include +#include +#include + +#if USE_HDFS +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +template +Range createRangeFromOrcStatistics(const S * stats) +{ + if (stats->hasMinimum() && stats->hasMaximum()) + { + return Range(T(stats->getMinimum()), true, T(stats->getMaximum()), true); + } + else if (stats->hasMinimum()) + { + return Range::createLeftBounded(T(stats->getMinimum()), true); + } + else if (stats->hasMaximum()) + { + return Range::createRightBounded(T(stats->getMaximum()), true); + } + else + { + return Range(); + } +} + +template +Range createRangeFromParquetStatistics(std::shared_ptr stats) +{ + if (!stats->HasMinMax()) + return Range(); + return Range(T(stats->min()), true, T(stats->max()), true); +} + +Range createRangeFromParquetStatistics(std::shared_ptr stats) +{ + if (!stats->HasMinMax()) + return Range(); + String min_val(reinterpret_cast(stats->min().ptr), stats->min().len); + String max_val(reinterpret_cast(stats->max().ptr), stats->max().len); + return Range(min_val, true, max_val, true); +} + +Range HiveOrcFile::buildRange(const orc::ColumnStatistics * col_stats) +{ + if (!col_stats || col_stats->hasNull()) + return {}; + + if (const auto * int_stats = dynamic_cast(col_stats)) + { + return createRangeFromOrcStatistics(int_stats); + } + else if (const auto * double_stats = dynamic_cast(col_stats)) + { + return createRangeFromOrcStatistics(double_stats); + } + else if (const auto * string_stats = dynamic_cast(col_stats)) + { + return createRangeFromOrcStatistics(string_stats); + } + else if (const auto * bool_stats = dynamic_cast(col_stats)) + { + auto false_cnt = bool_stats->getFalseCount(); + auto true_cnt = bool_stats->getTrueCount(); + if (false_cnt && true_cnt) + { + return Range(UInt8(0), true, UInt8(1), true); + } + else if (false_cnt) + { + return Range::createLeftBounded(UInt8(0), true); + } + else if (true_cnt) + { + return Range::createRightBounded(UInt8(1), true); + } + } + else if (const auto * timestamp_stats = dynamic_cast(col_stats)) + { + return createRangeFromOrcStatistics(timestamp_stats); + } + else if (const auto * date_stats = dynamic_cast(col_stats)) + { + return createRangeFromOrcStatistics(date_stats); + } + return {}; +} + +void HiveOrcFile::prepareReader() +{ + // TODO To be implemented +} + +void HiveOrcFile::prepareColumnMapping() +{ + // TODO To be implemented +} + +bool HiveOrcFile::hasMinMaxIndex() const +{ + return false; +} + + +std::unique_ptr HiveOrcFile::buildMinMaxIndex(const orc::Statistics * /*statistics*/) +{ + // TODO To be implemented + return {}; +} + + +void HiveOrcFile::loadMinMaxIndex() +{ + // TODO To be implemented +} + +bool HiveOrcFile::hasSubMinMaxIndex() const +{ + return false; +} + + +void HiveOrcFile::loadSubMinMaxIndex() +{ + // TODO To be implemented +} + +bool HiveParquetFile::hasSubMinMaxIndex() const +{ + // TODO To be implemented + return false; +} + +void HiveParquetFile::prepareReader() +{ + // TODO To be implemented +} + + +void HiveParquetFile::loadSubMinMaxIndex() +{ + // TODO To be implemented +} + +} +#endif diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h new file mode 100644 index 00000000000..ada46ff0038 --- /dev/null +++ b/src/Storages/Hive/HiveFile.h @@ -0,0 +1,230 @@ +#pragma once +#include +#include "Core/Block.h" + +#if USE_HDFS +#include +#include + +#include +#include +#include + +#include +#include +#include +#include + +namespace orc +{ +class Reader; +} + +namespace parquet +{ +class ParquetFileReader; +namespace arrow +{ + class FileReader; +} +} + +namespace arrow +{ +namespace io +{ + class RandomAccessFile; +} + +class Buffer; +} + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class IHiveFile : public WithContext +{ +public: + IHiveFile( + const FieldVector & values_, + const String & namenode_url_, + const String & path_, + UInt64 ts_, + size_t size_, + const NamesAndTypesList & index_names_and_types_, + const std::shared_ptr & storage_settings_, + ContextPtr context_) + : WithContext(context_) + , partition_values(values_) + , namenode_url(namenode_url_) + , path(path_) + , last_mod_ts(ts_) + , size(size_) + , index_names_and_types(index_names_and_types_) + , storage_settings(storage_settings_) + { + // std::cout << "1delim:" << storage_settings->hive_text_field_delimeter << std::endl; + // std::cout << "1disable orc:" << storage_settings->disable_orc_stripe_minmax_index << std::endl; + // std::cout << "1disable parquet:" << storage_settings->disable_parquet_rowgroup_minmax_index << std::endl; + } + virtual ~IHiveFile() = default; + + using FileFormat = StorageHive::FileFormat; + virtual FileFormat getFormat() const = 0; + + virtual String getName() const = 0; + + virtual String getPath() const { return path; } + + virtual FieldVector getPartitionValues() const { return partition_values; } + + virtual String getNamenodeUrl() { return namenode_url; } + + virtual bool hasMinMaxIndex() const { return false; } + + virtual void loadMinMaxIndex() + { + throw Exception("Method loadMinMaxIndex is not supported by hive file:" + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + + virtual std::shared_ptr getMinMaxIndex() const { return minmax_idx; } + + // Do hive file contains sub-file level minmax index? + virtual bool hasSubMinMaxIndex() const { return false; } + + virtual void loadSubMinMaxIndex() + { + throw Exception("Method loadSubMinMaxIndex is not supported by hive file:" + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + + virtual const std::vector> & getSubMinMaxIndexes() const { return sub_minmax_idxes; } + + virtual void setSkipSplits(const std::set & splits) { skip_splits = splits; } + + virtual const std::set & getSkipSplits() const { return skip_splits; } + + inline std::string describeMinMaxIndex(const std::shared_ptr & idx) const + { + if (!idx) + return ""; + + std::vector strs; + strs.reserve(index_names_and_types.size()); + size_t i = 0; + for (const auto & name_type : index_names_and_types) + { + strs.push_back(name_type.name + ":" + name_type.type->getName() + idx->hyperrectangle[i++].toString()); + } + return boost::algorithm::join(strs, "|"); + } + + inline UInt64 getLastModTs() const { return last_mod_ts; } + inline size_t getSize() const { return size; } + +protected: + FieldVector partition_values; + String namenode_url; + String path; + UInt64 last_mod_ts; + size_t size; + NamesAndTypesList index_names_and_types; + std::shared_ptr minmax_idx; + std::vector> sub_minmax_idxes; + std::set skip_splits; // skip splits for this file after applying minmax index (if any) + std::shared_ptr storage_settings; +}; + +class HiveTextFile : public IHiveFile +{ +public: + HiveTextFile( + const FieldVector & values_, + const String & namenode_url_, + const String & path_, + UInt64 ts_, + size_t size_, + const NamesAndTypesList & index_names_and_types_, + const std::shared_ptr & hive_settings_, + ContextPtr context_) + : IHiveFile(values_, namenode_url_, path_, ts_, size_, index_names_and_types_, hive_settings_, context_) + { + } + + using FileFormat = StorageHive::FileFormat; + virtual FileFormat getFormat() const override { return FileFormat::TEXT; } + virtual String getName() const override { return "TEXT"; } +}; + +class HiveOrcFile : public IHiveFile +{ +public: + HiveOrcFile( + const FieldVector & values_, + const String & namenode_url_, + const String & path_, + UInt64 ts_, + size_t size_, + const NamesAndTypesList & index_names_and_types_, + const std::shared_ptr & hive_settings_, + ContextPtr context_) + : IHiveFile(values_, namenode_url_, path_, ts_, size_, index_names_and_types_, hive_settings_, context_) + { + } + + using FileFormat = StorageHive::FileFormat; + virtual FileFormat getFormat() const override { return FileFormat::ORC; } + virtual String getName() const override { return "ORC"; } + virtual bool hasMinMaxIndex() const override; + virtual void loadMinMaxIndex() override; + + virtual bool hasSubMinMaxIndex() const override; + virtual void loadSubMinMaxIndex() override; + +protected: + virtual std::unique_ptr buildMinMaxIndex(const orc::Statistics * statistics); + virtual Range buildRange(const orc::ColumnStatistics * col_stats); + virtual void prepareReader(); + virtual void prepareColumnMapping(); + + std::shared_ptr reader; + std::map orc_column_positions; +}; + +class HiveParquetFile : public IHiveFile +{ +public: + HiveParquetFile( + const FieldVector & values_, + const String & namenode_url_, + const String & path_, + UInt64 ts_, + size_t size_, + const NamesAndTypesList & index_names_and_types_, + const std::shared_ptr & hive_settings_, + ContextPtr context_) + : IHiveFile(values_, namenode_url_, path_, ts_, size_, index_names_and_types_, hive_settings_, context_) + { + } + + using FileFormat = StorageHive::FileFormat; + virtual FileFormat getFormat() const override { return FileFormat::PARQUET; } + virtual String getName() const override { return "PARQUET"; } + + virtual bool hasSubMinMaxIndex() const override; + virtual void loadSubMinMaxIndex() override; + +protected: + virtual void prepareReader(); + + std::shared_ptr fs; + std::shared_ptr reader; + std::map parquet_column_positions; +}; +} + + +#endif diff --git a/src/Storages/Hive/HiveSettings.cpp b/src/Storages/Hive/HiveSettings.cpp new file mode 100644 index 00000000000..efdfc1c954b --- /dev/null +++ b/src/Storages/Hive/HiveSettings.cpp @@ -0,0 +1,64 @@ +#include + +#if USE_HDFS +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} + +IMPLEMENT_SETTINGS_TRAITS(HiveSettingsTraits, LIST_OF_HIVE_SETTINGS) + +void HiveSettings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config) +{ + if (!config.has(config_elem)) + return; + + Poco::Util::AbstractConfiguration::Keys config_keys; + config.keys(config_elem, config_keys); + + try + { + for (const String & key : config_keys) + set(key, config.getString(config_elem + "." + key)); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + e.addMessage("in MergeTree config"); + throw; + } +} + +void HiveSettings::loadFromQuery(ASTStorage & storage_def) +{ + if (storage_def.settings) + { + try + { + applyChanges(storage_def.settings->changes); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + e.addMessage("for storage " + storage_def.engine->name); + throw; + } + } + else + { + auto settings_ast = std::make_shared(); + settings_ast->is_standalone = false; + storage_def.set(storage_def.settings, settings_ast); + } +} + +} +#endif diff --git a/src/Storages/Hive/HiveSettings.h b/src/Storages/Hive/HiveSettings.h new file mode 100644 index 00000000000..8bd7c85a855 --- /dev/null +++ b/src/Storages/Hive/HiveSettings.h @@ -0,0 +1,40 @@ +#pragma once + +#if !defined(ARCADIA_BUILD) +#include +#endif + +#if USE_HDFS +#include +#include +#include + +namespace DB +{ +class ASTStorage; + +#define HIVE_RELATED_SETTINGS(M) \ + M(Char, hive_text_field_delimeter, '\x01', "How to split one row of hive data with format text", 0) \ + M(Bool, disable_orc_stripe_minmax_index, true, "Disable using ORC stripe level minmax index.", 0) \ + M(Bool, disable_parquet_rowgroup_minmax_index, true, "Disable using Parquet row-group level minmax index.", 0) \ + M(Bool, disable_orc_file_minmax_index, false, "Disable using ORC file level minmax index.", 0) + +#define LIST_OF_HIVE_SETTINGS(M) \ + HIVE_RELATED_SETTINGS(M) \ + FORMAT_FACTORY_SETTINGS(M) + +DECLARE_SETTINGS_TRAITS(HiveSettingsTraits, LIST_OF_HIVE_SETTINGS) + + +/** Settings for the Hive engine. + * Could be loaded from a CREATE TABLE query (SETTINGS clause). + */ +class HiveSettings : public BaseSettings +{ +public: + void loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config); + void loadFromQuery(ASTStorage & storage_def); +}; +} + +#endif diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp new file mode 100644 index 00000000000..265a25d7566 --- /dev/null +++ b/src/Storages/Hive/StorageHive.cpp @@ -0,0 +1,636 @@ +#include +#include "Interpreters/TreeRewriter.h" + +#if USE_HDFS + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#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 NOT_IMPLEMENTED; + extern const int INVALID_PARTITION_VALUE; + extern const int BAD_ARGUMENTS; + extern const int CANNOT_OPEN_FILE; +} + +using HiveFilePtr = std::shared_ptr; +using HiveFiles = std::vector; + +static std::string getBaseName(const String & path) +{ + size_t basename_start = path.rfind('/'); + return std::string{&path[basename_start + 1], path.length() - basename_start - 1}; +} + +class HiveSource : public SourceWithProgress, WithContext +{ +public: + struct SourcesInfo + { + HMSClientPtr hms_client; + std::string database; + std::string table_name; + HiveFiles hive_files; + NamesAndTypesList partition_name_types; + + std::atomic next_uri_to_read = 0; + + bool need_path_column = false; + bool need_file_column = false; + }; + + using SourcesInfoPtr = std::shared_ptr; + + static Block getHeader(Block header, const SourcesInfoPtr & source_info) + { + if (source_info->need_path_column) + header.insert({DataTypeString().createColumn(), std::make_shared(), "_path"}); + if (source_info->need_file_column) + header.insert({DataTypeString().createColumn(), std::make_shared(), "_file"}); + + return header; + } + + HiveSource( + SourcesInfoPtr source_info_, + String hdfs_namenode_url_, + String format_, + String compression_method_, + Block sample_block_, + ContextPtr context_, + UInt64 max_block_size_, + const Names & text_input_field_names_ = {}) + : SourceWithProgress(getHeader(sample_block_, source_info_)) + , WithContext(context_) + , source_info(std::move(source_info_)) + , hdfs_namenode_url(hdfs_namenode_url_) + , format(std::move(format_)) + , compression_method(compression_method_) + , max_block_size(max_block_size_) + , sample_block(std::move(sample_block_)) + , to_read_block(sample_block) + , text_input_field_names(text_input_field_names_) + , format_settings(getFormatSettings(getContext())) + { + to_read_block = sample_block; + for (const auto & name_type : source_info->partition_name_types) + { + to_read_block.erase(name_type.name); + } + format_settings.csv.delimiter = '\x01'; + format_settings.csv.input_field_names = text_input_field_names; + } + + String getName() const override { return "Hive"; } + + Chunk generate() override + { + while (true) + { + if (!reader) + { + current_idx = source_info->next_uri_to_read.fetch_add(1); + if (current_idx >= source_info->hive_files.size()) + return {}; + + const auto & curr_file = source_info->hive_files[current_idx]; + current_path = curr_file->getPath(); + + String uri_with_path = hdfs_namenode_url + current_path; + auto compression = chooseCompressionMethod(current_path, compression_method); + std::unique_ptr raw_read_buf; + try + { + raw_read_buf = std::make_unique( + hdfs_namenode_url, current_path, getContext()->getGlobalContext()->getConfigRef()); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::CANNOT_OPEN_FILE) + { + source_info->hms_client->clearTableMeta(source_info->database, source_info->table_name); + throw; + } + } + + std::unique_ptr remote_read_buf = RemoteReadBuffer::create( + "Hive", + getNameNodeCluster(hdfs_namenode_url), + uri_with_path, + curr_file->getLastModTs(), + curr_file->getSize(), + std::move(raw_read_buf)); + // std::unique_ptr remote_read_buf = std::move(raw_read_buf); + if (curr_file->getFormat() == StorageHive::FileFormat::TEXT) + read_buf = wrapReadBufferWithCompressionMethod(std::move(remote_read_buf), compression); + else + read_buf = std::move(remote_read_buf); + + auto input_format = FormatFactory::instance().getInputFormat( + format, *read_buf, to_read_block, getContext(), max_block_size, format_settings); + pipeline = QueryPipeline(std::move(input_format)); + reader = std::make_unique(pipeline); + } + + Block res; + if (reader->pull(res)) + { + Columns columns = res.getColumns(); + UInt64 num_rows = res.rows(); + auto types = source_info->partition_name_types.getTypes(); + for (size_t i = 0; i < types.size(); ++i) + { + auto column = types[i]->createColumnConst(num_rows, source_info->hive_files[current_idx]->getPartitionValues()[i]); + auto previous_idx = sample_block.getPositionByName(source_info->partition_name_types.getNames()[i]); + columns.insert(columns.begin() + previous_idx, column->convertToFullColumnIfConst()); + } + + /// Enrich with virtual columns. + if (source_info->need_path_column) + { + auto column = DataTypeString().createColumnConst(num_rows, current_path); + columns.push_back(column->convertToFullColumnIfConst()); + } + + if (source_info->need_file_column) + { + size_t last_slash_pos = current_path.find_last_of('/'); + auto file_name = current_path.substr(last_slash_pos + 1); + + auto column = DataTypeString().createColumnConst(num_rows, std::move(file_name)); + columns.push_back(column->convertToFullColumnIfConst()); + } + return Chunk(std::move(columns), num_rows); + } + reader.reset(); + pipeline.reset(); + read_buf.reset(); + } + } + +private: + std::unique_ptr read_buf; + QueryPipeline pipeline; + std::unique_ptr reader; + SourcesInfoPtr source_info; + String hdfs_namenode_url; + String format; + String compression_method; + UInt64 max_block_size; + Block sample_block; + Block to_read_block; + const Names & text_input_field_names; + FormatSettings format_settings; + + String current_path; + size_t current_idx = 0; + + Poco::Logger * log = &Poco::Logger::get("StorageHive"); +}; + + +StorageHive::StorageHive( + const String & hms_url_, + const String & hive_database_, + const String & hive_table_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment_, + const ASTPtr & partition_by_ast_, + std::unique_ptr storage_settings_, + ContextPtr context_) + : IStorage(table_id_) + , WithContext(context_) + , hms_url(hms_url_) + , hive_database(hive_database_) + , hive_table(hive_table_) + , partition_by_ast(partition_by_ast_) + , storage_settings(std::move(storage_settings_)) +{ + getContext()->getRemoteHostFilter().checkURL(Poco::URI(hms_url)); + + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); + storage_metadata.setComment(comment_); + setInMemoryMetadata(storage_metadata); + + auto hms_client = getContext()->getHMSClient(hms_url); + auto table_meta = hms_client->getTableMeta(hive_database, hive_table); + + hdfs_namenode_url = getNameNodeUrl(table_meta->getTable()->sd.location); + table_schema = table_meta->getTable()->sd.cols; + + FileFormat hdfs_file_format = toFileFormat(table_meta->getTable()->sd.inputFormat); + switch (hdfs_file_format) + { + case FileFormat::TEXT: + format_name = "CSVWithNames"; + break; + case FileFormat::LZO_TEXT: + format_name = "CSVWithNames"; + break; + case FileFormat::RC_FILE: + // TODO to be implemented + throw Exception("Unsopported hive format rc_file", ErrorCodes::NOT_IMPLEMENTED); + case FileFormat::SEQUENCE_FILE: + // TODO to be implemented + throw Exception("Unsopported hive format sequence_file", ErrorCodes::NOT_IMPLEMENTED); + case FileFormat::AVRO: + format_name = "Avro"; + break; + case FileFormat::PARQUET: + format_name = "Parquet"; + break; + case FileFormat::ORC: + format_name = "ORC"; + break; + } + + // Need to specify text_input_fields_names from table_schema for TextInputFormated Hive table + if (format_name == "CSVWithNames") + { + size_t i = 0; + text_input_field_names.resize(table_schema.size()); + for (const auto & field : table_schema) + { + String name{field.name}; + boost::to_lower(name); + text_input_field_names[i++] = std::move(name); + } + } + initMinMaxIndexExpression(); +} + +void StorageHive::initMinMaxIndexExpression() +{ + auto metadata_snapshot = getInMemoryMetadataPtr(); + ASTPtr partition_key_expr_list = extractKeyExpressionList(partition_by_ast); + if (!partition_key_expr_list->children.empty()) + { + auto syntax_result = TreeRewriter(getContext()).analyze(partition_key_expr_list, metadata_snapshot->getColumns().getAllPhysical()); + partition_key_expr = ExpressionAnalyzer(partition_key_expr_list, syntax_result, getContext()).getActions(false); + + /// Add all columns used in the partition key to the min-max index. + partition_name_types = partition_key_expr->getRequiredColumnsWithTypes(); + partition_minmax_idx_expr = std::make_shared( + std::make_shared(partition_name_types), ExpressionActionsSettings::fromContext(getContext())); + } + + NamesAndTypesList all_name_types = metadata_snapshot->getColumns().getAllPhysical(); + for (const auto & column : all_name_types) + { + if (!partition_name_types.contains(column.name)) + hivefile_name_types.push_back(column); + } + hivefile_minmax_idx_expr = std::make_shared( + std::make_shared(hivefile_name_types), ExpressionActionsSettings::fromContext(getContext())); +} + +ASTPtr StorageHive::extractKeyExpressionList(const ASTPtr & node) +{ + if (!node) + return std::make_shared(); + + const auto * expr_func = node->as(); + if (expr_func && expr_func->name == "tuple") + { + /// Primary key is specified in tuple, extract its arguments. + return expr_func->arguments->clone(); + } + else + { + /// Primary key consists of one column. + auto res = std::make_shared(); + res->children.push_back(node); + return res; + } +} + + +HiveFilePtr createHiveFile( + const String & format_name, + const FieldVector & fields, + const String & namenode_url, + const String & path, + UInt64 ts, + size_t size, + const NamesAndTypesList & index_names_and_types, + const std::shared_ptr & hive_settings, + ContextPtr context) +{ + HiveFilePtr hive_file; + if (format_name == "CSVWithNames") + { + hive_file = std::make_shared(fields, namenode_url, path, ts, size, index_names_and_types, hive_settings, context); + } + else if (format_name == "ORC") + { + hive_file = std::make_shared(fields, namenode_url, path, ts, size, index_names_and_types, hive_settings, context); + } + else if (format_name == "Parquet") + { + hive_file = std::make_shared(fields, namenode_url, path, ts, size, index_names_and_types, hive_settings, context); + } + else + { + throw Exception("IHiveFile not implemented for format " + format_name, ErrorCodes::NOT_IMPLEMENTED); + } + return hive_file; +} + +Pipe StorageHive::read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + SelectQueryInfo & query_info, + ContextPtr context_, + QueryProcessingStage::Enum /* processed_stage */, + size_t max_block_size, + unsigned num_streams) +{ + HDFSBuilderWrapper builder = createHDFSBuilder(hdfs_namenode_url, context_->getGlobalContext()->getConfigRef()); + HDFSFSPtr fs = createHDFSFS(builder.get()); + auto hms_client = context_->getHMSClient(hms_url); + auto table_meta_cntrl = hms_client->getTableMeta(hive_database, hive_table); + + // List files under partition directory in HDFS + auto list_paths = [table_meta_cntrl, &fs](const String & path) { return table_meta_cntrl->getLocationFiles(fs, path); }; + + std::vector partitions = table_meta_cntrl->getPartitions(); + HiveFiles hive_files; // hive files to read + std::mutex hive_files_mutex; // Mutext to protect hive_files, which maybe appended in multiple threads + + auto append_hive_files = [&](const HMSClient::FileInfo & hfile, const FieldVector & fields) + { + String filename = getBaseName(hfile.path); + + // Skip temporary files starts with '.' + if (filename.find('.') == 0) + return; + + auto file = createHiveFile( + format_name, + fields, + hdfs_namenode_url, + hfile.path, + hfile.last_mod_ts, + hfile.size, + hivefile_name_types, + storage_settings, + context_); + + // Load file level minmax index and apply + const KeyCondition hivefile_key_condition(query_info, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); + if (file->hasMinMaxIndex()) + { + file->loadMinMaxIndex(); + if (!hivefile_key_condition.checkInHyperrectangle(file->getMinMaxIndex()->hyperrectangle, hivefile_name_types.getTypes()) + .can_be_true) + { + LOG_DEBUG(log, "skip file:{} index:{}", file->getPath(), file->describeMinMaxIndex(file->getMinMaxIndex())); + return; + } + } + + // Load sub-file level minmax index and appy + std::set skip_splits; + if (file->hasSubMinMaxIndex()) + { + file->loadSubMinMaxIndex(); + const auto & sub_minmax_idxes = file->getSubMinMaxIndexes(); + for (size_t i = 0; i < sub_minmax_idxes.size(); ++i) + { + if (!hivefile_key_condition.checkInHyperrectangle(sub_minmax_idxes[i]->hyperrectangle, hivefile_name_types.getTypes()) + .can_be_true) + { + LOG_DEBUG(log, "skip split:{} in file {}", i, file->getPath()); + skip_splits.insert(i); + } + } + file->setSkipSplits(skip_splits); + } + + { + std::lock_guard lock{hive_files_mutex}; + hive_files.push_back(file); + } + }; + + ThreadPool pool{num_streams}; + if (!partitions.empty()) + { + const auto partition_names = partition_name_types.getNames(); + const auto partition_types = partition_name_types.getTypes(); + + for (const auto & p : partitions) + { + auto f = [&]() + { + // Skip partition "__HIVE_DEFAULT_PARTITION__" + bool has_default_partition = false; + for (const auto & value : p.values) + { + if (value == "__HIVE_DEFAULT_PARTITION__") + { + has_default_partition = true; + break; + } + } + if (has_default_partition) + { + LOG_DEBUG(log, "skip partition:__HIVE_DEFAULT_PARTITION__"); + return; + } + + std::vector ranges; + WriteBufferFromOwnString wb; + if (p.values.size() != partition_names.size()) + throw Exception( + fmt::format("Partition value size not match, expect {}, but got {}", partition_names.size(), p.values.size()), + ErrorCodes::INVALID_PARTITION_VALUE); + + for (size_t i = 0; i < p.values.size(); ++i) + { + if (i != 0) + writeString(",", wb); + writeString(p.values[i], wb); + } + writeString("\n", wb); + + ReadBufferFromString buffer(wb.str()); + auto format = FormatFactory::instance().getInputFormat( + "CSV", buffer, partition_key_expr->getSampleBlock(), getContext(), getContext()->getSettingsRef().max_block_size); + auto pipeline = QueryPipeline(std::move(format)); + auto reader = std::make_unique(pipeline); + Block block; + if (!reader->pull(block) || !block.rows()) + throw Exception("Could not parse partition value: " + wb.str(), ErrorCodes::INVALID_PARTITION_VALUE); + + FieldVector fields(partition_names.size()); + for (size_t i = 0; i < partition_names.size(); ++i) + { + block.getByPosition(i).column->get(0, fields[i]); + ranges.emplace_back(fields[i]); + } + + const KeyCondition partition_key_condition(query_info, getContext(), partition_names, partition_minmax_idx_expr); + if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true) + { + LOG_DEBUG(log, "skip partition:{}", boost::algorithm::join(p.values, "|")); + return; + } + + auto paths = list_paths(p.sd.location); + for (const auto & path : paths) + { + append_hive_files(path, fields); + } + }; + pool.scheduleOrThrowOnError(f); + } + pool.wait(); + } + else if (partition_name_types.empty()) // Partition keys is empty + { + auto paths = list_paths(table_meta_cntrl->getTable()->sd.location); + for (const auto & path : paths) + { + pool.scheduleOrThrowOnError([&] { append_hive_files(path, {}); }); + } + pool.wait(); + } + else // Partition keys is not empty but partitions is empty + { + return {}; + } + + auto sources_info = std::make_shared(); + sources_info->hive_files = std::move(hive_files); + sources_info->database = hive_database; + sources_info->table_name = hive_table; + sources_info->hms_client = hms_client; + sources_info->partition_name_types = partition_name_types; + for (const auto & column : column_names) + { + if (column == "_path") + sources_info->need_path_column = true; + if (column == "_file") + sources_info->need_file_column = true; + } + + if (num_streams > sources_info->hive_files.size()) + num_streams = sources_info->hive_files.size(); + + Pipes pipes; + for (size_t i = 0; i < num_streams; ++i) + { + pipes.emplace_back(std::make_shared( + sources_info, + hdfs_namenode_url, + format_name, + compression_method, + metadata_snapshot->getSampleBlock(), + getContext(), + max_block_size, + text_input_field_names)); + } + return Pipe::unitePipes(std::move(pipes)); +} + +SinkToStoragePtr StorageHive::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /* metadata_snapshot*/, ContextPtr /*context*/) +{ + throw Exception("Method write is not implemented for StorageHive", ErrorCodes::NOT_IMPLEMENTED); +} + +NamesAndTypesList StorageHive::getVirtuals() const +{ + return NamesAndTypesList{{"_path", std::make_shared()}, {"_file", std::make_shared()}}; +} + +void registerStorageHive(StorageFactory & factory) +{ + factory.registerStorage( + "Hive", + [](const StorageFactory::Arguments & args) + { + bool have_settings = args.storage_def->settings; + std::unique_ptr hive_settings = std::make_unique(); + if (have_settings) + hive_settings->loadFromQuery(*args.storage_def); + + ASTs & engine_args = args.engine_args; + if (engine_args.size() != 3) + throw Exception( + "Storage Hive requires 3 arguments: hive metastore address, hive database and hive table", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + auto * partition_by = args.storage_def->partition_by; + if (!partition_by) + throw Exception("Storage Hive requires partition by clause", ErrorCodes::BAD_ARGUMENTS); + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext()); + + const String & hms_url = engine_args[0]->as().value.safeGet(); + const String & hive_database = engine_args[1]->as().value.safeGet(); + const String & hive_table = engine_args[2]->as().value.safeGet(); + return StorageHive::create( + hms_url, + hive_database, + hive_table, + args.table_id, + args.columns, + args.constraints, + args.comment, + partition_by ? partition_by->ptr() : nullptr, + std::move(hive_settings), + args.getContext()); + }, + StorageFactory::StorageFeatures{ + .supports_settings = true, + .supports_sort_order = true, + }); +} + +} +#endif diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h new file mode 100644 index 00000000000..c289d29161a --- /dev/null +++ b/src/Storages/Hive/StorageHive.h @@ -0,0 +1,152 @@ +#pragma once + +#if !defined(ARCADIA_BUILD) +#include +#endif + +#if USE_HDFS + +#include +#include +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class HiveSettings; +/** + * This class represents table engine for external hdfs files. + * Read method is supported for now. + */ +class StorageHive final : public shared_ptr_helper, public IStorage, WithContext +{ + friend struct shared_ptr_helper; + +public: + enum class FileFormat + { + RC_FILE, + TEXT, + LZO_TEXT, + SEQUENCE_FILE, + AVRO, + PARQUET, + ORC, + }; + + // TODO: json support + inline static const String RCFILE_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.RCFileInputFormat"; + inline static const String TEXT_INPUT_FORMAT = "org.apache.hadoop.mapred.TextInputFormat"; + inline static const String LZO_TEXT_INPUT_FORMAT = "com.hadoop.mapred.DeprecatedLzoTextInputFormat"; + inline static const String SEQUENCE_INPUT_FORMAT = "org.apache.hadoop.mapred.SequenceFileInputFormat"; + inline static const String PARQUET_INPUT_FORMAT = "com.cloudera.impala.hive.serde.ParquetInputFormat"; + inline static const String MR_PARQUET_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"; + inline static const String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"; + inline static const String ORC_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"; + inline static const std::map VALID_HDFS_FORMATS = { + {RCFILE_INPUT_FORMAT, FileFormat::RC_FILE}, + {TEXT_INPUT_FORMAT, FileFormat::TEXT}, + {LZO_TEXT_INPUT_FORMAT, FileFormat::LZO_TEXT}, + {SEQUENCE_INPUT_FORMAT, FileFormat::SEQUENCE_FILE}, + {PARQUET_INPUT_FORMAT, FileFormat::PARQUET}, + {MR_PARQUET_INPUT_FORMAT, FileFormat::PARQUET}, + {AVRO_INPUT_FORMAT, FileFormat::AVRO}, + {ORC_INPUT_FORMAT, FileFormat::ORC}, + }; + + static inline bool isFormatClass(const String & format_class) { return VALID_HDFS_FORMATS.count(format_class) > 0; } + static inline FileFormat toFileFormat(const String & format_class) + { + if (isFormatClass(format_class)) + { + return VALID_HDFS_FORMATS.find(format_class)->second; + } + throw Exception("Unsupported hdfs file format " + format_class, ErrorCodes::NOT_IMPLEMENTED); + } + + + String getName() const override { return "Hive"; } + + bool supportsIndexForIn() const override { return true; } + bool mayBenefitFromIndexForIn( + const ASTPtr & /* left_in_operand */, + ContextPtr /* query_context */, + const StorageMetadataPtr & /* metadata_snapshot */) const override + { + return false; + } + + + Pipe read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) override; + + NamesAndTypesList getVirtuals() const override; + +protected: + StorageHive( + const String & hms_url_, + const String & hive_database_, + const String & hive_table_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment_, + const ASTPtr & partition_by_ast_, + std::unique_ptr storage_settings_, + ContextPtr context_); + + void initMinMaxIndexExpression(); + static ASTPtr extractKeyExpressionList(const ASTPtr & node); + +private: + // hive metastore url + String hms_url; + + // hive database and table + String hive_database; + String hive_table; + + // hive table meta + std::vector table_schema; + Names text_input_field_names; // Defines schema of hive file, only used when text input format is TEXT + + // hdfs relative information + String hdfs_namenode_url; + + String format_name; + String compression_method; + + const ASTPtr partition_by_ast; + NamesAndTypesList partition_name_types; + ExpressionActionsPtr partition_key_expr; + ExpressionActionsPtr partition_minmax_idx_expr; + + NamesAndTypesList hivefile_name_types; + ExpressionActionsPtr hivefile_minmax_idx_expr; + + std::shared_ptr storage_settings; + + Poco::Logger * log = &Poco::Logger::get("StorageHive"); +}; +} + +#endif From bdca69f3544b425b8c218ef2fb76fe06b0f5fd55 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 6 Nov 2021 10:52:53 +0800 Subject: [PATCH 0016/1260] add binary thrift-compiler to generate code from hive_metastore.thrift --- contrib/arrow-cmake/CMakeLists.txt | 6 +++--- contrib/hive-metastore/CMakeLists.txt | 5 +++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 9b4428afb18..c2ce24b5916 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -50,10 +50,10 @@ target_link_libraries (${THRIFT_LIBRARY} PRIVATE boost::headers_only) set(COMPILER_DIR "${ClickHouse_SOURCE_DIR}/contrib/thrift/compiler/cpp") configure_file(${COMPILER_DIR}/src/thrift/version.h.in ${COMPILER_DIR}/src/thrift/version.h) -set(thrift-compiler_SOURCES +set(thrift_compiler_SOURCES ${COMPILER_DIR}/src/thrift/main.cc ${COMPILER_DIR}/src/thrift/audit/t_audit.cpp - ${COMPILER_DIR}/src/thrift/generate/t_cpp_generator.cpp + ${COMPILER_DIR}/src/thrift/generate/t_cpp_generator.cc ${COMPILER_DIR}/src/thrift/common.cc ${COMPILER_DIR}/src/thrift/generate/t_generator.cc ${COMPILER_DIR}/src/thrift/parse/t_typedef.cc @@ -63,7 +63,7 @@ set(thrift-compiler_SOURCES ${COMPILER_DIR}/src/thrift/thriftl.cc ${COMPILER_DIR}/src/thrift/thrifty.hh ) -add_executable(thrift-compiler ${thrift-compiler_SOURCES}) +add_executable(thrift-compiler ${thrift_compiler_SOURCES}) include_directories(thrift-compiler SYSTEM ${COMPILER_DIR}/src) # === orc diff --git a/contrib/hive-metastore/CMakeLists.txt b/contrib/hive-metastore/CMakeLists.txt index 6e6a74758eb..54aecc584fb 100644 --- a/contrib/hive-metastore/CMakeLists.txt +++ b/contrib/hive-metastore/CMakeLists.txt @@ -4,15 +4,16 @@ set (SRCS ${CMAKE_CURRENT_BINARY_DIR}/ThriftHiveMetastore.cpp ) +set(THRIFT_COMPILER ${ClickHouse_BINARY_DIR}/contrib/arrow-cmake/thrift-compiler) add_custom_command( OUTPUT ${SRCS} - COMMAND thrift + COMMAND thrift-compiler ARGS -r --gen cpp -out ${CMAKE_CURRENT_BINARY_DIR} ${CMAKE_CURRENT_LIST_DIR}/hive_metastore.thrift COMMENT "Running cpp thrift compiler for hive_metastore.thrift" VERBATIM) add_library(hivemetastore ${SRCS}) -add_dependencies(hivemetastore ${THRIFT_LIBRARY}) +add_dependencies(hivemetastore ${THRIFT_LIBRARY} thrift-compiler) target_link_libraries(hivemetastore PUBLIC ${THRIFT_LIBRARY}) target_include_directories(hivemetastore SYSTEM PUBLIC ${CMAKE_CURRENT_BINARY_DIR}) From dd98fc74b699d4f40b8776af4df1cba9000f16af Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 6 Nov 2021 11:01:58 +0800 Subject: [PATCH 0017/1260] fix typos --- src/Storages/Hive/HiveCommon.cpp | 4 ++-- src/Storages/Hive/StorageHive.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 9be89f5c78d..b94e98e6356 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -23,7 +23,7 @@ std::shared_ptr HMSClient::getTableMeta(const std::str client->get_table(*table, db_name, table_name); /** - * query the lastest partition info to check new change + * query the latest partition info to check new change */ client->get_partitions(partitions, db_name, table_name, -1); } @@ -54,7 +54,7 @@ std::shared_ptr HMSClient::getTableMeta(const std::str auto & pinfo = partition_infos[partition.sd.location]; pinfo.partition = partition; - // query files under the patition by hdfs api is costly, we reuse the files in case the partion has no change + // query files under the partition by hdfs api is costly, we reuse the files in case the partition has no change if (result) { auto it = old_partition_infos.find(partition.sd.location); diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 265a25d7566..8f23b036628 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -432,7 +432,7 @@ Pipe StorageHive::read( } } - // Load sub-file level minmax index and appy + // Load sub-file level minmax index and apply std::set skip_splits; if (file->hasSubMinMaxIndex()) { From fa95571164ad642b8b601e1e5190b3911b80bdee Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 6 Nov 2021 11:19:30 +0800 Subject: [PATCH 0018/1260] fix build error --- contrib/hive-metastore/CMakeLists.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/contrib/hive-metastore/CMakeLists.txt b/contrib/hive-metastore/CMakeLists.txt index 54aecc584fb..0b2e32c2442 100644 --- a/contrib/hive-metastore/CMakeLists.txt +++ b/contrib/hive-metastore/CMakeLists.txt @@ -7,13 +7,13 @@ set (SRCS set(THRIFT_COMPILER ${ClickHouse_BINARY_DIR}/contrib/arrow-cmake/thrift-compiler) add_custom_command( OUTPUT ${SRCS} - COMMAND thrift-compiler + COMMAND ${THRIFT_COMPILER} ARGS -r --gen cpp -out ${CMAKE_CURRENT_BINARY_DIR} ${CMAKE_CURRENT_LIST_DIR}/hive_metastore.thrift COMMENT "Running cpp thrift compiler for hive_metastore.thrift" + DEPENDS thrift-compiler VERBATIM) add_library(hivemetastore ${SRCS}) -add_dependencies(hivemetastore ${THRIFT_LIBRARY} thrift-compiler) +add_dependencies(hivemetastore ${THRIFT_LIBRARY}) target_link_libraries(hivemetastore PUBLIC ${THRIFT_LIBRARY}) target_include_directories(hivemetastore SYSTEM PUBLIC ${CMAKE_CURRENT_BINARY_DIR}) - From 1b9f0dd5e1184cd2b5611aead4b1ab05fd5a2c41 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 6 Nov 2021 11:31:01 +0800 Subject: [PATCH 0019/1260] fix build error --- contrib/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index ca537f4d4f7..6aa6e9b4d25 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -37,7 +37,6 @@ add_subdirectory (murmurhash) add_subdirectory (replxx-cmake) add_subdirectory (unixodbc-cmake) add_subdirectory (nanodbc-cmake) -add_subdirectory (hive-metastore) if (USE_INTERNAL_CAPNP_LIBRARY AND NOT MISSING_INTERNAL_CAPNP_LIBRARY) add_subdirectory(capnproto-cmake) @@ -183,6 +182,7 @@ if (USE_INTERNAL_PARQUET_LIBRARY) # {BUILD_DIR}/contrib/arrow/cpp/build/release/parquet-reader some_file.parquet add_subdirectory(arrow-cmake) + add_subdirectory (hive-metastore) # The library is large - avoid bloat. target_compile_options (${ARROW_LIBRARY} PRIVATE -g0) From dd78bbf35c60525bb1418f1ef466e5e8925c8df2 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 6 Nov 2021 16:34:05 +0800 Subject: [PATCH 0020/1260] fix build error --- contrib/hive-metastore/CMakeLists.txt | 3 ++- src/CMakeLists.txt | 3 +++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/contrib/hive-metastore/CMakeLists.txt b/contrib/hive-metastore/CMakeLists.txt index 0b2e32c2442..973730c9d32 100644 --- a/contrib/hive-metastore/CMakeLists.txt +++ b/contrib/hive-metastore/CMakeLists.txt @@ -16,4 +16,5 @@ add_custom_command( add_library(hivemetastore ${SRCS}) add_dependencies(hivemetastore ${THRIFT_LIBRARY}) target_link_libraries(hivemetastore PUBLIC ${THRIFT_LIBRARY}) -target_include_directories(hivemetastore SYSTEM PUBLIC ${CMAKE_CURRENT_BINARY_DIR}) +# target_include_directories(hivemetastore SYSTEM PUBLIC ${CMAKE_CURRENT_BINARY_DIR}) +include_directories(SYSTEM ${CMAKE_CURRENT_BINARY_DIR}) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 3696b91108b..3ced029ad16 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -445,6 +445,9 @@ if (USE_HDFS) dbms_target_link_libraries(PRIVATE ${HDFS3_LIBRARY}) dbms_target_include_directories (SYSTEM BEFORE PUBLIC ${HDFS3_INCLUDE_DIR}) + foreach (module ${all_modules}) + target_include_directories (${module} hivemetastore) + endforeach () dbms_target_link_libraries(PRIVATE hivemetastore ${THRIFT_LIBRARY}) endif() From ed942a58f3555679f3ee201bfb55b47f52e74bc1 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sun, 7 Nov 2021 17:44:31 +0800 Subject: [PATCH 0021/1260] fix build error --- src/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 3ced029ad16..1e3245e12b3 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -446,7 +446,7 @@ if (USE_HDFS) dbms_target_include_directories (SYSTEM BEFORE PUBLIC ${HDFS3_INCLUDE_DIR}) foreach (module ${all_modules}) - target_include_directories (${module} hivemetastore) + target_include_directories (${module} ${ClickHouse_BINARY_DIR}/contrib/hive-metastore) endforeach () dbms_target_link_libraries(PRIVATE hivemetastore ${THRIFT_LIBRARY}) endif() From 290f5a0c61beabcbd415eb1e03519f397d694de6 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 8 Nov 2021 10:52:06 +0800 Subject: [PATCH 0022/1260] fix build --- src/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 1e3245e12b3..71ede3e6b2c 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -446,7 +446,7 @@ if (USE_HDFS) dbms_target_include_directories (SYSTEM BEFORE PUBLIC ${HDFS3_INCLUDE_DIR}) foreach (module ${all_modules}) - target_include_directories (${module} ${ClickHouse_BINARY_DIR}/contrib/hive-metastore) + target_include_directories (${module} SYSTEM BEFORE PUBLIC ${ClickHouse_BINARY_DIR}/contrib/hive-metastore) endforeach () dbms_target_link_libraries(PRIVATE hivemetastore ${THRIFT_LIBRARY}) endif() From 9e20775968ad3407323d704c8b1a68039a99da2d Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 8 Nov 2021 15:37:48 +0800 Subject: [PATCH 0023/1260] add comment --- CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index ede70b09d94..c79762736ff 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -564,7 +564,7 @@ include (cmake/find/poco.cmake) include (cmake/find/curl.cmake) include (cmake/find/s3.cmake) include (cmake/find/base64.cmake) -include (cmake/find/parquet.cmake) +include (cmake/find/parquet.cmake) # uses protobuf include (cmake/find/simdjson.cmake) include (cmake/find/fast_float.cmake) include (cmake/find/rapidjson.cmake) From cf47449e16eb82c746b27e7ba62ac203ca41eca0 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 8 Nov 2021 18:21:49 +0800 Subject: [PATCH 0024/1260] fix build errror --- CMakeLists.txt | 1 + contrib/CMakeLists.txt | 8 +++- contrib/arrow-cmake/CMakeLists.txt | 65 --------------------------- contrib/hive-metastore/CMakeLists.txt | 10 ++--- src/CMakeLists.txt | 8 ++-- src/Interpreters/Context.cpp | 21 +++++---- src/Interpreters/Context.h | 14 +++--- 7 files changed, 36 insertions(+), 91 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index c79762736ff..c4cc95bdac5 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -556,6 +556,7 @@ include (cmake/find/h3.cmake) include (cmake/find/libxml2.cmake) include (cmake/find/brotli.cmake) include (cmake/find/protobuf.cmake) +include (cmake/find/thrift.cmake) include (cmake/find/grpc.cmake) include (cmake/find/pdqsort.cmake) include (cmake/find/miniselect.cmake) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 6aa6e9b4d25..3d1296f01ed 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -182,11 +182,9 @@ if (USE_INTERNAL_PARQUET_LIBRARY) # {BUILD_DIR}/contrib/arrow/cpp/build/release/parquet-reader some_file.parquet add_subdirectory(arrow-cmake) - add_subdirectory (hive-metastore) # The library is large - avoid bloat. target_compile_options (${ARROW_LIBRARY} PRIVATE -g0) - target_compile_options (${THRIFT_LIBRARY} PRIVATE -g0) target_compile_options (${PARQUET_LIBRARY} PRIVATE -g0) endif() @@ -246,6 +244,10 @@ if (USE_INTERNAL_PROTOBUF_LIBRARY) add_subdirectory(protobuf-cmake) endif () +if (USE_INTERNAL_THRIFT_LIBRARY) + add_subdirectory(thrift-cmake) +endif () + if (USE_INTERNAL_HDFS3_LIBRARY) add_subdirectory(libhdfs3-cmake) endif () @@ -347,6 +349,8 @@ if (USE_S2_GEOMETRY) add_subdirectory(s2geometry-cmake) endif() +add_subdirectory (hive-metastore) + # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear # in "contrib/..." as originally planned, so we workaround this by fixing FOLDER properties of all targets manually, diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index c2ce24b5916..e3ecfbd31dd 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -1,73 +1,8 @@ set (CMAKE_CXX_STANDARD 17) -# === thrift -set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp") -# contrib/thrift/lib/cpp/CMakeLists.txt -set(thriftcpp_SOURCES - "${LIBRARY_DIR}/src/thrift/TApplicationException.cpp" - "${LIBRARY_DIR}/src/thrift/TOutput.cpp" - "${LIBRARY_DIR}/src/thrift/async/TAsyncChannel.cpp" - "${LIBRARY_DIR}/src/thrift/async/TAsyncProtocolProcessor.cpp" - "${LIBRARY_DIR}/src/thrift/async/TConcurrentClientSyncInfo.h" - "${LIBRARY_DIR}/src/thrift/async/TConcurrentClientSyncInfo.cpp" - "${LIBRARY_DIR}/src/thrift/concurrency/ThreadManager.cpp" - "${LIBRARY_DIR}/src/thrift/concurrency/TimerManager.cpp" - "${LIBRARY_DIR}/src/thrift/concurrency/Util.cpp" - "${LIBRARY_DIR}/src/thrift/processor/PeekProcessor.cpp" - "${LIBRARY_DIR}/src/thrift/protocol/TBase64Utils.cpp" - "${LIBRARY_DIR}/src/thrift/protocol/TDebugProtocol.cpp" - "${LIBRARY_DIR}/src/thrift/protocol/TJSONProtocol.cpp" - "${LIBRARY_DIR}/src/thrift/protocol/TMultiplexedProtocol.cpp" - "${LIBRARY_DIR}/src/thrift/protocol/TProtocol.cpp" - "${LIBRARY_DIR}/src/thrift/transport/TTransportException.cpp" - "${LIBRARY_DIR}/src/thrift/transport/TFDTransport.cpp" - "${LIBRARY_DIR}/src/thrift/transport/TSimpleFileTransport.cpp" - "${LIBRARY_DIR}/src/thrift/transport/THttpTransport.cpp" - "${LIBRARY_DIR}/src/thrift/transport/THttpClient.cpp" - "${LIBRARY_DIR}/src/thrift/transport/THttpServer.cpp" - "${LIBRARY_DIR}/src/thrift/transport/TSocket.cpp" - "${LIBRARY_DIR}/src/thrift/transport/TSocketPool.cpp" - "${LIBRARY_DIR}/src/thrift/transport/TServerSocket.cpp" - "${LIBRARY_DIR}/src/thrift/transport/TTransportUtils.cpp" - "${LIBRARY_DIR}/src/thrift/transport/TBufferTransports.cpp" - "${LIBRARY_DIR}/src/thrift/server/TConnectedClient.cpp" - "${LIBRARY_DIR}/src/thrift/server/TServerFramework.cpp" - "${LIBRARY_DIR}/src/thrift/server/TSimpleServer.cpp" - "${LIBRARY_DIR}/src/thrift/server/TThreadPoolServer.cpp" - "${LIBRARY_DIR}/src/thrift/server/TThreadedServer.cpp" - ) -set(thriftcpp_threads_SOURCES - "${LIBRARY_DIR}/src/thrift/concurrency/ThreadFactory.cpp" - "${LIBRARY_DIR}/src/thrift/concurrency/Thread.cpp" - "${LIBRARY_DIR}/src/thrift/concurrency/Monitor.cpp" - "${LIBRARY_DIR}/src/thrift/concurrency/Mutex.cpp" - ) -add_library(${THRIFT_LIBRARY} ${thriftcpp_SOURCES} ${thriftcpp_threads_SOURCES}) -set_target_properties(${THRIFT_LIBRARY} PROPERTIES CXX_STANDARD 14) # REMOVE after https://github.com/apache/thrift/pull/1641 -target_include_directories(${THRIFT_LIBRARY} SYSTEM PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src") -target_link_libraries (${THRIFT_LIBRARY} PRIVATE boost::headers_only) - -set(COMPILER_DIR "${ClickHouse_SOURCE_DIR}/contrib/thrift/compiler/cpp") -configure_file(${COMPILER_DIR}/src/thrift/version.h.in ${COMPILER_DIR}/src/thrift/version.h) -set(thrift_compiler_SOURCES - ${COMPILER_DIR}/src/thrift/main.cc - ${COMPILER_DIR}/src/thrift/audit/t_audit.cpp - ${COMPILER_DIR}/src/thrift/generate/t_cpp_generator.cc - ${COMPILER_DIR}/src/thrift/common.cc - ${COMPILER_DIR}/src/thrift/generate/t_generator.cc - ${COMPILER_DIR}/src/thrift/parse/t_typedef.cc - ${COMPILER_DIR}/src/thrift/parse/parse.cc - ${COMPILER_DIR}/src/thrift/version.h - ${COMPILER_DIR}/src/thrift/thrifty.cc - ${COMPILER_DIR}/src/thrift/thriftl.cc - ${COMPILER_DIR}/src/thrift/thrifty.hh -) -add_executable(thrift-compiler ${thrift_compiler_SOURCES}) -include_directories(thrift-compiler SYSTEM ${COMPILER_DIR}/src) # === orc - set(ORC_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/orc/c++") set(ORC_INCLUDE_DIR "${ORC_SOURCE_DIR}/include") set(ORC_SOURCE_SRC_DIR "${ORC_SOURCE_DIR}/src") diff --git a/contrib/hive-metastore/CMakeLists.txt b/contrib/hive-metastore/CMakeLists.txt index 973730c9d32..591b7025b87 100644 --- a/contrib/hive-metastore/CMakeLists.txt +++ b/contrib/hive-metastore/CMakeLists.txt @@ -4,17 +4,15 @@ set (SRCS ${CMAKE_CURRENT_BINARY_DIR}/ThriftHiveMetastore.cpp ) -set(THRIFT_COMPILER ${ClickHouse_BINARY_DIR}/contrib/arrow-cmake/thrift-compiler) +set(THRIFT_COMPILER_PATH ${ClickHouse_BINARY_DIR}/contrib/thrift-cmake/thrift-compiler) add_custom_command( OUTPUT ${SRCS} - COMMAND ${THRIFT_COMPILER} + COMMAND ${THRIFT_COMPILER_PATH} ARGS -r --gen cpp -out ${CMAKE_CURRENT_BINARY_DIR} ${CMAKE_CURRENT_LIST_DIR}/hive_metastore.thrift COMMENT "Running cpp thrift compiler for hive_metastore.thrift" - DEPENDS thrift-compiler + DEPENDS ${THRIFT_COMPILER} VERBATIM) add_library(hivemetastore ${SRCS}) -add_dependencies(hivemetastore ${THRIFT_LIBRARY}) target_link_libraries(hivemetastore PUBLIC ${THRIFT_LIBRARY}) -# target_include_directories(hivemetastore SYSTEM PUBLIC ${CMAKE_CURRENT_BINARY_DIR}) -include_directories(SYSTEM ${CMAKE_CURRENT_BINARY_DIR}) +target_include_directories(hivemetastore SYSTEM PUBLIC ${CMAKE_CURRENT_BINARY_DIR}) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 71ede3e6b2c..eb8d8ebd3ca 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -445,10 +445,10 @@ if (USE_HDFS) dbms_target_link_libraries(PRIVATE ${HDFS3_LIBRARY}) dbms_target_include_directories (SYSTEM BEFORE PUBLIC ${HDFS3_INCLUDE_DIR}) - foreach (module ${all_modules}) - target_include_directories (${module} SYSTEM BEFORE PUBLIC ${ClickHouse_BINARY_DIR}/contrib/hive-metastore) - endforeach () - dbms_target_link_libraries(PRIVATE hivemetastore ${THRIFT_LIBRARY}) + # foreach (module ${all_modules}) + # target_include_directories (${module} SYSTEM BEFORE PUBLIC ${ClickHouse_BINARY_DIR}/contrib/hive-metastore) + # endforeach () + dbms_target_link_libraries(PRIVATE hivemetastore) endif() if (USE_AWS_S3) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 8b2a6144708..a580c6dc923 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include #include @@ -27,7 +26,6 @@ #include #include #include -#include #include #include #include @@ -86,14 +84,18 @@ #include #include #include -#include #include #include +#include + +#if USE_HDFS +#include #include #include #include -#include - +#include +#include +#endif namespace fs = std::filesystem; @@ -161,10 +163,10 @@ struct ContextSharedPart mutable std::mutex storage_policies_mutex; /// Separate mutex for re-initialization of zookeeper session. This operation could take a long time and must not interfere with another operations. mutable std::mutex zookeeper_mutex; +#if USE_HDFS /// Separate mutex for re-initialization of hive metastore client. This operation could take a long time and must not interfere with another operations. mutable std::mutex hive_metastore_mutex; - /// Separate mutex for re-initialization of hdfs file system. This operation could take a long time and must not interfere with another operations. - mutable std::mutex hdfs_filesystem_mutex; +#endif mutable zkutil::ZooKeeperPtr zookeeper; /// Client for ZooKeeper. ConfigurationPtr zookeeper_config; /// Stores zookeeper configs @@ -177,8 +179,9 @@ struct ContextSharedPart mutable std::map auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients. ConfigurationPtr auxiliary_zookeepers_config; /// Stores auxiliary zookeepers configs +#if USE_HDFS mutable std::map hive_metastore_clients; /// Map for hive metastore clients - // mutable std::map hdfs_filesystems; /// Map for hdfs file systems. +#endif String interserver_io_host; /// The host name by which this server is available for other servers. UInt16 interserver_io_port = 0; /// and port. @@ -1821,6 +1824,7 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const return shared->zookeeper; } +#if USE_HDFS HMSClientPtr Context::getHMSClient(const String & name) const { using namespace apache::thrift; @@ -1868,6 +1872,7 @@ HMSClientPtr Context::getHMSClient(const String & name) const } return it->second; } +#endif namespace { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 448efecc295..18421bf997c 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -18,6 +18,10 @@ #include "config_core.h" +#if !defined(ARCADIA_BUILD) +#include +#endif + #include #include #include @@ -27,8 +31,6 @@ namespace Poco::Net { class IPAddress; } namespace zkutil { class ZooKeeper; } -namespace hdfs { class FileSystem; } -// namespace Apache { namespace Hadoop { namespace Hive { class ThriftHiveMetastoreClient; }}} namespace DB { @@ -148,9 +150,6 @@ using InputBlocksReader = std::function; /// Used in distributed task processing using ReadTaskCallback = std::function; -// class HDFSFSPtr; -// using HDFSFileSystemPtr = std::shared_ptr; - /// An empty interface for an arbitrary object that may be attached by a shared pointer /// to query context, when using ClickHouse as a library. struct IHostContext @@ -178,8 +177,10 @@ private: std::unique_ptr shared; }; +#if USE_HDFS class HMSClient; using HMSClientPtr = std::shared_ptr; +#endif /** A set of known objects that can be used in the query. * Consists of a shared part (always common to all sessions and queries) @@ -695,8 +696,9 @@ public: // Reload Zookeeper void reloadZooKeeperIfChanged(const ConfigurationPtr & config) const; +#if USE_HDFS HMSClientPtr getHMSClient(const String & name) const; - // HDFSFileSystemPtr getHDFSFileSystem(const String & name) const; +#endif void setSystemZooKeeperLogAfterInitializationIfNeeded(); From bb0feaa191ed1fc8f40b636675284232affa33d7 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 8 Nov 2021 18:40:41 +0800 Subject: [PATCH 0025/1260] remove defined(ARCADIA_BUILD) --- src/Interpreters/Context.h | 5 +---- src/Storages/Hive/HiveCommon.h | 2 -- src/Storages/Hive/HiveSettings.h | 2 -- src/Storages/Hive/StorageHive.h | 2 -- 4 files changed, 1 insertion(+), 10 deletions(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 18421bf997c..47f35b8acf0 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -14,14 +14,11 @@ #include #include #include +#include #include #include "config_core.h" -#if !defined(ARCADIA_BUILD) -#include -#endif - #include #include #include diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index 647bbde8f89..0a8fba5c07c 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -1,8 +1,6 @@ #pragma once -#if !defined(ARCADIA_BUILD) #include -#endif #if USE_HDFS #include diff --git a/src/Storages/Hive/HiveSettings.h b/src/Storages/Hive/HiveSettings.h index 8bd7c85a855..60505ee6f11 100644 --- a/src/Storages/Hive/HiveSettings.h +++ b/src/Storages/Hive/HiveSettings.h @@ -1,8 +1,6 @@ #pragma once -#if !defined(ARCADIA_BUILD) #include -#endif #if USE_HDFS #include diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index c289d29161a..b94153fd4c3 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -1,8 +1,6 @@ #pragma once -#if !defined(ARCADIA_BUILD) #include -#endif #if USE_HDFS From cd28324e1c34a4e01633457a905037d5b5f93a29 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 8 Nov 2021 18:41:30 +0800 Subject: [PATCH 0026/1260] add missing files --- cmake/find/thrift.cmake | 23 +++++++++++ contrib/thrift-cmake/CMakeLists.txt | 64 +++++++++++++++++++++++++++++ 2 files changed, 87 insertions(+) create mode 100644 cmake/find/thrift.cmake create mode 100644 contrib/thrift-cmake/CMakeLists.txt diff --git a/cmake/find/thrift.cmake b/cmake/find/thrift.cmake new file mode 100644 index 00000000000..92be0f62005 --- /dev/null +++ b/cmake/find/thrift.cmake @@ -0,0 +1,23 @@ +option(ENABLE_THRIFT "Enable Thrift" ON) +option(USE_INTERNAL_THRIFT_LIBRARY "Set to FALSE to use system thrift library instead of bundled" ${NOT_UNBUNDLED}) + +if(ENABLE_THRIFT) + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/thrift") + message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") + set (MISSING_THRIFT 1) + endif() + + if (USE_INTERNAL_THRIFT_LIBRARY AND NOT MISSING_THRIFT) + if (MAKE_STATIC_LIBRARIES) + set(THRIFT_LIBRARY thrift_static) + else() + set(THRIFT_LIBRARY thrift) + endif() + set (THRIFT_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src") + set (THRIFT_COMPILER "thrift-compiler") + else() + find_library(THRIFT_LIBRARY thrift) + endif () +endif() + +message (STATUS "Using_THRIFT=${ENABLE_THRIFT}: ${THRIFT_INCLUDE_DIR} : ${THRIFT_LIBRARY}") diff --git a/contrib/thrift-cmake/CMakeLists.txt b/contrib/thrift-cmake/CMakeLists.txt new file mode 100644 index 00000000000..1a4e5bba898 --- /dev/null +++ b/contrib/thrift-cmake/CMakeLists.txt @@ -0,0 +1,64 @@ +# === thrift +set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp") +set(thriftcpp_SOURCES + "${LIBRARY_DIR}/src/thrift/TApplicationException.cpp" + "${LIBRARY_DIR}/src/thrift/TOutput.cpp" + "${LIBRARY_DIR}/src/thrift/async/TAsyncChannel.cpp" + "${LIBRARY_DIR}/src/thrift/async/TAsyncProtocolProcessor.cpp" + "${LIBRARY_DIR}/src/thrift/async/TConcurrentClientSyncInfo.h" + "${LIBRARY_DIR}/src/thrift/async/TConcurrentClientSyncInfo.cpp" + "${LIBRARY_DIR}/src/thrift/concurrency/ThreadManager.cpp" + "${LIBRARY_DIR}/src/thrift/concurrency/TimerManager.cpp" + "${LIBRARY_DIR}/src/thrift/concurrency/Util.cpp" + "${LIBRARY_DIR}/src/thrift/processor/PeekProcessor.cpp" + "${LIBRARY_DIR}/src/thrift/protocol/TBase64Utils.cpp" + "${LIBRARY_DIR}/src/thrift/protocol/TDebugProtocol.cpp" + "${LIBRARY_DIR}/src/thrift/protocol/TJSONProtocol.cpp" + "${LIBRARY_DIR}/src/thrift/protocol/TMultiplexedProtocol.cpp" + "${LIBRARY_DIR}/src/thrift/protocol/TProtocol.cpp" + "${LIBRARY_DIR}/src/thrift/transport/TTransportException.cpp" + "${LIBRARY_DIR}/src/thrift/transport/TFDTransport.cpp" + "${LIBRARY_DIR}/src/thrift/transport/TSimpleFileTransport.cpp" + "${LIBRARY_DIR}/src/thrift/transport/THttpTransport.cpp" + "${LIBRARY_DIR}/src/thrift/transport/THttpClient.cpp" + "${LIBRARY_DIR}/src/thrift/transport/THttpServer.cpp" + "${LIBRARY_DIR}/src/thrift/transport/TSocket.cpp" + "${LIBRARY_DIR}/src/thrift/transport/TSocketPool.cpp" + "${LIBRARY_DIR}/src/thrift/transport/TServerSocket.cpp" + "${LIBRARY_DIR}/src/thrift/transport/TTransportUtils.cpp" + "${LIBRARY_DIR}/src/thrift/transport/TBufferTransports.cpp" + "${LIBRARY_DIR}/src/thrift/server/TConnectedClient.cpp" + "${LIBRARY_DIR}/src/thrift/server/TServerFramework.cpp" + "${LIBRARY_DIR}/src/thrift/server/TSimpleServer.cpp" + "${LIBRARY_DIR}/src/thrift/server/TThreadPoolServer.cpp" + "${LIBRARY_DIR}/src/thrift/server/TThreadedServer.cpp" + ) +set(thriftcpp_threads_SOURCES + "${LIBRARY_DIR}/src/thrift/concurrency/ThreadFactory.cpp" + "${LIBRARY_DIR}/src/thrift/concurrency/Thread.cpp" + "${LIBRARY_DIR}/src/thrift/concurrency/Monitor.cpp" + "${LIBRARY_DIR}/src/thrift/concurrency/Mutex.cpp" + ) +add_library(${THRIFT_LIBRARY} ${thriftcpp_SOURCES} ${thriftcpp_threads_SOURCES}) +# set_target_properties(${THRIFT_LIBRARY} PROPERTIES CXX_STANDARD 14) # REMOVE after https://github.com/apache/thrift/pull/1641 +# target_include_directories(${THRIFT_LIBRARY} SYSTEM PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src") +target_include_directories(${THRIFT_LIBRARY} SYSTEM PUBLIC "${THRIFT_INCLUDE_DIR}") +target_link_libraries (${THRIFT_LIBRARY} PRIVATE boost::headers_only) + +set(COMPILER_DIR "${ClickHouse_SOURCE_DIR}/contrib/thrift/compiler/cpp") +configure_file(${COMPILER_DIR}/src/thrift/version.h.in ${COMPILER_DIR}/src/thrift/version.h) +set(thrift_compiler_SOURCES + ${COMPILER_DIR}/src/thrift/main.cc + ${COMPILER_DIR}/src/thrift/audit/t_audit.cpp + ${COMPILER_DIR}/src/thrift/generate/t_cpp_generator.cc + ${COMPILER_DIR}/src/thrift/common.cc + ${COMPILER_DIR}/src/thrift/generate/t_generator.cc + ${COMPILER_DIR}/src/thrift/parse/t_typedef.cc + ${COMPILER_DIR}/src/thrift/parse/parse.cc + ${COMPILER_DIR}/src/thrift/version.h + ${COMPILER_DIR}/src/thrift/thrifty.cc + ${COMPILER_DIR}/src/thrift/thriftl.cc + ${COMPILER_DIR}/src/thrift/thrifty.hh +) +add_executable(${THRIFT_COMPILER} ${thrift_compiler_SOURCES}) +include_directories(${THRIFT_COMPILER} SYSTEM ${COMPILER_DIR}/src) From 65c4c61e3329328b63adba3bbf0660006fe4d2bb Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 9 Nov 2021 10:31:22 +0800 Subject: [PATCH 0027/1260] compile hive-metastore from pre-generated source files --- cmake/find/parquet.cmake | 6 +- cmake/find/thrift.cmake | 2 +- contrib/hive-metastore/CMakeLists.txt | 18 +- .../hive-metastore/ThriftHiveMetastore.cpp | 79541 ++++++++++++++++ contrib/hive-metastore/ThriftHiveMetastore.h | 25246 +++++ .../hive_metastore_constants.cpp | 65 + .../hive-metastore/hive_metastore_constants.h | 48 + .../hive-metastore/hive_metastore_types.cpp | 26465 +++++ contrib/hive-metastore/hive_metastore_types.h | 10002 ++ contrib/thrift-cmake/CMakeLists.txt | 36 +- 10 files changed, 141397 insertions(+), 32 deletions(-) create mode 100644 contrib/hive-metastore/ThriftHiveMetastore.cpp create mode 100644 contrib/hive-metastore/ThriftHiveMetastore.h create mode 100644 contrib/hive-metastore/hive_metastore_constants.cpp create mode 100644 contrib/hive-metastore/hive_metastore_constants.h create mode 100644 contrib/hive-metastore/hive_metastore_types.cpp create mode 100644 contrib/hive-metastore/hive_metastore_types.h diff --git a/cmake/find/parquet.cmake b/cmake/find/parquet.cmake index 4b56a829df5..a294ccb00c4 100644 --- a/cmake/find/parquet.cmake +++ b/cmake/find/parquet.cmake @@ -34,7 +34,7 @@ endif() if(NOT USE_INTERNAL_PARQUET_LIBRARY) find_package(Arrow) find_package(Parquet) - find_library(THRIFT_LIBRARY thrift) + # find_library(THRIFT_LIBRARY thrift) find_library(UTF8_PROC_LIBRARY utf8proc) find_package(BZip2) @@ -145,12 +145,12 @@ if(NOT EXTERNAL_PARQUET_FOUND AND NOT MISSING_INTERNAL_PARQUET_LIBRARY AND NOT O set(FLATBUFFERS_LIBRARY flatbuffers) set(ARROW_LIBRARY arrow_static) set(PARQUET_LIBRARY parquet_static) - set(THRIFT_LIBRARY thrift_static) + # set(THRIFT_LIBRARY thrift_static) else() set(FLATBUFFERS_LIBRARY flatbuffers_shared) set(ARROW_LIBRARY arrow_shared) set(PARQUET_LIBRARY parquet_shared) - set(THRIFT_LIBRARY thrift) + # set(THRIFT_LIBRARY thrift) endif() set(USE_PARQUET 1) diff --git a/cmake/find/thrift.cmake b/cmake/find/thrift.cmake index 92be0f62005..0a872e76abe 100644 --- a/cmake/find/thrift.cmake +++ b/cmake/find/thrift.cmake @@ -14,7 +14,7 @@ if(ENABLE_THRIFT) set(THRIFT_LIBRARY thrift) endif() set (THRIFT_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src") - set (THRIFT_COMPILER "thrift-compiler") + # set (THRIFT_COMPILER "thrift-compiler") else() find_library(THRIFT_LIBRARY thrift) endif () diff --git a/contrib/hive-metastore/CMakeLists.txt b/contrib/hive-metastore/CMakeLists.txt index 591b7025b87..2b1129f0618 100644 --- a/contrib/hive-metastore/CMakeLists.txt +++ b/contrib/hive-metastore/CMakeLists.txt @@ -4,15 +4,15 @@ set (SRCS ${CMAKE_CURRENT_BINARY_DIR}/ThriftHiveMetastore.cpp ) -set(THRIFT_COMPILER_PATH ${ClickHouse_BINARY_DIR}/contrib/thrift-cmake/thrift-compiler) -add_custom_command( - OUTPUT ${SRCS} - COMMAND ${THRIFT_COMPILER_PATH} - ARGS -r --gen cpp -out ${CMAKE_CURRENT_BINARY_DIR} ${CMAKE_CURRENT_LIST_DIR}/hive_metastore.thrift - COMMENT "Running cpp thrift compiler for hive_metastore.thrift" - DEPENDS ${THRIFT_COMPILER} - VERBATIM) +# set(THRIFT_COMPILER_PATH ${ClickHouse_BINARY_DIR}/contrib/thrift-cmake/thrift-compiler) +# add_custom_command( +# OUTPUT ${SRCS} +# COMMAND ${THRIFT_COMPILER_PATH} +# ARGS -r --gen cpp -out ${CMAKE_CURRENT_BINARY_DIR} ${CMAKE_CURRENT_LIST_DIR}/hive_metastore.thrift +# COMMENT "Running cpp thrift compiler for hive_metastore.thrift" +# DEPENDS ${THRIFT_COMPILER} +# VERBATIM) add_library(hivemetastore ${SRCS}) target_link_libraries(hivemetastore PUBLIC ${THRIFT_LIBRARY}) -target_include_directories(hivemetastore SYSTEM PUBLIC ${CMAKE_CURRENT_BINARY_DIR}) +target_include_directories(hivemetastore SYSTEM PUBLIC ${CMAKE_CURRENT_LIST_DIR}) diff --git a/contrib/hive-metastore/ThriftHiveMetastore.cpp b/contrib/hive-metastore/ThriftHiveMetastore.cpp new file mode 100644 index 00000000000..f6209bac7b0 --- /dev/null +++ b/contrib/hive-metastore/ThriftHiveMetastore.cpp @@ -0,0 +1,79541 @@ +/** + * Autogenerated by Thrift Compiler () + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +#include "ThriftHiveMetastore.h" + +namespace Apache { namespace Hadoop { namespace Hive { + + +ThriftHiveMetastore_getMetaConf_args::~ThriftHiveMetastore_getMetaConf_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_getMetaConf_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->key); + this->__isset.key = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_getMetaConf_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_getMetaConf_args"); + + xfer += oprot->writeFieldBegin("key", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->key); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_getMetaConf_pargs::~ThriftHiveMetastore_getMetaConf_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_getMetaConf_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_getMetaConf_pargs"); + + xfer += oprot->writeFieldBegin("key", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->key))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_getMetaConf_result::~ThriftHiveMetastore_getMetaConf_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_getMetaConf_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_getMetaConf_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_getMetaConf_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0); + xfer += oprot->writeString(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_getMetaConf_presult::~ThriftHiveMetastore_getMetaConf_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_getMetaConf_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_setMetaConf_args::~ThriftHiveMetastore_setMetaConf_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_setMetaConf_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->key); + this->__isset.key = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->value); + this->__isset.value = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_setMetaConf_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_setMetaConf_args"); + + xfer += oprot->writeFieldBegin("key", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->key); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("value", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->value); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_setMetaConf_pargs::~ThriftHiveMetastore_setMetaConf_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_setMetaConf_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_setMetaConf_pargs"); + + xfer += oprot->writeFieldBegin("key", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->key))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("value", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->value))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_setMetaConf_result::~ThriftHiveMetastore_setMetaConf_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_setMetaConf_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_setMetaConf_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_setMetaConf_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_setMetaConf_presult::~ThriftHiveMetastore_setMetaConf_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_setMetaConf_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_create_database_args::~ThriftHiveMetastore_create_database_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_database_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->database.read(iprot); + this->__isset.database = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_database_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_database_args"); + + xfer += oprot->writeFieldBegin("database", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->database.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_database_pargs::~ThriftHiveMetastore_create_database_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_database_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_database_pargs"); + + xfer += oprot->writeFieldBegin("database", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->database)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_database_result::~ThriftHiveMetastore_create_database_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_database_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_database_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_database_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_database_presult::~ThriftHiveMetastore_create_database_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_database_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_database_args::~ThriftHiveMetastore_get_database_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_database_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->name); + this->__isset.name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_database_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_database_args"); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_database_pargs::~ThriftHiveMetastore_get_database_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_database_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_database_pargs"); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_database_result::~ThriftHiveMetastore_get_database_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_database_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_database_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_database_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_database_presult::~ThriftHiveMetastore_get_database_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_database_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_drop_database_args::~ThriftHiveMetastore_drop_database_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_database_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->name); + this->__isset.name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->deleteData); + this->__isset.deleteData = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->cascade); + this->__isset.cascade = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_database_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_database_args"); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 2); + xfer += oprot->writeBool(this->deleteData); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("cascade", ::apache::thrift::protocol::T_BOOL, 3); + xfer += oprot->writeBool(this->cascade); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_database_pargs::~ThriftHiveMetastore_drop_database_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_database_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_database_pargs"); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 2); + xfer += oprot->writeBool((*(this->deleteData))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("cascade", ::apache::thrift::protocol::T_BOOL, 3); + xfer += oprot->writeBool((*(this->cascade))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_database_result::~ThriftHiveMetastore_drop_database_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_database_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_database_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_database_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_database_presult::~ThriftHiveMetastore_drop_database_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_database_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_databases_args::~ThriftHiveMetastore_get_databases_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_databases_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->pattern); + this->__isset.pattern = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_databases_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_databases_args"); + + xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->pattern); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_databases_pargs::~ThriftHiveMetastore_get_databases_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_databases_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_databases_pargs"); + + xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->pattern))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_databases_result::~ThriftHiveMetastore_get_databases_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_databases_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size982; + ::apache::thrift::protocol::TType _etype985; + xfer += iprot->readListBegin(_etype985, _size982); + this->success.resize(_size982); + uint32_t _i986; + for (_i986 = 0; _i986 < _size982; ++_i986) + { + xfer += iprot->readString(this->success[_i986]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_databases_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::vector ::const_iterator _iter987; + for (_iter987 = this->success.begin(); _iter987 != this->success.end(); ++_iter987) + { + xfer += oprot->writeString((*_iter987)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_databases_presult::~ThriftHiveMetastore_get_databases_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_databases_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size988; + ::apache::thrift::protocol::TType _etype991; + xfer += iprot->readListBegin(_etype991, _size988); + (*(this->success)).resize(_size988); + uint32_t _i992; + for (_i992 = 0; _i992 < _size988; ++_i992) + { + xfer += iprot->readString((*(this->success))[_i992]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_all_databases_args::~ThriftHiveMetastore_get_all_databases_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_databases_args::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 ThriftHiveMetastore_get_all_databases_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_databases_args"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_all_databases_pargs::~ThriftHiveMetastore_get_all_databases_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_databases_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_databases_pargs"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_all_databases_result::~ThriftHiveMetastore_get_all_databases_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_databases_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size993; + ::apache::thrift::protocol::TType _etype996; + xfer += iprot->readListBegin(_etype996, _size993); + this->success.resize(_size993); + uint32_t _i997; + for (_i997 = 0; _i997 < _size993; ++_i997) + { + xfer += iprot->readString(this->success[_i997]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_databases_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::vector ::const_iterator _iter998; + for (_iter998 = this->success.begin(); _iter998 != this->success.end(); ++_iter998) + { + xfer += oprot->writeString((*_iter998)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_all_databases_presult::~ThriftHiveMetastore_get_all_databases_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_databases_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size999; + ::apache::thrift::protocol::TType _etype1002; + xfer += iprot->readListBegin(_etype1002, _size999); + (*(this->success)).resize(_size999); + uint32_t _i1003; + for (_i1003 = 0; _i1003 < _size999; ++_i1003) + { + xfer += iprot->readString((*(this->success))[_i1003]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_alter_database_args::~ThriftHiveMetastore_alter_database_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_database_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbname); + this->__isset.dbname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->db.read(iprot); + this->__isset.db = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_database_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_database_args"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbname); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("db", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->db.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_database_pargs::~ThriftHiveMetastore_alter_database_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_database_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_database_pargs"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbname))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("db", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += (*(this->db)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_database_result::~ThriftHiveMetastore_alter_database_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_database_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_database_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_database_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_database_presult::~ThriftHiveMetastore_alter_database_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_database_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_type_args::~ThriftHiveMetastore_get_type_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_type_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->name); + this->__isset.name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_type_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_args"); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_type_pargs::~ThriftHiveMetastore_get_type_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_type_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_pargs"); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_type_result::~ThriftHiveMetastore_get_type_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_type_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_type_presult::~ThriftHiveMetastore_get_type_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_type_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_create_type_args::~ThriftHiveMetastore_create_type_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_type_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->type.read(iprot); + this->__isset.type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_type_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_type_args"); + + xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->type.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_type_pargs::~ThriftHiveMetastore_create_type_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_type_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_type_pargs"); + + xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->type)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_type_result::~ThriftHiveMetastore_create_type_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_type_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_type_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_type_presult::~ThriftHiveMetastore_create_type_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_type_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_drop_type_args::~ThriftHiveMetastore_drop_type_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_type_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->type); + this->__isset.type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_type_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_type_args"); + + xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->type); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_type_pargs::~ThriftHiveMetastore_drop_type_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_type_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_type_pargs"); + + xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->type))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_type_result::~ThriftHiveMetastore_drop_type_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_type_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_type_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_type_presult::~ThriftHiveMetastore_drop_type_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_type_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_type_all_args::~ThriftHiveMetastore_get_type_all_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_type_all_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->name); + this->__isset.name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_type_all_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_all_args"); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_type_all_pargs::~ThriftHiveMetastore_get_type_all_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_type_all_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_all_pargs"); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_type_all_result::~ThriftHiveMetastore_get_type_all_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_type_all_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->success.clear(); + uint32_t _size1004; + ::apache::thrift::protocol::TType _ktype1005; + ::apache::thrift::protocol::TType _vtype1006; + xfer += iprot->readMapBegin(_ktype1005, _vtype1006, _size1004); + uint32_t _i1008; + for (_i1008 = 0; _i1008 < _size1004; ++_i1008) + { + std::string _key1009; + xfer += iprot->readString(_key1009); + Type& _val1010 = this->success[_key1009]; + xfer += _val1010.read(iprot); + } + xfer += iprot->readMapEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_all_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::map ::const_iterator _iter1011; + for (_iter1011 = this->success.begin(); _iter1011 != this->success.end(); ++_iter1011) + { + xfer += oprot->writeString(_iter1011->first); + xfer += _iter1011->second.write(oprot); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_type_all_presult::~ThriftHiveMetastore_get_type_all_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_type_all_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + (*(this->success)).clear(); + uint32_t _size1012; + ::apache::thrift::protocol::TType _ktype1013; + ::apache::thrift::protocol::TType _vtype1014; + xfer += iprot->readMapBegin(_ktype1013, _vtype1014, _size1012); + uint32_t _i1016; + for (_i1016 = 0; _i1016 < _size1012; ++_i1016) + { + std::string _key1017; + xfer += iprot->readString(_key1017); + Type& _val1018 = (*(this->success))[_key1017]; + xfer += _val1018.read(iprot); + } + xfer += iprot->readMapEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_fields_args::~ThriftHiveMetastore_get_fields_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_fields_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->table_name); + this->__isset.table_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_fields_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->table_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_fields_pargs::~ThriftHiveMetastore_get_fields_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_fields_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->table_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_fields_result::~ThriftHiveMetastore_get_fields_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_fields_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1019; + ::apache::thrift::protocol::TType _etype1022; + xfer += iprot->readListBegin(_etype1022, _size1019); + this->success.resize(_size1019); + uint32_t _i1023; + for (_i1023 = 0; _i1023 < _size1019; ++_i1023) + { + xfer += this->success[_i1023].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter1024; + for (_iter1024 = this->success.begin(); _iter1024 != this->success.end(); ++_iter1024) + { + xfer += (*_iter1024).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_fields_presult::~ThriftHiveMetastore_get_fields_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_fields_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1025; + ::apache::thrift::protocol::TType _etype1028; + xfer += iprot->readListBegin(_etype1028, _size1025); + (*(this->success)).resize(_size1025); + uint32_t _i1029; + for (_i1029 = 0; _i1029 < _size1025; ++_i1029) + { + xfer += (*(this->success))[_i1029].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_fields_with_environment_context_args::~ThriftHiveMetastore_get_fields_with_environment_context_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_fields_with_environment_context_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->table_name); + this->__isset.table_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->environment_context.read(iprot); + this->__isset.environment_context = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_fields_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_with_environment_context_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->table_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->environment_context.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_fields_with_environment_context_pargs::~ThriftHiveMetastore_get_fields_with_environment_context_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_fields_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_with_environment_context_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->table_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += (*(this->environment_context)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_fields_with_environment_context_result::~ThriftHiveMetastore_get_fields_with_environment_context_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1030; + ::apache::thrift::protocol::TType _etype1033; + xfer += iprot->readListBegin(_etype1033, _size1030); + this->success.resize(_size1030); + uint32_t _i1034; + for (_i1034 = 0; _i1034 < _size1030; ++_i1034) + { + xfer += this->success[_i1034].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_with_environment_context_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter1035; + for (_iter1035 = this->success.begin(); _iter1035 != this->success.end(); ++_iter1035) + { + xfer += (*_iter1035).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_fields_with_environment_context_presult::~ThriftHiveMetastore_get_fields_with_environment_context_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1036; + ::apache::thrift::protocol::TType _etype1039; + xfer += iprot->readListBegin(_etype1039, _size1036); + (*(this->success)).resize(_size1036); + uint32_t _i1040; + for (_i1040 = 0; _i1040 < _size1036; ++_i1040) + { + xfer += (*(this->success))[_i1040].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_schema_args::~ThriftHiveMetastore_get_schema_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_schema_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->table_name); + this->__isset.table_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_schema_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_schema_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->table_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_schema_pargs::~ThriftHiveMetastore_get_schema_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_schema_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_schema_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->table_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_schema_result::~ThriftHiveMetastore_get_schema_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_schema_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1041; + ::apache::thrift::protocol::TType _etype1044; + xfer += iprot->readListBegin(_etype1044, _size1041); + this->success.resize(_size1041); + uint32_t _i1045; + for (_i1045 = 0; _i1045 < _size1041; ++_i1045) + { + xfer += this->success[_i1045].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_schema_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter1046; + for (_iter1046 = this->success.begin(); _iter1046 != this->success.end(); ++_iter1046) + { + xfer += (*_iter1046).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_schema_presult::~ThriftHiveMetastore_get_schema_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_schema_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1047; + ::apache::thrift::protocol::TType _etype1050; + xfer += iprot->readListBegin(_etype1050, _size1047); + (*(this->success)).resize(_size1047); + uint32_t _i1051; + for (_i1051 = 0; _i1051 < _size1047; ++_i1051) + { + xfer += (*(this->success))[_i1051].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_schema_with_environment_context_args::~ThriftHiveMetastore_get_schema_with_environment_context_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_schema_with_environment_context_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->table_name); + this->__isset.table_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->environment_context.read(iprot); + this->__isset.environment_context = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_schema_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_schema_with_environment_context_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->table_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->environment_context.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_schema_with_environment_context_pargs::~ThriftHiveMetastore_get_schema_with_environment_context_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_schema_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_schema_with_environment_context_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->table_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += (*(this->environment_context)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_schema_with_environment_context_result::~ThriftHiveMetastore_get_schema_with_environment_context_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1052; + ::apache::thrift::protocol::TType _etype1055; + xfer += iprot->readListBegin(_etype1055, _size1052); + this->success.resize(_size1052); + uint32_t _i1056; + for (_i1056 = 0; _i1056 < _size1052; ++_i1056) + { + xfer += this->success[_i1056].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_schema_with_environment_context_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter1057; + for (_iter1057 = this->success.begin(); _iter1057 != this->success.end(); ++_iter1057) + { + xfer += (*_iter1057).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_schema_with_environment_context_presult::~ThriftHiveMetastore_get_schema_with_environment_context_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1058; + ::apache::thrift::protocol::TType _etype1061; + xfer += iprot->readListBegin(_etype1061, _size1058); + (*(this->success)).resize(_size1058); + uint32_t _i1062; + for (_i1062 = 0; _i1062 < _size1058; ++_i1062) + { + xfer += (*(this->success))[_i1062].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_create_table_args::~ThriftHiveMetastore_create_table_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_table_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->tbl.read(iprot); + this->__isset.tbl = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_table_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_args"); + + xfer += oprot->writeFieldBegin("tbl", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->tbl.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_table_pargs::~ThriftHiveMetastore_create_table_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_table_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_pargs"); + + xfer += oprot->writeFieldBegin("tbl", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->tbl)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_table_result::~ThriftHiveMetastore_create_table_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_table_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_table_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_table_presult::~ThriftHiveMetastore_create_table_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_table_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_create_table_with_environment_context_args::~ThriftHiveMetastore_create_table_with_environment_context_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_table_with_environment_context_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->tbl.read(iprot); + this->__isset.tbl = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->environment_context.read(iprot); + this->__isset.environment_context = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_table_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_with_environment_context_args"); + + xfer += oprot->writeFieldBegin("tbl", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->tbl.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->environment_context.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_table_with_environment_context_pargs::~ThriftHiveMetastore_create_table_with_environment_context_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_table_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_with_environment_context_pargs"); + + xfer += oprot->writeFieldBegin("tbl", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->tbl)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += (*(this->environment_context)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_table_with_environment_context_result::~ThriftHiveMetastore_create_table_with_environment_context_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_table_with_environment_context_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_table_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_with_environment_context_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_table_with_environment_context_presult::~ThriftHiveMetastore_create_table_with_environment_context_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_table_with_environment_context_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_create_table_with_constraints_args::~ThriftHiveMetastore_create_table_with_constraints_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_table_with_constraints_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->tbl.read(iprot); + this->__isset.tbl = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->primaryKeys.clear(); + uint32_t _size1063; + ::apache::thrift::protocol::TType _etype1066; + xfer += iprot->readListBegin(_etype1066, _size1063); + this->primaryKeys.resize(_size1063); + uint32_t _i1067; + for (_i1067 = 0; _i1067 < _size1063; ++_i1067) + { + xfer += this->primaryKeys[_i1067].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.primaryKeys = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->foreignKeys.clear(); + uint32_t _size1068; + ::apache::thrift::protocol::TType _etype1071; + xfer += iprot->readListBegin(_etype1071, _size1068); + this->foreignKeys.resize(_size1068); + uint32_t _i1072; + for (_i1072 = 0; _i1072 < _size1068; ++_i1072) + { + xfer += this->foreignKeys[_i1072].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.foreignKeys = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->uniqueConstraints.clear(); + uint32_t _size1073; + ::apache::thrift::protocol::TType _etype1076; + xfer += iprot->readListBegin(_etype1076, _size1073); + this->uniqueConstraints.resize(_size1073); + uint32_t _i1077; + for (_i1077 = 0; _i1077 < _size1073; ++_i1077) + { + xfer += this->uniqueConstraints[_i1077].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.uniqueConstraints = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->notNullConstraints.clear(); + uint32_t _size1078; + ::apache::thrift::protocol::TType _etype1081; + xfer += iprot->readListBegin(_etype1081, _size1078); + this->notNullConstraints.resize(_size1078); + uint32_t _i1082; + for (_i1082 = 0; _i1082 < _size1078; ++_i1082) + { + xfer += this->notNullConstraints[_i1082].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.notNullConstraints = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_with_constraints_args"); + + xfer += oprot->writeFieldBegin("tbl", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->tbl.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->primaryKeys.size())); + std::vector ::const_iterator _iter1083; + for (_iter1083 = this->primaryKeys.begin(); _iter1083 != this->primaryKeys.end(); ++_iter1083) + { + xfer += (*_iter1083).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->foreignKeys.size())); + std::vector ::const_iterator _iter1084; + for (_iter1084 = this->foreignKeys.begin(); _iter1084 != this->foreignKeys.end(); ++_iter1084) + { + xfer += (*_iter1084).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->uniqueConstraints.size())); + std::vector ::const_iterator _iter1085; + for (_iter1085 = this->uniqueConstraints.begin(); _iter1085 != this->uniqueConstraints.end(); ++_iter1085) + { + xfer += (*_iter1085).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->notNullConstraints.size())); + std::vector ::const_iterator _iter1086; + for (_iter1086 = this->notNullConstraints.begin(); _iter1086 != this->notNullConstraints.end(); ++_iter1086) + { + xfer += (*_iter1086).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_table_with_constraints_pargs::~ThriftHiveMetastore_create_table_with_constraints_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_with_constraints_pargs"); + + xfer += oprot->writeFieldBegin("tbl", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->tbl)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->primaryKeys)).size())); + std::vector ::const_iterator _iter1087; + for (_iter1087 = (*(this->primaryKeys)).begin(); _iter1087 != (*(this->primaryKeys)).end(); ++_iter1087) + { + xfer += (*_iter1087).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->foreignKeys)).size())); + std::vector ::const_iterator _iter1088; + for (_iter1088 = (*(this->foreignKeys)).begin(); _iter1088 != (*(this->foreignKeys)).end(); ++_iter1088) + { + xfer += (*_iter1088).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->uniqueConstraints)).size())); + std::vector ::const_iterator _iter1089; + for (_iter1089 = (*(this->uniqueConstraints)).begin(); _iter1089 != (*(this->uniqueConstraints)).end(); ++_iter1089) + { + xfer += (*_iter1089).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->notNullConstraints)).size())); + std::vector ::const_iterator _iter1090; + for (_iter1090 = (*(this->notNullConstraints)).begin(); _iter1090 != (*(this->notNullConstraints)).end(); ++_iter1090) + { + xfer += (*_iter1090).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_table_with_constraints_result::~ThriftHiveMetastore_create_table_with_constraints_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_table_with_constraints_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_table_with_constraints_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_with_constraints_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_table_with_constraints_presult::~ThriftHiveMetastore_create_table_with_constraints_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_table_with_constraints_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_drop_constraint_args::~ThriftHiveMetastore_drop_constraint_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_constraint_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->req.read(iprot); + this->__isset.req = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_constraint_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_constraint_args"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->req.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_constraint_pargs::~ThriftHiveMetastore_drop_constraint_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_constraint_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_constraint_pargs"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->req)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_constraint_result::~ThriftHiveMetastore_drop_constraint_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_constraint_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_constraint_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_constraint_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_constraint_presult::~ThriftHiveMetastore_drop_constraint_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_constraint_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_add_primary_key_args::~ThriftHiveMetastore_add_primary_key_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_primary_key_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->req.read(iprot); + this->__isset.req = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_primary_key_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_primary_key_args"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->req.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_primary_key_pargs::~ThriftHiveMetastore_add_primary_key_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_primary_key_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_primary_key_pargs"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->req)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_primary_key_result::~ThriftHiveMetastore_add_primary_key_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_primary_key_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_primary_key_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_primary_key_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_primary_key_presult::~ThriftHiveMetastore_add_primary_key_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_primary_key_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_add_foreign_key_args::~ThriftHiveMetastore_add_foreign_key_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_foreign_key_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->req.read(iprot); + this->__isset.req = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_foreign_key_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_foreign_key_args"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->req.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_foreign_key_pargs::~ThriftHiveMetastore_add_foreign_key_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_foreign_key_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_foreign_key_pargs"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->req)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_foreign_key_result::~ThriftHiveMetastore_add_foreign_key_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_foreign_key_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_foreign_key_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_foreign_key_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_foreign_key_presult::~ThriftHiveMetastore_add_foreign_key_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_foreign_key_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_add_unique_constraint_args::~ThriftHiveMetastore_add_unique_constraint_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_unique_constraint_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->req.read(iprot); + this->__isset.req = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_unique_constraint_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_unique_constraint_args"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->req.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_unique_constraint_pargs::~ThriftHiveMetastore_add_unique_constraint_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_unique_constraint_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_unique_constraint_pargs"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->req)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_unique_constraint_result::~ThriftHiveMetastore_add_unique_constraint_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_unique_constraint_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_unique_constraint_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_unique_constraint_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_unique_constraint_presult::~ThriftHiveMetastore_add_unique_constraint_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_unique_constraint_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_add_not_null_constraint_args::~ThriftHiveMetastore_add_not_null_constraint_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_not_null_constraint_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->req.read(iprot); + this->__isset.req = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_not_null_constraint_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_not_null_constraint_args"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->req.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_not_null_constraint_pargs::~ThriftHiveMetastore_add_not_null_constraint_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_not_null_constraint_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_not_null_constraint_pargs"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->req)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_not_null_constraint_result::~ThriftHiveMetastore_add_not_null_constraint_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_not_null_constraint_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_not_null_constraint_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_not_null_constraint_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_not_null_constraint_presult::~ThriftHiveMetastore_add_not_null_constraint_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_not_null_constraint_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_drop_table_args::~ThriftHiveMetastore_drop_table_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_table_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbname); + this->__isset.dbname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->name); + this->__isset.name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->deleteData); + this->__isset.deleteData = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_table_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_args"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbname); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 3); + xfer += oprot->writeBool(this->deleteData); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_table_pargs::~ThriftHiveMetastore_drop_table_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_table_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_pargs"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbname))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 3); + xfer += oprot->writeBool((*(this->deleteData))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_table_result::~ThriftHiveMetastore_drop_table_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_table_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_table_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_table_presult::~ThriftHiveMetastore_drop_table_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_table_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_drop_table_with_environment_context_args::~ThriftHiveMetastore_drop_table_with_environment_context_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbname); + this->__isset.dbname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->name); + this->__isset.name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->deleteData); + this->__isset.deleteData = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->environment_context.read(iprot); + this->__isset.environment_context = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_args"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbname); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 3); + xfer += oprot->writeBool(this->deleteData); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->environment_context.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_table_with_environment_context_pargs::~ThriftHiveMetastore_drop_table_with_environment_context_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_table_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_pargs"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbname))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 3); + xfer += oprot->writeBool((*(this->deleteData))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += (*(this->environment_context)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_table_with_environment_context_result::~ThriftHiveMetastore_drop_table_with_environment_context_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_table_with_environment_context_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_table_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_table_with_environment_context_presult::~ThriftHiveMetastore_drop_table_with_environment_context_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_table_with_environment_context_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_truncate_table_args::~ThriftHiveMetastore_truncate_table_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_truncate_table_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tableName); + this->__isset.tableName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->partNames.clear(); + uint32_t _size1091; + ::apache::thrift::protocol::TType _etype1094; + xfer += iprot->readListBegin(_etype1094, _size1091); + this->partNames.resize(_size1091); + uint32_t _i1095; + for (_i1095 = 0; _i1095 < _size1091; ++_i1095) + { + xfer += iprot->readString(this->partNames[_i1095]); + } + xfer += iprot->readListEnd(); + } + this->__isset.partNames = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_truncate_table_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_truncate_table_args"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tableName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partNames.size())); + std::vector ::const_iterator _iter1096; + for (_iter1096 = this->partNames.begin(); _iter1096 != this->partNames.end(); ++_iter1096) + { + xfer += oprot->writeString((*_iter1096)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_truncate_table_pargs::~ThriftHiveMetastore_truncate_table_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_truncate_table_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_truncate_table_pargs"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbName))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tableName))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->partNames)).size())); + std::vector ::const_iterator _iter1097; + for (_iter1097 = (*(this->partNames)).begin(); _iter1097 != (*(this->partNames)).end(); ++_iter1097) + { + xfer += oprot->writeString((*_iter1097)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_truncate_table_result::~ThriftHiveMetastore_truncate_table_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_truncate_table_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_truncate_table_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_truncate_table_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_truncate_table_presult::~ThriftHiveMetastore_truncate_table_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_truncate_table_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_tables_args::~ThriftHiveMetastore_get_tables_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_tables_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->pattern); + this->__isset.pattern = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_tables_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->pattern); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_tables_pargs::~ThriftHiveMetastore_get_tables_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_tables_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->pattern))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_tables_result::~ThriftHiveMetastore_get_tables_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_tables_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1098; + ::apache::thrift::protocol::TType _etype1101; + xfer += iprot->readListBegin(_etype1101, _size1098); + this->success.resize(_size1098); + uint32_t _i1102; + for (_i1102 = 0; _i1102 < _size1098; ++_i1102) + { + xfer += iprot->readString(this->success[_i1102]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::vector ::const_iterator _iter1103; + for (_iter1103 = this->success.begin(); _iter1103 != this->success.end(); ++_iter1103) + { + xfer += oprot->writeString((*_iter1103)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_tables_presult::~ThriftHiveMetastore_get_tables_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_tables_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1104; + ::apache::thrift::protocol::TType _etype1107; + xfer += iprot->readListBegin(_etype1107, _size1104); + (*(this->success)).resize(_size1104); + uint32_t _i1108; + for (_i1108 = 0; _i1108 < _size1104; ++_i1108) + { + xfer += iprot->readString((*(this->success))[_i1108]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_tables_by_type_args::~ThriftHiveMetastore_get_tables_by_type_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_tables_by_type_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->pattern); + this->__isset.pattern = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tableType); + this->__isset.tableType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_tables_by_type_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->pattern); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->tableType); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_tables_by_type_pargs::~ThriftHiveMetastore_get_tables_by_type_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_tables_by_type_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->pattern))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->tableType))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_tables_by_type_result::~ThriftHiveMetastore_get_tables_by_type_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_tables_by_type_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1109; + ::apache::thrift::protocol::TType _etype1112; + xfer += iprot->readListBegin(_etype1112, _size1109); + this->success.resize(_size1109); + uint32_t _i1113; + for (_i1113 = 0; _i1113 < _size1109; ++_i1113) + { + xfer += iprot->readString(this->success[_i1113]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_tables_by_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::vector ::const_iterator _iter1114; + for (_iter1114 = this->success.begin(); _iter1114 != this->success.end(); ++_iter1114) + { + xfer += oprot->writeString((*_iter1114)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_tables_by_type_presult::~ThriftHiveMetastore_get_tables_by_type_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_tables_by_type_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1115; + ::apache::thrift::protocol::TType _etype1118; + xfer += iprot->readListBegin(_etype1118, _size1115); + (*(this->success)).resize(_size1115); + uint32_t _i1119; + for (_i1119 = 0; _i1119 < _size1115; ++_i1119) + { + xfer += iprot->readString((*(this->success))[_i1119]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_table_meta_args::~ThriftHiveMetastore_get_table_meta_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_meta_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_patterns); + this->__isset.db_patterns = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_patterns); + this->__isset.tbl_patterns = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->tbl_types.clear(); + uint32_t _size1120; + ::apache::thrift::protocol::TType _etype1123; + xfer += iprot->readListBegin(_etype1123, _size1120); + this->tbl_types.resize(_size1120); + uint32_t _i1124; + for (_i1124 = 0; _i1124 < _size1120; ++_i1124) + { + xfer += iprot->readString(this->tbl_types[_i1124]); + } + xfer += iprot->readListEnd(); + } + this->__isset.tbl_types = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_meta_args"); + + xfer += oprot->writeFieldBegin("db_patterns", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_patterns); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_patterns", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_patterns); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->tbl_types.size())); + std::vector ::const_iterator _iter1125; + for (_iter1125 = this->tbl_types.begin(); _iter1125 != this->tbl_types.end(); ++_iter1125) + { + xfer += oprot->writeString((*_iter1125)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_meta_pargs::~ThriftHiveMetastore_get_table_meta_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_meta_pargs"); + + xfer += oprot->writeFieldBegin("db_patterns", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_patterns))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_patterns", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_patterns))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->tbl_types)).size())); + std::vector ::const_iterator _iter1126; + for (_iter1126 = (*(this->tbl_types)).begin(); _iter1126 != (*(this->tbl_types)).end(); ++_iter1126) + { + xfer += oprot->writeString((*_iter1126)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_meta_result::~ThriftHiveMetastore_get_table_meta_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_meta_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1127; + ::apache::thrift::protocol::TType _etype1130; + xfer += iprot->readListBegin(_etype1130, _size1127); + this->success.resize(_size1127); + uint32_t _i1131; + for (_i1131 = 0; _i1131 < _size1127; ++_i1131) + { + xfer += this->success[_i1131].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_table_meta_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_meta_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter1132; + for (_iter1132 = this->success.begin(); _iter1132 != this->success.end(); ++_iter1132) + { + xfer += (*_iter1132).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_meta_presult::~ThriftHiveMetastore_get_table_meta_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_meta_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1133; + ::apache::thrift::protocol::TType _etype1136; + xfer += iprot->readListBegin(_etype1136, _size1133); + (*(this->success)).resize(_size1133); + uint32_t _i1137; + for (_i1137 = 0; _i1137 < _size1133; ++_i1137) + { + xfer += (*(this->success))[_i1137].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_all_tables_args::~ThriftHiveMetastore_get_all_tables_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_tables_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_all_tables_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_tables_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_all_tables_pargs::~ThriftHiveMetastore_get_all_tables_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_tables_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_tables_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_all_tables_result::~ThriftHiveMetastore_get_all_tables_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_tables_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1138; + ::apache::thrift::protocol::TType _etype1141; + xfer += iprot->readListBegin(_etype1141, _size1138); + this->success.resize(_size1138); + uint32_t _i1142; + for (_i1142 = 0; _i1142 < _size1138; ++_i1142) + { + xfer += iprot->readString(this->success[_i1142]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_tables_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::vector ::const_iterator _iter1143; + for (_iter1143 = this->success.begin(); _iter1143 != this->success.end(); ++_iter1143) + { + xfer += oprot->writeString((*_iter1143)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_all_tables_presult::~ThriftHiveMetastore_get_all_tables_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_tables_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1144; + ::apache::thrift::protocol::TType _etype1147; + xfer += iprot->readListBegin(_etype1147, _size1144); + (*(this->success)).resize(_size1144); + uint32_t _i1148; + for (_i1148 = 0; _i1148 < _size1144; ++_i1148) + { + xfer += iprot->readString((*(this->success))[_i1148]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_table_args::~ThriftHiveMetastore_get_table_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbname); + this->__isset.dbname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_table_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_args"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbname); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_pargs::~ThriftHiveMetastore_get_table_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_pargs"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbname))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_result::~ThriftHiveMetastore_get_table_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_table_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_presult::~ThriftHiveMetastore_get_table_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_table_objects_by_name_args::~ThriftHiveMetastore_get_table_objects_by_name_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbname); + this->__isset.dbname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->tbl_names.clear(); + uint32_t _size1149; + ::apache::thrift::protocol::TType _etype1152; + xfer += iprot->readListBegin(_etype1152, _size1149); + this->tbl_names.resize(_size1149); + uint32_t _i1153; + for (_i1153 = 0; _i1153 < _size1149; ++_i1153) + { + xfer += iprot->readString(this->tbl_names[_i1153]); + } + xfer += iprot->readListEnd(); + } + this->__isset.tbl_names = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_objects_by_name_args"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbname); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->tbl_names.size())); + std::vector ::const_iterator _iter1154; + for (_iter1154 = this->tbl_names.begin(); _iter1154 != this->tbl_names.end(); ++_iter1154) + { + xfer += oprot->writeString((*_iter1154)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_objects_by_name_pargs::~ThriftHiveMetastore_get_table_objects_by_name_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_objects_by_name_pargs"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbname))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->tbl_names)).size())); + std::vector ::const_iterator _iter1155; + for (_iter1155 = (*(this->tbl_names)).begin(); _iter1155 != (*(this->tbl_names)).end(); ++_iter1155) + { + xfer += oprot->writeString((*_iter1155)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_objects_by_name_result::~ThriftHiveMetastore_get_table_objects_by_name_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1156; + ::apache::thrift::protocol::TType _etype1159; + xfer += iprot->readListBegin(_etype1159, _size1156); + this->success.resize(_size1156); + uint32_t _i1160; + for (_i1160 = 0; _i1160 < _size1156; ++_i1160) + { + xfer += this->success[_i1160].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_objects_by_name_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector
::const_iterator _iter1161; + for (_iter1161 = this->success.begin(); _iter1161 != this->success.end(); ++_iter1161) + { + xfer += (*_iter1161).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_objects_by_name_presult::~ThriftHiveMetastore_get_table_objects_by_name_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_objects_by_name_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1162; + ::apache::thrift::protocol::TType _etype1165; + xfer += iprot->readListBegin(_etype1165, _size1162); + (*(this->success)).resize(_size1162); + uint32_t _i1166; + for (_i1166 = 0; _i1166 < _size1162; ++_i1166) + { + xfer += (*(this->success))[_i1166].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_table_req_args::~ThriftHiveMetastore_get_table_req_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_req_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->req.read(iprot); + this->__isset.req = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_table_req_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_req_args"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->req.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_req_pargs::~ThriftHiveMetastore_get_table_req_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_req_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_req_pargs"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->req)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_req_result::~ThriftHiveMetastore_get_table_req_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_req_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_table_req_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_req_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_req_presult::~ThriftHiveMetastore_get_table_req_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_req_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_table_objects_by_name_req_args::~ThriftHiveMetastore_get_table_objects_by_name_req_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_objects_by_name_req_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->req.read(iprot); + this->__isset.req = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_table_objects_by_name_req_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_objects_by_name_req_args"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->req.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_objects_by_name_req_pargs::~ThriftHiveMetastore_get_table_objects_by_name_req_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_objects_by_name_req_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_objects_by_name_req_pargs"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->req)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_objects_by_name_req_result::~ThriftHiveMetastore_get_table_objects_by_name_req_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_objects_by_name_req_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_table_objects_by_name_req_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_objects_by_name_req_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_objects_by_name_req_presult::~ThriftHiveMetastore_get_table_objects_by_name_req_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_objects_by_name_req_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_table_names_by_filter_args::~ThriftHiveMetastore_get_table_names_by_filter_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_names_by_filter_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbname); + this->__isset.dbname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->filter); + this->__isset.filter = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I16) { + xfer += iprot->readI16(this->max_tables); + this->__isset.max_tables = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_table_names_by_filter_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_names_by_filter_args"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbname); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->filter); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_tables", ::apache::thrift::protocol::T_I16, 3); + xfer += oprot->writeI16(this->max_tables); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_names_by_filter_pargs::~ThriftHiveMetastore_get_table_names_by_filter_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_names_by_filter_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_names_by_filter_pargs"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbname))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->filter))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_tables", ::apache::thrift::protocol::T_I16, 3); + xfer += oprot->writeI16((*(this->max_tables))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_names_by_filter_result::~ThriftHiveMetastore_get_table_names_by_filter_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1167; + ::apache::thrift::protocol::TType _etype1170; + xfer += iprot->readListBegin(_etype1170, _size1167); + this->success.resize(_size1167); + uint32_t _i1171; + for (_i1171 = 0; _i1171 < _size1167; ++_i1171) + { + xfer += iprot->readString(this->success[_i1171]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_names_by_filter_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::vector ::const_iterator _iter1172; + for (_iter1172 = this->success.begin(); _iter1172 != this->success.end(); ++_iter1172) + { + xfer += oprot->writeString((*_iter1172)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_names_by_filter_presult::~ThriftHiveMetastore_get_table_names_by_filter_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_names_by_filter_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1173; + ::apache::thrift::protocol::TType _etype1176; + xfer += iprot->readListBegin(_etype1176, _size1173); + (*(this->success)).resize(_size1173); + uint32_t _i1177; + for (_i1177 = 0; _i1177 < _size1173; ++_i1177) + { + xfer += iprot->readString((*(this->success))[_i1177]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_alter_table_args::~ThriftHiveMetastore_alter_table_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_table_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbname); + this->__isset.dbname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->new_tbl.read(iprot); + this->__isset.new_tbl = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_table_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_args"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbname); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_tbl", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->new_tbl.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_table_pargs::~ThriftHiveMetastore_alter_table_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_table_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_pargs"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbname))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_tbl", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += (*(this->new_tbl)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_table_result::~ThriftHiveMetastore_alter_table_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_table_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_table_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_table_presult::~ThriftHiveMetastore_alter_table_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_table_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_alter_table_with_environment_context_args::~ThriftHiveMetastore_alter_table_with_environment_context_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_table_with_environment_context_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbname); + this->__isset.dbname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->new_tbl.read(iprot); + this->__isset.new_tbl = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->environment_context.read(iprot); + this->__isset.environment_context = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_table_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_with_environment_context_args"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbname); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_tbl", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->new_tbl.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->environment_context.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_table_with_environment_context_pargs::~ThriftHiveMetastore_alter_table_with_environment_context_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_table_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_with_environment_context_pargs"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbname))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_tbl", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += (*(this->new_tbl)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += (*(this->environment_context)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_table_with_environment_context_result::~ThriftHiveMetastore_alter_table_with_environment_context_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_table_with_environment_context_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_table_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_with_environment_context_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_table_with_environment_context_presult::~ThriftHiveMetastore_alter_table_with_environment_context_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_table_with_environment_context_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_alter_table_with_cascade_args::~ThriftHiveMetastore_alter_table_with_cascade_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_table_with_cascade_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbname); + this->__isset.dbname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->new_tbl.read(iprot); + this->__isset.new_tbl = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->cascade); + this->__isset.cascade = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_table_with_cascade_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_with_cascade_args"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbname); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_tbl", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->new_tbl.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("cascade", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool(this->cascade); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_table_with_cascade_pargs::~ThriftHiveMetastore_alter_table_with_cascade_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_table_with_cascade_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_with_cascade_pargs"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbname))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_tbl", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += (*(this->new_tbl)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("cascade", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool((*(this->cascade))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_table_with_cascade_result::~ThriftHiveMetastore_alter_table_with_cascade_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_table_with_cascade_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_table_with_cascade_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_with_cascade_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_table_with_cascade_presult::~ThriftHiveMetastore_alter_table_with_cascade_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_table_with_cascade_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_add_partition_args::~ThriftHiveMetastore_add_partition_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partition_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->new_part.read(iprot); + this->__isset.new_part = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_partition_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partition_args"); + + xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->new_part.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_partition_pargs::~ThriftHiveMetastore_add_partition_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partition_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partition_pargs"); + + xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->new_part)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_partition_result::~ThriftHiveMetastore_add_partition_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partition_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_partition_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partition_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_partition_presult::~ThriftHiveMetastore_add_partition_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partition_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_add_partition_with_environment_context_args::~ThriftHiveMetastore_add_partition_with_environment_context_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partition_with_environment_context_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->new_part.read(iprot); + this->__isset.new_part = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->environment_context.read(iprot); + this->__isset.environment_context = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_partition_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partition_with_environment_context_args"); + + xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->new_part.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->environment_context.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_partition_with_environment_context_pargs::~ThriftHiveMetastore_add_partition_with_environment_context_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partition_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partition_with_environment_context_pargs"); + + xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->new_part)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += (*(this->environment_context)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_partition_with_environment_context_result::~ThriftHiveMetastore_add_partition_with_environment_context_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partition_with_environment_context_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_partition_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partition_with_environment_context_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_partition_with_environment_context_presult::~ThriftHiveMetastore_add_partition_with_environment_context_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partition_with_environment_context_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_add_partitions_args::~ThriftHiveMetastore_add_partitions_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partitions_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->new_parts.clear(); + uint32_t _size1178; + ::apache::thrift::protocol::TType _etype1181; + xfer += iprot->readListBegin(_etype1181, _size1178); + this->new_parts.resize(_size1178); + uint32_t _i1182; + for (_i1182 = 0; _i1182 < _size1178; ++_i1182) + { + xfer += this->new_parts[_i1182].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.new_parts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_partitions_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_args"); + + xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->new_parts.size())); + std::vector ::const_iterator _iter1183; + for (_iter1183 = this->new_parts.begin(); _iter1183 != this->new_parts.end(); ++_iter1183) + { + xfer += (*_iter1183).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_partitions_pargs::~ThriftHiveMetastore_add_partitions_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partitions_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_pargs"); + + xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->new_parts)).size())); + std::vector ::const_iterator _iter1184; + for (_iter1184 = (*(this->new_parts)).begin(); _iter1184 != (*(this->new_parts)).end(); ++_iter1184) + { + xfer += (*_iter1184).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_partitions_result::~ThriftHiveMetastore_add_partitions_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partitions_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_partitions_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_I32, 0); + xfer += oprot->writeI32(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_partitions_presult::~ThriftHiveMetastore_add_partitions_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partitions_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_add_partitions_pspec_args::~ThriftHiveMetastore_add_partitions_pspec_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partitions_pspec_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->new_parts.clear(); + uint32_t _size1185; + ::apache::thrift::protocol::TType _etype1188; + xfer += iprot->readListBegin(_etype1188, _size1185); + this->new_parts.resize(_size1185); + uint32_t _i1189; + for (_i1189 = 0; _i1189 < _size1185; ++_i1189) + { + xfer += this->new_parts[_i1189].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.new_parts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_partitions_pspec_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_pspec_args"); + + xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->new_parts.size())); + std::vector ::const_iterator _iter1190; + for (_iter1190 = this->new_parts.begin(); _iter1190 != this->new_parts.end(); ++_iter1190) + { + xfer += (*_iter1190).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_partitions_pspec_pargs::~ThriftHiveMetastore_add_partitions_pspec_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partitions_pspec_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_pspec_pargs"); + + xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->new_parts)).size())); + std::vector ::const_iterator _iter1191; + for (_iter1191 = (*(this->new_parts)).begin(); _iter1191 != (*(this->new_parts)).end(); ++_iter1191) + { + xfer += (*_iter1191).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_partitions_pspec_result::~ThriftHiveMetastore_add_partitions_pspec_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partitions_pspec_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_partitions_pspec_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_pspec_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_I32, 0); + xfer += oprot->writeI32(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_partitions_pspec_presult::~ThriftHiveMetastore_add_partitions_pspec_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partitions_pspec_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_append_partition_args::~ThriftHiveMetastore_append_partition_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_append_partition_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->part_vals.clear(); + uint32_t _size1192; + ::apache::thrift::protocol::TType _etype1195; + xfer += iprot->readListBegin(_etype1195, _size1192); + this->part_vals.resize(_size1192); + uint32_t _i1196; + for (_i1196 = 0; _i1196 < _size1192; ++_i1196) + { + xfer += iprot->readString(this->part_vals[_i1196]); + } + xfer += iprot->readListEnd(); + } + this->__isset.part_vals = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_append_partition_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); + std::vector ::const_iterator _iter1197; + for (_iter1197 = this->part_vals.begin(); _iter1197 != this->part_vals.end(); ++_iter1197) + { + xfer += oprot->writeString((*_iter1197)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_append_partition_pargs::~ThriftHiveMetastore_append_partition_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_append_partition_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); + std::vector ::const_iterator _iter1198; + for (_iter1198 = (*(this->part_vals)).begin(); _iter1198 != (*(this->part_vals)).end(); ++_iter1198) + { + xfer += oprot->writeString((*_iter1198)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_append_partition_result::~ThriftHiveMetastore_append_partition_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_append_partition_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_append_partition_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_append_partition_presult::~ThriftHiveMetastore_append_partition_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_append_partition_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_add_partitions_req_args::~ThriftHiveMetastore_add_partitions_req_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partitions_req_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_partitions_req_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_req_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_partitions_req_pargs::~ThriftHiveMetastore_add_partitions_req_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partitions_req_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_req_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_partitions_req_result::~ThriftHiveMetastore_add_partitions_req_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partitions_req_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_partitions_req_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_req_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_partitions_req_presult::~ThriftHiveMetastore_add_partitions_req_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_partitions_req_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_append_partition_with_environment_context_args::~ThriftHiveMetastore_append_partition_with_environment_context_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->part_vals.clear(); + uint32_t _size1199; + ::apache::thrift::protocol::TType _etype1202; + xfer += iprot->readListBegin(_etype1202, _size1199); + this->part_vals.resize(_size1199); + uint32_t _i1203; + for (_i1203 = 0; _i1203 < _size1199; ++_i1203) + { + xfer += iprot->readString(this->part_vals[_i1203]); + } + xfer += iprot->readListEnd(); + } + this->__isset.part_vals = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->environment_context.read(iprot); + this->__isset.environment_context = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_with_environment_context_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); + std::vector ::const_iterator _iter1204; + for (_iter1204 = this->part_vals.begin(); _iter1204 != this->part_vals.end(); ++_iter1204) + { + xfer += oprot->writeString((*_iter1204)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->environment_context.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_append_partition_with_environment_context_pargs::~ThriftHiveMetastore_append_partition_with_environment_context_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_append_partition_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_with_environment_context_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); + std::vector ::const_iterator _iter1205; + for (_iter1205 = (*(this->part_vals)).begin(); _iter1205 != (*(this->part_vals)).end(); ++_iter1205) + { + xfer += oprot->writeString((*_iter1205)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += (*(this->environment_context)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_append_partition_with_environment_context_result::~ThriftHiveMetastore_append_partition_with_environment_context_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_append_partition_with_environment_context_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_append_partition_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_with_environment_context_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_append_partition_with_environment_context_presult::~ThriftHiveMetastore_append_partition_with_environment_context_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_append_partition_with_environment_context_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_append_partition_by_name_args::~ThriftHiveMetastore_append_partition_by_name_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_append_partition_by_name_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->part_name); + this->__isset.part_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_append_partition_by_name_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_by_name_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->part_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_append_partition_by_name_pargs::~ThriftHiveMetastore_append_partition_by_name_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_append_partition_by_name_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_by_name_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->part_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_append_partition_by_name_result::~ThriftHiveMetastore_append_partition_by_name_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_append_partition_by_name_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_append_partition_by_name_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_by_name_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_append_partition_by_name_presult::~ThriftHiveMetastore_append_partition_by_name_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_append_partition_by_name_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_append_partition_by_name_with_environment_context_args::~ThriftHiveMetastore_append_partition_by_name_with_environment_context_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_append_partition_by_name_with_environment_context_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->part_name); + this->__isset.part_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->environment_context.read(iprot); + this->__isset.environment_context = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_append_partition_by_name_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_by_name_with_environment_context_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->part_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->environment_context.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs::~ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->part_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += (*(this->environment_context)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_append_partition_by_name_with_environment_context_result::~ThriftHiveMetastore_append_partition_by_name_with_environment_context_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_append_partition_by_name_with_environment_context_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_append_partition_by_name_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_by_name_with_environment_context_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult::~ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_drop_partition_args::~ThriftHiveMetastore_drop_partition_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partition_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->part_vals.clear(); + uint32_t _size1206; + ::apache::thrift::protocol::TType _etype1209; + xfer += iprot->readListBegin(_etype1209, _size1206); + this->part_vals.resize(_size1206); + uint32_t _i1210; + for (_i1210 = 0; _i1210 < _size1206; ++_i1210) + { + xfer += iprot->readString(this->part_vals[_i1210]); + } + xfer += iprot->readListEnd(); + } + this->__isset.part_vals = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->deleteData); + this->__isset.deleteData = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_partition_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); + std::vector ::const_iterator _iter1211; + for (_iter1211 = this->part_vals.begin(); _iter1211 != this->part_vals.end(); ++_iter1211) + { + xfer += oprot->writeString((*_iter1211)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool(this->deleteData); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_partition_pargs::~ThriftHiveMetastore_drop_partition_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partition_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); + std::vector ::const_iterator _iter1212; + for (_iter1212 = (*(this->part_vals)).begin(); _iter1212 != (*(this->part_vals)).end(); ++_iter1212) + { + xfer += oprot->writeString((*_iter1212)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool((*(this->deleteData))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_partition_result::~ThriftHiveMetastore_drop_partition_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partition_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_partition_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_partition_presult::~ThriftHiveMetastore_drop_partition_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partition_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_drop_partition_with_environment_context_args::~ThriftHiveMetastore_drop_partition_with_environment_context_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->part_vals.clear(); + uint32_t _size1213; + ::apache::thrift::protocol::TType _etype1216; + xfer += iprot->readListBegin(_etype1216, _size1213); + this->part_vals.resize(_size1213); + uint32_t _i1217; + for (_i1217 = 0; _i1217 < _size1213; ++_i1217) + { + xfer += iprot->readString(this->part_vals[_i1217]); + } + xfer += iprot->readListEnd(); + } + this->__isset.part_vals = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->deleteData); + this->__isset.deleteData = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->environment_context.read(iprot); + this->__isset.environment_context = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_with_environment_context_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); + std::vector ::const_iterator _iter1218; + for (_iter1218 = this->part_vals.begin(); _iter1218 != this->part_vals.end(); ++_iter1218) + { + xfer += oprot->writeString((*_iter1218)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool(this->deleteData); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 5); + xfer += this->environment_context.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_partition_with_environment_context_pargs::~ThriftHiveMetastore_drop_partition_with_environment_context_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_with_environment_context_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); + std::vector ::const_iterator _iter1219; + for (_iter1219 = (*(this->part_vals)).begin(); _iter1219 != (*(this->part_vals)).end(); ++_iter1219) + { + xfer += oprot->writeString((*_iter1219)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool((*(this->deleteData))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 5); + xfer += (*(this->environment_context)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_partition_with_environment_context_result::~ThriftHiveMetastore_drop_partition_with_environment_context_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_with_environment_context_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_partition_with_environment_context_presult::~ThriftHiveMetastore_drop_partition_with_environment_context_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_drop_partition_by_name_args::~ThriftHiveMetastore_drop_partition_by_name_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partition_by_name_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->part_name); + this->__isset.part_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->deleteData); + this->__isset.deleteData = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_partition_by_name_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_by_name_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->part_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool(this->deleteData); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_partition_by_name_pargs::~ThriftHiveMetastore_drop_partition_by_name_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partition_by_name_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_by_name_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->part_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool((*(this->deleteData))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_partition_by_name_result::~ThriftHiveMetastore_drop_partition_by_name_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partition_by_name_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_partition_by_name_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_by_name_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_partition_by_name_presult::~ThriftHiveMetastore_drop_partition_by_name_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partition_by_name_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args::~ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->part_name); + this->__isset.part_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->deleteData); + this->__isset.deleteData = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->environment_context.read(iprot); + this->__isset.environment_context = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->part_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool(this->deleteData); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 5); + xfer += this->environment_context.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs::~ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->part_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool((*(this->deleteData))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 5); + xfer += (*(this->environment_context)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result::~ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult::~ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_drop_partitions_req_args::~ThriftHiveMetastore_drop_partitions_req_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partitions_req_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->req.read(iprot); + this->__isset.req = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_partitions_req_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partitions_req_args"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->req.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_partitions_req_pargs::~ThriftHiveMetastore_drop_partitions_req_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partitions_req_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partitions_req_pargs"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->req)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_partitions_req_result::~ThriftHiveMetastore_drop_partitions_req_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partitions_req_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_partitions_req_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partitions_req_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_partitions_req_presult::~ThriftHiveMetastore_drop_partitions_req_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_partitions_req_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_partition_args::~ThriftHiveMetastore_get_partition_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->part_vals.clear(); + uint32_t _size1220; + ::apache::thrift::protocol::TType _etype1223; + xfer += iprot->readListBegin(_etype1223, _size1220); + this->part_vals.resize(_size1220); + uint32_t _i1224; + for (_i1224 = 0; _i1224 < _size1220; ++_i1224) + { + xfer += iprot->readString(this->part_vals[_i1224]); + } + xfer += iprot->readListEnd(); + } + this->__isset.part_vals = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partition_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); + std::vector ::const_iterator _iter1225; + for (_iter1225 = this->part_vals.begin(); _iter1225 != this->part_vals.end(); ++_iter1225) + { + xfer += oprot->writeString((*_iter1225)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_pargs::~ThriftHiveMetastore_get_partition_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); + std::vector ::const_iterator _iter1226; + for (_iter1226 = (*(this->part_vals)).begin(); _iter1226 != (*(this->part_vals)).end(); ++_iter1226) + { + xfer += oprot->writeString((*_iter1226)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_result::~ThriftHiveMetastore_get_partition_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partition_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_presult::~ThriftHiveMetastore_get_partition_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_exchange_partition_args::~ThriftHiveMetastore_exchange_partition_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_exchange_partition_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->partitionSpecs.clear(); + uint32_t _size1227; + ::apache::thrift::protocol::TType _ktype1228; + ::apache::thrift::protocol::TType _vtype1229; + xfer += iprot->readMapBegin(_ktype1228, _vtype1229, _size1227); + uint32_t _i1231; + for (_i1231 = 0; _i1231 < _size1227; ++_i1231) + { + std::string _key1232; + xfer += iprot->readString(_key1232); + std::string& _val1233 = this->partitionSpecs[_key1232]; + xfer += iprot->readString(_val1233); + } + xfer += iprot->readMapEnd(); + } + this->__isset.partitionSpecs = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->source_db); + this->__isset.source_db = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->source_table_name); + this->__isset.source_table_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dest_db); + this->__isset.dest_db = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dest_table_name); + this->__isset.dest_table_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_exchange_partition_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_exchange_partition_args"); + + xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->partitionSpecs.size())); + std::map ::const_iterator _iter1234; + for (_iter1234 = this->partitionSpecs.begin(); _iter1234 != this->partitionSpecs.end(); ++_iter1234) + { + xfer += oprot->writeString(_iter1234->first); + xfer += oprot->writeString(_iter1234->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("source_db", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->source_db); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("source_table_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->source_table_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dest_db", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->dest_db); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dest_table_name", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->dest_table_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_exchange_partition_pargs::~ThriftHiveMetastore_exchange_partition_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_exchange_partition_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_exchange_partition_pargs"); + + xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast((*(this->partitionSpecs)).size())); + std::map ::const_iterator _iter1235; + for (_iter1235 = (*(this->partitionSpecs)).begin(); _iter1235 != (*(this->partitionSpecs)).end(); ++_iter1235) + { + xfer += oprot->writeString(_iter1235->first); + xfer += oprot->writeString(_iter1235->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("source_db", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->source_db))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("source_table_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->source_table_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dest_db", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString((*(this->dest_db))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dest_table_name", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString((*(this->dest_table_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_exchange_partition_result::~ThriftHiveMetastore_exchange_partition_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_exchange_partition_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_exchange_partition_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_exchange_partition_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_exchange_partition_presult::~ThriftHiveMetastore_exchange_partition_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_exchange_partition_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_exchange_partitions_args::~ThriftHiveMetastore_exchange_partitions_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_exchange_partitions_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->partitionSpecs.clear(); + uint32_t _size1236; + ::apache::thrift::protocol::TType _ktype1237; + ::apache::thrift::protocol::TType _vtype1238; + xfer += iprot->readMapBegin(_ktype1237, _vtype1238, _size1236); + uint32_t _i1240; + for (_i1240 = 0; _i1240 < _size1236; ++_i1240) + { + std::string _key1241; + xfer += iprot->readString(_key1241); + std::string& _val1242 = this->partitionSpecs[_key1241]; + xfer += iprot->readString(_val1242); + } + xfer += iprot->readMapEnd(); + } + this->__isset.partitionSpecs = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->source_db); + this->__isset.source_db = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->source_table_name); + this->__isset.source_table_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dest_db); + this->__isset.dest_db = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dest_table_name); + this->__isset.dest_table_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_exchange_partitions_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_exchange_partitions_args"); + + xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->partitionSpecs.size())); + std::map ::const_iterator _iter1243; + for (_iter1243 = this->partitionSpecs.begin(); _iter1243 != this->partitionSpecs.end(); ++_iter1243) + { + xfer += oprot->writeString(_iter1243->first); + xfer += oprot->writeString(_iter1243->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("source_db", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->source_db); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("source_table_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->source_table_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dest_db", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->dest_db); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dest_table_name", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->dest_table_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_exchange_partitions_pargs::~ThriftHiveMetastore_exchange_partitions_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_exchange_partitions_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_exchange_partitions_pargs"); + + xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast((*(this->partitionSpecs)).size())); + std::map ::const_iterator _iter1244; + for (_iter1244 = (*(this->partitionSpecs)).begin(); _iter1244 != (*(this->partitionSpecs)).end(); ++_iter1244) + { + xfer += oprot->writeString(_iter1244->first); + xfer += oprot->writeString(_iter1244->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("source_db", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->source_db))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("source_table_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->source_table_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dest_db", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString((*(this->dest_db))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dest_table_name", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString((*(this->dest_table_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_exchange_partitions_result::~ThriftHiveMetastore_exchange_partitions_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_exchange_partitions_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1245; + ::apache::thrift::protocol::TType _etype1248; + xfer += iprot->readListBegin(_etype1248, _size1245); + this->success.resize(_size1245); + uint32_t _i1249; + for (_i1249 = 0; _i1249 < _size1245; ++_i1249) + { + xfer += this->success[_i1249].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_exchange_partitions_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_exchange_partitions_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter1250; + for (_iter1250 = this->success.begin(); _iter1250 != this->success.end(); ++_iter1250) + { + xfer += (*_iter1250).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_exchange_partitions_presult::~ThriftHiveMetastore_exchange_partitions_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_exchange_partitions_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1251; + ::apache::thrift::protocol::TType _etype1254; + xfer += iprot->readListBegin(_etype1254, _size1251); + (*(this->success)).resize(_size1251); + uint32_t _i1255; + for (_i1255 = 0; _i1255 < _size1251; ++_i1255) + { + xfer += (*(this->success))[_i1255].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_partition_with_auth_args::~ThriftHiveMetastore_get_partition_with_auth_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_with_auth_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->part_vals.clear(); + uint32_t _size1256; + ::apache::thrift::protocol::TType _etype1259; + xfer += iprot->readListBegin(_etype1259, _size1256); + this->part_vals.resize(_size1256); + uint32_t _i1260; + for (_i1260 = 0; _i1260 < _size1256; ++_i1260) + { + xfer += iprot->readString(this->part_vals[_i1260]); + } + xfer += iprot->readListEnd(); + } + this->__isset.part_vals = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->user_name); + this->__isset.user_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->group_names.clear(); + uint32_t _size1261; + ::apache::thrift::protocol::TType _etype1264; + xfer += iprot->readListBegin(_etype1264, _size1261); + this->group_names.resize(_size1261); + uint32_t _i1265; + for (_i1265 = 0; _i1265 < _size1261; ++_i1265) + { + xfer += iprot->readString(this->group_names[_i1265]); + } + xfer += iprot->readListEnd(); + } + this->__isset.group_names = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_with_auth_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); + std::vector ::const_iterator _iter1266; + for (_iter1266 = this->part_vals.begin(); _iter1266 != this->part_vals.end(); ++_iter1266) + { + xfer += oprot->writeString((*_iter1266)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->user_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); + std::vector ::const_iterator _iter1267; + for (_iter1267 = this->group_names.begin(); _iter1267 != this->group_names.end(); ++_iter1267) + { + xfer += oprot->writeString((*_iter1267)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_with_auth_pargs::~ThriftHiveMetastore_get_partition_with_auth_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_with_auth_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); + std::vector ::const_iterator _iter1268; + for (_iter1268 = (*(this->part_vals)).begin(); _iter1268 != (*(this->part_vals)).end(); ++_iter1268) + { + xfer += oprot->writeString((*_iter1268)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString((*(this->user_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); + std::vector ::const_iterator _iter1269; + for (_iter1269 = (*(this->group_names)).begin(); _iter1269 != (*(this->group_names)).end(); ++_iter1269) + { + xfer += oprot->writeString((*_iter1269)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_with_auth_result::~ThriftHiveMetastore_get_partition_with_auth_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_with_auth_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partition_with_auth_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_with_auth_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_with_auth_presult::~ThriftHiveMetastore_get_partition_with_auth_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_with_auth_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_partition_by_name_args::~ThriftHiveMetastore_get_partition_by_name_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_by_name_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->part_name); + this->__isset.part_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partition_by_name_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_by_name_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->part_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_by_name_pargs::~ThriftHiveMetastore_get_partition_by_name_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_by_name_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_by_name_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->part_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_by_name_result::~ThriftHiveMetastore_get_partition_by_name_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_by_name_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partition_by_name_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_by_name_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_by_name_presult::~ThriftHiveMetastore_get_partition_by_name_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_by_name_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_partitions_args::~ThriftHiveMetastore_get_partitions_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I16) { + xfer += iprot->readI16(this->max_parts); + this->__isset.max_parts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 3); + xfer += oprot->writeI16(this->max_parts); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_pargs::~ThriftHiveMetastore_get_partitions_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 3); + xfer += oprot->writeI16((*(this->max_parts))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_result::~ThriftHiveMetastore_get_partitions_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1270; + ::apache::thrift::protocol::TType _etype1273; + xfer += iprot->readListBegin(_etype1273, _size1270); + this->success.resize(_size1270); + uint32_t _i1274; + for (_i1274 = 0; _i1274 < _size1270; ++_i1274) + { + xfer += this->success[_i1274].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter1275; + for (_iter1275 = this->success.begin(); _iter1275 != this->success.end(); ++_iter1275) + { + xfer += (*_iter1275).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_presult::~ThriftHiveMetastore_get_partitions_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1276; + ::apache::thrift::protocol::TType _etype1279; + xfer += iprot->readListBegin(_etype1279, _size1276); + (*(this->success)).resize(_size1276); + uint32_t _i1280; + for (_i1280 = 0; _i1280 < _size1276; ++_i1280) + { + xfer += (*(this->success))[_i1280].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_partitions_with_auth_args::~ThriftHiveMetastore_get_partitions_with_auth_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I16) { + xfer += iprot->readI16(this->max_parts); + this->__isset.max_parts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->user_name); + this->__isset.user_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->group_names.clear(); + uint32_t _size1281; + ::apache::thrift::protocol::TType _etype1284; + xfer += iprot->readListBegin(_etype1284, _size1281); + this->group_names.resize(_size1281); + uint32_t _i1285; + for (_i1285 = 0; _i1285 < _size1281; ++_i1285) + { + xfer += iprot->readString(this->group_names[_i1285]); + } + xfer += iprot->readListEnd(); + } + this->__isset.group_names = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_with_auth_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 3); + xfer += oprot->writeI16(this->max_parts); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->user_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); + std::vector ::const_iterator _iter1286; + for (_iter1286 = this->group_names.begin(); _iter1286 != this->group_names.end(); ++_iter1286) + { + xfer += oprot->writeString((*_iter1286)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_with_auth_pargs::~ThriftHiveMetastore_get_partitions_with_auth_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_with_auth_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_with_auth_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 3); + xfer += oprot->writeI16((*(this->max_parts))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString((*(this->user_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); + std::vector ::const_iterator _iter1287; + for (_iter1287 = (*(this->group_names)).begin(); _iter1287 != (*(this->group_names)).end(); ++_iter1287) + { + xfer += oprot->writeString((*_iter1287)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_with_auth_result::~ThriftHiveMetastore_get_partitions_with_auth_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1288; + ::apache::thrift::protocol::TType _etype1291; + xfer += iprot->readListBegin(_etype1291, _size1288); + this->success.resize(_size1288); + uint32_t _i1292; + for (_i1292 = 0; _i1292 < _size1288; ++_i1292) + { + xfer += this->success[_i1292].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_with_auth_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter1293; + for (_iter1293 = this->success.begin(); _iter1293 != this->success.end(); ++_iter1293) + { + xfer += (*_iter1293).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_with_auth_presult::~ThriftHiveMetastore_get_partitions_with_auth_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1294; + ::apache::thrift::protocol::TType _etype1297; + xfer += iprot->readListBegin(_etype1297, _size1294); + (*(this->success)).resize(_size1294); + uint32_t _i1298; + for (_i1298 = 0; _i1298 < _size1294; ++_i1298) + { + xfer += (*(this->success))[_i1298].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_partitions_pspec_args::~ThriftHiveMetastore_get_partitions_pspec_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_pspec_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->max_parts); + this->__isset.max_parts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_pspec_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_pspec_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I32, 3); + xfer += oprot->writeI32(this->max_parts); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_pspec_pargs::~ThriftHiveMetastore_get_partitions_pspec_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_pspec_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_pspec_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I32, 3); + xfer += oprot->writeI32((*(this->max_parts))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_pspec_result::~ThriftHiveMetastore_get_partitions_pspec_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_pspec_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1299; + ::apache::thrift::protocol::TType _etype1302; + xfer += iprot->readListBegin(_etype1302, _size1299); + this->success.resize(_size1299); + uint32_t _i1303; + for (_i1303 = 0; _i1303 < _size1299; ++_i1303) + { + xfer += this->success[_i1303].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_pspec_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_pspec_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter1304; + for (_iter1304 = this->success.begin(); _iter1304 != this->success.end(); ++_iter1304) + { + xfer += (*_iter1304).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_pspec_presult::~ThriftHiveMetastore_get_partitions_pspec_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1305; + ::apache::thrift::protocol::TType _etype1308; + xfer += iprot->readListBegin(_etype1308, _size1305); + (*(this->success)).resize(_size1305); + uint32_t _i1309; + for (_i1309 = 0; _i1309 < _size1305; ++_i1309) + { + xfer += (*(this->success))[_i1309].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_partition_names_args::~ThriftHiveMetastore_get_partition_names_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_names_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I16) { + xfer += iprot->readI16(this->max_parts); + this->__isset.max_parts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partition_names_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_names_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 3); + xfer += oprot->writeI16(this->max_parts); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_names_pargs::~ThriftHiveMetastore_get_partition_names_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_names_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_names_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 3); + xfer += oprot->writeI16((*(this->max_parts))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_names_result::~ThriftHiveMetastore_get_partition_names_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_names_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1310; + ::apache::thrift::protocol::TType _etype1313; + xfer += iprot->readListBegin(_etype1313, _size1310); + this->success.resize(_size1310); + uint32_t _i1314; + for (_i1314 = 0; _i1314 < _size1310; ++_i1314) + { + xfer += iprot->readString(this->success[_i1314]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partition_names_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_names_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::vector ::const_iterator _iter1315; + for (_iter1315 = this->success.begin(); _iter1315 != this->success.end(); ++_iter1315) + { + xfer += oprot->writeString((*_iter1315)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_names_presult::~ThriftHiveMetastore_get_partition_names_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_names_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1316; + ::apache::thrift::protocol::TType _etype1319; + xfer += iprot->readListBegin(_etype1319, _size1316); + (*(this->success)).resize(_size1316); + uint32_t _i1320; + for (_i1320 = 0; _i1320 < _size1316; ++_i1320) + { + xfer += iprot->readString((*(this->success))[_i1320]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_partition_values_args::~ThriftHiveMetastore_get_partition_values_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_values_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partition_values_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_values_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_values_pargs::~ThriftHiveMetastore_get_partition_values_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_values_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_values_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_values_result::~ThriftHiveMetastore_get_partition_values_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_values_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partition_values_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_values_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_values_presult::~ThriftHiveMetastore_get_partition_values_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_values_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_partitions_ps_args::~ThriftHiveMetastore_get_partitions_ps_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_ps_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->part_vals.clear(); + uint32_t _size1321; + ::apache::thrift::protocol::TType _etype1324; + xfer += iprot->readListBegin(_etype1324, _size1321); + this->part_vals.resize(_size1321); + uint32_t _i1325; + for (_i1325 = 0; _i1325 < _size1321; ++_i1325) + { + xfer += iprot->readString(this->part_vals[_i1325]); + } + xfer += iprot->readListEnd(); + } + this->__isset.part_vals = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I16) { + xfer += iprot->readI16(this->max_parts); + this->__isset.max_parts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_ps_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_ps_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); + std::vector ::const_iterator _iter1326; + for (_iter1326 = this->part_vals.begin(); _iter1326 != this->part_vals.end(); ++_iter1326) + { + xfer += oprot->writeString((*_iter1326)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4); + xfer += oprot->writeI16(this->max_parts); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_ps_pargs::~ThriftHiveMetastore_get_partitions_ps_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_ps_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_ps_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); + std::vector ::const_iterator _iter1327; + for (_iter1327 = (*(this->part_vals)).begin(); _iter1327 != (*(this->part_vals)).end(); ++_iter1327) + { + xfer += oprot->writeString((*_iter1327)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4); + xfer += oprot->writeI16((*(this->max_parts))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_ps_result::~ThriftHiveMetastore_get_partitions_ps_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_ps_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1328; + ::apache::thrift::protocol::TType _etype1331; + xfer += iprot->readListBegin(_etype1331, _size1328); + this->success.resize(_size1328); + uint32_t _i1332; + for (_i1332 = 0; _i1332 < _size1328; ++_i1332) + { + xfer += this->success[_i1332].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_ps_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_ps_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter1333; + for (_iter1333 = this->success.begin(); _iter1333 != this->success.end(); ++_iter1333) + { + xfer += (*_iter1333).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_ps_presult::~ThriftHiveMetastore_get_partitions_ps_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_ps_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1334; + ::apache::thrift::protocol::TType _etype1337; + xfer += iprot->readListBegin(_etype1337, _size1334); + (*(this->success)).resize(_size1334); + uint32_t _i1338; + for (_i1338 = 0; _i1338 < _size1334; ++_i1338) + { + xfer += (*(this->success))[_i1338].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_partitions_ps_with_auth_args::~ThriftHiveMetastore_get_partitions_ps_with_auth_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->part_vals.clear(); + uint32_t _size1339; + ::apache::thrift::protocol::TType _etype1342; + xfer += iprot->readListBegin(_etype1342, _size1339); + this->part_vals.resize(_size1339); + uint32_t _i1343; + for (_i1343 = 0; _i1343 < _size1339; ++_i1343) + { + xfer += iprot->readString(this->part_vals[_i1343]); + } + xfer += iprot->readListEnd(); + } + this->__isset.part_vals = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I16) { + xfer += iprot->readI16(this->max_parts); + this->__isset.max_parts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->user_name); + this->__isset.user_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->group_names.clear(); + uint32_t _size1344; + ::apache::thrift::protocol::TType _etype1347; + xfer += iprot->readListBegin(_etype1347, _size1344); + this->group_names.resize(_size1344); + uint32_t _i1348; + for (_i1348 = 0; _i1348 < _size1344; ++_i1348) + { + xfer += iprot->readString(this->group_names[_i1348]); + } + xfer += iprot->readListEnd(); + } + this->__isset.group_names = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_ps_with_auth_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); + std::vector ::const_iterator _iter1349; + for (_iter1349 = this->part_vals.begin(); _iter1349 != this->part_vals.end(); ++_iter1349) + { + xfer += oprot->writeString((*_iter1349)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4); + xfer += oprot->writeI16(this->max_parts); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->user_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); + std::vector ::const_iterator _iter1350; + for (_iter1350 = this->group_names.begin(); _iter1350 != this->group_names.end(); ++_iter1350) + { + xfer += oprot->writeString((*_iter1350)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::~ThriftHiveMetastore_get_partitions_ps_with_auth_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_ps_with_auth_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); + std::vector ::const_iterator _iter1351; + for (_iter1351 = (*(this->part_vals)).begin(); _iter1351 != (*(this->part_vals)).end(); ++_iter1351) + { + xfer += oprot->writeString((*_iter1351)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4); + xfer += oprot->writeI16((*(this->max_parts))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString((*(this->user_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); + std::vector ::const_iterator _iter1352; + for (_iter1352 = (*(this->group_names)).begin(); _iter1352 != (*(this->group_names)).end(); ++_iter1352) + { + xfer += oprot->writeString((*_iter1352)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_ps_with_auth_result::~ThriftHiveMetastore_get_partitions_ps_with_auth_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1353; + ::apache::thrift::protocol::TType _etype1356; + xfer += iprot->readListBegin(_etype1356, _size1353); + this->success.resize(_size1353); + uint32_t _i1357; + for (_i1357 = 0; _i1357 < _size1353; ++_i1357) + { + xfer += this->success[_i1357].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_ps_with_auth_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter1358; + for (_iter1358 = this->success.begin(); _iter1358 != this->success.end(); ++_iter1358) + { + xfer += (*_iter1358).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_ps_with_auth_presult::~ThriftHiveMetastore_get_partitions_ps_with_auth_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1359; + ::apache::thrift::protocol::TType _etype1362; + xfer += iprot->readListBegin(_etype1362, _size1359); + (*(this->success)).resize(_size1359); + uint32_t _i1363; + for (_i1363 = 0; _i1363 < _size1359; ++_i1363) + { + xfer += (*(this->success))[_i1363].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_partition_names_ps_args::~ThriftHiveMetastore_get_partition_names_ps_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_names_ps_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->part_vals.clear(); + uint32_t _size1364; + ::apache::thrift::protocol::TType _etype1367; + xfer += iprot->readListBegin(_etype1367, _size1364); + this->part_vals.resize(_size1364); + uint32_t _i1368; + for (_i1368 = 0; _i1368 < _size1364; ++_i1368) + { + xfer += iprot->readString(this->part_vals[_i1368]); + } + xfer += iprot->readListEnd(); + } + this->__isset.part_vals = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I16) { + xfer += iprot->readI16(this->max_parts); + this->__isset.max_parts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partition_names_ps_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_names_ps_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); + std::vector ::const_iterator _iter1369; + for (_iter1369 = this->part_vals.begin(); _iter1369 != this->part_vals.end(); ++_iter1369) + { + xfer += oprot->writeString((*_iter1369)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4); + xfer += oprot->writeI16(this->max_parts); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_names_ps_pargs::~ThriftHiveMetastore_get_partition_names_ps_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_names_ps_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_names_ps_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); + std::vector ::const_iterator _iter1370; + for (_iter1370 = (*(this->part_vals)).begin(); _iter1370 != (*(this->part_vals)).end(); ++_iter1370) + { + xfer += oprot->writeString((*_iter1370)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4); + xfer += oprot->writeI16((*(this->max_parts))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_names_ps_result::~ThriftHiveMetastore_get_partition_names_ps_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_names_ps_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1371; + ::apache::thrift::protocol::TType _etype1374; + xfer += iprot->readListBegin(_etype1374, _size1371); + this->success.resize(_size1371); + uint32_t _i1375; + for (_i1375 = 0; _i1375 < _size1371; ++_i1375) + { + xfer += iprot->readString(this->success[_i1375]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partition_names_ps_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_names_ps_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::vector ::const_iterator _iter1376; + for (_iter1376 = this->success.begin(); _iter1376 != this->success.end(); ++_iter1376) + { + xfer += oprot->writeString((*_iter1376)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_names_ps_presult::~ThriftHiveMetastore_get_partition_names_ps_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_names_ps_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1377; + ::apache::thrift::protocol::TType _etype1380; + xfer += iprot->readListBegin(_etype1380, _size1377); + (*(this->success)).resize(_size1377); + uint32_t _i1381; + for (_i1381 = 0; _i1381 < _size1377; ++_i1381) + { + xfer += iprot->readString((*(this->success))[_i1381]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_partitions_by_filter_args::~ThriftHiveMetastore_get_partitions_by_filter_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_by_filter_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->filter); + this->__isset.filter = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I16) { + xfer += iprot->readI16(this->max_parts); + this->__isset.max_parts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_by_filter_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_filter_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->filter); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4); + xfer += oprot->writeI16(this->max_parts); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_by_filter_pargs::~ThriftHiveMetastore_get_partitions_by_filter_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_by_filter_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_filter_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->filter))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4); + xfer += oprot->writeI16((*(this->max_parts))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_by_filter_result::~ThriftHiveMetastore_get_partitions_by_filter_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_by_filter_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1382; + ::apache::thrift::protocol::TType _etype1385; + xfer += iprot->readListBegin(_etype1385, _size1382); + this->success.resize(_size1382); + uint32_t _i1386; + for (_i1386 = 0; _i1386 < _size1382; ++_i1386) + { + xfer += this->success[_i1386].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_by_filter_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_filter_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter1387; + for (_iter1387 = this->success.begin(); _iter1387 != this->success.end(); ++_iter1387) + { + xfer += (*_iter1387).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_by_filter_presult::~ThriftHiveMetastore_get_partitions_by_filter_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_by_filter_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1388; + ::apache::thrift::protocol::TType _etype1391; + xfer += iprot->readListBegin(_etype1391, _size1388); + (*(this->success)).resize(_size1388); + uint32_t _i1392; + for (_i1392 = 0; _i1392 < _size1388; ++_i1392) + { + xfer += (*(this->success))[_i1392].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_part_specs_by_filter_args::~ThriftHiveMetastore_get_part_specs_by_filter_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_part_specs_by_filter_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->filter); + this->__isset.filter = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->max_parts); + this->__isset.max_parts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_part_specs_by_filter_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_part_specs_by_filter_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->filter); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32(this->max_parts); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_part_specs_by_filter_pargs::~ThriftHiveMetastore_get_part_specs_by_filter_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_part_specs_by_filter_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_part_specs_by_filter_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->filter))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32((*(this->max_parts))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_part_specs_by_filter_result::~ThriftHiveMetastore_get_part_specs_by_filter_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_part_specs_by_filter_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1393; + ::apache::thrift::protocol::TType _etype1396; + xfer += iprot->readListBegin(_etype1396, _size1393); + this->success.resize(_size1393); + uint32_t _i1397; + for (_i1397 = 0; _i1397 < _size1393; ++_i1397) + { + xfer += this->success[_i1397].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_part_specs_by_filter_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_part_specs_by_filter_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter1398; + for (_iter1398 = this->success.begin(); _iter1398 != this->success.end(); ++_iter1398) + { + xfer += (*_iter1398).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_part_specs_by_filter_presult::~ThriftHiveMetastore_get_part_specs_by_filter_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_part_specs_by_filter_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1399; + ::apache::thrift::protocol::TType _etype1402; + xfer += iprot->readListBegin(_etype1402, _size1399); + (*(this->success)).resize(_size1399); + uint32_t _i1403; + for (_i1403 = 0; _i1403 < _size1399; ++_i1403) + { + xfer += (*(this->success))[_i1403].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_partitions_by_expr_args::~ThriftHiveMetastore_get_partitions_by_expr_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_by_expr_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->req.read(iprot); + this->__isset.req = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_by_expr_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_expr_args"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->req.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_by_expr_pargs::~ThriftHiveMetastore_get_partitions_by_expr_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_by_expr_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_expr_pargs"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->req)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_by_expr_result::~ThriftHiveMetastore_get_partitions_by_expr_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_by_expr_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_by_expr_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_expr_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_by_expr_presult::~ThriftHiveMetastore_get_partitions_by_expr_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_by_expr_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_num_partitions_by_filter_args::~ThriftHiveMetastore_get_num_partitions_by_filter_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_num_partitions_by_filter_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->filter); + this->__isset.filter = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_num_partitions_by_filter_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_num_partitions_by_filter_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->filter); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_num_partitions_by_filter_pargs::~ThriftHiveMetastore_get_num_partitions_by_filter_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_num_partitions_by_filter_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_num_partitions_by_filter_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->filter))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_num_partitions_by_filter_result::~ThriftHiveMetastore_get_num_partitions_by_filter_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_num_partitions_by_filter_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_num_partitions_by_filter_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_num_partitions_by_filter_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_I32, 0); + xfer += oprot->writeI32(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_num_partitions_by_filter_presult::~ThriftHiveMetastore_get_num_partitions_by_filter_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_num_partitions_by_filter_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_partitions_by_names_args::~ThriftHiveMetastore_get_partitions_by_names_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_by_names_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->names.clear(); + uint32_t _size1404; + ::apache::thrift::protocol::TType _etype1407; + xfer += iprot->readListBegin(_etype1407, _size1404); + this->names.resize(_size1404); + uint32_t _i1408; + for (_i1408 = 0; _i1408 < _size1404; ++_i1408) + { + xfer += iprot->readString(this->names[_i1408]); + } + xfer += iprot->readListEnd(); + } + this->__isset.names = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_by_names_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_names_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->names.size())); + std::vector ::const_iterator _iter1409; + for (_iter1409 = this->names.begin(); _iter1409 != this->names.end(); ++_iter1409) + { + xfer += oprot->writeString((*_iter1409)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_by_names_pargs::~ThriftHiveMetastore_get_partitions_by_names_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_by_names_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_names_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->names)).size())); + std::vector ::const_iterator _iter1410; + for (_iter1410 = (*(this->names)).begin(); _iter1410 != (*(this->names)).end(); ++_iter1410) + { + xfer += oprot->writeString((*_iter1410)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_by_names_result::~ThriftHiveMetastore_get_partitions_by_names_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_by_names_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1411; + ::apache::thrift::protocol::TType _etype1414; + xfer += iprot->readListBegin(_etype1414, _size1411); + this->success.resize(_size1411); + uint32_t _i1415; + for (_i1415 = 0; _i1415 < _size1411; ++_i1415) + { + xfer += this->success[_i1415].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_by_names_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_names_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter1416; + for (_iter1416 = this->success.begin(); _iter1416 != this->success.end(); ++_iter1416) + { + xfer += (*_iter1416).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_by_names_presult::~ThriftHiveMetastore_get_partitions_by_names_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_by_names_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1417; + ::apache::thrift::protocol::TType _etype1420; + xfer += iprot->readListBegin(_etype1420, _size1417); + (*(this->success)).resize(_size1417); + uint32_t _i1421; + for (_i1421 = 0; _i1421 < _size1417; ++_i1421) + { + xfer += (*(this->success))[_i1421].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_alter_partition_args::~ThriftHiveMetastore_alter_partition_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_partition_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->new_part.read(iprot); + this->__isset.new_part = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_partition_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partition_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->new_part.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_partition_pargs::~ThriftHiveMetastore_alter_partition_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_partition_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partition_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += (*(this->new_part)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_partition_result::~ThriftHiveMetastore_alter_partition_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_partition_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_partition_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partition_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_partition_presult::~ThriftHiveMetastore_alter_partition_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_partition_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_alter_partitions_args::~ThriftHiveMetastore_alter_partitions_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_partitions_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->new_parts.clear(); + uint32_t _size1422; + ::apache::thrift::protocol::TType _etype1425; + xfer += iprot->readListBegin(_etype1425, _size1422); + this->new_parts.resize(_size1422); + uint32_t _i1426; + for (_i1426 = 0; _i1426 < _size1422; ++_i1426) + { + xfer += this->new_parts[_i1426].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.new_parts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_partitions_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partitions_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->new_parts.size())); + std::vector ::const_iterator _iter1427; + for (_iter1427 = this->new_parts.begin(); _iter1427 != this->new_parts.end(); ++_iter1427) + { + xfer += (*_iter1427).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_partitions_pargs::~ThriftHiveMetastore_alter_partitions_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_partitions_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partitions_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->new_parts)).size())); + std::vector ::const_iterator _iter1428; + for (_iter1428 = (*(this->new_parts)).begin(); _iter1428 != (*(this->new_parts)).end(); ++_iter1428) + { + xfer += (*_iter1428).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_partitions_result::~ThriftHiveMetastore_alter_partitions_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_partitions_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_partitions_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partitions_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_partitions_presult::~ThriftHiveMetastore_alter_partitions_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_partitions_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_alter_partitions_with_environment_context_args::~ThriftHiveMetastore_alter_partitions_with_environment_context_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->new_parts.clear(); + uint32_t _size1429; + ::apache::thrift::protocol::TType _etype1432; + xfer += iprot->readListBegin(_etype1432, _size1429); + this->new_parts.resize(_size1429); + uint32_t _i1433; + for (_i1433 = 0; _i1433 < _size1429; ++_i1433) + { + xfer += this->new_parts[_i1433].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.new_parts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->environment_context.read(iprot); + this->__isset.environment_context = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partitions_with_environment_context_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->new_parts.size())); + std::vector ::const_iterator _iter1434; + for (_iter1434 = this->new_parts.begin(); _iter1434 != this->new_parts.end(); ++_iter1434) + { + xfer += (*_iter1434).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->environment_context.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_partitions_with_environment_context_pargs::~ThriftHiveMetastore_alter_partitions_with_environment_context_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partitions_with_environment_context_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->new_parts)).size())); + std::vector ::const_iterator _iter1435; + for (_iter1435 = (*(this->new_parts)).begin(); _iter1435 != (*(this->new_parts)).end(); ++_iter1435) + { + xfer += (*_iter1435).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += (*(this->environment_context)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_partitions_with_environment_context_result::~ThriftHiveMetastore_alter_partitions_with_environment_context_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partitions_with_environment_context_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_partitions_with_environment_context_presult::~ThriftHiveMetastore_alter_partitions_with_environment_context_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_alter_partition_with_environment_context_args::~ThriftHiveMetastore_alter_partition_with_environment_context_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_partition_with_environment_context_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->new_part.read(iprot); + this->__isset.new_part = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->environment_context.read(iprot); + this->__isset.environment_context = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_partition_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partition_with_environment_context_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->new_part.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->environment_context.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_partition_with_environment_context_pargs::~ThriftHiveMetastore_alter_partition_with_environment_context_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_partition_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partition_with_environment_context_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += (*(this->new_part)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += (*(this->environment_context)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_partition_with_environment_context_result::~ThriftHiveMetastore_alter_partition_with_environment_context_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_partition_with_environment_context_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_partition_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partition_with_environment_context_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_partition_with_environment_context_presult::~ThriftHiveMetastore_alter_partition_with_environment_context_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_partition_with_environment_context_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_rename_partition_args::~ThriftHiveMetastore_rename_partition_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_rename_partition_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->part_vals.clear(); + uint32_t _size1436; + ::apache::thrift::protocol::TType _etype1439; + xfer += iprot->readListBegin(_etype1439, _size1436); + this->part_vals.resize(_size1436); + uint32_t _i1440; + for (_i1440 = 0; _i1440 < _size1436; ++_i1440) + { + xfer += iprot->readString(this->part_vals[_i1440]); + } + xfer += iprot->readListEnd(); + } + this->__isset.part_vals = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->new_part.read(iprot); + this->__isset.new_part = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_rename_partition_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_rename_partition_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); + std::vector ::const_iterator _iter1441; + for (_iter1441 = this->part_vals.begin(); _iter1441 != this->part_vals.end(); ++_iter1441) + { + xfer += oprot->writeString((*_iter1441)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->new_part.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_rename_partition_pargs::~ThriftHiveMetastore_rename_partition_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_rename_partition_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_rename_partition_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); + std::vector ::const_iterator _iter1442; + for (_iter1442 = (*(this->part_vals)).begin(); _iter1442 != (*(this->part_vals)).end(); ++_iter1442) + { + xfer += oprot->writeString((*_iter1442)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += (*(this->new_part)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_rename_partition_result::~ThriftHiveMetastore_rename_partition_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_rename_partition_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_rename_partition_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_rename_partition_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_rename_partition_presult::~ThriftHiveMetastore_rename_partition_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_rename_partition_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_partition_name_has_valid_characters_args::~ThriftHiveMetastore_partition_name_has_valid_characters_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->part_vals.clear(); + uint32_t _size1443; + ::apache::thrift::protocol::TType _etype1446; + xfer += iprot->readListBegin(_etype1446, _size1443); + this->part_vals.resize(_size1443); + uint32_t _i1447; + for (_i1447 = 0; _i1447 < _size1443; ++_i1447) + { + xfer += iprot->readString(this->part_vals[_i1447]); + } + xfer += iprot->readListEnd(); + } + this->__isset.part_vals = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->throw_exception); + this->__isset.throw_exception = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_has_valid_characters_args"); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); + std::vector ::const_iterator _iter1448; + for (_iter1448 = this->part_vals.begin(); _iter1448 != this->part_vals.end(); ++_iter1448) + { + xfer += oprot->writeString((*_iter1448)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("throw_exception", ::apache::thrift::protocol::T_BOOL, 2); + xfer += oprot->writeBool(this->throw_exception); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_partition_name_has_valid_characters_pargs::~ThriftHiveMetastore_partition_name_has_valid_characters_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_has_valid_characters_pargs"); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); + std::vector ::const_iterator _iter1449; + for (_iter1449 = (*(this->part_vals)).begin(); _iter1449 != (*(this->part_vals)).end(); ++_iter1449) + { + xfer += oprot->writeString((*_iter1449)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("throw_exception", ::apache::thrift::protocol::T_BOOL, 2); + xfer += oprot->writeBool((*(this->throw_exception))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_partition_name_has_valid_characters_result::~ThriftHiveMetastore_partition_name_has_valid_characters_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_has_valid_characters_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_partition_name_has_valid_characters_presult::~ThriftHiveMetastore_partition_name_has_valid_characters_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_config_value_args::~ThriftHiveMetastore_get_config_value_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_config_value_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->name); + this->__isset.name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->defaultValue); + this->__isset.defaultValue = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_config_value_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_config_value_args"); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("defaultValue", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->defaultValue); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_config_value_pargs::~ThriftHiveMetastore_get_config_value_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_config_value_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_config_value_pargs"); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("defaultValue", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->defaultValue))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_config_value_result::~ThriftHiveMetastore_get_config_value_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_config_value_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_config_value_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_config_value_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0); + xfer += oprot->writeString(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_config_value_presult::~ThriftHiveMetastore_get_config_value_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_config_value_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_partition_name_to_vals_args::~ThriftHiveMetastore_partition_name_to_vals_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_partition_name_to_vals_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->part_name); + this->__isset.part_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_partition_name_to_vals_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_to_vals_args"); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->part_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_partition_name_to_vals_pargs::~ThriftHiveMetastore_partition_name_to_vals_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_partition_name_to_vals_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_to_vals_pargs"); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->part_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_partition_name_to_vals_result::~ThriftHiveMetastore_partition_name_to_vals_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_partition_name_to_vals_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1450; + ::apache::thrift::protocol::TType _etype1453; + xfer += iprot->readListBegin(_etype1453, _size1450); + this->success.resize(_size1450); + uint32_t _i1454; + for (_i1454 = 0; _i1454 < _size1450; ++_i1454) + { + xfer += iprot->readString(this->success[_i1454]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_partition_name_to_vals_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_to_vals_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::vector ::const_iterator _iter1455; + for (_iter1455 = this->success.begin(); _iter1455 != this->success.end(); ++_iter1455) + { + xfer += oprot->writeString((*_iter1455)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_partition_name_to_vals_presult::~ThriftHiveMetastore_partition_name_to_vals_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_partition_name_to_vals_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1456; + ::apache::thrift::protocol::TType _etype1459; + xfer += iprot->readListBegin(_etype1459, _size1456); + (*(this->success)).resize(_size1456); + uint32_t _i1460; + for (_i1460 = 0; _i1460 < _size1456; ++_i1460) + { + xfer += iprot->readString((*(this->success))[_i1460]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_partition_name_to_spec_args::~ThriftHiveMetastore_partition_name_to_spec_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_partition_name_to_spec_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->part_name); + this->__isset.part_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_partition_name_to_spec_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_to_spec_args"); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->part_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_partition_name_to_spec_pargs::~ThriftHiveMetastore_partition_name_to_spec_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_partition_name_to_spec_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_to_spec_pargs"); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->part_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_partition_name_to_spec_result::~ThriftHiveMetastore_partition_name_to_spec_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_partition_name_to_spec_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->success.clear(); + uint32_t _size1461; + ::apache::thrift::protocol::TType _ktype1462; + ::apache::thrift::protocol::TType _vtype1463; + xfer += iprot->readMapBegin(_ktype1462, _vtype1463, _size1461); + uint32_t _i1465; + for (_i1465 = 0; _i1465 < _size1461; ++_i1465) + { + std::string _key1466; + xfer += iprot->readString(_key1466); + std::string& _val1467 = this->success[_key1466]; + xfer += iprot->readString(_val1467); + } + xfer += iprot->readMapEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_partition_name_to_spec_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_to_spec_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::map ::const_iterator _iter1468; + for (_iter1468 = this->success.begin(); _iter1468 != this->success.end(); ++_iter1468) + { + xfer += oprot->writeString(_iter1468->first); + xfer += oprot->writeString(_iter1468->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_partition_name_to_spec_presult::~ThriftHiveMetastore_partition_name_to_spec_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_partition_name_to_spec_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + (*(this->success)).clear(); + uint32_t _size1469; + ::apache::thrift::protocol::TType _ktype1470; + ::apache::thrift::protocol::TType _vtype1471; + xfer += iprot->readMapBegin(_ktype1470, _vtype1471, _size1469); + uint32_t _i1473; + for (_i1473 = 0; _i1473 < _size1469; ++_i1473) + { + std::string _key1474; + xfer += iprot->readString(_key1474); + std::string& _val1475 = (*(this->success))[_key1474]; + xfer += iprot->readString(_val1475); + } + xfer += iprot->readMapEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_markPartitionForEvent_args::~ThriftHiveMetastore_markPartitionForEvent_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_markPartitionForEvent_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->part_vals.clear(); + uint32_t _size1476; + ::apache::thrift::protocol::TType _ktype1477; + ::apache::thrift::protocol::TType _vtype1478; + xfer += iprot->readMapBegin(_ktype1477, _vtype1478, _size1476); + uint32_t _i1480; + for (_i1480 = 0; _i1480 < _size1476; ++_i1480) + { + std::string _key1481; + xfer += iprot->readString(_key1481); + std::string& _val1482 = this->part_vals[_key1481]; + xfer += iprot->readString(_val1482); + } + xfer += iprot->readMapEnd(); + } + this->__isset.part_vals = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast1483; + xfer += iprot->readI32(ecast1483); + this->eventType = (PartitionEventType::type)ecast1483; + this->__isset.eventType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_markPartitionForEvent_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_markPartitionForEvent_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); + std::map ::const_iterator _iter1484; + for (_iter1484 = this->part_vals.begin(); _iter1484 != this->part_vals.end(); ++_iter1484) + { + xfer += oprot->writeString(_iter1484->first); + xfer += oprot->writeString(_iter1484->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("eventType", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32((int32_t)this->eventType); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_markPartitionForEvent_pargs::~ThriftHiveMetastore_markPartitionForEvent_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_markPartitionForEvent_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_markPartitionForEvent_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); + std::map ::const_iterator _iter1485; + for (_iter1485 = (*(this->part_vals)).begin(); _iter1485 != (*(this->part_vals)).end(); ++_iter1485) + { + xfer += oprot->writeString(_iter1485->first); + xfer += oprot->writeString(_iter1485->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("eventType", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32((int32_t)(*(this->eventType))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_markPartitionForEvent_result::~ThriftHiveMetastore_markPartitionForEvent_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_markPartitionForEvent_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o5.read(iprot); + this->__isset.o5 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o6.read(iprot); + this->__isset.o6 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_markPartitionForEvent_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_markPartitionForEvent_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o5) { + xfer += oprot->writeFieldBegin("o5", ::apache::thrift::protocol::T_STRUCT, 5); + xfer += this->o5.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o6) { + xfer += oprot->writeFieldBegin("o6", ::apache::thrift::protocol::T_STRUCT, 6); + xfer += this->o6.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_markPartitionForEvent_presult::~ThriftHiveMetastore_markPartitionForEvent_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_markPartitionForEvent_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o5.read(iprot); + this->__isset.o5 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o6.read(iprot); + this->__isset.o6 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_isPartitionMarkedForEvent_args::~ThriftHiveMetastore_isPartitionMarkedForEvent_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->part_vals.clear(); + uint32_t _size1486; + ::apache::thrift::protocol::TType _ktype1487; + ::apache::thrift::protocol::TType _vtype1488; + xfer += iprot->readMapBegin(_ktype1487, _vtype1488, _size1486); + uint32_t _i1490; + for (_i1490 = 0; _i1490 < _size1486; ++_i1490) + { + std::string _key1491; + xfer += iprot->readString(_key1491); + std::string& _val1492 = this->part_vals[_key1491]; + xfer += iprot->readString(_val1492); + } + xfer += iprot->readMapEnd(); + } + this->__isset.part_vals = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast1493; + xfer += iprot->readI32(ecast1493); + this->eventType = (PartitionEventType::type)ecast1493; + this->__isset.eventType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_isPartitionMarkedForEvent_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); + std::map ::const_iterator _iter1494; + for (_iter1494 = this->part_vals.begin(); _iter1494 != this->part_vals.end(); ++_iter1494) + { + xfer += oprot->writeString(_iter1494->first); + xfer += oprot->writeString(_iter1494->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("eventType", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32((int32_t)this->eventType); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_isPartitionMarkedForEvent_pargs::~ThriftHiveMetastore_isPartitionMarkedForEvent_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_isPartitionMarkedForEvent_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); + std::map ::const_iterator _iter1495; + for (_iter1495 = (*(this->part_vals)).begin(); _iter1495 != (*(this->part_vals)).end(); ++_iter1495) + { + xfer += oprot->writeString(_iter1495->first); + xfer += oprot->writeString(_iter1495->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("eventType", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32((int32_t)(*(this->eventType))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_isPartitionMarkedForEvent_result::~ThriftHiveMetastore_isPartitionMarkedForEvent_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o5.read(iprot); + this->__isset.o5 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o6.read(iprot); + this->__isset.o6 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_isPartitionMarkedForEvent_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o5) { + xfer += oprot->writeFieldBegin("o5", ::apache::thrift::protocol::T_STRUCT, 5); + xfer += this->o5.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o6) { + xfer += oprot->writeFieldBegin("o6", ::apache::thrift::protocol::T_STRUCT, 6); + xfer += this->o6.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_isPartitionMarkedForEvent_presult::~ThriftHiveMetastore_isPartitionMarkedForEvent_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o5.read(iprot); + this->__isset.o5 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o6.read(iprot); + this->__isset.o6 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_add_index_args::~ThriftHiveMetastore_add_index_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_index_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->new_index.read(iprot); + this->__isset.new_index = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->index_table.read(iprot); + this->__isset.index_table = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_index_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_index_args"); + + xfer += oprot->writeFieldBegin("new_index", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->new_index.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("index_table", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->index_table.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_index_pargs::~ThriftHiveMetastore_add_index_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_index_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_index_pargs"); + + xfer += oprot->writeFieldBegin("new_index", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->new_index)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("index_table", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += (*(this->index_table)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_index_result::~ThriftHiveMetastore_add_index_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_index_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_index_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_index_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_index_presult::~ThriftHiveMetastore_add_index_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_index_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_alter_index_args::~ThriftHiveMetastore_alter_index_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_index_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbname); + this->__isset.dbname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->base_tbl_name); + this->__isset.base_tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->idx_name); + this->__isset.idx_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->new_idx.read(iprot); + this->__isset.new_idx = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_index_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_index_args"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbname); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("base_tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->base_tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("idx_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->idx_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_idx", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->new_idx.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_index_pargs::~ThriftHiveMetastore_alter_index_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_index_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_index_pargs"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbname))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("base_tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->base_tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("idx_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->idx_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("new_idx", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += (*(this->new_idx)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_index_result::~ThriftHiveMetastore_alter_index_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_index_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_index_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_index_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_index_presult::~ThriftHiveMetastore_alter_index_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_index_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_drop_index_by_name_args::~ThriftHiveMetastore_drop_index_by_name_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_index_by_name_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->index_name); + this->__isset.index_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->deleteData); + this->__isset.deleteData = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_index_by_name_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_index_by_name_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("index_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->index_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool(this->deleteData); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_index_by_name_pargs::~ThriftHiveMetastore_drop_index_by_name_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_index_by_name_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_index_by_name_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("index_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->index_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool((*(this->deleteData))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_index_by_name_result::~ThriftHiveMetastore_drop_index_by_name_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_index_by_name_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_index_by_name_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_index_by_name_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_index_by_name_presult::~ThriftHiveMetastore_drop_index_by_name_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_index_by_name_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_index_by_name_args::~ThriftHiveMetastore_get_index_by_name_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_index_by_name_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->index_name); + this->__isset.index_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_index_by_name_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_index_by_name_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("index_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->index_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_index_by_name_pargs::~ThriftHiveMetastore_get_index_by_name_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_index_by_name_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_index_by_name_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("index_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->index_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_index_by_name_result::~ThriftHiveMetastore_get_index_by_name_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_index_by_name_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_index_by_name_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_index_by_name_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_index_by_name_presult::~ThriftHiveMetastore_get_index_by_name_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_index_by_name_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_indexes_args::~ThriftHiveMetastore_get_indexes_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_indexes_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I16) { + xfer += iprot->readI16(this->max_indexes); + this->__isset.max_indexes = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_indexes_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_indexes_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_indexes", ::apache::thrift::protocol::T_I16, 3); + xfer += oprot->writeI16(this->max_indexes); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_indexes_pargs::~ThriftHiveMetastore_get_indexes_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_indexes_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_indexes_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_indexes", ::apache::thrift::protocol::T_I16, 3); + xfer += oprot->writeI16((*(this->max_indexes))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_indexes_result::~ThriftHiveMetastore_get_indexes_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_indexes_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1496; + ::apache::thrift::protocol::TType _etype1499; + xfer += iprot->readListBegin(_etype1499, _size1496); + this->success.resize(_size1496); + uint32_t _i1500; + for (_i1500 = 0; _i1500 < _size1496; ++_i1500) + { + xfer += this->success[_i1500].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_indexes_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_indexes_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter1501; + for (_iter1501 = this->success.begin(); _iter1501 != this->success.end(); ++_iter1501) + { + xfer += (*_iter1501).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_indexes_presult::~ThriftHiveMetastore_get_indexes_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_indexes_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1502; + ::apache::thrift::protocol::TType _etype1505; + xfer += iprot->readListBegin(_etype1505, _size1502); + (*(this->success)).resize(_size1502); + uint32_t _i1506; + for (_i1506 = 0; _i1506 < _size1502; ++_i1506) + { + xfer += (*(this->success))[_i1506].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_index_names_args::~ThriftHiveMetastore_get_index_names_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_index_names_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I16) { + xfer += iprot->readI16(this->max_indexes); + this->__isset.max_indexes = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_index_names_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_index_names_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_indexes", ::apache::thrift::protocol::T_I16, 3); + xfer += oprot->writeI16(this->max_indexes); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_index_names_pargs::~ThriftHiveMetastore_get_index_names_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_index_names_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_index_names_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max_indexes", ::apache::thrift::protocol::T_I16, 3); + xfer += oprot->writeI16((*(this->max_indexes))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_index_names_result::~ThriftHiveMetastore_get_index_names_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_index_names_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1507; + ::apache::thrift::protocol::TType _etype1510; + xfer += iprot->readListBegin(_etype1510, _size1507); + this->success.resize(_size1507); + uint32_t _i1511; + for (_i1511 = 0; _i1511 < _size1507; ++_i1511) + { + xfer += iprot->readString(this->success[_i1511]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_index_names_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_index_names_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::vector ::const_iterator _iter1512; + for (_iter1512 = this->success.begin(); _iter1512 != this->success.end(); ++_iter1512) + { + xfer += oprot->writeString((*_iter1512)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_index_names_presult::~ThriftHiveMetastore_get_index_names_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_index_names_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1513; + ::apache::thrift::protocol::TType _etype1516; + xfer += iprot->readListBegin(_etype1516, _size1513); + (*(this->success)).resize(_size1513); + uint32_t _i1517; + for (_i1517 = 0; _i1517 < _size1513; ++_i1517) + { + xfer += iprot->readString((*(this->success))[_i1517]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_primary_keys_args::~ThriftHiveMetastore_get_primary_keys_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_primary_keys_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_primary_keys_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_primary_keys_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_primary_keys_pargs::~ThriftHiveMetastore_get_primary_keys_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_primary_keys_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_primary_keys_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_primary_keys_result::~ThriftHiveMetastore_get_primary_keys_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_primary_keys_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_primary_keys_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_primary_keys_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_primary_keys_presult::~ThriftHiveMetastore_get_primary_keys_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_primary_keys_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_foreign_keys_args::~ThriftHiveMetastore_get_foreign_keys_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_foreign_keys_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_foreign_keys_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_foreign_keys_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_foreign_keys_pargs::~ThriftHiveMetastore_get_foreign_keys_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_foreign_keys_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_foreign_keys_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_foreign_keys_result::~ThriftHiveMetastore_get_foreign_keys_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_foreign_keys_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_foreign_keys_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_foreign_keys_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_foreign_keys_presult::~ThriftHiveMetastore_get_foreign_keys_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_foreign_keys_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_unique_constraints_args::~ThriftHiveMetastore_get_unique_constraints_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_unique_constraints_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_unique_constraints_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_unique_constraints_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_unique_constraints_pargs::~ThriftHiveMetastore_get_unique_constraints_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_unique_constraints_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_unique_constraints_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_unique_constraints_result::~ThriftHiveMetastore_get_unique_constraints_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_unique_constraints_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_unique_constraints_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_unique_constraints_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_unique_constraints_presult::~ThriftHiveMetastore_get_unique_constraints_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_unique_constraints_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_not_null_constraints_args::~ThriftHiveMetastore_get_not_null_constraints_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_not_null_constraints_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_not_null_constraints_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_not_null_constraints_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_not_null_constraints_pargs::~ThriftHiveMetastore_get_not_null_constraints_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_not_null_constraints_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_not_null_constraints_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_not_null_constraints_result::~ThriftHiveMetastore_get_not_null_constraints_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_not_null_constraints_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_not_null_constraints_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_not_null_constraints_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_not_null_constraints_presult::~ThriftHiveMetastore_get_not_null_constraints_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_not_null_constraints_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_update_table_column_statistics_args::~ThriftHiveMetastore_update_table_column_statistics_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_update_table_column_statistics_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->stats_obj.read(iprot); + this->__isset.stats_obj = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_update_table_column_statistics_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_table_column_statistics_args"); + + xfer += oprot->writeFieldBegin("stats_obj", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->stats_obj.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_update_table_column_statistics_pargs::~ThriftHiveMetastore_update_table_column_statistics_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_update_table_column_statistics_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_table_column_statistics_pargs"); + + xfer += oprot->writeFieldBegin("stats_obj", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->stats_obj)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_update_table_column_statistics_result::~ThriftHiveMetastore_update_table_column_statistics_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_update_table_column_statistics_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_update_table_column_statistics_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_table_column_statistics_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_update_table_column_statistics_presult::~ThriftHiveMetastore_update_table_column_statistics_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_update_table_column_statistics_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_update_partition_column_statistics_args::~ThriftHiveMetastore_update_partition_column_statistics_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_update_partition_column_statistics_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->stats_obj.read(iprot); + this->__isset.stats_obj = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_update_partition_column_statistics_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_partition_column_statistics_args"); + + xfer += oprot->writeFieldBegin("stats_obj", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->stats_obj.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_update_partition_column_statistics_pargs::~ThriftHiveMetastore_update_partition_column_statistics_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_update_partition_column_statistics_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_partition_column_statistics_pargs"); + + xfer += oprot->writeFieldBegin("stats_obj", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->stats_obj)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_update_partition_column_statistics_result::~ThriftHiveMetastore_update_partition_column_statistics_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_update_partition_column_statistics_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_update_partition_column_statistics_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_partition_column_statistics_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_update_partition_column_statistics_presult::~ThriftHiveMetastore_update_partition_column_statistics_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_update_partition_column_statistics_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_table_column_statistics_args::~ThriftHiveMetastore_get_table_column_statistics_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_column_statistics_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->col_name); + this->__isset.col_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_table_column_statistics_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_column_statistics_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("col_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->col_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_column_statistics_pargs::~ThriftHiveMetastore_get_table_column_statistics_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_column_statistics_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_column_statistics_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("col_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->col_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_column_statistics_result::~ThriftHiveMetastore_get_table_column_statistics_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_column_statistics_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_table_column_statistics_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_column_statistics_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_column_statistics_presult::~ThriftHiveMetastore_get_table_column_statistics_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_column_statistics_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_partition_column_statistics_args::~ThriftHiveMetastore_get_partition_column_statistics_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_column_statistics_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->part_name); + this->__isset.part_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->col_name); + this->__isset.col_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partition_column_statistics_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_column_statistics_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->part_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("col_name", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->col_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_column_statistics_pargs::~ThriftHiveMetastore_get_partition_column_statistics_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_column_statistics_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_column_statistics_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->part_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("col_name", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString((*(this->col_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_column_statistics_result::~ThriftHiveMetastore_get_partition_column_statistics_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_column_statistics_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partition_column_statistics_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_column_statistics_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partition_column_statistics_presult::~ThriftHiveMetastore_get_partition_column_statistics_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partition_column_statistics_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_table_statistics_req_args::~ThriftHiveMetastore_get_table_statistics_req_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_statistics_req_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_table_statistics_req_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_statistics_req_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_statistics_req_pargs::~ThriftHiveMetastore_get_table_statistics_req_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_statistics_req_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_statistics_req_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_statistics_req_result::~ThriftHiveMetastore_get_table_statistics_req_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_statistics_req_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_table_statistics_req_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_statistics_req_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_table_statistics_req_presult::~ThriftHiveMetastore_get_table_statistics_req_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_table_statistics_req_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_partitions_statistics_req_args::~ThriftHiveMetastore_get_partitions_statistics_req_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_statistics_req_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_statistics_req_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_statistics_req_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_statistics_req_pargs::~ThriftHiveMetastore_get_partitions_statistics_req_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_statistics_req_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_statistics_req_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_statistics_req_result::~ThriftHiveMetastore_get_partitions_statistics_req_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_statistics_req_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_partitions_statistics_req_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_statistics_req_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_partitions_statistics_req_presult::~ThriftHiveMetastore_get_partitions_statistics_req_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_partitions_statistics_req_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_aggr_stats_for_args::~ThriftHiveMetastore_get_aggr_stats_for_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_aggr_stats_for_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_aggr_stats_for_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_aggr_stats_for_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_aggr_stats_for_pargs::~ThriftHiveMetastore_get_aggr_stats_for_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_aggr_stats_for_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_aggr_stats_for_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_aggr_stats_for_result::~ThriftHiveMetastore_get_aggr_stats_for_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_aggr_stats_for_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_aggr_stats_for_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_aggr_stats_for_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_aggr_stats_for_presult::~ThriftHiveMetastore_get_aggr_stats_for_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_aggr_stats_for_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_set_aggr_stats_for_args::~ThriftHiveMetastore_set_aggr_stats_for_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_set_aggr_stats_for_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_set_aggr_stats_for_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_aggr_stats_for_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_set_aggr_stats_for_pargs::~ThriftHiveMetastore_set_aggr_stats_for_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_set_aggr_stats_for_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_aggr_stats_for_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_set_aggr_stats_for_result::~ThriftHiveMetastore_set_aggr_stats_for_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_set_aggr_stats_for_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_set_aggr_stats_for_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_aggr_stats_for_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_set_aggr_stats_for_presult::~ThriftHiveMetastore_set_aggr_stats_for_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_set_aggr_stats_for_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_delete_partition_column_statistics_args::~ThriftHiveMetastore_delete_partition_column_statistics_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_delete_partition_column_statistics_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->part_name); + this->__isset.part_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->col_name); + this->__isset.col_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_delete_partition_column_statistics_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_delete_partition_column_statistics_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->part_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("col_name", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->col_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_delete_partition_column_statistics_pargs::~ThriftHiveMetastore_delete_partition_column_statistics_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_delete_partition_column_statistics_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_delete_partition_column_statistics_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->part_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("col_name", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString((*(this->col_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_delete_partition_column_statistics_result::~ThriftHiveMetastore_delete_partition_column_statistics_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_delete_partition_column_statistics_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_delete_partition_column_statistics_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_delete_partition_column_statistics_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_delete_partition_column_statistics_presult::~ThriftHiveMetastore_delete_partition_column_statistics_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_delete_partition_column_statistics_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_delete_table_column_statistics_args::~ThriftHiveMetastore_delete_table_column_statistics_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_delete_table_column_statistics_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->db_name); + this->__isset.db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + this->__isset.tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->col_name); + this->__isset.col_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_delete_table_column_statistics_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_delete_table_column_statistics_args"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("col_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->col_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_delete_table_column_statistics_pargs::~ThriftHiveMetastore_delete_table_column_statistics_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_delete_table_column_statistics_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_delete_table_column_statistics_pargs"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->db_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->tbl_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("col_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString((*(this->col_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_delete_table_column_statistics_result::~ThriftHiveMetastore_delete_table_column_statistics_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_delete_table_column_statistics_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_delete_table_column_statistics_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_delete_table_column_statistics_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_delete_table_column_statistics_presult::~ThriftHiveMetastore_delete_table_column_statistics_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_delete_table_column_statistics_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_create_function_args::~ThriftHiveMetastore_create_function_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_function_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->func.read(iprot); + this->__isset.func = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_function_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_function_args"); + + xfer += oprot->writeFieldBegin("func", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->func.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_function_pargs::~ThriftHiveMetastore_create_function_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_function_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_function_pargs"); + + xfer += oprot->writeFieldBegin("func", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->func)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_function_result::~ThriftHiveMetastore_create_function_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_function_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_function_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_function_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_function_presult::~ThriftHiveMetastore_create_function_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_function_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_drop_function_args::~ThriftHiveMetastore_drop_function_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_function_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->funcName); + this->__isset.funcName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_function_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_function_args"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("funcName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->funcName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_function_pargs::~ThriftHiveMetastore_drop_function_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_function_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_function_pargs"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbName))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("funcName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->funcName))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_function_result::~ThriftHiveMetastore_drop_function_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_function_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_function_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_function_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_function_presult::~ThriftHiveMetastore_drop_function_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_function_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_alter_function_args::~ThriftHiveMetastore_alter_function_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_function_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->funcName); + this->__isset.funcName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->newFunc.read(iprot); + this->__isset.newFunc = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_function_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_function_args"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("funcName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->funcName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("newFunc", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->newFunc.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_function_pargs::~ThriftHiveMetastore_alter_function_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_function_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_function_pargs"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbName))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("funcName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->funcName))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("newFunc", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += (*(this->newFunc)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_function_result::~ThriftHiveMetastore_alter_function_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_function_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_function_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_function_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_function_presult::~ThriftHiveMetastore_alter_function_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_function_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_functions_args::~ThriftHiveMetastore_get_functions_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_functions_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->pattern); + this->__isset.pattern = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_functions_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_functions_args"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->pattern); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_functions_pargs::~ThriftHiveMetastore_get_functions_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_functions_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_functions_pargs"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbName))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->pattern))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_functions_result::~ThriftHiveMetastore_get_functions_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_functions_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1518; + ::apache::thrift::protocol::TType _etype1521; + xfer += iprot->readListBegin(_etype1521, _size1518); + this->success.resize(_size1518); + uint32_t _i1522; + for (_i1522 = 0; _i1522 < _size1518; ++_i1522) + { + xfer += iprot->readString(this->success[_i1522]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_functions_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_functions_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::vector ::const_iterator _iter1523; + for (_iter1523 = this->success.begin(); _iter1523 != this->success.end(); ++_iter1523) + { + xfer += oprot->writeString((*_iter1523)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_functions_presult::~ThriftHiveMetastore_get_functions_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_functions_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1524; + ::apache::thrift::protocol::TType _etype1527; + xfer += iprot->readListBegin(_etype1527, _size1524); + (*(this->success)).resize(_size1524); + uint32_t _i1528; + for (_i1528 = 0; _i1528 < _size1524; ++_i1528) + { + xfer += iprot->readString((*(this->success))[_i1528]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_function_args::~ThriftHiveMetastore_get_function_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_function_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->funcName); + this->__isset.funcName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_function_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_function_args"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("funcName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->funcName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_function_pargs::~ThriftHiveMetastore_get_function_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_function_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_function_pargs"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->dbName))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("funcName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->funcName))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_function_result::~ThriftHiveMetastore_get_function_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_function_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_function_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_function_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_function_presult::~ThriftHiveMetastore_get_function_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_function_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_all_functions_args::~ThriftHiveMetastore_get_all_functions_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_functions_args::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 ThriftHiveMetastore_get_all_functions_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_functions_args"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_all_functions_pargs::~ThriftHiveMetastore_get_all_functions_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_functions_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_functions_pargs"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_all_functions_result::~ThriftHiveMetastore_get_all_functions_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_functions_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_all_functions_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_functions_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_all_functions_presult::~ThriftHiveMetastore_get_all_functions_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_functions_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_create_role_args::~ThriftHiveMetastore_create_role_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_role_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->role.read(iprot); + this->__isset.role = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_role_args"); + + xfer += oprot->writeFieldBegin("role", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->role.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_role_pargs::~ThriftHiveMetastore_create_role_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_role_pargs"); + + xfer += oprot->writeFieldBegin("role", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->role)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_role_result::~ThriftHiveMetastore_create_role_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_role_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_role_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_role_presult::~ThriftHiveMetastore_create_role_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_role_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_drop_role_args::~ThriftHiveMetastore_drop_role_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_role_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->role_name); + this->__isset.role_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_role_args"); + + xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->role_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_role_pargs::~ThriftHiveMetastore_drop_role_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_role_pargs"); + + xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->role_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_role_result::~ThriftHiveMetastore_drop_role_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_role_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_role_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_role_presult::~ThriftHiveMetastore_drop_role_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_role_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_role_names_args::~ThriftHiveMetastore_get_role_names_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_role_names_args::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 ThriftHiveMetastore_get_role_names_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_names_args"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_role_names_pargs::~ThriftHiveMetastore_get_role_names_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_role_names_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_names_pargs"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_role_names_result::~ThriftHiveMetastore_get_role_names_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_role_names_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1529; + ::apache::thrift::protocol::TType _etype1532; + xfer += iprot->readListBegin(_etype1532, _size1529); + this->success.resize(_size1529); + uint32_t _i1533; + for (_i1533 = 0; _i1533 < _size1529; ++_i1533) + { + xfer += iprot->readString(this->success[_i1533]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_role_names_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_names_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::vector ::const_iterator _iter1534; + for (_iter1534 = this->success.begin(); _iter1534 != this->success.end(); ++_iter1534) + { + xfer += oprot->writeString((*_iter1534)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_role_names_presult::~ThriftHiveMetastore_get_role_names_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_role_names_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1535; + ::apache::thrift::protocol::TType _etype1538; + xfer += iprot->readListBegin(_etype1538, _size1535); + (*(this->success)).resize(_size1535); + uint32_t _i1539; + for (_i1539 = 0; _i1539 < _size1535; ++_i1539) + { + xfer += iprot->readString((*(this->success))[_i1539]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_grant_role_args::~ThriftHiveMetastore_grant_role_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_grant_role_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->role_name); + this->__isset.role_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->principal_name); + this->__isset.principal_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast1540; + xfer += iprot->readI32(ecast1540); + this->principal_type = (PrincipalType::type)ecast1540; + this->__isset.principal_type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->grantor); + this->__isset.grantor = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast1541; + xfer += iprot->readI32(ecast1541); + this->grantorType = (PrincipalType::type)ecast1541; + this->__isset.grantorType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->grant_option); + this->__isset.grant_option = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_grant_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_role_args"); + + xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->role_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->principal_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 3); + xfer += oprot->writeI32((int32_t)this->principal_type); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grantor", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->grantor); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grantorType", ::apache::thrift::protocol::T_I32, 5); + xfer += oprot->writeI32((int32_t)this->grantorType); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grant_option", ::apache::thrift::protocol::T_BOOL, 6); + xfer += oprot->writeBool(this->grant_option); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_grant_role_pargs::~ThriftHiveMetastore_grant_role_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_grant_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_role_pargs"); + + xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->role_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->principal_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 3); + xfer += oprot->writeI32((int32_t)(*(this->principal_type))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grantor", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString((*(this->grantor))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grantorType", ::apache::thrift::protocol::T_I32, 5); + xfer += oprot->writeI32((int32_t)(*(this->grantorType))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grant_option", ::apache::thrift::protocol::T_BOOL, 6); + xfer += oprot->writeBool((*(this->grant_option))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_grant_role_result::~ThriftHiveMetastore_grant_role_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_grant_role_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_grant_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_role_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_grant_role_presult::~ThriftHiveMetastore_grant_role_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_grant_role_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_revoke_role_args::~ThriftHiveMetastore_revoke_role_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_revoke_role_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->role_name); + this->__isset.role_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->principal_name); + this->__isset.principal_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast1542; + xfer += iprot->readI32(ecast1542); + this->principal_type = (PrincipalType::type)ecast1542; + this->__isset.principal_type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_revoke_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_role_args"); + + xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->role_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->principal_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 3); + xfer += oprot->writeI32((int32_t)this->principal_type); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_revoke_role_pargs::~ThriftHiveMetastore_revoke_role_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_revoke_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_role_pargs"); + + xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->role_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->principal_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 3); + xfer += oprot->writeI32((int32_t)(*(this->principal_type))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_revoke_role_result::~ThriftHiveMetastore_revoke_role_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_revoke_role_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_revoke_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_role_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_revoke_role_presult::~ThriftHiveMetastore_revoke_role_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_revoke_role_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_list_roles_args::~ThriftHiveMetastore_list_roles_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_list_roles_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->principal_name); + this->__isset.principal_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast1543; + xfer += iprot->readI32(ecast1543); + this->principal_type = (PrincipalType::type)ecast1543; + this->__isset.principal_type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_list_roles_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_roles_args"); + + xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->principal_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32((int32_t)this->principal_type); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_list_roles_pargs::~ThriftHiveMetastore_list_roles_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_list_roles_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_roles_pargs"); + + xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->principal_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32((int32_t)(*(this->principal_type))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_list_roles_result::~ThriftHiveMetastore_list_roles_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_list_roles_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1544; + ::apache::thrift::protocol::TType _etype1547; + xfer += iprot->readListBegin(_etype1547, _size1544); + this->success.resize(_size1544); + uint32_t _i1548; + for (_i1548 = 0; _i1548 < _size1544; ++_i1548) + { + xfer += this->success[_i1548].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_list_roles_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_roles_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter1549; + for (_iter1549 = this->success.begin(); _iter1549 != this->success.end(); ++_iter1549) + { + xfer += (*_iter1549).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_list_roles_presult::~ThriftHiveMetastore_list_roles_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_list_roles_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1550; + ::apache::thrift::protocol::TType _etype1553; + xfer += iprot->readListBegin(_etype1553, _size1550); + (*(this->success)).resize(_size1550); + uint32_t _i1554; + for (_i1554 = 0; _i1554 < _size1550; ++_i1554) + { + xfer += (*(this->success))[_i1554].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_grant_revoke_role_args::~ThriftHiveMetastore_grant_revoke_role_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_grant_revoke_role_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_grant_revoke_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_revoke_role_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_grant_revoke_role_pargs::~ThriftHiveMetastore_grant_revoke_role_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_grant_revoke_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_revoke_role_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_grant_revoke_role_result::~ThriftHiveMetastore_grant_revoke_role_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_grant_revoke_role_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_grant_revoke_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_revoke_role_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_grant_revoke_role_presult::~ThriftHiveMetastore_grant_revoke_role_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_grant_revoke_role_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_principals_in_role_args::~ThriftHiveMetastore_get_principals_in_role_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_principals_in_role_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_principals_in_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_principals_in_role_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_principals_in_role_pargs::~ThriftHiveMetastore_get_principals_in_role_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_principals_in_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_principals_in_role_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_principals_in_role_result::~ThriftHiveMetastore_get_principals_in_role_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_principals_in_role_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_principals_in_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_principals_in_role_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_principals_in_role_presult::~ThriftHiveMetastore_get_principals_in_role_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_principals_in_role_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_role_grants_for_principal_args::~ThriftHiveMetastore_get_role_grants_for_principal_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_role_grants_for_principal_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_role_grants_for_principal_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_grants_for_principal_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_role_grants_for_principal_pargs::~ThriftHiveMetastore_get_role_grants_for_principal_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_role_grants_for_principal_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_grants_for_principal_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_role_grants_for_principal_result::~ThriftHiveMetastore_get_role_grants_for_principal_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_role_grants_for_principal_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_role_grants_for_principal_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_grants_for_principal_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_role_grants_for_principal_presult::~ThriftHiveMetastore_get_role_grants_for_principal_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_role_grants_for_principal_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_privilege_set_args::~ThriftHiveMetastore_get_privilege_set_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_privilege_set_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->hiveObject.read(iprot); + this->__isset.hiveObject = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->user_name); + this->__isset.user_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->group_names.clear(); + uint32_t _size1555; + ::apache::thrift::protocol::TType _etype1558; + xfer += iprot->readListBegin(_etype1558, _size1555); + this->group_names.resize(_size1555); + uint32_t _i1559; + for (_i1559 = 0; _i1559 < _size1555; ++_i1559) + { + xfer += iprot->readString(this->group_names[_i1559]); + } + xfer += iprot->readListEnd(); + } + this->__isset.group_names = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_privilege_set_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_privilege_set_args"); + + xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->hiveObject.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->user_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); + std::vector ::const_iterator _iter1560; + for (_iter1560 = this->group_names.begin(); _iter1560 != this->group_names.end(); ++_iter1560) + { + xfer += oprot->writeString((*_iter1560)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_privilege_set_pargs::~ThriftHiveMetastore_get_privilege_set_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_privilege_set_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_privilege_set_pargs"); + + xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->hiveObject)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->user_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); + std::vector ::const_iterator _iter1561; + for (_iter1561 = (*(this->group_names)).begin(); _iter1561 != (*(this->group_names)).end(); ++_iter1561) + { + xfer += oprot->writeString((*_iter1561)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_privilege_set_result::~ThriftHiveMetastore_get_privilege_set_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_privilege_set_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_privilege_set_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_privilege_set_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_privilege_set_presult::~ThriftHiveMetastore_get_privilege_set_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_privilege_set_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_list_privileges_args::~ThriftHiveMetastore_list_privileges_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_list_privileges_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->principal_name); + this->__isset.principal_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast1562; + xfer += iprot->readI32(ecast1562); + this->principal_type = (PrincipalType::type)ecast1562; + this->__isset.principal_type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->hiveObject.read(iprot); + this->__isset.hiveObject = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_list_privileges_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_privileges_args"); + + xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->principal_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32((int32_t)this->principal_type); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->hiveObject.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_list_privileges_pargs::~ThriftHiveMetastore_list_privileges_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_list_privileges_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_privileges_pargs"); + + xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->principal_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32((int32_t)(*(this->principal_type))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += (*(this->hiveObject)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_list_privileges_result::~ThriftHiveMetastore_list_privileges_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_list_privileges_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1563; + ::apache::thrift::protocol::TType _etype1566; + xfer += iprot->readListBegin(_etype1566, _size1563); + this->success.resize(_size1563); + uint32_t _i1567; + for (_i1567 = 0; _i1567 < _size1563; ++_i1567) + { + xfer += this->success[_i1567].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_list_privileges_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_privileges_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); + std::vector ::const_iterator _iter1568; + for (_iter1568 = this->success.begin(); _iter1568 != this->success.end(); ++_iter1568) + { + xfer += (*_iter1568).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_list_privileges_presult::~ThriftHiveMetastore_list_privileges_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_list_privileges_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1569; + ::apache::thrift::protocol::TType _etype1572; + xfer += iprot->readListBegin(_etype1572, _size1569); + (*(this->success)).resize(_size1569); + uint32_t _i1573; + for (_i1573 = 0; _i1573 < _size1569; ++_i1573) + { + xfer += (*(this->success))[_i1573].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_grant_privileges_args::~ThriftHiveMetastore_grant_privileges_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_grant_privileges_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->privileges.read(iprot); + this->__isset.privileges = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_grant_privileges_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_privileges_args"); + + xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->privileges.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_grant_privileges_pargs::~ThriftHiveMetastore_grant_privileges_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_grant_privileges_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_privileges_pargs"); + + xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->privileges)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_grant_privileges_result::~ThriftHiveMetastore_grant_privileges_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_grant_privileges_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_grant_privileges_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_privileges_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_grant_privileges_presult::~ThriftHiveMetastore_grant_privileges_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_grant_privileges_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_revoke_privileges_args::~ThriftHiveMetastore_revoke_privileges_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_revoke_privileges_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->privileges.read(iprot); + this->__isset.privileges = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_revoke_privileges_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_privileges_args"); + + xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->privileges.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_revoke_privileges_pargs::~ThriftHiveMetastore_revoke_privileges_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_revoke_privileges_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_privileges_pargs"); + + xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->privileges)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_revoke_privileges_result::~ThriftHiveMetastore_revoke_privileges_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_revoke_privileges_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_revoke_privileges_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_privileges_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_revoke_privileges_presult::~ThriftHiveMetastore_revoke_privileges_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_revoke_privileges_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_grant_revoke_privileges_args::~ThriftHiveMetastore_grant_revoke_privileges_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_grant_revoke_privileges_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_grant_revoke_privileges_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_revoke_privileges_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_grant_revoke_privileges_pargs::~ThriftHiveMetastore_grant_revoke_privileges_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_grant_revoke_privileges_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_revoke_privileges_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_grant_revoke_privileges_result::~ThriftHiveMetastore_grant_revoke_privileges_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_grant_revoke_privileges_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_grant_revoke_privileges_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_revoke_privileges_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_grant_revoke_privileges_presult::~ThriftHiveMetastore_grant_revoke_privileges_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_grant_revoke_privileges_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_set_ugi_args::~ThriftHiveMetastore_set_ugi_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_set_ugi_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->user_name); + this->__isset.user_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->group_names.clear(); + uint32_t _size1574; + ::apache::thrift::protocol::TType _etype1577; + xfer += iprot->readListBegin(_etype1577, _size1574); + this->group_names.resize(_size1574); + uint32_t _i1578; + for (_i1578 = 0; _i1578 < _size1574; ++_i1578) + { + xfer += iprot->readString(this->group_names[_i1578]); + } + xfer += iprot->readListEnd(); + } + this->__isset.group_names = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_set_ugi_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_ugi_args"); + + xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->user_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); + std::vector ::const_iterator _iter1579; + for (_iter1579 = this->group_names.begin(); _iter1579 != this->group_names.end(); ++_iter1579) + { + xfer += oprot->writeString((*_iter1579)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_set_ugi_pargs::~ThriftHiveMetastore_set_ugi_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_set_ugi_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_ugi_pargs"); + + xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->user_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); + std::vector ::const_iterator _iter1580; + for (_iter1580 = (*(this->group_names)).begin(); _iter1580 != (*(this->group_names)).end(); ++_iter1580) + { + xfer += oprot->writeString((*_iter1580)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_set_ugi_result::~ThriftHiveMetastore_set_ugi_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_set_ugi_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1581; + ::apache::thrift::protocol::TType _etype1584; + xfer += iprot->readListBegin(_etype1584, _size1581); + this->success.resize(_size1581); + uint32_t _i1585; + for (_i1585 = 0; _i1585 < _size1581; ++_i1585) + { + xfer += iprot->readString(this->success[_i1585]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_set_ugi_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_ugi_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::vector ::const_iterator _iter1586; + for (_iter1586 = this->success.begin(); _iter1586 != this->success.end(); ++_iter1586) + { + xfer += oprot->writeString((*_iter1586)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_set_ugi_presult::~ThriftHiveMetastore_set_ugi_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_set_ugi_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1587; + ::apache::thrift::protocol::TType _etype1590; + xfer += iprot->readListBegin(_etype1590, _size1587); + (*(this->success)).resize(_size1587); + uint32_t _i1591; + for (_i1591 = 0; _i1591 < _size1587; ++_i1591) + { + xfer += iprot->readString((*(this->success))[_i1591]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_delegation_token_args::~ThriftHiveMetastore_get_delegation_token_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_delegation_token_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->token_owner); + this->__isset.token_owner = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->renewer_kerberos_principal_name); + this->__isset.renewer_kerberos_principal_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_delegation_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_delegation_token_args"); + + xfer += oprot->writeFieldBegin("token_owner", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->token_owner); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("renewer_kerberos_principal_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->renewer_kerberos_principal_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_delegation_token_pargs::~ThriftHiveMetastore_get_delegation_token_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_delegation_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_delegation_token_pargs"); + + xfer += oprot->writeFieldBegin("token_owner", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->token_owner))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("renewer_kerberos_principal_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->renewer_kerberos_principal_name))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_delegation_token_result::~ThriftHiveMetastore_get_delegation_token_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_delegation_token_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_delegation_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_delegation_token_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0); + xfer += oprot->writeString(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_delegation_token_presult::~ThriftHiveMetastore_get_delegation_token_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_delegation_token_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_renew_delegation_token_args::~ThriftHiveMetastore_renew_delegation_token_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_renew_delegation_token_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->token_str_form); + this->__isset.token_str_form = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_renew_delegation_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_renew_delegation_token_args"); + + xfer += oprot->writeFieldBegin("token_str_form", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->token_str_form); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_renew_delegation_token_pargs::~ThriftHiveMetastore_renew_delegation_token_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_renew_delegation_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_renew_delegation_token_pargs"); + + xfer += oprot->writeFieldBegin("token_str_form", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->token_str_form))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_renew_delegation_token_result::~ThriftHiveMetastore_renew_delegation_token_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_renew_delegation_token_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_renew_delegation_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_renew_delegation_token_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_I64, 0); + xfer += oprot->writeI64(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_renew_delegation_token_presult::~ThriftHiveMetastore_renew_delegation_token_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_renew_delegation_token_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_cancel_delegation_token_args::~ThriftHiveMetastore_cancel_delegation_token_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_cancel_delegation_token_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->token_str_form); + this->__isset.token_str_form = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_cancel_delegation_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_cancel_delegation_token_args"); + + xfer += oprot->writeFieldBegin("token_str_form", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->token_str_form); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_cancel_delegation_token_pargs::~ThriftHiveMetastore_cancel_delegation_token_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_cancel_delegation_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_cancel_delegation_token_pargs"); + + xfer += oprot->writeFieldBegin("token_str_form", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->token_str_form))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_cancel_delegation_token_result::~ThriftHiveMetastore_cancel_delegation_token_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_cancel_delegation_token_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_cancel_delegation_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_cancel_delegation_token_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_cancel_delegation_token_presult::~ThriftHiveMetastore_cancel_delegation_token_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_cancel_delegation_token_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_add_token_args::~ThriftHiveMetastore_add_token_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_token_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->token_identifier); + this->__isset.token_identifier = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->delegation_token); + this->__isset.delegation_token = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_token_args"); + + xfer += oprot->writeFieldBegin("token_identifier", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->token_identifier); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("delegation_token", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->delegation_token); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_token_pargs::~ThriftHiveMetastore_add_token_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_token_pargs"); + + xfer += oprot->writeFieldBegin("token_identifier", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->token_identifier))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("delegation_token", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->delegation_token))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_token_result::~ThriftHiveMetastore_add_token_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_token_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_token_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_token_presult::~ThriftHiveMetastore_add_token_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_token_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_remove_token_args::~ThriftHiveMetastore_remove_token_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_remove_token_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->token_identifier); + this->__isset.token_identifier = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_remove_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_remove_token_args"); + + xfer += oprot->writeFieldBegin("token_identifier", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->token_identifier); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_remove_token_pargs::~ThriftHiveMetastore_remove_token_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_remove_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_remove_token_pargs"); + + xfer += oprot->writeFieldBegin("token_identifier", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->token_identifier))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_remove_token_result::~ThriftHiveMetastore_remove_token_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_remove_token_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_remove_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_remove_token_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_remove_token_presult::~ThriftHiveMetastore_remove_token_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_remove_token_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_token_args::~ThriftHiveMetastore_get_token_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_token_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->token_identifier); + this->__isset.token_identifier = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_token_args"); + + xfer += oprot->writeFieldBegin("token_identifier", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->token_identifier); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_token_pargs::~ThriftHiveMetastore_get_token_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_token_pargs"); + + xfer += oprot->writeFieldBegin("token_identifier", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->token_identifier))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_token_result::~ThriftHiveMetastore_get_token_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_token_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_token_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0); + xfer += oprot->writeString(this->success); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_token_presult::~ThriftHiveMetastore_get_token_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_token_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_all_token_identifiers_args::~ThriftHiveMetastore_get_all_token_identifiers_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_token_identifiers_args::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 ThriftHiveMetastore_get_all_token_identifiers_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_token_identifiers_args"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_all_token_identifiers_pargs::~ThriftHiveMetastore_get_all_token_identifiers_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_token_identifiers_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_token_identifiers_pargs"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_all_token_identifiers_result::~ThriftHiveMetastore_get_all_token_identifiers_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_token_identifiers_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1592; + ::apache::thrift::protocol::TType _etype1595; + xfer += iprot->readListBegin(_etype1595, _size1592); + this->success.resize(_size1592); + uint32_t _i1596; + for (_i1596 = 0; _i1596 < _size1592; ++_i1596) + { + xfer += iprot->readString(this->success[_i1596]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_all_token_identifiers_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_token_identifiers_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::vector ::const_iterator _iter1597; + for (_iter1597 = this->success.begin(); _iter1597 != this->success.end(); ++_iter1597) + { + xfer += oprot->writeString((*_iter1597)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_all_token_identifiers_presult::~ThriftHiveMetastore_get_all_token_identifiers_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_token_identifiers_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1598; + ::apache::thrift::protocol::TType _etype1601; + xfer += iprot->readListBegin(_etype1601, _size1598); + (*(this->success)).resize(_size1598); + uint32_t _i1602; + for (_i1602 = 0; _i1602 < _size1598; ++_i1602) + { + xfer += iprot->readString((*(this->success))[_i1602]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_add_master_key_args::~ThriftHiveMetastore_add_master_key_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_master_key_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->key); + this->__isset.key = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_master_key_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_master_key_args"); + + xfer += oprot->writeFieldBegin("key", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->key); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_master_key_pargs::~ThriftHiveMetastore_add_master_key_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_master_key_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_master_key_pargs"); + + xfer += oprot->writeFieldBegin("key", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString((*(this->key))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_master_key_result::~ThriftHiveMetastore_add_master_key_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_master_key_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_master_key_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_master_key_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_I32, 0); + xfer += oprot->writeI32(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_master_key_presult::~ThriftHiveMetastore_add_master_key_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_master_key_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_update_master_key_args::~ThriftHiveMetastore_update_master_key_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_update_master_key_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->seq_number); + this->__isset.seq_number = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->key); + this->__isset.key = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_update_master_key_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_master_key_args"); + + xfer += oprot->writeFieldBegin("seq_number", ::apache::thrift::protocol::T_I32, 1); + xfer += oprot->writeI32(this->seq_number); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("key", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->key); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_update_master_key_pargs::~ThriftHiveMetastore_update_master_key_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_update_master_key_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_master_key_pargs"); + + xfer += oprot->writeFieldBegin("seq_number", ::apache::thrift::protocol::T_I32, 1); + xfer += oprot->writeI32((*(this->seq_number))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("key", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString((*(this->key))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_update_master_key_result::~ThriftHiveMetastore_update_master_key_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_update_master_key_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_update_master_key_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_master_key_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_update_master_key_presult::~ThriftHiveMetastore_update_master_key_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_update_master_key_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_remove_master_key_args::~ThriftHiveMetastore_remove_master_key_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_remove_master_key_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->key_seq); + this->__isset.key_seq = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_remove_master_key_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_remove_master_key_args"); + + xfer += oprot->writeFieldBegin("key_seq", ::apache::thrift::protocol::T_I32, 1); + xfer += oprot->writeI32(this->key_seq); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_remove_master_key_pargs::~ThriftHiveMetastore_remove_master_key_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_remove_master_key_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_remove_master_key_pargs"); + + xfer += oprot->writeFieldBegin("key_seq", ::apache::thrift::protocol::T_I32, 1); + xfer += oprot->writeI32((*(this->key_seq))); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_remove_master_key_result::~ThriftHiveMetastore_remove_master_key_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_remove_master_key_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_remove_master_key_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_remove_master_key_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_remove_master_key_presult::~ThriftHiveMetastore_remove_master_key_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_remove_master_key_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_master_keys_args::~ThriftHiveMetastore_get_master_keys_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_master_keys_args::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 ThriftHiveMetastore_get_master_keys_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_master_keys_args"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_master_keys_pargs::~ThriftHiveMetastore_get_master_keys_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_master_keys_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_master_keys_pargs"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_master_keys_result::~ThriftHiveMetastore_get_master_keys_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_master_keys_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->success.clear(); + uint32_t _size1603; + ::apache::thrift::protocol::TType _etype1606; + xfer += iprot->readListBegin(_etype1606, _size1603); + this->success.resize(_size1603); + uint32_t _i1607; + for (_i1607 = 0; _i1607 < _size1603; ++_i1607) + { + xfer += iprot->readString(this->success[_i1607]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_master_keys_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_master_keys_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); + std::vector ::const_iterator _iter1608; + for (_iter1608 = this->success.begin(); _iter1608 != this->success.end(); ++_iter1608) + { + xfer += oprot->writeString((*_iter1608)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_master_keys_presult::~ThriftHiveMetastore_get_master_keys_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_master_keys_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + (*(this->success)).clear(); + uint32_t _size1609; + ::apache::thrift::protocol::TType _etype1612; + xfer += iprot->readListBegin(_etype1612, _size1609); + (*(this->success)).resize(_size1609); + uint32_t _i1613; + for (_i1613 = 0; _i1613 < _size1609; ++_i1613) + { + xfer += iprot->readString((*(this->success))[_i1613]); + } + xfer += iprot->readListEnd(); + } + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_open_txns_args::~ThriftHiveMetastore_get_open_txns_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_open_txns_args::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 ThriftHiveMetastore_get_open_txns_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_open_txns_args"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_open_txns_pargs::~ThriftHiveMetastore_get_open_txns_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_open_txns_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_open_txns_pargs"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_open_txns_result::~ThriftHiveMetastore_get_open_txns_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_open_txns_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_open_txns_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_open_txns_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_open_txns_presult::~ThriftHiveMetastore_get_open_txns_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_open_txns_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_open_txns_info_args::~ThriftHiveMetastore_get_open_txns_info_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_open_txns_info_args::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 ThriftHiveMetastore_get_open_txns_info_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_open_txns_info_args"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_open_txns_info_pargs::~ThriftHiveMetastore_get_open_txns_info_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_open_txns_info_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_open_txns_info_pargs"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_open_txns_info_result::~ThriftHiveMetastore_get_open_txns_info_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_open_txns_info_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_open_txns_info_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_open_txns_info_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_open_txns_info_presult::~ThriftHiveMetastore_get_open_txns_info_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_open_txns_info_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_open_txns_args::~ThriftHiveMetastore_open_txns_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_open_txns_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->rqst.read(iprot); + this->__isset.rqst = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_open_txns_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_open_txns_args"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->rqst.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_open_txns_pargs::~ThriftHiveMetastore_open_txns_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_open_txns_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_open_txns_pargs"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_open_txns_result::~ThriftHiveMetastore_open_txns_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_open_txns_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_open_txns_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_open_txns_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_open_txns_presult::~ThriftHiveMetastore_open_txns_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_open_txns_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_abort_txn_args::~ThriftHiveMetastore_abort_txn_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_abort_txn_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->rqst.read(iprot); + this->__isset.rqst = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_abort_txn_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_abort_txn_args"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->rqst.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_abort_txn_pargs::~ThriftHiveMetastore_abort_txn_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_abort_txn_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_abort_txn_pargs"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_abort_txn_result::~ThriftHiveMetastore_abort_txn_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_abort_txn_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_abort_txn_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_abort_txn_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_abort_txn_presult::~ThriftHiveMetastore_abort_txn_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_abort_txn_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_abort_txns_args::~ThriftHiveMetastore_abort_txns_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_abort_txns_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->rqst.read(iprot); + this->__isset.rqst = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_abort_txns_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_abort_txns_args"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->rqst.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_abort_txns_pargs::~ThriftHiveMetastore_abort_txns_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_abort_txns_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_abort_txns_pargs"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_abort_txns_result::~ThriftHiveMetastore_abort_txns_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_abort_txns_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_abort_txns_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_abort_txns_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_abort_txns_presult::~ThriftHiveMetastore_abort_txns_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_abort_txns_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_commit_txn_args::~ThriftHiveMetastore_commit_txn_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_commit_txn_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->rqst.read(iprot); + this->__isset.rqst = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_commit_txn_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_commit_txn_args"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->rqst.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_commit_txn_pargs::~ThriftHiveMetastore_commit_txn_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_commit_txn_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_commit_txn_pargs"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_commit_txn_result::~ThriftHiveMetastore_commit_txn_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_commit_txn_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_commit_txn_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_commit_txn_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_commit_txn_presult::~ThriftHiveMetastore_commit_txn_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_commit_txn_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_lock_args::~ThriftHiveMetastore_lock_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_lock_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->rqst.read(iprot); + this->__isset.rqst = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_lock_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_lock_args"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->rqst.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_lock_pargs::~ThriftHiveMetastore_lock_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_lock_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_lock_pargs"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_lock_result::~ThriftHiveMetastore_lock_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_lock_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_lock_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_lock_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_lock_presult::~ThriftHiveMetastore_lock_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_lock_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_check_lock_args::~ThriftHiveMetastore_check_lock_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_check_lock_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->rqst.read(iprot); + this->__isset.rqst = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_check_lock_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_check_lock_args"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->rqst.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_check_lock_pargs::~ThriftHiveMetastore_check_lock_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_check_lock_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_check_lock_pargs"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_check_lock_result::~ThriftHiveMetastore_check_lock_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_check_lock_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_check_lock_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_check_lock_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_check_lock_presult::~ThriftHiveMetastore_check_lock_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_check_lock_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_unlock_args::~ThriftHiveMetastore_unlock_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_unlock_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->rqst.read(iprot); + this->__isset.rqst = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_unlock_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_unlock_args"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->rqst.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_unlock_pargs::~ThriftHiveMetastore_unlock_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_unlock_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_unlock_pargs"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_unlock_result::~ThriftHiveMetastore_unlock_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_unlock_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_unlock_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_unlock_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_unlock_presult::~ThriftHiveMetastore_unlock_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_unlock_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_show_locks_args::~ThriftHiveMetastore_show_locks_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_show_locks_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->rqst.read(iprot); + this->__isset.rqst = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_show_locks_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_show_locks_args"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->rqst.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_show_locks_pargs::~ThriftHiveMetastore_show_locks_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_show_locks_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_show_locks_pargs"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_show_locks_result::~ThriftHiveMetastore_show_locks_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_show_locks_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_show_locks_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_show_locks_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_show_locks_presult::~ThriftHiveMetastore_show_locks_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_show_locks_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_heartbeat_args::~ThriftHiveMetastore_heartbeat_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_heartbeat_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->ids.read(iprot); + this->__isset.ids = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_heartbeat_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_heartbeat_args"); + + xfer += oprot->writeFieldBegin("ids", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->ids.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_heartbeat_pargs::~ThriftHiveMetastore_heartbeat_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_heartbeat_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_heartbeat_pargs"); + + xfer += oprot->writeFieldBegin("ids", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->ids)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_heartbeat_result::~ThriftHiveMetastore_heartbeat_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_heartbeat_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_heartbeat_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_heartbeat_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_heartbeat_presult::~ThriftHiveMetastore_heartbeat_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_heartbeat_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_heartbeat_txn_range_args::~ThriftHiveMetastore_heartbeat_txn_range_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_heartbeat_txn_range_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->txns.read(iprot); + this->__isset.txns = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_heartbeat_txn_range_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_heartbeat_txn_range_args"); + + xfer += oprot->writeFieldBegin("txns", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->txns.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_heartbeat_txn_range_pargs::~ThriftHiveMetastore_heartbeat_txn_range_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_heartbeat_txn_range_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_heartbeat_txn_range_pargs"); + + xfer += oprot->writeFieldBegin("txns", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->txns)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_heartbeat_txn_range_result::~ThriftHiveMetastore_heartbeat_txn_range_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_heartbeat_txn_range_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_heartbeat_txn_range_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_heartbeat_txn_range_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_heartbeat_txn_range_presult::~ThriftHiveMetastore_heartbeat_txn_range_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_heartbeat_txn_range_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_compact_args::~ThriftHiveMetastore_compact_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_compact_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->rqst.read(iprot); + this->__isset.rqst = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_compact_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_compact_args"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->rqst.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_compact_pargs::~ThriftHiveMetastore_compact_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_compact_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_compact_pargs"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_compact_result::~ThriftHiveMetastore_compact_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_compact_result::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 ThriftHiveMetastore_compact_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_compact_result"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_compact_presult::~ThriftHiveMetastore_compact_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_compact_presult::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; +} + + +ThriftHiveMetastore_compact2_args::~ThriftHiveMetastore_compact2_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_compact2_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->rqst.read(iprot); + this->__isset.rqst = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_compact2_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_compact2_args"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->rqst.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_compact2_pargs::~ThriftHiveMetastore_compact2_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_compact2_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_compact2_pargs"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_compact2_result::~ThriftHiveMetastore_compact2_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_compact2_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_compact2_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_compact2_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_compact2_presult::~ThriftHiveMetastore_compact2_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_compact2_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_show_compact_args::~ThriftHiveMetastore_show_compact_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_show_compact_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->rqst.read(iprot); + this->__isset.rqst = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_show_compact_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_show_compact_args"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->rqst.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_show_compact_pargs::~ThriftHiveMetastore_show_compact_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_show_compact_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_show_compact_pargs"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_show_compact_result::~ThriftHiveMetastore_show_compact_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_show_compact_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_show_compact_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_show_compact_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_show_compact_presult::~ThriftHiveMetastore_show_compact_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_show_compact_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_add_dynamic_partitions_args::~ThriftHiveMetastore_add_dynamic_partitions_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_dynamic_partitions_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->rqst.read(iprot); + this->__isset.rqst = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_dynamic_partitions_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_dynamic_partitions_args"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->rqst.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_dynamic_partitions_pargs::~ThriftHiveMetastore_add_dynamic_partitions_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_dynamic_partitions_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_dynamic_partitions_pargs"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_dynamic_partitions_result::~ThriftHiveMetastore_add_dynamic_partitions_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_dynamic_partitions_result::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_add_dynamic_partitions_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_dynamic_partitions_result"); + + if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_add_dynamic_partitions_presult::~ThriftHiveMetastore_add_dynamic_partitions_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_add_dynamic_partitions_presult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_next_notification_args::~ThriftHiveMetastore_get_next_notification_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_next_notification_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->rqst.read(iprot); + this->__isset.rqst = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_next_notification_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_next_notification_args"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->rqst.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_next_notification_pargs::~ThriftHiveMetastore_get_next_notification_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_next_notification_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_next_notification_pargs"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_next_notification_result::~ThriftHiveMetastore_get_next_notification_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_next_notification_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_next_notification_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_next_notification_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_next_notification_presult::~ThriftHiveMetastore_get_next_notification_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_next_notification_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_current_notificationEventId_args::~ThriftHiveMetastore_get_current_notificationEventId_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_current_notificationEventId_args::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 ThriftHiveMetastore_get_current_notificationEventId_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_current_notificationEventId_args"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_current_notificationEventId_pargs::~ThriftHiveMetastore_get_current_notificationEventId_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_current_notificationEventId_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_current_notificationEventId_pargs"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_current_notificationEventId_result::~ThriftHiveMetastore_get_current_notificationEventId_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_current_notificationEventId_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_current_notificationEventId_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_current_notificationEventId_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_current_notificationEventId_presult::~ThriftHiveMetastore_get_current_notificationEventId_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_current_notificationEventId_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_notification_events_count_args::~ThriftHiveMetastore_get_notification_events_count_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_notification_events_count_args::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; + } + switch (fid) + { + case -1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->rqst.read(iprot); + this->__isset.rqst = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_notification_events_count_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_notification_events_count_args"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, -1); + xfer += this->rqst.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_notification_events_count_pargs::~ThriftHiveMetastore_get_notification_events_count_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_notification_events_count_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_notification_events_count_pargs"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, -1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_notification_events_count_result::~ThriftHiveMetastore_get_notification_events_count_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_notification_events_count_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_notification_events_count_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_notification_events_count_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_notification_events_count_presult::~ThriftHiveMetastore_get_notification_events_count_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_notification_events_count_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_fire_listener_event_args::~ThriftHiveMetastore_fire_listener_event_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_fire_listener_event_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->rqst.read(iprot); + this->__isset.rqst = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_fire_listener_event_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_fire_listener_event_args"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->rqst.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_fire_listener_event_pargs::~ThriftHiveMetastore_fire_listener_event_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_fire_listener_event_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_fire_listener_event_pargs"); + + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_fire_listener_event_result::~ThriftHiveMetastore_fire_listener_event_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_fire_listener_event_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_fire_listener_event_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_fire_listener_event_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_fire_listener_event_presult::~ThriftHiveMetastore_fire_listener_event_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_fire_listener_event_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_flushCache_args::~ThriftHiveMetastore_flushCache_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_flushCache_args::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 ThriftHiveMetastore_flushCache_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_flushCache_args"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_flushCache_pargs::~ThriftHiveMetastore_flushCache_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_flushCache_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_flushCache_pargs"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_flushCache_result::~ThriftHiveMetastore_flushCache_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_flushCache_result::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 ThriftHiveMetastore_flushCache_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_flushCache_result"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_flushCache_presult::~ThriftHiveMetastore_flushCache_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_flushCache_presult::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; +} + + +ThriftHiveMetastore_cm_recycle_args::~ThriftHiveMetastore_cm_recycle_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_cm_recycle_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_cm_recycle_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_cm_recycle_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_cm_recycle_pargs::~ThriftHiveMetastore_cm_recycle_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_cm_recycle_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_cm_recycle_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_cm_recycle_result::~ThriftHiveMetastore_cm_recycle_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_cm_recycle_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_cm_recycle_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_cm_recycle_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_cm_recycle_presult::~ThriftHiveMetastore_cm_recycle_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_cm_recycle_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_file_metadata_by_expr_args::~ThriftHiveMetastore_get_file_metadata_by_expr_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->req.read(iprot); + this->__isset.req = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_by_expr_args"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->req.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_file_metadata_by_expr_pargs::~ThriftHiveMetastore_get_file_metadata_by_expr_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_by_expr_pargs"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->req)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_file_metadata_by_expr_result::~ThriftHiveMetastore_get_file_metadata_by_expr_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_by_expr_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_file_metadata_by_expr_presult::~ThriftHiveMetastore_get_file_metadata_by_expr_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_file_metadata_args::~ThriftHiveMetastore_get_file_metadata_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_file_metadata_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->req.read(iprot); + this->__isset.req = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_file_metadata_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_args"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->req.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_file_metadata_pargs::~ThriftHiveMetastore_get_file_metadata_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_file_metadata_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_pargs"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->req)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_file_metadata_result::~ThriftHiveMetastore_get_file_metadata_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_file_metadata_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_file_metadata_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_file_metadata_presult::~ThriftHiveMetastore_get_file_metadata_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_file_metadata_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_put_file_metadata_args::~ThriftHiveMetastore_put_file_metadata_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_put_file_metadata_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->req.read(iprot); + this->__isset.req = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_put_file_metadata_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_put_file_metadata_args"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->req.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_put_file_metadata_pargs::~ThriftHiveMetastore_put_file_metadata_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_put_file_metadata_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_put_file_metadata_pargs"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->req)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_put_file_metadata_result::~ThriftHiveMetastore_put_file_metadata_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_put_file_metadata_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_put_file_metadata_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_put_file_metadata_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_put_file_metadata_presult::~ThriftHiveMetastore_put_file_metadata_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_put_file_metadata_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_clear_file_metadata_args::~ThriftHiveMetastore_clear_file_metadata_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_clear_file_metadata_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->req.read(iprot); + this->__isset.req = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_clear_file_metadata_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_clear_file_metadata_args"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->req.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_clear_file_metadata_pargs::~ThriftHiveMetastore_clear_file_metadata_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_clear_file_metadata_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_clear_file_metadata_pargs"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->req)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_clear_file_metadata_result::~ThriftHiveMetastore_clear_file_metadata_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_clear_file_metadata_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_clear_file_metadata_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_clear_file_metadata_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_clear_file_metadata_presult::~ThriftHiveMetastore_clear_file_metadata_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_clear_file_metadata_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_cache_file_metadata_args::~ThriftHiveMetastore_cache_file_metadata_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_cache_file_metadata_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->req.read(iprot); + this->__isset.req = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_cache_file_metadata_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_cache_file_metadata_args"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->req.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_cache_file_metadata_pargs::~ThriftHiveMetastore_cache_file_metadata_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_cache_file_metadata_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_cache_file_metadata_pargs"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->req)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_cache_file_metadata_result::~ThriftHiveMetastore_cache_file_metadata_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_cache_file_metadata_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_cache_file_metadata_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_cache_file_metadata_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_cache_file_metadata_presult::~ThriftHiveMetastore_cache_file_metadata_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_cache_file_metadata_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_metastore_db_uuid_args::~ThriftHiveMetastore_get_metastore_db_uuid_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_metastore_db_uuid_args::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 ThriftHiveMetastore_get_metastore_db_uuid_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_metastore_db_uuid_args"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_metastore_db_uuid_pargs::~ThriftHiveMetastore_get_metastore_db_uuid_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_metastore_db_uuid_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_metastore_db_uuid_pargs"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_metastore_db_uuid_result::~ThriftHiveMetastore_get_metastore_db_uuid_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_metastore_db_uuid_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_metastore_db_uuid_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_metastore_db_uuid_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0); + xfer += oprot->writeString(this->success); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_metastore_db_uuid_presult::~ThriftHiveMetastore_get_metastore_db_uuid_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_metastore_db_uuid_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString((*(this->success))); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_create_resource_plan_args::~ThriftHiveMetastore_create_resource_plan_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_resource_plan_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_resource_plan_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_resource_plan_pargs::~ThriftHiveMetastore_create_resource_plan_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_resource_plan_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_resource_plan_result::~ThriftHiveMetastore_create_resource_plan_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_resource_plan_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_resource_plan_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_resource_plan_presult::~ThriftHiveMetastore_create_resource_plan_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_resource_plan_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_resource_plan_args::~ThriftHiveMetastore_get_resource_plan_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_resource_plan_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_resource_plan_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_resource_plan_pargs::~ThriftHiveMetastore_get_resource_plan_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_resource_plan_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_resource_plan_result::~ThriftHiveMetastore_get_resource_plan_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_resource_plan_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_resource_plan_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_resource_plan_presult::~ThriftHiveMetastore_get_resource_plan_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_resource_plan_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_active_resource_plan_args::~ThriftHiveMetastore_get_active_resource_plan_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_active_resource_plan_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_active_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_active_resource_plan_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_active_resource_plan_pargs::~ThriftHiveMetastore_get_active_resource_plan_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_active_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_active_resource_plan_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_active_resource_plan_result::~ThriftHiveMetastore_get_active_resource_plan_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_active_resource_plan_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_active_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_active_resource_plan_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_active_resource_plan_presult::~ThriftHiveMetastore_get_active_resource_plan_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_active_resource_plan_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_all_resource_plans_args::~ThriftHiveMetastore_get_all_resource_plans_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_resource_plans_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_all_resource_plans_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_resource_plans_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_all_resource_plans_pargs::~ThriftHiveMetastore_get_all_resource_plans_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_resource_plans_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_resource_plans_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_all_resource_plans_result::~ThriftHiveMetastore_get_all_resource_plans_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_resource_plans_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_all_resource_plans_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_resource_plans_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_all_resource_plans_presult::~ThriftHiveMetastore_get_all_resource_plans_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_all_resource_plans_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_alter_resource_plan_args::~ThriftHiveMetastore_alter_resource_plan_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_resource_plan_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_resource_plan_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_resource_plan_pargs::~ThriftHiveMetastore_alter_resource_plan_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_resource_plan_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_resource_plan_result::~ThriftHiveMetastore_alter_resource_plan_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_resource_plan_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_resource_plan_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_resource_plan_presult::~ThriftHiveMetastore_alter_resource_plan_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_resource_plan_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_validate_resource_plan_args::~ThriftHiveMetastore_validate_resource_plan_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_validate_resource_plan_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_validate_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_validate_resource_plan_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_validate_resource_plan_pargs::~ThriftHiveMetastore_validate_resource_plan_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_validate_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_validate_resource_plan_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_validate_resource_plan_result::~ThriftHiveMetastore_validate_resource_plan_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_validate_resource_plan_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_validate_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_validate_resource_plan_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_validate_resource_plan_presult::~ThriftHiveMetastore_validate_resource_plan_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_validate_resource_plan_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_drop_resource_plan_args::~ThriftHiveMetastore_drop_resource_plan_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_resource_plan_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_resource_plan_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_resource_plan_pargs::~ThriftHiveMetastore_drop_resource_plan_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_resource_plan_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_resource_plan_result::~ThriftHiveMetastore_drop_resource_plan_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_resource_plan_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_resource_plan_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_resource_plan_presult::~ThriftHiveMetastore_drop_resource_plan_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_resource_plan_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_create_wm_trigger_args::~ThriftHiveMetastore_create_wm_trigger_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_wm_trigger_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_wm_trigger_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_wm_trigger_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_wm_trigger_pargs::~ThriftHiveMetastore_create_wm_trigger_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_wm_trigger_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_wm_trigger_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_wm_trigger_result::~ThriftHiveMetastore_create_wm_trigger_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_wm_trigger_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_create_wm_trigger_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_wm_trigger_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o4) { + xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->o4.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_create_wm_trigger_presult::~ThriftHiveMetastore_create_wm_trigger_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_create_wm_trigger_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o4.read(iprot); + this->__isset.o4 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_alter_wm_trigger_args::~ThriftHiveMetastore_alter_wm_trigger_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_wm_trigger_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_wm_trigger_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_wm_trigger_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_wm_trigger_pargs::~ThriftHiveMetastore_alter_wm_trigger_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_wm_trigger_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_wm_trigger_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_wm_trigger_result::~ThriftHiveMetastore_alter_wm_trigger_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_wm_trigger_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_alter_wm_trigger_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_wm_trigger_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_alter_wm_trigger_presult::~ThriftHiveMetastore_alter_wm_trigger_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_alter_wm_trigger_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_drop_wm_trigger_args::~ThriftHiveMetastore_drop_wm_trigger_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_wm_trigger_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_wm_trigger_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_wm_trigger_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_wm_trigger_pargs::~ThriftHiveMetastore_drop_wm_trigger_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_wm_trigger_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_wm_trigger_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_wm_trigger_result::~ThriftHiveMetastore_drop_wm_trigger_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_wm_trigger_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_drop_wm_trigger_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_wm_trigger_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o3) { + xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->o3.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_drop_wm_trigger_presult::~ThriftHiveMetastore_drop_wm_trigger_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_drop_wm_trigger_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o3.read(iprot); + this->__isset.o3 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + + +ThriftHiveMetastore_get_triggers_for_resourceplan_args::~ThriftHiveMetastore_get_triggers_for_resourceplan_args() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_triggers_for_resourceplan_args::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->request.read(iprot); + this->__isset.request = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_triggers_for_resourceplan_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_triggers_for_resourceplan_args"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->request.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_triggers_for_resourceplan_pargs::~ThriftHiveMetastore_get_triggers_for_resourceplan_pargs() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_triggers_for_resourceplan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_triggers_for_resourceplan_pargs"); + + xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->request)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_triggers_for_resourceplan_result::~ThriftHiveMetastore_get_triggers_for_resourceplan_result() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_triggers_for_resourceplan_result::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ThriftHiveMetastore_get_triggers_for_resourceplan_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_triggers_for_resourceplan_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); + } else if (this->__isset.o2) { + xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->o2.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +ThriftHiveMetastore_get_triggers_for_resourceplan_presult::~ThriftHiveMetastore_get_triggers_for_resourceplan_presult() noexcept { +} + + +uint32_t ThriftHiveMetastore_get_triggers_for_resourceplan_presult::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; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o1.read(iprot); + this->__isset.o1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->o2.read(iprot); + this->__isset.o2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +void ThriftHiveMetastoreClient::getMetaConf(std::string& _return, const std::string& key) +{ + send_getMetaConf(key); + recv_getMetaConf(_return); +} + +void ThriftHiveMetastoreClient::send_getMetaConf(const std::string& key) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("getMetaConf", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_getMetaConf_pargs args; + args.key = &key; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_getMetaConf(std::string& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("getMetaConf") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_getMetaConf_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "getMetaConf failed: unknown result"); +} + +void ThriftHiveMetastoreClient::setMetaConf(const std::string& key, const std::string& value) +{ + send_setMetaConf(key, value); + recv_setMetaConf(); +} + +void ThriftHiveMetastoreClient::send_setMetaConf(const std::string& key, const std::string& value) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("setMetaConf", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_setMetaConf_pargs args; + args.key = &key; + args.value = &value; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_setMetaConf() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("setMetaConf") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_setMetaConf_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + return; +} + +void ThriftHiveMetastoreClient::create_database(const Database& database) +{ + send_create_database(database); + recv_create_database(); +} + +void ThriftHiveMetastoreClient::send_create_database(const Database& database) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("create_database", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_database_pargs args; + args.database = &database; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_create_database() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_database") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_create_database_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + return; +} + +void ThriftHiveMetastoreClient::get_database(Database& _return, const std::string& name) +{ + send_get_database(name); + recv_get_database(_return); +} + +void ThriftHiveMetastoreClient::send_get_database(const std::string& name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_database", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_database_pargs args; + args.name = &name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_database(Database& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_database") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_database_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_database failed: unknown result"); +} + +void ThriftHiveMetastoreClient::drop_database(const std::string& name, const bool deleteData, const bool cascade) +{ + send_drop_database(name, deleteData, cascade); + recv_drop_database(); +} + +void ThriftHiveMetastoreClient::send_drop_database(const std::string& name, const bool deleteData, const bool cascade) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("drop_database", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_database_pargs args; + args.name = &name; + args.deleteData = &deleteData; + args.cascade = &cascade; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_drop_database() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_database") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_drop_database_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + return; +} + +void ThriftHiveMetastoreClient::get_databases(std::vector & _return, const std::string& pattern) +{ + send_get_databases(pattern); + recv_get_databases(_return); +} + +void ThriftHiveMetastoreClient::send_get_databases(const std::string& pattern) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_databases", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_databases_pargs args; + args.pattern = &pattern; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_databases(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_databases") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_databases_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_databases failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_all_databases(std::vector & _return) +{ + send_get_all_databases(); + recv_get_all_databases(_return); +} + +void ThriftHiveMetastoreClient::send_get_all_databases() +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_all_databases", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_all_databases_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_all_databases(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_all_databases") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_all_databases_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_databases failed: unknown result"); +} + +void ThriftHiveMetastoreClient::alter_database(const std::string& dbname, const Database& db) +{ + send_alter_database(dbname, db); + recv_alter_database(); +} + +void ThriftHiveMetastoreClient::send_alter_database(const std::string& dbname, const Database& db) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("alter_database", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_database_pargs args; + args.dbname = &dbname; + args.db = &db; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_alter_database() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_database") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_alter_database_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +void ThriftHiveMetastoreClient::get_type(Type& _return, const std::string& name) +{ + send_get_type(name); + recv_get_type(_return); +} + +void ThriftHiveMetastoreClient::send_get_type(const std::string& name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_type", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_type_pargs args; + args.name = &name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_type(Type& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_type") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_type_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_type failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::create_type(const Type& type) +{ + send_create_type(type); + return recv_create_type(); +} + +void ThriftHiveMetastoreClient::send_create_type(const Type& type) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("create_type", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_type_pargs args; + args.type = &type; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_create_type() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_type") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_create_type_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_type failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::drop_type(const std::string& type) +{ + send_drop_type(type); + return recv_drop_type(); +} + +void ThriftHiveMetastoreClient::send_drop_type(const std::string& type) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("drop_type", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_type_pargs args; + args.type = &type; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_drop_type() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_type") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_drop_type_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_type failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_type_all(std::map & _return, const std::string& name) +{ + send_get_type_all(name); + recv_get_type_all(_return); +} + +void ThriftHiveMetastoreClient::send_get_type_all(const std::string& name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_type_all", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_type_all_pargs args; + args.name = &name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_type_all(std::map & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_type_all") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_type_all_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_type_all failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_fields(std::vector & _return, const std::string& db_name, const std::string& table_name) +{ + send_get_fields(db_name, table_name); + recv_get_fields(_return); +} + +void ThriftHiveMetastoreClient::send_get_fields(const std::string& db_name, const std::string& table_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_fields", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_fields_pargs args; + args.db_name = &db_name; + args.table_name = &table_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_fields(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_fields") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_fields_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_fields failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_fields_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) +{ + send_get_fields_with_environment_context(db_name, table_name, environment_context); + recv_get_fields_with_environment_context(_return); +} + +void ThriftHiveMetastoreClient::send_get_fields_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_fields_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_fields_with_environment_context_pargs args; + args.db_name = &db_name; + args.table_name = &table_name; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_fields_with_environment_context(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_fields_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_fields_with_environment_context_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_fields_with_environment_context failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_schema(std::vector & _return, const std::string& db_name, const std::string& table_name) +{ + send_get_schema(db_name, table_name); + recv_get_schema(_return); +} + +void ThriftHiveMetastoreClient::send_get_schema(const std::string& db_name, const std::string& table_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_schema", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_schema_pargs args; + args.db_name = &db_name; + args.table_name = &table_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_schema(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_schema") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_schema_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_schema failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_schema_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) +{ + send_get_schema_with_environment_context(db_name, table_name, environment_context); + recv_get_schema_with_environment_context(_return); +} + +void ThriftHiveMetastoreClient::send_get_schema_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_schema_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_schema_with_environment_context_pargs args; + args.db_name = &db_name; + args.table_name = &table_name; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_schema_with_environment_context(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_schema_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_schema_with_environment_context_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_schema_with_environment_context failed: unknown result"); +} + +void ThriftHiveMetastoreClient::create_table(const Table& tbl) +{ + send_create_table(tbl); + recv_create_table(); +} + +void ThriftHiveMetastoreClient::send_create_table(const Table& tbl) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("create_table", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_table_pargs args; + args.tbl = &tbl; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_create_table() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_table") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_create_table_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + return; +} + +void ThriftHiveMetastoreClient::create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) +{ + send_create_table_with_environment_context(tbl, environment_context); + recv_create_table_with_environment_context(); +} + +void ThriftHiveMetastoreClient::send_create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("create_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_table_with_environment_context_pargs args; + args.tbl = &tbl; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_create_table_with_environment_context() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_table_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_create_table_with_environment_context_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + return; +} + +void ThriftHiveMetastoreClient::create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints) +{ + send_create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints); + recv_create_table_with_constraints(); +} + +void ThriftHiveMetastoreClient::send_create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("create_table_with_constraints", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_table_with_constraints_pargs args; + args.tbl = &tbl; + args.primaryKeys = &primaryKeys; + args.foreignKeys = &foreignKeys; + args.uniqueConstraints = &uniqueConstraints; + args.notNullConstraints = ¬NullConstraints; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_create_table_with_constraints() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_table_with_constraints") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_create_table_with_constraints_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + return; +} + +void ThriftHiveMetastoreClient::drop_constraint(const DropConstraintRequest& req) +{ + send_drop_constraint(req); + recv_drop_constraint(); +} + +void ThriftHiveMetastoreClient::send_drop_constraint(const DropConstraintRequest& req) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("drop_constraint", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_constraint_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_drop_constraint() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_constraint") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_drop_constraint_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o3) { + throw result.o3; + } + return; +} + +void ThriftHiveMetastoreClient::add_primary_key(const AddPrimaryKeyRequest& req) +{ + send_add_primary_key(req); + recv_add_primary_key(); +} + +void ThriftHiveMetastoreClient::send_add_primary_key(const AddPrimaryKeyRequest& req) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("add_primary_key", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_primary_key_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_add_primary_key() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_primary_key") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_add_primary_key_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +void ThriftHiveMetastoreClient::add_foreign_key(const AddForeignKeyRequest& req) +{ + send_add_foreign_key(req); + recv_add_foreign_key(); +} + +void ThriftHiveMetastoreClient::send_add_foreign_key(const AddForeignKeyRequest& req) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("add_foreign_key", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_foreign_key_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_add_foreign_key() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_foreign_key") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_add_foreign_key_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +void ThriftHiveMetastoreClient::add_unique_constraint(const AddUniqueConstraintRequest& req) +{ + send_add_unique_constraint(req); + recv_add_unique_constraint(); +} + +void ThriftHiveMetastoreClient::send_add_unique_constraint(const AddUniqueConstraintRequest& req) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("add_unique_constraint", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_unique_constraint_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_add_unique_constraint() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_unique_constraint") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_add_unique_constraint_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +void ThriftHiveMetastoreClient::add_not_null_constraint(const AddNotNullConstraintRequest& req) +{ + send_add_not_null_constraint(req); + recv_add_not_null_constraint(); +} + +void ThriftHiveMetastoreClient::send_add_not_null_constraint(const AddNotNullConstraintRequest& req) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("add_not_null_constraint", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_not_null_constraint_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_add_not_null_constraint() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_not_null_constraint") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_add_not_null_constraint_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +void ThriftHiveMetastoreClient::drop_table(const std::string& dbname, const std::string& name, const bool deleteData) +{ + send_drop_table(dbname, name, deleteData); + recv_drop_table(); +} + +void ThriftHiveMetastoreClient::send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("drop_table", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_table_pargs args; + args.dbname = &dbname; + args.name = &name; + args.deleteData = &deleteData; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_drop_table() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_table") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_drop_table_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o3) { + throw result.o3; + } + return; +} + +void ThriftHiveMetastoreClient::drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) +{ + send_drop_table_with_environment_context(dbname, name, deleteData, environment_context); + recv_drop_table_with_environment_context(); +} + +void ThriftHiveMetastoreClient::send_drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("drop_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_table_with_environment_context_pargs args; + args.dbname = &dbname; + args.name = &name; + args.deleteData = &deleteData; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_drop_table_with_environment_context() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_table_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_drop_table_with_environment_context_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o3) { + throw result.o3; + } + return; +} + +void ThriftHiveMetastoreClient::truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames) +{ + send_truncate_table(dbName, tableName, partNames); + recv_truncate_table(); +} + +void ThriftHiveMetastoreClient::send_truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("truncate_table", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_truncate_table_pargs args; + args.dbName = &dbName; + args.tableName = &tableName; + args.partNames = &partNames; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_truncate_table() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("truncate_table") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_truncate_table_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + return; +} + +void ThriftHiveMetastoreClient::get_tables(std::vector & _return, const std::string& db_name, const std::string& pattern) +{ + send_get_tables(db_name, pattern); + recv_get_tables(_return); +} + +void ThriftHiveMetastoreClient::send_get_tables(const std::string& db_name, const std::string& pattern) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_tables", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_tables_pargs args; + args.db_name = &db_name; + args.pattern = &pattern; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_tables(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_tables") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_tables_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_tables failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_tables_by_type(std::vector & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType) +{ + send_get_tables_by_type(db_name, pattern, tableType); + recv_get_tables_by_type(_return); +} + +void ThriftHiveMetastoreClient::send_get_tables_by_type(const std::string& db_name, const std::string& pattern, const std::string& tableType) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_tables_by_type", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_tables_by_type_pargs args; + args.db_name = &db_name; + args.pattern = &pattern; + args.tableType = &tableType; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_tables_by_type(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_tables_by_type") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_tables_by_type_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_tables_by_type failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_table_meta(std::vector & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types) +{ + send_get_table_meta(db_patterns, tbl_patterns, tbl_types); + recv_get_table_meta(_return); +} + +void ThriftHiveMetastoreClient::send_get_table_meta(const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_table_meta", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_table_meta_pargs args; + args.db_patterns = &db_patterns; + args.tbl_patterns = &tbl_patterns; + args.tbl_types = &tbl_types; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_table_meta(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_table_meta") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_table_meta_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_meta failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_all_tables(std::vector & _return, const std::string& db_name) +{ + send_get_all_tables(db_name); + recv_get_all_tables(_return); +} + +void ThriftHiveMetastoreClient::send_get_all_tables(const std::string& db_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_all_tables", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_all_tables_pargs args; + args.db_name = &db_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_all_tables(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_all_tables") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_all_tables_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_tables failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_table(Table& _return, const std::string& dbname, const std::string& tbl_name) +{ + send_get_table(dbname, tbl_name); + recv_get_table(_return); +} + +void ThriftHiveMetastoreClient::send_get_table(const std::string& dbname, const std::string& tbl_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_table", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_table_pargs args; + args.dbname = &dbname; + args.tbl_name = &tbl_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_table(Table& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_table") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_table_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_table_objects_by_name(std::vector
& _return, const std::string& dbname, const std::vector & tbl_names) +{ + send_get_table_objects_by_name(dbname, tbl_names); + recv_get_table_objects_by_name(_return); +} + +void ThriftHiveMetastoreClient::send_get_table_objects_by_name(const std::string& dbname, const std::vector & tbl_names) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_table_objects_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_table_objects_by_name_pargs args; + args.dbname = &dbname; + args.tbl_names = &tbl_names; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_table_objects_by_name(std::vector
& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_table_objects_by_name") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_table_objects_by_name_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_objects_by_name failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_table_req(GetTableResult& _return, const GetTableRequest& req) +{ + send_get_table_req(req); + recv_get_table_req(_return); +} + +void ThriftHiveMetastoreClient::send_get_table_req(const GetTableRequest& req) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_table_req", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_table_req_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_table_req(GetTableResult& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_table_req") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_table_req_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_req failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req) +{ + send_get_table_objects_by_name_req(req); + recv_get_table_objects_by_name_req(_return); +} + +void ThriftHiveMetastoreClient::send_get_table_objects_by_name_req(const GetTablesRequest& req) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_table_objects_by_name_req", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_table_objects_by_name_req_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_table_objects_by_name_req(GetTablesResult& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_table_objects_by_name_req") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_table_objects_by_name_req_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_objects_by_name_req failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_table_names_by_filter(std::vector & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables) +{ + send_get_table_names_by_filter(dbname, filter, max_tables); + recv_get_table_names_by_filter(_return); +} + +void ThriftHiveMetastoreClient::send_get_table_names_by_filter(const std::string& dbname, const std::string& filter, const int16_t max_tables) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_table_names_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_table_names_by_filter_pargs args; + args.dbname = &dbname; + args.filter = &filter; + args.max_tables = &max_tables; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_table_names_by_filter(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_table_names_by_filter") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_table_names_by_filter_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_names_by_filter failed: unknown result"); +} + +void ThriftHiveMetastoreClient::alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) +{ + send_alter_table(dbname, tbl_name, new_tbl); + recv_alter_table(); +} + +void ThriftHiveMetastoreClient::send_alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("alter_table", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_table_pargs args; + args.dbname = &dbname; + args.tbl_name = &tbl_name; + args.new_tbl = &new_tbl; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_alter_table() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_table") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_alter_table_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +void ThriftHiveMetastoreClient::alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) +{ + send_alter_table_with_environment_context(dbname, tbl_name, new_tbl, environment_context); + recv_alter_table_with_environment_context(); +} + +void ThriftHiveMetastoreClient::send_alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("alter_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_table_with_environment_context_pargs args; + args.dbname = &dbname; + args.tbl_name = &tbl_name; + args.new_tbl = &new_tbl; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_alter_table_with_environment_context() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_table_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_alter_table_with_environment_context_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +void ThriftHiveMetastoreClient::alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade) +{ + send_alter_table_with_cascade(dbname, tbl_name, new_tbl, cascade); + recv_alter_table_with_cascade(); +} + +void ThriftHiveMetastoreClient::send_alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("alter_table_with_cascade", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_table_with_cascade_pargs args; + args.dbname = &dbname; + args.tbl_name = &tbl_name; + args.new_tbl = &new_tbl; + args.cascade = &cascade; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_alter_table_with_cascade() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_table_with_cascade") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_alter_table_with_cascade_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +void ThriftHiveMetastoreClient::add_partition(Partition& _return, const Partition& new_part) +{ + send_add_partition(new_part); + recv_add_partition(_return); +} + +void ThriftHiveMetastoreClient::send_add_partition(const Partition& new_part) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("add_partition", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_partition_pargs args; + args.new_part = &new_part; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_add_partition(Partition& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_partition") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_add_partition_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partition failed: unknown result"); +} + +void ThriftHiveMetastoreClient::add_partition_with_environment_context(Partition& _return, const Partition& new_part, const EnvironmentContext& environment_context) +{ + send_add_partition_with_environment_context(new_part, environment_context); + recv_add_partition_with_environment_context(_return); +} + +void ThriftHiveMetastoreClient::send_add_partition_with_environment_context(const Partition& new_part, const EnvironmentContext& environment_context) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("add_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_partition_with_environment_context_pargs args; + args.new_part = &new_part; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_add_partition_with_environment_context(Partition& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_partition_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_add_partition_with_environment_context_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partition_with_environment_context failed: unknown result"); +} + +int32_t ThriftHiveMetastoreClient::add_partitions(const std::vector & new_parts) +{ + send_add_partitions(new_parts); + return recv_add_partitions(); +} + +void ThriftHiveMetastoreClient::send_add_partitions(const std::vector & new_parts) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("add_partitions", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_partitions_pargs args; + args.new_parts = &new_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +int32_t ThriftHiveMetastoreClient::recv_add_partitions() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_partitions") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + int32_t _return; + ThriftHiveMetastore_add_partitions_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partitions failed: unknown result"); +} + +int32_t ThriftHiveMetastoreClient::add_partitions_pspec(const std::vector & new_parts) +{ + send_add_partitions_pspec(new_parts); + return recv_add_partitions_pspec(); +} + +void ThriftHiveMetastoreClient::send_add_partitions_pspec(const std::vector & new_parts) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("add_partitions_pspec", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_partitions_pspec_pargs args; + args.new_parts = &new_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +int32_t ThriftHiveMetastoreClient::recv_add_partitions_pspec() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_partitions_pspec") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + int32_t _return; + ThriftHiveMetastore_add_partitions_pspec_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partitions_pspec failed: unknown result"); +} + +void ThriftHiveMetastoreClient::append_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) +{ + send_append_partition(db_name, tbl_name, part_vals); + recv_append_partition(_return); +} + +void ThriftHiveMetastoreClient::send_append_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("append_partition", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_append_partition_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_append_partition(Partition& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("append_partition") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_append_partition_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition failed: unknown result"); +} + +void ThriftHiveMetastoreClient::add_partitions_req(AddPartitionsResult& _return, const AddPartitionsRequest& request) +{ + send_add_partitions_req(request); + recv_add_partitions_req(_return); +} + +void ThriftHiveMetastoreClient::send_add_partitions_req(const AddPartitionsRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("add_partitions_req", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_partitions_req_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_add_partitions_req(AddPartitionsResult& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_partitions_req") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_add_partitions_req_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partitions_req failed: unknown result"); +} + +void ThriftHiveMetastoreClient::append_partition_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context) +{ + send_append_partition_with_environment_context(db_name, tbl_name, part_vals, environment_context); + recv_append_partition_with_environment_context(_return); +} + +void ThriftHiveMetastoreClient::send_append_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_append_partition_with_environment_context_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_append_partition_with_environment_context(Partition& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("append_partition_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_append_partition_with_environment_context_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition_with_environment_context failed: unknown result"); +} + +void ThriftHiveMetastoreClient::append_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) +{ + send_append_partition_by_name(db_name, tbl_name, part_name); + recv_append_partition_by_name(_return); +} + +void ThriftHiveMetastoreClient::send_append_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("append_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_append_partition_by_name_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_append_partition_by_name(Partition& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("append_partition_by_name") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_append_partition_by_name_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition_by_name failed: unknown result"); +} + +void ThriftHiveMetastoreClient::append_partition_by_name_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context) +{ + send_append_partition_by_name_with_environment_context(db_name, tbl_name, part_name, environment_context); + recv_append_partition_by_name_with_environment_context(_return); +} + +void ThriftHiveMetastoreClient::send_append_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_append_partition_by_name_with_environment_context(Partition& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("append_partition_by_name_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition_by_name_with_environment_context failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) +{ + send_drop_partition(db_name, tbl_name, part_vals, deleteData); + return recv_drop_partition(); +} + +void ThriftHiveMetastoreClient::send_drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("drop_partition", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_partition_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.deleteData = &deleteData; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_drop_partition() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_partition") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_drop_partition_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context) +{ + send_drop_partition_with_environment_context(db_name, tbl_name, part_vals, deleteData, environment_context); + return recv_drop_partition_with_environment_context(); +} + +void ThriftHiveMetastoreClient::send_drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("drop_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_partition_with_environment_context_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.deleteData = &deleteData; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_drop_partition_with_environment_context() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_partition_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_drop_partition_with_environment_context_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_with_environment_context failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData) +{ + send_drop_partition_by_name(db_name, tbl_name, part_name, deleteData); + return recv_drop_partition_by_name(); +} + +void ThriftHiveMetastoreClient::send_drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("drop_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_partition_by_name_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; + args.deleteData = &deleteData; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_drop_partition_by_name() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_partition_by_name") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_drop_partition_by_name_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_by_name failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context) +{ + send_drop_partition_by_name_with_environment_context(db_name, tbl_name, part_name, deleteData, environment_context); + return recv_drop_partition_by_name_with_environment_context(); +} + +void ThriftHiveMetastoreClient::send_drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; + args.deleteData = &deleteData; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_drop_partition_by_name_with_environment_context() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_partition_by_name_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_by_name_with_environment_context failed: unknown result"); +} + +void ThriftHiveMetastoreClient::drop_partitions_req(DropPartitionsResult& _return, const DropPartitionsRequest& req) +{ + send_drop_partitions_req(req); + recv_drop_partitions_req(_return); +} + +void ThriftHiveMetastoreClient::send_drop_partitions_req(const DropPartitionsRequest& req) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("drop_partitions_req", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_partitions_req_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_drop_partitions_req(DropPartitionsResult& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_partitions_req") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_drop_partitions_req_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partitions_req failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) +{ + send_get_partition(db_name, tbl_name, part_vals); + recv_get_partition(_return); +} + +void ThriftHiveMetastoreClient::send_get_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partition_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_partition(Partition& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partition") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_partition_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition failed: unknown result"); +} + +void ThriftHiveMetastoreClient::exchange_partition(Partition& _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) +{ + send_exchange_partition(partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); + recv_exchange_partition(_return); +} + +void ThriftHiveMetastoreClient::send_exchange_partition(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_exchange_partition_pargs args; + args.partitionSpecs = &partitionSpecs; + args.source_db = &source_db; + args.source_table_name = &source_table_name; + args.dest_db = &dest_db; + args.dest_table_name = &dest_table_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_exchange_partition(Partition& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("exchange_partition") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_exchange_partition_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "exchange_partition failed: unknown result"); +} + +void ThriftHiveMetastoreClient::exchange_partitions(std::vector & _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) +{ + send_exchange_partitions(partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); + recv_exchange_partitions(_return); +} + +void ThriftHiveMetastoreClient::send_exchange_partitions(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("exchange_partitions", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_exchange_partitions_pargs args; + args.partitionSpecs = &partitionSpecs; + args.source_db = &source_db; + args.source_table_name = &source_table_name; + args.dest_db = &dest_db; + args.dest_table_name = &dest_table_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_exchange_partitions(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("exchange_partitions") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_exchange_partitions_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "exchange_partitions failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_partition_with_auth(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names) +{ + send_get_partition_with_auth(db_name, tbl_name, part_vals, user_name, group_names); + recv_get_partition_with_auth(_return); +} + +void ThriftHiveMetastoreClient::send_get_partition_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_partition_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partition_with_auth_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.user_name = &user_name; + args.group_names = &group_names; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_partition_with_auth(Partition& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partition_with_auth") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_partition_with_auth_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_with_auth failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) +{ + send_get_partition_by_name(db_name, tbl_name, part_name); + recv_get_partition_by_name(_return); +} + +void ThriftHiveMetastoreClient::send_get_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partition_by_name_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_partition_by_name(Partition& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partition_by_name") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_partition_by_name_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_by_name failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_partitions(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) +{ + send_get_partitions(db_name, tbl_name, max_parts); + recv_get_partitions(_return); +} + +void ThriftHiveMetastoreClient::send_get_partitions(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.max_parts = &max_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_partitions(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_partitions_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_partitions_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) +{ + send_get_partitions_with_auth(db_name, tbl_name, max_parts, user_name, group_names); + recv_get_partitions_with_auth(_return); +} + +void ThriftHiveMetastoreClient::send_get_partitions_with_auth(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_with_auth_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.max_parts = &max_parts; + args.user_name = &user_name; + args.group_names = &group_names; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_partitions_with_auth(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions_with_auth") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_partitions_with_auth_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_with_auth failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_partitions_pspec(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int32_t max_parts) +{ + send_get_partitions_pspec(db_name, tbl_name, max_parts); + recv_get_partitions_pspec(_return); +} + +void ThriftHiveMetastoreClient::send_get_partitions_pspec(const std::string& db_name, const std::string& tbl_name, const int32_t max_parts) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_partitions_pspec", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_pspec_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.max_parts = &max_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_partitions_pspec(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions_pspec") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_partitions_pspec_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_pspec failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_partition_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) +{ + send_get_partition_names(db_name, tbl_name, max_parts); + recv_get_partition_names(_return); +} + +void ThriftHiveMetastoreClient::send_get_partition_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partition_names_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.max_parts = &max_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_partition_names(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partition_names") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_partition_names_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_names failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_partition_values(PartitionValuesResponse& _return, const PartitionValuesRequest& request) +{ + send_get_partition_values(request); + recv_get_partition_values(_return); +} + +void ThriftHiveMetastoreClient::send_get_partition_values(const PartitionValuesRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_partition_values", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partition_values_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_partition_values(PartitionValuesResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partition_values") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_partition_values_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_values failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_partitions_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) +{ + send_get_partitions_ps(db_name, tbl_name, part_vals, max_parts); + recv_get_partitions_ps(_return); +} + +void ThriftHiveMetastoreClient::send_get_partitions_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_ps_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.max_parts = &max_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_partitions_ps(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions_ps") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_partitions_ps_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_ps failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_partitions_ps_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) +{ + send_get_partitions_ps_with_auth(db_name, tbl_name, part_vals, max_parts, user_name, group_names); + recv_get_partitions_ps_with_auth(_return); +} + +void ThriftHiveMetastoreClient::send_get_partitions_ps_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_ps_with_auth_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.max_parts = &max_parts; + args.user_name = &user_name; + args.group_names = &group_names; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_partitions_ps_with_auth(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions_ps_with_auth") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_partitions_ps_with_auth_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_ps_with_auth failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_partition_names_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) +{ + send_get_partition_names_ps(db_name, tbl_name, part_vals, max_parts); + recv_get_partition_names_ps(_return); +} + +void ThriftHiveMetastoreClient::send_get_partition_names_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partition_names_ps_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.max_parts = &max_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_partition_names_ps(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partition_names_ps") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_partition_names_ps_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_names_ps failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_partitions_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts) +{ + send_get_partitions_by_filter(db_name, tbl_name, filter, max_parts); + recv_get_partitions_by_filter(_return); +} + +void ThriftHiveMetastoreClient::send_get_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_by_filter_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.filter = &filter; + args.max_parts = &max_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_partitions_by_filter(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions_by_filter") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_partitions_by_filter_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_filter failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_part_specs_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts) +{ + send_get_part_specs_by_filter(db_name, tbl_name, filter, max_parts); + recv_get_part_specs_by_filter(_return); +} + +void ThriftHiveMetastoreClient::send_get_part_specs_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_part_specs_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_part_specs_by_filter_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.filter = &filter; + args.max_parts = &max_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_part_specs_by_filter(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_part_specs_by_filter") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_part_specs_by_filter_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_part_specs_by_filter failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_partitions_by_expr(PartitionsByExprResult& _return, const PartitionsByExprRequest& req) +{ + send_get_partitions_by_expr(req); + recv_get_partitions_by_expr(_return); +} + +void ThriftHiveMetastoreClient::send_get_partitions_by_expr(const PartitionsByExprRequest& req) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_partitions_by_expr", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_by_expr_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_partitions_by_expr(PartitionsByExprResult& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions_by_expr") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_partitions_by_expr_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_expr failed: unknown result"); +} + +int32_t ThriftHiveMetastoreClient::get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter) +{ + send_get_num_partitions_by_filter(db_name, tbl_name, filter); + return recv_get_num_partitions_by_filter(); +} + +void ThriftHiveMetastoreClient::send_get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_num_partitions_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_num_partitions_by_filter_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.filter = &filter; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +int32_t ThriftHiveMetastoreClient::recv_get_num_partitions_by_filter() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_num_partitions_by_filter") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + int32_t _return; + ThriftHiveMetastore_get_num_partitions_by_filter_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_num_partitions_by_filter failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_partitions_by_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & names) +{ + send_get_partitions_by_names(db_name, tbl_name, names); + recv_get_partitions_by_names(_return); +} + +void ThriftHiveMetastoreClient::send_get_partitions_by_names(const std::string& db_name, const std::string& tbl_name, const std::vector & names) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_by_names_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.names = &names; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_partitions_by_names(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions_by_names") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_partitions_by_names_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_names failed: unknown result"); +} + +void ThriftHiveMetastoreClient::alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part) +{ + send_alter_partition(db_name, tbl_name, new_part); + recv_alter_partition(); +} + +void ThriftHiveMetastoreClient::send_alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_partition_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.new_part = &new_part; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_alter_partition() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_partition") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_alter_partition_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +void ThriftHiveMetastoreClient::alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) +{ + send_alter_partitions(db_name, tbl_name, new_parts); + recv_alter_partitions(); +} + +void ThriftHiveMetastoreClient::send_alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_partitions_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.new_parts = &new_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_alter_partitions() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_partitions") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_alter_partitions_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +void ThriftHiveMetastoreClient::alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context) +{ + send_alter_partitions_with_environment_context(db_name, tbl_name, new_parts, environment_context); + recv_alter_partitions_with_environment_context(); +} + +void ThriftHiveMetastoreClient::send_alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("alter_partitions_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_partitions_with_environment_context_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.new_parts = &new_parts; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_alter_partitions_with_environment_context() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_partitions_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_alter_partitions_with_environment_context_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +void ThriftHiveMetastoreClient::alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context) +{ + send_alter_partition_with_environment_context(db_name, tbl_name, new_part, environment_context); + recv_alter_partition_with_environment_context(); +} + +void ThriftHiveMetastoreClient::send_alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_partition_with_environment_context_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.new_part = &new_part; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_alter_partition_with_environment_context() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_partition_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_alter_partition_with_environment_context_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +void ThriftHiveMetastoreClient::rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) +{ + send_rename_partition(db_name, tbl_name, part_vals, new_part); + recv_rename_partition(); +} + +void ThriftHiveMetastoreClient::send_rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_rename_partition_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.new_part = &new_part; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_rename_partition() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("rename_partition") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_rename_partition_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +bool ThriftHiveMetastoreClient::partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) +{ + send_partition_name_has_valid_characters(part_vals, throw_exception); + return recv_partition_name_has_valid_characters(); +} + +void ThriftHiveMetastoreClient::send_partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("partition_name_has_valid_characters", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_partition_name_has_valid_characters_pargs args; + args.part_vals = &part_vals; + args.throw_exception = &throw_exception; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_partition_name_has_valid_characters() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("partition_name_has_valid_characters") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_partition_name_has_valid_characters_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_has_valid_characters failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_config_value(std::string& _return, const std::string& name, const std::string& defaultValue) +{ + send_get_config_value(name, defaultValue); + recv_get_config_value(_return); +} + +void ThriftHiveMetastoreClient::send_get_config_value(const std::string& name, const std::string& defaultValue) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_config_value_pargs args; + args.name = &name; + args.defaultValue = &defaultValue; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_config_value(std::string& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_config_value") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_config_value_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_config_value failed: unknown result"); +} + +void ThriftHiveMetastoreClient::partition_name_to_vals(std::vector & _return, const std::string& part_name) +{ + send_partition_name_to_vals(part_name); + recv_partition_name_to_vals(_return); +} + +void ThriftHiveMetastoreClient::send_partition_name_to_vals(const std::string& part_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_partition_name_to_vals_pargs args; + args.part_name = &part_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_partition_name_to_vals(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("partition_name_to_vals") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_partition_name_to_vals_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_to_vals failed: unknown result"); +} + +void ThriftHiveMetastoreClient::partition_name_to_spec(std::map & _return, const std::string& part_name) +{ + send_partition_name_to_spec(part_name); + recv_partition_name_to_spec(_return); +} + +void ThriftHiveMetastoreClient::send_partition_name_to_spec(const std::string& part_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_partition_name_to_spec_pargs args; + args.part_name = &part_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_partition_name_to_spec(std::map & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("partition_name_to_spec") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_partition_name_to_spec_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_to_spec failed: unknown result"); +} + +void ThriftHiveMetastoreClient::markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) +{ + send_markPartitionForEvent(db_name, tbl_name, part_vals, eventType); + recv_markPartitionForEvent(); +} + +void ThriftHiveMetastoreClient::send_markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_markPartitionForEvent_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.eventType = &eventType; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_markPartitionForEvent() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("markPartitionForEvent") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_markPartitionForEvent_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + if (result.__isset.o5) { + throw result.o5; + } + if (result.__isset.o6) { + throw result.o6; + } + return; +} + +bool ThriftHiveMetastoreClient::isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) +{ + send_isPartitionMarkedForEvent(db_name, tbl_name, part_vals, eventType); + return recv_isPartitionMarkedForEvent(); +} + +void ThriftHiveMetastoreClient::send_isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_isPartitionMarkedForEvent_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.eventType = &eventType; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_isPartitionMarkedForEvent() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("isPartitionMarkedForEvent") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_isPartitionMarkedForEvent_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + if (result.__isset.o5) { + throw result.o5; + } + if (result.__isset.o6) { + throw result.o6; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "isPartitionMarkedForEvent failed: unknown result"); +} + +void ThriftHiveMetastoreClient::add_index(Index& _return, const Index& new_index, const Table& index_table) +{ + send_add_index(new_index, index_table); + recv_add_index(_return); +} + +void ThriftHiveMetastoreClient::send_add_index(const Index& new_index, const Table& index_table) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("add_index", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_index_pargs args; + args.new_index = &new_index; + args.index_table = &index_table; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_add_index(Index& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_index") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_add_index_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_index failed: unknown result"); +} + +void ThriftHiveMetastoreClient::alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) +{ + send_alter_index(dbname, base_tbl_name, idx_name, new_idx); + recv_alter_index(); +} + +void ThriftHiveMetastoreClient::send_alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_index_pargs args; + args.dbname = &dbname; + args.base_tbl_name = &base_tbl_name; + args.idx_name = &idx_name; + args.new_idx = &new_idx; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_alter_index() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_index") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_alter_index_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +bool ThriftHiveMetastoreClient::drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData) +{ + send_drop_index_by_name(db_name, tbl_name, index_name, deleteData); + return recv_drop_index_by_name(); +} + +void ThriftHiveMetastoreClient::send_drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_index_by_name_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.index_name = &index_name; + args.deleteData = &deleteData; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_drop_index_by_name() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_index_by_name") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_drop_index_by_name_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_index_by_name failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_index_by_name(Index& _return, const std::string& db_name, const std::string& tbl_name, const std::string& index_name) +{ + send_get_index_by_name(db_name, tbl_name, index_name); + recv_get_index_by_name(_return); +} + +void ThriftHiveMetastoreClient::send_get_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_index_by_name_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.index_name = &index_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_index_by_name(Index& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_index_by_name") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_index_by_name_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_index_by_name failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_indexes(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) +{ + send_get_indexes(db_name, tbl_name, max_indexes); + recv_get_indexes(_return); +} + +void ThriftHiveMetastoreClient::send_get_indexes(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_indexes_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.max_indexes = &max_indexes; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_indexes(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_indexes") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_indexes_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_indexes failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_index_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) +{ + send_get_index_names(db_name, tbl_name, max_indexes); + recv_get_index_names(_return); +} + +void ThriftHiveMetastoreClient::send_get_index_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_index_names_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.max_indexes = &max_indexes; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_index_names(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_index_names") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_index_names_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_index_names failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request) +{ + send_get_primary_keys(request); + recv_get_primary_keys(_return); +} + +void ThriftHiveMetastoreClient::send_get_primary_keys(const PrimaryKeysRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_primary_keys", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_primary_keys_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_primary_keys(PrimaryKeysResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_primary_keys") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_primary_keys_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_primary_keys failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request) +{ + send_get_foreign_keys(request); + recv_get_foreign_keys(_return); +} + +void ThriftHiveMetastoreClient::send_get_foreign_keys(const ForeignKeysRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_foreign_keys", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_foreign_keys_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_foreign_keys(ForeignKeysResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_foreign_keys") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_foreign_keys_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_foreign_keys failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_unique_constraints(UniqueConstraintsResponse& _return, const UniqueConstraintsRequest& request) +{ + send_get_unique_constraints(request); + recv_get_unique_constraints(_return); +} + +void ThriftHiveMetastoreClient::send_get_unique_constraints(const UniqueConstraintsRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_unique_constraints", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_unique_constraints_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_unique_constraints(UniqueConstraintsResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_unique_constraints") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_unique_constraints_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_unique_constraints failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_not_null_constraints(NotNullConstraintsResponse& _return, const NotNullConstraintsRequest& request) +{ + send_get_not_null_constraints(request); + recv_get_not_null_constraints(_return); +} + +void ThriftHiveMetastoreClient::send_get_not_null_constraints(const NotNullConstraintsRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_not_null_constraints", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_not_null_constraints_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_not_null_constraints(NotNullConstraintsResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_not_null_constraints") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_not_null_constraints_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_not_null_constraints failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::update_table_column_statistics(const ColumnStatistics& stats_obj) +{ + send_update_table_column_statistics(stats_obj); + return recv_update_table_column_statistics(); +} + +void ThriftHiveMetastoreClient::send_update_table_column_statistics(const ColumnStatistics& stats_obj) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("update_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_update_table_column_statistics_pargs args; + args.stats_obj = &stats_obj; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_update_table_column_statistics() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("update_table_column_statistics") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_update_table_column_statistics_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "update_table_column_statistics failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::update_partition_column_statistics(const ColumnStatistics& stats_obj) +{ + send_update_partition_column_statistics(stats_obj); + return recv_update_partition_column_statistics(); +} + +void ThriftHiveMetastoreClient::send_update_partition_column_statistics(const ColumnStatistics& stats_obj) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_update_partition_column_statistics_pargs args; + args.stats_obj = &stats_obj; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_update_partition_column_statistics() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("update_partition_column_statistics") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_update_partition_column_statistics_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "update_partition_column_statistics failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_table_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& col_name) +{ + send_get_table_column_statistics(db_name, tbl_name, col_name); + recv_get_table_column_statistics(_return); +} + +void ThriftHiveMetastoreClient::send_get_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_table_column_statistics_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.col_name = &col_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_table_column_statistics(ColumnStatistics& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_table_column_statistics") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_table_column_statistics_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_column_statistics failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_partition_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) +{ + send_get_partition_column_statistics(db_name, tbl_name, part_name, col_name); + recv_get_partition_column_statistics(_return); +} + +void ThriftHiveMetastoreClient::send_get_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partition_column_statistics_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; + args.col_name = &col_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_partition_column_statistics(ColumnStatistics& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partition_column_statistics") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_partition_column_statistics_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_column_statistics failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_table_statistics_req(TableStatsResult& _return, const TableStatsRequest& request) +{ + send_get_table_statistics_req(request); + recv_get_table_statistics_req(_return); +} + +void ThriftHiveMetastoreClient::send_get_table_statistics_req(const TableStatsRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_table_statistics_req", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_table_statistics_req_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_table_statistics_req(TableStatsResult& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_table_statistics_req") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_table_statistics_req_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_statistics_req failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_partitions_statistics_req(PartitionsStatsResult& _return, const PartitionsStatsRequest& request) +{ + send_get_partitions_statistics_req(request); + recv_get_partitions_statistics_req(_return); +} + +void ThriftHiveMetastoreClient::send_get_partitions_statistics_req(const PartitionsStatsRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_partitions_statistics_req", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_statistics_req_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_partitions_statistics_req(PartitionsStatsResult& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions_statistics_req") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_partitions_statistics_req_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_statistics_req failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_aggr_stats_for(AggrStats& _return, const PartitionsStatsRequest& request) +{ + send_get_aggr_stats_for(request); + recv_get_aggr_stats_for(_return); +} + +void ThriftHiveMetastoreClient::send_get_aggr_stats_for(const PartitionsStatsRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_aggr_stats_for", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_aggr_stats_for_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_aggr_stats_for(AggrStats& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_aggr_stats_for") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_aggr_stats_for_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_aggr_stats_for failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::set_aggr_stats_for(const SetPartitionsStatsRequest& request) +{ + send_set_aggr_stats_for(request); + return recv_set_aggr_stats_for(); +} + +void ThriftHiveMetastoreClient::send_set_aggr_stats_for(const SetPartitionsStatsRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("set_aggr_stats_for", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_set_aggr_stats_for_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_set_aggr_stats_for() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("set_aggr_stats_for") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_set_aggr_stats_for_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "set_aggr_stats_for failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) +{ + send_delete_partition_column_statistics(db_name, tbl_name, part_name, col_name); + return recv_delete_partition_column_statistics(); +} + +void ThriftHiveMetastoreClient::send_delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_delete_partition_column_statistics_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; + args.col_name = &col_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_delete_partition_column_statistics() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("delete_partition_column_statistics") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_delete_partition_column_statistics_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "delete_partition_column_statistics failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) +{ + send_delete_table_column_statistics(db_name, tbl_name, col_name); + return recv_delete_table_column_statistics(); +} + +void ThriftHiveMetastoreClient::send_delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_delete_table_column_statistics_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.col_name = &col_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_delete_table_column_statistics() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("delete_table_column_statistics") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_delete_table_column_statistics_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "delete_table_column_statistics failed: unknown result"); +} + +void ThriftHiveMetastoreClient::create_function(const Function& func) +{ + send_create_function(func); + recv_create_function(); +} + +void ThriftHiveMetastoreClient::send_create_function(const Function& func) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("create_function", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_function_pargs args; + args.func = &func; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_create_function() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_function") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_create_function_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + return; +} + +void ThriftHiveMetastoreClient::drop_function(const std::string& dbName, const std::string& funcName) +{ + send_drop_function(dbName, funcName); + recv_drop_function(); +} + +void ThriftHiveMetastoreClient::send_drop_function(const std::string& dbName, const std::string& funcName) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("drop_function", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_function_pargs args; + args.dbName = &dbName; + args.funcName = &funcName; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_drop_function() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_function") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_drop_function_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o3) { + throw result.o3; + } + return; +} + +void ThriftHiveMetastoreClient::alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc) +{ + send_alter_function(dbName, funcName, newFunc); + recv_alter_function(); +} + +void ThriftHiveMetastoreClient::send_alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("alter_function", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_function_pargs args; + args.dbName = &dbName; + args.funcName = &funcName; + args.newFunc = &newFunc; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_alter_function() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_function") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_alter_function_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +void ThriftHiveMetastoreClient::get_functions(std::vector & _return, const std::string& dbName, const std::string& pattern) +{ + send_get_functions(dbName, pattern); + recv_get_functions(_return); +} + +void ThriftHiveMetastoreClient::send_get_functions(const std::string& dbName, const std::string& pattern) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_functions", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_functions_pargs args; + args.dbName = &dbName; + args.pattern = &pattern; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_functions(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_functions") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_functions_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_functions failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_function(Function& _return, const std::string& dbName, const std::string& funcName) +{ + send_get_function(dbName, funcName); + recv_get_function(_return); +} + +void ThriftHiveMetastoreClient::send_get_function(const std::string& dbName, const std::string& funcName) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_function", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_function_pargs args; + args.dbName = &dbName; + args.funcName = &funcName; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_function(Function& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_function") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_function_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_function failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_all_functions(GetAllFunctionsResponse& _return) +{ + send_get_all_functions(); + recv_get_all_functions(_return); +} + +void ThriftHiveMetastoreClient::send_get_all_functions() +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_all_functions", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_all_functions_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_all_functions(GetAllFunctionsResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_all_functions") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_all_functions_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_functions failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::create_role(const Role& role) +{ + send_create_role(role); + return recv_create_role(); +} + +void ThriftHiveMetastoreClient::send_create_role(const Role& role) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("create_role", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_role_pargs args; + args.role = &role; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_create_role() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_role") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_create_role_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_role failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::drop_role(const std::string& role_name) +{ + send_drop_role(role_name); + return recv_drop_role(); +} + +void ThriftHiveMetastoreClient::send_drop_role(const std::string& role_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("drop_role", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_role_pargs args; + args.role_name = &role_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_drop_role() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_role") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_drop_role_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_role failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_role_names(std::vector & _return) +{ + send_get_role_names(); + recv_get_role_names(_return); +} + +void ThriftHiveMetastoreClient::send_get_role_names() +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_role_names", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_role_names_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_role_names(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_role_names") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_role_names_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_role_names failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option) +{ + send_grant_role(role_name, principal_name, principal_type, grantor, grantorType, grant_option); + return recv_grant_role(); +} + +void ThriftHiveMetastoreClient::send_grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("grant_role", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_grant_role_pargs args; + args.role_name = &role_name; + args.principal_name = &principal_name; + args.principal_type = &principal_type; + args.grantor = &grantor; + args.grantorType = &grantorType; + args.grant_option = &grant_option; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_grant_role() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("grant_role") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_grant_role_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_role failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type) +{ + send_revoke_role(role_name, principal_name, principal_type); + return recv_revoke_role(); +} + +void ThriftHiveMetastoreClient::send_revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("revoke_role", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_revoke_role_pargs args; + args.role_name = &role_name; + args.principal_name = &principal_name; + args.principal_type = &principal_type; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_revoke_role() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("revoke_role") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_revoke_role_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "revoke_role failed: unknown result"); +} + +void ThriftHiveMetastoreClient::list_roles(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type) +{ + send_list_roles(principal_name, principal_type); + recv_list_roles(_return); +} + +void ThriftHiveMetastoreClient::send_list_roles(const std::string& principal_name, const PrincipalType::type principal_type) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("list_roles", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_list_roles_pargs args; + args.principal_name = &principal_name; + args.principal_type = &principal_type; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_list_roles(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("list_roles") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_list_roles_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "list_roles failed: unknown result"); +} + +void ThriftHiveMetastoreClient::grant_revoke_role(GrantRevokeRoleResponse& _return, const GrantRevokeRoleRequest& request) +{ + send_grant_revoke_role(request); + recv_grant_revoke_role(_return); +} + +void ThriftHiveMetastoreClient::send_grant_revoke_role(const GrantRevokeRoleRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("grant_revoke_role", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_grant_revoke_role_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_grant_revoke_role(GrantRevokeRoleResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("grant_revoke_role") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_grant_revoke_role_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_revoke_role failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_principals_in_role(GetPrincipalsInRoleResponse& _return, const GetPrincipalsInRoleRequest& request) +{ + send_get_principals_in_role(request); + recv_get_principals_in_role(_return); +} + +void ThriftHiveMetastoreClient::send_get_principals_in_role(const GetPrincipalsInRoleRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_principals_in_role", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_principals_in_role_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_principals_in_role(GetPrincipalsInRoleResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_principals_in_role") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_principals_in_role_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_principals_in_role failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return, const GetRoleGrantsForPrincipalRequest& request) +{ + send_get_role_grants_for_principal(request); + recv_get_role_grants_for_principal(_return); +} + +void ThriftHiveMetastoreClient::send_get_role_grants_for_principal(const GetRoleGrantsForPrincipalRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_role_grants_for_principal", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_role_grants_for_principal_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_role_grants_for_principal") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_role_grants_for_principal_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_role_grants_for_principal failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_privilege_set(PrincipalPrivilegeSet& _return, const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names) +{ + send_get_privilege_set(hiveObject, user_name, group_names); + recv_get_privilege_set(_return); +} + +void ThriftHiveMetastoreClient::send_get_privilege_set(const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_privilege_set", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_privilege_set_pargs args; + args.hiveObject = &hiveObject; + args.user_name = &user_name; + args.group_names = &group_names; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_privilege_set(PrincipalPrivilegeSet& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_privilege_set") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_privilege_set_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_privilege_set failed: unknown result"); +} + +void ThriftHiveMetastoreClient::list_privileges(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) +{ + send_list_privileges(principal_name, principal_type, hiveObject); + recv_list_privileges(_return); +} + +void ThriftHiveMetastoreClient::send_list_privileges(const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("list_privileges", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_list_privileges_pargs args; + args.principal_name = &principal_name; + args.principal_type = &principal_type; + args.hiveObject = &hiveObject; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_list_privileges(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("list_privileges") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_list_privileges_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "list_privileges failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::grant_privileges(const PrivilegeBag& privileges) +{ + send_grant_privileges(privileges); + return recv_grant_privileges(); +} + +void ThriftHiveMetastoreClient::send_grant_privileges(const PrivilegeBag& privileges) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("grant_privileges", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_grant_privileges_pargs args; + args.privileges = &privileges; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_grant_privileges() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("grant_privileges") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_grant_privileges_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_privileges failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::revoke_privileges(const PrivilegeBag& privileges) +{ + send_revoke_privileges(privileges); + return recv_revoke_privileges(); +} + +void ThriftHiveMetastoreClient::send_revoke_privileges(const PrivilegeBag& privileges) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("revoke_privileges", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_revoke_privileges_pargs args; + args.privileges = &privileges; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_revoke_privileges() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("revoke_privileges") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_revoke_privileges_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "revoke_privileges failed: unknown result"); +} + +void ThriftHiveMetastoreClient::grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const GrantRevokePrivilegeRequest& request) +{ + send_grant_revoke_privileges(request); + recv_grant_revoke_privileges(_return); +} + +void ThriftHiveMetastoreClient::send_grant_revoke_privileges(const GrantRevokePrivilegeRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("grant_revoke_privileges", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_grant_revoke_privileges_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_grant_revoke_privileges(GrantRevokePrivilegeResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("grant_revoke_privileges") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_grant_revoke_privileges_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_revoke_privileges failed: unknown result"); +} + +void ThriftHiveMetastoreClient::set_ugi(std::vector & _return, const std::string& user_name, const std::vector & group_names) +{ + send_set_ugi(user_name, group_names); + recv_set_ugi(_return); +} + +void ThriftHiveMetastoreClient::send_set_ugi(const std::string& user_name, const std::vector & group_names) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("set_ugi", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_set_ugi_pargs args; + args.user_name = &user_name; + args.group_names = &group_names; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_set_ugi(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("set_ugi") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_set_ugi_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "set_ugi failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name) +{ + send_get_delegation_token(token_owner, renewer_kerberos_principal_name); + recv_get_delegation_token(_return); +} + +void ThriftHiveMetastoreClient::send_get_delegation_token(const std::string& token_owner, const std::string& renewer_kerberos_principal_name) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_delegation_token_pargs args; + args.token_owner = &token_owner; + args.renewer_kerberos_principal_name = &renewer_kerberos_principal_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_delegation_token(std::string& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_delegation_token") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_delegation_token_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_delegation_token failed: unknown result"); +} + +int64_t ThriftHiveMetastoreClient::renew_delegation_token(const std::string& token_str_form) +{ + send_renew_delegation_token(token_str_form); + return recv_renew_delegation_token(); +} + +void ThriftHiveMetastoreClient::send_renew_delegation_token(const std::string& token_str_form) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("renew_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_renew_delegation_token_pargs args; + args.token_str_form = &token_str_form; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +int64_t ThriftHiveMetastoreClient::recv_renew_delegation_token() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("renew_delegation_token") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + int64_t _return; + ThriftHiveMetastore_renew_delegation_token_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "renew_delegation_token failed: unknown result"); +} + +void ThriftHiveMetastoreClient::cancel_delegation_token(const std::string& token_str_form) +{ + send_cancel_delegation_token(token_str_form); + recv_cancel_delegation_token(); +} + +void ThriftHiveMetastoreClient::send_cancel_delegation_token(const std::string& token_str_form) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("cancel_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_cancel_delegation_token_pargs args; + args.token_str_form = &token_str_form; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_cancel_delegation_token() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("cancel_delegation_token") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_cancel_delegation_token_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + return; +} + +bool ThriftHiveMetastoreClient::add_token(const std::string& token_identifier, const std::string& delegation_token) +{ + send_add_token(token_identifier, delegation_token); + return recv_add_token(); +} + +void ThriftHiveMetastoreClient::send_add_token(const std::string& token_identifier, const std::string& delegation_token) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("add_token", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_token_pargs args; + args.token_identifier = &token_identifier; + args.delegation_token = &delegation_token; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_add_token() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_token") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_add_token_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_token failed: unknown result"); +} + +bool ThriftHiveMetastoreClient::remove_token(const std::string& token_identifier) +{ + send_remove_token(token_identifier); + return recv_remove_token(); +} + +void ThriftHiveMetastoreClient::send_remove_token(const std::string& token_identifier) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("remove_token", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_remove_token_pargs args; + args.token_identifier = &token_identifier; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_remove_token() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("remove_token") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_remove_token_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "remove_token failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_token(std::string& _return, const std::string& token_identifier) +{ + send_get_token(token_identifier); + recv_get_token(_return); +} + +void ThriftHiveMetastoreClient::send_get_token(const std::string& token_identifier) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_token", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_token_pargs args; + args.token_identifier = &token_identifier; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_token(std::string& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_token") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_token_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_token failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_all_token_identifiers(std::vector & _return) +{ + send_get_all_token_identifiers(); + recv_get_all_token_identifiers(_return); +} + +void ThriftHiveMetastoreClient::send_get_all_token_identifiers() +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_all_token_identifiers", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_all_token_identifiers_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_all_token_identifiers(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_all_token_identifiers") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_all_token_identifiers_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_token_identifiers failed: unknown result"); +} + +int32_t ThriftHiveMetastoreClient::add_master_key(const std::string& key) +{ + send_add_master_key(key); + return recv_add_master_key(); +} + +void ThriftHiveMetastoreClient::send_add_master_key(const std::string& key) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("add_master_key", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_master_key_pargs args; + args.key = &key; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +int32_t ThriftHiveMetastoreClient::recv_add_master_key() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_master_key") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + int32_t _return; + ThriftHiveMetastore_add_master_key_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_master_key failed: unknown result"); +} + +void ThriftHiveMetastoreClient::update_master_key(const int32_t seq_number, const std::string& key) +{ + send_update_master_key(seq_number, key); + recv_update_master_key(); +} + +void ThriftHiveMetastoreClient::send_update_master_key(const int32_t seq_number, const std::string& key) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("update_master_key", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_update_master_key_pargs args; + args.seq_number = &seq_number; + args.key = &key; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_update_master_key() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("update_master_key") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_update_master_key_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +bool ThriftHiveMetastoreClient::remove_master_key(const int32_t key_seq) +{ + send_remove_master_key(key_seq); + return recv_remove_master_key(); +} + +void ThriftHiveMetastoreClient::send_remove_master_key(const int32_t key_seq) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("remove_master_key", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_remove_master_key_pargs args; + args.key_seq = &key_seq; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +bool ThriftHiveMetastoreClient::recv_remove_master_key() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("remove_master_key") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + bool _return; + ThriftHiveMetastore_remove_master_key_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + return _return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "remove_master_key failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_master_keys(std::vector & _return) +{ + send_get_master_keys(); + recv_get_master_keys(_return); +} + +void ThriftHiveMetastoreClient::send_get_master_keys() +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_master_keys", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_master_keys_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_master_keys(std::vector & _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_master_keys") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_master_keys_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_master_keys failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_open_txns(GetOpenTxnsResponse& _return) +{ + send_get_open_txns(); + recv_get_open_txns(_return); +} + +void ThriftHiveMetastoreClient::send_get_open_txns() +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_open_txns", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_open_txns_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_open_txns(GetOpenTxnsResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_open_txns") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_open_txns_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_open_txns failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_open_txns_info(GetOpenTxnsInfoResponse& _return) +{ + send_get_open_txns_info(); + recv_get_open_txns_info(_return); +} + +void ThriftHiveMetastoreClient::send_get_open_txns_info() +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_open_txns_info", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_open_txns_info_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_open_txns_info(GetOpenTxnsInfoResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_open_txns_info") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_open_txns_info_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_open_txns_info failed: unknown result"); +} + +void ThriftHiveMetastoreClient::open_txns(OpenTxnsResponse& _return, const OpenTxnRequest& rqst) +{ + send_open_txns(rqst); + recv_open_txns(_return); +} + +void ThriftHiveMetastoreClient::send_open_txns(const OpenTxnRequest& rqst) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("open_txns", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_open_txns_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_open_txns(OpenTxnsResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("open_txns") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_open_txns_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "open_txns failed: unknown result"); +} + +void ThriftHiveMetastoreClient::abort_txn(const AbortTxnRequest& rqst) +{ + send_abort_txn(rqst); + recv_abort_txn(); +} + +void ThriftHiveMetastoreClient::send_abort_txn(const AbortTxnRequest& rqst) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("abort_txn", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_abort_txn_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_abort_txn() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("abort_txn") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_abort_txn_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + return; +} + +void ThriftHiveMetastoreClient::abort_txns(const AbortTxnsRequest& rqst) +{ + send_abort_txns(rqst); + recv_abort_txns(); +} + +void ThriftHiveMetastoreClient::send_abort_txns(const AbortTxnsRequest& rqst) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("abort_txns", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_abort_txns_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_abort_txns() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("abort_txns") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_abort_txns_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + return; +} + +void ThriftHiveMetastoreClient::commit_txn(const CommitTxnRequest& rqst) +{ + send_commit_txn(rqst); + recv_commit_txn(); +} + +void ThriftHiveMetastoreClient::send_commit_txn(const CommitTxnRequest& rqst) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("commit_txn", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_commit_txn_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_commit_txn() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("commit_txn") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_commit_txn_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +void ThriftHiveMetastoreClient::lock(LockResponse& _return, const LockRequest& rqst) +{ + send_lock(rqst); + recv_lock(_return); +} + +void ThriftHiveMetastoreClient::send_lock(const LockRequest& rqst) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("lock", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_lock_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_lock(LockResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("lock") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_lock_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "lock failed: unknown result"); +} + +void ThriftHiveMetastoreClient::check_lock(LockResponse& _return, const CheckLockRequest& rqst) +{ + send_check_lock(rqst); + recv_check_lock(_return); +} + +void ThriftHiveMetastoreClient::send_check_lock(const CheckLockRequest& rqst) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("check_lock", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_check_lock_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_check_lock(LockResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("check_lock") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_check_lock_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "check_lock failed: unknown result"); +} + +void ThriftHiveMetastoreClient::unlock(const UnlockRequest& rqst) +{ + send_unlock(rqst); + recv_unlock(); +} + +void ThriftHiveMetastoreClient::send_unlock(const UnlockRequest& rqst) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("unlock", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_unlock_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_unlock() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("unlock") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_unlock_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +void ThriftHiveMetastoreClient::show_locks(ShowLocksResponse& _return, const ShowLocksRequest& rqst) +{ + send_show_locks(rqst); + recv_show_locks(_return); +} + +void ThriftHiveMetastoreClient::send_show_locks(const ShowLocksRequest& rqst) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("show_locks", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_show_locks_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_show_locks(ShowLocksResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("show_locks") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_show_locks_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "show_locks failed: unknown result"); +} + +void ThriftHiveMetastoreClient::heartbeat(const HeartbeatRequest& ids) +{ + send_heartbeat(ids); + recv_heartbeat(); +} + +void ThriftHiveMetastoreClient::send_heartbeat(const HeartbeatRequest& ids) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("heartbeat", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_heartbeat_pargs args; + args.ids = &ids; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_heartbeat() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("heartbeat") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_heartbeat_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + return; +} + +void ThriftHiveMetastoreClient::heartbeat_txn_range(HeartbeatTxnRangeResponse& _return, const HeartbeatTxnRangeRequest& txns) +{ + send_heartbeat_txn_range(txns); + recv_heartbeat_txn_range(_return); +} + +void ThriftHiveMetastoreClient::send_heartbeat_txn_range(const HeartbeatTxnRangeRequest& txns) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("heartbeat_txn_range", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_heartbeat_txn_range_pargs args; + args.txns = &txns; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_heartbeat_txn_range(HeartbeatTxnRangeResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("heartbeat_txn_range") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_heartbeat_txn_range_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "heartbeat_txn_range failed: unknown result"); +} + +void ThriftHiveMetastoreClient::compact(const CompactionRequest& rqst) +{ + send_compact(rqst); + recv_compact(); +} + +void ThriftHiveMetastoreClient::send_compact(const CompactionRequest& rqst) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("compact", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_compact_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_compact() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("compact") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_compact_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + return; +} + +void ThriftHiveMetastoreClient::compact2(CompactionResponse& _return, const CompactionRequest& rqst) +{ + send_compact2(rqst); + recv_compact2(_return); +} + +void ThriftHiveMetastoreClient::send_compact2(const CompactionRequest& rqst) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("compact2", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_compact2_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_compact2(CompactionResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("compact2") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_compact2_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "compact2 failed: unknown result"); +} + +void ThriftHiveMetastoreClient::show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst) +{ + send_show_compact(rqst); + recv_show_compact(_return); +} + +void ThriftHiveMetastoreClient::send_show_compact(const ShowCompactRequest& rqst) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("show_compact", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_show_compact_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_show_compact(ShowCompactResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("show_compact") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_show_compact_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "show_compact failed: unknown result"); +} + +void ThriftHiveMetastoreClient::add_dynamic_partitions(const AddDynamicPartitions& rqst) +{ + send_add_dynamic_partitions(rqst); + recv_add_dynamic_partitions(); +} + +void ThriftHiveMetastoreClient::send_add_dynamic_partitions(const AddDynamicPartitions& rqst) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("add_dynamic_partitions", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_dynamic_partitions_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_add_dynamic_partitions() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_dynamic_partitions") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_add_dynamic_partitions_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + return; +} + +void ThriftHiveMetastoreClient::get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst) +{ + send_get_next_notification(rqst); + recv_get_next_notification(_return); +} + +void ThriftHiveMetastoreClient::send_get_next_notification(const NotificationEventRequest& rqst) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_next_notification", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_next_notification_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_next_notification(NotificationEventResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_next_notification") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_next_notification_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_next_notification failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_current_notificationEventId(CurrentNotificationEventId& _return) +{ + send_get_current_notificationEventId(); + recv_get_current_notificationEventId(_return); +} + +void ThriftHiveMetastoreClient::send_get_current_notificationEventId() +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_current_notificationEventId", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_current_notificationEventId_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_current_notificationEventId(CurrentNotificationEventId& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_current_notificationEventId") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_current_notificationEventId_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_current_notificationEventId failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_notification_events_count(NotificationEventsCountResponse& _return, const NotificationEventsCountRequest& rqst) +{ + send_get_notification_events_count(rqst); + recv_get_notification_events_count(_return); +} + +void ThriftHiveMetastoreClient::send_get_notification_events_count(const NotificationEventsCountRequest& rqst) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_notification_events_count", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_notification_events_count_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_notification_events_count(NotificationEventsCountResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_notification_events_count") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_notification_events_count_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_notification_events_count failed: unknown result"); +} + +void ThriftHiveMetastoreClient::fire_listener_event(FireEventResponse& _return, const FireEventRequest& rqst) +{ + send_fire_listener_event(rqst); + recv_fire_listener_event(_return); +} + +void ThriftHiveMetastoreClient::send_fire_listener_event(const FireEventRequest& rqst) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("fire_listener_event", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_fire_listener_event_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_fire_listener_event(FireEventResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("fire_listener_event") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_fire_listener_event_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "fire_listener_event failed: unknown result"); +} + +void ThriftHiveMetastoreClient::flushCache() +{ + send_flushCache(); + recv_flushCache(); +} + +void ThriftHiveMetastoreClient::send_flushCache() +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("flushCache", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_flushCache_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_flushCache() +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("flushCache") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_flushCache_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + return; +} + +void ThriftHiveMetastoreClient::cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request) +{ + send_cm_recycle(request); + recv_cm_recycle(_return); +} + +void ThriftHiveMetastoreClient::send_cm_recycle(const CmRecycleRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("cm_recycle", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_cm_recycle_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_cm_recycle(CmRecycleResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("cm_recycle") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_cm_recycle_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "cm_recycle failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req) +{ + send_get_file_metadata_by_expr(req); + recv_get_file_metadata_by_expr(_return); +} + +void ThriftHiveMetastoreClient::send_get_file_metadata_by_expr(const GetFileMetadataByExprRequest& req) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_file_metadata_by_expr", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_file_metadata_by_expr_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_file_metadata_by_expr(GetFileMetadataByExprResult& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_file_metadata_by_expr") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_file_metadata_by_expr_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_file_metadata_by_expr failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req) +{ + send_get_file_metadata(req); + recv_get_file_metadata(_return); +} + +void ThriftHiveMetastoreClient::send_get_file_metadata(const GetFileMetadataRequest& req) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_file_metadata", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_file_metadata_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_file_metadata(GetFileMetadataResult& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_file_metadata") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_file_metadata_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_file_metadata failed: unknown result"); +} + +void ThriftHiveMetastoreClient::put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req) +{ + send_put_file_metadata(req); + recv_put_file_metadata(_return); +} + +void ThriftHiveMetastoreClient::send_put_file_metadata(const PutFileMetadataRequest& req) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("put_file_metadata", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_put_file_metadata_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_put_file_metadata(PutFileMetadataResult& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("put_file_metadata") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_put_file_metadata_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "put_file_metadata failed: unknown result"); +} + +void ThriftHiveMetastoreClient::clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req) +{ + send_clear_file_metadata(req); + recv_clear_file_metadata(_return); +} + +void ThriftHiveMetastoreClient::send_clear_file_metadata(const ClearFileMetadataRequest& req) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("clear_file_metadata", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_clear_file_metadata_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_clear_file_metadata(ClearFileMetadataResult& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("clear_file_metadata") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_clear_file_metadata_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "clear_file_metadata failed: unknown result"); +} + +void ThriftHiveMetastoreClient::cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req) +{ + send_cache_file_metadata(req); + recv_cache_file_metadata(_return); +} + +void ThriftHiveMetastoreClient::send_cache_file_metadata(const CacheFileMetadataRequest& req) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("cache_file_metadata", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_cache_file_metadata_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_cache_file_metadata(CacheFileMetadataResult& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("cache_file_metadata") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_cache_file_metadata_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "cache_file_metadata failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_metastore_db_uuid(std::string& _return) +{ + send_get_metastore_db_uuid(); + recv_get_metastore_db_uuid(_return); +} + +void ThriftHiveMetastoreClient::send_get_metastore_db_uuid() +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_metastore_db_uuid", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_metastore_db_uuid_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_metastore_db_uuid(std::string& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_metastore_db_uuid") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_metastore_db_uuid_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_metastore_db_uuid failed: unknown result"); +} + +void ThriftHiveMetastoreClient::create_resource_plan(WMCreateResourcePlanResponse& _return, const WMCreateResourcePlanRequest& request) +{ + send_create_resource_plan(request); + recv_create_resource_plan(_return); +} + +void ThriftHiveMetastoreClient::send_create_resource_plan(const WMCreateResourcePlanRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("create_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_resource_plan_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_create_resource_plan(WMCreateResourcePlanResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_resource_plan") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_create_resource_plan_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_resource_plan failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_resource_plan(WMGetResourcePlanResponse& _return, const WMGetResourcePlanRequest& request) +{ + send_get_resource_plan(request); + recv_get_resource_plan(_return); +} + +void ThriftHiveMetastoreClient::send_get_resource_plan(const WMGetResourcePlanRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_resource_plan_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_resource_plan(WMGetResourcePlanResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_resource_plan") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_resource_plan_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_resource_plan failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_active_resource_plan(WMGetActiveResourcePlanResponse& _return, const WMGetActiveResourcePlanRequest& request) +{ + send_get_active_resource_plan(request); + recv_get_active_resource_plan(_return); +} + +void ThriftHiveMetastoreClient::send_get_active_resource_plan(const WMGetActiveResourcePlanRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_active_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_active_resource_plan_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_active_resource_plan(WMGetActiveResourcePlanResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_active_resource_plan") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_active_resource_plan_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_active_resource_plan failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_all_resource_plans(WMGetAllResourcePlanResponse& _return, const WMGetAllResourcePlanRequest& request) +{ + send_get_all_resource_plans(request); + recv_get_all_resource_plans(_return); +} + +void ThriftHiveMetastoreClient::send_get_all_resource_plans(const WMGetAllResourcePlanRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_all_resource_plans", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_all_resource_plans_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_all_resource_plans(WMGetAllResourcePlanResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_all_resource_plans") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_all_resource_plans_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_resource_plans failed: unknown result"); +} + +void ThriftHiveMetastoreClient::alter_resource_plan(WMAlterResourcePlanResponse& _return, const WMAlterResourcePlanRequest& request) +{ + send_alter_resource_plan(request); + recv_alter_resource_plan(_return); +} + +void ThriftHiveMetastoreClient::send_alter_resource_plan(const WMAlterResourcePlanRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("alter_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_resource_plan_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_alter_resource_plan(WMAlterResourcePlanResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_resource_plan") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_alter_resource_plan_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "alter_resource_plan failed: unknown result"); +} + +void ThriftHiveMetastoreClient::validate_resource_plan(WMValidateResourcePlanResponse& _return, const WMValidateResourcePlanRequest& request) +{ + send_validate_resource_plan(request); + recv_validate_resource_plan(_return); +} + +void ThriftHiveMetastoreClient::send_validate_resource_plan(const WMValidateResourcePlanRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("validate_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_validate_resource_plan_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_validate_resource_plan(WMValidateResourcePlanResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("validate_resource_plan") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_validate_resource_plan_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "validate_resource_plan failed: unknown result"); +} + +void ThriftHiveMetastoreClient::drop_resource_plan(WMDropResourcePlanResponse& _return, const WMDropResourcePlanRequest& request) +{ + send_drop_resource_plan(request); + recv_drop_resource_plan(_return); +} + +void ThriftHiveMetastoreClient::send_drop_resource_plan(const WMDropResourcePlanRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("drop_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_resource_plan_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_drop_resource_plan(WMDropResourcePlanResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_resource_plan") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_drop_resource_plan_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_resource_plan failed: unknown result"); +} + +void ThriftHiveMetastoreClient::create_wm_trigger(WMCreateTriggerResponse& _return, const WMCreateTriggerRequest& request) +{ + send_create_wm_trigger(request); + recv_create_wm_trigger(_return); +} + +void ThriftHiveMetastoreClient::send_create_wm_trigger(const WMCreateTriggerRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("create_wm_trigger", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_wm_trigger_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_create_wm_trigger(WMCreateTriggerResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_wm_trigger") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_create_wm_trigger_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_wm_trigger failed: unknown result"); +} + +void ThriftHiveMetastoreClient::alter_wm_trigger(WMAlterTriggerResponse& _return, const WMAlterTriggerRequest& request) +{ + send_alter_wm_trigger(request); + recv_alter_wm_trigger(_return); +} + +void ThriftHiveMetastoreClient::send_alter_wm_trigger(const WMAlterTriggerRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("alter_wm_trigger", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_wm_trigger_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_alter_wm_trigger(WMAlterTriggerResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_wm_trigger") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_alter_wm_trigger_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "alter_wm_trigger failed: unknown result"); +} + +void ThriftHiveMetastoreClient::drop_wm_trigger(WMDropTriggerResponse& _return, const WMDropTriggerRequest& request) +{ + send_drop_wm_trigger(request); + recv_drop_wm_trigger(_return); +} + +void ThriftHiveMetastoreClient::send_drop_wm_trigger(const WMDropTriggerRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("drop_wm_trigger", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_wm_trigger_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_drop_wm_trigger(WMDropTriggerResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_wm_trigger") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_drop_wm_trigger_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_wm_trigger failed: unknown result"); +} + +void ThriftHiveMetastoreClient::get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return, const WMGetTriggersForResourePlanRequest& request) +{ + send_get_triggers_for_resourceplan(request); + recv_get_triggers_for_resourceplan(_return); +} + +void ThriftHiveMetastoreClient::send_get_triggers_for_resourceplan(const WMGetTriggersForResourePlanRequest& request) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("get_triggers_for_resourceplan", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_triggers_for_resourceplan_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void ThriftHiveMetastoreClient::recv_get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_triggers_for_resourceplan") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + ThriftHiveMetastore_get_triggers_for_resourceplan_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + if (result.__isset.o1) { + throw result.o1; + } + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_triggers_for_resourceplan failed: unknown result"); +} + +bool ThriftHiveMetastoreProcessor::dispatchCall(::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, const std::string& fname, int32_t seqid, void* callContext) { + ProcessMap::iterator pfn; + pfn = processMap_.find(fname); + if (pfn == processMap_.end()) { + iprot->skip(::apache::thrift::protocol::T_STRUCT); + iprot->readMessageEnd(); + iprot->getTransport()->readEnd(); + ::apache::thrift::TApplicationException x(::apache::thrift::TApplicationException::UNKNOWN_METHOD, "Invalid method name: '"+fname+"'"); + oprot->writeMessageBegin(fname, ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return true; + } + (this->*(pfn->second))(seqid, iprot, oprot, callContext); + return true; +} + +void ThriftHiveMetastoreProcessor::process_getMetaConf(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.getMetaConf", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.getMetaConf"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.getMetaConf"); + } + + ThriftHiveMetastore_getMetaConf_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.getMetaConf", bytes); + } + + ThriftHiveMetastore_getMetaConf_result result; + try { + iface_->getMetaConf(result.success, args.key); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.getMetaConf"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("getMetaConf", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.getMetaConf"); + } + + oprot->writeMessageBegin("getMetaConf", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.getMetaConf", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_setMetaConf(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.setMetaConf", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.setMetaConf"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.setMetaConf"); + } + + ThriftHiveMetastore_setMetaConf_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.setMetaConf", bytes); + } + + ThriftHiveMetastore_setMetaConf_result result; + try { + iface_->setMetaConf(args.key, args.value); + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.setMetaConf"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("setMetaConf", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.setMetaConf"); + } + + oprot->writeMessageBegin("setMetaConf", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.setMetaConf", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_create_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_database", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_database"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_database"); + } + + ThriftHiveMetastore_create_database_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_database", bytes); + } + + ThriftHiveMetastore_create_database_result result; + try { + iface_->create_database(args.database); + } catch (AlreadyExistsException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_database"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("create_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_database"); + } + + oprot->writeMessageBegin("create_database", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_database", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_database", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_database"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_database"); + } + + ThriftHiveMetastore_get_database_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_database", bytes); + } + + ThriftHiveMetastore_get_database_result result; + try { + iface_->get_database(result.success, args.name); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_database"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_database"); + } + + oprot->writeMessageBegin("get_database", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_database", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_drop_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_database", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_database"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_database"); + } + + ThriftHiveMetastore_drop_database_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_database", bytes); + } + + ThriftHiveMetastore_drop_database_result result; + try { + iface_->drop_database(args.name, args.deleteData, args.cascade); + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidOperationException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_database"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("drop_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_database"); + } + + oprot->writeMessageBegin("drop_database", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_database", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_databases(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_databases", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_databases"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_databases"); + } + + ThriftHiveMetastore_get_databases_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_databases", bytes); + } + + ThriftHiveMetastore_get_databases_result result; + try { + iface_->get_databases(result.success, args.pattern); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_databases"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_databases", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_databases"); + } + + oprot->writeMessageBegin("get_databases", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_databases", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_all_databases(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_all_databases", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_all_databases"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_databases"); + } + + ThriftHiveMetastore_get_all_databases_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_all_databases", bytes); + } + + ThriftHiveMetastore_get_all_databases_result result; + try { + iface_->get_all_databases(result.success); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_all_databases"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_all_databases", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_databases"); + } + + oprot->writeMessageBegin("get_all_databases", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_all_databases", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_alter_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_database", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_database"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_database"); + } + + ThriftHiveMetastore_alter_database_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_database", bytes); + } + + ThriftHiveMetastore_alter_database_result result; + try { + iface_->alter_database(args.dbname, args.db); + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_database"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("alter_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_database"); + } + + oprot->writeMessageBegin("alter_database", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_database", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_type", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_type"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_type"); + } + + ThriftHiveMetastore_get_type_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_type", bytes); + } + + ThriftHiveMetastore_get_type_result result; + try { + iface_->get_type(result.success, args.name); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_type"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_type", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_type"); + } + + oprot->writeMessageBegin("get_type", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_type", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_create_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_type", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_type"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_type"); + } + + ThriftHiveMetastore_create_type_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_type", bytes); + } + + ThriftHiveMetastore_create_type_result result; + try { + result.success = iface_->create_type(args.type); + result.__isset.success = true; + } catch (AlreadyExistsException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_type"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("create_type", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_type"); + } + + oprot->writeMessageBegin("create_type", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_type", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_drop_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_type", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_type"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_type"); + } + + ThriftHiveMetastore_drop_type_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_type", bytes); + } + + ThriftHiveMetastore_drop_type_result result; + try { + result.success = iface_->drop_type(args.type); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_type"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("drop_type", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_type"); + } + + oprot->writeMessageBegin("drop_type", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_type", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_type_all(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_type_all", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_type_all"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_type_all"); + } + + ThriftHiveMetastore_get_type_all_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_type_all", bytes); + } + + ThriftHiveMetastore_get_type_all_result result; + try { + iface_->get_type_all(result.success, args.name); + result.__isset.success = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_type_all"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_type_all", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_type_all"); + } + + oprot->writeMessageBegin("get_type_all", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_type_all", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_fields(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_fields", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_fields"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_fields"); + } + + ThriftHiveMetastore_get_fields_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_fields", bytes); + } + + ThriftHiveMetastore_get_fields_result result; + try { + iface_->get_fields(result.success, args.db_name, args.table_name); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (UnknownTableException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (UnknownDBException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_fields"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_fields", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_fields"); + } + + oprot->writeMessageBegin("get_fields", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_fields", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_fields_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_fields_with_environment_context", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_fields_with_environment_context"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_fields_with_environment_context"); + } + + ThriftHiveMetastore_get_fields_with_environment_context_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_fields_with_environment_context", bytes); + } + + ThriftHiveMetastore_get_fields_with_environment_context_result result; + try { + iface_->get_fields_with_environment_context(result.success, args.db_name, args.table_name, args.environment_context); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (UnknownTableException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (UnknownDBException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_fields_with_environment_context"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_fields_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_fields_with_environment_context"); + } + + oprot->writeMessageBegin("get_fields_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_fields_with_environment_context", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_schema(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_schema", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_schema"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_schema"); + } + + ThriftHiveMetastore_get_schema_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_schema", bytes); + } + + ThriftHiveMetastore_get_schema_result result; + try { + iface_->get_schema(result.success, args.db_name, args.table_name); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (UnknownTableException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (UnknownDBException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_schema"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_schema", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_schema"); + } + + oprot->writeMessageBegin("get_schema", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_schema", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_schema_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_schema_with_environment_context", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_schema_with_environment_context"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_schema_with_environment_context"); + } + + ThriftHiveMetastore_get_schema_with_environment_context_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_schema_with_environment_context", bytes); + } + + ThriftHiveMetastore_get_schema_with_environment_context_result result; + try { + iface_->get_schema_with_environment_context(result.success, args.db_name, args.table_name, args.environment_context); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (UnknownTableException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (UnknownDBException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_schema_with_environment_context"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_schema_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_schema_with_environment_context"); + } + + oprot->writeMessageBegin("get_schema_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_schema_with_environment_context", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_create_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_table", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_table"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_table"); + } + + ThriftHiveMetastore_create_table_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_table", bytes); + } + + ThriftHiveMetastore_create_table_result result; + try { + iface_->create_table(args.tbl); + } catch (AlreadyExistsException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (NoSuchObjectException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_table"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("create_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_table"); + } + + oprot->writeMessageBegin("create_table", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_table", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_create_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_table_with_environment_context", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_table_with_environment_context"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_table_with_environment_context"); + } + + ThriftHiveMetastore_create_table_with_environment_context_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_table_with_environment_context", bytes); + } + + ThriftHiveMetastore_create_table_with_environment_context_result result; + try { + iface_->create_table_with_environment_context(args.tbl, args.environment_context); + } catch (AlreadyExistsException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (NoSuchObjectException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_table_with_environment_context"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("create_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_table_with_environment_context"); + } + + oprot->writeMessageBegin("create_table_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_table_with_environment_context", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_create_table_with_constraints(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_table_with_constraints", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_table_with_constraints"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_table_with_constraints"); + } + + ThriftHiveMetastore_create_table_with_constraints_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_table_with_constraints", bytes); + } + + ThriftHiveMetastore_create_table_with_constraints_result result; + try { + iface_->create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys, args.uniqueConstraints, args.notNullConstraints); + } catch (AlreadyExistsException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (NoSuchObjectException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_table_with_constraints"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("create_table_with_constraints", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_table_with_constraints"); + } + + oprot->writeMessageBegin("create_table_with_constraints", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_table_with_constraints", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_drop_constraint(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_constraint", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_constraint"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_constraint"); + } + + ThriftHiveMetastore_drop_constraint_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_constraint", bytes); + } + + ThriftHiveMetastore_drop_constraint_result result; + try { + iface_->drop_constraint(args.req); + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_constraint"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("drop_constraint", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_constraint"); + } + + oprot->writeMessageBegin("drop_constraint", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_constraint", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_add_primary_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_primary_key", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_primary_key"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_primary_key"); + } + + ThriftHiveMetastore_add_primary_key_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_primary_key", bytes); + } + + ThriftHiveMetastore_add_primary_key_result result; + try { + iface_->add_primary_key(args.req); + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_primary_key"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("add_primary_key", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_primary_key"); + } + + oprot->writeMessageBegin("add_primary_key", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_primary_key", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_add_foreign_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_foreign_key", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_foreign_key"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_foreign_key"); + } + + ThriftHiveMetastore_add_foreign_key_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_foreign_key", bytes); + } + + ThriftHiveMetastore_add_foreign_key_result result; + try { + iface_->add_foreign_key(args.req); + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_foreign_key"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("add_foreign_key", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_foreign_key"); + } + + oprot->writeMessageBegin("add_foreign_key", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_foreign_key", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_add_unique_constraint(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_unique_constraint", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_unique_constraint"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_unique_constraint"); + } + + ThriftHiveMetastore_add_unique_constraint_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_unique_constraint", bytes); + } + + ThriftHiveMetastore_add_unique_constraint_result result; + try { + iface_->add_unique_constraint(args.req); + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_unique_constraint"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("add_unique_constraint", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_unique_constraint"); + } + + oprot->writeMessageBegin("add_unique_constraint", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_unique_constraint", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_add_not_null_constraint(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_not_null_constraint", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_not_null_constraint"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_not_null_constraint"); + } + + ThriftHiveMetastore_add_not_null_constraint_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_not_null_constraint", bytes); + } + + ThriftHiveMetastore_add_not_null_constraint_result result; + try { + iface_->add_not_null_constraint(args.req); + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_not_null_constraint"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("add_not_null_constraint", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_not_null_constraint"); + } + + oprot->writeMessageBegin("add_not_null_constraint", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_not_null_constraint", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_drop_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_table", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_table"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_table"); + } + + ThriftHiveMetastore_drop_table_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_table", bytes); + } + + ThriftHiveMetastore_drop_table_result result; + try { + iface_->drop_table(args.dbname, args.name, args.deleteData); + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_table"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("drop_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_table"); + } + + oprot->writeMessageBegin("drop_table", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_table", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_drop_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_table_with_environment_context", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); + } + + ThriftHiveMetastore_drop_table_with_environment_context_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_table_with_environment_context", bytes); + } + + ThriftHiveMetastore_drop_table_with_environment_context_result result; + try { + iface_->drop_table_with_environment_context(args.dbname, args.name, args.deleteData, args.environment_context); + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("drop_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); + } + + oprot->writeMessageBegin("drop_table_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_table_with_environment_context", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_truncate_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.truncate_table", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.truncate_table"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.truncate_table"); + } + + ThriftHiveMetastore_truncate_table_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.truncate_table", bytes); + } + + ThriftHiveMetastore_truncate_table_result result; + try { + iface_->truncate_table(args.dbName, args.tableName, args.partNames); + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.truncate_table"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("truncate_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.truncate_table"); + } + + oprot->writeMessageBegin("truncate_table", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.truncate_table", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_tables", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_tables"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_tables"); + } + + ThriftHiveMetastore_get_tables_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_tables", bytes); + } + + ThriftHiveMetastore_get_tables_result result; + try { + iface_->get_tables(result.success, args.db_name, args.pattern); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_tables"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_tables", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_tables"); + } + + oprot->writeMessageBegin("get_tables", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_tables", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_tables_by_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_tables_by_type", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_tables_by_type"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_tables_by_type"); + } + + ThriftHiveMetastore_get_tables_by_type_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_tables_by_type", bytes); + } + + ThriftHiveMetastore_get_tables_by_type_result result; + try { + iface_->get_tables_by_type(result.success, args.db_name, args.pattern, args.tableType); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_tables_by_type"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_tables_by_type", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_tables_by_type"); + } + + oprot->writeMessageBegin("get_tables_by_type", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_tables_by_type", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_table_meta(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_meta", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_meta"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_meta"); + } + + ThriftHiveMetastore_get_table_meta_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_meta", bytes); + } + + ThriftHiveMetastore_get_table_meta_result result; + try { + iface_->get_table_meta(result.success, args.db_patterns, args.tbl_patterns, args.tbl_types); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_meta"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_table_meta", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_meta"); + } + + oprot->writeMessageBegin("get_table_meta", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_meta", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_all_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_all_tables", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_all_tables"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_tables"); + } + + ThriftHiveMetastore_get_all_tables_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_all_tables", bytes); + } + + ThriftHiveMetastore_get_all_tables_result result; + try { + iface_->get_all_tables(result.success, args.db_name); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_all_tables"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_all_tables", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_tables"); + } + + oprot->writeMessageBegin("get_all_tables", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_all_tables", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table"); + } + + ThriftHiveMetastore_get_table_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table", bytes); + } + + ThriftHiveMetastore_get_table_result result; + try { + iface_->get_table(result.success, args.dbname, args.tbl_name); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table"); + } + + oprot->writeMessageBegin("get_table", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_table_objects_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_objects_by_name", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_objects_by_name"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_objects_by_name"); + } + + ThriftHiveMetastore_get_table_objects_by_name_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_objects_by_name", bytes); + } + + ThriftHiveMetastore_get_table_objects_by_name_result result; + try { + iface_->get_table_objects_by_name(result.success, args.dbname, args.tbl_names); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_objects_by_name"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_table_objects_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_objects_by_name"); + } + + oprot->writeMessageBegin("get_table_objects_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_objects_by_name", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_table_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_req", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_req"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_req"); + } + + ThriftHiveMetastore_get_table_req_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_req", bytes); + } + + ThriftHiveMetastore_get_table_req_result result; + try { + iface_->get_table_req(result.success, args.req); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_req"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_table_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_req"); + } + + oprot->writeMessageBegin("get_table_req", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_req", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_table_objects_by_name_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_objects_by_name_req", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_objects_by_name_req"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_objects_by_name_req"); + } + + ThriftHiveMetastore_get_table_objects_by_name_req_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_objects_by_name_req", bytes); + } + + ThriftHiveMetastore_get_table_objects_by_name_req_result result; + try { + iface_->get_table_objects_by_name_req(result.success, args.req); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidOperationException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (UnknownDBException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_objects_by_name_req"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_table_objects_by_name_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_objects_by_name_req"); + } + + oprot->writeMessageBegin("get_table_objects_by_name_req", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_objects_by_name_req", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_table_names_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_names_by_filter", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_names_by_filter"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_names_by_filter"); + } + + ThriftHiveMetastore_get_table_names_by_filter_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_names_by_filter", bytes); + } + + ThriftHiveMetastore_get_table_names_by_filter_result result; + try { + iface_->get_table_names_by_filter(result.success, args.dbname, args.filter, args.max_tables); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidOperationException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (UnknownDBException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_names_by_filter"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_table_names_by_filter", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_names_by_filter"); + } + + oprot->writeMessageBegin("get_table_names_by_filter", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_names_by_filter", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_alter_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_table", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_table"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_table"); + } + + ThriftHiveMetastore_alter_table_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_table", bytes); + } + + ThriftHiveMetastore_alter_table_result result; + try { + iface_->alter_table(args.dbname, args.tbl_name, args.new_tbl); + } catch (InvalidOperationException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_table"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("alter_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_table"); + } + + oprot->writeMessageBegin("alter_table", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_table", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_alter_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_table_with_environment_context", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); + } + + ThriftHiveMetastore_alter_table_with_environment_context_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_table_with_environment_context", bytes); + } + + ThriftHiveMetastore_alter_table_with_environment_context_result result; + try { + iface_->alter_table_with_environment_context(args.dbname, args.tbl_name, args.new_tbl, args.environment_context); + } catch (InvalidOperationException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("alter_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); + } + + oprot->writeMessageBegin("alter_table_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_table_with_environment_context", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_alter_table_with_cascade(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_table_with_cascade", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_table_with_cascade"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_table_with_cascade"); + } + + ThriftHiveMetastore_alter_table_with_cascade_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_table_with_cascade", bytes); + } + + ThriftHiveMetastore_alter_table_with_cascade_result result; + try { + iface_->alter_table_with_cascade(args.dbname, args.tbl_name, args.new_tbl, args.cascade); + } catch (InvalidOperationException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_table_with_cascade"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("alter_table_with_cascade", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_table_with_cascade"); + } + + oprot->writeMessageBegin("alter_table_with_cascade", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_table_with_cascade", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_add_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partition", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partition"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partition"); + } + + ThriftHiveMetastore_add_partition_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partition", bytes); + } + + ThriftHiveMetastore_add_partition_result result; + try { + iface_->add_partition(result.success, args.new_part); + result.__isset.success = true; + } catch (InvalidObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (AlreadyExistsException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partition"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("add_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partition"); + } + + oprot->writeMessageBegin("add_partition", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partition", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_add_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partition_with_environment_context", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); + } + + ThriftHiveMetastore_add_partition_with_environment_context_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partition_with_environment_context", bytes); + } + + ThriftHiveMetastore_add_partition_with_environment_context_result result; + try { + iface_->add_partition_with_environment_context(result.success, args.new_part, args.environment_context); + result.__isset.success = true; + } catch (InvalidObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (AlreadyExistsException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("add_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); + } + + oprot->writeMessageBegin("add_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partition_with_environment_context", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_add_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partitions", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partitions"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partitions"); + } + + ThriftHiveMetastore_add_partitions_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partitions", bytes); + } + + ThriftHiveMetastore_add_partitions_result result; + try { + result.success = iface_->add_partitions(args.new_parts); + result.__isset.success = true; + } catch (InvalidObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (AlreadyExistsException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partitions"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("add_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partitions"); + } + + oprot->writeMessageBegin("add_partitions", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partitions", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_add_partitions_pspec(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partitions_pspec", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partitions_pspec"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partitions_pspec"); + } + + ThriftHiveMetastore_add_partitions_pspec_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partitions_pspec", bytes); + } + + ThriftHiveMetastore_add_partitions_pspec_result result; + try { + result.success = iface_->add_partitions_pspec(args.new_parts); + result.__isset.success = true; + } catch (InvalidObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (AlreadyExistsException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partitions_pspec"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("add_partitions_pspec", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partitions_pspec"); + } + + oprot->writeMessageBegin("add_partitions_pspec", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partitions_pspec", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_append_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition"); + } + + ThriftHiveMetastore_append_partition_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition", bytes); + } + + ThriftHiveMetastore_append_partition_result result; + try { + iface_->append_partition(result.success, args.db_name, args.tbl_name, args.part_vals); + result.__isset.success = true; + } catch (InvalidObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (AlreadyExistsException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("append_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition"); + } + + oprot->writeMessageBegin("append_partition", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_add_partitions_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partitions_req", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partitions_req"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partitions_req"); + } + + ThriftHiveMetastore_add_partitions_req_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partitions_req", bytes); + } + + ThriftHiveMetastore_add_partitions_req_result result; + try { + iface_->add_partitions_req(result.success, args.request); + result.__isset.success = true; + } catch (InvalidObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (AlreadyExistsException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partitions_req"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("add_partitions_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partitions_req"); + } + + oprot->writeMessageBegin("add_partitions_req", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partitions_req", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_append_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition_with_environment_context", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); + } + + ThriftHiveMetastore_append_partition_with_environment_context_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition_with_environment_context", bytes); + } + + ThriftHiveMetastore_append_partition_with_environment_context_result result; + try { + iface_->append_partition_with_environment_context(result.success, args.db_name, args.tbl_name, args.part_vals, args.environment_context); + result.__isset.success = true; + } catch (InvalidObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (AlreadyExistsException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); + } + + oprot->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition_with_environment_context", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_append_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition_by_name", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition_by_name"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_by_name"); + } + + ThriftHiveMetastore_append_partition_by_name_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition_by_name", bytes); + } + + ThriftHiveMetastore_append_partition_by_name_result result; + try { + iface_->append_partition_by_name(result.success, args.db_name, args.tbl_name, args.part_name); + result.__isset.success = true; + } catch (InvalidObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (AlreadyExistsException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition_by_name"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("append_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_by_name"); + } + + oprot->writeMessageBegin("append_partition_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition_by_name", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_append_partition_by_name_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition_by_name_with_environment_context", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); + } + + ThriftHiveMetastore_append_partition_by_name_with_environment_context_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context", bytes); + } + + ThriftHiveMetastore_append_partition_by_name_with_environment_context_result result; + try { + iface_->append_partition_by_name_with_environment_context(result.success, args.db_name, args.tbl_name, args.part_name, args.environment_context); + result.__isset.success = true; + } catch (InvalidObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (AlreadyExistsException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); + } + + oprot->writeMessageBegin("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_drop_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition"); + } + + ThriftHiveMetastore_drop_partition_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition", bytes); + } + + ThriftHiveMetastore_drop_partition_result result; + try { + result.success = iface_->drop_partition(args.db_name, args.tbl_name, args.part_vals, args.deleteData); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("drop_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition"); + } + + oprot->writeMessageBegin("drop_partition", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_drop_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition_with_environment_context", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); + } + + ThriftHiveMetastore_drop_partition_with_environment_context_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context", bytes); + } + + ThriftHiveMetastore_drop_partition_with_environment_context_result result; + try { + result.success = iface_->drop_partition_with_environment_context(args.db_name, args.tbl_name, args.part_vals, args.deleteData, args.environment_context); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("drop_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); + } + + oprot->writeMessageBegin("drop_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_drop_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition_by_name", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition_by_name"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_by_name"); + } + + ThriftHiveMetastore_drop_partition_by_name_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition_by_name", bytes); + } + + ThriftHiveMetastore_drop_partition_by_name_result result; + try { + result.success = iface_->drop_partition_by_name(args.db_name, args.tbl_name, args.part_name, args.deleteData); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition_by_name"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("drop_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name"); + } + + oprot->writeMessageBegin("drop_partition_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_drop_partition_by_name_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition_by_name_with_environment_context", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); + } + + ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context", bytes); + } + + ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result result; + try { + result.success = iface_->drop_partition_by_name_with_environment_context(args.db_name, args.tbl_name, args.part_name, args.deleteData, args.environment_context); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); + } + + oprot->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_drop_partitions_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partitions_req", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partitions_req"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partitions_req"); + } + + ThriftHiveMetastore_drop_partitions_req_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partitions_req", bytes); + } + + ThriftHiveMetastore_drop_partitions_req_result result; + try { + iface_->drop_partitions_req(result.success, args.req); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partitions_req"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("drop_partitions_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partitions_req"); + } + + oprot->writeMessageBegin("drop_partitions_req", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partitions_req", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition"); + } + + ThriftHiveMetastore_get_partition_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition", bytes); + } + + ThriftHiveMetastore_get_partition_result result; + try { + iface_->get_partition(result.success, args.db_name, args.tbl_name, args.part_vals); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition"); + } + + oprot->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_exchange_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.exchange_partition", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.exchange_partition"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.exchange_partition"); + } + + ThriftHiveMetastore_exchange_partition_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.exchange_partition", bytes); + } + + ThriftHiveMetastore_exchange_partition_result result; + try { + iface_->exchange_partition(result.success, args.partitionSpecs, args.source_db, args.source_table_name, args.dest_db, args.dest_table_name); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (InvalidObjectException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (InvalidInputException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.exchange_partition"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.exchange_partition"); + } + + oprot->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.exchange_partition", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_exchange_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.exchange_partitions", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.exchange_partitions"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.exchange_partitions"); + } + + ThriftHiveMetastore_exchange_partitions_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.exchange_partitions", bytes); + } + + ThriftHiveMetastore_exchange_partitions_result result; + try { + iface_->exchange_partitions(result.success, args.partitionSpecs, args.source_db, args.source_table_name, args.dest_db, args.dest_table_name); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (InvalidObjectException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (InvalidInputException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.exchange_partitions"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("exchange_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.exchange_partitions"); + } + + oprot->writeMessageBegin("exchange_partitions", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.exchange_partitions", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_partition_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_with_auth", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_with_auth"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_with_auth"); + } + + ThriftHiveMetastore_get_partition_with_auth_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_with_auth", bytes); + } + + ThriftHiveMetastore_get_partition_with_auth_result result; + try { + iface_->get_partition_with_auth(result.success, args.db_name, args.tbl_name, args.part_vals, args.user_name, args.group_names); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_with_auth"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_partition_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_with_auth"); + } + + oprot->writeMessageBegin("get_partition_with_auth", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_with_auth", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_by_name", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_by_name"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_by_name"); + } + + ThriftHiveMetastore_get_partition_by_name_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_by_name", bytes); + } + + ThriftHiveMetastore_get_partition_by_name_result result; + try { + iface_->get_partition_by_name(result.success, args.db_name, args.tbl_name, args.part_name); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_by_name"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_by_name"); + } + + oprot->writeMessageBegin("get_partition_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_by_name", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions"); + } + + ThriftHiveMetastore_get_partitions_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions", bytes); + } + + ThriftHiveMetastore_get_partitions_result result; + try { + iface_->get_partitions(result.success, args.db_name, args.tbl_name, args.max_parts); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions"); + } + + oprot->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_partitions_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_with_auth", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_with_auth"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_with_auth"); + } + + ThriftHiveMetastore_get_partitions_with_auth_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_with_auth", bytes); + } + + ThriftHiveMetastore_get_partitions_with_auth_result result; + try { + iface_->get_partitions_with_auth(result.success, args.db_name, args.tbl_name, args.max_parts, args.user_name, args.group_names); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_with_auth"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_with_auth"); + } + + oprot->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_with_auth", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_partitions_pspec(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_pspec", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_pspec"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_pspec"); + } + + ThriftHiveMetastore_get_partitions_pspec_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_pspec", bytes); + } + + ThriftHiveMetastore_get_partitions_pspec_result result; + try { + iface_->get_partitions_pspec(result.success, args.db_name, args.tbl_name, args.max_parts); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_pspec"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_partitions_pspec", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_pspec"); + } + + oprot->writeMessageBegin("get_partitions_pspec", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_pspec", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_partition_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_names", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_names"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_names"); + } + + ThriftHiveMetastore_get_partition_names_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_names", bytes); + } + + ThriftHiveMetastore_get_partition_names_result result; + try { + iface_->get_partition_names(result.success, args.db_name, args.tbl_name, args.max_parts); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_names"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_names"); + } + + oprot->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_names", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_partition_values(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_values", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_values"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_values"); + } + + ThriftHiveMetastore_get_partition_values_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_values", bytes); + } + + ThriftHiveMetastore_get_partition_values_result result; + try { + iface_->get_partition_values(result.success, args.request); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_values"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_partition_values", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_values"); + } + + oprot->writeMessageBegin("get_partition_values", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_values", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_partitions_ps(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_ps", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_ps"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_ps"); + } + + ThriftHiveMetastore_get_partitions_ps_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_ps", bytes); + } + + ThriftHiveMetastore_get_partitions_ps_result result; + try { + iface_->get_partitions_ps(result.success, args.db_name, args.tbl_name, args.part_vals, args.max_parts); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_ps"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_ps"); + } + + oprot->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_ps", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_partitions_ps_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_ps_with_auth", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); + } + + ThriftHiveMetastore_get_partitions_ps_with_auth_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth", bytes); + } + + ThriftHiveMetastore_get_partitions_ps_with_auth_result result; + try { + iface_->get_partitions_ps_with_auth(result.success, args.db_name, args.tbl_name, args.part_vals, args.max_parts, args.user_name, args.group_names); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); + } + + oprot->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_partition_names_ps(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_names_ps", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_names_ps"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_names_ps"); + } + + ThriftHiveMetastore_get_partition_names_ps_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_names_ps", bytes); + } + + ThriftHiveMetastore_get_partition_names_ps_result result; + try { + iface_->get_partition_names_ps(result.success, args.db_name, args.tbl_name, args.part_vals, args.max_parts); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_names_ps"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_names_ps"); + } + + oprot->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_names_ps", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_partitions_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_by_filter", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_by_filter"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_by_filter"); + } + + ThriftHiveMetastore_get_partitions_by_filter_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_by_filter", bytes); + } + + ThriftHiveMetastore_get_partitions_by_filter_result result; + try { + iface_->get_partitions_by_filter(result.success, args.db_name, args.tbl_name, args.filter, args.max_parts); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_by_filter"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_by_filter"); + } + + oprot->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_by_filter", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_part_specs_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_part_specs_by_filter", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_part_specs_by_filter"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_part_specs_by_filter"); + } + + ThriftHiveMetastore_get_part_specs_by_filter_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_part_specs_by_filter", bytes); + } + + ThriftHiveMetastore_get_part_specs_by_filter_result result; + try { + iface_->get_part_specs_by_filter(result.success, args.db_name, args.tbl_name, args.filter, args.max_parts); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_part_specs_by_filter"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_part_specs_by_filter", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_part_specs_by_filter"); + } + + oprot->writeMessageBegin("get_part_specs_by_filter", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_part_specs_by_filter", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_partitions_by_expr(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_by_expr", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_by_expr"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_by_expr"); + } + + ThriftHiveMetastore_get_partitions_by_expr_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_by_expr", bytes); + } + + ThriftHiveMetastore_get_partitions_by_expr_result result; + try { + iface_->get_partitions_by_expr(result.success, args.req); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_by_expr"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_partitions_by_expr", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_by_expr"); + } + + oprot->writeMessageBegin("get_partitions_by_expr", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_by_expr", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_num_partitions_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_num_partitions_by_filter", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_num_partitions_by_filter"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_num_partitions_by_filter"); + } + + ThriftHiveMetastore_get_num_partitions_by_filter_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_num_partitions_by_filter", bytes); + } + + ThriftHiveMetastore_get_num_partitions_by_filter_result result; + try { + result.success = iface_->get_num_partitions_by_filter(args.db_name, args.tbl_name, args.filter); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_num_partitions_by_filter"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_num_partitions_by_filter", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_num_partitions_by_filter"); + } + + oprot->writeMessageBegin("get_num_partitions_by_filter", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_num_partitions_by_filter", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_partitions_by_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_by_names", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_by_names"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_by_names"); + } + + ThriftHiveMetastore_get_partitions_by_names_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_by_names", bytes); + } + + ThriftHiveMetastore_get_partitions_by_names_result result; + try { + iface_->get_partitions_by_names(result.success, args.db_name, args.tbl_name, args.names); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_by_names"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_by_names"); + } + + oprot->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_by_names", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_alter_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_partition", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partition"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partition"); + } + + ThriftHiveMetastore_alter_partition_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_partition", bytes); + } + + ThriftHiveMetastore_alter_partition_result result; + try { + iface_->alter_partition(args.db_name, args.tbl_name, args.new_part); + } catch (InvalidOperationException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partition"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partition"); + } + + oprot->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_partition", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_alter_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_partitions", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partitions"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partitions"); + } + + ThriftHiveMetastore_alter_partitions_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_partitions", bytes); + } + + ThriftHiveMetastore_alter_partitions_result result; + try { + iface_->alter_partitions(args.db_name, args.tbl_name, args.new_parts); + } catch (InvalidOperationException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partitions"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partitions"); + } + + oprot->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_partitions", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_alter_partitions_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_partitions_with_environment_context", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partitions_with_environment_context"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partitions_with_environment_context"); + } + + ThriftHiveMetastore_alter_partitions_with_environment_context_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_partitions_with_environment_context", bytes); + } + + ThriftHiveMetastore_alter_partitions_with_environment_context_result result; + try { + iface_->alter_partitions_with_environment_context(args.db_name, args.tbl_name, args.new_parts, args.environment_context); + } catch (InvalidOperationException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partitions_with_environment_context"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("alter_partitions_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partitions_with_environment_context"); + } + + oprot->writeMessageBegin("alter_partitions_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_partitions_with_environment_context", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_alter_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_partition_with_environment_context", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); + } + + ThriftHiveMetastore_alter_partition_with_environment_context_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context", bytes); + } + + ThriftHiveMetastore_alter_partition_with_environment_context_result result; + try { + iface_->alter_partition_with_environment_context(args.db_name, args.tbl_name, args.new_part, args.environment_context); + } catch (InvalidOperationException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); + } + + oprot->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_rename_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.rename_partition", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.rename_partition"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.rename_partition"); + } + + ThriftHiveMetastore_rename_partition_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.rename_partition", bytes); + } + + ThriftHiveMetastore_rename_partition_result result; + try { + iface_->rename_partition(args.db_name, args.tbl_name, args.part_vals, args.new_part); + } catch (InvalidOperationException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.rename_partition"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.rename_partition"); + } + + oprot->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.rename_partition", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_partition_name_has_valid_characters(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.partition_name_has_valid_characters", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); + } + + ThriftHiveMetastore_partition_name_has_valid_characters_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters", bytes); + } + + ThriftHiveMetastore_partition_name_has_valid_characters_result result; + try { + result.success = iface_->partition_name_has_valid_characters(args.part_vals, args.throw_exception); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("partition_name_has_valid_characters", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); + } + + oprot->writeMessageBegin("partition_name_has_valid_characters", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_config_value(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_config_value", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_config_value"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_config_value"); + } + + ThriftHiveMetastore_get_config_value_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_config_value", bytes); + } + + ThriftHiveMetastore_get_config_value_result result; + try { + iface_->get_config_value(result.success, args.name, args.defaultValue); + result.__isset.success = true; + } catch (ConfigValSecurityException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_config_value"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_config_value"); + } + + oprot->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_config_value", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_partition_name_to_vals(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.partition_name_to_vals", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_to_vals"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_to_vals"); + } + + ThriftHiveMetastore_partition_name_to_vals_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.partition_name_to_vals", bytes); + } + + ThriftHiveMetastore_partition_name_to_vals_result result; + try { + iface_->partition_name_to_vals(result.success, args.part_name); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.partition_name_to_vals"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_to_vals"); + } + + oprot->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.partition_name_to_vals", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_partition_name_to_spec(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.partition_name_to_spec", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_to_spec"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_to_spec"); + } + + ThriftHiveMetastore_partition_name_to_spec_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.partition_name_to_spec", bytes); + } + + ThriftHiveMetastore_partition_name_to_spec_result result; + try { + iface_->partition_name_to_spec(result.success, args.part_name); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.partition_name_to_spec"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_to_spec"); + } + + oprot->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.partition_name_to_spec", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_markPartitionForEvent(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.markPartitionForEvent", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.markPartitionForEvent"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.markPartitionForEvent"); + } + + ThriftHiveMetastore_markPartitionForEvent_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.markPartitionForEvent", bytes); + } + + ThriftHiveMetastore_markPartitionForEvent_result result; + try { + iface_->markPartitionForEvent(args.db_name, args.tbl_name, args.part_vals, args.eventType); + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (UnknownDBException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (UnknownTableException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (UnknownPartitionException &o5) { + result.o5 = o5; + result.__isset.o5 = true; + } catch (InvalidPartitionException &o6) { + result.o6 = o6; + result.__isset.o6 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.markPartitionForEvent"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.markPartitionForEvent"); + } + + oprot->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.markPartitionForEvent", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_isPartitionMarkedForEvent(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.isPartitionMarkedForEvent", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); + } + + ThriftHiveMetastore_isPartitionMarkedForEvent_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent", bytes); + } + + ThriftHiveMetastore_isPartitionMarkedForEvent_result result; + try { + result.success = iface_->isPartitionMarkedForEvent(args.db_name, args.tbl_name, args.part_vals, args.eventType); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (UnknownDBException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (UnknownTableException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (UnknownPartitionException &o5) { + result.o5 = o5; + result.__isset.o5 = true; + } catch (InvalidPartitionException &o6) { + result.o6 = o6; + result.__isset.o6 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); + } + + oprot->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_add_index(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_index", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_index"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_index"); + } + + ThriftHiveMetastore_add_index_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_index", bytes); + } + + ThriftHiveMetastore_add_index_result result; + try { + iface_->add_index(result.success, args.new_index, args.index_table); + result.__isset.success = true; + } catch (InvalidObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (AlreadyExistsException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_index"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("add_index", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_index"); + } + + oprot->writeMessageBegin("add_index", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_index", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_alter_index(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_index", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_index"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_index"); + } + + ThriftHiveMetastore_alter_index_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_index", bytes); + } + + ThriftHiveMetastore_alter_index_result result; + try { + iface_->alter_index(args.dbname, args.base_tbl_name, args.idx_name, args.new_idx); + } catch (InvalidOperationException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_index"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_index"); + } + + oprot->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_index", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_drop_index_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_index_by_name", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_index_by_name"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_index_by_name"); + } + + ThriftHiveMetastore_drop_index_by_name_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_index_by_name", bytes); + } + + ThriftHiveMetastore_drop_index_by_name_result result; + try { + result.success = iface_->drop_index_by_name(args.db_name, args.tbl_name, args.index_name, args.deleteData); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_index_by_name"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_index_by_name"); + } + + oprot->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_index_by_name", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_index_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_index_by_name", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_index_by_name"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_index_by_name"); + } + + ThriftHiveMetastore_get_index_by_name_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_index_by_name", bytes); + } + + ThriftHiveMetastore_get_index_by_name_result result; + try { + iface_->get_index_by_name(result.success, args.db_name, args.tbl_name, args.index_name); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_index_by_name"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_index_by_name"); + } + + oprot->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_index_by_name", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_indexes(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_indexes", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_indexes"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_indexes"); + } + + ThriftHiveMetastore_get_indexes_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_indexes", bytes); + } + + ThriftHiveMetastore_get_indexes_result result; + try { + iface_->get_indexes(result.success, args.db_name, args.tbl_name, args.max_indexes); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_indexes"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_indexes"); + } + + oprot->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_indexes", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_index_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_index_names", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_index_names"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_index_names"); + } + + ThriftHiveMetastore_get_index_names_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_index_names", bytes); + } + + ThriftHiveMetastore_get_index_names_result result; + try { + iface_->get_index_names(result.success, args.db_name, args.tbl_name, args.max_indexes); + result.__isset.success = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_index_names"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_index_names"); + } + + oprot->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_index_names", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_primary_keys(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_primary_keys", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_primary_keys"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_primary_keys"); + } + + ThriftHiveMetastore_get_primary_keys_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_primary_keys", bytes); + } + + ThriftHiveMetastore_get_primary_keys_result result; + try { + iface_->get_primary_keys(result.success, args.request); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_primary_keys"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_primary_keys", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_primary_keys"); + } + + oprot->writeMessageBegin("get_primary_keys", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_primary_keys", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_foreign_keys(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_foreign_keys", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_foreign_keys"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_foreign_keys"); + } + + ThriftHiveMetastore_get_foreign_keys_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_foreign_keys", bytes); + } + + ThriftHiveMetastore_get_foreign_keys_result result; + try { + iface_->get_foreign_keys(result.success, args.request); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_foreign_keys"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_foreign_keys", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_foreign_keys"); + } + + oprot->writeMessageBegin("get_foreign_keys", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_foreign_keys", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_unique_constraints(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_unique_constraints", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_unique_constraints"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_unique_constraints"); + } + + ThriftHiveMetastore_get_unique_constraints_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_unique_constraints", bytes); + } + + ThriftHiveMetastore_get_unique_constraints_result result; + try { + iface_->get_unique_constraints(result.success, args.request); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_unique_constraints"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_unique_constraints", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_unique_constraints"); + } + + oprot->writeMessageBegin("get_unique_constraints", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_unique_constraints", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_not_null_constraints(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_not_null_constraints", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_not_null_constraints"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_not_null_constraints"); + } + + ThriftHiveMetastore_get_not_null_constraints_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_not_null_constraints", bytes); + } + + ThriftHiveMetastore_get_not_null_constraints_result result; + try { + iface_->get_not_null_constraints(result.success, args.request); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_not_null_constraints"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_not_null_constraints", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_not_null_constraints"); + } + + oprot->writeMessageBegin("get_not_null_constraints", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_not_null_constraints", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_update_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.update_table_column_statistics", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.update_table_column_statistics"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.update_table_column_statistics"); + } + + ThriftHiveMetastore_update_table_column_statistics_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.update_table_column_statistics", bytes); + } + + ThriftHiveMetastore_update_table_column_statistics_result result; + try { + result.success = iface_->update_table_column_statistics(args.stats_obj); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (InvalidInputException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.update_table_column_statistics"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("update_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.update_table_column_statistics"); + } + + oprot->writeMessageBegin("update_table_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.update_table_column_statistics", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_update_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.update_partition_column_statistics", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.update_partition_column_statistics"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.update_partition_column_statistics"); + } + + ThriftHiveMetastore_update_partition_column_statistics_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.update_partition_column_statistics", bytes); + } + + ThriftHiveMetastore_update_partition_column_statistics_result result; + try { + result.success = iface_->update_partition_column_statistics(args.stats_obj); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (InvalidInputException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.update_partition_column_statistics"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.update_partition_column_statistics"); + } + + oprot->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.update_partition_column_statistics", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_column_statistics", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_column_statistics"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_column_statistics"); + } + + ThriftHiveMetastore_get_table_column_statistics_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_column_statistics", bytes); + } + + ThriftHiveMetastore_get_table_column_statistics_result result; + try { + iface_->get_table_column_statistics(result.success, args.db_name, args.tbl_name, args.col_name); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (InvalidInputException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (InvalidObjectException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_column_statistics"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_column_statistics"); + } + + oprot->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_column_statistics", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_column_statistics", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_column_statistics"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_column_statistics"); + } + + ThriftHiveMetastore_get_partition_column_statistics_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_column_statistics", bytes); + } + + ThriftHiveMetastore_get_partition_column_statistics_result result; + try { + iface_->get_partition_column_statistics(result.success, args.db_name, args.tbl_name, args.part_name, args.col_name); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (InvalidInputException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (InvalidObjectException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_column_statistics"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_column_statistics"); + } + + oprot->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_column_statistics", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_table_statistics_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_statistics_req", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_statistics_req"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_statistics_req"); + } + + ThriftHiveMetastore_get_table_statistics_req_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_statistics_req", bytes); + } + + ThriftHiveMetastore_get_table_statistics_req_result result; + try { + iface_->get_table_statistics_req(result.success, args.request); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_statistics_req"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_table_statistics_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_statistics_req"); + } + + oprot->writeMessageBegin("get_table_statistics_req", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_statistics_req", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_partitions_statistics_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_statistics_req", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_statistics_req"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_statistics_req"); + } + + ThriftHiveMetastore_get_partitions_statistics_req_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_statistics_req", bytes); + } + + ThriftHiveMetastore_get_partitions_statistics_req_result result; + try { + iface_->get_partitions_statistics_req(result.success, args.request); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_statistics_req"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_partitions_statistics_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_statistics_req"); + } + + oprot->writeMessageBegin("get_partitions_statistics_req", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_statistics_req", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_aggr_stats_for(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_aggr_stats_for", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_aggr_stats_for"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_aggr_stats_for"); + } + + ThriftHiveMetastore_get_aggr_stats_for_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_aggr_stats_for", bytes); + } + + ThriftHiveMetastore_get_aggr_stats_for_result result; + try { + iface_->get_aggr_stats_for(result.success, args.request); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_aggr_stats_for"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_aggr_stats_for", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_aggr_stats_for"); + } + + oprot->writeMessageBegin("get_aggr_stats_for", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_aggr_stats_for", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_set_aggr_stats_for(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.set_aggr_stats_for", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.set_aggr_stats_for"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.set_aggr_stats_for"); + } + + ThriftHiveMetastore_set_aggr_stats_for_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.set_aggr_stats_for", bytes); + } + + ThriftHiveMetastore_set_aggr_stats_for_result result; + try { + result.success = iface_->set_aggr_stats_for(args.request); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (InvalidInputException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.set_aggr_stats_for"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("set_aggr_stats_for", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.set_aggr_stats_for"); + } + + oprot->writeMessageBegin("set_aggr_stats_for", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.set_aggr_stats_for", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_delete_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.delete_partition_column_statistics", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); + } + + ThriftHiveMetastore_delete_partition_column_statistics_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.delete_partition_column_statistics", bytes); + } + + ThriftHiveMetastore_delete_partition_column_statistics_result result; + try { + result.success = iface_->delete_partition_column_statistics(args.db_name, args.tbl_name, args.part_name, args.col_name); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (InvalidObjectException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (InvalidInputException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); + } + + oprot->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.delete_partition_column_statistics", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_delete_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.delete_table_column_statistics", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.delete_table_column_statistics"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.delete_table_column_statistics"); + } + + ThriftHiveMetastore_delete_table_column_statistics_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.delete_table_column_statistics", bytes); + } + + ThriftHiveMetastore_delete_table_column_statistics_result result; + try { + result.success = iface_->delete_table_column_statistics(args.db_name, args.tbl_name, args.col_name); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (InvalidObjectException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (InvalidInputException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.delete_table_column_statistics"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.delete_table_column_statistics"); + } + + oprot->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.delete_table_column_statistics", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_create_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_function", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_function"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_function"); + } + + ThriftHiveMetastore_create_function_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_function", bytes); + } + + ThriftHiveMetastore_create_function_result result; + try { + iface_->create_function(args.func); + } catch (AlreadyExistsException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (NoSuchObjectException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_function"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("create_function", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_function"); + } + + oprot->writeMessageBegin("create_function", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_function", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_drop_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_function", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_function"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_function"); + } + + ThriftHiveMetastore_drop_function_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_function", bytes); + } + + ThriftHiveMetastore_drop_function_result result; + try { + iface_->drop_function(args.dbName, args.funcName); + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_function"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("drop_function", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_function"); + } + + oprot->writeMessageBegin("drop_function", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_function", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_alter_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_function", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_function"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_function"); + } + + ThriftHiveMetastore_alter_function_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_function", bytes); + } + + ThriftHiveMetastore_alter_function_result result; + try { + iface_->alter_function(args.dbName, args.funcName, args.newFunc); + } catch (InvalidOperationException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_function"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("alter_function", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_function"); + } + + oprot->writeMessageBegin("alter_function", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_function", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_functions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_functions", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_functions"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_functions"); + } + + ThriftHiveMetastore_get_functions_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_functions", bytes); + } + + ThriftHiveMetastore_get_functions_result result; + try { + iface_->get_functions(result.success, args.dbName, args.pattern); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_functions"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_functions", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_functions"); + } + + oprot->writeMessageBegin("get_functions", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_functions", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_function", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_function"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_function"); + } + + ThriftHiveMetastore_get_function_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_function", bytes); + } + + ThriftHiveMetastore_get_function_result result; + try { + iface_->get_function(result.success, args.dbName, args.funcName); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_function"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_function", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_function"); + } + + oprot->writeMessageBegin("get_function", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_function", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_all_functions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_all_functions", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_all_functions"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_functions"); + } + + ThriftHiveMetastore_get_all_functions_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_all_functions", bytes); + } + + ThriftHiveMetastore_get_all_functions_result result; + try { + iface_->get_all_functions(result.success); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_all_functions"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_all_functions", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_functions"); + } + + oprot->writeMessageBegin("get_all_functions", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_all_functions", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_create_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_role", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_role"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_role"); + } + + ThriftHiveMetastore_create_role_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_role", bytes); + } + + ThriftHiveMetastore_create_role_result result; + try { + result.success = iface_->create_role(args.role); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_role"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("create_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_role"); + } + + oprot->writeMessageBegin("create_role", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_role", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_drop_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_role", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_role"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_role"); + } + + ThriftHiveMetastore_drop_role_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_role", bytes); + } + + ThriftHiveMetastore_drop_role_result result; + try { + result.success = iface_->drop_role(args.role_name); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_role"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("drop_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_role"); + } + + oprot->writeMessageBegin("drop_role", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_role", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_role_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_role_names", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_role_names"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_role_names"); + } + + ThriftHiveMetastore_get_role_names_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_role_names", bytes); + } + + ThriftHiveMetastore_get_role_names_result result; + try { + iface_->get_role_names(result.success); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_role_names"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_role_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_role_names"); + } + + oprot->writeMessageBegin("get_role_names", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_role_names", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_grant_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.grant_role", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.grant_role"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.grant_role"); + } + + ThriftHiveMetastore_grant_role_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.grant_role", bytes); + } + + ThriftHiveMetastore_grant_role_result result; + try { + result.success = iface_->grant_role(args.role_name, args.principal_name, args.principal_type, args.grantor, args.grantorType, args.grant_option); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.grant_role"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("grant_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.grant_role"); + } + + oprot->writeMessageBegin("grant_role", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.grant_role", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_revoke_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.revoke_role", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.revoke_role"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.revoke_role"); + } + + ThriftHiveMetastore_revoke_role_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.revoke_role", bytes); + } + + ThriftHiveMetastore_revoke_role_result result; + try { + result.success = iface_->revoke_role(args.role_name, args.principal_name, args.principal_type); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.revoke_role"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("revoke_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.revoke_role"); + } + + oprot->writeMessageBegin("revoke_role", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.revoke_role", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_list_roles(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.list_roles", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.list_roles"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.list_roles"); + } + + ThriftHiveMetastore_list_roles_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.list_roles", bytes); + } + + ThriftHiveMetastore_list_roles_result result; + try { + iface_->list_roles(result.success, args.principal_name, args.principal_type); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.list_roles"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("list_roles", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.list_roles"); + } + + oprot->writeMessageBegin("list_roles", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.list_roles", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_grant_revoke_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.grant_revoke_role", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.grant_revoke_role"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.grant_revoke_role"); + } + + ThriftHiveMetastore_grant_revoke_role_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.grant_revoke_role", bytes); + } + + ThriftHiveMetastore_grant_revoke_role_result result; + try { + iface_->grant_revoke_role(result.success, args.request); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.grant_revoke_role"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("grant_revoke_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.grant_revoke_role"); + } + + oprot->writeMessageBegin("grant_revoke_role", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.grant_revoke_role", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_principals_in_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_principals_in_role", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_principals_in_role"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_principals_in_role"); + } + + ThriftHiveMetastore_get_principals_in_role_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_principals_in_role", bytes); + } + + ThriftHiveMetastore_get_principals_in_role_result result; + try { + iface_->get_principals_in_role(result.success, args.request); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_principals_in_role"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_principals_in_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_principals_in_role"); + } + + oprot->writeMessageBegin("get_principals_in_role", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_principals_in_role", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_role_grants_for_principal(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_role_grants_for_principal", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_role_grants_for_principal"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_role_grants_for_principal"); + } + + ThriftHiveMetastore_get_role_grants_for_principal_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_role_grants_for_principal", bytes); + } + + ThriftHiveMetastore_get_role_grants_for_principal_result result; + try { + iface_->get_role_grants_for_principal(result.success, args.request); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_role_grants_for_principal"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_role_grants_for_principal", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_role_grants_for_principal"); + } + + oprot->writeMessageBegin("get_role_grants_for_principal", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_role_grants_for_principal", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_privilege_set(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_privilege_set", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_privilege_set"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_privilege_set"); + } + + ThriftHiveMetastore_get_privilege_set_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_privilege_set", bytes); + } + + ThriftHiveMetastore_get_privilege_set_result result; + try { + iface_->get_privilege_set(result.success, args.hiveObject, args.user_name, args.group_names); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_privilege_set"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_privilege_set", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_privilege_set"); + } + + oprot->writeMessageBegin("get_privilege_set", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_privilege_set", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_list_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.list_privileges", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.list_privileges"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.list_privileges"); + } + + ThriftHiveMetastore_list_privileges_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.list_privileges", bytes); + } + + ThriftHiveMetastore_list_privileges_result result; + try { + iface_->list_privileges(result.success, args.principal_name, args.principal_type, args.hiveObject); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.list_privileges"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("list_privileges", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.list_privileges"); + } + + oprot->writeMessageBegin("list_privileges", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.list_privileges", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_grant_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.grant_privileges", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.grant_privileges"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.grant_privileges"); + } + + ThriftHiveMetastore_grant_privileges_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.grant_privileges", bytes); + } + + ThriftHiveMetastore_grant_privileges_result result; + try { + result.success = iface_->grant_privileges(args.privileges); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.grant_privileges"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("grant_privileges", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.grant_privileges"); + } + + oprot->writeMessageBegin("grant_privileges", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.grant_privileges", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_revoke_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.revoke_privileges", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.revoke_privileges"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.revoke_privileges"); + } + + ThriftHiveMetastore_revoke_privileges_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.revoke_privileges", bytes); + } + + ThriftHiveMetastore_revoke_privileges_result result; + try { + result.success = iface_->revoke_privileges(args.privileges); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.revoke_privileges"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("revoke_privileges", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.revoke_privileges"); + } + + oprot->writeMessageBegin("revoke_privileges", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.revoke_privileges", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_grant_revoke_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.grant_revoke_privileges", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.grant_revoke_privileges"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.grant_revoke_privileges"); + } + + ThriftHiveMetastore_grant_revoke_privileges_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.grant_revoke_privileges", bytes); + } + + ThriftHiveMetastore_grant_revoke_privileges_result result; + try { + iface_->grant_revoke_privileges(result.success, args.request); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.grant_revoke_privileges"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("grant_revoke_privileges", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.grant_revoke_privileges"); + } + + oprot->writeMessageBegin("grant_revoke_privileges", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.grant_revoke_privileges", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_set_ugi(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.set_ugi", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.set_ugi"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.set_ugi"); + } + + ThriftHiveMetastore_set_ugi_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.set_ugi", bytes); + } + + ThriftHiveMetastore_set_ugi_result result; + try { + iface_->set_ugi(result.success, args.user_name, args.group_names); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.set_ugi"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("set_ugi", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.set_ugi"); + } + + oprot->writeMessageBegin("set_ugi", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.set_ugi", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_delegation_token", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_delegation_token"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_delegation_token"); + } + + ThriftHiveMetastore_get_delegation_token_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_delegation_token", bytes); + } + + ThriftHiveMetastore_get_delegation_token_result result; + try { + iface_->get_delegation_token(result.success, args.token_owner, args.renewer_kerberos_principal_name); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_delegation_token"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_delegation_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_delegation_token"); + } + + oprot->writeMessageBegin("get_delegation_token", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_delegation_token", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_renew_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.renew_delegation_token", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.renew_delegation_token"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.renew_delegation_token"); + } + + ThriftHiveMetastore_renew_delegation_token_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.renew_delegation_token", bytes); + } + + ThriftHiveMetastore_renew_delegation_token_result result; + try { + result.success = iface_->renew_delegation_token(args.token_str_form); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.renew_delegation_token"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("renew_delegation_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.renew_delegation_token"); + } + + oprot->writeMessageBegin("renew_delegation_token", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.renew_delegation_token", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_cancel_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.cancel_delegation_token", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.cancel_delegation_token"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.cancel_delegation_token"); + } + + ThriftHiveMetastore_cancel_delegation_token_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.cancel_delegation_token", bytes); + } + + ThriftHiveMetastore_cancel_delegation_token_result result; + try { + iface_->cancel_delegation_token(args.token_str_form); + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.cancel_delegation_token"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("cancel_delegation_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.cancel_delegation_token"); + } + + oprot->writeMessageBegin("cancel_delegation_token", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.cancel_delegation_token", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_add_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_token", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_token"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_token"); + } + + ThriftHiveMetastore_add_token_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_token", bytes); + } + + ThriftHiveMetastore_add_token_result result; + try { + result.success = iface_->add_token(args.token_identifier, args.delegation_token); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_token"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("add_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_token"); + } + + oprot->writeMessageBegin("add_token", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_token", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_remove_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.remove_token", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.remove_token"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.remove_token"); + } + + ThriftHiveMetastore_remove_token_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.remove_token", bytes); + } + + ThriftHiveMetastore_remove_token_result result; + try { + result.success = iface_->remove_token(args.token_identifier); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.remove_token"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("remove_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.remove_token"); + } + + oprot->writeMessageBegin("remove_token", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.remove_token", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_token", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_token"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_token"); + } + + ThriftHiveMetastore_get_token_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_token", bytes); + } + + ThriftHiveMetastore_get_token_result result; + try { + iface_->get_token(result.success, args.token_identifier); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_token"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_token"); + } + + oprot->writeMessageBegin("get_token", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_token", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_all_token_identifiers(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_all_token_identifiers", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_all_token_identifiers"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_token_identifiers"); + } + + ThriftHiveMetastore_get_all_token_identifiers_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_all_token_identifiers", bytes); + } + + ThriftHiveMetastore_get_all_token_identifiers_result result; + try { + iface_->get_all_token_identifiers(result.success); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_all_token_identifiers"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_all_token_identifiers", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_token_identifiers"); + } + + oprot->writeMessageBegin("get_all_token_identifiers", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_all_token_identifiers", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_add_master_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_master_key", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_master_key"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_master_key"); + } + + ThriftHiveMetastore_add_master_key_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_master_key", bytes); + } + + ThriftHiveMetastore_add_master_key_result result; + try { + result.success = iface_->add_master_key(args.key); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_master_key"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("add_master_key", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_master_key"); + } + + oprot->writeMessageBegin("add_master_key", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_master_key", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_update_master_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.update_master_key", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.update_master_key"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.update_master_key"); + } + + ThriftHiveMetastore_update_master_key_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.update_master_key", bytes); + } + + ThriftHiveMetastore_update_master_key_result result; + try { + iface_->update_master_key(args.seq_number, args.key); + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.update_master_key"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("update_master_key", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.update_master_key"); + } + + oprot->writeMessageBegin("update_master_key", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.update_master_key", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_remove_master_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.remove_master_key", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.remove_master_key"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.remove_master_key"); + } + + ThriftHiveMetastore_remove_master_key_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.remove_master_key", bytes); + } + + ThriftHiveMetastore_remove_master_key_result result; + try { + result.success = iface_->remove_master_key(args.key_seq); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.remove_master_key"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("remove_master_key", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.remove_master_key"); + } + + oprot->writeMessageBegin("remove_master_key", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.remove_master_key", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_master_keys(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_master_keys", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_master_keys"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_master_keys"); + } + + ThriftHiveMetastore_get_master_keys_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_master_keys", bytes); + } + + ThriftHiveMetastore_get_master_keys_result result; + try { + iface_->get_master_keys(result.success); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_master_keys"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_master_keys", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_master_keys"); + } + + oprot->writeMessageBegin("get_master_keys", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_master_keys", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_open_txns(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_open_txns", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_open_txns"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_open_txns"); + } + + ThriftHiveMetastore_get_open_txns_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_open_txns", bytes); + } + + ThriftHiveMetastore_get_open_txns_result result; + try { + iface_->get_open_txns(result.success); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_open_txns"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_open_txns", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_open_txns"); + } + + oprot->writeMessageBegin("get_open_txns", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_open_txns", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_open_txns_info(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_open_txns_info", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_open_txns_info"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_open_txns_info"); + } + + ThriftHiveMetastore_get_open_txns_info_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_open_txns_info", bytes); + } + + ThriftHiveMetastore_get_open_txns_info_result result; + try { + iface_->get_open_txns_info(result.success); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_open_txns_info"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_open_txns_info", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_open_txns_info"); + } + + oprot->writeMessageBegin("get_open_txns_info", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_open_txns_info", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_open_txns(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.open_txns", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.open_txns"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.open_txns"); + } + + ThriftHiveMetastore_open_txns_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.open_txns", bytes); + } + + ThriftHiveMetastore_open_txns_result result; + try { + iface_->open_txns(result.success, args.rqst); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.open_txns"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("open_txns", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.open_txns"); + } + + oprot->writeMessageBegin("open_txns", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.open_txns", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_abort_txn(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.abort_txn", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.abort_txn"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.abort_txn"); + } + + ThriftHiveMetastore_abort_txn_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.abort_txn", bytes); + } + + ThriftHiveMetastore_abort_txn_result result; + try { + iface_->abort_txn(args.rqst); + } catch (NoSuchTxnException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.abort_txn"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("abort_txn", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.abort_txn"); + } + + oprot->writeMessageBegin("abort_txn", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.abort_txn", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_abort_txns(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.abort_txns", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.abort_txns"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.abort_txns"); + } + + ThriftHiveMetastore_abort_txns_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.abort_txns", bytes); + } + + ThriftHiveMetastore_abort_txns_result result; + try { + iface_->abort_txns(args.rqst); + } catch (NoSuchTxnException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.abort_txns"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("abort_txns", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.abort_txns"); + } + + oprot->writeMessageBegin("abort_txns", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.abort_txns", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_commit_txn(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.commit_txn", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.commit_txn"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.commit_txn"); + } + + ThriftHiveMetastore_commit_txn_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.commit_txn", bytes); + } + + ThriftHiveMetastore_commit_txn_result result; + try { + iface_->commit_txn(args.rqst); + } catch (NoSuchTxnException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (TxnAbortedException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.commit_txn"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("commit_txn", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.commit_txn"); + } + + oprot->writeMessageBegin("commit_txn", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.commit_txn", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_lock(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.lock", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.lock"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.lock"); + } + + ThriftHiveMetastore_lock_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.lock", bytes); + } + + ThriftHiveMetastore_lock_result result; + try { + iface_->lock(result.success, args.rqst); + result.__isset.success = true; + } catch (NoSuchTxnException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (TxnAbortedException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.lock"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("lock", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.lock"); + } + + oprot->writeMessageBegin("lock", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.lock", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_check_lock(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.check_lock", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.check_lock"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.check_lock"); + } + + ThriftHiveMetastore_check_lock_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.check_lock", bytes); + } + + ThriftHiveMetastore_check_lock_result result; + try { + iface_->check_lock(result.success, args.rqst); + result.__isset.success = true; + } catch (NoSuchTxnException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (TxnAbortedException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (NoSuchLockException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.check_lock"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("check_lock", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.check_lock"); + } + + oprot->writeMessageBegin("check_lock", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.check_lock", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_unlock(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.unlock", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.unlock"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.unlock"); + } + + ThriftHiveMetastore_unlock_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.unlock", bytes); + } + + ThriftHiveMetastore_unlock_result result; + try { + iface_->unlock(args.rqst); + } catch (NoSuchLockException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (TxnOpenException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.unlock"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("unlock", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.unlock"); + } + + oprot->writeMessageBegin("unlock", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.unlock", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_show_locks(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.show_locks", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.show_locks"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.show_locks"); + } + + ThriftHiveMetastore_show_locks_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.show_locks", bytes); + } + + ThriftHiveMetastore_show_locks_result result; + try { + iface_->show_locks(result.success, args.rqst); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.show_locks"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("show_locks", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.show_locks"); + } + + oprot->writeMessageBegin("show_locks", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.show_locks", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_heartbeat(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.heartbeat", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.heartbeat"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.heartbeat"); + } + + ThriftHiveMetastore_heartbeat_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.heartbeat", bytes); + } + + ThriftHiveMetastore_heartbeat_result result; + try { + iface_->heartbeat(args.ids); + } catch (NoSuchLockException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchTxnException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (TxnAbortedException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.heartbeat"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("heartbeat", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.heartbeat"); + } + + oprot->writeMessageBegin("heartbeat", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.heartbeat", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_heartbeat_txn_range(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.heartbeat_txn_range", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.heartbeat_txn_range"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.heartbeat_txn_range"); + } + + ThriftHiveMetastore_heartbeat_txn_range_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.heartbeat_txn_range", bytes); + } + + ThriftHiveMetastore_heartbeat_txn_range_result result; + try { + iface_->heartbeat_txn_range(result.success, args.txns); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.heartbeat_txn_range"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("heartbeat_txn_range", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.heartbeat_txn_range"); + } + + oprot->writeMessageBegin("heartbeat_txn_range", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.heartbeat_txn_range", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_compact(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.compact", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.compact"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.compact"); + } + + ThriftHiveMetastore_compact_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.compact", bytes); + } + + ThriftHiveMetastore_compact_result result; + try { + iface_->compact(args.rqst); + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.compact"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("compact", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.compact"); + } + + oprot->writeMessageBegin("compact", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.compact", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_compact2(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.compact2", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.compact2"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.compact2"); + } + + ThriftHiveMetastore_compact2_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.compact2", bytes); + } + + ThriftHiveMetastore_compact2_result result; + try { + iface_->compact2(result.success, args.rqst); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.compact2"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("compact2", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.compact2"); + } + + oprot->writeMessageBegin("compact2", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.compact2", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_show_compact(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.show_compact", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.show_compact"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.show_compact"); + } + + ThriftHiveMetastore_show_compact_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.show_compact", bytes); + } + + ThriftHiveMetastore_show_compact_result result; + try { + iface_->show_compact(result.success, args.rqst); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.show_compact"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("show_compact", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.show_compact"); + } + + oprot->writeMessageBegin("show_compact", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.show_compact", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_add_dynamic_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_dynamic_partitions", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_dynamic_partitions"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_dynamic_partitions"); + } + + ThriftHiveMetastore_add_dynamic_partitions_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_dynamic_partitions", bytes); + } + + ThriftHiveMetastore_add_dynamic_partitions_result result; + try { + iface_->add_dynamic_partitions(args.rqst); + } catch (NoSuchTxnException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (TxnAbortedException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_dynamic_partitions"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("add_dynamic_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_dynamic_partitions"); + } + + oprot->writeMessageBegin("add_dynamic_partitions", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_dynamic_partitions", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_next_notification(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_next_notification", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_next_notification"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_next_notification"); + } + + ThriftHiveMetastore_get_next_notification_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_next_notification", bytes); + } + + ThriftHiveMetastore_get_next_notification_result result; + try { + iface_->get_next_notification(result.success, args.rqst); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_next_notification"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_next_notification", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_next_notification"); + } + + oprot->writeMessageBegin("get_next_notification", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_next_notification", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_current_notificationEventId(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_current_notificationEventId", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_current_notificationEventId"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_current_notificationEventId"); + } + + ThriftHiveMetastore_get_current_notificationEventId_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_current_notificationEventId", bytes); + } + + ThriftHiveMetastore_get_current_notificationEventId_result result; + try { + iface_->get_current_notificationEventId(result.success); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_current_notificationEventId"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_current_notificationEventId", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_current_notificationEventId"); + } + + oprot->writeMessageBegin("get_current_notificationEventId", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_current_notificationEventId", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_notification_events_count(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_notification_events_count", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_notification_events_count"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_notification_events_count"); + } + + ThriftHiveMetastore_get_notification_events_count_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_notification_events_count", bytes); + } + + ThriftHiveMetastore_get_notification_events_count_result result; + try { + iface_->get_notification_events_count(result.success, args.rqst); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_notification_events_count"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_notification_events_count", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_notification_events_count"); + } + + oprot->writeMessageBegin("get_notification_events_count", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_notification_events_count", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_fire_listener_event(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.fire_listener_event", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.fire_listener_event"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.fire_listener_event"); + } + + ThriftHiveMetastore_fire_listener_event_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.fire_listener_event", bytes); + } + + ThriftHiveMetastore_fire_listener_event_result result; + try { + iface_->fire_listener_event(result.success, args.rqst); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.fire_listener_event"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("fire_listener_event", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.fire_listener_event"); + } + + oprot->writeMessageBegin("fire_listener_event", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.fire_listener_event", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_flushCache(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.flushCache", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.flushCache"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.flushCache"); + } + + ThriftHiveMetastore_flushCache_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.flushCache", bytes); + } + + ThriftHiveMetastore_flushCache_result result; + try { + iface_->flushCache(); + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.flushCache"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("flushCache", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.flushCache"); + } + + oprot->writeMessageBegin("flushCache", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.flushCache", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_cm_recycle(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.cm_recycle", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.cm_recycle"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.cm_recycle"); + } + + ThriftHiveMetastore_cm_recycle_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.cm_recycle", bytes); + } + + ThriftHiveMetastore_cm_recycle_result result; + try { + iface_->cm_recycle(result.success, args.request); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.cm_recycle"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("cm_recycle", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.cm_recycle"); + } + + oprot->writeMessageBegin("cm_recycle", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.cm_recycle", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_file_metadata_by_expr(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_file_metadata_by_expr", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_file_metadata_by_expr"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_file_metadata_by_expr"); + } + + ThriftHiveMetastore_get_file_metadata_by_expr_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_file_metadata_by_expr", bytes); + } + + ThriftHiveMetastore_get_file_metadata_by_expr_result result; + try { + iface_->get_file_metadata_by_expr(result.success, args.req); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_file_metadata_by_expr"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_file_metadata_by_expr", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_file_metadata_by_expr"); + } + + oprot->writeMessageBegin("get_file_metadata_by_expr", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_file_metadata_by_expr", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_file_metadata", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_file_metadata"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_file_metadata"); + } + + ThriftHiveMetastore_get_file_metadata_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_file_metadata", bytes); + } + + ThriftHiveMetastore_get_file_metadata_result result; + try { + iface_->get_file_metadata(result.success, args.req); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_file_metadata"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_file_metadata", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_file_metadata"); + } + + oprot->writeMessageBegin("get_file_metadata", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_file_metadata", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_put_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.put_file_metadata", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.put_file_metadata"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.put_file_metadata"); + } + + ThriftHiveMetastore_put_file_metadata_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.put_file_metadata", bytes); + } + + ThriftHiveMetastore_put_file_metadata_result result; + try { + iface_->put_file_metadata(result.success, args.req); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.put_file_metadata"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("put_file_metadata", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.put_file_metadata"); + } + + oprot->writeMessageBegin("put_file_metadata", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.put_file_metadata", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_clear_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.clear_file_metadata", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.clear_file_metadata"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.clear_file_metadata"); + } + + ThriftHiveMetastore_clear_file_metadata_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.clear_file_metadata", bytes); + } + + ThriftHiveMetastore_clear_file_metadata_result result; + try { + iface_->clear_file_metadata(result.success, args.req); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.clear_file_metadata"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("clear_file_metadata", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.clear_file_metadata"); + } + + oprot->writeMessageBegin("clear_file_metadata", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.clear_file_metadata", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_cache_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.cache_file_metadata", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.cache_file_metadata"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.cache_file_metadata"); + } + + ThriftHiveMetastore_cache_file_metadata_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.cache_file_metadata", bytes); + } + + ThriftHiveMetastore_cache_file_metadata_result result; + try { + iface_->cache_file_metadata(result.success, args.req); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.cache_file_metadata"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("cache_file_metadata", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.cache_file_metadata"); + } + + oprot->writeMessageBegin("cache_file_metadata", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.cache_file_metadata", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_metastore_db_uuid(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_metastore_db_uuid", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_metastore_db_uuid"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_metastore_db_uuid"); + } + + ThriftHiveMetastore_get_metastore_db_uuid_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_metastore_db_uuid", bytes); + } + + ThriftHiveMetastore_get_metastore_db_uuid_result result; + try { + iface_->get_metastore_db_uuid(result.success); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_metastore_db_uuid"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_metastore_db_uuid", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_metastore_db_uuid"); + } + + oprot->writeMessageBegin("get_metastore_db_uuid", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_metastore_db_uuid", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_create_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_resource_plan", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_resource_plan"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_resource_plan"); + } + + ThriftHiveMetastore_create_resource_plan_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_resource_plan", bytes); + } + + ThriftHiveMetastore_create_resource_plan_result result; + try { + iface_->create_resource_plan(result.success, args.request); + result.__isset.success = true; + } catch (AlreadyExistsException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_resource_plan"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("create_resource_plan", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_resource_plan"); + } + + oprot->writeMessageBegin("create_resource_plan", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_resource_plan", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_resource_plan", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_resource_plan"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_resource_plan"); + } + + ThriftHiveMetastore_get_resource_plan_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_resource_plan", bytes); + } + + ThriftHiveMetastore_get_resource_plan_result result; + try { + iface_->get_resource_plan(result.success, args.request); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_resource_plan"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_resource_plan", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_resource_plan"); + } + + oprot->writeMessageBegin("get_resource_plan", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_resource_plan", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_active_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_active_resource_plan", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_active_resource_plan"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_active_resource_plan"); + } + + ThriftHiveMetastore_get_active_resource_plan_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_active_resource_plan", bytes); + } + + ThriftHiveMetastore_get_active_resource_plan_result result; + try { + iface_->get_active_resource_plan(result.success, args.request); + result.__isset.success = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_active_resource_plan"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_active_resource_plan", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_active_resource_plan"); + } + + oprot->writeMessageBegin("get_active_resource_plan", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_active_resource_plan", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_all_resource_plans(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_all_resource_plans", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_all_resource_plans"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_resource_plans"); + } + + ThriftHiveMetastore_get_all_resource_plans_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_all_resource_plans", bytes); + } + + ThriftHiveMetastore_get_all_resource_plans_result result; + try { + iface_->get_all_resource_plans(result.success, args.request); + result.__isset.success = true; + } catch (MetaException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_all_resource_plans"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_all_resource_plans", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_resource_plans"); + } + + oprot->writeMessageBegin("get_all_resource_plans", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_all_resource_plans", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_alter_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_resource_plan", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_resource_plan"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_resource_plan"); + } + + ThriftHiveMetastore_alter_resource_plan_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_resource_plan", bytes); + } + + ThriftHiveMetastore_alter_resource_plan_result result; + try { + iface_->alter_resource_plan(result.success, args.request); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidOperationException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_resource_plan"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("alter_resource_plan", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_resource_plan"); + } + + oprot->writeMessageBegin("alter_resource_plan", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_resource_plan", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_validate_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.validate_resource_plan", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.validate_resource_plan"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.validate_resource_plan"); + } + + ThriftHiveMetastore_validate_resource_plan_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.validate_resource_plan", bytes); + } + + ThriftHiveMetastore_validate_resource_plan_result result; + try { + iface_->validate_resource_plan(result.success, args.request); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.validate_resource_plan"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("validate_resource_plan", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.validate_resource_plan"); + } + + oprot->writeMessageBegin("validate_resource_plan", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.validate_resource_plan", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_drop_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_resource_plan", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_resource_plan"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_resource_plan"); + } + + ThriftHiveMetastore_drop_resource_plan_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_resource_plan", bytes); + } + + ThriftHiveMetastore_drop_resource_plan_result result; + try { + iface_->drop_resource_plan(result.success, args.request); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidOperationException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_resource_plan"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("drop_resource_plan", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_resource_plan"); + } + + oprot->writeMessageBegin("drop_resource_plan", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_resource_plan", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_create_wm_trigger(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_wm_trigger", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_wm_trigger"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_wm_trigger"); + } + + ThriftHiveMetastore_create_wm_trigger_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_wm_trigger", bytes); + } + + ThriftHiveMetastore_create_wm_trigger_result result; + try { + iface_->create_wm_trigger(result.success, args.request); + result.__isset.success = true; + } catch (AlreadyExistsException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (NoSuchObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (InvalidObjectException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (MetaException &o4) { + result.o4 = o4; + result.__isset.o4 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_wm_trigger"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("create_wm_trigger", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_wm_trigger"); + } + + oprot->writeMessageBegin("create_wm_trigger", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_wm_trigger", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_alter_wm_trigger(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_wm_trigger", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_wm_trigger"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_wm_trigger"); + } + + ThriftHiveMetastore_alter_wm_trigger_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_wm_trigger", bytes); + } + + ThriftHiveMetastore_alter_wm_trigger_result result; + try { + iface_->alter_wm_trigger(result.success, args.request); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidObjectException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_wm_trigger"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("alter_wm_trigger", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_wm_trigger"); + } + + oprot->writeMessageBegin("alter_wm_trigger", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_wm_trigger", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_drop_wm_trigger(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_wm_trigger", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_wm_trigger"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_wm_trigger"); + } + + ThriftHiveMetastore_drop_wm_trigger_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_wm_trigger", bytes); + } + + ThriftHiveMetastore_drop_wm_trigger_result result; + try { + iface_->drop_wm_trigger(result.success, args.request); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (InvalidOperationException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (MetaException &o3) { + result.o3 = o3; + result.__isset.o3 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_wm_trigger"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("drop_wm_trigger", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_wm_trigger"); + } + + oprot->writeMessageBegin("drop_wm_trigger", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_wm_trigger", bytes); + } +} + +void ThriftHiveMetastoreProcessor::process_get_triggers_for_resourceplan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_triggers_for_resourceplan", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_triggers_for_resourceplan"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_triggers_for_resourceplan"); + } + + ThriftHiveMetastore_get_triggers_for_resourceplan_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_triggers_for_resourceplan", bytes); + } + + ThriftHiveMetastore_get_triggers_for_resourceplan_result result; + try { + iface_->get_triggers_for_resourceplan(result.success, args.request); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { + result.o1 = o1; + result.__isset.o1 = true; + } catch (MetaException &o2) { + result.o2 = o2; + result.__isset.o2 = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_triggers_for_resourceplan"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("get_triggers_for_resourceplan", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_triggers_for_resourceplan"); + } + + oprot->writeMessageBegin("get_triggers_for_resourceplan", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_triggers_for_resourceplan", bytes); + } +} + +::std::shared_ptr< ::apache::thrift::TProcessor > ThriftHiveMetastoreProcessorFactory::getProcessor(const ::apache::thrift::TConnectionInfo& connInfo) { + ::apache::thrift::ReleaseHandler< ThriftHiveMetastoreIfFactory > cleanup(handlerFactory_); + ::std::shared_ptr< ThriftHiveMetastoreIf > handler(handlerFactory_->getHandler(connInfo), cleanup); + ::std::shared_ptr< ::apache::thrift::TProcessor > processor(new ThriftHiveMetastoreProcessor(handler)); + return processor; +} + +void ThriftHiveMetastoreConcurrentClient::getMetaConf(std::string& _return, const std::string& key) +{ + int32_t seqid = send_getMetaConf(key); + recv_getMetaConf(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_getMetaConf(const std::string& key) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("getMetaConf", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_getMetaConf_pargs args; + args.key = &key; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_getMetaConf(std::string& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("getMetaConf") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_getMetaConf_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "getMetaConf failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::setMetaConf(const std::string& key, const std::string& value) +{ + int32_t seqid = send_setMetaConf(key, value); + recv_setMetaConf(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_setMetaConf(const std::string& key, const std::string& value) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("setMetaConf", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_setMetaConf_pargs args; + args.key = &key; + args.value = &value; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_setMetaConf(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("setMetaConf") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_setMetaConf_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::create_database(const Database& database) +{ + int32_t seqid = send_create_database(database); + recv_create_database(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_create_database(const Database& database) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("create_database", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_database_pargs args; + args.database = &database; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_create_database(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_database") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_create_database_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_database(Database& _return, const std::string& name) +{ + int32_t seqid = send_get_database(name); + recv_get_database(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_database(const std::string& name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_database", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_database_pargs args; + args.name = &name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_database(Database& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_database") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_database_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_database failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::drop_database(const std::string& name, const bool deleteData, const bool cascade) +{ + int32_t seqid = send_drop_database(name, deleteData, cascade); + recv_drop_database(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_drop_database(const std::string& name, const bool deleteData, const bool cascade) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("drop_database", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_database_pargs args; + args.name = &name; + args.deleteData = &deleteData; + args.cascade = &cascade; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_drop_database(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_database") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_drop_database_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_databases(std::vector & _return, const std::string& pattern) +{ + int32_t seqid = send_get_databases(pattern); + recv_get_databases(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_databases(const std::string& pattern) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_databases", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_databases_pargs args; + args.pattern = &pattern; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_databases(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_databases") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_databases_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_databases failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_all_databases(std::vector & _return) +{ + int32_t seqid = send_get_all_databases(); + recv_get_all_databases(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_all_databases() +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_all_databases", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_all_databases_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_all_databases(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_all_databases") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_all_databases_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_databases failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::alter_database(const std::string& dbname, const Database& db) +{ + int32_t seqid = send_alter_database(dbname, db); + recv_alter_database(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_alter_database(const std::string& dbname, const Database& db) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("alter_database", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_database_pargs args; + args.dbname = &dbname; + args.db = &db; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_alter_database(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_database") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_alter_database_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_type(Type& _return, const std::string& name) +{ + int32_t seqid = send_get_type(name); + recv_get_type(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_type(const std::string& name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_type", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_type_pargs args; + args.name = &name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_type(Type& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_type") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_type_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_type failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::create_type(const Type& type) +{ + int32_t seqid = send_create_type(type); + return recv_create_type(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_create_type(const Type& type) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("create_type", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_type_pargs args; + args.type = &type; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_create_type(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_type") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_create_type_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_type failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::drop_type(const std::string& type) +{ + int32_t seqid = send_drop_type(type); + return recv_drop_type(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_drop_type(const std::string& type) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("drop_type", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_type_pargs args; + args.type = &type; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_drop_type(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_type") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_drop_type_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_type failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_type_all(std::map & _return, const std::string& name) +{ + int32_t seqid = send_get_type_all(name); + recv_get_type_all(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_type_all(const std::string& name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_type_all", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_type_all_pargs args; + args.name = &name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_type_all(std::map & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_type_all") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_type_all_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_type_all failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_fields(std::vector & _return, const std::string& db_name, const std::string& table_name) +{ + int32_t seqid = send_get_fields(db_name, table_name); + recv_get_fields(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_fields(const std::string& db_name, const std::string& table_name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_fields", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_fields_pargs args; + args.db_name = &db_name; + args.table_name = &table_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_fields(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_fields") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_fields_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_fields failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_fields_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) +{ + int32_t seqid = send_get_fields_with_environment_context(db_name, table_name, environment_context); + recv_get_fields_with_environment_context(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_fields_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_fields_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_fields_with_environment_context_pargs args; + args.db_name = &db_name; + args.table_name = &table_name; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_fields_with_environment_context(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_fields_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_fields_with_environment_context_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_fields_with_environment_context failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_schema(std::vector & _return, const std::string& db_name, const std::string& table_name) +{ + int32_t seqid = send_get_schema(db_name, table_name); + recv_get_schema(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_schema(const std::string& db_name, const std::string& table_name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_schema", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_schema_pargs args; + args.db_name = &db_name; + args.table_name = &table_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_schema(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_schema") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_schema_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_schema failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_schema_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) +{ + int32_t seqid = send_get_schema_with_environment_context(db_name, table_name, environment_context); + recv_get_schema_with_environment_context(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_schema_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_schema_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_schema_with_environment_context_pargs args; + args.db_name = &db_name; + args.table_name = &table_name; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_schema_with_environment_context(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_schema_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_schema_with_environment_context_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_schema_with_environment_context failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::create_table(const Table& tbl) +{ + int32_t seqid = send_create_table(tbl); + recv_create_table(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_create_table(const Table& tbl) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("create_table", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_table_pargs args; + args.tbl = &tbl; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_create_table(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_table") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_create_table_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + if (result.__isset.o4) { + sentry.commit(); + throw result.o4; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) +{ + int32_t seqid = send_create_table_with_environment_context(tbl, environment_context); + recv_create_table_with_environment_context(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("create_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_table_with_environment_context_pargs args; + args.tbl = &tbl; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_create_table_with_environment_context(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_table_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_create_table_with_environment_context_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + if (result.__isset.o4) { + sentry.commit(); + throw result.o4; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints) +{ + int32_t seqid = send_create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints); + recv_create_table_with_constraints(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("create_table_with_constraints", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_table_with_constraints_pargs args; + args.tbl = &tbl; + args.primaryKeys = &primaryKeys; + args.foreignKeys = &foreignKeys; + args.uniqueConstraints = &uniqueConstraints; + args.notNullConstraints = ¬NullConstraints; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_create_table_with_constraints(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_table_with_constraints") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_create_table_with_constraints_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + if (result.__isset.o4) { + sentry.commit(); + throw result.o4; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::drop_constraint(const DropConstraintRequest& req) +{ + int32_t seqid = send_drop_constraint(req); + recv_drop_constraint(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_drop_constraint(const DropConstraintRequest& req) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("drop_constraint", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_constraint_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_drop_constraint(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_constraint") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_drop_constraint_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::add_primary_key(const AddPrimaryKeyRequest& req) +{ + int32_t seqid = send_add_primary_key(req); + recv_add_primary_key(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_add_primary_key(const AddPrimaryKeyRequest& req) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("add_primary_key", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_primary_key_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_add_primary_key(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_primary_key") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_add_primary_key_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::add_foreign_key(const AddForeignKeyRequest& req) +{ + int32_t seqid = send_add_foreign_key(req); + recv_add_foreign_key(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_add_foreign_key(const AddForeignKeyRequest& req) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("add_foreign_key", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_foreign_key_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_add_foreign_key(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_foreign_key") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_add_foreign_key_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::add_unique_constraint(const AddUniqueConstraintRequest& req) +{ + int32_t seqid = send_add_unique_constraint(req); + recv_add_unique_constraint(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_add_unique_constraint(const AddUniqueConstraintRequest& req) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("add_unique_constraint", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_unique_constraint_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_add_unique_constraint(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_unique_constraint") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_add_unique_constraint_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::add_not_null_constraint(const AddNotNullConstraintRequest& req) +{ + int32_t seqid = send_add_not_null_constraint(req); + recv_add_not_null_constraint(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_add_not_null_constraint(const AddNotNullConstraintRequest& req) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("add_not_null_constraint", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_not_null_constraint_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_add_not_null_constraint(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_not_null_constraint") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_add_not_null_constraint_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::drop_table(const std::string& dbname, const std::string& name, const bool deleteData) +{ + int32_t seqid = send_drop_table(dbname, name, deleteData); + recv_drop_table(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("drop_table", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_table_pargs args; + args.dbname = &dbname; + args.name = &name; + args.deleteData = &deleteData; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_drop_table(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_table") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_drop_table_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) +{ + int32_t seqid = send_drop_table_with_environment_context(dbname, name, deleteData, environment_context); + recv_drop_table_with_environment_context(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("drop_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_table_with_environment_context_pargs args; + args.dbname = &dbname; + args.name = &name; + args.deleteData = &deleteData; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_drop_table_with_environment_context(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_table_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_drop_table_with_environment_context_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames) +{ + int32_t seqid = send_truncate_table(dbName, tableName, partNames); + recv_truncate_table(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("truncate_table", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_truncate_table_pargs args; + args.dbName = &dbName; + args.tableName = &tableName; + args.partNames = &partNames; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_truncate_table(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("truncate_table") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_truncate_table_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_tables(std::vector & _return, const std::string& db_name, const std::string& pattern) +{ + int32_t seqid = send_get_tables(db_name, pattern); + recv_get_tables(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_tables(const std::string& db_name, const std::string& pattern) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_tables", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_tables_pargs args; + args.db_name = &db_name; + args.pattern = &pattern; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_tables(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_tables") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_tables_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_tables failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_tables_by_type(std::vector & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType) +{ + int32_t seqid = send_get_tables_by_type(db_name, pattern, tableType); + recv_get_tables_by_type(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_tables_by_type(const std::string& db_name, const std::string& pattern, const std::string& tableType) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_tables_by_type", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_tables_by_type_pargs args; + args.db_name = &db_name; + args.pattern = &pattern; + args.tableType = &tableType; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_tables_by_type(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_tables_by_type") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_tables_by_type_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_tables_by_type failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_table_meta(std::vector & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types) +{ + int32_t seqid = send_get_table_meta(db_patterns, tbl_patterns, tbl_types); + recv_get_table_meta(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_table_meta(const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_table_meta", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_table_meta_pargs args; + args.db_patterns = &db_patterns; + args.tbl_patterns = &tbl_patterns; + args.tbl_types = &tbl_types; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_table_meta(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_table_meta") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_table_meta_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_meta failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_all_tables(std::vector & _return, const std::string& db_name) +{ + int32_t seqid = send_get_all_tables(db_name); + recv_get_all_tables(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_all_tables(const std::string& db_name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_all_tables", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_all_tables_pargs args; + args.db_name = &db_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_all_tables(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_all_tables") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_all_tables_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_tables failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_table(Table& _return, const std::string& dbname, const std::string& tbl_name) +{ + int32_t seqid = send_get_table(dbname, tbl_name); + recv_get_table(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_table(const std::string& dbname, const std::string& tbl_name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_table", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_table_pargs args; + args.dbname = &dbname; + args.tbl_name = &tbl_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_table(Table& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_table") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_table_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_table_objects_by_name(std::vector
& _return, const std::string& dbname, const std::vector & tbl_names) +{ + int32_t seqid = send_get_table_objects_by_name(dbname, tbl_names); + recv_get_table_objects_by_name(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_table_objects_by_name(const std::string& dbname, const std::vector & tbl_names) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_table_objects_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_table_objects_by_name_pargs args; + args.dbname = &dbname; + args.tbl_names = &tbl_names; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_table_objects_by_name(std::vector
& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_table_objects_by_name") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_table_objects_by_name_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_objects_by_name failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_table_req(GetTableResult& _return, const GetTableRequest& req) +{ + int32_t seqid = send_get_table_req(req); + recv_get_table_req(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_table_req(const GetTableRequest& req) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_table_req", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_table_req_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_table_req(GetTableResult& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_table_req") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_table_req_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_req failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req) +{ + int32_t seqid = send_get_table_objects_by_name_req(req); + recv_get_table_objects_by_name_req(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_table_objects_by_name_req(const GetTablesRequest& req) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_table_objects_by_name_req", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_table_objects_by_name_req_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_table_objects_by_name_req(GetTablesResult& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_table_objects_by_name_req") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_table_objects_by_name_req_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_objects_by_name_req failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_table_names_by_filter(std::vector & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables) +{ + int32_t seqid = send_get_table_names_by_filter(dbname, filter, max_tables); + recv_get_table_names_by_filter(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_table_names_by_filter(const std::string& dbname, const std::string& filter, const int16_t max_tables) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_table_names_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_table_names_by_filter_pargs args; + args.dbname = &dbname; + args.filter = &filter; + args.max_tables = &max_tables; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_table_names_by_filter(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_table_names_by_filter") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_table_names_by_filter_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_names_by_filter failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) +{ + int32_t seqid = send_alter_table(dbname, tbl_name, new_tbl); + recv_alter_table(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("alter_table", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_table_pargs args; + args.dbname = &dbname; + args.tbl_name = &tbl_name; + args.new_tbl = &new_tbl; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_alter_table(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_table") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_alter_table_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) +{ + int32_t seqid = send_alter_table_with_environment_context(dbname, tbl_name, new_tbl, environment_context); + recv_alter_table_with_environment_context(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("alter_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_table_with_environment_context_pargs args; + args.dbname = &dbname; + args.tbl_name = &tbl_name; + args.new_tbl = &new_tbl; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_alter_table_with_environment_context(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_table_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_alter_table_with_environment_context_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade) +{ + int32_t seqid = send_alter_table_with_cascade(dbname, tbl_name, new_tbl, cascade); + recv_alter_table_with_cascade(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("alter_table_with_cascade", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_table_with_cascade_pargs args; + args.dbname = &dbname; + args.tbl_name = &tbl_name; + args.new_tbl = &new_tbl; + args.cascade = &cascade; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_alter_table_with_cascade(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_table_with_cascade") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_alter_table_with_cascade_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::add_partition(Partition& _return, const Partition& new_part) +{ + int32_t seqid = send_add_partition(new_part); + recv_add_partition(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_add_partition(const Partition& new_part) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("add_partition", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_partition_pargs args; + args.new_part = &new_part; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_add_partition(Partition& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_partition") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_add_partition_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partition failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::add_partition_with_environment_context(Partition& _return, const Partition& new_part, const EnvironmentContext& environment_context) +{ + int32_t seqid = send_add_partition_with_environment_context(new_part, environment_context); + recv_add_partition_with_environment_context(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_add_partition_with_environment_context(const Partition& new_part, const EnvironmentContext& environment_context) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("add_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_partition_with_environment_context_pargs args; + args.new_part = &new_part; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_add_partition_with_environment_context(Partition& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_partition_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_add_partition_with_environment_context_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partition_with_environment_context failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +int32_t ThriftHiveMetastoreConcurrentClient::add_partitions(const std::vector & new_parts) +{ + int32_t seqid = send_add_partitions(new_parts); + return recv_add_partitions(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_add_partitions(const std::vector & new_parts) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("add_partitions", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_partitions_pargs args; + args.new_parts = &new_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +int32_t ThriftHiveMetastoreConcurrentClient::recv_add_partitions(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_partitions") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + int32_t _return; + ThriftHiveMetastore_add_partitions_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partitions failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +int32_t ThriftHiveMetastoreConcurrentClient::add_partitions_pspec(const std::vector & new_parts) +{ + int32_t seqid = send_add_partitions_pspec(new_parts); + return recv_add_partitions_pspec(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_add_partitions_pspec(const std::vector & new_parts) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("add_partitions_pspec", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_partitions_pspec_pargs args; + args.new_parts = &new_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +int32_t ThriftHiveMetastoreConcurrentClient::recv_add_partitions_pspec(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_partitions_pspec") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + int32_t _return; + ThriftHiveMetastore_add_partitions_pspec_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partitions_pspec failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::append_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) +{ + int32_t seqid = send_append_partition(db_name, tbl_name, part_vals); + recv_append_partition(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_append_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("append_partition", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_append_partition_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_append_partition(Partition& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("append_partition") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_append_partition_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::add_partitions_req(AddPartitionsResult& _return, const AddPartitionsRequest& request) +{ + int32_t seqid = send_add_partitions_req(request); + recv_add_partitions_req(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_add_partitions_req(const AddPartitionsRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("add_partitions_req", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_partitions_req_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_add_partitions_req(AddPartitionsResult& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_partitions_req") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_add_partitions_req_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partitions_req failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::append_partition_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context) +{ + int32_t seqid = send_append_partition_with_environment_context(db_name, tbl_name, part_vals, environment_context); + recv_append_partition_with_environment_context(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_append_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_append_partition_with_environment_context_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_append_partition_with_environment_context(Partition& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("append_partition_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_append_partition_with_environment_context_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition_with_environment_context failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::append_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) +{ + int32_t seqid = send_append_partition_by_name(db_name, tbl_name, part_name); + recv_append_partition_by_name(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_append_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("append_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_append_partition_by_name_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_append_partition_by_name(Partition& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("append_partition_by_name") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_append_partition_by_name_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition_by_name failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::append_partition_by_name_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context) +{ + int32_t seqid = send_append_partition_by_name_with_environment_context(db_name, tbl_name, part_name, environment_context); + recv_append_partition_by_name_with_environment_context(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_append_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_append_partition_by_name_with_environment_context(Partition& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("append_partition_by_name_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition_by_name_with_environment_context failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) +{ + int32_t seqid = send_drop_partition(db_name, tbl_name, part_vals, deleteData); + return recv_drop_partition(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("drop_partition", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_partition_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.deleteData = &deleteData; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_drop_partition(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_partition") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_drop_partition_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context) +{ + int32_t seqid = send_drop_partition_with_environment_context(db_name, tbl_name, part_vals, deleteData, environment_context); + return recv_drop_partition_with_environment_context(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("drop_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_partition_with_environment_context_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.deleteData = &deleteData; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_drop_partition_with_environment_context(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_partition_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_drop_partition_with_environment_context_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_with_environment_context failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData) +{ + int32_t seqid = send_drop_partition_by_name(db_name, tbl_name, part_name, deleteData); + return recv_drop_partition_by_name(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("drop_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_partition_by_name_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; + args.deleteData = &deleteData; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_drop_partition_by_name(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_partition_by_name") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_drop_partition_by_name_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_by_name failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context) +{ + int32_t seqid = send_drop_partition_by_name_with_environment_context(db_name, tbl_name, part_name, deleteData, environment_context); + return recv_drop_partition_by_name_with_environment_context(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; + args.deleteData = &deleteData; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_drop_partition_by_name_with_environment_context(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_partition_by_name_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_by_name_with_environment_context failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::drop_partitions_req(DropPartitionsResult& _return, const DropPartitionsRequest& req) +{ + int32_t seqid = send_drop_partitions_req(req); + recv_drop_partitions_req(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_drop_partitions_req(const DropPartitionsRequest& req) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("drop_partitions_req", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_partitions_req_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_drop_partitions_req(DropPartitionsResult& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_partitions_req") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_drop_partitions_req_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partitions_req failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) +{ + int32_t seqid = send_get_partition(db_name, tbl_name, part_vals); + recv_get_partition(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partition_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_partition(Partition& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partition") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_partition_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::exchange_partition(Partition& _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) +{ + int32_t seqid = send_exchange_partition(partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); + recv_exchange_partition(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_exchange_partition(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_exchange_partition_pargs args; + args.partitionSpecs = &partitionSpecs; + args.source_db = &source_db; + args.source_table_name = &source_table_name; + args.dest_db = &dest_db; + args.dest_table_name = &dest_table_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_exchange_partition(Partition& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("exchange_partition") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_exchange_partition_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + if (result.__isset.o4) { + sentry.commit(); + throw result.o4; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "exchange_partition failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::exchange_partitions(std::vector & _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) +{ + int32_t seqid = send_exchange_partitions(partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); + recv_exchange_partitions(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_exchange_partitions(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("exchange_partitions", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_exchange_partitions_pargs args; + args.partitionSpecs = &partitionSpecs; + args.source_db = &source_db; + args.source_table_name = &source_table_name; + args.dest_db = &dest_db; + args.dest_table_name = &dest_table_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_exchange_partitions(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("exchange_partitions") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_exchange_partitions_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + if (result.__isset.o4) { + sentry.commit(); + throw result.o4; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "exchange_partitions failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_partition_with_auth(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names) +{ + int32_t seqid = send_get_partition_with_auth(db_name, tbl_name, part_vals, user_name, group_names); + recv_get_partition_with_auth(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_partition_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_partition_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partition_with_auth_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.user_name = &user_name; + args.group_names = &group_names; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_partition_with_auth(Partition& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partition_with_auth") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_partition_with_auth_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_with_auth failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) +{ + int32_t seqid = send_get_partition_by_name(db_name, tbl_name, part_name); + recv_get_partition_by_name(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partition_by_name_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_partition_by_name(Partition& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partition_by_name") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_partition_by_name_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_by_name failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_partitions(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) +{ + int32_t seqid = send_get_partitions(db_name, tbl_name, max_parts); + recv_get_partitions(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.max_parts = &max_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_partitions(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_partitions_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_partitions_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) +{ + int32_t seqid = send_get_partitions_with_auth(db_name, tbl_name, max_parts, user_name, group_names); + recv_get_partitions_with_auth(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions_with_auth(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_with_auth_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.max_parts = &max_parts; + args.user_name = &user_name; + args.group_names = &group_names; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_partitions_with_auth(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions_with_auth") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_partitions_with_auth_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_with_auth failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_partitions_pspec(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int32_t max_parts) +{ + int32_t seqid = send_get_partitions_pspec(db_name, tbl_name, max_parts); + recv_get_partitions_pspec(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions_pspec(const std::string& db_name, const std::string& tbl_name, const int32_t max_parts) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_partitions_pspec", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_pspec_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.max_parts = &max_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_partitions_pspec(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions_pspec") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_partitions_pspec_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_pspec failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_partition_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) +{ + int32_t seqid = send_get_partition_names(db_name, tbl_name, max_parts); + recv_get_partition_names(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_partition_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partition_names_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.max_parts = &max_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_partition_names(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partition_names") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_partition_names_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_names failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_partition_values(PartitionValuesResponse& _return, const PartitionValuesRequest& request) +{ + int32_t seqid = send_get_partition_values(request); + recv_get_partition_values(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_partition_values(const PartitionValuesRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_partition_values", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partition_values_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_partition_values(PartitionValuesResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partition_values") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_partition_values_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_values failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_partitions_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) +{ + int32_t seqid = send_get_partitions_ps(db_name, tbl_name, part_vals, max_parts); + recv_get_partitions_ps(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_ps_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.max_parts = &max_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_partitions_ps(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions_ps") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_partitions_ps_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_ps failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_partitions_ps_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) +{ + int32_t seqid = send_get_partitions_ps_with_auth(db_name, tbl_name, part_vals, max_parts, user_name, group_names); + recv_get_partitions_ps_with_auth(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions_ps_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_ps_with_auth_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.max_parts = &max_parts; + args.user_name = &user_name; + args.group_names = &group_names; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_partitions_ps_with_auth(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions_ps_with_auth") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_partitions_ps_with_auth_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_ps_with_auth failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_partition_names_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) +{ + int32_t seqid = send_get_partition_names_ps(db_name, tbl_name, part_vals, max_parts); + recv_get_partition_names_ps(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_partition_names_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partition_names_ps_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.max_parts = &max_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_partition_names_ps(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partition_names_ps") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_partition_names_ps_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_names_ps failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_partitions_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts) +{ + int32_t seqid = send_get_partitions_by_filter(db_name, tbl_name, filter, max_parts); + recv_get_partitions_by_filter(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_by_filter_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.filter = &filter; + args.max_parts = &max_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_partitions_by_filter(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions_by_filter") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_partitions_by_filter_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_filter failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_part_specs_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts) +{ + int32_t seqid = send_get_part_specs_by_filter(db_name, tbl_name, filter, max_parts); + recv_get_part_specs_by_filter(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_part_specs_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_part_specs_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_part_specs_by_filter_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.filter = &filter; + args.max_parts = &max_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_part_specs_by_filter(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_part_specs_by_filter") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_part_specs_by_filter_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_part_specs_by_filter failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_partitions_by_expr(PartitionsByExprResult& _return, const PartitionsByExprRequest& req) +{ + int32_t seqid = send_get_partitions_by_expr(req); + recv_get_partitions_by_expr(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions_by_expr(const PartitionsByExprRequest& req) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_partitions_by_expr", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_by_expr_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_partitions_by_expr(PartitionsByExprResult& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions_by_expr") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_partitions_by_expr_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_expr failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +int32_t ThriftHiveMetastoreConcurrentClient::get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter) +{ + int32_t seqid = send_get_num_partitions_by_filter(db_name, tbl_name, filter); + return recv_get_num_partitions_by_filter(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_num_partitions_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_num_partitions_by_filter_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.filter = &filter; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +int32_t ThriftHiveMetastoreConcurrentClient::recv_get_num_partitions_by_filter(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_num_partitions_by_filter") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + int32_t _return; + ThriftHiveMetastore_get_num_partitions_by_filter_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_num_partitions_by_filter failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_partitions_by_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & names) +{ + int32_t seqid = send_get_partitions_by_names(db_name, tbl_name, names); + recv_get_partitions_by_names(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions_by_names(const std::string& db_name, const std::string& tbl_name, const std::vector & names) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_by_names_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.names = &names; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_partitions_by_names(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions_by_names") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_partitions_by_names_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_names failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part) +{ + int32_t seqid = send_alter_partition(db_name, tbl_name, new_part); + recv_alter_partition(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_partition_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.new_part = &new_part; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_alter_partition(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_partition") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_alter_partition_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) +{ + int32_t seqid = send_alter_partitions(db_name, tbl_name, new_parts); + recv_alter_partitions(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_partitions_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.new_parts = &new_parts; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_alter_partitions(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_partitions") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_alter_partitions_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context) +{ + int32_t seqid = send_alter_partitions_with_environment_context(db_name, tbl_name, new_parts, environment_context); + recv_alter_partitions_with_environment_context(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("alter_partitions_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_partitions_with_environment_context_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.new_parts = &new_parts; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_alter_partitions_with_environment_context(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_partitions_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_alter_partitions_with_environment_context_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context) +{ + int32_t seqid = send_alter_partition_with_environment_context(db_name, tbl_name, new_part, environment_context); + recv_alter_partition_with_environment_context(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_partition_with_environment_context_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.new_part = &new_part; + args.environment_context = &environment_context; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_alter_partition_with_environment_context(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_partition_with_environment_context") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_alter_partition_with_environment_context_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) +{ + int32_t seqid = send_rename_partition(db_name, tbl_name, part_vals, new_part); + recv_rename_partition(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_rename_partition_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.new_part = &new_part; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_rename_partition(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("rename_partition") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_rename_partition_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) +{ + int32_t seqid = send_partition_name_has_valid_characters(part_vals, throw_exception); + return recv_partition_name_has_valid_characters(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("partition_name_has_valid_characters", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_partition_name_has_valid_characters_pargs args; + args.part_vals = &part_vals; + args.throw_exception = &throw_exception; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_partition_name_has_valid_characters(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("partition_name_has_valid_characters") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_partition_name_has_valid_characters_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_has_valid_characters failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_config_value(std::string& _return, const std::string& name, const std::string& defaultValue) +{ + int32_t seqid = send_get_config_value(name, defaultValue); + recv_get_config_value(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_config_value(const std::string& name, const std::string& defaultValue) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_config_value_pargs args; + args.name = &name; + args.defaultValue = &defaultValue; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_config_value(std::string& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_config_value") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_config_value_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_config_value failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::partition_name_to_vals(std::vector & _return, const std::string& part_name) +{ + int32_t seqid = send_partition_name_to_vals(part_name); + recv_partition_name_to_vals(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_partition_name_to_vals(const std::string& part_name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_partition_name_to_vals_pargs args; + args.part_name = &part_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_partition_name_to_vals(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("partition_name_to_vals") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_partition_name_to_vals_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_to_vals failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::partition_name_to_spec(std::map & _return, const std::string& part_name) +{ + int32_t seqid = send_partition_name_to_spec(part_name); + recv_partition_name_to_spec(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_partition_name_to_spec(const std::string& part_name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_partition_name_to_spec_pargs args; + args.part_name = &part_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_partition_name_to_spec(std::map & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("partition_name_to_spec") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_partition_name_to_spec_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_to_spec failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) +{ + int32_t seqid = send_markPartitionForEvent(db_name, tbl_name, part_vals, eventType); + recv_markPartitionForEvent(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_markPartitionForEvent_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.eventType = &eventType; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_markPartitionForEvent(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("markPartitionForEvent") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_markPartitionForEvent_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + if (result.__isset.o4) { + sentry.commit(); + throw result.o4; + } + if (result.__isset.o5) { + sentry.commit(); + throw result.o5; + } + if (result.__isset.o6) { + sentry.commit(); + throw result.o6; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) +{ + int32_t seqid = send_isPartitionMarkedForEvent(db_name, tbl_name, part_vals, eventType); + return recv_isPartitionMarkedForEvent(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_isPartitionMarkedForEvent_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_vals = &part_vals; + args.eventType = &eventType; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_isPartitionMarkedForEvent(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("isPartitionMarkedForEvent") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_isPartitionMarkedForEvent_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + if (result.__isset.o4) { + sentry.commit(); + throw result.o4; + } + if (result.__isset.o5) { + sentry.commit(); + throw result.o5; + } + if (result.__isset.o6) { + sentry.commit(); + throw result.o6; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "isPartitionMarkedForEvent failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::add_index(Index& _return, const Index& new_index, const Table& index_table) +{ + int32_t seqid = send_add_index(new_index, index_table); + recv_add_index(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_add_index(const Index& new_index, const Table& index_table) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("add_index", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_index_pargs args; + args.new_index = &new_index; + args.index_table = &index_table; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_add_index(Index& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_index") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_add_index_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_index failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) +{ + int32_t seqid = send_alter_index(dbname, base_tbl_name, idx_name, new_idx); + recv_alter_index(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_index_pargs args; + args.dbname = &dbname; + args.base_tbl_name = &base_tbl_name; + args.idx_name = &idx_name; + args.new_idx = &new_idx; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_alter_index(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_index") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_alter_index_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData) +{ + int32_t seqid = send_drop_index_by_name(db_name, tbl_name, index_name, deleteData); + return recv_drop_index_by_name(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_index_by_name_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.index_name = &index_name; + args.deleteData = &deleteData; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_drop_index_by_name(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_index_by_name") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_drop_index_by_name_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_index_by_name failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_index_by_name(Index& _return, const std::string& db_name, const std::string& tbl_name, const std::string& index_name) +{ + int32_t seqid = send_get_index_by_name(db_name, tbl_name, index_name); + recv_get_index_by_name(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_index_by_name_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.index_name = &index_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_index_by_name(Index& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_index_by_name") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_index_by_name_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_index_by_name failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_indexes(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) +{ + int32_t seqid = send_get_indexes(db_name, tbl_name, max_indexes); + recv_get_indexes(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_indexes(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_indexes_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.max_indexes = &max_indexes; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_indexes(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_indexes") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_indexes_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_indexes failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_index_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) +{ + int32_t seqid = send_get_index_names(db_name, tbl_name, max_indexes); + recv_get_index_names(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_index_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_index_names_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.max_indexes = &max_indexes; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_index_names(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_index_names") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_index_names_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_index_names failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request) +{ + int32_t seqid = send_get_primary_keys(request); + recv_get_primary_keys(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_primary_keys(const PrimaryKeysRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_primary_keys", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_primary_keys_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_primary_keys(PrimaryKeysResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_primary_keys") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_primary_keys_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_primary_keys failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request) +{ + int32_t seqid = send_get_foreign_keys(request); + recv_get_foreign_keys(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_foreign_keys(const ForeignKeysRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_foreign_keys", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_foreign_keys_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_foreign_keys(ForeignKeysResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_foreign_keys") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_foreign_keys_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_foreign_keys failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_unique_constraints(UniqueConstraintsResponse& _return, const UniqueConstraintsRequest& request) +{ + int32_t seqid = send_get_unique_constraints(request); + recv_get_unique_constraints(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_unique_constraints(const UniqueConstraintsRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_unique_constraints", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_unique_constraints_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_unique_constraints(UniqueConstraintsResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_unique_constraints") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_unique_constraints_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_unique_constraints failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_not_null_constraints(NotNullConstraintsResponse& _return, const NotNullConstraintsRequest& request) +{ + int32_t seqid = send_get_not_null_constraints(request); + recv_get_not_null_constraints(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_not_null_constraints(const NotNullConstraintsRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_not_null_constraints", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_not_null_constraints_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_not_null_constraints(NotNullConstraintsResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_not_null_constraints") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_not_null_constraints_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_not_null_constraints failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::update_table_column_statistics(const ColumnStatistics& stats_obj) +{ + int32_t seqid = send_update_table_column_statistics(stats_obj); + return recv_update_table_column_statistics(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_update_table_column_statistics(const ColumnStatistics& stats_obj) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("update_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_update_table_column_statistics_pargs args; + args.stats_obj = &stats_obj; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_update_table_column_statistics(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("update_table_column_statistics") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_update_table_column_statistics_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + if (result.__isset.o4) { + sentry.commit(); + throw result.o4; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "update_table_column_statistics failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::update_partition_column_statistics(const ColumnStatistics& stats_obj) +{ + int32_t seqid = send_update_partition_column_statistics(stats_obj); + return recv_update_partition_column_statistics(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_update_partition_column_statistics(const ColumnStatistics& stats_obj) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_update_partition_column_statistics_pargs args; + args.stats_obj = &stats_obj; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_update_partition_column_statistics(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("update_partition_column_statistics") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_update_partition_column_statistics_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + if (result.__isset.o4) { + sentry.commit(); + throw result.o4; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "update_partition_column_statistics failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_table_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& col_name) +{ + int32_t seqid = send_get_table_column_statistics(db_name, tbl_name, col_name); + recv_get_table_column_statistics(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_table_column_statistics_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.col_name = &col_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_table_column_statistics(ColumnStatistics& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_table_column_statistics") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_table_column_statistics_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + if (result.__isset.o4) { + sentry.commit(); + throw result.o4; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_column_statistics failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_partition_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) +{ + int32_t seqid = send_get_partition_column_statistics(db_name, tbl_name, part_name, col_name); + recv_get_partition_column_statistics(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partition_column_statistics_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; + args.col_name = &col_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_partition_column_statistics(ColumnStatistics& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partition_column_statistics") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_partition_column_statistics_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + if (result.__isset.o4) { + sentry.commit(); + throw result.o4; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_column_statistics failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_table_statistics_req(TableStatsResult& _return, const TableStatsRequest& request) +{ + int32_t seqid = send_get_table_statistics_req(request); + recv_get_table_statistics_req(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_table_statistics_req(const TableStatsRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_table_statistics_req", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_table_statistics_req_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_table_statistics_req(TableStatsResult& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_table_statistics_req") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_table_statistics_req_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_statistics_req failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_partitions_statistics_req(PartitionsStatsResult& _return, const PartitionsStatsRequest& request) +{ + int32_t seqid = send_get_partitions_statistics_req(request); + recv_get_partitions_statistics_req(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions_statistics_req(const PartitionsStatsRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_partitions_statistics_req", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_partitions_statistics_req_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_partitions_statistics_req(PartitionsStatsResult& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_partitions_statistics_req") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_partitions_statistics_req_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_statistics_req failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_aggr_stats_for(AggrStats& _return, const PartitionsStatsRequest& request) +{ + int32_t seqid = send_get_aggr_stats_for(request); + recv_get_aggr_stats_for(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_aggr_stats_for(const PartitionsStatsRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_aggr_stats_for", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_aggr_stats_for_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_aggr_stats_for(AggrStats& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_aggr_stats_for") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_aggr_stats_for_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_aggr_stats_for failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::set_aggr_stats_for(const SetPartitionsStatsRequest& request) +{ + int32_t seqid = send_set_aggr_stats_for(request); + return recv_set_aggr_stats_for(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_set_aggr_stats_for(const SetPartitionsStatsRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("set_aggr_stats_for", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_set_aggr_stats_for_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_set_aggr_stats_for(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("set_aggr_stats_for") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_set_aggr_stats_for_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + if (result.__isset.o4) { + sentry.commit(); + throw result.o4; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "set_aggr_stats_for failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) +{ + int32_t seqid = send_delete_partition_column_statistics(db_name, tbl_name, part_name, col_name); + return recv_delete_partition_column_statistics(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_delete_partition_column_statistics_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.part_name = &part_name; + args.col_name = &col_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_delete_partition_column_statistics(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("delete_partition_column_statistics") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_delete_partition_column_statistics_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + if (result.__isset.o4) { + sentry.commit(); + throw result.o4; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "delete_partition_column_statistics failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) +{ + int32_t seqid = send_delete_table_column_statistics(db_name, tbl_name, col_name); + return recv_delete_table_column_statistics(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_delete_table_column_statistics_pargs args; + args.db_name = &db_name; + args.tbl_name = &tbl_name; + args.col_name = &col_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_delete_table_column_statistics(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("delete_table_column_statistics") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_delete_table_column_statistics_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + if (result.__isset.o4) { + sentry.commit(); + throw result.o4; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "delete_table_column_statistics failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::create_function(const Function& func) +{ + int32_t seqid = send_create_function(func); + recv_create_function(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_create_function(const Function& func) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("create_function", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_function_pargs args; + args.func = &func; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_create_function(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_function") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_create_function_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + if (result.__isset.o4) { + sentry.commit(); + throw result.o4; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::drop_function(const std::string& dbName, const std::string& funcName) +{ + int32_t seqid = send_drop_function(dbName, funcName); + recv_drop_function(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_drop_function(const std::string& dbName, const std::string& funcName) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("drop_function", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_function_pargs args; + args.dbName = &dbName; + args.funcName = &funcName; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_drop_function(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_function") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_drop_function_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc) +{ + int32_t seqid = send_alter_function(dbName, funcName, newFunc); + recv_alter_function(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("alter_function", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_function_pargs args; + args.dbName = &dbName; + args.funcName = &funcName; + args.newFunc = &newFunc; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_alter_function(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_function") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_alter_function_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_functions(std::vector & _return, const std::string& dbName, const std::string& pattern) +{ + int32_t seqid = send_get_functions(dbName, pattern); + recv_get_functions(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_functions(const std::string& dbName, const std::string& pattern) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_functions", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_functions_pargs args; + args.dbName = &dbName; + args.pattern = &pattern; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_functions(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_functions") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_functions_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_functions failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_function(Function& _return, const std::string& dbName, const std::string& funcName) +{ + int32_t seqid = send_get_function(dbName, funcName); + recv_get_function(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_function(const std::string& dbName, const std::string& funcName) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_function", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_function_pargs args; + args.dbName = &dbName; + args.funcName = &funcName; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_function(Function& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_function") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_function_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_function failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_all_functions(GetAllFunctionsResponse& _return) +{ + int32_t seqid = send_get_all_functions(); + recv_get_all_functions(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_all_functions() +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_all_functions", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_all_functions_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_all_functions(GetAllFunctionsResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_all_functions") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_all_functions_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_functions failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::create_role(const Role& role) +{ + int32_t seqid = send_create_role(role); + return recv_create_role(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_create_role(const Role& role) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("create_role", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_role_pargs args; + args.role = &role; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_create_role(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_role") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_create_role_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_role failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::drop_role(const std::string& role_name) +{ + int32_t seqid = send_drop_role(role_name); + return recv_drop_role(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_drop_role(const std::string& role_name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("drop_role", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_role_pargs args; + args.role_name = &role_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_drop_role(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_role") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_drop_role_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_role failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_role_names(std::vector & _return) +{ + int32_t seqid = send_get_role_names(); + recv_get_role_names(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_role_names() +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_role_names", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_role_names_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_role_names(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_role_names") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_role_names_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_role_names failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option) +{ + int32_t seqid = send_grant_role(role_name, principal_name, principal_type, grantor, grantorType, grant_option); + return recv_grant_role(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("grant_role", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_grant_role_pargs args; + args.role_name = &role_name; + args.principal_name = &principal_name; + args.principal_type = &principal_type; + args.grantor = &grantor; + args.grantorType = &grantorType; + args.grant_option = &grant_option; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_grant_role(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("grant_role") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_grant_role_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_role failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type) +{ + int32_t seqid = send_revoke_role(role_name, principal_name, principal_type); + return recv_revoke_role(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("revoke_role", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_revoke_role_pargs args; + args.role_name = &role_name; + args.principal_name = &principal_name; + args.principal_type = &principal_type; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_revoke_role(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("revoke_role") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_revoke_role_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "revoke_role failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::list_roles(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type) +{ + int32_t seqid = send_list_roles(principal_name, principal_type); + recv_list_roles(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_list_roles(const std::string& principal_name, const PrincipalType::type principal_type) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("list_roles", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_list_roles_pargs args; + args.principal_name = &principal_name; + args.principal_type = &principal_type; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_list_roles(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("list_roles") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_list_roles_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "list_roles failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::grant_revoke_role(GrantRevokeRoleResponse& _return, const GrantRevokeRoleRequest& request) +{ + int32_t seqid = send_grant_revoke_role(request); + recv_grant_revoke_role(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_grant_revoke_role(const GrantRevokeRoleRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("grant_revoke_role", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_grant_revoke_role_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_grant_revoke_role(GrantRevokeRoleResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("grant_revoke_role") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_grant_revoke_role_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_revoke_role failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_principals_in_role(GetPrincipalsInRoleResponse& _return, const GetPrincipalsInRoleRequest& request) +{ + int32_t seqid = send_get_principals_in_role(request); + recv_get_principals_in_role(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_principals_in_role(const GetPrincipalsInRoleRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_principals_in_role", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_principals_in_role_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_principals_in_role(GetPrincipalsInRoleResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_principals_in_role") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_principals_in_role_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_principals_in_role failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return, const GetRoleGrantsForPrincipalRequest& request) +{ + int32_t seqid = send_get_role_grants_for_principal(request); + recv_get_role_grants_for_principal(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_role_grants_for_principal(const GetRoleGrantsForPrincipalRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_role_grants_for_principal", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_role_grants_for_principal_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_role_grants_for_principal") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_role_grants_for_principal_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_role_grants_for_principal failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_privilege_set(PrincipalPrivilegeSet& _return, const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names) +{ + int32_t seqid = send_get_privilege_set(hiveObject, user_name, group_names); + recv_get_privilege_set(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_privilege_set(const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_privilege_set", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_privilege_set_pargs args; + args.hiveObject = &hiveObject; + args.user_name = &user_name; + args.group_names = &group_names; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_privilege_set(PrincipalPrivilegeSet& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_privilege_set") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_privilege_set_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_privilege_set failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::list_privileges(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) +{ + int32_t seqid = send_list_privileges(principal_name, principal_type, hiveObject); + recv_list_privileges(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_list_privileges(const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("list_privileges", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_list_privileges_pargs args; + args.principal_name = &principal_name; + args.principal_type = &principal_type; + args.hiveObject = &hiveObject; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_list_privileges(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("list_privileges") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_list_privileges_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "list_privileges failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::grant_privileges(const PrivilegeBag& privileges) +{ + int32_t seqid = send_grant_privileges(privileges); + return recv_grant_privileges(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_grant_privileges(const PrivilegeBag& privileges) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("grant_privileges", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_grant_privileges_pargs args; + args.privileges = &privileges; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_grant_privileges(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("grant_privileges") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_grant_privileges_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_privileges failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::revoke_privileges(const PrivilegeBag& privileges) +{ + int32_t seqid = send_revoke_privileges(privileges); + return recv_revoke_privileges(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_revoke_privileges(const PrivilegeBag& privileges) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("revoke_privileges", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_revoke_privileges_pargs args; + args.privileges = &privileges; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_revoke_privileges(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("revoke_privileges") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_revoke_privileges_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "revoke_privileges failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const GrantRevokePrivilegeRequest& request) +{ + int32_t seqid = send_grant_revoke_privileges(request); + recv_grant_revoke_privileges(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_grant_revoke_privileges(const GrantRevokePrivilegeRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("grant_revoke_privileges", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_grant_revoke_privileges_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("grant_revoke_privileges") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_grant_revoke_privileges_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_revoke_privileges failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::set_ugi(std::vector & _return, const std::string& user_name, const std::vector & group_names) +{ + int32_t seqid = send_set_ugi(user_name, group_names); + recv_set_ugi(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_set_ugi(const std::string& user_name, const std::vector & group_names) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("set_ugi", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_set_ugi_pargs args; + args.user_name = &user_name; + args.group_names = &group_names; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_set_ugi(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("set_ugi") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_set_ugi_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "set_ugi failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name) +{ + int32_t seqid = send_get_delegation_token(token_owner, renewer_kerberos_principal_name); + recv_get_delegation_token(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_delegation_token(const std::string& token_owner, const std::string& renewer_kerberos_principal_name) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_delegation_token_pargs args; + args.token_owner = &token_owner; + args.renewer_kerberos_principal_name = &renewer_kerberos_principal_name; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_delegation_token(std::string& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_delegation_token") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_delegation_token_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_delegation_token failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +int64_t ThriftHiveMetastoreConcurrentClient::renew_delegation_token(const std::string& token_str_form) +{ + int32_t seqid = send_renew_delegation_token(token_str_form); + return recv_renew_delegation_token(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_renew_delegation_token(const std::string& token_str_form) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("renew_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_renew_delegation_token_pargs args; + args.token_str_form = &token_str_form; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +int64_t ThriftHiveMetastoreConcurrentClient::recv_renew_delegation_token(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("renew_delegation_token") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + int64_t _return; + ThriftHiveMetastore_renew_delegation_token_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "renew_delegation_token failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::cancel_delegation_token(const std::string& token_str_form) +{ + int32_t seqid = send_cancel_delegation_token(token_str_form); + recv_cancel_delegation_token(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_cancel_delegation_token(const std::string& token_str_form) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("cancel_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_cancel_delegation_token_pargs args; + args.token_str_form = &token_str_form; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_cancel_delegation_token(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("cancel_delegation_token") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_cancel_delegation_token_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::add_token(const std::string& token_identifier, const std::string& delegation_token) +{ + int32_t seqid = send_add_token(token_identifier, delegation_token); + return recv_add_token(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_add_token(const std::string& token_identifier, const std::string& delegation_token) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("add_token", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_token_pargs args; + args.token_identifier = &token_identifier; + args.delegation_token = &delegation_token; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_add_token(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_token") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_add_token_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_token failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::remove_token(const std::string& token_identifier) +{ + int32_t seqid = send_remove_token(token_identifier); + return recv_remove_token(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_remove_token(const std::string& token_identifier) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("remove_token", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_remove_token_pargs args; + args.token_identifier = &token_identifier; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_remove_token(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("remove_token") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_remove_token_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "remove_token failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_token(std::string& _return, const std::string& token_identifier) +{ + int32_t seqid = send_get_token(token_identifier); + recv_get_token(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_token(const std::string& token_identifier) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_token", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_token_pargs args; + args.token_identifier = &token_identifier; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_token(std::string& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_token") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_token_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_token failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_all_token_identifiers(std::vector & _return) +{ + int32_t seqid = send_get_all_token_identifiers(); + recv_get_all_token_identifiers(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_all_token_identifiers() +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_all_token_identifiers", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_all_token_identifiers_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_all_token_identifiers(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_all_token_identifiers") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_all_token_identifiers_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_token_identifiers failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +int32_t ThriftHiveMetastoreConcurrentClient::add_master_key(const std::string& key) +{ + int32_t seqid = send_add_master_key(key); + return recv_add_master_key(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_add_master_key(const std::string& key) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("add_master_key", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_master_key_pargs args; + args.key = &key; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +int32_t ThriftHiveMetastoreConcurrentClient::recv_add_master_key(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_master_key") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + int32_t _return; + ThriftHiveMetastore_add_master_key_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_master_key failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::update_master_key(const int32_t seq_number, const std::string& key) +{ + int32_t seqid = send_update_master_key(seq_number, key); + recv_update_master_key(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_update_master_key(const int32_t seq_number, const std::string& key) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("update_master_key", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_update_master_key_pargs args; + args.seq_number = &seq_number; + args.key = &key; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_update_master_key(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("update_master_key") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_update_master_key_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +bool ThriftHiveMetastoreConcurrentClient::remove_master_key(const int32_t key_seq) +{ + int32_t seqid = send_remove_master_key(key_seq); + return recv_remove_master_key(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_remove_master_key(const int32_t key_seq) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("remove_master_key", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_remove_master_key_pargs args; + args.key_seq = &key_seq; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +bool ThriftHiveMetastoreConcurrentClient::recv_remove_master_key(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("remove_master_key") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + bool _return; + ThriftHiveMetastore_remove_master_key_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + sentry.commit(); + return _return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "remove_master_key failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_master_keys(std::vector & _return) +{ + int32_t seqid = send_get_master_keys(); + recv_get_master_keys(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_master_keys() +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_master_keys", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_master_keys_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_master_keys(std::vector & _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_master_keys") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_master_keys_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_master_keys failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_open_txns(GetOpenTxnsResponse& _return) +{ + int32_t seqid = send_get_open_txns(); + recv_get_open_txns(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_open_txns() +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_open_txns", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_open_txns_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_open_txns(GetOpenTxnsResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_open_txns") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_open_txns_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_open_txns failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_open_txns_info(GetOpenTxnsInfoResponse& _return) +{ + int32_t seqid = send_get_open_txns_info(); + recv_get_open_txns_info(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_open_txns_info() +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_open_txns_info", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_open_txns_info_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_open_txns_info(GetOpenTxnsInfoResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_open_txns_info") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_open_txns_info_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_open_txns_info failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::open_txns(OpenTxnsResponse& _return, const OpenTxnRequest& rqst) +{ + int32_t seqid = send_open_txns(rqst); + recv_open_txns(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_open_txns(const OpenTxnRequest& rqst) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("open_txns", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_open_txns_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_open_txns(OpenTxnsResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("open_txns") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_open_txns_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "open_txns failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::abort_txn(const AbortTxnRequest& rqst) +{ + int32_t seqid = send_abort_txn(rqst); + recv_abort_txn(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_abort_txn(const AbortTxnRequest& rqst) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("abort_txn", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_abort_txn_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_abort_txn(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("abort_txn") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_abort_txn_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::abort_txns(const AbortTxnsRequest& rqst) +{ + int32_t seqid = send_abort_txns(rqst); + recv_abort_txns(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_abort_txns(const AbortTxnsRequest& rqst) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("abort_txns", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_abort_txns_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_abort_txns(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("abort_txns") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_abort_txns_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::commit_txn(const CommitTxnRequest& rqst) +{ + int32_t seqid = send_commit_txn(rqst); + recv_commit_txn(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_commit_txn(const CommitTxnRequest& rqst) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("commit_txn", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_commit_txn_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_commit_txn(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("commit_txn") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_commit_txn_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::lock(LockResponse& _return, const LockRequest& rqst) +{ + int32_t seqid = send_lock(rqst); + recv_lock(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_lock(const LockRequest& rqst) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("lock", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_lock_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_lock(LockResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("lock") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_lock_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "lock failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::check_lock(LockResponse& _return, const CheckLockRequest& rqst) +{ + int32_t seqid = send_check_lock(rqst); + recv_check_lock(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_check_lock(const CheckLockRequest& rqst) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("check_lock", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_check_lock_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_check_lock(LockResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("check_lock") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_check_lock_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "check_lock failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::unlock(const UnlockRequest& rqst) +{ + int32_t seqid = send_unlock(rqst); + recv_unlock(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_unlock(const UnlockRequest& rqst) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("unlock", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_unlock_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_unlock(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("unlock") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_unlock_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::show_locks(ShowLocksResponse& _return, const ShowLocksRequest& rqst) +{ + int32_t seqid = send_show_locks(rqst); + recv_show_locks(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_show_locks(const ShowLocksRequest& rqst) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("show_locks", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_show_locks_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_show_locks(ShowLocksResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("show_locks") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_show_locks_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "show_locks failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::heartbeat(const HeartbeatRequest& ids) +{ + int32_t seqid = send_heartbeat(ids); + recv_heartbeat(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_heartbeat(const HeartbeatRequest& ids) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("heartbeat", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_heartbeat_pargs args; + args.ids = &ids; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_heartbeat(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("heartbeat") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_heartbeat_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::heartbeat_txn_range(HeartbeatTxnRangeResponse& _return, const HeartbeatTxnRangeRequest& txns) +{ + int32_t seqid = send_heartbeat_txn_range(txns); + recv_heartbeat_txn_range(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_heartbeat_txn_range(const HeartbeatTxnRangeRequest& txns) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("heartbeat_txn_range", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_heartbeat_txn_range_pargs args; + args.txns = &txns; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_heartbeat_txn_range(HeartbeatTxnRangeResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("heartbeat_txn_range") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_heartbeat_txn_range_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "heartbeat_txn_range failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::compact(const CompactionRequest& rqst) +{ + int32_t seqid = send_compact(rqst); + recv_compact(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_compact(const CompactionRequest& rqst) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("compact", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_compact_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_compact(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("compact") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_compact_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::compact2(CompactionResponse& _return, const CompactionRequest& rqst) +{ + int32_t seqid = send_compact2(rqst); + recv_compact2(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_compact2(const CompactionRequest& rqst) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("compact2", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_compact2_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_compact2(CompactionResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("compact2") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_compact2_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "compact2 failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst) +{ + int32_t seqid = send_show_compact(rqst); + recv_show_compact(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_show_compact(const ShowCompactRequest& rqst) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("show_compact", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_show_compact_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_show_compact(ShowCompactResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("show_compact") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_show_compact_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "show_compact failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::add_dynamic_partitions(const AddDynamicPartitions& rqst) +{ + int32_t seqid = send_add_dynamic_partitions(rqst); + recv_add_dynamic_partitions(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_add_dynamic_partitions(const AddDynamicPartitions& rqst) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("add_dynamic_partitions", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_add_dynamic_partitions_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_add_dynamic_partitions(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("add_dynamic_partitions") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_add_dynamic_partitions_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst) +{ + int32_t seqid = send_get_next_notification(rqst); + recv_get_next_notification(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_next_notification(const NotificationEventRequest& rqst) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_next_notification", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_next_notification_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_next_notification(NotificationEventResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_next_notification") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_next_notification_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_next_notification failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_current_notificationEventId(CurrentNotificationEventId& _return) +{ + int32_t seqid = send_get_current_notificationEventId(); + recv_get_current_notificationEventId(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_current_notificationEventId() +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_current_notificationEventId", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_current_notificationEventId_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_current_notificationEventId(CurrentNotificationEventId& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_current_notificationEventId") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_current_notificationEventId_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_current_notificationEventId failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_notification_events_count(NotificationEventsCountResponse& _return, const NotificationEventsCountRequest& rqst) +{ + int32_t seqid = send_get_notification_events_count(rqst); + recv_get_notification_events_count(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_notification_events_count(const NotificationEventsCountRequest& rqst) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_notification_events_count", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_notification_events_count_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_notification_events_count(NotificationEventsCountResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_notification_events_count") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_notification_events_count_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_notification_events_count failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::fire_listener_event(FireEventResponse& _return, const FireEventRequest& rqst) +{ + int32_t seqid = send_fire_listener_event(rqst); + recv_fire_listener_event(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_fire_listener_event(const FireEventRequest& rqst) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("fire_listener_event", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_fire_listener_event_pargs args; + args.rqst = &rqst; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_fire_listener_event(FireEventResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("fire_listener_event") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_fire_listener_event_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "fire_listener_event failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::flushCache() +{ + int32_t seqid = send_flushCache(); + recv_flushCache(seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_flushCache() +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("flushCache", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_flushCache_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_flushCache(const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("flushCache") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_flushCache_presult result; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + sentry.commit(); + return; + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request) +{ + int32_t seqid = send_cm_recycle(request); + recv_cm_recycle(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_cm_recycle(const CmRecycleRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("cm_recycle", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_cm_recycle_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_cm_recycle(CmRecycleResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("cm_recycle") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_cm_recycle_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "cm_recycle failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req) +{ + int32_t seqid = send_get_file_metadata_by_expr(req); + recv_get_file_metadata_by_expr(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_file_metadata_by_expr(const GetFileMetadataByExprRequest& req) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_file_metadata_by_expr", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_file_metadata_by_expr_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_file_metadata_by_expr") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_file_metadata_by_expr_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_file_metadata_by_expr failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req) +{ + int32_t seqid = send_get_file_metadata(req); + recv_get_file_metadata(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_file_metadata(const GetFileMetadataRequest& req) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_file_metadata", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_file_metadata_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_file_metadata(GetFileMetadataResult& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_file_metadata") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_file_metadata_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_file_metadata failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req) +{ + int32_t seqid = send_put_file_metadata(req); + recv_put_file_metadata(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_put_file_metadata(const PutFileMetadataRequest& req) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("put_file_metadata", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_put_file_metadata_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_put_file_metadata(PutFileMetadataResult& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("put_file_metadata") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_put_file_metadata_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "put_file_metadata failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req) +{ + int32_t seqid = send_clear_file_metadata(req); + recv_clear_file_metadata(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_clear_file_metadata(const ClearFileMetadataRequest& req) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("clear_file_metadata", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_clear_file_metadata_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_clear_file_metadata(ClearFileMetadataResult& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("clear_file_metadata") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_clear_file_metadata_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "clear_file_metadata failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req) +{ + int32_t seqid = send_cache_file_metadata(req); + recv_cache_file_metadata(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_cache_file_metadata(const CacheFileMetadataRequest& req) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("cache_file_metadata", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_cache_file_metadata_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_cache_file_metadata(CacheFileMetadataResult& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("cache_file_metadata") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_cache_file_metadata_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "cache_file_metadata failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_metastore_db_uuid(std::string& _return) +{ + int32_t seqid = send_get_metastore_db_uuid(); + recv_get_metastore_db_uuid(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_metastore_db_uuid() +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_metastore_db_uuid", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_metastore_db_uuid_pargs args; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_metastore_db_uuid(std::string& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_metastore_db_uuid") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_metastore_db_uuid_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_metastore_db_uuid failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::create_resource_plan(WMCreateResourcePlanResponse& _return, const WMCreateResourcePlanRequest& request) +{ + int32_t seqid = send_create_resource_plan(request); + recv_create_resource_plan(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_create_resource_plan(const WMCreateResourcePlanRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("create_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_resource_plan_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_create_resource_plan(WMCreateResourcePlanResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_resource_plan") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_create_resource_plan_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_resource_plan failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_resource_plan(WMGetResourcePlanResponse& _return, const WMGetResourcePlanRequest& request) +{ + int32_t seqid = send_get_resource_plan(request); + recv_get_resource_plan(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_resource_plan(const WMGetResourcePlanRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_resource_plan_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_resource_plan(WMGetResourcePlanResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_resource_plan") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_resource_plan_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_resource_plan failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_active_resource_plan(WMGetActiveResourcePlanResponse& _return, const WMGetActiveResourcePlanRequest& request) +{ + int32_t seqid = send_get_active_resource_plan(request); + recv_get_active_resource_plan(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_active_resource_plan(const WMGetActiveResourcePlanRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_active_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_active_resource_plan_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_active_resource_plan(WMGetActiveResourcePlanResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_active_resource_plan") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_active_resource_plan_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_active_resource_plan failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_all_resource_plans(WMGetAllResourcePlanResponse& _return, const WMGetAllResourcePlanRequest& request) +{ + int32_t seqid = send_get_all_resource_plans(request); + recv_get_all_resource_plans(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_all_resource_plans(const WMGetAllResourcePlanRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_all_resource_plans", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_all_resource_plans_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_all_resource_plans(WMGetAllResourcePlanResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_all_resource_plans") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_all_resource_plans_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_resource_plans failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::alter_resource_plan(WMAlterResourcePlanResponse& _return, const WMAlterResourcePlanRequest& request) +{ + int32_t seqid = send_alter_resource_plan(request); + recv_alter_resource_plan(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_alter_resource_plan(const WMAlterResourcePlanRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("alter_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_resource_plan_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_alter_resource_plan(WMAlterResourcePlanResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_resource_plan") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_alter_resource_plan_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "alter_resource_plan failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::validate_resource_plan(WMValidateResourcePlanResponse& _return, const WMValidateResourcePlanRequest& request) +{ + int32_t seqid = send_validate_resource_plan(request); + recv_validate_resource_plan(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_validate_resource_plan(const WMValidateResourcePlanRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("validate_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_validate_resource_plan_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_validate_resource_plan(WMValidateResourcePlanResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("validate_resource_plan") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_validate_resource_plan_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "validate_resource_plan failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::drop_resource_plan(WMDropResourcePlanResponse& _return, const WMDropResourcePlanRequest& request) +{ + int32_t seqid = send_drop_resource_plan(request); + recv_drop_resource_plan(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_drop_resource_plan(const WMDropResourcePlanRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("drop_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_resource_plan_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_drop_resource_plan(WMDropResourcePlanResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_resource_plan") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_drop_resource_plan_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_resource_plan failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::create_wm_trigger(WMCreateTriggerResponse& _return, const WMCreateTriggerRequest& request) +{ + int32_t seqid = send_create_wm_trigger(request); + recv_create_wm_trigger(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_create_wm_trigger(const WMCreateTriggerRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("create_wm_trigger", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_create_wm_trigger_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_create_wm_trigger(WMCreateTriggerResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("create_wm_trigger") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_create_wm_trigger_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + if (result.__isset.o4) { + sentry.commit(); + throw result.o4; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_wm_trigger failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::alter_wm_trigger(WMAlterTriggerResponse& _return, const WMAlterTriggerRequest& request) +{ + int32_t seqid = send_alter_wm_trigger(request); + recv_alter_wm_trigger(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_alter_wm_trigger(const WMAlterTriggerRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("alter_wm_trigger", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_alter_wm_trigger_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_alter_wm_trigger(WMAlterTriggerResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("alter_wm_trigger") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_alter_wm_trigger_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "alter_wm_trigger failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::drop_wm_trigger(WMDropTriggerResponse& _return, const WMDropTriggerRequest& request) +{ + int32_t seqid = send_drop_wm_trigger(request); + recv_drop_wm_trigger(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_drop_wm_trigger(const WMDropTriggerRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("drop_wm_trigger", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_drop_wm_trigger_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_drop_wm_trigger(WMDropTriggerResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("drop_wm_trigger") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_drop_wm_trigger_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + if (result.__isset.o3) { + sentry.commit(); + throw result.o3; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_wm_trigger failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +void ThriftHiveMetastoreConcurrentClient::get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return, const WMGetTriggersForResourePlanRequest& request) +{ + int32_t seqid = send_get_triggers_for_resourceplan(request); + recv_get_triggers_for_resourceplan(_return, seqid); +} + +int32_t ThriftHiveMetastoreConcurrentClient::send_get_triggers_for_resourceplan(const WMGetTriggersForResourePlanRequest& request) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("get_triggers_for_resourceplan", ::apache::thrift::protocol::T_CALL, cseqid); + + ThriftHiveMetastore_get_triggers_for_resourceplan_pargs args; + args.request = &request; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void ThriftHiveMetastoreConcurrentClient::recv_get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("get_triggers_for_resourceplan") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + ThriftHiveMetastore_get_triggers_for_resourceplan_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + if (result.__isset.o1) { + sentry.commit(); + throw result.o1; + } + if (result.__isset.o2) { + sentry.commit(); + throw result.o2; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_triggers_for_resourceplan failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + +}}} // namespace + diff --git a/contrib/hive-metastore/ThriftHiveMetastore.h b/contrib/hive-metastore/ThriftHiveMetastore.h new file mode 100644 index 00000000000..e86933e5011 --- /dev/null +++ b/contrib/hive-metastore/ThriftHiveMetastore.h @@ -0,0 +1,25246 @@ +/** + * Autogenerated by Thrift Compiler () + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +#ifndef ThriftHiveMetastore_H +#define ThriftHiveMetastore_H + +#include +#include +#include "hive_metastore_types.h" + +namespace Apache { namespace Hadoop { namespace Hive { + +#ifdef _MSC_VER + #pragma warning( push ) + #pragma warning (disable : 4250 ) //inheriting methods via dominance +#endif + +class ThriftHiveMetastoreIf { + public: + virtual ~ThriftHiveMetastoreIf() {} + virtual void getMetaConf(std::string& _return, const std::string& key) = 0; + virtual void setMetaConf(const std::string& key, const std::string& value) = 0; + virtual void create_database(const Database& database) = 0; + virtual void get_database(Database& _return, const std::string& name) = 0; + virtual void drop_database(const std::string& name, const bool deleteData, const bool cascade) = 0; + virtual void get_databases(std::vector & _return, const std::string& pattern) = 0; + virtual void get_all_databases(std::vector & _return) = 0; + virtual void alter_database(const std::string& dbname, const Database& db) = 0; + virtual void get_type(Type& _return, const std::string& name) = 0; + virtual bool create_type(const Type& type) = 0; + virtual bool drop_type(const std::string& type) = 0; + virtual void get_type_all(std::map & _return, const std::string& name) = 0; + virtual void get_fields(std::vector & _return, const std::string& db_name, const std::string& table_name) = 0; + virtual void get_fields_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) = 0; + virtual void get_schema(std::vector & _return, const std::string& db_name, const std::string& table_name) = 0; + virtual void get_schema_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) = 0; + virtual void create_table(const Table& tbl) = 0; + virtual void create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) = 0; + virtual void create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints) = 0; + virtual void drop_constraint(const DropConstraintRequest& req) = 0; + virtual void add_primary_key(const AddPrimaryKeyRequest& req) = 0; + virtual void add_foreign_key(const AddForeignKeyRequest& req) = 0; + virtual void add_unique_constraint(const AddUniqueConstraintRequest& req) = 0; + virtual void add_not_null_constraint(const AddNotNullConstraintRequest& req) = 0; + virtual void drop_table(const std::string& dbname, const std::string& name, const bool deleteData) = 0; + virtual void drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) = 0; + virtual void truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames) = 0; + virtual void get_tables(std::vector & _return, const std::string& db_name, const std::string& pattern) = 0; + virtual void get_tables_by_type(std::vector & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType) = 0; + virtual void get_table_meta(std::vector & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types) = 0; + virtual void get_all_tables(std::vector & _return, const std::string& db_name) = 0; + virtual void get_table(Table& _return, const std::string& dbname, const std::string& tbl_name) = 0; + virtual void get_table_objects_by_name(std::vector
& _return, const std::string& dbname, const std::vector & tbl_names) = 0; + virtual void get_table_req(GetTableResult& _return, const GetTableRequest& req) = 0; + virtual void get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req) = 0; + virtual void get_table_names_by_filter(std::vector & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables) = 0; + virtual void alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) = 0; + virtual void alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) = 0; + virtual void alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade) = 0; + virtual void add_partition(Partition& _return, const Partition& new_part) = 0; + virtual void add_partition_with_environment_context(Partition& _return, const Partition& new_part, const EnvironmentContext& environment_context) = 0; + virtual int32_t add_partitions(const std::vector & new_parts) = 0; + virtual int32_t add_partitions_pspec(const std::vector & new_parts) = 0; + virtual void append_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) = 0; + virtual void add_partitions_req(AddPartitionsResult& _return, const AddPartitionsRequest& request) = 0; + virtual void append_partition_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context) = 0; + virtual void append_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) = 0; + virtual void append_partition_by_name_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context) = 0; + virtual bool drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) = 0; + virtual bool drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context) = 0; + virtual bool drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData) = 0; + virtual bool drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context) = 0; + virtual void drop_partitions_req(DropPartitionsResult& _return, const DropPartitionsRequest& req) = 0; + virtual void get_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) = 0; + virtual void exchange_partition(Partition& _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) = 0; + virtual void exchange_partitions(std::vector & _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) = 0; + virtual void get_partition_with_auth(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names) = 0; + virtual void get_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) = 0; + virtual void get_partitions(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) = 0; + virtual void get_partitions_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) = 0; + virtual void get_partitions_pspec(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int32_t max_parts) = 0; + virtual void get_partition_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) = 0; + virtual void get_partition_values(PartitionValuesResponse& _return, const PartitionValuesRequest& request) = 0; + virtual void get_partitions_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) = 0; + virtual void get_partitions_ps_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) = 0; + virtual void get_partition_names_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) = 0; + virtual void get_partitions_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts) = 0; + virtual void get_part_specs_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts) = 0; + virtual void get_partitions_by_expr(PartitionsByExprResult& _return, const PartitionsByExprRequest& req) = 0; + virtual int32_t get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter) = 0; + virtual void get_partitions_by_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & names) = 0; + virtual void alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part) = 0; + virtual void alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) = 0; + virtual void alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context) = 0; + virtual void alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context) = 0; + virtual void rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) = 0; + virtual bool partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) = 0; + virtual void get_config_value(std::string& _return, const std::string& name, const std::string& defaultValue) = 0; + virtual void partition_name_to_vals(std::vector & _return, const std::string& part_name) = 0; + virtual void partition_name_to_spec(std::map & _return, const std::string& part_name) = 0; + virtual void markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) = 0; + virtual bool isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) = 0; + virtual void add_index(Index& _return, const Index& new_index, const Table& index_table) = 0; + virtual void alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) = 0; + virtual bool drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData) = 0; + virtual void get_index_by_name(Index& _return, const std::string& db_name, const std::string& tbl_name, const std::string& index_name) = 0; + virtual void get_indexes(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) = 0; + virtual void get_index_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) = 0; + virtual void get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request) = 0; + virtual void get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request) = 0; + virtual void get_unique_constraints(UniqueConstraintsResponse& _return, const UniqueConstraintsRequest& request) = 0; + virtual void get_not_null_constraints(NotNullConstraintsResponse& _return, const NotNullConstraintsRequest& request) = 0; + virtual bool update_table_column_statistics(const ColumnStatistics& stats_obj) = 0; + virtual bool update_partition_column_statistics(const ColumnStatistics& stats_obj) = 0; + virtual void get_table_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& col_name) = 0; + virtual void get_partition_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) = 0; + virtual void get_table_statistics_req(TableStatsResult& _return, const TableStatsRequest& request) = 0; + virtual void get_partitions_statistics_req(PartitionsStatsResult& _return, const PartitionsStatsRequest& request) = 0; + virtual void get_aggr_stats_for(AggrStats& _return, const PartitionsStatsRequest& request) = 0; + virtual bool set_aggr_stats_for(const SetPartitionsStatsRequest& request) = 0; + virtual bool delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) = 0; + virtual bool delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) = 0; + virtual void create_function(const Function& func) = 0; + virtual void drop_function(const std::string& dbName, const std::string& funcName) = 0; + virtual void alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc) = 0; + virtual void get_functions(std::vector & _return, const std::string& dbName, const std::string& pattern) = 0; + virtual void get_function(Function& _return, const std::string& dbName, const std::string& funcName) = 0; + virtual void get_all_functions(GetAllFunctionsResponse& _return) = 0; + virtual bool create_role(const Role& role) = 0; + virtual bool drop_role(const std::string& role_name) = 0; + virtual void get_role_names(std::vector & _return) = 0; + virtual bool grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option) = 0; + virtual bool revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type) = 0; + virtual void list_roles(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type) = 0; + virtual void grant_revoke_role(GrantRevokeRoleResponse& _return, const GrantRevokeRoleRequest& request) = 0; + virtual void get_principals_in_role(GetPrincipalsInRoleResponse& _return, const GetPrincipalsInRoleRequest& request) = 0; + virtual void get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return, const GetRoleGrantsForPrincipalRequest& request) = 0; + virtual void get_privilege_set(PrincipalPrivilegeSet& _return, const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names) = 0; + virtual void list_privileges(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) = 0; + virtual bool grant_privileges(const PrivilegeBag& privileges) = 0; + virtual bool revoke_privileges(const PrivilegeBag& privileges) = 0; + virtual void grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const GrantRevokePrivilegeRequest& request) = 0; + virtual void set_ugi(std::vector & _return, const std::string& user_name, const std::vector & group_names) = 0; + virtual void get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name) = 0; + virtual int64_t renew_delegation_token(const std::string& token_str_form) = 0; + virtual void cancel_delegation_token(const std::string& token_str_form) = 0; + virtual bool add_token(const std::string& token_identifier, const std::string& delegation_token) = 0; + virtual bool remove_token(const std::string& token_identifier) = 0; + virtual void get_token(std::string& _return, const std::string& token_identifier) = 0; + virtual void get_all_token_identifiers(std::vector & _return) = 0; + virtual int32_t add_master_key(const std::string& key) = 0; + virtual void update_master_key(const int32_t seq_number, const std::string& key) = 0; + virtual bool remove_master_key(const int32_t key_seq) = 0; + virtual void get_master_keys(std::vector & _return) = 0; + virtual void get_open_txns(GetOpenTxnsResponse& _return) = 0; + virtual void get_open_txns_info(GetOpenTxnsInfoResponse& _return) = 0; + virtual void open_txns(OpenTxnsResponse& _return, const OpenTxnRequest& rqst) = 0; + virtual void abort_txn(const AbortTxnRequest& rqst) = 0; + virtual void abort_txns(const AbortTxnsRequest& rqst) = 0; + virtual void commit_txn(const CommitTxnRequest& rqst) = 0; + virtual void lock(LockResponse& _return, const LockRequest& rqst) = 0; + virtual void check_lock(LockResponse& _return, const CheckLockRequest& rqst) = 0; + virtual void unlock(const UnlockRequest& rqst) = 0; + virtual void show_locks(ShowLocksResponse& _return, const ShowLocksRequest& rqst) = 0; + virtual void heartbeat(const HeartbeatRequest& ids) = 0; + virtual void heartbeat_txn_range(HeartbeatTxnRangeResponse& _return, const HeartbeatTxnRangeRequest& txns) = 0; + virtual void compact(const CompactionRequest& rqst) = 0; + virtual void compact2(CompactionResponse& _return, const CompactionRequest& rqst) = 0; + virtual void show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst) = 0; + virtual void add_dynamic_partitions(const AddDynamicPartitions& rqst) = 0; + virtual void get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst) = 0; + virtual void get_current_notificationEventId(CurrentNotificationEventId& _return) = 0; + virtual void get_notification_events_count(NotificationEventsCountResponse& _return, const NotificationEventsCountRequest& rqst) = 0; + virtual void fire_listener_event(FireEventResponse& _return, const FireEventRequest& rqst) = 0; + virtual void flushCache() = 0; + virtual void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request) = 0; + virtual void get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req) = 0; + virtual void get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req) = 0; + virtual void put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req) = 0; + virtual void clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req) = 0; + virtual void cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req) = 0; + virtual void get_metastore_db_uuid(std::string& _return) = 0; + virtual void create_resource_plan(WMCreateResourcePlanResponse& _return, const WMCreateResourcePlanRequest& request) = 0; + virtual void get_resource_plan(WMGetResourcePlanResponse& _return, const WMGetResourcePlanRequest& request) = 0; + virtual void get_active_resource_plan(WMGetActiveResourcePlanResponse& _return, const WMGetActiveResourcePlanRequest& request) = 0; + virtual void get_all_resource_plans(WMGetAllResourcePlanResponse& _return, const WMGetAllResourcePlanRequest& request) = 0; + virtual void alter_resource_plan(WMAlterResourcePlanResponse& _return, const WMAlterResourcePlanRequest& request) = 0; + virtual void validate_resource_plan(WMValidateResourcePlanResponse& _return, const WMValidateResourcePlanRequest& request) = 0; + virtual void drop_resource_plan(WMDropResourcePlanResponse& _return, const WMDropResourcePlanRequest& request) = 0; + virtual void create_wm_trigger(WMCreateTriggerResponse& _return, const WMCreateTriggerRequest& request) = 0; + virtual void alter_wm_trigger(WMAlterTriggerResponse& _return, const WMAlterTriggerRequest& request) = 0; + virtual void drop_wm_trigger(WMDropTriggerResponse& _return, const WMDropTriggerRequest& request) = 0; + virtual void get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return, const WMGetTriggersForResourePlanRequest& request) = 0; +}; + +class ThriftHiveMetastoreIfFactory { + public: + typedef ThriftHiveMetastoreIf Handler; + + virtual ~ThriftHiveMetastoreIfFactory() {} + + virtual ThriftHiveMetastoreIf* getHandler(const ::apache::thrift::TConnectionInfo& connInfo) = 0; + virtual void releaseHandler(ThriftHiveMetastoreIf* /* handler */) = 0; +}; + +class ThriftHiveMetastoreIfSingletonFactory : virtual public ThriftHiveMetastoreIfFactory { + public: + ThriftHiveMetastoreIfSingletonFactory(const ::std::shared_ptr& iface) : iface_(iface) {} + virtual ~ThriftHiveMetastoreIfSingletonFactory() {} + + virtual ThriftHiveMetastoreIf* getHandler(const ::apache::thrift::TConnectionInfo&) { + return iface_.get(); + } + virtual void releaseHandler(ThriftHiveMetastoreIf* /* handler */) {} + + protected: + ::std::shared_ptr iface_; +}; + +class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf { + public: + virtual ~ThriftHiveMetastoreNull() {} + void getMetaConf(std::string& /* _return */, const std::string& /* key */) { + return; + } + void setMetaConf(const std::string& /* key */, const std::string& /* value */) { + return; + } + void create_database(const Database& /* database */) { + return; + } + void get_database(Database& /* _return */, const std::string& /* name */) { + return; + } + void drop_database(const std::string& /* name */, const bool /* deleteData */, const bool /* cascade */) { + return; + } + void get_databases(std::vector & /* _return */, const std::string& /* pattern */) { + return; + } + void get_all_databases(std::vector & /* _return */) { + return; + } + void alter_database(const std::string& /* dbname */, const Database& /* db */) { + return; + } + void get_type(Type& /* _return */, const std::string& /* name */) { + return; + } + bool create_type(const Type& /* type */) { + bool _return = false; + return _return; + } + bool drop_type(const std::string& /* type */) { + bool _return = false; + return _return; + } + void get_type_all(std::map & /* _return */, const std::string& /* name */) { + return; + } + void get_fields(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* table_name */) { + return; + } + void get_fields_with_environment_context(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* table_name */, const EnvironmentContext& /* environment_context */) { + return; + } + void get_schema(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* table_name */) { + return; + } + void get_schema_with_environment_context(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* table_name */, const EnvironmentContext& /* environment_context */) { + return; + } + void create_table(const Table& /* tbl */) { + return; + } + void create_table_with_environment_context(const Table& /* tbl */, const EnvironmentContext& /* environment_context */) { + return; + } + void create_table_with_constraints(const Table& /* tbl */, const std::vector & /* primaryKeys */, const std::vector & /* foreignKeys */, const std::vector & /* uniqueConstraints */, const std::vector & /* notNullConstraints */) { + return; + } + void drop_constraint(const DropConstraintRequest& /* req */) { + return; + } + void add_primary_key(const AddPrimaryKeyRequest& /* req */) { + return; + } + void add_foreign_key(const AddForeignKeyRequest& /* req */) { + return; + } + void add_unique_constraint(const AddUniqueConstraintRequest& /* req */) { + return; + } + void add_not_null_constraint(const AddNotNullConstraintRequest& /* req */) { + return; + } + void drop_table(const std::string& /* dbname */, const std::string& /* name */, const bool /* deleteData */) { + return; + } + void drop_table_with_environment_context(const std::string& /* dbname */, const std::string& /* name */, const bool /* deleteData */, const EnvironmentContext& /* environment_context */) { + return; + } + void truncate_table(const std::string& /* dbName */, const std::string& /* tableName */, const std::vector & /* partNames */) { + return; + } + void get_tables(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* pattern */) { + return; + } + void get_tables_by_type(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* pattern */, const std::string& /* tableType */) { + return; + } + void get_table_meta(std::vector & /* _return */, const std::string& /* db_patterns */, const std::string& /* tbl_patterns */, const std::vector & /* tbl_types */) { + return; + } + void get_all_tables(std::vector & /* _return */, const std::string& /* db_name */) { + return; + } + void get_table(Table& /* _return */, const std::string& /* dbname */, const std::string& /* tbl_name */) { + return; + } + void get_table_objects_by_name(std::vector
& /* _return */, const std::string& /* dbname */, const std::vector & /* tbl_names */) { + return; + } + void get_table_req(GetTableResult& /* _return */, const GetTableRequest& /* req */) { + return; + } + void get_table_objects_by_name_req(GetTablesResult& /* _return */, const GetTablesRequest& /* req */) { + return; + } + void get_table_names_by_filter(std::vector & /* _return */, const std::string& /* dbname */, const std::string& /* filter */, const int16_t /* max_tables */) { + return; + } + void alter_table(const std::string& /* dbname */, const std::string& /* tbl_name */, const Table& /* new_tbl */) { + return; + } + void alter_table_with_environment_context(const std::string& /* dbname */, const std::string& /* tbl_name */, const Table& /* new_tbl */, const EnvironmentContext& /* environment_context */) { + return; + } + void alter_table_with_cascade(const std::string& /* dbname */, const std::string& /* tbl_name */, const Table& /* new_tbl */, const bool /* cascade */) { + return; + } + void add_partition(Partition& /* _return */, const Partition& /* new_part */) { + return; + } + void add_partition_with_environment_context(Partition& /* _return */, const Partition& /* new_part */, const EnvironmentContext& /* environment_context */) { + return; + } + int32_t add_partitions(const std::vector & /* new_parts */) { + int32_t _return = 0; + return _return; + } + int32_t add_partitions_pspec(const std::vector & /* new_parts */) { + int32_t _return = 0; + return _return; + } + void append_partition(Partition& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */) { + return; + } + void add_partitions_req(AddPartitionsResult& /* _return */, const AddPartitionsRequest& /* request */) { + return; + } + void append_partition_with_environment_context(Partition& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */, const EnvironmentContext& /* environment_context */) { + return; + } + void append_partition_by_name(Partition& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* part_name */) { + return; + } + void append_partition_by_name_with_environment_context(Partition& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* part_name */, const EnvironmentContext& /* environment_context */) { + return; + } + bool drop_partition(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */, const bool /* deleteData */) { + bool _return = false; + return _return; + } + bool drop_partition_with_environment_context(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */, const bool /* deleteData */, const EnvironmentContext& /* environment_context */) { + bool _return = false; + return _return; + } + bool drop_partition_by_name(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* part_name */, const bool /* deleteData */) { + bool _return = false; + return _return; + } + bool drop_partition_by_name_with_environment_context(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* part_name */, const bool /* deleteData */, const EnvironmentContext& /* environment_context */) { + bool _return = false; + return _return; + } + void drop_partitions_req(DropPartitionsResult& /* _return */, const DropPartitionsRequest& /* req */) { + return; + } + void get_partition(Partition& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */) { + return; + } + void exchange_partition(Partition& /* _return */, const std::map & /* partitionSpecs */, const std::string& /* source_db */, const std::string& /* source_table_name */, const std::string& /* dest_db */, const std::string& /* dest_table_name */) { + return; + } + void exchange_partitions(std::vector & /* _return */, const std::map & /* partitionSpecs */, const std::string& /* source_db */, const std::string& /* source_table_name */, const std::string& /* dest_db */, const std::string& /* dest_table_name */) { + return; + } + void get_partition_with_auth(Partition& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */, const std::string& /* user_name */, const std::vector & /* group_names */) { + return; + } + void get_partition_by_name(Partition& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* part_name */) { + return; + } + void get_partitions(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const int16_t /* max_parts */) { + return; + } + void get_partitions_with_auth(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const int16_t /* max_parts */, const std::string& /* user_name */, const std::vector & /* group_names */) { + return; + } + void get_partitions_pspec(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const int32_t /* max_parts */) { + return; + } + void get_partition_names(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const int16_t /* max_parts */) { + return; + } + void get_partition_values(PartitionValuesResponse& /* _return */, const PartitionValuesRequest& /* request */) { + return; + } + void get_partitions_ps(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */, const int16_t /* max_parts */) { + return; + } + void get_partitions_ps_with_auth(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */, const int16_t /* max_parts */, const std::string& /* user_name */, const std::vector & /* group_names */) { + return; + } + void get_partition_names_ps(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */, const int16_t /* max_parts */) { + return; + } + void get_partitions_by_filter(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* filter */, const int16_t /* max_parts */) { + return; + } + void get_part_specs_by_filter(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* filter */, const int32_t /* max_parts */) { + return; + } + void get_partitions_by_expr(PartitionsByExprResult& /* _return */, const PartitionsByExprRequest& /* req */) { + return; + } + int32_t get_num_partitions_by_filter(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* filter */) { + int32_t _return = 0; + return _return; + } + void get_partitions_by_names(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* names */) { + return; + } + void alter_partition(const std::string& /* db_name */, const std::string& /* tbl_name */, const Partition& /* new_part */) { + return; + } + void alter_partitions(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* new_parts */) { + return; + } + void alter_partitions_with_environment_context(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* new_parts */, const EnvironmentContext& /* environment_context */) { + return; + } + void alter_partition_with_environment_context(const std::string& /* db_name */, const std::string& /* tbl_name */, const Partition& /* new_part */, const EnvironmentContext& /* environment_context */) { + return; + } + void rename_partition(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */, const Partition& /* new_part */) { + return; + } + bool partition_name_has_valid_characters(const std::vector & /* part_vals */, const bool /* throw_exception */) { + bool _return = false; + return _return; + } + void get_config_value(std::string& /* _return */, const std::string& /* name */, const std::string& /* defaultValue */) { + return; + } + void partition_name_to_vals(std::vector & /* _return */, const std::string& /* part_name */) { + return; + } + void partition_name_to_spec(std::map & /* _return */, const std::string& /* part_name */) { + return; + } + void markPartitionForEvent(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::map & /* part_vals */, const PartitionEventType::type /* eventType */) { + return; + } + bool isPartitionMarkedForEvent(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::map & /* part_vals */, const PartitionEventType::type /* eventType */) { + bool _return = false; + return _return; + } + void add_index(Index& /* _return */, const Index& /* new_index */, const Table& /* index_table */) { + return; + } + void alter_index(const std::string& /* dbname */, const std::string& /* base_tbl_name */, const std::string& /* idx_name */, const Index& /* new_idx */) { + return; + } + bool drop_index_by_name(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* index_name */, const bool /* deleteData */) { + bool _return = false; + return _return; + } + void get_index_by_name(Index& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* index_name */) { + return; + } + void get_indexes(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const int16_t /* max_indexes */) { + return; + } + void get_index_names(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const int16_t /* max_indexes */) { + return; + } + void get_primary_keys(PrimaryKeysResponse& /* _return */, const PrimaryKeysRequest& /* request */) { + return; + } + void get_foreign_keys(ForeignKeysResponse& /* _return */, const ForeignKeysRequest& /* request */) { + return; + } + void get_unique_constraints(UniqueConstraintsResponse& /* _return */, const UniqueConstraintsRequest& /* request */) { + return; + } + void get_not_null_constraints(NotNullConstraintsResponse& /* _return */, const NotNullConstraintsRequest& /* request */) { + return; + } + bool update_table_column_statistics(const ColumnStatistics& /* stats_obj */) { + bool _return = false; + return _return; + } + bool update_partition_column_statistics(const ColumnStatistics& /* stats_obj */) { + bool _return = false; + return _return; + } + void get_table_column_statistics(ColumnStatistics& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* col_name */) { + return; + } + void get_partition_column_statistics(ColumnStatistics& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* part_name */, const std::string& /* col_name */) { + return; + } + void get_table_statistics_req(TableStatsResult& /* _return */, const TableStatsRequest& /* request */) { + return; + } + void get_partitions_statistics_req(PartitionsStatsResult& /* _return */, const PartitionsStatsRequest& /* request */) { + return; + } + void get_aggr_stats_for(AggrStats& /* _return */, const PartitionsStatsRequest& /* request */) { + return; + } + bool set_aggr_stats_for(const SetPartitionsStatsRequest& /* request */) { + bool _return = false; + return _return; + } + bool delete_partition_column_statistics(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* part_name */, const std::string& /* col_name */) { + bool _return = false; + return _return; + } + bool delete_table_column_statistics(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* col_name */) { + bool _return = false; + return _return; + } + void create_function(const Function& /* func */) { + return; + } + void drop_function(const std::string& /* dbName */, const std::string& /* funcName */) { + return; + } + void alter_function(const std::string& /* dbName */, const std::string& /* funcName */, const Function& /* newFunc */) { + return; + } + void get_functions(std::vector & /* _return */, const std::string& /* dbName */, const std::string& /* pattern */) { + return; + } + void get_function(Function& /* _return */, const std::string& /* dbName */, const std::string& /* funcName */) { + return; + } + void get_all_functions(GetAllFunctionsResponse& /* _return */) { + return; + } + bool create_role(const Role& /* role */) { + bool _return = false; + return _return; + } + bool drop_role(const std::string& /* role_name */) { + bool _return = false; + return _return; + } + void get_role_names(std::vector & /* _return */) { + return; + } + bool grant_role(const std::string& /* role_name */, const std::string& /* principal_name */, const PrincipalType::type /* principal_type */, const std::string& /* grantor */, const PrincipalType::type /* grantorType */, const bool /* grant_option */) { + bool _return = false; + return _return; + } + bool revoke_role(const std::string& /* role_name */, const std::string& /* principal_name */, const PrincipalType::type /* principal_type */) { + bool _return = false; + return _return; + } + void list_roles(std::vector & /* _return */, const std::string& /* principal_name */, const PrincipalType::type /* principal_type */) { + return; + } + void grant_revoke_role(GrantRevokeRoleResponse& /* _return */, const GrantRevokeRoleRequest& /* request */) { + return; + } + void get_principals_in_role(GetPrincipalsInRoleResponse& /* _return */, const GetPrincipalsInRoleRequest& /* request */) { + return; + } + void get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& /* _return */, const GetRoleGrantsForPrincipalRequest& /* request */) { + return; + } + void get_privilege_set(PrincipalPrivilegeSet& /* _return */, const HiveObjectRef& /* hiveObject */, const std::string& /* user_name */, const std::vector & /* group_names */) { + return; + } + void list_privileges(std::vector & /* _return */, const std::string& /* principal_name */, const PrincipalType::type /* principal_type */, const HiveObjectRef& /* hiveObject */) { + return; + } + bool grant_privileges(const PrivilegeBag& /* privileges */) { + bool _return = false; + return _return; + } + bool revoke_privileges(const PrivilegeBag& /* privileges */) { + bool _return = false; + return _return; + } + void grant_revoke_privileges(GrantRevokePrivilegeResponse& /* _return */, const GrantRevokePrivilegeRequest& /* request */) { + return; + } + void set_ugi(std::vector & /* _return */, const std::string& /* user_name */, const std::vector & /* group_names */) { + return; + } + void get_delegation_token(std::string& /* _return */, const std::string& /* token_owner */, const std::string& /* renewer_kerberos_principal_name */) { + return; + } + int64_t renew_delegation_token(const std::string& /* token_str_form */) { + int64_t _return = 0; + return _return; + } + void cancel_delegation_token(const std::string& /* token_str_form */) { + return; + } + bool add_token(const std::string& /* token_identifier */, const std::string& /* delegation_token */) { + bool _return = false; + return _return; + } + bool remove_token(const std::string& /* token_identifier */) { + bool _return = false; + return _return; + } + void get_token(std::string& /* _return */, const std::string& /* token_identifier */) { + return; + } + void get_all_token_identifiers(std::vector & /* _return */) { + return; + } + int32_t add_master_key(const std::string& /* key */) { + int32_t _return = 0; + return _return; + } + void update_master_key(const int32_t /* seq_number */, const std::string& /* key */) { + return; + } + bool remove_master_key(const int32_t /* key_seq */) { + bool _return = false; + return _return; + } + void get_master_keys(std::vector & /* _return */) { + return; + } + void get_open_txns(GetOpenTxnsResponse& /* _return */) { + return; + } + void get_open_txns_info(GetOpenTxnsInfoResponse& /* _return */) { + return; + } + void open_txns(OpenTxnsResponse& /* _return */, const OpenTxnRequest& /* rqst */) { + return; + } + void abort_txn(const AbortTxnRequest& /* rqst */) { + return; + } + void abort_txns(const AbortTxnsRequest& /* rqst */) { + return; + } + void commit_txn(const CommitTxnRequest& /* rqst */) { + return; + } + void lock(LockResponse& /* _return */, const LockRequest& /* rqst */) { + return; + } + void check_lock(LockResponse& /* _return */, const CheckLockRequest& /* rqst */) { + return; + } + void unlock(const UnlockRequest& /* rqst */) { + return; + } + void show_locks(ShowLocksResponse& /* _return */, const ShowLocksRequest& /* rqst */) { + return; + } + void heartbeat(const HeartbeatRequest& /* ids */) { + return; + } + void heartbeat_txn_range(HeartbeatTxnRangeResponse& /* _return */, const HeartbeatTxnRangeRequest& /* txns */) { + return; + } + void compact(const CompactionRequest& /* rqst */) { + return; + } + void compact2(CompactionResponse& /* _return */, const CompactionRequest& /* rqst */) { + return; + } + void show_compact(ShowCompactResponse& /* _return */, const ShowCompactRequest& /* rqst */) { + return; + } + void add_dynamic_partitions(const AddDynamicPartitions& /* rqst */) { + return; + } + void get_next_notification(NotificationEventResponse& /* _return */, const NotificationEventRequest& /* rqst */) { + return; + } + void get_current_notificationEventId(CurrentNotificationEventId& /* _return */) { + return; + } + void get_notification_events_count(NotificationEventsCountResponse& /* _return */, const NotificationEventsCountRequest& /* rqst */) { + return; + } + void fire_listener_event(FireEventResponse& /* _return */, const FireEventRequest& /* rqst */) { + return; + } + void flushCache() { + return; + } + void cm_recycle(CmRecycleResponse& /* _return */, const CmRecycleRequest& /* request */) { + return; + } + void get_file_metadata_by_expr(GetFileMetadataByExprResult& /* _return */, const GetFileMetadataByExprRequest& /* req */) { + return; + } + void get_file_metadata(GetFileMetadataResult& /* _return */, const GetFileMetadataRequest& /* req */) { + return; + } + void put_file_metadata(PutFileMetadataResult& /* _return */, const PutFileMetadataRequest& /* req */) { + return; + } + void clear_file_metadata(ClearFileMetadataResult& /* _return */, const ClearFileMetadataRequest& /* req */) { + return; + } + void cache_file_metadata(CacheFileMetadataResult& /* _return */, const CacheFileMetadataRequest& /* req */) { + return; + } + void get_metastore_db_uuid(std::string& /* _return */) { + return; + } + void create_resource_plan(WMCreateResourcePlanResponse& /* _return */, const WMCreateResourcePlanRequest& /* request */) { + return; + } + void get_resource_plan(WMGetResourcePlanResponse& /* _return */, const WMGetResourcePlanRequest& /* request */) { + return; + } + void get_active_resource_plan(WMGetActiveResourcePlanResponse& /* _return */, const WMGetActiveResourcePlanRequest& /* request */) { + return; + } + void get_all_resource_plans(WMGetAllResourcePlanResponse& /* _return */, const WMGetAllResourcePlanRequest& /* request */) { + return; + } + void alter_resource_plan(WMAlterResourcePlanResponse& /* _return */, const WMAlterResourcePlanRequest& /* request */) { + return; + } + void validate_resource_plan(WMValidateResourcePlanResponse& /* _return */, const WMValidateResourcePlanRequest& /* request */) { + return; + } + void drop_resource_plan(WMDropResourcePlanResponse& /* _return */, const WMDropResourcePlanRequest& /* request */) { + return; + } + void create_wm_trigger(WMCreateTriggerResponse& /* _return */, const WMCreateTriggerRequest& /* request */) { + return; + } + void alter_wm_trigger(WMAlterTriggerResponse& /* _return */, const WMAlterTriggerRequest& /* request */) { + return; + } + void drop_wm_trigger(WMDropTriggerResponse& /* _return */, const WMDropTriggerRequest& /* request */) { + return; + } + void get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& /* _return */, const WMGetTriggersForResourePlanRequest& /* request */) { + return; + } +}; + +typedef struct _ThriftHiveMetastore_getMetaConf_args__isset { + _ThriftHiveMetastore_getMetaConf_args__isset() : key(false) {} + bool key :1; +} _ThriftHiveMetastore_getMetaConf_args__isset; + +class ThriftHiveMetastore_getMetaConf_args { + public: + + ThriftHiveMetastore_getMetaConf_args(const ThriftHiveMetastore_getMetaConf_args&); + ThriftHiveMetastore_getMetaConf_args& operator=(const ThriftHiveMetastore_getMetaConf_args&); + ThriftHiveMetastore_getMetaConf_args() : key() { + } + + virtual ~ThriftHiveMetastore_getMetaConf_args() noexcept; + std::string key; + + _ThriftHiveMetastore_getMetaConf_args__isset __isset; + + void __set_key(const std::string& val); + + bool operator == (const ThriftHiveMetastore_getMetaConf_args & rhs) const + { + if (!(key == rhs.key)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_getMetaConf_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_getMetaConf_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_getMetaConf_pargs { + public: + + + virtual ~ThriftHiveMetastore_getMetaConf_pargs() noexcept; + const std::string* key; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_getMetaConf_result__isset { + _ThriftHiveMetastore_getMetaConf_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_getMetaConf_result__isset; + +class ThriftHiveMetastore_getMetaConf_result { + public: + + ThriftHiveMetastore_getMetaConf_result(const ThriftHiveMetastore_getMetaConf_result&); + ThriftHiveMetastore_getMetaConf_result& operator=(const ThriftHiveMetastore_getMetaConf_result&); + ThriftHiveMetastore_getMetaConf_result() : success() { + } + + virtual ~ThriftHiveMetastore_getMetaConf_result() noexcept; + std::string success; + MetaException o1; + + _ThriftHiveMetastore_getMetaConf_result__isset __isset; + + void __set_success(const std::string& val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_getMetaConf_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_getMetaConf_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_getMetaConf_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_getMetaConf_presult__isset { + _ThriftHiveMetastore_getMetaConf_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_getMetaConf_presult__isset; + +class ThriftHiveMetastore_getMetaConf_presult { + public: + + + virtual ~ThriftHiveMetastore_getMetaConf_presult() noexcept; + std::string* success; + MetaException o1; + + _ThriftHiveMetastore_getMetaConf_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_setMetaConf_args__isset { + _ThriftHiveMetastore_setMetaConf_args__isset() : key(false), value(false) {} + bool key :1; + bool value :1; +} _ThriftHiveMetastore_setMetaConf_args__isset; + +class ThriftHiveMetastore_setMetaConf_args { + public: + + ThriftHiveMetastore_setMetaConf_args(const ThriftHiveMetastore_setMetaConf_args&); + ThriftHiveMetastore_setMetaConf_args& operator=(const ThriftHiveMetastore_setMetaConf_args&); + ThriftHiveMetastore_setMetaConf_args() : key(), value() { + } + + virtual ~ThriftHiveMetastore_setMetaConf_args() noexcept; + std::string key; + std::string value; + + _ThriftHiveMetastore_setMetaConf_args__isset __isset; + + void __set_key(const std::string& val); + + void __set_value(const std::string& val); + + bool operator == (const ThriftHiveMetastore_setMetaConf_args & rhs) const + { + if (!(key == rhs.key)) + return false; + if (!(value == rhs.value)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_setMetaConf_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_setMetaConf_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_setMetaConf_pargs { + public: + + + virtual ~ThriftHiveMetastore_setMetaConf_pargs() noexcept; + const std::string* key; + const std::string* value; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_setMetaConf_result__isset { + _ThriftHiveMetastore_setMetaConf_result__isset() : o1(false) {} + bool o1 :1; +} _ThriftHiveMetastore_setMetaConf_result__isset; + +class ThriftHiveMetastore_setMetaConf_result { + public: + + ThriftHiveMetastore_setMetaConf_result(const ThriftHiveMetastore_setMetaConf_result&); + ThriftHiveMetastore_setMetaConf_result& operator=(const ThriftHiveMetastore_setMetaConf_result&); + ThriftHiveMetastore_setMetaConf_result() { + } + + virtual ~ThriftHiveMetastore_setMetaConf_result() noexcept; + MetaException o1; + + _ThriftHiveMetastore_setMetaConf_result__isset __isset; + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_setMetaConf_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_setMetaConf_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_setMetaConf_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_setMetaConf_presult__isset { + _ThriftHiveMetastore_setMetaConf_presult__isset() : o1(false) {} + bool o1 :1; +} _ThriftHiveMetastore_setMetaConf_presult__isset; + +class ThriftHiveMetastore_setMetaConf_presult { + public: + + + virtual ~ThriftHiveMetastore_setMetaConf_presult() noexcept; + MetaException o1; + + _ThriftHiveMetastore_setMetaConf_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_create_database_args__isset { + _ThriftHiveMetastore_create_database_args__isset() : database(false) {} + bool database :1; +} _ThriftHiveMetastore_create_database_args__isset; + +class ThriftHiveMetastore_create_database_args { + public: + + ThriftHiveMetastore_create_database_args(const ThriftHiveMetastore_create_database_args&); + ThriftHiveMetastore_create_database_args& operator=(const ThriftHiveMetastore_create_database_args&); + ThriftHiveMetastore_create_database_args() { + } + + virtual ~ThriftHiveMetastore_create_database_args() noexcept; + Database database; + + _ThriftHiveMetastore_create_database_args__isset __isset; + + void __set_database(const Database& val); + + bool operator == (const ThriftHiveMetastore_create_database_args & rhs) const + { + if (!(database == rhs.database)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_database_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_database_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_create_database_pargs { + public: + + + virtual ~ThriftHiveMetastore_create_database_pargs() noexcept; + const Database* database; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_database_result__isset { + _ThriftHiveMetastore_create_database_result__isset() : o1(false), o2(false), o3(false) {} + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_create_database_result__isset; + +class ThriftHiveMetastore_create_database_result { + public: + + ThriftHiveMetastore_create_database_result(const ThriftHiveMetastore_create_database_result&); + ThriftHiveMetastore_create_database_result& operator=(const ThriftHiveMetastore_create_database_result&); + ThriftHiveMetastore_create_database_result() { + } + + virtual ~ThriftHiveMetastore_create_database_result() noexcept; + AlreadyExistsException o1; + InvalidObjectException o2; + MetaException o3; + + _ThriftHiveMetastore_create_database_result__isset __isset; + + void __set_o1(const AlreadyExistsException& val); + + void __set_o2(const InvalidObjectException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_create_database_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_database_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_database_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_database_presult__isset { + _ThriftHiveMetastore_create_database_presult__isset() : o1(false), o2(false), o3(false) {} + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_create_database_presult__isset; + +class ThriftHiveMetastore_create_database_presult { + public: + + + virtual ~ThriftHiveMetastore_create_database_presult() noexcept; + AlreadyExistsException o1; + InvalidObjectException o2; + MetaException o3; + + _ThriftHiveMetastore_create_database_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_database_args__isset { + _ThriftHiveMetastore_get_database_args__isset() : name(false) {} + bool name :1; +} _ThriftHiveMetastore_get_database_args__isset; + +class ThriftHiveMetastore_get_database_args { + public: + + ThriftHiveMetastore_get_database_args(const ThriftHiveMetastore_get_database_args&); + ThriftHiveMetastore_get_database_args& operator=(const ThriftHiveMetastore_get_database_args&); + ThriftHiveMetastore_get_database_args() : name() { + } + + virtual ~ThriftHiveMetastore_get_database_args() noexcept; + std::string name; + + _ThriftHiveMetastore_get_database_args__isset __isset; + + void __set_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_database_args & rhs) const + { + if (!(name == rhs.name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_database_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_database_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_database_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_database_pargs() noexcept; + const std::string* name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_database_result__isset { + _ThriftHiveMetastore_get_database_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_database_result__isset; + +class ThriftHiveMetastore_get_database_result { + public: + + ThriftHiveMetastore_get_database_result(const ThriftHiveMetastore_get_database_result&); + ThriftHiveMetastore_get_database_result& operator=(const ThriftHiveMetastore_get_database_result&); + ThriftHiveMetastore_get_database_result() { + } + + virtual ~ThriftHiveMetastore_get_database_result() noexcept; + Database success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_database_result__isset __isset; + + void __set_success(const Database& val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_database_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_database_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_database_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_database_presult__isset { + _ThriftHiveMetastore_get_database_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_database_presult__isset; + +class ThriftHiveMetastore_get_database_presult { + public: + + + virtual ~ThriftHiveMetastore_get_database_presult() noexcept; + Database* success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_database_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_drop_database_args__isset { + _ThriftHiveMetastore_drop_database_args__isset() : name(false), deleteData(false), cascade(false) {} + bool name :1; + bool deleteData :1; + bool cascade :1; +} _ThriftHiveMetastore_drop_database_args__isset; + +class ThriftHiveMetastore_drop_database_args { + public: + + ThriftHiveMetastore_drop_database_args(const ThriftHiveMetastore_drop_database_args&); + ThriftHiveMetastore_drop_database_args& operator=(const ThriftHiveMetastore_drop_database_args&); + ThriftHiveMetastore_drop_database_args() : name(), deleteData(0), cascade(0) { + } + + virtual ~ThriftHiveMetastore_drop_database_args() noexcept; + std::string name; + bool deleteData; + bool cascade; + + _ThriftHiveMetastore_drop_database_args__isset __isset; + + void __set_name(const std::string& val); + + void __set_deleteData(const bool val); + + void __set_cascade(const bool val); + + bool operator == (const ThriftHiveMetastore_drop_database_args & rhs) const + { + if (!(name == rhs.name)) + return false; + if (!(deleteData == rhs.deleteData)) + return false; + if (!(cascade == rhs.cascade)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_database_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_database_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_drop_database_pargs { + public: + + + virtual ~ThriftHiveMetastore_drop_database_pargs() noexcept; + const std::string* name; + const bool* deleteData; + const bool* cascade; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_database_result__isset { + _ThriftHiveMetastore_drop_database_result__isset() : o1(false), o2(false), o3(false) {} + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_drop_database_result__isset; + +class ThriftHiveMetastore_drop_database_result { + public: + + ThriftHiveMetastore_drop_database_result(const ThriftHiveMetastore_drop_database_result&); + ThriftHiveMetastore_drop_database_result& operator=(const ThriftHiveMetastore_drop_database_result&); + ThriftHiveMetastore_drop_database_result() { + } + + virtual ~ThriftHiveMetastore_drop_database_result() noexcept; + NoSuchObjectException o1; + InvalidOperationException o2; + MetaException o3; + + _ThriftHiveMetastore_drop_database_result__isset __isset; + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const InvalidOperationException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_drop_database_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_database_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_database_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_database_presult__isset { + _ThriftHiveMetastore_drop_database_presult__isset() : o1(false), o2(false), o3(false) {} + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_drop_database_presult__isset; + +class ThriftHiveMetastore_drop_database_presult { + public: + + + virtual ~ThriftHiveMetastore_drop_database_presult() noexcept; + NoSuchObjectException o1; + InvalidOperationException o2; + MetaException o3; + + _ThriftHiveMetastore_drop_database_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_databases_args__isset { + _ThriftHiveMetastore_get_databases_args__isset() : pattern(false) {} + bool pattern :1; +} _ThriftHiveMetastore_get_databases_args__isset; + +class ThriftHiveMetastore_get_databases_args { + public: + + ThriftHiveMetastore_get_databases_args(const ThriftHiveMetastore_get_databases_args&); + ThriftHiveMetastore_get_databases_args& operator=(const ThriftHiveMetastore_get_databases_args&); + ThriftHiveMetastore_get_databases_args() : pattern() { + } + + virtual ~ThriftHiveMetastore_get_databases_args() noexcept; + std::string pattern; + + _ThriftHiveMetastore_get_databases_args__isset __isset; + + void __set_pattern(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_databases_args & rhs) const + { + if (!(pattern == rhs.pattern)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_databases_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_databases_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_databases_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_databases_pargs() noexcept; + const std::string* pattern; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_databases_result__isset { + _ThriftHiveMetastore_get_databases_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_databases_result__isset; + +class ThriftHiveMetastore_get_databases_result { + public: + + ThriftHiveMetastore_get_databases_result(const ThriftHiveMetastore_get_databases_result&); + ThriftHiveMetastore_get_databases_result& operator=(const ThriftHiveMetastore_get_databases_result&); + ThriftHiveMetastore_get_databases_result() { + } + + virtual ~ThriftHiveMetastore_get_databases_result() noexcept; + std::vector success; + MetaException o1; + + _ThriftHiveMetastore_get_databases_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_databases_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_databases_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_databases_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_databases_presult__isset { + _ThriftHiveMetastore_get_databases_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_databases_presult__isset; + +class ThriftHiveMetastore_get_databases_presult { + public: + + + virtual ~ThriftHiveMetastore_get_databases_presult() noexcept; + std::vector * success; + MetaException o1; + + _ThriftHiveMetastore_get_databases_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + + +class ThriftHiveMetastore_get_all_databases_args { + public: + + ThriftHiveMetastore_get_all_databases_args(const ThriftHiveMetastore_get_all_databases_args&); + ThriftHiveMetastore_get_all_databases_args& operator=(const ThriftHiveMetastore_get_all_databases_args&); + ThriftHiveMetastore_get_all_databases_args() { + } + + virtual ~ThriftHiveMetastore_get_all_databases_args() noexcept; + + bool operator == (const ThriftHiveMetastore_get_all_databases_args & /* rhs */) const + { + return true; + } + bool operator != (const ThriftHiveMetastore_get_all_databases_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_all_databases_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_all_databases_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_all_databases_pargs() noexcept; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_all_databases_result__isset { + _ThriftHiveMetastore_get_all_databases_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_all_databases_result__isset; + +class ThriftHiveMetastore_get_all_databases_result { + public: + + ThriftHiveMetastore_get_all_databases_result(const ThriftHiveMetastore_get_all_databases_result&); + ThriftHiveMetastore_get_all_databases_result& operator=(const ThriftHiveMetastore_get_all_databases_result&); + ThriftHiveMetastore_get_all_databases_result() { + } + + virtual ~ThriftHiveMetastore_get_all_databases_result() noexcept; + std::vector success; + MetaException o1; + + _ThriftHiveMetastore_get_all_databases_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_all_databases_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_all_databases_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_all_databases_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_all_databases_presult__isset { + _ThriftHiveMetastore_get_all_databases_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_all_databases_presult__isset; + +class ThriftHiveMetastore_get_all_databases_presult { + public: + + + virtual ~ThriftHiveMetastore_get_all_databases_presult() noexcept; + std::vector * success; + MetaException o1; + + _ThriftHiveMetastore_get_all_databases_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_alter_database_args__isset { + _ThriftHiveMetastore_alter_database_args__isset() : dbname(false), db(false) {} + bool dbname :1; + bool db :1; +} _ThriftHiveMetastore_alter_database_args__isset; + +class ThriftHiveMetastore_alter_database_args { + public: + + ThriftHiveMetastore_alter_database_args(const ThriftHiveMetastore_alter_database_args&); + ThriftHiveMetastore_alter_database_args& operator=(const ThriftHiveMetastore_alter_database_args&); + ThriftHiveMetastore_alter_database_args() : dbname() { + } + + virtual ~ThriftHiveMetastore_alter_database_args() noexcept; + std::string dbname; + Database db; + + _ThriftHiveMetastore_alter_database_args__isset __isset; + + void __set_dbname(const std::string& val); + + void __set_db(const Database& val); + + bool operator == (const ThriftHiveMetastore_alter_database_args & rhs) const + { + if (!(dbname == rhs.dbname)) + return false; + if (!(db == rhs.db)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_database_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_database_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_alter_database_pargs { + public: + + + virtual ~ThriftHiveMetastore_alter_database_pargs() noexcept; + const std::string* dbname; + const Database* db; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_database_result__isset { + _ThriftHiveMetastore_alter_database_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_database_result__isset; + +class ThriftHiveMetastore_alter_database_result { + public: + + ThriftHiveMetastore_alter_database_result(const ThriftHiveMetastore_alter_database_result&); + ThriftHiveMetastore_alter_database_result& operator=(const ThriftHiveMetastore_alter_database_result&); + ThriftHiveMetastore_alter_database_result() { + } + + virtual ~ThriftHiveMetastore_alter_database_result() noexcept; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_alter_database_result__isset __isset; + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_alter_database_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_database_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_database_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_database_presult__isset { + _ThriftHiveMetastore_alter_database_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_database_presult__isset; + +class ThriftHiveMetastore_alter_database_presult { + public: + + + virtual ~ThriftHiveMetastore_alter_database_presult() noexcept; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_alter_database_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_type_args__isset { + _ThriftHiveMetastore_get_type_args__isset() : name(false) {} + bool name :1; +} _ThriftHiveMetastore_get_type_args__isset; + +class ThriftHiveMetastore_get_type_args { + public: + + ThriftHiveMetastore_get_type_args(const ThriftHiveMetastore_get_type_args&); + ThriftHiveMetastore_get_type_args& operator=(const ThriftHiveMetastore_get_type_args&); + ThriftHiveMetastore_get_type_args() : name() { + } + + virtual ~ThriftHiveMetastore_get_type_args() noexcept; + std::string name; + + _ThriftHiveMetastore_get_type_args__isset __isset; + + void __set_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_type_args & rhs) const + { + if (!(name == rhs.name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_type_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_type_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_type_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_type_pargs() noexcept; + const std::string* name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_type_result__isset { + _ThriftHiveMetastore_get_type_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_type_result__isset; + +class ThriftHiveMetastore_get_type_result { + public: + + ThriftHiveMetastore_get_type_result(const ThriftHiveMetastore_get_type_result&); + ThriftHiveMetastore_get_type_result& operator=(const ThriftHiveMetastore_get_type_result&); + ThriftHiveMetastore_get_type_result() { + } + + virtual ~ThriftHiveMetastore_get_type_result() noexcept; + Type success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_type_result__isset __isset; + + void __set_success(const Type& val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_type_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_type_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_type_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_type_presult__isset { + _ThriftHiveMetastore_get_type_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_type_presult__isset; + +class ThriftHiveMetastore_get_type_presult { + public: + + + virtual ~ThriftHiveMetastore_get_type_presult() noexcept; + Type* success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_type_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_create_type_args__isset { + _ThriftHiveMetastore_create_type_args__isset() : type(false) {} + bool type :1; +} _ThriftHiveMetastore_create_type_args__isset; + +class ThriftHiveMetastore_create_type_args { + public: + + ThriftHiveMetastore_create_type_args(const ThriftHiveMetastore_create_type_args&); + ThriftHiveMetastore_create_type_args& operator=(const ThriftHiveMetastore_create_type_args&); + ThriftHiveMetastore_create_type_args() { + } + + virtual ~ThriftHiveMetastore_create_type_args() noexcept; + Type type; + + _ThriftHiveMetastore_create_type_args__isset __isset; + + void __set_type(const Type& val); + + bool operator == (const ThriftHiveMetastore_create_type_args & rhs) const + { + if (!(type == rhs.type)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_type_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_type_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_create_type_pargs { + public: + + + virtual ~ThriftHiveMetastore_create_type_pargs() noexcept; + const Type* type; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_type_result__isset { + _ThriftHiveMetastore_create_type_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_create_type_result__isset; + +class ThriftHiveMetastore_create_type_result { + public: + + ThriftHiveMetastore_create_type_result(const ThriftHiveMetastore_create_type_result&); + ThriftHiveMetastore_create_type_result& operator=(const ThriftHiveMetastore_create_type_result&); + ThriftHiveMetastore_create_type_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_create_type_result() noexcept; + bool success; + AlreadyExistsException o1; + InvalidObjectException o2; + MetaException o3; + + _ThriftHiveMetastore_create_type_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const AlreadyExistsException& val); + + void __set_o2(const InvalidObjectException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_create_type_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_type_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_type_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_type_presult__isset { + _ThriftHiveMetastore_create_type_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_create_type_presult__isset; + +class ThriftHiveMetastore_create_type_presult { + public: + + + virtual ~ThriftHiveMetastore_create_type_presult() noexcept; + bool* success; + AlreadyExistsException o1; + InvalidObjectException o2; + MetaException o3; + + _ThriftHiveMetastore_create_type_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_drop_type_args__isset { + _ThriftHiveMetastore_drop_type_args__isset() : type(false) {} + bool type :1; +} _ThriftHiveMetastore_drop_type_args__isset; + +class ThriftHiveMetastore_drop_type_args { + public: + + ThriftHiveMetastore_drop_type_args(const ThriftHiveMetastore_drop_type_args&); + ThriftHiveMetastore_drop_type_args& operator=(const ThriftHiveMetastore_drop_type_args&); + ThriftHiveMetastore_drop_type_args() : type() { + } + + virtual ~ThriftHiveMetastore_drop_type_args() noexcept; + std::string type; + + _ThriftHiveMetastore_drop_type_args__isset __isset; + + void __set_type(const std::string& val); + + bool operator == (const ThriftHiveMetastore_drop_type_args & rhs) const + { + if (!(type == rhs.type)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_type_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_type_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_drop_type_pargs { + public: + + + virtual ~ThriftHiveMetastore_drop_type_pargs() noexcept; + const std::string* type; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_type_result__isset { + _ThriftHiveMetastore_drop_type_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_drop_type_result__isset; + +class ThriftHiveMetastore_drop_type_result { + public: + + ThriftHiveMetastore_drop_type_result(const ThriftHiveMetastore_drop_type_result&); + ThriftHiveMetastore_drop_type_result& operator=(const ThriftHiveMetastore_drop_type_result&); + ThriftHiveMetastore_drop_type_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_drop_type_result() noexcept; + bool success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_drop_type_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_drop_type_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_type_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_type_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_type_presult__isset { + _ThriftHiveMetastore_drop_type_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_drop_type_presult__isset; + +class ThriftHiveMetastore_drop_type_presult { + public: + + + virtual ~ThriftHiveMetastore_drop_type_presult() noexcept; + bool* success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_drop_type_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_type_all_args__isset { + _ThriftHiveMetastore_get_type_all_args__isset() : name(false) {} + bool name :1; +} _ThriftHiveMetastore_get_type_all_args__isset; + +class ThriftHiveMetastore_get_type_all_args { + public: + + ThriftHiveMetastore_get_type_all_args(const ThriftHiveMetastore_get_type_all_args&); + ThriftHiveMetastore_get_type_all_args& operator=(const ThriftHiveMetastore_get_type_all_args&); + ThriftHiveMetastore_get_type_all_args() : name() { + } + + virtual ~ThriftHiveMetastore_get_type_all_args() noexcept; + std::string name; + + _ThriftHiveMetastore_get_type_all_args__isset __isset; + + void __set_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_type_all_args & rhs) const + { + if (!(name == rhs.name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_type_all_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_type_all_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_type_all_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_type_all_pargs() noexcept; + const std::string* name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_type_all_result__isset { + _ThriftHiveMetastore_get_type_all_result__isset() : success(false), o2(false) {} + bool success :1; + bool o2 :1; +} _ThriftHiveMetastore_get_type_all_result__isset; + +class ThriftHiveMetastore_get_type_all_result { + public: + + ThriftHiveMetastore_get_type_all_result(const ThriftHiveMetastore_get_type_all_result&); + ThriftHiveMetastore_get_type_all_result& operator=(const ThriftHiveMetastore_get_type_all_result&); + ThriftHiveMetastore_get_type_all_result() { + } + + virtual ~ThriftHiveMetastore_get_type_all_result() noexcept; + std::map success; + MetaException o2; + + _ThriftHiveMetastore_get_type_all_result__isset __isset; + + void __set_success(const std::map & val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_type_all_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_type_all_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_type_all_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_type_all_presult__isset { + _ThriftHiveMetastore_get_type_all_presult__isset() : success(false), o2(false) {} + bool success :1; + bool o2 :1; +} _ThriftHiveMetastore_get_type_all_presult__isset; + +class ThriftHiveMetastore_get_type_all_presult { + public: + + + virtual ~ThriftHiveMetastore_get_type_all_presult() noexcept; + std::map * success; + MetaException o2; + + _ThriftHiveMetastore_get_type_all_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_fields_args__isset { + _ThriftHiveMetastore_get_fields_args__isset() : db_name(false), table_name(false) {} + bool db_name :1; + bool table_name :1; +} _ThriftHiveMetastore_get_fields_args__isset; + +class ThriftHiveMetastore_get_fields_args { + public: + + ThriftHiveMetastore_get_fields_args(const ThriftHiveMetastore_get_fields_args&); + ThriftHiveMetastore_get_fields_args& operator=(const ThriftHiveMetastore_get_fields_args&); + ThriftHiveMetastore_get_fields_args() : db_name(), table_name() { + } + + virtual ~ThriftHiveMetastore_get_fields_args() noexcept; + std::string db_name; + std::string table_name; + + _ThriftHiveMetastore_get_fields_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_table_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_fields_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(table_name == rhs.table_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_fields_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_fields_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_fields_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_fields_pargs() noexcept; + const std::string* db_name; + const std::string* table_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_fields_result__isset { + _ThriftHiveMetastore_get_fields_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_get_fields_result__isset; + +class ThriftHiveMetastore_get_fields_result { + public: + + ThriftHiveMetastore_get_fields_result(const ThriftHiveMetastore_get_fields_result&); + ThriftHiveMetastore_get_fields_result& operator=(const ThriftHiveMetastore_get_fields_result&); + ThriftHiveMetastore_get_fields_result() { + } + + virtual ~ThriftHiveMetastore_get_fields_result() noexcept; + std::vector success; + MetaException o1; + UnknownTableException o2; + UnknownDBException o3; + + _ThriftHiveMetastore_get_fields_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + void __set_o2(const UnknownTableException& val); + + void __set_o3(const UnknownDBException& val); + + bool operator == (const ThriftHiveMetastore_get_fields_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_fields_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_fields_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_fields_presult__isset { + _ThriftHiveMetastore_get_fields_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_get_fields_presult__isset; + +class ThriftHiveMetastore_get_fields_presult { + public: + + + virtual ~ThriftHiveMetastore_get_fields_presult() noexcept; + std::vector * success; + MetaException o1; + UnknownTableException o2; + UnknownDBException o3; + + _ThriftHiveMetastore_get_fields_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_fields_with_environment_context_args__isset { + _ThriftHiveMetastore_get_fields_with_environment_context_args__isset() : db_name(false), table_name(false), environment_context(false) {} + bool db_name :1; + bool table_name :1; + bool environment_context :1; +} _ThriftHiveMetastore_get_fields_with_environment_context_args__isset; + +class ThriftHiveMetastore_get_fields_with_environment_context_args { + public: + + ThriftHiveMetastore_get_fields_with_environment_context_args(const ThriftHiveMetastore_get_fields_with_environment_context_args&); + ThriftHiveMetastore_get_fields_with_environment_context_args& operator=(const ThriftHiveMetastore_get_fields_with_environment_context_args&); + ThriftHiveMetastore_get_fields_with_environment_context_args() : db_name(), table_name() { + } + + virtual ~ThriftHiveMetastore_get_fields_with_environment_context_args() noexcept; + std::string db_name; + std::string table_name; + EnvironmentContext environment_context; + + _ThriftHiveMetastore_get_fields_with_environment_context_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_table_name(const std::string& val); + + void __set_environment_context(const EnvironmentContext& val); + + bool operator == (const ThriftHiveMetastore_get_fields_with_environment_context_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(table_name == rhs.table_name)) + return false; + if (!(environment_context == rhs.environment_context)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_fields_with_environment_context_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_fields_with_environment_context_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_fields_with_environment_context_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_fields_with_environment_context_pargs() noexcept; + const std::string* db_name; + const std::string* table_name; + const EnvironmentContext* environment_context; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_fields_with_environment_context_result__isset { + _ThriftHiveMetastore_get_fields_with_environment_context_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_get_fields_with_environment_context_result__isset; + +class ThriftHiveMetastore_get_fields_with_environment_context_result { + public: + + ThriftHiveMetastore_get_fields_with_environment_context_result(const ThriftHiveMetastore_get_fields_with_environment_context_result&); + ThriftHiveMetastore_get_fields_with_environment_context_result& operator=(const ThriftHiveMetastore_get_fields_with_environment_context_result&); + ThriftHiveMetastore_get_fields_with_environment_context_result() { + } + + virtual ~ThriftHiveMetastore_get_fields_with_environment_context_result() noexcept; + std::vector success; + MetaException o1; + UnknownTableException o2; + UnknownDBException o3; + + _ThriftHiveMetastore_get_fields_with_environment_context_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + void __set_o2(const UnknownTableException& val); + + void __set_o3(const UnknownDBException& val); + + bool operator == (const ThriftHiveMetastore_get_fields_with_environment_context_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_fields_with_environment_context_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_fields_with_environment_context_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_fields_with_environment_context_presult__isset { + _ThriftHiveMetastore_get_fields_with_environment_context_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_get_fields_with_environment_context_presult__isset; + +class ThriftHiveMetastore_get_fields_with_environment_context_presult { + public: + + + virtual ~ThriftHiveMetastore_get_fields_with_environment_context_presult() noexcept; + std::vector * success; + MetaException o1; + UnknownTableException o2; + UnknownDBException o3; + + _ThriftHiveMetastore_get_fields_with_environment_context_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_schema_args__isset { + _ThriftHiveMetastore_get_schema_args__isset() : db_name(false), table_name(false) {} + bool db_name :1; + bool table_name :1; +} _ThriftHiveMetastore_get_schema_args__isset; + +class ThriftHiveMetastore_get_schema_args { + public: + + ThriftHiveMetastore_get_schema_args(const ThriftHiveMetastore_get_schema_args&); + ThriftHiveMetastore_get_schema_args& operator=(const ThriftHiveMetastore_get_schema_args&); + ThriftHiveMetastore_get_schema_args() : db_name(), table_name() { + } + + virtual ~ThriftHiveMetastore_get_schema_args() noexcept; + std::string db_name; + std::string table_name; + + _ThriftHiveMetastore_get_schema_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_table_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_schema_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(table_name == rhs.table_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_schema_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_schema_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_schema_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_schema_pargs() noexcept; + const std::string* db_name; + const std::string* table_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_schema_result__isset { + _ThriftHiveMetastore_get_schema_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_get_schema_result__isset; + +class ThriftHiveMetastore_get_schema_result { + public: + + ThriftHiveMetastore_get_schema_result(const ThriftHiveMetastore_get_schema_result&); + ThriftHiveMetastore_get_schema_result& operator=(const ThriftHiveMetastore_get_schema_result&); + ThriftHiveMetastore_get_schema_result() { + } + + virtual ~ThriftHiveMetastore_get_schema_result() noexcept; + std::vector success; + MetaException o1; + UnknownTableException o2; + UnknownDBException o3; + + _ThriftHiveMetastore_get_schema_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + void __set_o2(const UnknownTableException& val); + + void __set_o3(const UnknownDBException& val); + + bool operator == (const ThriftHiveMetastore_get_schema_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_schema_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_schema_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_schema_presult__isset { + _ThriftHiveMetastore_get_schema_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_get_schema_presult__isset; + +class ThriftHiveMetastore_get_schema_presult { + public: + + + virtual ~ThriftHiveMetastore_get_schema_presult() noexcept; + std::vector * success; + MetaException o1; + UnknownTableException o2; + UnknownDBException o3; + + _ThriftHiveMetastore_get_schema_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_schema_with_environment_context_args__isset { + _ThriftHiveMetastore_get_schema_with_environment_context_args__isset() : db_name(false), table_name(false), environment_context(false) {} + bool db_name :1; + bool table_name :1; + bool environment_context :1; +} _ThriftHiveMetastore_get_schema_with_environment_context_args__isset; + +class ThriftHiveMetastore_get_schema_with_environment_context_args { + public: + + ThriftHiveMetastore_get_schema_with_environment_context_args(const ThriftHiveMetastore_get_schema_with_environment_context_args&); + ThriftHiveMetastore_get_schema_with_environment_context_args& operator=(const ThriftHiveMetastore_get_schema_with_environment_context_args&); + ThriftHiveMetastore_get_schema_with_environment_context_args() : db_name(), table_name() { + } + + virtual ~ThriftHiveMetastore_get_schema_with_environment_context_args() noexcept; + std::string db_name; + std::string table_name; + EnvironmentContext environment_context; + + _ThriftHiveMetastore_get_schema_with_environment_context_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_table_name(const std::string& val); + + void __set_environment_context(const EnvironmentContext& val); + + bool operator == (const ThriftHiveMetastore_get_schema_with_environment_context_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(table_name == rhs.table_name)) + return false; + if (!(environment_context == rhs.environment_context)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_schema_with_environment_context_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_schema_with_environment_context_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_schema_with_environment_context_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_schema_with_environment_context_pargs() noexcept; + const std::string* db_name; + const std::string* table_name; + const EnvironmentContext* environment_context; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_schema_with_environment_context_result__isset { + _ThriftHiveMetastore_get_schema_with_environment_context_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_get_schema_with_environment_context_result__isset; + +class ThriftHiveMetastore_get_schema_with_environment_context_result { + public: + + ThriftHiveMetastore_get_schema_with_environment_context_result(const ThriftHiveMetastore_get_schema_with_environment_context_result&); + ThriftHiveMetastore_get_schema_with_environment_context_result& operator=(const ThriftHiveMetastore_get_schema_with_environment_context_result&); + ThriftHiveMetastore_get_schema_with_environment_context_result() { + } + + virtual ~ThriftHiveMetastore_get_schema_with_environment_context_result() noexcept; + std::vector success; + MetaException o1; + UnknownTableException o2; + UnknownDBException o3; + + _ThriftHiveMetastore_get_schema_with_environment_context_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + void __set_o2(const UnknownTableException& val); + + void __set_o3(const UnknownDBException& val); + + bool operator == (const ThriftHiveMetastore_get_schema_with_environment_context_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_schema_with_environment_context_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_schema_with_environment_context_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_schema_with_environment_context_presult__isset { + _ThriftHiveMetastore_get_schema_with_environment_context_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_get_schema_with_environment_context_presult__isset; + +class ThriftHiveMetastore_get_schema_with_environment_context_presult { + public: + + + virtual ~ThriftHiveMetastore_get_schema_with_environment_context_presult() noexcept; + std::vector * success; + MetaException o1; + UnknownTableException o2; + UnknownDBException o3; + + _ThriftHiveMetastore_get_schema_with_environment_context_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_create_table_args__isset { + _ThriftHiveMetastore_create_table_args__isset() : tbl(false) {} + bool tbl :1; +} _ThriftHiveMetastore_create_table_args__isset; + +class ThriftHiveMetastore_create_table_args { + public: + + ThriftHiveMetastore_create_table_args(const ThriftHiveMetastore_create_table_args&); + ThriftHiveMetastore_create_table_args& operator=(const ThriftHiveMetastore_create_table_args&); + ThriftHiveMetastore_create_table_args() { + } + + virtual ~ThriftHiveMetastore_create_table_args() noexcept; + Table tbl; + + _ThriftHiveMetastore_create_table_args__isset __isset; + + void __set_tbl(const Table& val); + + bool operator == (const ThriftHiveMetastore_create_table_args & rhs) const + { + if (!(tbl == rhs.tbl)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_table_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_table_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_create_table_pargs { + public: + + + virtual ~ThriftHiveMetastore_create_table_pargs() noexcept; + const Table* tbl; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_table_result__isset { + _ThriftHiveMetastore_create_table_result__isset() : o1(false), o2(false), o3(false), o4(false) {} + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_create_table_result__isset; + +class ThriftHiveMetastore_create_table_result { + public: + + ThriftHiveMetastore_create_table_result(const ThriftHiveMetastore_create_table_result&); + ThriftHiveMetastore_create_table_result& operator=(const ThriftHiveMetastore_create_table_result&); + ThriftHiveMetastore_create_table_result() { + } + + virtual ~ThriftHiveMetastore_create_table_result() noexcept; + AlreadyExistsException o1; + InvalidObjectException o2; + MetaException o3; + NoSuchObjectException o4; + + _ThriftHiveMetastore_create_table_result__isset __isset; + + void __set_o1(const AlreadyExistsException& val); + + void __set_o2(const InvalidObjectException& val); + + void __set_o3(const MetaException& val); + + void __set_o4(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_create_table_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_table_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_table_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_table_presult__isset { + _ThriftHiveMetastore_create_table_presult__isset() : o1(false), o2(false), o3(false), o4(false) {} + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_create_table_presult__isset; + +class ThriftHiveMetastore_create_table_presult { + public: + + + virtual ~ThriftHiveMetastore_create_table_presult() noexcept; + AlreadyExistsException o1; + InvalidObjectException o2; + MetaException o3; + NoSuchObjectException o4; + + _ThriftHiveMetastore_create_table_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_create_table_with_environment_context_args__isset { + _ThriftHiveMetastore_create_table_with_environment_context_args__isset() : tbl(false), environment_context(false) {} + bool tbl :1; + bool environment_context :1; +} _ThriftHiveMetastore_create_table_with_environment_context_args__isset; + +class ThriftHiveMetastore_create_table_with_environment_context_args { + public: + + ThriftHiveMetastore_create_table_with_environment_context_args(const ThriftHiveMetastore_create_table_with_environment_context_args&); + ThriftHiveMetastore_create_table_with_environment_context_args& operator=(const ThriftHiveMetastore_create_table_with_environment_context_args&); + ThriftHiveMetastore_create_table_with_environment_context_args() { + } + + virtual ~ThriftHiveMetastore_create_table_with_environment_context_args() noexcept; + Table tbl; + EnvironmentContext environment_context; + + _ThriftHiveMetastore_create_table_with_environment_context_args__isset __isset; + + void __set_tbl(const Table& val); + + void __set_environment_context(const EnvironmentContext& val); + + bool operator == (const ThriftHiveMetastore_create_table_with_environment_context_args & rhs) const + { + if (!(tbl == rhs.tbl)) + return false; + if (!(environment_context == rhs.environment_context)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_table_with_environment_context_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_table_with_environment_context_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_create_table_with_environment_context_pargs { + public: + + + virtual ~ThriftHiveMetastore_create_table_with_environment_context_pargs() noexcept; + const Table* tbl; + const EnvironmentContext* environment_context; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_table_with_environment_context_result__isset { + _ThriftHiveMetastore_create_table_with_environment_context_result__isset() : o1(false), o2(false), o3(false), o4(false) {} + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_create_table_with_environment_context_result__isset; + +class ThriftHiveMetastore_create_table_with_environment_context_result { + public: + + ThriftHiveMetastore_create_table_with_environment_context_result(const ThriftHiveMetastore_create_table_with_environment_context_result&); + ThriftHiveMetastore_create_table_with_environment_context_result& operator=(const ThriftHiveMetastore_create_table_with_environment_context_result&); + ThriftHiveMetastore_create_table_with_environment_context_result() { + } + + virtual ~ThriftHiveMetastore_create_table_with_environment_context_result() noexcept; + AlreadyExistsException o1; + InvalidObjectException o2; + MetaException o3; + NoSuchObjectException o4; + + _ThriftHiveMetastore_create_table_with_environment_context_result__isset __isset; + + void __set_o1(const AlreadyExistsException& val); + + void __set_o2(const InvalidObjectException& val); + + void __set_o3(const MetaException& val); + + void __set_o4(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_create_table_with_environment_context_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_table_with_environment_context_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_table_with_environment_context_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_table_with_environment_context_presult__isset { + _ThriftHiveMetastore_create_table_with_environment_context_presult__isset() : o1(false), o2(false), o3(false), o4(false) {} + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_create_table_with_environment_context_presult__isset; + +class ThriftHiveMetastore_create_table_with_environment_context_presult { + public: + + + virtual ~ThriftHiveMetastore_create_table_with_environment_context_presult() noexcept; + AlreadyExistsException o1; + InvalidObjectException o2; + MetaException o3; + NoSuchObjectException o4; + + _ThriftHiveMetastore_create_table_with_environment_context_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_create_table_with_constraints_args__isset { + _ThriftHiveMetastore_create_table_with_constraints_args__isset() : tbl(false), primaryKeys(false), foreignKeys(false), uniqueConstraints(false), notNullConstraints(false) {} + bool tbl :1; + bool primaryKeys :1; + bool foreignKeys :1; + bool uniqueConstraints :1; + bool notNullConstraints :1; +} _ThriftHiveMetastore_create_table_with_constraints_args__isset; + +class ThriftHiveMetastore_create_table_with_constraints_args { + public: + + ThriftHiveMetastore_create_table_with_constraints_args(const ThriftHiveMetastore_create_table_with_constraints_args&); + ThriftHiveMetastore_create_table_with_constraints_args& operator=(const ThriftHiveMetastore_create_table_with_constraints_args&); + ThriftHiveMetastore_create_table_with_constraints_args() { + } + + virtual ~ThriftHiveMetastore_create_table_with_constraints_args() noexcept; + Table tbl; + std::vector primaryKeys; + std::vector foreignKeys; + std::vector uniqueConstraints; + std::vector notNullConstraints; + + _ThriftHiveMetastore_create_table_with_constraints_args__isset __isset; + + void __set_tbl(const Table& val); + + void __set_primaryKeys(const std::vector & val); + + void __set_foreignKeys(const std::vector & val); + + void __set_uniqueConstraints(const std::vector & val); + + void __set_notNullConstraints(const std::vector & val); + + bool operator == (const ThriftHiveMetastore_create_table_with_constraints_args & rhs) const + { + if (!(tbl == rhs.tbl)) + return false; + if (!(primaryKeys == rhs.primaryKeys)) + return false; + if (!(foreignKeys == rhs.foreignKeys)) + return false; + if (!(uniqueConstraints == rhs.uniqueConstraints)) + return false; + if (!(notNullConstraints == rhs.notNullConstraints)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_table_with_constraints_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_table_with_constraints_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_create_table_with_constraints_pargs { + public: + + + virtual ~ThriftHiveMetastore_create_table_with_constraints_pargs() noexcept; + const Table* tbl; + const std::vector * primaryKeys; + const std::vector * foreignKeys; + const std::vector * uniqueConstraints; + const std::vector * notNullConstraints; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_table_with_constraints_result__isset { + _ThriftHiveMetastore_create_table_with_constraints_result__isset() : o1(false), o2(false), o3(false), o4(false) {} + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_create_table_with_constraints_result__isset; + +class ThriftHiveMetastore_create_table_with_constraints_result { + public: + + ThriftHiveMetastore_create_table_with_constraints_result(const ThriftHiveMetastore_create_table_with_constraints_result&); + ThriftHiveMetastore_create_table_with_constraints_result& operator=(const ThriftHiveMetastore_create_table_with_constraints_result&); + ThriftHiveMetastore_create_table_with_constraints_result() { + } + + virtual ~ThriftHiveMetastore_create_table_with_constraints_result() noexcept; + AlreadyExistsException o1; + InvalidObjectException o2; + MetaException o3; + NoSuchObjectException o4; + + _ThriftHiveMetastore_create_table_with_constraints_result__isset __isset; + + void __set_o1(const AlreadyExistsException& val); + + void __set_o2(const InvalidObjectException& val); + + void __set_o3(const MetaException& val); + + void __set_o4(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_create_table_with_constraints_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_table_with_constraints_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_table_with_constraints_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_table_with_constraints_presult__isset { + _ThriftHiveMetastore_create_table_with_constraints_presult__isset() : o1(false), o2(false), o3(false), o4(false) {} + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_create_table_with_constraints_presult__isset; + +class ThriftHiveMetastore_create_table_with_constraints_presult { + public: + + + virtual ~ThriftHiveMetastore_create_table_with_constraints_presult() noexcept; + AlreadyExistsException o1; + InvalidObjectException o2; + MetaException o3; + NoSuchObjectException o4; + + _ThriftHiveMetastore_create_table_with_constraints_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_drop_constraint_args__isset { + _ThriftHiveMetastore_drop_constraint_args__isset() : req(false) {} + bool req :1; +} _ThriftHiveMetastore_drop_constraint_args__isset; + +class ThriftHiveMetastore_drop_constraint_args { + public: + + ThriftHiveMetastore_drop_constraint_args(const ThriftHiveMetastore_drop_constraint_args&); + ThriftHiveMetastore_drop_constraint_args& operator=(const ThriftHiveMetastore_drop_constraint_args&); + ThriftHiveMetastore_drop_constraint_args() { + } + + virtual ~ThriftHiveMetastore_drop_constraint_args() noexcept; + DropConstraintRequest req; + + _ThriftHiveMetastore_drop_constraint_args__isset __isset; + + void __set_req(const DropConstraintRequest& val); + + bool operator == (const ThriftHiveMetastore_drop_constraint_args & rhs) const + { + if (!(req == rhs.req)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_constraint_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_constraint_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_drop_constraint_pargs { + public: + + + virtual ~ThriftHiveMetastore_drop_constraint_pargs() noexcept; + const DropConstraintRequest* req; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_constraint_result__isset { + _ThriftHiveMetastore_drop_constraint_result__isset() : o1(false), o3(false) {} + bool o1 :1; + bool o3 :1; +} _ThriftHiveMetastore_drop_constraint_result__isset; + +class ThriftHiveMetastore_drop_constraint_result { + public: + + ThriftHiveMetastore_drop_constraint_result(const ThriftHiveMetastore_drop_constraint_result&); + ThriftHiveMetastore_drop_constraint_result& operator=(const ThriftHiveMetastore_drop_constraint_result&); + ThriftHiveMetastore_drop_constraint_result() { + } + + virtual ~ThriftHiveMetastore_drop_constraint_result() noexcept; + NoSuchObjectException o1; + MetaException o3; + + _ThriftHiveMetastore_drop_constraint_result__isset __isset; + + void __set_o1(const NoSuchObjectException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_drop_constraint_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_constraint_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_constraint_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_constraint_presult__isset { + _ThriftHiveMetastore_drop_constraint_presult__isset() : o1(false), o3(false) {} + bool o1 :1; + bool o3 :1; +} _ThriftHiveMetastore_drop_constraint_presult__isset; + +class ThriftHiveMetastore_drop_constraint_presult { + public: + + + virtual ~ThriftHiveMetastore_drop_constraint_presult() noexcept; + NoSuchObjectException o1; + MetaException o3; + + _ThriftHiveMetastore_drop_constraint_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_add_primary_key_args__isset { + _ThriftHiveMetastore_add_primary_key_args__isset() : req(false) {} + bool req :1; +} _ThriftHiveMetastore_add_primary_key_args__isset; + +class ThriftHiveMetastore_add_primary_key_args { + public: + + ThriftHiveMetastore_add_primary_key_args(const ThriftHiveMetastore_add_primary_key_args&); + ThriftHiveMetastore_add_primary_key_args& operator=(const ThriftHiveMetastore_add_primary_key_args&); + ThriftHiveMetastore_add_primary_key_args() { + } + + virtual ~ThriftHiveMetastore_add_primary_key_args() noexcept; + AddPrimaryKeyRequest req; + + _ThriftHiveMetastore_add_primary_key_args__isset __isset; + + void __set_req(const AddPrimaryKeyRequest& val); + + bool operator == (const ThriftHiveMetastore_add_primary_key_args & rhs) const + { + if (!(req == rhs.req)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_primary_key_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_primary_key_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_add_primary_key_pargs { + public: + + + virtual ~ThriftHiveMetastore_add_primary_key_pargs() noexcept; + const AddPrimaryKeyRequest* req; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_primary_key_result__isset { + _ThriftHiveMetastore_add_primary_key_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_add_primary_key_result__isset; + +class ThriftHiveMetastore_add_primary_key_result { + public: + + ThriftHiveMetastore_add_primary_key_result(const ThriftHiveMetastore_add_primary_key_result&); + ThriftHiveMetastore_add_primary_key_result& operator=(const ThriftHiveMetastore_add_primary_key_result&); + ThriftHiveMetastore_add_primary_key_result() { + } + + virtual ~ThriftHiveMetastore_add_primary_key_result() noexcept; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_add_primary_key_result__isset __isset; + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_add_primary_key_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_primary_key_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_primary_key_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_primary_key_presult__isset { + _ThriftHiveMetastore_add_primary_key_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_add_primary_key_presult__isset; + +class ThriftHiveMetastore_add_primary_key_presult { + public: + + + virtual ~ThriftHiveMetastore_add_primary_key_presult() noexcept; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_add_primary_key_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_add_foreign_key_args__isset { + _ThriftHiveMetastore_add_foreign_key_args__isset() : req(false) {} + bool req :1; +} _ThriftHiveMetastore_add_foreign_key_args__isset; + +class ThriftHiveMetastore_add_foreign_key_args { + public: + + ThriftHiveMetastore_add_foreign_key_args(const ThriftHiveMetastore_add_foreign_key_args&); + ThriftHiveMetastore_add_foreign_key_args& operator=(const ThriftHiveMetastore_add_foreign_key_args&); + ThriftHiveMetastore_add_foreign_key_args() { + } + + virtual ~ThriftHiveMetastore_add_foreign_key_args() noexcept; + AddForeignKeyRequest req; + + _ThriftHiveMetastore_add_foreign_key_args__isset __isset; + + void __set_req(const AddForeignKeyRequest& val); + + bool operator == (const ThriftHiveMetastore_add_foreign_key_args & rhs) const + { + if (!(req == rhs.req)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_foreign_key_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_foreign_key_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_add_foreign_key_pargs { + public: + + + virtual ~ThriftHiveMetastore_add_foreign_key_pargs() noexcept; + const AddForeignKeyRequest* req; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_foreign_key_result__isset { + _ThriftHiveMetastore_add_foreign_key_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_add_foreign_key_result__isset; + +class ThriftHiveMetastore_add_foreign_key_result { + public: + + ThriftHiveMetastore_add_foreign_key_result(const ThriftHiveMetastore_add_foreign_key_result&); + ThriftHiveMetastore_add_foreign_key_result& operator=(const ThriftHiveMetastore_add_foreign_key_result&); + ThriftHiveMetastore_add_foreign_key_result() { + } + + virtual ~ThriftHiveMetastore_add_foreign_key_result() noexcept; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_add_foreign_key_result__isset __isset; + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_add_foreign_key_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_foreign_key_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_foreign_key_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_foreign_key_presult__isset { + _ThriftHiveMetastore_add_foreign_key_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_add_foreign_key_presult__isset; + +class ThriftHiveMetastore_add_foreign_key_presult { + public: + + + virtual ~ThriftHiveMetastore_add_foreign_key_presult() noexcept; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_add_foreign_key_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_add_unique_constraint_args__isset { + _ThriftHiveMetastore_add_unique_constraint_args__isset() : req(false) {} + bool req :1; +} _ThriftHiveMetastore_add_unique_constraint_args__isset; + +class ThriftHiveMetastore_add_unique_constraint_args { + public: + + ThriftHiveMetastore_add_unique_constraint_args(const ThriftHiveMetastore_add_unique_constraint_args&); + ThriftHiveMetastore_add_unique_constraint_args& operator=(const ThriftHiveMetastore_add_unique_constraint_args&); + ThriftHiveMetastore_add_unique_constraint_args() { + } + + virtual ~ThriftHiveMetastore_add_unique_constraint_args() noexcept; + AddUniqueConstraintRequest req; + + _ThriftHiveMetastore_add_unique_constraint_args__isset __isset; + + void __set_req(const AddUniqueConstraintRequest& val); + + bool operator == (const ThriftHiveMetastore_add_unique_constraint_args & rhs) const + { + if (!(req == rhs.req)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_unique_constraint_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_unique_constraint_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_add_unique_constraint_pargs { + public: + + + virtual ~ThriftHiveMetastore_add_unique_constraint_pargs() noexcept; + const AddUniqueConstraintRequest* req; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_unique_constraint_result__isset { + _ThriftHiveMetastore_add_unique_constraint_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_add_unique_constraint_result__isset; + +class ThriftHiveMetastore_add_unique_constraint_result { + public: + + ThriftHiveMetastore_add_unique_constraint_result(const ThriftHiveMetastore_add_unique_constraint_result&); + ThriftHiveMetastore_add_unique_constraint_result& operator=(const ThriftHiveMetastore_add_unique_constraint_result&); + ThriftHiveMetastore_add_unique_constraint_result() { + } + + virtual ~ThriftHiveMetastore_add_unique_constraint_result() noexcept; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_add_unique_constraint_result__isset __isset; + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_add_unique_constraint_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_unique_constraint_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_unique_constraint_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_unique_constraint_presult__isset { + _ThriftHiveMetastore_add_unique_constraint_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_add_unique_constraint_presult__isset; + +class ThriftHiveMetastore_add_unique_constraint_presult { + public: + + + virtual ~ThriftHiveMetastore_add_unique_constraint_presult() noexcept; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_add_unique_constraint_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_add_not_null_constraint_args__isset { + _ThriftHiveMetastore_add_not_null_constraint_args__isset() : req(false) {} + bool req :1; +} _ThriftHiveMetastore_add_not_null_constraint_args__isset; + +class ThriftHiveMetastore_add_not_null_constraint_args { + public: + + ThriftHiveMetastore_add_not_null_constraint_args(const ThriftHiveMetastore_add_not_null_constraint_args&); + ThriftHiveMetastore_add_not_null_constraint_args& operator=(const ThriftHiveMetastore_add_not_null_constraint_args&); + ThriftHiveMetastore_add_not_null_constraint_args() { + } + + virtual ~ThriftHiveMetastore_add_not_null_constraint_args() noexcept; + AddNotNullConstraintRequest req; + + _ThriftHiveMetastore_add_not_null_constraint_args__isset __isset; + + void __set_req(const AddNotNullConstraintRequest& val); + + bool operator == (const ThriftHiveMetastore_add_not_null_constraint_args & rhs) const + { + if (!(req == rhs.req)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_not_null_constraint_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_not_null_constraint_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_add_not_null_constraint_pargs { + public: + + + virtual ~ThriftHiveMetastore_add_not_null_constraint_pargs() noexcept; + const AddNotNullConstraintRequest* req; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_not_null_constraint_result__isset { + _ThriftHiveMetastore_add_not_null_constraint_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_add_not_null_constraint_result__isset; + +class ThriftHiveMetastore_add_not_null_constraint_result { + public: + + ThriftHiveMetastore_add_not_null_constraint_result(const ThriftHiveMetastore_add_not_null_constraint_result&); + ThriftHiveMetastore_add_not_null_constraint_result& operator=(const ThriftHiveMetastore_add_not_null_constraint_result&); + ThriftHiveMetastore_add_not_null_constraint_result() { + } + + virtual ~ThriftHiveMetastore_add_not_null_constraint_result() noexcept; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_add_not_null_constraint_result__isset __isset; + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_add_not_null_constraint_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_not_null_constraint_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_not_null_constraint_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_not_null_constraint_presult__isset { + _ThriftHiveMetastore_add_not_null_constraint_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_add_not_null_constraint_presult__isset; + +class ThriftHiveMetastore_add_not_null_constraint_presult { + public: + + + virtual ~ThriftHiveMetastore_add_not_null_constraint_presult() noexcept; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_add_not_null_constraint_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_drop_table_args__isset { + _ThriftHiveMetastore_drop_table_args__isset() : dbname(false), name(false), deleteData(false) {} + bool dbname :1; + bool name :1; + bool deleteData :1; +} _ThriftHiveMetastore_drop_table_args__isset; + +class ThriftHiveMetastore_drop_table_args { + public: + + ThriftHiveMetastore_drop_table_args(const ThriftHiveMetastore_drop_table_args&); + ThriftHiveMetastore_drop_table_args& operator=(const ThriftHiveMetastore_drop_table_args&); + ThriftHiveMetastore_drop_table_args() : dbname(), name(), deleteData(0) { + } + + virtual ~ThriftHiveMetastore_drop_table_args() noexcept; + std::string dbname; + std::string name; + bool deleteData; + + _ThriftHiveMetastore_drop_table_args__isset __isset; + + void __set_dbname(const std::string& val); + + void __set_name(const std::string& val); + + void __set_deleteData(const bool val); + + bool operator == (const ThriftHiveMetastore_drop_table_args & rhs) const + { + if (!(dbname == rhs.dbname)) + return false; + if (!(name == rhs.name)) + return false; + if (!(deleteData == rhs.deleteData)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_table_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_table_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_drop_table_pargs { + public: + + + virtual ~ThriftHiveMetastore_drop_table_pargs() noexcept; + const std::string* dbname; + const std::string* name; + const bool* deleteData; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_table_result__isset { + _ThriftHiveMetastore_drop_table_result__isset() : o1(false), o3(false) {} + bool o1 :1; + bool o3 :1; +} _ThriftHiveMetastore_drop_table_result__isset; + +class ThriftHiveMetastore_drop_table_result { + public: + + ThriftHiveMetastore_drop_table_result(const ThriftHiveMetastore_drop_table_result&); + ThriftHiveMetastore_drop_table_result& operator=(const ThriftHiveMetastore_drop_table_result&); + ThriftHiveMetastore_drop_table_result() { + } + + virtual ~ThriftHiveMetastore_drop_table_result() noexcept; + NoSuchObjectException o1; + MetaException o3; + + _ThriftHiveMetastore_drop_table_result__isset __isset; + + void __set_o1(const NoSuchObjectException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_drop_table_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_table_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_table_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_table_presult__isset { + _ThriftHiveMetastore_drop_table_presult__isset() : o1(false), o3(false) {} + bool o1 :1; + bool o3 :1; +} _ThriftHiveMetastore_drop_table_presult__isset; + +class ThriftHiveMetastore_drop_table_presult { + public: + + + virtual ~ThriftHiveMetastore_drop_table_presult() noexcept; + NoSuchObjectException o1; + MetaException o3; + + _ThriftHiveMetastore_drop_table_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_drop_table_with_environment_context_args__isset { + _ThriftHiveMetastore_drop_table_with_environment_context_args__isset() : dbname(false), name(false), deleteData(false), environment_context(false) {} + bool dbname :1; + bool name :1; + bool deleteData :1; + bool environment_context :1; +} _ThriftHiveMetastore_drop_table_with_environment_context_args__isset; + +class ThriftHiveMetastore_drop_table_with_environment_context_args { + public: + + ThriftHiveMetastore_drop_table_with_environment_context_args(const ThriftHiveMetastore_drop_table_with_environment_context_args&); + ThriftHiveMetastore_drop_table_with_environment_context_args& operator=(const ThriftHiveMetastore_drop_table_with_environment_context_args&); + ThriftHiveMetastore_drop_table_with_environment_context_args() : dbname(), name(), deleteData(0) { + } + + virtual ~ThriftHiveMetastore_drop_table_with_environment_context_args() noexcept; + std::string dbname; + std::string name; + bool deleteData; + EnvironmentContext environment_context; + + _ThriftHiveMetastore_drop_table_with_environment_context_args__isset __isset; + + void __set_dbname(const std::string& val); + + void __set_name(const std::string& val); + + void __set_deleteData(const bool val); + + void __set_environment_context(const EnvironmentContext& val); + + bool operator == (const ThriftHiveMetastore_drop_table_with_environment_context_args & rhs) const + { + if (!(dbname == rhs.dbname)) + return false; + if (!(name == rhs.name)) + return false; + if (!(deleteData == rhs.deleteData)) + return false; + if (!(environment_context == rhs.environment_context)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_table_with_environment_context_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_table_with_environment_context_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_drop_table_with_environment_context_pargs { + public: + + + virtual ~ThriftHiveMetastore_drop_table_with_environment_context_pargs() noexcept; + const std::string* dbname; + const std::string* name; + const bool* deleteData; + const EnvironmentContext* environment_context; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_table_with_environment_context_result__isset { + _ThriftHiveMetastore_drop_table_with_environment_context_result__isset() : o1(false), o3(false) {} + bool o1 :1; + bool o3 :1; +} _ThriftHiveMetastore_drop_table_with_environment_context_result__isset; + +class ThriftHiveMetastore_drop_table_with_environment_context_result { + public: + + ThriftHiveMetastore_drop_table_with_environment_context_result(const ThriftHiveMetastore_drop_table_with_environment_context_result&); + ThriftHiveMetastore_drop_table_with_environment_context_result& operator=(const ThriftHiveMetastore_drop_table_with_environment_context_result&); + ThriftHiveMetastore_drop_table_with_environment_context_result() { + } + + virtual ~ThriftHiveMetastore_drop_table_with_environment_context_result() noexcept; + NoSuchObjectException o1; + MetaException o3; + + _ThriftHiveMetastore_drop_table_with_environment_context_result__isset __isset; + + void __set_o1(const NoSuchObjectException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_drop_table_with_environment_context_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_table_with_environment_context_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_table_with_environment_context_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_table_with_environment_context_presult__isset { + _ThriftHiveMetastore_drop_table_with_environment_context_presult__isset() : o1(false), o3(false) {} + bool o1 :1; + bool o3 :1; +} _ThriftHiveMetastore_drop_table_with_environment_context_presult__isset; + +class ThriftHiveMetastore_drop_table_with_environment_context_presult { + public: + + + virtual ~ThriftHiveMetastore_drop_table_with_environment_context_presult() noexcept; + NoSuchObjectException o1; + MetaException o3; + + _ThriftHiveMetastore_drop_table_with_environment_context_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_truncate_table_args__isset { + _ThriftHiveMetastore_truncate_table_args__isset() : dbName(false), tableName(false), partNames(false) {} + bool dbName :1; + bool tableName :1; + bool partNames :1; +} _ThriftHiveMetastore_truncate_table_args__isset; + +class ThriftHiveMetastore_truncate_table_args { + public: + + ThriftHiveMetastore_truncate_table_args(const ThriftHiveMetastore_truncate_table_args&); + ThriftHiveMetastore_truncate_table_args& operator=(const ThriftHiveMetastore_truncate_table_args&); + ThriftHiveMetastore_truncate_table_args() : dbName(), tableName() { + } + + virtual ~ThriftHiveMetastore_truncate_table_args() noexcept; + std::string dbName; + std::string tableName; + std::vector partNames; + + _ThriftHiveMetastore_truncate_table_args__isset __isset; + + void __set_dbName(const std::string& val); + + void __set_tableName(const std::string& val); + + void __set_partNames(const std::vector & val); + + bool operator == (const ThriftHiveMetastore_truncate_table_args & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(tableName == rhs.tableName)) + return false; + if (!(partNames == rhs.partNames)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_truncate_table_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_truncate_table_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_truncate_table_pargs { + public: + + + virtual ~ThriftHiveMetastore_truncate_table_pargs() noexcept; + const std::string* dbName; + const std::string* tableName; + const std::vector * partNames; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_truncate_table_result__isset { + _ThriftHiveMetastore_truncate_table_result__isset() : o1(false) {} + bool o1 :1; +} _ThriftHiveMetastore_truncate_table_result__isset; + +class ThriftHiveMetastore_truncate_table_result { + public: + + ThriftHiveMetastore_truncate_table_result(const ThriftHiveMetastore_truncate_table_result&); + ThriftHiveMetastore_truncate_table_result& operator=(const ThriftHiveMetastore_truncate_table_result&); + ThriftHiveMetastore_truncate_table_result() { + } + + virtual ~ThriftHiveMetastore_truncate_table_result() noexcept; + MetaException o1; + + _ThriftHiveMetastore_truncate_table_result__isset __isset; + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_truncate_table_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_truncate_table_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_truncate_table_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_truncate_table_presult__isset { + _ThriftHiveMetastore_truncate_table_presult__isset() : o1(false) {} + bool o1 :1; +} _ThriftHiveMetastore_truncate_table_presult__isset; + +class ThriftHiveMetastore_truncate_table_presult { + public: + + + virtual ~ThriftHiveMetastore_truncate_table_presult() noexcept; + MetaException o1; + + _ThriftHiveMetastore_truncate_table_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_tables_args__isset { + _ThriftHiveMetastore_get_tables_args__isset() : db_name(false), pattern(false) {} + bool db_name :1; + bool pattern :1; +} _ThriftHiveMetastore_get_tables_args__isset; + +class ThriftHiveMetastore_get_tables_args { + public: + + ThriftHiveMetastore_get_tables_args(const ThriftHiveMetastore_get_tables_args&); + ThriftHiveMetastore_get_tables_args& operator=(const ThriftHiveMetastore_get_tables_args&); + ThriftHiveMetastore_get_tables_args() : db_name(), pattern() { + } + + virtual ~ThriftHiveMetastore_get_tables_args() noexcept; + std::string db_name; + std::string pattern; + + _ThriftHiveMetastore_get_tables_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_pattern(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_tables_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(pattern == rhs.pattern)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_tables_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_tables_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_tables_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_tables_pargs() noexcept; + const std::string* db_name; + const std::string* pattern; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_tables_result__isset { + _ThriftHiveMetastore_get_tables_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_tables_result__isset; + +class ThriftHiveMetastore_get_tables_result { + public: + + ThriftHiveMetastore_get_tables_result(const ThriftHiveMetastore_get_tables_result&); + ThriftHiveMetastore_get_tables_result& operator=(const ThriftHiveMetastore_get_tables_result&); + ThriftHiveMetastore_get_tables_result() { + } + + virtual ~ThriftHiveMetastore_get_tables_result() noexcept; + std::vector success; + MetaException o1; + + _ThriftHiveMetastore_get_tables_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_tables_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_tables_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_tables_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_tables_presult__isset { + _ThriftHiveMetastore_get_tables_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_tables_presult__isset; + +class ThriftHiveMetastore_get_tables_presult { + public: + + + virtual ~ThriftHiveMetastore_get_tables_presult() noexcept; + std::vector * success; + MetaException o1; + + _ThriftHiveMetastore_get_tables_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_tables_by_type_args__isset { + _ThriftHiveMetastore_get_tables_by_type_args__isset() : db_name(false), pattern(false), tableType(false) {} + bool db_name :1; + bool pattern :1; + bool tableType :1; +} _ThriftHiveMetastore_get_tables_by_type_args__isset; + +class ThriftHiveMetastore_get_tables_by_type_args { + public: + + ThriftHiveMetastore_get_tables_by_type_args(const ThriftHiveMetastore_get_tables_by_type_args&); + ThriftHiveMetastore_get_tables_by_type_args& operator=(const ThriftHiveMetastore_get_tables_by_type_args&); + ThriftHiveMetastore_get_tables_by_type_args() : db_name(), pattern(), tableType() { + } + + virtual ~ThriftHiveMetastore_get_tables_by_type_args() noexcept; + std::string db_name; + std::string pattern; + std::string tableType; + + _ThriftHiveMetastore_get_tables_by_type_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_pattern(const std::string& val); + + void __set_tableType(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_tables_by_type_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(pattern == rhs.pattern)) + return false; + if (!(tableType == rhs.tableType)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_tables_by_type_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_tables_by_type_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_tables_by_type_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_tables_by_type_pargs() noexcept; + const std::string* db_name; + const std::string* pattern; + const std::string* tableType; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_tables_by_type_result__isset { + _ThriftHiveMetastore_get_tables_by_type_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_tables_by_type_result__isset; + +class ThriftHiveMetastore_get_tables_by_type_result { + public: + + ThriftHiveMetastore_get_tables_by_type_result(const ThriftHiveMetastore_get_tables_by_type_result&); + ThriftHiveMetastore_get_tables_by_type_result& operator=(const ThriftHiveMetastore_get_tables_by_type_result&); + ThriftHiveMetastore_get_tables_by_type_result() { + } + + virtual ~ThriftHiveMetastore_get_tables_by_type_result() noexcept; + std::vector success; + MetaException o1; + + _ThriftHiveMetastore_get_tables_by_type_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_tables_by_type_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_tables_by_type_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_tables_by_type_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_tables_by_type_presult__isset { + _ThriftHiveMetastore_get_tables_by_type_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_tables_by_type_presult__isset; + +class ThriftHiveMetastore_get_tables_by_type_presult { + public: + + + virtual ~ThriftHiveMetastore_get_tables_by_type_presult() noexcept; + std::vector * success; + MetaException o1; + + _ThriftHiveMetastore_get_tables_by_type_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_table_meta_args__isset { + _ThriftHiveMetastore_get_table_meta_args__isset() : db_patterns(false), tbl_patterns(false), tbl_types(false) {} + bool db_patterns :1; + bool tbl_patterns :1; + bool tbl_types :1; +} _ThriftHiveMetastore_get_table_meta_args__isset; + +class ThriftHiveMetastore_get_table_meta_args { + public: + + ThriftHiveMetastore_get_table_meta_args(const ThriftHiveMetastore_get_table_meta_args&); + ThriftHiveMetastore_get_table_meta_args& operator=(const ThriftHiveMetastore_get_table_meta_args&); + ThriftHiveMetastore_get_table_meta_args() : db_patterns(), tbl_patterns() { + } + + virtual ~ThriftHiveMetastore_get_table_meta_args() noexcept; + std::string db_patterns; + std::string tbl_patterns; + std::vector tbl_types; + + _ThriftHiveMetastore_get_table_meta_args__isset __isset; + + void __set_db_patterns(const std::string& val); + + void __set_tbl_patterns(const std::string& val); + + void __set_tbl_types(const std::vector & val); + + bool operator == (const ThriftHiveMetastore_get_table_meta_args & rhs) const + { + if (!(db_patterns == rhs.db_patterns)) + return false; + if (!(tbl_patterns == rhs.tbl_patterns)) + return false; + if (!(tbl_types == rhs.tbl_types)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_table_meta_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_table_meta_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_table_meta_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_table_meta_pargs() noexcept; + const std::string* db_patterns; + const std::string* tbl_patterns; + const std::vector * tbl_types; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_table_meta_result__isset { + _ThriftHiveMetastore_get_table_meta_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_table_meta_result__isset; + +class ThriftHiveMetastore_get_table_meta_result { + public: + + ThriftHiveMetastore_get_table_meta_result(const ThriftHiveMetastore_get_table_meta_result&); + ThriftHiveMetastore_get_table_meta_result& operator=(const ThriftHiveMetastore_get_table_meta_result&); + ThriftHiveMetastore_get_table_meta_result() { + } + + virtual ~ThriftHiveMetastore_get_table_meta_result() noexcept; + std::vector success; + MetaException o1; + + _ThriftHiveMetastore_get_table_meta_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_table_meta_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_table_meta_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_table_meta_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_table_meta_presult__isset { + _ThriftHiveMetastore_get_table_meta_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_table_meta_presult__isset; + +class ThriftHiveMetastore_get_table_meta_presult { + public: + + + virtual ~ThriftHiveMetastore_get_table_meta_presult() noexcept; + std::vector * success; + MetaException o1; + + _ThriftHiveMetastore_get_table_meta_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_all_tables_args__isset { + _ThriftHiveMetastore_get_all_tables_args__isset() : db_name(false) {} + bool db_name :1; +} _ThriftHiveMetastore_get_all_tables_args__isset; + +class ThriftHiveMetastore_get_all_tables_args { + public: + + ThriftHiveMetastore_get_all_tables_args(const ThriftHiveMetastore_get_all_tables_args&); + ThriftHiveMetastore_get_all_tables_args& operator=(const ThriftHiveMetastore_get_all_tables_args&); + ThriftHiveMetastore_get_all_tables_args() : db_name() { + } + + virtual ~ThriftHiveMetastore_get_all_tables_args() noexcept; + std::string db_name; + + _ThriftHiveMetastore_get_all_tables_args__isset __isset; + + void __set_db_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_all_tables_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_all_tables_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_all_tables_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_all_tables_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_all_tables_pargs() noexcept; + const std::string* db_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_all_tables_result__isset { + _ThriftHiveMetastore_get_all_tables_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_all_tables_result__isset; + +class ThriftHiveMetastore_get_all_tables_result { + public: + + ThriftHiveMetastore_get_all_tables_result(const ThriftHiveMetastore_get_all_tables_result&); + ThriftHiveMetastore_get_all_tables_result& operator=(const ThriftHiveMetastore_get_all_tables_result&); + ThriftHiveMetastore_get_all_tables_result() { + } + + virtual ~ThriftHiveMetastore_get_all_tables_result() noexcept; + std::vector success; + MetaException o1; + + _ThriftHiveMetastore_get_all_tables_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_all_tables_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_all_tables_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_all_tables_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_all_tables_presult__isset { + _ThriftHiveMetastore_get_all_tables_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_all_tables_presult__isset; + +class ThriftHiveMetastore_get_all_tables_presult { + public: + + + virtual ~ThriftHiveMetastore_get_all_tables_presult() noexcept; + std::vector * success; + MetaException o1; + + _ThriftHiveMetastore_get_all_tables_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_table_args__isset { + _ThriftHiveMetastore_get_table_args__isset() : dbname(false), tbl_name(false) {} + bool dbname :1; + bool tbl_name :1; +} _ThriftHiveMetastore_get_table_args__isset; + +class ThriftHiveMetastore_get_table_args { + public: + + ThriftHiveMetastore_get_table_args(const ThriftHiveMetastore_get_table_args&); + ThriftHiveMetastore_get_table_args& operator=(const ThriftHiveMetastore_get_table_args&); + ThriftHiveMetastore_get_table_args() : dbname(), tbl_name() { + } + + virtual ~ThriftHiveMetastore_get_table_args() noexcept; + std::string dbname; + std::string tbl_name; + + _ThriftHiveMetastore_get_table_args__isset __isset; + + void __set_dbname(const std::string& val); + + void __set_tbl_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_table_args & rhs) const + { + if (!(dbname == rhs.dbname)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_table_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_table_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_table_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_table_pargs() noexcept; + const std::string* dbname; + const std::string* tbl_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_table_result__isset { + _ThriftHiveMetastore_get_table_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_table_result__isset; + +class ThriftHiveMetastore_get_table_result { + public: + + ThriftHiveMetastore_get_table_result(const ThriftHiveMetastore_get_table_result&); + ThriftHiveMetastore_get_table_result& operator=(const ThriftHiveMetastore_get_table_result&); + ThriftHiveMetastore_get_table_result() { + } + + virtual ~ThriftHiveMetastore_get_table_result() noexcept; + Table success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_table_result__isset __isset; + + void __set_success(const Table& val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_table_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_table_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_table_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_table_presult__isset { + _ThriftHiveMetastore_get_table_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_table_presult__isset; + +class ThriftHiveMetastore_get_table_presult { + public: + + + virtual ~ThriftHiveMetastore_get_table_presult() noexcept; + Table* success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_table_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_table_objects_by_name_args__isset { + _ThriftHiveMetastore_get_table_objects_by_name_args__isset() : dbname(false), tbl_names(false) {} + bool dbname :1; + bool tbl_names :1; +} _ThriftHiveMetastore_get_table_objects_by_name_args__isset; + +class ThriftHiveMetastore_get_table_objects_by_name_args { + public: + + ThriftHiveMetastore_get_table_objects_by_name_args(const ThriftHiveMetastore_get_table_objects_by_name_args&); + ThriftHiveMetastore_get_table_objects_by_name_args& operator=(const ThriftHiveMetastore_get_table_objects_by_name_args&); + ThriftHiveMetastore_get_table_objects_by_name_args() : dbname() { + } + + virtual ~ThriftHiveMetastore_get_table_objects_by_name_args() noexcept; + std::string dbname; + std::vector tbl_names; + + _ThriftHiveMetastore_get_table_objects_by_name_args__isset __isset; + + void __set_dbname(const std::string& val); + + void __set_tbl_names(const std::vector & val); + + bool operator == (const ThriftHiveMetastore_get_table_objects_by_name_args & rhs) const + { + if (!(dbname == rhs.dbname)) + return false; + if (!(tbl_names == rhs.tbl_names)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_table_objects_by_name_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_table_objects_by_name_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_table_objects_by_name_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_table_objects_by_name_pargs() noexcept; + const std::string* dbname; + const std::vector * tbl_names; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_table_objects_by_name_result__isset { + _ThriftHiveMetastore_get_table_objects_by_name_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_table_objects_by_name_result__isset; + +class ThriftHiveMetastore_get_table_objects_by_name_result { + public: + + ThriftHiveMetastore_get_table_objects_by_name_result(const ThriftHiveMetastore_get_table_objects_by_name_result&); + ThriftHiveMetastore_get_table_objects_by_name_result& operator=(const ThriftHiveMetastore_get_table_objects_by_name_result&); + ThriftHiveMetastore_get_table_objects_by_name_result() { + } + + virtual ~ThriftHiveMetastore_get_table_objects_by_name_result() noexcept; + std::vector
success; + + _ThriftHiveMetastore_get_table_objects_by_name_result__isset __isset; + + void __set_success(const std::vector
& val); + + bool operator == (const ThriftHiveMetastore_get_table_objects_by_name_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_table_objects_by_name_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_table_objects_by_name_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_table_objects_by_name_presult__isset { + _ThriftHiveMetastore_get_table_objects_by_name_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_table_objects_by_name_presult__isset; + +class ThriftHiveMetastore_get_table_objects_by_name_presult { + public: + + + virtual ~ThriftHiveMetastore_get_table_objects_by_name_presult() noexcept; + std::vector
* success; + + _ThriftHiveMetastore_get_table_objects_by_name_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_table_req_args__isset { + _ThriftHiveMetastore_get_table_req_args__isset() : req(false) {} + bool req :1; +} _ThriftHiveMetastore_get_table_req_args__isset; + +class ThriftHiveMetastore_get_table_req_args { + public: + + ThriftHiveMetastore_get_table_req_args(const ThriftHiveMetastore_get_table_req_args&); + ThriftHiveMetastore_get_table_req_args& operator=(const ThriftHiveMetastore_get_table_req_args&); + ThriftHiveMetastore_get_table_req_args() { + } + + virtual ~ThriftHiveMetastore_get_table_req_args() noexcept; + GetTableRequest req; + + _ThriftHiveMetastore_get_table_req_args__isset __isset; + + void __set_req(const GetTableRequest& val); + + bool operator == (const ThriftHiveMetastore_get_table_req_args & rhs) const + { + if (!(req == rhs.req)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_table_req_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_table_req_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_table_req_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_table_req_pargs() noexcept; + const GetTableRequest* req; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_table_req_result__isset { + _ThriftHiveMetastore_get_table_req_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_table_req_result__isset; + +class ThriftHiveMetastore_get_table_req_result { + public: + + ThriftHiveMetastore_get_table_req_result(const ThriftHiveMetastore_get_table_req_result&); + ThriftHiveMetastore_get_table_req_result& operator=(const ThriftHiveMetastore_get_table_req_result&); + ThriftHiveMetastore_get_table_req_result() { + } + + virtual ~ThriftHiveMetastore_get_table_req_result() noexcept; + GetTableResult success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_table_req_result__isset __isset; + + void __set_success(const GetTableResult& val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_table_req_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_table_req_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_table_req_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_table_req_presult__isset { + _ThriftHiveMetastore_get_table_req_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_table_req_presult__isset; + +class ThriftHiveMetastore_get_table_req_presult { + public: + + + virtual ~ThriftHiveMetastore_get_table_req_presult() noexcept; + GetTableResult* success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_table_req_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_table_objects_by_name_req_args__isset { + _ThriftHiveMetastore_get_table_objects_by_name_req_args__isset() : req(false) {} + bool req :1; +} _ThriftHiveMetastore_get_table_objects_by_name_req_args__isset; + +class ThriftHiveMetastore_get_table_objects_by_name_req_args { + public: + + ThriftHiveMetastore_get_table_objects_by_name_req_args(const ThriftHiveMetastore_get_table_objects_by_name_req_args&); + ThriftHiveMetastore_get_table_objects_by_name_req_args& operator=(const ThriftHiveMetastore_get_table_objects_by_name_req_args&); + ThriftHiveMetastore_get_table_objects_by_name_req_args() { + } + + virtual ~ThriftHiveMetastore_get_table_objects_by_name_req_args() noexcept; + GetTablesRequest req; + + _ThriftHiveMetastore_get_table_objects_by_name_req_args__isset __isset; + + void __set_req(const GetTablesRequest& val); + + bool operator == (const ThriftHiveMetastore_get_table_objects_by_name_req_args & rhs) const + { + if (!(req == rhs.req)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_table_objects_by_name_req_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_table_objects_by_name_req_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_table_objects_by_name_req_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_table_objects_by_name_req_pargs() noexcept; + const GetTablesRequest* req; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_table_objects_by_name_req_result__isset { + _ThriftHiveMetastore_get_table_objects_by_name_req_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_get_table_objects_by_name_req_result__isset; + +class ThriftHiveMetastore_get_table_objects_by_name_req_result { + public: + + ThriftHiveMetastore_get_table_objects_by_name_req_result(const ThriftHiveMetastore_get_table_objects_by_name_req_result&); + ThriftHiveMetastore_get_table_objects_by_name_req_result& operator=(const ThriftHiveMetastore_get_table_objects_by_name_req_result&); + ThriftHiveMetastore_get_table_objects_by_name_req_result() { + } + + virtual ~ThriftHiveMetastore_get_table_objects_by_name_req_result() noexcept; + GetTablesResult success; + MetaException o1; + InvalidOperationException o2; + UnknownDBException o3; + + _ThriftHiveMetastore_get_table_objects_by_name_req_result__isset __isset; + + void __set_success(const GetTablesResult& val); + + void __set_o1(const MetaException& val); + + void __set_o2(const InvalidOperationException& val); + + void __set_o3(const UnknownDBException& val); + + bool operator == (const ThriftHiveMetastore_get_table_objects_by_name_req_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_table_objects_by_name_req_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_table_objects_by_name_req_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_table_objects_by_name_req_presult__isset { + _ThriftHiveMetastore_get_table_objects_by_name_req_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_get_table_objects_by_name_req_presult__isset; + +class ThriftHiveMetastore_get_table_objects_by_name_req_presult { + public: + + + virtual ~ThriftHiveMetastore_get_table_objects_by_name_req_presult() noexcept; + GetTablesResult* success; + MetaException o1; + InvalidOperationException o2; + UnknownDBException o3; + + _ThriftHiveMetastore_get_table_objects_by_name_req_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_table_names_by_filter_args__isset { + _ThriftHiveMetastore_get_table_names_by_filter_args__isset() : dbname(false), filter(false), max_tables(true) {} + bool dbname :1; + bool filter :1; + bool max_tables :1; +} _ThriftHiveMetastore_get_table_names_by_filter_args__isset; + +class ThriftHiveMetastore_get_table_names_by_filter_args { + public: + + ThriftHiveMetastore_get_table_names_by_filter_args(const ThriftHiveMetastore_get_table_names_by_filter_args&); + ThriftHiveMetastore_get_table_names_by_filter_args& operator=(const ThriftHiveMetastore_get_table_names_by_filter_args&); + ThriftHiveMetastore_get_table_names_by_filter_args() : dbname(), filter(), max_tables(-1) { + } + + virtual ~ThriftHiveMetastore_get_table_names_by_filter_args() noexcept; + std::string dbname; + std::string filter; + int16_t max_tables; + + _ThriftHiveMetastore_get_table_names_by_filter_args__isset __isset; + + void __set_dbname(const std::string& val); + + void __set_filter(const std::string& val); + + void __set_max_tables(const int16_t val); + + bool operator == (const ThriftHiveMetastore_get_table_names_by_filter_args & rhs) const + { + if (!(dbname == rhs.dbname)) + return false; + if (!(filter == rhs.filter)) + return false; + if (!(max_tables == rhs.max_tables)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_table_names_by_filter_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_table_names_by_filter_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_table_names_by_filter_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_table_names_by_filter_pargs() noexcept; + const std::string* dbname; + const std::string* filter; + const int16_t* max_tables; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_table_names_by_filter_result__isset { + _ThriftHiveMetastore_get_table_names_by_filter_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_get_table_names_by_filter_result__isset; + +class ThriftHiveMetastore_get_table_names_by_filter_result { + public: + + ThriftHiveMetastore_get_table_names_by_filter_result(const ThriftHiveMetastore_get_table_names_by_filter_result&); + ThriftHiveMetastore_get_table_names_by_filter_result& operator=(const ThriftHiveMetastore_get_table_names_by_filter_result&); + ThriftHiveMetastore_get_table_names_by_filter_result() { + } + + virtual ~ThriftHiveMetastore_get_table_names_by_filter_result() noexcept; + std::vector success; + MetaException o1; + InvalidOperationException o2; + UnknownDBException o3; + + _ThriftHiveMetastore_get_table_names_by_filter_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + void __set_o2(const InvalidOperationException& val); + + void __set_o3(const UnknownDBException& val); + + bool operator == (const ThriftHiveMetastore_get_table_names_by_filter_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_table_names_by_filter_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_table_names_by_filter_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_table_names_by_filter_presult__isset { + _ThriftHiveMetastore_get_table_names_by_filter_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_get_table_names_by_filter_presult__isset; + +class ThriftHiveMetastore_get_table_names_by_filter_presult { + public: + + + virtual ~ThriftHiveMetastore_get_table_names_by_filter_presult() noexcept; + std::vector * success; + MetaException o1; + InvalidOperationException o2; + UnknownDBException o3; + + _ThriftHiveMetastore_get_table_names_by_filter_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_alter_table_args__isset { + _ThriftHiveMetastore_alter_table_args__isset() : dbname(false), tbl_name(false), new_tbl(false) {} + bool dbname :1; + bool tbl_name :1; + bool new_tbl :1; +} _ThriftHiveMetastore_alter_table_args__isset; + +class ThriftHiveMetastore_alter_table_args { + public: + + ThriftHiveMetastore_alter_table_args(const ThriftHiveMetastore_alter_table_args&); + ThriftHiveMetastore_alter_table_args& operator=(const ThriftHiveMetastore_alter_table_args&); + ThriftHiveMetastore_alter_table_args() : dbname(), tbl_name() { + } + + virtual ~ThriftHiveMetastore_alter_table_args() noexcept; + std::string dbname; + std::string tbl_name; + Table new_tbl; + + _ThriftHiveMetastore_alter_table_args__isset __isset; + + void __set_dbname(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_new_tbl(const Table& val); + + bool operator == (const ThriftHiveMetastore_alter_table_args & rhs) const + { + if (!(dbname == rhs.dbname)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(new_tbl == rhs.new_tbl)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_table_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_table_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_alter_table_pargs { + public: + + + virtual ~ThriftHiveMetastore_alter_table_pargs() noexcept; + const std::string* dbname; + const std::string* tbl_name; + const Table* new_tbl; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_table_result__isset { + _ThriftHiveMetastore_alter_table_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_table_result__isset; + +class ThriftHiveMetastore_alter_table_result { + public: + + ThriftHiveMetastore_alter_table_result(const ThriftHiveMetastore_alter_table_result&); + ThriftHiveMetastore_alter_table_result& operator=(const ThriftHiveMetastore_alter_table_result&); + ThriftHiveMetastore_alter_table_result() { + } + + virtual ~ThriftHiveMetastore_alter_table_result() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_table_result__isset __isset; + + void __set_o1(const InvalidOperationException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_alter_table_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_table_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_table_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_table_presult__isset { + _ThriftHiveMetastore_alter_table_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_table_presult__isset; + +class ThriftHiveMetastore_alter_table_presult { + public: + + + virtual ~ThriftHiveMetastore_alter_table_presult() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_table_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_alter_table_with_environment_context_args__isset { + _ThriftHiveMetastore_alter_table_with_environment_context_args__isset() : dbname(false), tbl_name(false), new_tbl(false), environment_context(false) {} + bool dbname :1; + bool tbl_name :1; + bool new_tbl :1; + bool environment_context :1; +} _ThriftHiveMetastore_alter_table_with_environment_context_args__isset; + +class ThriftHiveMetastore_alter_table_with_environment_context_args { + public: + + ThriftHiveMetastore_alter_table_with_environment_context_args(const ThriftHiveMetastore_alter_table_with_environment_context_args&); + ThriftHiveMetastore_alter_table_with_environment_context_args& operator=(const ThriftHiveMetastore_alter_table_with_environment_context_args&); + ThriftHiveMetastore_alter_table_with_environment_context_args() : dbname(), tbl_name() { + } + + virtual ~ThriftHiveMetastore_alter_table_with_environment_context_args() noexcept; + std::string dbname; + std::string tbl_name; + Table new_tbl; + EnvironmentContext environment_context; + + _ThriftHiveMetastore_alter_table_with_environment_context_args__isset __isset; + + void __set_dbname(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_new_tbl(const Table& val); + + void __set_environment_context(const EnvironmentContext& val); + + bool operator == (const ThriftHiveMetastore_alter_table_with_environment_context_args & rhs) const + { + if (!(dbname == rhs.dbname)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(new_tbl == rhs.new_tbl)) + return false; + if (!(environment_context == rhs.environment_context)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_table_with_environment_context_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_table_with_environment_context_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_alter_table_with_environment_context_pargs { + public: + + + virtual ~ThriftHiveMetastore_alter_table_with_environment_context_pargs() noexcept; + const std::string* dbname; + const std::string* tbl_name; + const Table* new_tbl; + const EnvironmentContext* environment_context; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_table_with_environment_context_result__isset { + _ThriftHiveMetastore_alter_table_with_environment_context_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_table_with_environment_context_result__isset; + +class ThriftHiveMetastore_alter_table_with_environment_context_result { + public: + + ThriftHiveMetastore_alter_table_with_environment_context_result(const ThriftHiveMetastore_alter_table_with_environment_context_result&); + ThriftHiveMetastore_alter_table_with_environment_context_result& operator=(const ThriftHiveMetastore_alter_table_with_environment_context_result&); + ThriftHiveMetastore_alter_table_with_environment_context_result() { + } + + virtual ~ThriftHiveMetastore_alter_table_with_environment_context_result() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_table_with_environment_context_result__isset __isset; + + void __set_o1(const InvalidOperationException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_alter_table_with_environment_context_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_table_with_environment_context_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_table_with_environment_context_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_table_with_environment_context_presult__isset { + _ThriftHiveMetastore_alter_table_with_environment_context_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_table_with_environment_context_presult__isset; + +class ThriftHiveMetastore_alter_table_with_environment_context_presult { + public: + + + virtual ~ThriftHiveMetastore_alter_table_with_environment_context_presult() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_table_with_environment_context_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_alter_table_with_cascade_args__isset { + _ThriftHiveMetastore_alter_table_with_cascade_args__isset() : dbname(false), tbl_name(false), new_tbl(false), cascade(false) {} + bool dbname :1; + bool tbl_name :1; + bool new_tbl :1; + bool cascade :1; +} _ThriftHiveMetastore_alter_table_with_cascade_args__isset; + +class ThriftHiveMetastore_alter_table_with_cascade_args { + public: + + ThriftHiveMetastore_alter_table_with_cascade_args(const ThriftHiveMetastore_alter_table_with_cascade_args&); + ThriftHiveMetastore_alter_table_with_cascade_args& operator=(const ThriftHiveMetastore_alter_table_with_cascade_args&); + ThriftHiveMetastore_alter_table_with_cascade_args() : dbname(), tbl_name(), cascade(0) { + } + + virtual ~ThriftHiveMetastore_alter_table_with_cascade_args() noexcept; + std::string dbname; + std::string tbl_name; + Table new_tbl; + bool cascade; + + _ThriftHiveMetastore_alter_table_with_cascade_args__isset __isset; + + void __set_dbname(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_new_tbl(const Table& val); + + void __set_cascade(const bool val); + + bool operator == (const ThriftHiveMetastore_alter_table_with_cascade_args & rhs) const + { + if (!(dbname == rhs.dbname)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(new_tbl == rhs.new_tbl)) + return false; + if (!(cascade == rhs.cascade)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_table_with_cascade_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_table_with_cascade_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_alter_table_with_cascade_pargs { + public: + + + virtual ~ThriftHiveMetastore_alter_table_with_cascade_pargs() noexcept; + const std::string* dbname; + const std::string* tbl_name; + const Table* new_tbl; + const bool* cascade; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_table_with_cascade_result__isset { + _ThriftHiveMetastore_alter_table_with_cascade_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_table_with_cascade_result__isset; + +class ThriftHiveMetastore_alter_table_with_cascade_result { + public: + + ThriftHiveMetastore_alter_table_with_cascade_result(const ThriftHiveMetastore_alter_table_with_cascade_result&); + ThriftHiveMetastore_alter_table_with_cascade_result& operator=(const ThriftHiveMetastore_alter_table_with_cascade_result&); + ThriftHiveMetastore_alter_table_with_cascade_result() { + } + + virtual ~ThriftHiveMetastore_alter_table_with_cascade_result() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_table_with_cascade_result__isset __isset; + + void __set_o1(const InvalidOperationException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_alter_table_with_cascade_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_table_with_cascade_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_table_with_cascade_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_table_with_cascade_presult__isset { + _ThriftHiveMetastore_alter_table_with_cascade_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_table_with_cascade_presult__isset; + +class ThriftHiveMetastore_alter_table_with_cascade_presult { + public: + + + virtual ~ThriftHiveMetastore_alter_table_with_cascade_presult() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_table_with_cascade_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_add_partition_args__isset { + _ThriftHiveMetastore_add_partition_args__isset() : new_part(false) {} + bool new_part :1; +} _ThriftHiveMetastore_add_partition_args__isset; + +class ThriftHiveMetastore_add_partition_args { + public: + + ThriftHiveMetastore_add_partition_args(const ThriftHiveMetastore_add_partition_args&); + ThriftHiveMetastore_add_partition_args& operator=(const ThriftHiveMetastore_add_partition_args&); + ThriftHiveMetastore_add_partition_args() { + } + + virtual ~ThriftHiveMetastore_add_partition_args() noexcept; + Partition new_part; + + _ThriftHiveMetastore_add_partition_args__isset __isset; + + void __set_new_part(const Partition& val); + + bool operator == (const ThriftHiveMetastore_add_partition_args & rhs) const + { + if (!(new_part == rhs.new_part)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_partition_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_partition_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_add_partition_pargs { + public: + + + virtual ~ThriftHiveMetastore_add_partition_pargs() noexcept; + const Partition* new_part; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_partition_result__isset { + _ThriftHiveMetastore_add_partition_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_add_partition_result__isset; + +class ThriftHiveMetastore_add_partition_result { + public: + + ThriftHiveMetastore_add_partition_result(const ThriftHiveMetastore_add_partition_result&); + ThriftHiveMetastore_add_partition_result& operator=(const ThriftHiveMetastore_add_partition_result&); + ThriftHiveMetastore_add_partition_result() { + } + + virtual ~ThriftHiveMetastore_add_partition_result() noexcept; + Partition success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_add_partition_result__isset __isset; + + void __set_success(const Partition& val); + + void __set_o1(const InvalidObjectException& val); + + void __set_o2(const AlreadyExistsException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_add_partition_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_partition_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_partition_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_partition_presult__isset { + _ThriftHiveMetastore_add_partition_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_add_partition_presult__isset; + +class ThriftHiveMetastore_add_partition_presult { + public: + + + virtual ~ThriftHiveMetastore_add_partition_presult() noexcept; + Partition* success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_add_partition_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_add_partition_with_environment_context_args__isset { + _ThriftHiveMetastore_add_partition_with_environment_context_args__isset() : new_part(false), environment_context(false) {} + bool new_part :1; + bool environment_context :1; +} _ThriftHiveMetastore_add_partition_with_environment_context_args__isset; + +class ThriftHiveMetastore_add_partition_with_environment_context_args { + public: + + ThriftHiveMetastore_add_partition_with_environment_context_args(const ThriftHiveMetastore_add_partition_with_environment_context_args&); + ThriftHiveMetastore_add_partition_with_environment_context_args& operator=(const ThriftHiveMetastore_add_partition_with_environment_context_args&); + ThriftHiveMetastore_add_partition_with_environment_context_args() { + } + + virtual ~ThriftHiveMetastore_add_partition_with_environment_context_args() noexcept; + Partition new_part; + EnvironmentContext environment_context; + + _ThriftHiveMetastore_add_partition_with_environment_context_args__isset __isset; + + void __set_new_part(const Partition& val); + + void __set_environment_context(const EnvironmentContext& val); + + bool operator == (const ThriftHiveMetastore_add_partition_with_environment_context_args & rhs) const + { + if (!(new_part == rhs.new_part)) + return false; + if (!(environment_context == rhs.environment_context)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_partition_with_environment_context_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_partition_with_environment_context_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_add_partition_with_environment_context_pargs { + public: + + + virtual ~ThriftHiveMetastore_add_partition_with_environment_context_pargs() noexcept; + const Partition* new_part; + const EnvironmentContext* environment_context; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_partition_with_environment_context_result__isset { + _ThriftHiveMetastore_add_partition_with_environment_context_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_add_partition_with_environment_context_result__isset; + +class ThriftHiveMetastore_add_partition_with_environment_context_result { + public: + + ThriftHiveMetastore_add_partition_with_environment_context_result(const ThriftHiveMetastore_add_partition_with_environment_context_result&); + ThriftHiveMetastore_add_partition_with_environment_context_result& operator=(const ThriftHiveMetastore_add_partition_with_environment_context_result&); + ThriftHiveMetastore_add_partition_with_environment_context_result() { + } + + virtual ~ThriftHiveMetastore_add_partition_with_environment_context_result() noexcept; + Partition success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_add_partition_with_environment_context_result__isset __isset; + + void __set_success(const Partition& val); + + void __set_o1(const InvalidObjectException& val); + + void __set_o2(const AlreadyExistsException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_add_partition_with_environment_context_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_partition_with_environment_context_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_partition_with_environment_context_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_partition_with_environment_context_presult__isset { + _ThriftHiveMetastore_add_partition_with_environment_context_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_add_partition_with_environment_context_presult__isset; + +class ThriftHiveMetastore_add_partition_with_environment_context_presult { + public: + + + virtual ~ThriftHiveMetastore_add_partition_with_environment_context_presult() noexcept; + Partition* success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_add_partition_with_environment_context_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_add_partitions_args__isset { + _ThriftHiveMetastore_add_partitions_args__isset() : new_parts(false) {} + bool new_parts :1; +} _ThriftHiveMetastore_add_partitions_args__isset; + +class ThriftHiveMetastore_add_partitions_args { + public: + + ThriftHiveMetastore_add_partitions_args(const ThriftHiveMetastore_add_partitions_args&); + ThriftHiveMetastore_add_partitions_args& operator=(const ThriftHiveMetastore_add_partitions_args&); + ThriftHiveMetastore_add_partitions_args() { + } + + virtual ~ThriftHiveMetastore_add_partitions_args() noexcept; + std::vector new_parts; + + _ThriftHiveMetastore_add_partitions_args__isset __isset; + + void __set_new_parts(const std::vector & val); + + bool operator == (const ThriftHiveMetastore_add_partitions_args & rhs) const + { + if (!(new_parts == rhs.new_parts)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_partitions_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_partitions_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_add_partitions_pargs { + public: + + + virtual ~ThriftHiveMetastore_add_partitions_pargs() noexcept; + const std::vector * new_parts; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_partitions_result__isset { + _ThriftHiveMetastore_add_partitions_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_add_partitions_result__isset; + +class ThriftHiveMetastore_add_partitions_result { + public: + + ThriftHiveMetastore_add_partitions_result(const ThriftHiveMetastore_add_partitions_result&); + ThriftHiveMetastore_add_partitions_result& operator=(const ThriftHiveMetastore_add_partitions_result&); + ThriftHiveMetastore_add_partitions_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_add_partitions_result() noexcept; + int32_t success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_add_partitions_result__isset __isset; + + void __set_success(const int32_t val); + + void __set_o1(const InvalidObjectException& val); + + void __set_o2(const AlreadyExistsException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_add_partitions_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_partitions_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_partitions_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_partitions_presult__isset { + _ThriftHiveMetastore_add_partitions_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_add_partitions_presult__isset; + +class ThriftHiveMetastore_add_partitions_presult { + public: + + + virtual ~ThriftHiveMetastore_add_partitions_presult() noexcept; + int32_t* success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_add_partitions_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_add_partitions_pspec_args__isset { + _ThriftHiveMetastore_add_partitions_pspec_args__isset() : new_parts(false) {} + bool new_parts :1; +} _ThriftHiveMetastore_add_partitions_pspec_args__isset; + +class ThriftHiveMetastore_add_partitions_pspec_args { + public: + + ThriftHiveMetastore_add_partitions_pspec_args(const ThriftHiveMetastore_add_partitions_pspec_args&); + ThriftHiveMetastore_add_partitions_pspec_args& operator=(const ThriftHiveMetastore_add_partitions_pspec_args&); + ThriftHiveMetastore_add_partitions_pspec_args() { + } + + virtual ~ThriftHiveMetastore_add_partitions_pspec_args() noexcept; + std::vector new_parts; + + _ThriftHiveMetastore_add_partitions_pspec_args__isset __isset; + + void __set_new_parts(const std::vector & val); + + bool operator == (const ThriftHiveMetastore_add_partitions_pspec_args & rhs) const + { + if (!(new_parts == rhs.new_parts)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_partitions_pspec_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_partitions_pspec_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_add_partitions_pspec_pargs { + public: + + + virtual ~ThriftHiveMetastore_add_partitions_pspec_pargs() noexcept; + const std::vector * new_parts; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_partitions_pspec_result__isset { + _ThriftHiveMetastore_add_partitions_pspec_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_add_partitions_pspec_result__isset; + +class ThriftHiveMetastore_add_partitions_pspec_result { + public: + + ThriftHiveMetastore_add_partitions_pspec_result(const ThriftHiveMetastore_add_partitions_pspec_result&); + ThriftHiveMetastore_add_partitions_pspec_result& operator=(const ThriftHiveMetastore_add_partitions_pspec_result&); + ThriftHiveMetastore_add_partitions_pspec_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_add_partitions_pspec_result() noexcept; + int32_t success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_add_partitions_pspec_result__isset __isset; + + void __set_success(const int32_t val); + + void __set_o1(const InvalidObjectException& val); + + void __set_o2(const AlreadyExistsException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_add_partitions_pspec_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_partitions_pspec_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_partitions_pspec_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_partitions_pspec_presult__isset { + _ThriftHiveMetastore_add_partitions_pspec_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_add_partitions_pspec_presult__isset; + +class ThriftHiveMetastore_add_partitions_pspec_presult { + public: + + + virtual ~ThriftHiveMetastore_add_partitions_pspec_presult() noexcept; + int32_t* success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_add_partitions_pspec_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_append_partition_args__isset { + _ThriftHiveMetastore_append_partition_args__isset() : db_name(false), tbl_name(false), part_vals(false) {} + bool db_name :1; + bool tbl_name :1; + bool part_vals :1; +} _ThriftHiveMetastore_append_partition_args__isset; + +class ThriftHiveMetastore_append_partition_args { + public: + + ThriftHiveMetastore_append_partition_args(const ThriftHiveMetastore_append_partition_args&); + ThriftHiveMetastore_append_partition_args& operator=(const ThriftHiveMetastore_append_partition_args&); + ThriftHiveMetastore_append_partition_args() : db_name(), tbl_name() { + } + + virtual ~ThriftHiveMetastore_append_partition_args() noexcept; + std::string db_name; + std::string tbl_name; + std::vector part_vals; + + _ThriftHiveMetastore_append_partition_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_vals(const std::vector & val); + + bool operator == (const ThriftHiveMetastore_append_partition_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_vals == rhs.part_vals)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_append_partition_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_append_partition_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_append_partition_pargs { + public: + + + virtual ~ThriftHiveMetastore_append_partition_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::vector * part_vals; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_append_partition_result__isset { + _ThriftHiveMetastore_append_partition_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_append_partition_result__isset; + +class ThriftHiveMetastore_append_partition_result { + public: + + ThriftHiveMetastore_append_partition_result(const ThriftHiveMetastore_append_partition_result&); + ThriftHiveMetastore_append_partition_result& operator=(const ThriftHiveMetastore_append_partition_result&); + ThriftHiveMetastore_append_partition_result() { + } + + virtual ~ThriftHiveMetastore_append_partition_result() noexcept; + Partition success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_append_partition_result__isset __isset; + + void __set_success(const Partition& val); + + void __set_o1(const InvalidObjectException& val); + + void __set_o2(const AlreadyExistsException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_append_partition_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_append_partition_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_append_partition_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_append_partition_presult__isset { + _ThriftHiveMetastore_append_partition_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_append_partition_presult__isset; + +class ThriftHiveMetastore_append_partition_presult { + public: + + + virtual ~ThriftHiveMetastore_append_partition_presult() noexcept; + Partition* success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_append_partition_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_add_partitions_req_args__isset { + _ThriftHiveMetastore_add_partitions_req_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_add_partitions_req_args__isset; + +class ThriftHiveMetastore_add_partitions_req_args { + public: + + ThriftHiveMetastore_add_partitions_req_args(const ThriftHiveMetastore_add_partitions_req_args&); + ThriftHiveMetastore_add_partitions_req_args& operator=(const ThriftHiveMetastore_add_partitions_req_args&); + ThriftHiveMetastore_add_partitions_req_args() { + } + + virtual ~ThriftHiveMetastore_add_partitions_req_args() noexcept; + AddPartitionsRequest request; + + _ThriftHiveMetastore_add_partitions_req_args__isset __isset; + + void __set_request(const AddPartitionsRequest& val); + + bool operator == (const ThriftHiveMetastore_add_partitions_req_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_partitions_req_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_partitions_req_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_add_partitions_req_pargs { + public: + + + virtual ~ThriftHiveMetastore_add_partitions_req_pargs() noexcept; + const AddPartitionsRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_partitions_req_result__isset { + _ThriftHiveMetastore_add_partitions_req_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_add_partitions_req_result__isset; + +class ThriftHiveMetastore_add_partitions_req_result { + public: + + ThriftHiveMetastore_add_partitions_req_result(const ThriftHiveMetastore_add_partitions_req_result&); + ThriftHiveMetastore_add_partitions_req_result& operator=(const ThriftHiveMetastore_add_partitions_req_result&); + ThriftHiveMetastore_add_partitions_req_result() { + } + + virtual ~ThriftHiveMetastore_add_partitions_req_result() noexcept; + AddPartitionsResult success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_add_partitions_req_result__isset __isset; + + void __set_success(const AddPartitionsResult& val); + + void __set_o1(const InvalidObjectException& val); + + void __set_o2(const AlreadyExistsException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_add_partitions_req_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_partitions_req_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_partitions_req_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_partitions_req_presult__isset { + _ThriftHiveMetastore_add_partitions_req_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_add_partitions_req_presult__isset; + +class ThriftHiveMetastore_add_partitions_req_presult { + public: + + + virtual ~ThriftHiveMetastore_add_partitions_req_presult() noexcept; + AddPartitionsResult* success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_add_partitions_req_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_append_partition_with_environment_context_args__isset { + _ThriftHiveMetastore_append_partition_with_environment_context_args__isset() : db_name(false), tbl_name(false), part_vals(false), environment_context(false) {} + bool db_name :1; + bool tbl_name :1; + bool part_vals :1; + bool environment_context :1; +} _ThriftHiveMetastore_append_partition_with_environment_context_args__isset; + +class ThriftHiveMetastore_append_partition_with_environment_context_args { + public: + + ThriftHiveMetastore_append_partition_with_environment_context_args(const ThriftHiveMetastore_append_partition_with_environment_context_args&); + ThriftHiveMetastore_append_partition_with_environment_context_args& operator=(const ThriftHiveMetastore_append_partition_with_environment_context_args&); + ThriftHiveMetastore_append_partition_with_environment_context_args() : db_name(), tbl_name() { + } + + virtual ~ThriftHiveMetastore_append_partition_with_environment_context_args() noexcept; + std::string db_name; + std::string tbl_name; + std::vector part_vals; + EnvironmentContext environment_context; + + _ThriftHiveMetastore_append_partition_with_environment_context_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_vals(const std::vector & val); + + void __set_environment_context(const EnvironmentContext& val); + + bool operator == (const ThriftHiveMetastore_append_partition_with_environment_context_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_vals == rhs.part_vals)) + return false; + if (!(environment_context == rhs.environment_context)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_append_partition_with_environment_context_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_append_partition_with_environment_context_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_append_partition_with_environment_context_pargs { + public: + + + virtual ~ThriftHiveMetastore_append_partition_with_environment_context_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::vector * part_vals; + const EnvironmentContext* environment_context; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_append_partition_with_environment_context_result__isset { + _ThriftHiveMetastore_append_partition_with_environment_context_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_append_partition_with_environment_context_result__isset; + +class ThriftHiveMetastore_append_partition_with_environment_context_result { + public: + + ThriftHiveMetastore_append_partition_with_environment_context_result(const ThriftHiveMetastore_append_partition_with_environment_context_result&); + ThriftHiveMetastore_append_partition_with_environment_context_result& operator=(const ThriftHiveMetastore_append_partition_with_environment_context_result&); + ThriftHiveMetastore_append_partition_with_environment_context_result() { + } + + virtual ~ThriftHiveMetastore_append_partition_with_environment_context_result() noexcept; + Partition success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_append_partition_with_environment_context_result__isset __isset; + + void __set_success(const Partition& val); + + void __set_o1(const InvalidObjectException& val); + + void __set_o2(const AlreadyExistsException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_append_partition_with_environment_context_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_append_partition_with_environment_context_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_append_partition_with_environment_context_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_append_partition_with_environment_context_presult__isset { + _ThriftHiveMetastore_append_partition_with_environment_context_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_append_partition_with_environment_context_presult__isset; + +class ThriftHiveMetastore_append_partition_with_environment_context_presult { + public: + + + virtual ~ThriftHiveMetastore_append_partition_with_environment_context_presult() noexcept; + Partition* success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_append_partition_with_environment_context_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_append_partition_by_name_args__isset { + _ThriftHiveMetastore_append_partition_by_name_args__isset() : db_name(false), tbl_name(false), part_name(false) {} + bool db_name :1; + bool tbl_name :1; + bool part_name :1; +} _ThriftHiveMetastore_append_partition_by_name_args__isset; + +class ThriftHiveMetastore_append_partition_by_name_args { + public: + + ThriftHiveMetastore_append_partition_by_name_args(const ThriftHiveMetastore_append_partition_by_name_args&); + ThriftHiveMetastore_append_partition_by_name_args& operator=(const ThriftHiveMetastore_append_partition_by_name_args&); + ThriftHiveMetastore_append_partition_by_name_args() : db_name(), tbl_name(), part_name() { + } + + virtual ~ThriftHiveMetastore_append_partition_by_name_args() noexcept; + std::string db_name; + std::string tbl_name; + std::string part_name; + + _ThriftHiveMetastore_append_partition_by_name_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_append_partition_by_name_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_name == rhs.part_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_append_partition_by_name_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_append_partition_by_name_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_append_partition_by_name_pargs { + public: + + + virtual ~ThriftHiveMetastore_append_partition_by_name_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::string* part_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_append_partition_by_name_result__isset { + _ThriftHiveMetastore_append_partition_by_name_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_append_partition_by_name_result__isset; + +class ThriftHiveMetastore_append_partition_by_name_result { + public: + + ThriftHiveMetastore_append_partition_by_name_result(const ThriftHiveMetastore_append_partition_by_name_result&); + ThriftHiveMetastore_append_partition_by_name_result& operator=(const ThriftHiveMetastore_append_partition_by_name_result&); + ThriftHiveMetastore_append_partition_by_name_result() { + } + + virtual ~ThriftHiveMetastore_append_partition_by_name_result() noexcept; + Partition success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_append_partition_by_name_result__isset __isset; + + void __set_success(const Partition& val); + + void __set_o1(const InvalidObjectException& val); + + void __set_o2(const AlreadyExistsException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_append_partition_by_name_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_append_partition_by_name_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_append_partition_by_name_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_append_partition_by_name_presult__isset { + _ThriftHiveMetastore_append_partition_by_name_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_append_partition_by_name_presult__isset; + +class ThriftHiveMetastore_append_partition_by_name_presult { + public: + + + virtual ~ThriftHiveMetastore_append_partition_by_name_presult() noexcept; + Partition* success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_append_partition_by_name_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_append_partition_by_name_with_environment_context_args__isset { + _ThriftHiveMetastore_append_partition_by_name_with_environment_context_args__isset() : db_name(false), tbl_name(false), part_name(false), environment_context(false) {} + bool db_name :1; + bool tbl_name :1; + bool part_name :1; + bool environment_context :1; +} _ThriftHiveMetastore_append_partition_by_name_with_environment_context_args__isset; + +class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args { + public: + + ThriftHiveMetastore_append_partition_by_name_with_environment_context_args(const ThriftHiveMetastore_append_partition_by_name_with_environment_context_args&); + ThriftHiveMetastore_append_partition_by_name_with_environment_context_args& operator=(const ThriftHiveMetastore_append_partition_by_name_with_environment_context_args&); + ThriftHiveMetastore_append_partition_by_name_with_environment_context_args() : db_name(), tbl_name(), part_name() { + } + + virtual ~ThriftHiveMetastore_append_partition_by_name_with_environment_context_args() noexcept; + std::string db_name; + std::string tbl_name; + std::string part_name; + EnvironmentContext environment_context; + + _ThriftHiveMetastore_append_partition_by_name_with_environment_context_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_name(const std::string& val); + + void __set_environment_context(const EnvironmentContext& val); + + bool operator == (const ThriftHiveMetastore_append_partition_by_name_with_environment_context_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_name == rhs.part_name)) + return false; + if (!(environment_context == rhs.environment_context)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_append_partition_by_name_with_environment_context_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_append_partition_by_name_with_environment_context_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs { + public: + + + virtual ~ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::string* part_name; + const EnvironmentContext* environment_context; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_append_partition_by_name_with_environment_context_result__isset { + _ThriftHiveMetastore_append_partition_by_name_with_environment_context_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_append_partition_by_name_with_environment_context_result__isset; + +class ThriftHiveMetastore_append_partition_by_name_with_environment_context_result { + public: + + ThriftHiveMetastore_append_partition_by_name_with_environment_context_result(const ThriftHiveMetastore_append_partition_by_name_with_environment_context_result&); + ThriftHiveMetastore_append_partition_by_name_with_environment_context_result& operator=(const ThriftHiveMetastore_append_partition_by_name_with_environment_context_result&); + ThriftHiveMetastore_append_partition_by_name_with_environment_context_result() { + } + + virtual ~ThriftHiveMetastore_append_partition_by_name_with_environment_context_result() noexcept; + Partition success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_append_partition_by_name_with_environment_context_result__isset __isset; + + void __set_success(const Partition& val); + + void __set_o1(const InvalidObjectException& val); + + void __set_o2(const AlreadyExistsException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_append_partition_by_name_with_environment_context_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_append_partition_by_name_with_environment_context_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_append_partition_by_name_with_environment_context_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult__isset { + _ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult__isset; + +class ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult { + public: + + + virtual ~ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult() noexcept; + Partition* success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_drop_partition_args__isset { + _ThriftHiveMetastore_drop_partition_args__isset() : db_name(false), tbl_name(false), part_vals(false), deleteData(false) {} + bool db_name :1; + bool tbl_name :1; + bool part_vals :1; + bool deleteData :1; +} _ThriftHiveMetastore_drop_partition_args__isset; + +class ThriftHiveMetastore_drop_partition_args { + public: + + ThriftHiveMetastore_drop_partition_args(const ThriftHiveMetastore_drop_partition_args&); + ThriftHiveMetastore_drop_partition_args& operator=(const ThriftHiveMetastore_drop_partition_args&); + ThriftHiveMetastore_drop_partition_args() : db_name(), tbl_name(), deleteData(0) { + } + + virtual ~ThriftHiveMetastore_drop_partition_args() noexcept; + std::string db_name; + std::string tbl_name; + std::vector part_vals; + bool deleteData; + + _ThriftHiveMetastore_drop_partition_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_vals(const std::vector & val); + + void __set_deleteData(const bool val); + + bool operator == (const ThriftHiveMetastore_drop_partition_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_vals == rhs.part_vals)) + return false; + if (!(deleteData == rhs.deleteData)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_partition_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_partition_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_drop_partition_pargs { + public: + + + virtual ~ThriftHiveMetastore_drop_partition_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::vector * part_vals; + const bool* deleteData; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_partition_result__isset { + _ThriftHiveMetastore_drop_partition_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_drop_partition_result__isset; + +class ThriftHiveMetastore_drop_partition_result { + public: + + ThriftHiveMetastore_drop_partition_result(const ThriftHiveMetastore_drop_partition_result&); + ThriftHiveMetastore_drop_partition_result& operator=(const ThriftHiveMetastore_drop_partition_result&); + ThriftHiveMetastore_drop_partition_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_drop_partition_result() noexcept; + bool success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_drop_partition_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_drop_partition_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_partition_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_partition_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_partition_presult__isset { + _ThriftHiveMetastore_drop_partition_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_drop_partition_presult__isset; + +class ThriftHiveMetastore_drop_partition_presult { + public: + + + virtual ~ThriftHiveMetastore_drop_partition_presult() noexcept; + bool* success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_drop_partition_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_drop_partition_with_environment_context_args__isset { + _ThriftHiveMetastore_drop_partition_with_environment_context_args__isset() : db_name(false), tbl_name(false), part_vals(false), deleteData(false), environment_context(false) {} + bool db_name :1; + bool tbl_name :1; + bool part_vals :1; + bool deleteData :1; + bool environment_context :1; +} _ThriftHiveMetastore_drop_partition_with_environment_context_args__isset; + +class ThriftHiveMetastore_drop_partition_with_environment_context_args { + public: + + ThriftHiveMetastore_drop_partition_with_environment_context_args(const ThriftHiveMetastore_drop_partition_with_environment_context_args&); + ThriftHiveMetastore_drop_partition_with_environment_context_args& operator=(const ThriftHiveMetastore_drop_partition_with_environment_context_args&); + ThriftHiveMetastore_drop_partition_with_environment_context_args() : db_name(), tbl_name(), deleteData(0) { + } + + virtual ~ThriftHiveMetastore_drop_partition_with_environment_context_args() noexcept; + std::string db_name; + std::string tbl_name; + std::vector part_vals; + bool deleteData; + EnvironmentContext environment_context; + + _ThriftHiveMetastore_drop_partition_with_environment_context_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_vals(const std::vector & val); + + void __set_deleteData(const bool val); + + void __set_environment_context(const EnvironmentContext& val); + + bool operator == (const ThriftHiveMetastore_drop_partition_with_environment_context_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_vals == rhs.part_vals)) + return false; + if (!(deleteData == rhs.deleteData)) + return false; + if (!(environment_context == rhs.environment_context)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_partition_with_environment_context_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_partition_with_environment_context_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_drop_partition_with_environment_context_pargs { + public: + + + virtual ~ThriftHiveMetastore_drop_partition_with_environment_context_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::vector * part_vals; + const bool* deleteData; + const EnvironmentContext* environment_context; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_partition_with_environment_context_result__isset { + _ThriftHiveMetastore_drop_partition_with_environment_context_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_drop_partition_with_environment_context_result__isset; + +class ThriftHiveMetastore_drop_partition_with_environment_context_result { + public: + + ThriftHiveMetastore_drop_partition_with_environment_context_result(const ThriftHiveMetastore_drop_partition_with_environment_context_result&); + ThriftHiveMetastore_drop_partition_with_environment_context_result& operator=(const ThriftHiveMetastore_drop_partition_with_environment_context_result&); + ThriftHiveMetastore_drop_partition_with_environment_context_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_drop_partition_with_environment_context_result() noexcept; + bool success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_drop_partition_with_environment_context_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_drop_partition_with_environment_context_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_partition_with_environment_context_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_partition_with_environment_context_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_partition_with_environment_context_presult__isset { + _ThriftHiveMetastore_drop_partition_with_environment_context_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_drop_partition_with_environment_context_presult__isset; + +class ThriftHiveMetastore_drop_partition_with_environment_context_presult { + public: + + + virtual ~ThriftHiveMetastore_drop_partition_with_environment_context_presult() noexcept; + bool* success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_drop_partition_with_environment_context_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_drop_partition_by_name_args__isset { + _ThriftHiveMetastore_drop_partition_by_name_args__isset() : db_name(false), tbl_name(false), part_name(false), deleteData(false) {} + bool db_name :1; + bool tbl_name :1; + bool part_name :1; + bool deleteData :1; +} _ThriftHiveMetastore_drop_partition_by_name_args__isset; + +class ThriftHiveMetastore_drop_partition_by_name_args { + public: + + ThriftHiveMetastore_drop_partition_by_name_args(const ThriftHiveMetastore_drop_partition_by_name_args&); + ThriftHiveMetastore_drop_partition_by_name_args& operator=(const ThriftHiveMetastore_drop_partition_by_name_args&); + ThriftHiveMetastore_drop_partition_by_name_args() : db_name(), tbl_name(), part_name(), deleteData(0) { + } + + virtual ~ThriftHiveMetastore_drop_partition_by_name_args() noexcept; + std::string db_name; + std::string tbl_name; + std::string part_name; + bool deleteData; + + _ThriftHiveMetastore_drop_partition_by_name_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_name(const std::string& val); + + void __set_deleteData(const bool val); + + bool operator == (const ThriftHiveMetastore_drop_partition_by_name_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_name == rhs.part_name)) + return false; + if (!(deleteData == rhs.deleteData)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_partition_by_name_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_partition_by_name_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_drop_partition_by_name_pargs { + public: + + + virtual ~ThriftHiveMetastore_drop_partition_by_name_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::string* part_name; + const bool* deleteData; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_partition_by_name_result__isset { + _ThriftHiveMetastore_drop_partition_by_name_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_drop_partition_by_name_result__isset; + +class ThriftHiveMetastore_drop_partition_by_name_result { + public: + + ThriftHiveMetastore_drop_partition_by_name_result(const ThriftHiveMetastore_drop_partition_by_name_result&); + ThriftHiveMetastore_drop_partition_by_name_result& operator=(const ThriftHiveMetastore_drop_partition_by_name_result&); + ThriftHiveMetastore_drop_partition_by_name_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_drop_partition_by_name_result() noexcept; + bool success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_drop_partition_by_name_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_drop_partition_by_name_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_partition_by_name_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_partition_by_name_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_partition_by_name_presult__isset { + _ThriftHiveMetastore_drop_partition_by_name_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_drop_partition_by_name_presult__isset; + +class ThriftHiveMetastore_drop_partition_by_name_presult { + public: + + + virtual ~ThriftHiveMetastore_drop_partition_by_name_presult() noexcept; + bool* success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_drop_partition_by_name_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args__isset { + _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args__isset() : db_name(false), tbl_name(false), part_name(false), deleteData(false), environment_context(false) {} + bool db_name :1; + bool tbl_name :1; + bool part_name :1; + bool deleteData :1; + bool environment_context :1; +} _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args__isset; + +class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { + public: + + ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args(const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args&); + ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args& operator=(const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args&); + ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args() : db_name(), tbl_name(), part_name(), deleteData(0) { + } + + virtual ~ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args() noexcept; + std::string db_name; + std::string tbl_name; + std::string part_name; + bool deleteData; + EnvironmentContext environment_context; + + _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_name(const std::string& val); + + void __set_deleteData(const bool val); + + void __set_environment_context(const EnvironmentContext& val); + + bool operator == (const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_name == rhs.part_name)) + return false; + if (!(deleteData == rhs.deleteData)) + return false; + if (!(environment_context == rhs.environment_context)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs { + public: + + + virtual ~ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::string* part_name; + const bool* deleteData; + const EnvironmentContext* environment_context; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result__isset { + _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result__isset; + +class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result { + public: + + ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result(const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result&); + ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result& operator=(const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result&); + ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result() noexcept; + bool success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult__isset { + _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult__isset; + +class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult { + public: + + + virtual ~ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult() noexcept; + bool* success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_drop_partitions_req_args__isset { + _ThriftHiveMetastore_drop_partitions_req_args__isset() : req(false) {} + bool req :1; +} _ThriftHiveMetastore_drop_partitions_req_args__isset; + +class ThriftHiveMetastore_drop_partitions_req_args { + public: + + ThriftHiveMetastore_drop_partitions_req_args(const ThriftHiveMetastore_drop_partitions_req_args&); + ThriftHiveMetastore_drop_partitions_req_args& operator=(const ThriftHiveMetastore_drop_partitions_req_args&); + ThriftHiveMetastore_drop_partitions_req_args() { + } + + virtual ~ThriftHiveMetastore_drop_partitions_req_args() noexcept; + DropPartitionsRequest req; + + _ThriftHiveMetastore_drop_partitions_req_args__isset __isset; + + void __set_req(const DropPartitionsRequest& val); + + bool operator == (const ThriftHiveMetastore_drop_partitions_req_args & rhs) const + { + if (!(req == rhs.req)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_partitions_req_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_partitions_req_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_drop_partitions_req_pargs { + public: + + + virtual ~ThriftHiveMetastore_drop_partitions_req_pargs() noexcept; + const DropPartitionsRequest* req; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_partitions_req_result__isset { + _ThriftHiveMetastore_drop_partitions_req_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_drop_partitions_req_result__isset; + +class ThriftHiveMetastore_drop_partitions_req_result { + public: + + ThriftHiveMetastore_drop_partitions_req_result(const ThriftHiveMetastore_drop_partitions_req_result&); + ThriftHiveMetastore_drop_partitions_req_result& operator=(const ThriftHiveMetastore_drop_partitions_req_result&); + ThriftHiveMetastore_drop_partitions_req_result() { + } + + virtual ~ThriftHiveMetastore_drop_partitions_req_result() noexcept; + DropPartitionsResult success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_drop_partitions_req_result__isset __isset; + + void __set_success(const DropPartitionsResult& val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_drop_partitions_req_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_partitions_req_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_partitions_req_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_partitions_req_presult__isset { + _ThriftHiveMetastore_drop_partitions_req_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_drop_partitions_req_presult__isset; + +class ThriftHiveMetastore_drop_partitions_req_presult { + public: + + + virtual ~ThriftHiveMetastore_drop_partitions_req_presult() noexcept; + DropPartitionsResult* success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_drop_partitions_req_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_partition_args__isset { + _ThriftHiveMetastore_get_partition_args__isset() : db_name(false), tbl_name(false), part_vals(false) {} + bool db_name :1; + bool tbl_name :1; + bool part_vals :1; +} _ThriftHiveMetastore_get_partition_args__isset; + +class ThriftHiveMetastore_get_partition_args { + public: + + ThriftHiveMetastore_get_partition_args(const ThriftHiveMetastore_get_partition_args&); + ThriftHiveMetastore_get_partition_args& operator=(const ThriftHiveMetastore_get_partition_args&); + ThriftHiveMetastore_get_partition_args() : db_name(), tbl_name() { + } + + virtual ~ThriftHiveMetastore_get_partition_args() noexcept; + std::string db_name; + std::string tbl_name; + std::vector part_vals; + + _ThriftHiveMetastore_get_partition_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_vals(const std::vector & val); + + bool operator == (const ThriftHiveMetastore_get_partition_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_vals == rhs.part_vals)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partition_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partition_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_partition_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_partition_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::vector * part_vals; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partition_result__isset { + _ThriftHiveMetastore_get_partition_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partition_result__isset; + +class ThriftHiveMetastore_get_partition_result { + public: + + ThriftHiveMetastore_get_partition_result(const ThriftHiveMetastore_get_partition_result&); + ThriftHiveMetastore_get_partition_result& operator=(const ThriftHiveMetastore_get_partition_result&); + ThriftHiveMetastore_get_partition_result() { + } + + virtual ~ThriftHiveMetastore_get_partition_result() noexcept; + Partition success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partition_result__isset __isset; + + void __set_success(const Partition& val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_partition_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partition_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partition_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partition_presult__isset { + _ThriftHiveMetastore_get_partition_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partition_presult__isset; + +class ThriftHiveMetastore_get_partition_presult { + public: + + + virtual ~ThriftHiveMetastore_get_partition_presult() noexcept; + Partition* success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partition_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_exchange_partition_args__isset { + _ThriftHiveMetastore_exchange_partition_args__isset() : partitionSpecs(false), source_db(false), source_table_name(false), dest_db(false), dest_table_name(false) {} + bool partitionSpecs :1; + bool source_db :1; + bool source_table_name :1; + bool dest_db :1; + bool dest_table_name :1; +} _ThriftHiveMetastore_exchange_partition_args__isset; + +class ThriftHiveMetastore_exchange_partition_args { + public: + + ThriftHiveMetastore_exchange_partition_args(const ThriftHiveMetastore_exchange_partition_args&); + ThriftHiveMetastore_exchange_partition_args& operator=(const ThriftHiveMetastore_exchange_partition_args&); + ThriftHiveMetastore_exchange_partition_args() : source_db(), source_table_name(), dest_db(), dest_table_name() { + } + + virtual ~ThriftHiveMetastore_exchange_partition_args() noexcept; + std::map partitionSpecs; + std::string source_db; + std::string source_table_name; + std::string dest_db; + std::string dest_table_name; + + _ThriftHiveMetastore_exchange_partition_args__isset __isset; + + void __set_partitionSpecs(const std::map & val); + + void __set_source_db(const std::string& val); + + void __set_source_table_name(const std::string& val); + + void __set_dest_db(const std::string& val); + + void __set_dest_table_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_exchange_partition_args & rhs) const + { + if (!(partitionSpecs == rhs.partitionSpecs)) + return false; + if (!(source_db == rhs.source_db)) + return false; + if (!(source_table_name == rhs.source_table_name)) + return false; + if (!(dest_db == rhs.dest_db)) + return false; + if (!(dest_table_name == rhs.dest_table_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_exchange_partition_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_exchange_partition_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_exchange_partition_pargs { + public: + + + virtual ~ThriftHiveMetastore_exchange_partition_pargs() noexcept; + const std::map * partitionSpecs; + const std::string* source_db; + const std::string* source_table_name; + const std::string* dest_db; + const std::string* dest_table_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_exchange_partition_result__isset { + _ThriftHiveMetastore_exchange_partition_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_exchange_partition_result__isset; + +class ThriftHiveMetastore_exchange_partition_result { + public: + + ThriftHiveMetastore_exchange_partition_result(const ThriftHiveMetastore_exchange_partition_result&); + ThriftHiveMetastore_exchange_partition_result& operator=(const ThriftHiveMetastore_exchange_partition_result&); + ThriftHiveMetastore_exchange_partition_result() { + } + + virtual ~ThriftHiveMetastore_exchange_partition_result() noexcept; + Partition success; + MetaException o1; + NoSuchObjectException o2; + InvalidObjectException o3; + InvalidInputException o4; + + _ThriftHiveMetastore_exchange_partition_result__isset __isset; + + void __set_success(const Partition& val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + void __set_o3(const InvalidObjectException& val); + + void __set_o4(const InvalidInputException& val); + + bool operator == (const ThriftHiveMetastore_exchange_partition_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_exchange_partition_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_exchange_partition_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_exchange_partition_presult__isset { + _ThriftHiveMetastore_exchange_partition_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_exchange_partition_presult__isset; + +class ThriftHiveMetastore_exchange_partition_presult { + public: + + + virtual ~ThriftHiveMetastore_exchange_partition_presult() noexcept; + Partition* success; + MetaException o1; + NoSuchObjectException o2; + InvalidObjectException o3; + InvalidInputException o4; + + _ThriftHiveMetastore_exchange_partition_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_exchange_partitions_args__isset { + _ThriftHiveMetastore_exchange_partitions_args__isset() : partitionSpecs(false), source_db(false), source_table_name(false), dest_db(false), dest_table_name(false) {} + bool partitionSpecs :1; + bool source_db :1; + bool source_table_name :1; + bool dest_db :1; + bool dest_table_name :1; +} _ThriftHiveMetastore_exchange_partitions_args__isset; + +class ThriftHiveMetastore_exchange_partitions_args { + public: + + ThriftHiveMetastore_exchange_partitions_args(const ThriftHiveMetastore_exchange_partitions_args&); + ThriftHiveMetastore_exchange_partitions_args& operator=(const ThriftHiveMetastore_exchange_partitions_args&); + ThriftHiveMetastore_exchange_partitions_args() : source_db(), source_table_name(), dest_db(), dest_table_name() { + } + + virtual ~ThriftHiveMetastore_exchange_partitions_args() noexcept; + std::map partitionSpecs; + std::string source_db; + std::string source_table_name; + std::string dest_db; + std::string dest_table_name; + + _ThriftHiveMetastore_exchange_partitions_args__isset __isset; + + void __set_partitionSpecs(const std::map & val); + + void __set_source_db(const std::string& val); + + void __set_source_table_name(const std::string& val); + + void __set_dest_db(const std::string& val); + + void __set_dest_table_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_exchange_partitions_args & rhs) const + { + if (!(partitionSpecs == rhs.partitionSpecs)) + return false; + if (!(source_db == rhs.source_db)) + return false; + if (!(source_table_name == rhs.source_table_name)) + return false; + if (!(dest_db == rhs.dest_db)) + return false; + if (!(dest_table_name == rhs.dest_table_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_exchange_partitions_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_exchange_partitions_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_exchange_partitions_pargs { + public: + + + virtual ~ThriftHiveMetastore_exchange_partitions_pargs() noexcept; + const std::map * partitionSpecs; + const std::string* source_db; + const std::string* source_table_name; + const std::string* dest_db; + const std::string* dest_table_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_exchange_partitions_result__isset { + _ThriftHiveMetastore_exchange_partitions_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_exchange_partitions_result__isset; + +class ThriftHiveMetastore_exchange_partitions_result { + public: + + ThriftHiveMetastore_exchange_partitions_result(const ThriftHiveMetastore_exchange_partitions_result&); + ThriftHiveMetastore_exchange_partitions_result& operator=(const ThriftHiveMetastore_exchange_partitions_result&); + ThriftHiveMetastore_exchange_partitions_result() { + } + + virtual ~ThriftHiveMetastore_exchange_partitions_result() noexcept; + std::vector success; + MetaException o1; + NoSuchObjectException o2; + InvalidObjectException o3; + InvalidInputException o4; + + _ThriftHiveMetastore_exchange_partitions_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + void __set_o3(const InvalidObjectException& val); + + void __set_o4(const InvalidInputException& val); + + bool operator == (const ThriftHiveMetastore_exchange_partitions_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_exchange_partitions_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_exchange_partitions_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_exchange_partitions_presult__isset { + _ThriftHiveMetastore_exchange_partitions_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_exchange_partitions_presult__isset; + +class ThriftHiveMetastore_exchange_partitions_presult { + public: + + + virtual ~ThriftHiveMetastore_exchange_partitions_presult() noexcept; + std::vector * success; + MetaException o1; + NoSuchObjectException o2; + InvalidObjectException o3; + InvalidInputException o4; + + _ThriftHiveMetastore_exchange_partitions_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_partition_with_auth_args__isset { + _ThriftHiveMetastore_get_partition_with_auth_args__isset() : db_name(false), tbl_name(false), part_vals(false), user_name(false), group_names(false) {} + bool db_name :1; + bool tbl_name :1; + bool part_vals :1; + bool user_name :1; + bool group_names :1; +} _ThriftHiveMetastore_get_partition_with_auth_args__isset; + +class ThriftHiveMetastore_get_partition_with_auth_args { + public: + + ThriftHiveMetastore_get_partition_with_auth_args(const ThriftHiveMetastore_get_partition_with_auth_args&); + ThriftHiveMetastore_get_partition_with_auth_args& operator=(const ThriftHiveMetastore_get_partition_with_auth_args&); + ThriftHiveMetastore_get_partition_with_auth_args() : db_name(), tbl_name(), user_name() { + } + + virtual ~ThriftHiveMetastore_get_partition_with_auth_args() noexcept; + std::string db_name; + std::string tbl_name; + std::vector part_vals; + std::string user_name; + std::vector group_names; + + _ThriftHiveMetastore_get_partition_with_auth_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_vals(const std::vector & val); + + void __set_user_name(const std::string& val); + + void __set_group_names(const std::vector & val); + + bool operator == (const ThriftHiveMetastore_get_partition_with_auth_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_vals == rhs.part_vals)) + return false; + if (!(user_name == rhs.user_name)) + return false; + if (!(group_names == rhs.group_names)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partition_with_auth_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partition_with_auth_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_partition_with_auth_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_partition_with_auth_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::vector * part_vals; + const std::string* user_name; + const std::vector * group_names; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partition_with_auth_result__isset { + _ThriftHiveMetastore_get_partition_with_auth_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partition_with_auth_result__isset; + +class ThriftHiveMetastore_get_partition_with_auth_result { + public: + + ThriftHiveMetastore_get_partition_with_auth_result(const ThriftHiveMetastore_get_partition_with_auth_result&); + ThriftHiveMetastore_get_partition_with_auth_result& operator=(const ThriftHiveMetastore_get_partition_with_auth_result&); + ThriftHiveMetastore_get_partition_with_auth_result() { + } + + virtual ~ThriftHiveMetastore_get_partition_with_auth_result() noexcept; + Partition success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partition_with_auth_result__isset __isset; + + void __set_success(const Partition& val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_partition_with_auth_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partition_with_auth_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partition_with_auth_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partition_with_auth_presult__isset { + _ThriftHiveMetastore_get_partition_with_auth_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partition_with_auth_presult__isset; + +class ThriftHiveMetastore_get_partition_with_auth_presult { + public: + + + virtual ~ThriftHiveMetastore_get_partition_with_auth_presult() noexcept; + Partition* success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partition_with_auth_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_partition_by_name_args__isset { + _ThriftHiveMetastore_get_partition_by_name_args__isset() : db_name(false), tbl_name(false), part_name(false) {} + bool db_name :1; + bool tbl_name :1; + bool part_name :1; +} _ThriftHiveMetastore_get_partition_by_name_args__isset; + +class ThriftHiveMetastore_get_partition_by_name_args { + public: + + ThriftHiveMetastore_get_partition_by_name_args(const ThriftHiveMetastore_get_partition_by_name_args&); + ThriftHiveMetastore_get_partition_by_name_args& operator=(const ThriftHiveMetastore_get_partition_by_name_args&); + ThriftHiveMetastore_get_partition_by_name_args() : db_name(), tbl_name(), part_name() { + } + + virtual ~ThriftHiveMetastore_get_partition_by_name_args() noexcept; + std::string db_name; + std::string tbl_name; + std::string part_name; + + _ThriftHiveMetastore_get_partition_by_name_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_partition_by_name_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_name == rhs.part_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partition_by_name_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partition_by_name_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_partition_by_name_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_partition_by_name_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::string* part_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partition_by_name_result__isset { + _ThriftHiveMetastore_get_partition_by_name_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partition_by_name_result__isset; + +class ThriftHiveMetastore_get_partition_by_name_result { + public: + + ThriftHiveMetastore_get_partition_by_name_result(const ThriftHiveMetastore_get_partition_by_name_result&); + ThriftHiveMetastore_get_partition_by_name_result& operator=(const ThriftHiveMetastore_get_partition_by_name_result&); + ThriftHiveMetastore_get_partition_by_name_result() { + } + + virtual ~ThriftHiveMetastore_get_partition_by_name_result() noexcept; + Partition success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partition_by_name_result__isset __isset; + + void __set_success(const Partition& val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_partition_by_name_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partition_by_name_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partition_by_name_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partition_by_name_presult__isset { + _ThriftHiveMetastore_get_partition_by_name_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partition_by_name_presult__isset; + +class ThriftHiveMetastore_get_partition_by_name_presult { + public: + + + virtual ~ThriftHiveMetastore_get_partition_by_name_presult() noexcept; + Partition* success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partition_by_name_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_args__isset { + _ThriftHiveMetastore_get_partitions_args__isset() : db_name(false), tbl_name(false), max_parts(true) {} + bool db_name :1; + bool tbl_name :1; + bool max_parts :1; +} _ThriftHiveMetastore_get_partitions_args__isset; + +class ThriftHiveMetastore_get_partitions_args { + public: + + ThriftHiveMetastore_get_partitions_args(const ThriftHiveMetastore_get_partitions_args&); + ThriftHiveMetastore_get_partitions_args& operator=(const ThriftHiveMetastore_get_partitions_args&); + ThriftHiveMetastore_get_partitions_args() : db_name(), tbl_name(), max_parts(-1) { + } + + virtual ~ThriftHiveMetastore_get_partitions_args() noexcept; + std::string db_name; + std::string tbl_name; + int16_t max_parts; + + _ThriftHiveMetastore_get_partitions_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_max_parts(const int16_t val); + + bool operator == (const ThriftHiveMetastore_get_partitions_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(max_parts == rhs.max_parts)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_partitions_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const int16_t* max_parts; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_result__isset { + _ThriftHiveMetastore_get_partitions_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_result__isset; + +class ThriftHiveMetastore_get_partitions_result { + public: + + ThriftHiveMetastore_get_partitions_result(const ThriftHiveMetastore_get_partitions_result&); + ThriftHiveMetastore_get_partitions_result& operator=(const ThriftHiveMetastore_get_partitions_result&); + ThriftHiveMetastore_get_partitions_result() { + } + + virtual ~ThriftHiveMetastore_get_partitions_result() noexcept; + std::vector success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_partitions_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_partitions_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_presult__isset { + _ThriftHiveMetastore_get_partitions_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_presult__isset; + +class ThriftHiveMetastore_get_partitions_presult { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_presult() noexcept; + std::vector * success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_partitions_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_with_auth_args__isset { + _ThriftHiveMetastore_get_partitions_with_auth_args__isset() : db_name(false), tbl_name(false), max_parts(true), user_name(false), group_names(false) {} + bool db_name :1; + bool tbl_name :1; + bool max_parts :1; + bool user_name :1; + bool group_names :1; +} _ThriftHiveMetastore_get_partitions_with_auth_args__isset; + +class ThriftHiveMetastore_get_partitions_with_auth_args { + public: + + ThriftHiveMetastore_get_partitions_with_auth_args(const ThriftHiveMetastore_get_partitions_with_auth_args&); + ThriftHiveMetastore_get_partitions_with_auth_args& operator=(const ThriftHiveMetastore_get_partitions_with_auth_args&); + ThriftHiveMetastore_get_partitions_with_auth_args() : db_name(), tbl_name(), max_parts(-1), user_name() { + } + + virtual ~ThriftHiveMetastore_get_partitions_with_auth_args() noexcept; + std::string db_name; + std::string tbl_name; + int16_t max_parts; + std::string user_name; + std::vector group_names; + + _ThriftHiveMetastore_get_partitions_with_auth_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_max_parts(const int16_t val); + + void __set_user_name(const std::string& val); + + void __set_group_names(const std::vector & val); + + bool operator == (const ThriftHiveMetastore_get_partitions_with_auth_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(max_parts == rhs.max_parts)) + return false; + if (!(user_name == rhs.user_name)) + return false; + if (!(group_names == rhs.group_names)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_with_auth_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_with_auth_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_partitions_with_auth_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_with_auth_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const int16_t* max_parts; + const std::string* user_name; + const std::vector * group_names; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_with_auth_result__isset { + _ThriftHiveMetastore_get_partitions_with_auth_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_with_auth_result__isset; + +class ThriftHiveMetastore_get_partitions_with_auth_result { + public: + + ThriftHiveMetastore_get_partitions_with_auth_result(const ThriftHiveMetastore_get_partitions_with_auth_result&); + ThriftHiveMetastore_get_partitions_with_auth_result& operator=(const ThriftHiveMetastore_get_partitions_with_auth_result&); + ThriftHiveMetastore_get_partitions_with_auth_result() { + } + + virtual ~ThriftHiveMetastore_get_partitions_with_auth_result() noexcept; + std::vector success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_partitions_with_auth_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_partitions_with_auth_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_with_auth_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_with_auth_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_with_auth_presult__isset { + _ThriftHiveMetastore_get_partitions_with_auth_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_with_auth_presult__isset; + +class ThriftHiveMetastore_get_partitions_with_auth_presult { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_with_auth_presult() noexcept; + std::vector * success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_partitions_with_auth_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_pspec_args__isset { + _ThriftHiveMetastore_get_partitions_pspec_args__isset() : db_name(false), tbl_name(false), max_parts(true) {} + bool db_name :1; + bool tbl_name :1; + bool max_parts :1; +} _ThriftHiveMetastore_get_partitions_pspec_args__isset; + +class ThriftHiveMetastore_get_partitions_pspec_args { + public: + + ThriftHiveMetastore_get_partitions_pspec_args(const ThriftHiveMetastore_get_partitions_pspec_args&); + ThriftHiveMetastore_get_partitions_pspec_args& operator=(const ThriftHiveMetastore_get_partitions_pspec_args&); + ThriftHiveMetastore_get_partitions_pspec_args() : db_name(), tbl_name(), max_parts(-1) { + } + + virtual ~ThriftHiveMetastore_get_partitions_pspec_args() noexcept; + std::string db_name; + std::string tbl_name; + int32_t max_parts; + + _ThriftHiveMetastore_get_partitions_pspec_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_max_parts(const int32_t val); + + bool operator == (const ThriftHiveMetastore_get_partitions_pspec_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(max_parts == rhs.max_parts)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_pspec_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_pspec_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_partitions_pspec_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_pspec_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const int32_t* max_parts; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_pspec_result__isset { + _ThriftHiveMetastore_get_partitions_pspec_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_pspec_result__isset; + +class ThriftHiveMetastore_get_partitions_pspec_result { + public: + + ThriftHiveMetastore_get_partitions_pspec_result(const ThriftHiveMetastore_get_partitions_pspec_result&); + ThriftHiveMetastore_get_partitions_pspec_result& operator=(const ThriftHiveMetastore_get_partitions_pspec_result&); + ThriftHiveMetastore_get_partitions_pspec_result() { + } + + virtual ~ThriftHiveMetastore_get_partitions_pspec_result() noexcept; + std::vector success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_partitions_pspec_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_partitions_pspec_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_pspec_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_pspec_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_pspec_presult__isset { + _ThriftHiveMetastore_get_partitions_pspec_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_pspec_presult__isset; + +class ThriftHiveMetastore_get_partitions_pspec_presult { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_pspec_presult() noexcept; + std::vector * success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_partitions_pspec_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_partition_names_args__isset { + _ThriftHiveMetastore_get_partition_names_args__isset() : db_name(false), tbl_name(false), max_parts(true) {} + bool db_name :1; + bool tbl_name :1; + bool max_parts :1; +} _ThriftHiveMetastore_get_partition_names_args__isset; + +class ThriftHiveMetastore_get_partition_names_args { + public: + + ThriftHiveMetastore_get_partition_names_args(const ThriftHiveMetastore_get_partition_names_args&); + ThriftHiveMetastore_get_partition_names_args& operator=(const ThriftHiveMetastore_get_partition_names_args&); + ThriftHiveMetastore_get_partition_names_args() : db_name(), tbl_name(), max_parts(-1) { + } + + virtual ~ThriftHiveMetastore_get_partition_names_args() noexcept; + std::string db_name; + std::string tbl_name; + int16_t max_parts; + + _ThriftHiveMetastore_get_partition_names_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_max_parts(const int16_t val); + + bool operator == (const ThriftHiveMetastore_get_partition_names_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(max_parts == rhs.max_parts)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partition_names_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partition_names_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_partition_names_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_partition_names_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const int16_t* max_parts; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partition_names_result__isset { + _ThriftHiveMetastore_get_partition_names_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partition_names_result__isset; + +class ThriftHiveMetastore_get_partition_names_result { + public: + + ThriftHiveMetastore_get_partition_names_result(const ThriftHiveMetastore_get_partition_names_result&); + ThriftHiveMetastore_get_partition_names_result& operator=(const ThriftHiveMetastore_get_partition_names_result&); + ThriftHiveMetastore_get_partition_names_result() { + } + + virtual ~ThriftHiveMetastore_get_partition_names_result() noexcept; + std::vector success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_partition_names_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_partition_names_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partition_names_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partition_names_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partition_names_presult__isset { + _ThriftHiveMetastore_get_partition_names_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partition_names_presult__isset; + +class ThriftHiveMetastore_get_partition_names_presult { + public: + + + virtual ~ThriftHiveMetastore_get_partition_names_presult() noexcept; + std::vector * success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_partition_names_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_partition_values_args__isset { + _ThriftHiveMetastore_get_partition_values_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_get_partition_values_args__isset; + +class ThriftHiveMetastore_get_partition_values_args { + public: + + ThriftHiveMetastore_get_partition_values_args(const ThriftHiveMetastore_get_partition_values_args&); + ThriftHiveMetastore_get_partition_values_args& operator=(const ThriftHiveMetastore_get_partition_values_args&); + ThriftHiveMetastore_get_partition_values_args() { + } + + virtual ~ThriftHiveMetastore_get_partition_values_args() noexcept; + PartitionValuesRequest request; + + _ThriftHiveMetastore_get_partition_values_args__isset __isset; + + void __set_request(const PartitionValuesRequest& val); + + bool operator == (const ThriftHiveMetastore_get_partition_values_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partition_values_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partition_values_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_partition_values_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_partition_values_pargs() noexcept; + const PartitionValuesRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partition_values_result__isset { + _ThriftHiveMetastore_get_partition_values_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partition_values_result__isset; + +class ThriftHiveMetastore_get_partition_values_result { + public: + + ThriftHiveMetastore_get_partition_values_result(const ThriftHiveMetastore_get_partition_values_result&); + ThriftHiveMetastore_get_partition_values_result& operator=(const ThriftHiveMetastore_get_partition_values_result&); + ThriftHiveMetastore_get_partition_values_result() { + } + + virtual ~ThriftHiveMetastore_get_partition_values_result() noexcept; + PartitionValuesResponse success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partition_values_result__isset __isset; + + void __set_success(const PartitionValuesResponse& val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_partition_values_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partition_values_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partition_values_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partition_values_presult__isset { + _ThriftHiveMetastore_get_partition_values_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partition_values_presult__isset; + +class ThriftHiveMetastore_get_partition_values_presult { + public: + + + virtual ~ThriftHiveMetastore_get_partition_values_presult() noexcept; + PartitionValuesResponse* success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partition_values_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_ps_args__isset { + _ThriftHiveMetastore_get_partitions_ps_args__isset() : db_name(false), tbl_name(false), part_vals(false), max_parts(true) {} + bool db_name :1; + bool tbl_name :1; + bool part_vals :1; + bool max_parts :1; +} _ThriftHiveMetastore_get_partitions_ps_args__isset; + +class ThriftHiveMetastore_get_partitions_ps_args { + public: + + ThriftHiveMetastore_get_partitions_ps_args(const ThriftHiveMetastore_get_partitions_ps_args&); + ThriftHiveMetastore_get_partitions_ps_args& operator=(const ThriftHiveMetastore_get_partitions_ps_args&); + ThriftHiveMetastore_get_partitions_ps_args() : db_name(), tbl_name(), max_parts(-1) { + } + + virtual ~ThriftHiveMetastore_get_partitions_ps_args() noexcept; + std::string db_name; + std::string tbl_name; + std::vector part_vals; + int16_t max_parts; + + _ThriftHiveMetastore_get_partitions_ps_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_vals(const std::vector & val); + + void __set_max_parts(const int16_t val); + + bool operator == (const ThriftHiveMetastore_get_partitions_ps_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_vals == rhs.part_vals)) + return false; + if (!(max_parts == rhs.max_parts)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_ps_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_ps_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_partitions_ps_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_ps_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::vector * part_vals; + const int16_t* max_parts; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_ps_result__isset { + _ThriftHiveMetastore_get_partitions_ps_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_ps_result__isset; + +class ThriftHiveMetastore_get_partitions_ps_result { + public: + + ThriftHiveMetastore_get_partitions_ps_result(const ThriftHiveMetastore_get_partitions_ps_result&); + ThriftHiveMetastore_get_partitions_ps_result& operator=(const ThriftHiveMetastore_get_partitions_ps_result&); + ThriftHiveMetastore_get_partitions_ps_result() { + } + + virtual ~ThriftHiveMetastore_get_partitions_ps_result() noexcept; + std::vector success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partitions_ps_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_partitions_ps_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_ps_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_ps_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_ps_presult__isset { + _ThriftHiveMetastore_get_partitions_ps_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_ps_presult__isset; + +class ThriftHiveMetastore_get_partitions_ps_presult { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_ps_presult() noexcept; + std::vector * success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partitions_ps_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_ps_with_auth_args__isset { + _ThriftHiveMetastore_get_partitions_ps_with_auth_args__isset() : db_name(false), tbl_name(false), part_vals(false), max_parts(true), user_name(false), group_names(false) {} + bool db_name :1; + bool tbl_name :1; + bool part_vals :1; + bool max_parts :1; + bool user_name :1; + bool group_names :1; +} _ThriftHiveMetastore_get_partitions_ps_with_auth_args__isset; + +class ThriftHiveMetastore_get_partitions_ps_with_auth_args { + public: + + ThriftHiveMetastore_get_partitions_ps_with_auth_args(const ThriftHiveMetastore_get_partitions_ps_with_auth_args&); + ThriftHiveMetastore_get_partitions_ps_with_auth_args& operator=(const ThriftHiveMetastore_get_partitions_ps_with_auth_args&); + ThriftHiveMetastore_get_partitions_ps_with_auth_args() : db_name(), tbl_name(), max_parts(-1), user_name() { + } + + virtual ~ThriftHiveMetastore_get_partitions_ps_with_auth_args() noexcept; + std::string db_name; + std::string tbl_name; + std::vector part_vals; + int16_t max_parts; + std::string user_name; + std::vector group_names; + + _ThriftHiveMetastore_get_partitions_ps_with_auth_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_vals(const std::vector & val); + + void __set_max_parts(const int16_t val); + + void __set_user_name(const std::string& val); + + void __set_group_names(const std::vector & val); + + bool operator == (const ThriftHiveMetastore_get_partitions_ps_with_auth_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_vals == rhs.part_vals)) + return false; + if (!(max_parts == rhs.max_parts)) + return false; + if (!(user_name == rhs.user_name)) + return false; + if (!(group_names == rhs.group_names)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_ps_with_auth_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_ps_with_auth_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_partitions_ps_with_auth_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_ps_with_auth_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::vector * part_vals; + const int16_t* max_parts; + const std::string* user_name; + const std::vector * group_names; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_ps_with_auth_result__isset { + _ThriftHiveMetastore_get_partitions_ps_with_auth_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_ps_with_auth_result__isset; + +class ThriftHiveMetastore_get_partitions_ps_with_auth_result { + public: + + ThriftHiveMetastore_get_partitions_ps_with_auth_result(const ThriftHiveMetastore_get_partitions_ps_with_auth_result&); + ThriftHiveMetastore_get_partitions_ps_with_auth_result& operator=(const ThriftHiveMetastore_get_partitions_ps_with_auth_result&); + ThriftHiveMetastore_get_partitions_ps_with_auth_result() { + } + + virtual ~ThriftHiveMetastore_get_partitions_ps_with_auth_result() noexcept; + std::vector success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_partitions_ps_with_auth_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_partitions_ps_with_auth_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_ps_with_auth_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_ps_with_auth_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_ps_with_auth_presult__isset { + _ThriftHiveMetastore_get_partitions_ps_with_auth_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_ps_with_auth_presult__isset; + +class ThriftHiveMetastore_get_partitions_ps_with_auth_presult { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_ps_with_auth_presult() noexcept; + std::vector * success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_partitions_ps_with_auth_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_partition_names_ps_args__isset { + _ThriftHiveMetastore_get_partition_names_ps_args__isset() : db_name(false), tbl_name(false), part_vals(false), max_parts(true) {} + bool db_name :1; + bool tbl_name :1; + bool part_vals :1; + bool max_parts :1; +} _ThriftHiveMetastore_get_partition_names_ps_args__isset; + +class ThriftHiveMetastore_get_partition_names_ps_args { + public: + + ThriftHiveMetastore_get_partition_names_ps_args(const ThriftHiveMetastore_get_partition_names_ps_args&); + ThriftHiveMetastore_get_partition_names_ps_args& operator=(const ThriftHiveMetastore_get_partition_names_ps_args&); + ThriftHiveMetastore_get_partition_names_ps_args() : db_name(), tbl_name(), max_parts(-1) { + } + + virtual ~ThriftHiveMetastore_get_partition_names_ps_args() noexcept; + std::string db_name; + std::string tbl_name; + std::vector part_vals; + int16_t max_parts; + + _ThriftHiveMetastore_get_partition_names_ps_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_vals(const std::vector & val); + + void __set_max_parts(const int16_t val); + + bool operator == (const ThriftHiveMetastore_get_partition_names_ps_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_vals == rhs.part_vals)) + return false; + if (!(max_parts == rhs.max_parts)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partition_names_ps_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partition_names_ps_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_partition_names_ps_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_partition_names_ps_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::vector * part_vals; + const int16_t* max_parts; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partition_names_ps_result__isset { + _ThriftHiveMetastore_get_partition_names_ps_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partition_names_ps_result__isset; + +class ThriftHiveMetastore_get_partition_names_ps_result { + public: + + ThriftHiveMetastore_get_partition_names_ps_result(const ThriftHiveMetastore_get_partition_names_ps_result&); + ThriftHiveMetastore_get_partition_names_ps_result& operator=(const ThriftHiveMetastore_get_partition_names_ps_result&); + ThriftHiveMetastore_get_partition_names_ps_result() { + } + + virtual ~ThriftHiveMetastore_get_partition_names_ps_result() noexcept; + std::vector success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partition_names_ps_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_partition_names_ps_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partition_names_ps_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partition_names_ps_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partition_names_ps_presult__isset { + _ThriftHiveMetastore_get_partition_names_ps_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partition_names_ps_presult__isset; + +class ThriftHiveMetastore_get_partition_names_ps_presult { + public: + + + virtual ~ThriftHiveMetastore_get_partition_names_ps_presult() noexcept; + std::vector * success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partition_names_ps_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_by_filter_args__isset { + _ThriftHiveMetastore_get_partitions_by_filter_args__isset() : db_name(false), tbl_name(false), filter(false), max_parts(true) {} + bool db_name :1; + bool tbl_name :1; + bool filter :1; + bool max_parts :1; +} _ThriftHiveMetastore_get_partitions_by_filter_args__isset; + +class ThriftHiveMetastore_get_partitions_by_filter_args { + public: + + ThriftHiveMetastore_get_partitions_by_filter_args(const ThriftHiveMetastore_get_partitions_by_filter_args&); + ThriftHiveMetastore_get_partitions_by_filter_args& operator=(const ThriftHiveMetastore_get_partitions_by_filter_args&); + ThriftHiveMetastore_get_partitions_by_filter_args() : db_name(), tbl_name(), filter(), max_parts(-1) { + } + + virtual ~ThriftHiveMetastore_get_partitions_by_filter_args() noexcept; + std::string db_name; + std::string tbl_name; + std::string filter; + int16_t max_parts; + + _ThriftHiveMetastore_get_partitions_by_filter_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_filter(const std::string& val); + + void __set_max_parts(const int16_t val); + + bool operator == (const ThriftHiveMetastore_get_partitions_by_filter_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(filter == rhs.filter)) + return false; + if (!(max_parts == rhs.max_parts)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_by_filter_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_by_filter_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_partitions_by_filter_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_by_filter_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::string* filter; + const int16_t* max_parts; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_by_filter_result__isset { + _ThriftHiveMetastore_get_partitions_by_filter_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_by_filter_result__isset; + +class ThriftHiveMetastore_get_partitions_by_filter_result { + public: + + ThriftHiveMetastore_get_partitions_by_filter_result(const ThriftHiveMetastore_get_partitions_by_filter_result&); + ThriftHiveMetastore_get_partitions_by_filter_result& operator=(const ThriftHiveMetastore_get_partitions_by_filter_result&); + ThriftHiveMetastore_get_partitions_by_filter_result() { + } + + virtual ~ThriftHiveMetastore_get_partitions_by_filter_result() noexcept; + std::vector success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partitions_by_filter_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_partitions_by_filter_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_by_filter_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_by_filter_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_by_filter_presult__isset { + _ThriftHiveMetastore_get_partitions_by_filter_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_by_filter_presult__isset; + +class ThriftHiveMetastore_get_partitions_by_filter_presult { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_by_filter_presult() noexcept; + std::vector * success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partitions_by_filter_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_part_specs_by_filter_args__isset { + _ThriftHiveMetastore_get_part_specs_by_filter_args__isset() : db_name(false), tbl_name(false), filter(false), max_parts(true) {} + bool db_name :1; + bool tbl_name :1; + bool filter :1; + bool max_parts :1; +} _ThriftHiveMetastore_get_part_specs_by_filter_args__isset; + +class ThriftHiveMetastore_get_part_specs_by_filter_args { + public: + + ThriftHiveMetastore_get_part_specs_by_filter_args(const ThriftHiveMetastore_get_part_specs_by_filter_args&); + ThriftHiveMetastore_get_part_specs_by_filter_args& operator=(const ThriftHiveMetastore_get_part_specs_by_filter_args&); + ThriftHiveMetastore_get_part_specs_by_filter_args() : db_name(), tbl_name(), filter(), max_parts(-1) { + } + + virtual ~ThriftHiveMetastore_get_part_specs_by_filter_args() noexcept; + std::string db_name; + std::string tbl_name; + std::string filter; + int32_t max_parts; + + _ThriftHiveMetastore_get_part_specs_by_filter_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_filter(const std::string& val); + + void __set_max_parts(const int32_t val); + + bool operator == (const ThriftHiveMetastore_get_part_specs_by_filter_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(filter == rhs.filter)) + return false; + if (!(max_parts == rhs.max_parts)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_part_specs_by_filter_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_part_specs_by_filter_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_part_specs_by_filter_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_part_specs_by_filter_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::string* filter; + const int32_t* max_parts; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_part_specs_by_filter_result__isset { + _ThriftHiveMetastore_get_part_specs_by_filter_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_part_specs_by_filter_result__isset; + +class ThriftHiveMetastore_get_part_specs_by_filter_result { + public: + + ThriftHiveMetastore_get_part_specs_by_filter_result(const ThriftHiveMetastore_get_part_specs_by_filter_result&); + ThriftHiveMetastore_get_part_specs_by_filter_result& operator=(const ThriftHiveMetastore_get_part_specs_by_filter_result&); + ThriftHiveMetastore_get_part_specs_by_filter_result() { + } + + virtual ~ThriftHiveMetastore_get_part_specs_by_filter_result() noexcept; + std::vector success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_part_specs_by_filter_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_part_specs_by_filter_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_part_specs_by_filter_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_part_specs_by_filter_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_part_specs_by_filter_presult__isset { + _ThriftHiveMetastore_get_part_specs_by_filter_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_part_specs_by_filter_presult__isset; + +class ThriftHiveMetastore_get_part_specs_by_filter_presult { + public: + + + virtual ~ThriftHiveMetastore_get_part_specs_by_filter_presult() noexcept; + std::vector * success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_part_specs_by_filter_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_by_expr_args__isset { + _ThriftHiveMetastore_get_partitions_by_expr_args__isset() : req(false) {} + bool req :1; +} _ThriftHiveMetastore_get_partitions_by_expr_args__isset; + +class ThriftHiveMetastore_get_partitions_by_expr_args { + public: + + ThriftHiveMetastore_get_partitions_by_expr_args(const ThriftHiveMetastore_get_partitions_by_expr_args&); + ThriftHiveMetastore_get_partitions_by_expr_args& operator=(const ThriftHiveMetastore_get_partitions_by_expr_args&); + ThriftHiveMetastore_get_partitions_by_expr_args() { + } + + virtual ~ThriftHiveMetastore_get_partitions_by_expr_args() noexcept; + PartitionsByExprRequest req; + + _ThriftHiveMetastore_get_partitions_by_expr_args__isset __isset; + + void __set_req(const PartitionsByExprRequest& val); + + bool operator == (const ThriftHiveMetastore_get_partitions_by_expr_args & rhs) const + { + if (!(req == rhs.req)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_by_expr_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_by_expr_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_partitions_by_expr_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_by_expr_pargs() noexcept; + const PartitionsByExprRequest* req; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_by_expr_result__isset { + _ThriftHiveMetastore_get_partitions_by_expr_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_by_expr_result__isset; + +class ThriftHiveMetastore_get_partitions_by_expr_result { + public: + + ThriftHiveMetastore_get_partitions_by_expr_result(const ThriftHiveMetastore_get_partitions_by_expr_result&); + ThriftHiveMetastore_get_partitions_by_expr_result& operator=(const ThriftHiveMetastore_get_partitions_by_expr_result&); + ThriftHiveMetastore_get_partitions_by_expr_result() { + } + + virtual ~ThriftHiveMetastore_get_partitions_by_expr_result() noexcept; + PartitionsByExprResult success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partitions_by_expr_result__isset __isset; + + void __set_success(const PartitionsByExprResult& val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_partitions_by_expr_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_by_expr_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_by_expr_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_by_expr_presult__isset { + _ThriftHiveMetastore_get_partitions_by_expr_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_by_expr_presult__isset; + +class ThriftHiveMetastore_get_partitions_by_expr_presult { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_by_expr_presult() noexcept; + PartitionsByExprResult* success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partitions_by_expr_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_num_partitions_by_filter_args__isset { + _ThriftHiveMetastore_get_num_partitions_by_filter_args__isset() : db_name(false), tbl_name(false), filter(false) {} + bool db_name :1; + bool tbl_name :1; + bool filter :1; +} _ThriftHiveMetastore_get_num_partitions_by_filter_args__isset; + +class ThriftHiveMetastore_get_num_partitions_by_filter_args { + public: + + ThriftHiveMetastore_get_num_partitions_by_filter_args(const ThriftHiveMetastore_get_num_partitions_by_filter_args&); + ThriftHiveMetastore_get_num_partitions_by_filter_args& operator=(const ThriftHiveMetastore_get_num_partitions_by_filter_args&); + ThriftHiveMetastore_get_num_partitions_by_filter_args() : db_name(), tbl_name(), filter() { + } + + virtual ~ThriftHiveMetastore_get_num_partitions_by_filter_args() noexcept; + std::string db_name; + std::string tbl_name; + std::string filter; + + _ThriftHiveMetastore_get_num_partitions_by_filter_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_filter(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_num_partitions_by_filter_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(filter == rhs.filter)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_num_partitions_by_filter_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_num_partitions_by_filter_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_num_partitions_by_filter_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_num_partitions_by_filter_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::string* filter; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_num_partitions_by_filter_result__isset { + _ThriftHiveMetastore_get_num_partitions_by_filter_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_num_partitions_by_filter_result__isset; + +class ThriftHiveMetastore_get_num_partitions_by_filter_result { + public: + + ThriftHiveMetastore_get_num_partitions_by_filter_result(const ThriftHiveMetastore_get_num_partitions_by_filter_result&); + ThriftHiveMetastore_get_num_partitions_by_filter_result& operator=(const ThriftHiveMetastore_get_num_partitions_by_filter_result&); + ThriftHiveMetastore_get_num_partitions_by_filter_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_get_num_partitions_by_filter_result() noexcept; + int32_t success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_num_partitions_by_filter_result__isset __isset; + + void __set_success(const int32_t val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_num_partitions_by_filter_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_num_partitions_by_filter_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_num_partitions_by_filter_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_num_partitions_by_filter_presult__isset { + _ThriftHiveMetastore_get_num_partitions_by_filter_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_num_partitions_by_filter_presult__isset; + +class ThriftHiveMetastore_get_num_partitions_by_filter_presult { + public: + + + virtual ~ThriftHiveMetastore_get_num_partitions_by_filter_presult() noexcept; + int32_t* success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_num_partitions_by_filter_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_by_names_args__isset { + _ThriftHiveMetastore_get_partitions_by_names_args__isset() : db_name(false), tbl_name(false), names(false) {} + bool db_name :1; + bool tbl_name :1; + bool names :1; +} _ThriftHiveMetastore_get_partitions_by_names_args__isset; + +class ThriftHiveMetastore_get_partitions_by_names_args { + public: + + ThriftHiveMetastore_get_partitions_by_names_args(const ThriftHiveMetastore_get_partitions_by_names_args&); + ThriftHiveMetastore_get_partitions_by_names_args& operator=(const ThriftHiveMetastore_get_partitions_by_names_args&); + ThriftHiveMetastore_get_partitions_by_names_args() : db_name(), tbl_name() { + } + + virtual ~ThriftHiveMetastore_get_partitions_by_names_args() noexcept; + std::string db_name; + std::string tbl_name; + std::vector names; + + _ThriftHiveMetastore_get_partitions_by_names_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_names(const std::vector & val); + + bool operator == (const ThriftHiveMetastore_get_partitions_by_names_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(names == rhs.names)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_by_names_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_by_names_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_partitions_by_names_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_by_names_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::vector * names; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_by_names_result__isset { + _ThriftHiveMetastore_get_partitions_by_names_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_by_names_result__isset; + +class ThriftHiveMetastore_get_partitions_by_names_result { + public: + + ThriftHiveMetastore_get_partitions_by_names_result(const ThriftHiveMetastore_get_partitions_by_names_result&); + ThriftHiveMetastore_get_partitions_by_names_result& operator=(const ThriftHiveMetastore_get_partitions_by_names_result&); + ThriftHiveMetastore_get_partitions_by_names_result() { + } + + virtual ~ThriftHiveMetastore_get_partitions_by_names_result() noexcept; + std::vector success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partitions_by_names_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_partitions_by_names_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_by_names_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_by_names_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_by_names_presult__isset { + _ThriftHiveMetastore_get_partitions_by_names_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_by_names_presult__isset; + +class ThriftHiveMetastore_get_partitions_by_names_presult { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_by_names_presult() noexcept; + std::vector * success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_partitions_by_names_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_alter_partition_args__isset { + _ThriftHiveMetastore_alter_partition_args__isset() : db_name(false), tbl_name(false), new_part(false) {} + bool db_name :1; + bool tbl_name :1; + bool new_part :1; +} _ThriftHiveMetastore_alter_partition_args__isset; + +class ThriftHiveMetastore_alter_partition_args { + public: + + ThriftHiveMetastore_alter_partition_args(const ThriftHiveMetastore_alter_partition_args&); + ThriftHiveMetastore_alter_partition_args& operator=(const ThriftHiveMetastore_alter_partition_args&); + ThriftHiveMetastore_alter_partition_args() : db_name(), tbl_name() { + } + + virtual ~ThriftHiveMetastore_alter_partition_args() noexcept; + std::string db_name; + std::string tbl_name; + Partition new_part; + + _ThriftHiveMetastore_alter_partition_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_new_part(const Partition& val); + + bool operator == (const ThriftHiveMetastore_alter_partition_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(new_part == rhs.new_part)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_partition_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_partition_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_alter_partition_pargs { + public: + + + virtual ~ThriftHiveMetastore_alter_partition_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const Partition* new_part; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_partition_result__isset { + _ThriftHiveMetastore_alter_partition_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_partition_result__isset; + +class ThriftHiveMetastore_alter_partition_result { + public: + + ThriftHiveMetastore_alter_partition_result(const ThriftHiveMetastore_alter_partition_result&); + ThriftHiveMetastore_alter_partition_result& operator=(const ThriftHiveMetastore_alter_partition_result&); + ThriftHiveMetastore_alter_partition_result() { + } + + virtual ~ThriftHiveMetastore_alter_partition_result() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_partition_result__isset __isset; + + void __set_o1(const InvalidOperationException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_alter_partition_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_partition_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_partition_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_partition_presult__isset { + _ThriftHiveMetastore_alter_partition_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_partition_presult__isset; + +class ThriftHiveMetastore_alter_partition_presult { + public: + + + virtual ~ThriftHiveMetastore_alter_partition_presult() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_partition_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_alter_partitions_args__isset { + _ThriftHiveMetastore_alter_partitions_args__isset() : db_name(false), tbl_name(false), new_parts(false) {} + bool db_name :1; + bool tbl_name :1; + bool new_parts :1; +} _ThriftHiveMetastore_alter_partitions_args__isset; + +class ThriftHiveMetastore_alter_partitions_args { + public: + + ThriftHiveMetastore_alter_partitions_args(const ThriftHiveMetastore_alter_partitions_args&); + ThriftHiveMetastore_alter_partitions_args& operator=(const ThriftHiveMetastore_alter_partitions_args&); + ThriftHiveMetastore_alter_partitions_args() : db_name(), tbl_name() { + } + + virtual ~ThriftHiveMetastore_alter_partitions_args() noexcept; + std::string db_name; + std::string tbl_name; + std::vector new_parts; + + _ThriftHiveMetastore_alter_partitions_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_new_parts(const std::vector & val); + + bool operator == (const ThriftHiveMetastore_alter_partitions_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(new_parts == rhs.new_parts)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_partitions_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_partitions_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_alter_partitions_pargs { + public: + + + virtual ~ThriftHiveMetastore_alter_partitions_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::vector * new_parts; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_partitions_result__isset { + _ThriftHiveMetastore_alter_partitions_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_partitions_result__isset; + +class ThriftHiveMetastore_alter_partitions_result { + public: + + ThriftHiveMetastore_alter_partitions_result(const ThriftHiveMetastore_alter_partitions_result&); + ThriftHiveMetastore_alter_partitions_result& operator=(const ThriftHiveMetastore_alter_partitions_result&); + ThriftHiveMetastore_alter_partitions_result() { + } + + virtual ~ThriftHiveMetastore_alter_partitions_result() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_partitions_result__isset __isset; + + void __set_o1(const InvalidOperationException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_alter_partitions_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_partitions_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_partitions_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_partitions_presult__isset { + _ThriftHiveMetastore_alter_partitions_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_partitions_presult__isset; + +class ThriftHiveMetastore_alter_partitions_presult { + public: + + + virtual ~ThriftHiveMetastore_alter_partitions_presult() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_partitions_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_alter_partitions_with_environment_context_args__isset { + _ThriftHiveMetastore_alter_partitions_with_environment_context_args__isset() : db_name(false), tbl_name(false), new_parts(false), environment_context(false) {} + bool db_name :1; + bool tbl_name :1; + bool new_parts :1; + bool environment_context :1; +} _ThriftHiveMetastore_alter_partitions_with_environment_context_args__isset; + +class ThriftHiveMetastore_alter_partitions_with_environment_context_args { + public: + + ThriftHiveMetastore_alter_partitions_with_environment_context_args(const ThriftHiveMetastore_alter_partitions_with_environment_context_args&); + ThriftHiveMetastore_alter_partitions_with_environment_context_args& operator=(const ThriftHiveMetastore_alter_partitions_with_environment_context_args&); + ThriftHiveMetastore_alter_partitions_with_environment_context_args() : db_name(), tbl_name() { + } + + virtual ~ThriftHiveMetastore_alter_partitions_with_environment_context_args() noexcept; + std::string db_name; + std::string tbl_name; + std::vector new_parts; + EnvironmentContext environment_context; + + _ThriftHiveMetastore_alter_partitions_with_environment_context_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_new_parts(const std::vector & val); + + void __set_environment_context(const EnvironmentContext& val); + + bool operator == (const ThriftHiveMetastore_alter_partitions_with_environment_context_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(new_parts == rhs.new_parts)) + return false; + if (!(environment_context == rhs.environment_context)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_partitions_with_environment_context_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_partitions_with_environment_context_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_alter_partitions_with_environment_context_pargs { + public: + + + virtual ~ThriftHiveMetastore_alter_partitions_with_environment_context_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::vector * new_parts; + const EnvironmentContext* environment_context; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_partitions_with_environment_context_result__isset { + _ThriftHiveMetastore_alter_partitions_with_environment_context_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_partitions_with_environment_context_result__isset; + +class ThriftHiveMetastore_alter_partitions_with_environment_context_result { + public: + + ThriftHiveMetastore_alter_partitions_with_environment_context_result(const ThriftHiveMetastore_alter_partitions_with_environment_context_result&); + ThriftHiveMetastore_alter_partitions_with_environment_context_result& operator=(const ThriftHiveMetastore_alter_partitions_with_environment_context_result&); + ThriftHiveMetastore_alter_partitions_with_environment_context_result() { + } + + virtual ~ThriftHiveMetastore_alter_partitions_with_environment_context_result() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_partitions_with_environment_context_result__isset __isset; + + void __set_o1(const InvalidOperationException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_alter_partitions_with_environment_context_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_partitions_with_environment_context_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_partitions_with_environment_context_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_partitions_with_environment_context_presult__isset { + _ThriftHiveMetastore_alter_partitions_with_environment_context_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_partitions_with_environment_context_presult__isset; + +class ThriftHiveMetastore_alter_partitions_with_environment_context_presult { + public: + + + virtual ~ThriftHiveMetastore_alter_partitions_with_environment_context_presult() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_partitions_with_environment_context_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_alter_partition_with_environment_context_args__isset { + _ThriftHiveMetastore_alter_partition_with_environment_context_args__isset() : db_name(false), tbl_name(false), new_part(false), environment_context(false) {} + bool db_name :1; + bool tbl_name :1; + bool new_part :1; + bool environment_context :1; +} _ThriftHiveMetastore_alter_partition_with_environment_context_args__isset; + +class ThriftHiveMetastore_alter_partition_with_environment_context_args { + public: + + ThriftHiveMetastore_alter_partition_with_environment_context_args(const ThriftHiveMetastore_alter_partition_with_environment_context_args&); + ThriftHiveMetastore_alter_partition_with_environment_context_args& operator=(const ThriftHiveMetastore_alter_partition_with_environment_context_args&); + ThriftHiveMetastore_alter_partition_with_environment_context_args() : db_name(), tbl_name() { + } + + virtual ~ThriftHiveMetastore_alter_partition_with_environment_context_args() noexcept; + std::string db_name; + std::string tbl_name; + Partition new_part; + EnvironmentContext environment_context; + + _ThriftHiveMetastore_alter_partition_with_environment_context_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_new_part(const Partition& val); + + void __set_environment_context(const EnvironmentContext& val); + + bool operator == (const ThriftHiveMetastore_alter_partition_with_environment_context_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(new_part == rhs.new_part)) + return false; + if (!(environment_context == rhs.environment_context)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_partition_with_environment_context_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_partition_with_environment_context_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_alter_partition_with_environment_context_pargs { + public: + + + virtual ~ThriftHiveMetastore_alter_partition_with_environment_context_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const Partition* new_part; + const EnvironmentContext* environment_context; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_partition_with_environment_context_result__isset { + _ThriftHiveMetastore_alter_partition_with_environment_context_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_partition_with_environment_context_result__isset; + +class ThriftHiveMetastore_alter_partition_with_environment_context_result { + public: + + ThriftHiveMetastore_alter_partition_with_environment_context_result(const ThriftHiveMetastore_alter_partition_with_environment_context_result&); + ThriftHiveMetastore_alter_partition_with_environment_context_result& operator=(const ThriftHiveMetastore_alter_partition_with_environment_context_result&); + ThriftHiveMetastore_alter_partition_with_environment_context_result() { + } + + virtual ~ThriftHiveMetastore_alter_partition_with_environment_context_result() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_partition_with_environment_context_result__isset __isset; + + void __set_o1(const InvalidOperationException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_alter_partition_with_environment_context_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_partition_with_environment_context_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_partition_with_environment_context_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_partition_with_environment_context_presult__isset { + _ThriftHiveMetastore_alter_partition_with_environment_context_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_partition_with_environment_context_presult__isset; + +class ThriftHiveMetastore_alter_partition_with_environment_context_presult { + public: + + + virtual ~ThriftHiveMetastore_alter_partition_with_environment_context_presult() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_partition_with_environment_context_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_rename_partition_args__isset { + _ThriftHiveMetastore_rename_partition_args__isset() : db_name(false), tbl_name(false), part_vals(false), new_part(false) {} + bool db_name :1; + bool tbl_name :1; + bool part_vals :1; + bool new_part :1; +} _ThriftHiveMetastore_rename_partition_args__isset; + +class ThriftHiveMetastore_rename_partition_args { + public: + + ThriftHiveMetastore_rename_partition_args(const ThriftHiveMetastore_rename_partition_args&); + ThriftHiveMetastore_rename_partition_args& operator=(const ThriftHiveMetastore_rename_partition_args&); + ThriftHiveMetastore_rename_partition_args() : db_name(), tbl_name() { + } + + virtual ~ThriftHiveMetastore_rename_partition_args() noexcept; + std::string db_name; + std::string tbl_name; + std::vector part_vals; + Partition new_part; + + _ThriftHiveMetastore_rename_partition_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_vals(const std::vector & val); + + void __set_new_part(const Partition& val); + + bool operator == (const ThriftHiveMetastore_rename_partition_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_vals == rhs.part_vals)) + return false; + if (!(new_part == rhs.new_part)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_rename_partition_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_rename_partition_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_rename_partition_pargs { + public: + + + virtual ~ThriftHiveMetastore_rename_partition_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::vector * part_vals; + const Partition* new_part; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_rename_partition_result__isset { + _ThriftHiveMetastore_rename_partition_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_rename_partition_result__isset; + +class ThriftHiveMetastore_rename_partition_result { + public: + + ThriftHiveMetastore_rename_partition_result(const ThriftHiveMetastore_rename_partition_result&); + ThriftHiveMetastore_rename_partition_result& operator=(const ThriftHiveMetastore_rename_partition_result&); + ThriftHiveMetastore_rename_partition_result() { + } + + virtual ~ThriftHiveMetastore_rename_partition_result() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_rename_partition_result__isset __isset; + + void __set_o1(const InvalidOperationException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_rename_partition_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_rename_partition_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_rename_partition_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_rename_partition_presult__isset { + _ThriftHiveMetastore_rename_partition_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_rename_partition_presult__isset; + +class ThriftHiveMetastore_rename_partition_presult { + public: + + + virtual ~ThriftHiveMetastore_rename_partition_presult() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_rename_partition_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_partition_name_has_valid_characters_args__isset { + _ThriftHiveMetastore_partition_name_has_valid_characters_args__isset() : part_vals(false), throw_exception(false) {} + bool part_vals :1; + bool throw_exception :1; +} _ThriftHiveMetastore_partition_name_has_valid_characters_args__isset; + +class ThriftHiveMetastore_partition_name_has_valid_characters_args { + public: + + ThriftHiveMetastore_partition_name_has_valid_characters_args(const ThriftHiveMetastore_partition_name_has_valid_characters_args&); + ThriftHiveMetastore_partition_name_has_valid_characters_args& operator=(const ThriftHiveMetastore_partition_name_has_valid_characters_args&); + ThriftHiveMetastore_partition_name_has_valid_characters_args() : throw_exception(0) { + } + + virtual ~ThriftHiveMetastore_partition_name_has_valid_characters_args() noexcept; + std::vector part_vals; + bool throw_exception; + + _ThriftHiveMetastore_partition_name_has_valid_characters_args__isset __isset; + + void __set_part_vals(const std::vector & val); + + void __set_throw_exception(const bool val); + + bool operator == (const ThriftHiveMetastore_partition_name_has_valid_characters_args & rhs) const + { + if (!(part_vals == rhs.part_vals)) + return false; + if (!(throw_exception == rhs.throw_exception)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_partition_name_has_valid_characters_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_partition_name_has_valid_characters_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_partition_name_has_valid_characters_pargs { + public: + + + virtual ~ThriftHiveMetastore_partition_name_has_valid_characters_pargs() noexcept; + const std::vector * part_vals; + const bool* throw_exception; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_partition_name_has_valid_characters_result__isset { + _ThriftHiveMetastore_partition_name_has_valid_characters_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_partition_name_has_valid_characters_result__isset; + +class ThriftHiveMetastore_partition_name_has_valid_characters_result { + public: + + ThriftHiveMetastore_partition_name_has_valid_characters_result(const ThriftHiveMetastore_partition_name_has_valid_characters_result&); + ThriftHiveMetastore_partition_name_has_valid_characters_result& operator=(const ThriftHiveMetastore_partition_name_has_valid_characters_result&); + ThriftHiveMetastore_partition_name_has_valid_characters_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_partition_name_has_valid_characters_result() noexcept; + bool success; + MetaException o1; + + _ThriftHiveMetastore_partition_name_has_valid_characters_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_partition_name_has_valid_characters_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_partition_name_has_valid_characters_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_partition_name_has_valid_characters_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_partition_name_has_valid_characters_presult__isset { + _ThriftHiveMetastore_partition_name_has_valid_characters_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_partition_name_has_valid_characters_presult__isset; + +class ThriftHiveMetastore_partition_name_has_valid_characters_presult { + public: + + + virtual ~ThriftHiveMetastore_partition_name_has_valid_characters_presult() noexcept; + bool* success; + MetaException o1; + + _ThriftHiveMetastore_partition_name_has_valid_characters_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_config_value_args__isset { + _ThriftHiveMetastore_get_config_value_args__isset() : name(false), defaultValue(false) {} + bool name :1; + bool defaultValue :1; +} _ThriftHiveMetastore_get_config_value_args__isset; + +class ThriftHiveMetastore_get_config_value_args { + public: + + ThriftHiveMetastore_get_config_value_args(const ThriftHiveMetastore_get_config_value_args&); + ThriftHiveMetastore_get_config_value_args& operator=(const ThriftHiveMetastore_get_config_value_args&); + ThriftHiveMetastore_get_config_value_args() : name(), defaultValue() { + } + + virtual ~ThriftHiveMetastore_get_config_value_args() noexcept; + std::string name; + std::string defaultValue; + + _ThriftHiveMetastore_get_config_value_args__isset __isset; + + void __set_name(const std::string& val); + + void __set_defaultValue(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_config_value_args & rhs) const + { + if (!(name == rhs.name)) + return false; + if (!(defaultValue == rhs.defaultValue)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_config_value_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_config_value_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_config_value_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_config_value_pargs() noexcept; + const std::string* name; + const std::string* defaultValue; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_config_value_result__isset { + _ThriftHiveMetastore_get_config_value_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_config_value_result__isset; + +class ThriftHiveMetastore_get_config_value_result { + public: + + ThriftHiveMetastore_get_config_value_result(const ThriftHiveMetastore_get_config_value_result&); + ThriftHiveMetastore_get_config_value_result& operator=(const ThriftHiveMetastore_get_config_value_result&); + ThriftHiveMetastore_get_config_value_result() : success() { + } + + virtual ~ThriftHiveMetastore_get_config_value_result() noexcept; + std::string success; + ConfigValSecurityException o1; + + _ThriftHiveMetastore_get_config_value_result__isset __isset; + + void __set_success(const std::string& val); + + void __set_o1(const ConfigValSecurityException& val); + + bool operator == (const ThriftHiveMetastore_get_config_value_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_config_value_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_config_value_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_config_value_presult__isset { + _ThriftHiveMetastore_get_config_value_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_config_value_presult__isset; + +class ThriftHiveMetastore_get_config_value_presult { + public: + + + virtual ~ThriftHiveMetastore_get_config_value_presult() noexcept; + std::string* success; + ConfigValSecurityException o1; + + _ThriftHiveMetastore_get_config_value_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_partition_name_to_vals_args__isset { + _ThriftHiveMetastore_partition_name_to_vals_args__isset() : part_name(false) {} + bool part_name :1; +} _ThriftHiveMetastore_partition_name_to_vals_args__isset; + +class ThriftHiveMetastore_partition_name_to_vals_args { + public: + + ThriftHiveMetastore_partition_name_to_vals_args(const ThriftHiveMetastore_partition_name_to_vals_args&); + ThriftHiveMetastore_partition_name_to_vals_args& operator=(const ThriftHiveMetastore_partition_name_to_vals_args&); + ThriftHiveMetastore_partition_name_to_vals_args() : part_name() { + } + + virtual ~ThriftHiveMetastore_partition_name_to_vals_args() noexcept; + std::string part_name; + + _ThriftHiveMetastore_partition_name_to_vals_args__isset __isset; + + void __set_part_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_partition_name_to_vals_args & rhs) const + { + if (!(part_name == rhs.part_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_partition_name_to_vals_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_partition_name_to_vals_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_partition_name_to_vals_pargs { + public: + + + virtual ~ThriftHiveMetastore_partition_name_to_vals_pargs() noexcept; + const std::string* part_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_partition_name_to_vals_result__isset { + _ThriftHiveMetastore_partition_name_to_vals_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_partition_name_to_vals_result__isset; + +class ThriftHiveMetastore_partition_name_to_vals_result { + public: + + ThriftHiveMetastore_partition_name_to_vals_result(const ThriftHiveMetastore_partition_name_to_vals_result&); + ThriftHiveMetastore_partition_name_to_vals_result& operator=(const ThriftHiveMetastore_partition_name_to_vals_result&); + ThriftHiveMetastore_partition_name_to_vals_result() { + } + + virtual ~ThriftHiveMetastore_partition_name_to_vals_result() noexcept; + std::vector success; + MetaException o1; + + _ThriftHiveMetastore_partition_name_to_vals_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_partition_name_to_vals_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_partition_name_to_vals_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_partition_name_to_vals_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_partition_name_to_vals_presult__isset { + _ThriftHiveMetastore_partition_name_to_vals_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_partition_name_to_vals_presult__isset; + +class ThriftHiveMetastore_partition_name_to_vals_presult { + public: + + + virtual ~ThriftHiveMetastore_partition_name_to_vals_presult() noexcept; + std::vector * success; + MetaException o1; + + _ThriftHiveMetastore_partition_name_to_vals_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_partition_name_to_spec_args__isset { + _ThriftHiveMetastore_partition_name_to_spec_args__isset() : part_name(false) {} + bool part_name :1; +} _ThriftHiveMetastore_partition_name_to_spec_args__isset; + +class ThriftHiveMetastore_partition_name_to_spec_args { + public: + + ThriftHiveMetastore_partition_name_to_spec_args(const ThriftHiveMetastore_partition_name_to_spec_args&); + ThriftHiveMetastore_partition_name_to_spec_args& operator=(const ThriftHiveMetastore_partition_name_to_spec_args&); + ThriftHiveMetastore_partition_name_to_spec_args() : part_name() { + } + + virtual ~ThriftHiveMetastore_partition_name_to_spec_args() noexcept; + std::string part_name; + + _ThriftHiveMetastore_partition_name_to_spec_args__isset __isset; + + void __set_part_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_partition_name_to_spec_args & rhs) const + { + if (!(part_name == rhs.part_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_partition_name_to_spec_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_partition_name_to_spec_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_partition_name_to_spec_pargs { + public: + + + virtual ~ThriftHiveMetastore_partition_name_to_spec_pargs() noexcept; + const std::string* part_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_partition_name_to_spec_result__isset { + _ThriftHiveMetastore_partition_name_to_spec_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_partition_name_to_spec_result__isset; + +class ThriftHiveMetastore_partition_name_to_spec_result { + public: + + ThriftHiveMetastore_partition_name_to_spec_result(const ThriftHiveMetastore_partition_name_to_spec_result&); + ThriftHiveMetastore_partition_name_to_spec_result& operator=(const ThriftHiveMetastore_partition_name_to_spec_result&); + ThriftHiveMetastore_partition_name_to_spec_result() { + } + + virtual ~ThriftHiveMetastore_partition_name_to_spec_result() noexcept; + std::map success; + MetaException o1; + + _ThriftHiveMetastore_partition_name_to_spec_result__isset __isset; + + void __set_success(const std::map & val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_partition_name_to_spec_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_partition_name_to_spec_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_partition_name_to_spec_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_partition_name_to_spec_presult__isset { + _ThriftHiveMetastore_partition_name_to_spec_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_partition_name_to_spec_presult__isset; + +class ThriftHiveMetastore_partition_name_to_spec_presult { + public: + + + virtual ~ThriftHiveMetastore_partition_name_to_spec_presult() noexcept; + std::map * success; + MetaException o1; + + _ThriftHiveMetastore_partition_name_to_spec_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_markPartitionForEvent_args__isset { + _ThriftHiveMetastore_markPartitionForEvent_args__isset() : db_name(false), tbl_name(false), part_vals(false), eventType(false) {} + bool db_name :1; + bool tbl_name :1; + bool part_vals :1; + bool eventType :1; +} _ThriftHiveMetastore_markPartitionForEvent_args__isset; + +class ThriftHiveMetastore_markPartitionForEvent_args { + public: + + ThriftHiveMetastore_markPartitionForEvent_args(const ThriftHiveMetastore_markPartitionForEvent_args&); + ThriftHiveMetastore_markPartitionForEvent_args& operator=(const ThriftHiveMetastore_markPartitionForEvent_args&); + ThriftHiveMetastore_markPartitionForEvent_args() : db_name(), tbl_name(), eventType((PartitionEventType::type)0) { + } + + virtual ~ThriftHiveMetastore_markPartitionForEvent_args() noexcept; + std::string db_name; + std::string tbl_name; + std::map part_vals; + PartitionEventType::type eventType; + + _ThriftHiveMetastore_markPartitionForEvent_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_vals(const std::map & val); + + void __set_eventType(const PartitionEventType::type val); + + bool operator == (const ThriftHiveMetastore_markPartitionForEvent_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_vals == rhs.part_vals)) + return false; + if (!(eventType == rhs.eventType)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_markPartitionForEvent_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_markPartitionForEvent_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_markPartitionForEvent_pargs { + public: + + + virtual ~ThriftHiveMetastore_markPartitionForEvent_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::map * part_vals; + const PartitionEventType::type* eventType; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_markPartitionForEvent_result__isset { + _ThriftHiveMetastore_markPartitionForEvent_result__isset() : o1(false), o2(false), o3(false), o4(false), o5(false), o6(false) {} + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; + bool o5 :1; + bool o6 :1; +} _ThriftHiveMetastore_markPartitionForEvent_result__isset; + +class ThriftHiveMetastore_markPartitionForEvent_result { + public: + + ThriftHiveMetastore_markPartitionForEvent_result(const ThriftHiveMetastore_markPartitionForEvent_result&); + ThriftHiveMetastore_markPartitionForEvent_result& operator=(const ThriftHiveMetastore_markPartitionForEvent_result&); + ThriftHiveMetastore_markPartitionForEvent_result() { + } + + virtual ~ThriftHiveMetastore_markPartitionForEvent_result() noexcept; + MetaException o1; + NoSuchObjectException o2; + UnknownDBException o3; + UnknownTableException o4; + UnknownPartitionException o5; + InvalidPartitionException o6; + + _ThriftHiveMetastore_markPartitionForEvent_result__isset __isset; + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + void __set_o3(const UnknownDBException& val); + + void __set_o4(const UnknownTableException& val); + + void __set_o5(const UnknownPartitionException& val); + + void __set_o6(const InvalidPartitionException& val); + + bool operator == (const ThriftHiveMetastore_markPartitionForEvent_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + if (!(o5 == rhs.o5)) + return false; + if (!(o6 == rhs.o6)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_markPartitionForEvent_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_markPartitionForEvent_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_markPartitionForEvent_presult__isset { + _ThriftHiveMetastore_markPartitionForEvent_presult__isset() : o1(false), o2(false), o3(false), o4(false), o5(false), o6(false) {} + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; + bool o5 :1; + bool o6 :1; +} _ThriftHiveMetastore_markPartitionForEvent_presult__isset; + +class ThriftHiveMetastore_markPartitionForEvent_presult { + public: + + + virtual ~ThriftHiveMetastore_markPartitionForEvent_presult() noexcept; + MetaException o1; + NoSuchObjectException o2; + UnknownDBException o3; + UnknownTableException o4; + UnknownPartitionException o5; + InvalidPartitionException o6; + + _ThriftHiveMetastore_markPartitionForEvent_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_isPartitionMarkedForEvent_args__isset { + _ThriftHiveMetastore_isPartitionMarkedForEvent_args__isset() : db_name(false), tbl_name(false), part_vals(false), eventType(false) {} + bool db_name :1; + bool tbl_name :1; + bool part_vals :1; + bool eventType :1; +} _ThriftHiveMetastore_isPartitionMarkedForEvent_args__isset; + +class ThriftHiveMetastore_isPartitionMarkedForEvent_args { + public: + + ThriftHiveMetastore_isPartitionMarkedForEvent_args(const ThriftHiveMetastore_isPartitionMarkedForEvent_args&); + ThriftHiveMetastore_isPartitionMarkedForEvent_args& operator=(const ThriftHiveMetastore_isPartitionMarkedForEvent_args&); + ThriftHiveMetastore_isPartitionMarkedForEvent_args() : db_name(), tbl_name(), eventType((PartitionEventType::type)0) { + } + + virtual ~ThriftHiveMetastore_isPartitionMarkedForEvent_args() noexcept; + std::string db_name; + std::string tbl_name; + std::map part_vals; + PartitionEventType::type eventType; + + _ThriftHiveMetastore_isPartitionMarkedForEvent_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_vals(const std::map & val); + + void __set_eventType(const PartitionEventType::type val); + + bool operator == (const ThriftHiveMetastore_isPartitionMarkedForEvent_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_vals == rhs.part_vals)) + return false; + if (!(eventType == rhs.eventType)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_isPartitionMarkedForEvent_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_isPartitionMarkedForEvent_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_isPartitionMarkedForEvent_pargs { + public: + + + virtual ~ThriftHiveMetastore_isPartitionMarkedForEvent_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::map * part_vals; + const PartitionEventType::type* eventType; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_isPartitionMarkedForEvent_result__isset { + _ThriftHiveMetastore_isPartitionMarkedForEvent_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false), o5(false), o6(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; + bool o5 :1; + bool o6 :1; +} _ThriftHiveMetastore_isPartitionMarkedForEvent_result__isset; + +class ThriftHiveMetastore_isPartitionMarkedForEvent_result { + public: + + ThriftHiveMetastore_isPartitionMarkedForEvent_result(const ThriftHiveMetastore_isPartitionMarkedForEvent_result&); + ThriftHiveMetastore_isPartitionMarkedForEvent_result& operator=(const ThriftHiveMetastore_isPartitionMarkedForEvent_result&); + ThriftHiveMetastore_isPartitionMarkedForEvent_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_isPartitionMarkedForEvent_result() noexcept; + bool success; + MetaException o1; + NoSuchObjectException o2; + UnknownDBException o3; + UnknownTableException o4; + UnknownPartitionException o5; + InvalidPartitionException o6; + + _ThriftHiveMetastore_isPartitionMarkedForEvent_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + void __set_o3(const UnknownDBException& val); + + void __set_o4(const UnknownTableException& val); + + void __set_o5(const UnknownPartitionException& val); + + void __set_o6(const InvalidPartitionException& val); + + bool operator == (const ThriftHiveMetastore_isPartitionMarkedForEvent_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + if (!(o5 == rhs.o5)) + return false; + if (!(o6 == rhs.o6)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_isPartitionMarkedForEvent_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_isPartitionMarkedForEvent_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_isPartitionMarkedForEvent_presult__isset { + _ThriftHiveMetastore_isPartitionMarkedForEvent_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false), o5(false), o6(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; + bool o5 :1; + bool o6 :1; +} _ThriftHiveMetastore_isPartitionMarkedForEvent_presult__isset; + +class ThriftHiveMetastore_isPartitionMarkedForEvent_presult { + public: + + + virtual ~ThriftHiveMetastore_isPartitionMarkedForEvent_presult() noexcept; + bool* success; + MetaException o1; + NoSuchObjectException o2; + UnknownDBException o3; + UnknownTableException o4; + UnknownPartitionException o5; + InvalidPartitionException o6; + + _ThriftHiveMetastore_isPartitionMarkedForEvent_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_add_index_args__isset { + _ThriftHiveMetastore_add_index_args__isset() : new_index(false), index_table(false) {} + bool new_index :1; + bool index_table :1; +} _ThriftHiveMetastore_add_index_args__isset; + +class ThriftHiveMetastore_add_index_args { + public: + + ThriftHiveMetastore_add_index_args(const ThriftHiveMetastore_add_index_args&); + ThriftHiveMetastore_add_index_args& operator=(const ThriftHiveMetastore_add_index_args&); + ThriftHiveMetastore_add_index_args() { + } + + virtual ~ThriftHiveMetastore_add_index_args() noexcept; + Index new_index; + Table index_table; + + _ThriftHiveMetastore_add_index_args__isset __isset; + + void __set_new_index(const Index& val); + + void __set_index_table(const Table& val); + + bool operator == (const ThriftHiveMetastore_add_index_args & rhs) const + { + if (!(new_index == rhs.new_index)) + return false; + if (!(index_table == rhs.index_table)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_index_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_index_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_add_index_pargs { + public: + + + virtual ~ThriftHiveMetastore_add_index_pargs() noexcept; + const Index* new_index; + const Table* index_table; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_index_result__isset { + _ThriftHiveMetastore_add_index_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_add_index_result__isset; + +class ThriftHiveMetastore_add_index_result { + public: + + ThriftHiveMetastore_add_index_result(const ThriftHiveMetastore_add_index_result&); + ThriftHiveMetastore_add_index_result& operator=(const ThriftHiveMetastore_add_index_result&); + ThriftHiveMetastore_add_index_result() { + } + + virtual ~ThriftHiveMetastore_add_index_result() noexcept; + Index success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_add_index_result__isset __isset; + + void __set_success(const Index& val); + + void __set_o1(const InvalidObjectException& val); + + void __set_o2(const AlreadyExistsException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_add_index_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_index_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_index_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_index_presult__isset { + _ThriftHiveMetastore_add_index_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_add_index_presult__isset; + +class ThriftHiveMetastore_add_index_presult { + public: + + + virtual ~ThriftHiveMetastore_add_index_presult() noexcept; + Index* success; + InvalidObjectException o1; + AlreadyExistsException o2; + MetaException o3; + + _ThriftHiveMetastore_add_index_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_alter_index_args__isset { + _ThriftHiveMetastore_alter_index_args__isset() : dbname(false), base_tbl_name(false), idx_name(false), new_idx(false) {} + bool dbname :1; + bool base_tbl_name :1; + bool idx_name :1; + bool new_idx :1; +} _ThriftHiveMetastore_alter_index_args__isset; + +class ThriftHiveMetastore_alter_index_args { + public: + + ThriftHiveMetastore_alter_index_args(const ThriftHiveMetastore_alter_index_args&); + ThriftHiveMetastore_alter_index_args& operator=(const ThriftHiveMetastore_alter_index_args&); + ThriftHiveMetastore_alter_index_args() : dbname(), base_tbl_name(), idx_name() { + } + + virtual ~ThriftHiveMetastore_alter_index_args() noexcept; + std::string dbname; + std::string base_tbl_name; + std::string idx_name; + Index new_idx; + + _ThriftHiveMetastore_alter_index_args__isset __isset; + + void __set_dbname(const std::string& val); + + void __set_base_tbl_name(const std::string& val); + + void __set_idx_name(const std::string& val); + + void __set_new_idx(const Index& val); + + bool operator == (const ThriftHiveMetastore_alter_index_args & rhs) const + { + if (!(dbname == rhs.dbname)) + return false; + if (!(base_tbl_name == rhs.base_tbl_name)) + return false; + if (!(idx_name == rhs.idx_name)) + return false; + if (!(new_idx == rhs.new_idx)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_index_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_index_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_alter_index_pargs { + public: + + + virtual ~ThriftHiveMetastore_alter_index_pargs() noexcept; + const std::string* dbname; + const std::string* base_tbl_name; + const std::string* idx_name; + const Index* new_idx; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_index_result__isset { + _ThriftHiveMetastore_alter_index_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_index_result__isset; + +class ThriftHiveMetastore_alter_index_result { + public: + + ThriftHiveMetastore_alter_index_result(const ThriftHiveMetastore_alter_index_result&); + ThriftHiveMetastore_alter_index_result& operator=(const ThriftHiveMetastore_alter_index_result&); + ThriftHiveMetastore_alter_index_result() { + } + + virtual ~ThriftHiveMetastore_alter_index_result() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_index_result__isset __isset; + + void __set_o1(const InvalidOperationException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_alter_index_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_index_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_index_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_index_presult__isset { + _ThriftHiveMetastore_alter_index_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_index_presult__isset; + +class ThriftHiveMetastore_alter_index_presult { + public: + + + virtual ~ThriftHiveMetastore_alter_index_presult() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_index_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_drop_index_by_name_args__isset { + _ThriftHiveMetastore_drop_index_by_name_args__isset() : db_name(false), tbl_name(false), index_name(false), deleteData(false) {} + bool db_name :1; + bool tbl_name :1; + bool index_name :1; + bool deleteData :1; +} _ThriftHiveMetastore_drop_index_by_name_args__isset; + +class ThriftHiveMetastore_drop_index_by_name_args { + public: + + ThriftHiveMetastore_drop_index_by_name_args(const ThriftHiveMetastore_drop_index_by_name_args&); + ThriftHiveMetastore_drop_index_by_name_args& operator=(const ThriftHiveMetastore_drop_index_by_name_args&); + ThriftHiveMetastore_drop_index_by_name_args() : db_name(), tbl_name(), index_name(), deleteData(0) { + } + + virtual ~ThriftHiveMetastore_drop_index_by_name_args() noexcept; + std::string db_name; + std::string tbl_name; + std::string index_name; + bool deleteData; + + _ThriftHiveMetastore_drop_index_by_name_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_index_name(const std::string& val); + + void __set_deleteData(const bool val); + + bool operator == (const ThriftHiveMetastore_drop_index_by_name_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(index_name == rhs.index_name)) + return false; + if (!(deleteData == rhs.deleteData)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_index_by_name_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_index_by_name_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_drop_index_by_name_pargs { + public: + + + virtual ~ThriftHiveMetastore_drop_index_by_name_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::string* index_name; + const bool* deleteData; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_index_by_name_result__isset { + _ThriftHiveMetastore_drop_index_by_name_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_drop_index_by_name_result__isset; + +class ThriftHiveMetastore_drop_index_by_name_result { + public: + + ThriftHiveMetastore_drop_index_by_name_result(const ThriftHiveMetastore_drop_index_by_name_result&); + ThriftHiveMetastore_drop_index_by_name_result& operator=(const ThriftHiveMetastore_drop_index_by_name_result&); + ThriftHiveMetastore_drop_index_by_name_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_drop_index_by_name_result() noexcept; + bool success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_drop_index_by_name_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_drop_index_by_name_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_index_by_name_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_index_by_name_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_index_by_name_presult__isset { + _ThriftHiveMetastore_drop_index_by_name_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_drop_index_by_name_presult__isset; + +class ThriftHiveMetastore_drop_index_by_name_presult { + public: + + + virtual ~ThriftHiveMetastore_drop_index_by_name_presult() noexcept; + bool* success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_drop_index_by_name_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_index_by_name_args__isset { + _ThriftHiveMetastore_get_index_by_name_args__isset() : db_name(false), tbl_name(false), index_name(false) {} + bool db_name :1; + bool tbl_name :1; + bool index_name :1; +} _ThriftHiveMetastore_get_index_by_name_args__isset; + +class ThriftHiveMetastore_get_index_by_name_args { + public: + + ThriftHiveMetastore_get_index_by_name_args(const ThriftHiveMetastore_get_index_by_name_args&); + ThriftHiveMetastore_get_index_by_name_args& operator=(const ThriftHiveMetastore_get_index_by_name_args&); + ThriftHiveMetastore_get_index_by_name_args() : db_name(), tbl_name(), index_name() { + } + + virtual ~ThriftHiveMetastore_get_index_by_name_args() noexcept; + std::string db_name; + std::string tbl_name; + std::string index_name; + + _ThriftHiveMetastore_get_index_by_name_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_index_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_index_by_name_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(index_name == rhs.index_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_index_by_name_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_index_by_name_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_index_by_name_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_index_by_name_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::string* index_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_index_by_name_result__isset { + _ThriftHiveMetastore_get_index_by_name_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_index_by_name_result__isset; + +class ThriftHiveMetastore_get_index_by_name_result { + public: + + ThriftHiveMetastore_get_index_by_name_result(const ThriftHiveMetastore_get_index_by_name_result&); + ThriftHiveMetastore_get_index_by_name_result& operator=(const ThriftHiveMetastore_get_index_by_name_result&); + ThriftHiveMetastore_get_index_by_name_result() { + } + + virtual ~ThriftHiveMetastore_get_index_by_name_result() noexcept; + Index success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_index_by_name_result__isset __isset; + + void __set_success(const Index& val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_index_by_name_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_index_by_name_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_index_by_name_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_index_by_name_presult__isset { + _ThriftHiveMetastore_get_index_by_name_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_index_by_name_presult__isset; + +class ThriftHiveMetastore_get_index_by_name_presult { + public: + + + virtual ~ThriftHiveMetastore_get_index_by_name_presult() noexcept; + Index* success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_index_by_name_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_indexes_args__isset { + _ThriftHiveMetastore_get_indexes_args__isset() : db_name(false), tbl_name(false), max_indexes(true) {} + bool db_name :1; + bool tbl_name :1; + bool max_indexes :1; +} _ThriftHiveMetastore_get_indexes_args__isset; + +class ThriftHiveMetastore_get_indexes_args { + public: + + ThriftHiveMetastore_get_indexes_args(const ThriftHiveMetastore_get_indexes_args&); + ThriftHiveMetastore_get_indexes_args& operator=(const ThriftHiveMetastore_get_indexes_args&); + ThriftHiveMetastore_get_indexes_args() : db_name(), tbl_name(), max_indexes(-1) { + } + + virtual ~ThriftHiveMetastore_get_indexes_args() noexcept; + std::string db_name; + std::string tbl_name; + int16_t max_indexes; + + _ThriftHiveMetastore_get_indexes_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_max_indexes(const int16_t val); + + bool operator == (const ThriftHiveMetastore_get_indexes_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(max_indexes == rhs.max_indexes)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_indexes_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_indexes_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_indexes_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_indexes_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const int16_t* max_indexes; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_indexes_result__isset { + _ThriftHiveMetastore_get_indexes_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_indexes_result__isset; + +class ThriftHiveMetastore_get_indexes_result { + public: + + ThriftHiveMetastore_get_indexes_result(const ThriftHiveMetastore_get_indexes_result&); + ThriftHiveMetastore_get_indexes_result& operator=(const ThriftHiveMetastore_get_indexes_result&); + ThriftHiveMetastore_get_indexes_result() { + } + + virtual ~ThriftHiveMetastore_get_indexes_result() noexcept; + std::vector success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_indexes_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_indexes_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_indexes_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_indexes_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_indexes_presult__isset { + _ThriftHiveMetastore_get_indexes_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_indexes_presult__isset; + +class ThriftHiveMetastore_get_indexes_presult { + public: + + + virtual ~ThriftHiveMetastore_get_indexes_presult() noexcept; + std::vector * success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_indexes_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_index_names_args__isset { + _ThriftHiveMetastore_get_index_names_args__isset() : db_name(false), tbl_name(false), max_indexes(true) {} + bool db_name :1; + bool tbl_name :1; + bool max_indexes :1; +} _ThriftHiveMetastore_get_index_names_args__isset; + +class ThriftHiveMetastore_get_index_names_args { + public: + + ThriftHiveMetastore_get_index_names_args(const ThriftHiveMetastore_get_index_names_args&); + ThriftHiveMetastore_get_index_names_args& operator=(const ThriftHiveMetastore_get_index_names_args&); + ThriftHiveMetastore_get_index_names_args() : db_name(), tbl_name(), max_indexes(-1) { + } + + virtual ~ThriftHiveMetastore_get_index_names_args() noexcept; + std::string db_name; + std::string tbl_name; + int16_t max_indexes; + + _ThriftHiveMetastore_get_index_names_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_max_indexes(const int16_t val); + + bool operator == (const ThriftHiveMetastore_get_index_names_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(max_indexes == rhs.max_indexes)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_index_names_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_index_names_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_index_names_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_index_names_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const int16_t* max_indexes; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_index_names_result__isset { + _ThriftHiveMetastore_get_index_names_result__isset() : success(false), o2(false) {} + bool success :1; + bool o2 :1; +} _ThriftHiveMetastore_get_index_names_result__isset; + +class ThriftHiveMetastore_get_index_names_result { + public: + + ThriftHiveMetastore_get_index_names_result(const ThriftHiveMetastore_get_index_names_result&); + ThriftHiveMetastore_get_index_names_result& operator=(const ThriftHiveMetastore_get_index_names_result&); + ThriftHiveMetastore_get_index_names_result() { + } + + virtual ~ThriftHiveMetastore_get_index_names_result() noexcept; + std::vector success; + MetaException o2; + + _ThriftHiveMetastore_get_index_names_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_index_names_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_index_names_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_index_names_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_index_names_presult__isset { + _ThriftHiveMetastore_get_index_names_presult__isset() : success(false), o2(false) {} + bool success :1; + bool o2 :1; +} _ThriftHiveMetastore_get_index_names_presult__isset; + +class ThriftHiveMetastore_get_index_names_presult { + public: + + + virtual ~ThriftHiveMetastore_get_index_names_presult() noexcept; + std::vector * success; + MetaException o2; + + _ThriftHiveMetastore_get_index_names_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_primary_keys_args__isset { + _ThriftHiveMetastore_get_primary_keys_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_get_primary_keys_args__isset; + +class ThriftHiveMetastore_get_primary_keys_args { + public: + + ThriftHiveMetastore_get_primary_keys_args(const ThriftHiveMetastore_get_primary_keys_args&); + ThriftHiveMetastore_get_primary_keys_args& operator=(const ThriftHiveMetastore_get_primary_keys_args&); + ThriftHiveMetastore_get_primary_keys_args() { + } + + virtual ~ThriftHiveMetastore_get_primary_keys_args() noexcept; + PrimaryKeysRequest request; + + _ThriftHiveMetastore_get_primary_keys_args__isset __isset; + + void __set_request(const PrimaryKeysRequest& val); + + bool operator == (const ThriftHiveMetastore_get_primary_keys_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_primary_keys_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_primary_keys_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_primary_keys_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_primary_keys_pargs() noexcept; + const PrimaryKeysRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_primary_keys_result__isset { + _ThriftHiveMetastore_get_primary_keys_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_primary_keys_result__isset; + +class ThriftHiveMetastore_get_primary_keys_result { + public: + + ThriftHiveMetastore_get_primary_keys_result(const ThriftHiveMetastore_get_primary_keys_result&); + ThriftHiveMetastore_get_primary_keys_result& operator=(const ThriftHiveMetastore_get_primary_keys_result&); + ThriftHiveMetastore_get_primary_keys_result() { + } + + virtual ~ThriftHiveMetastore_get_primary_keys_result() noexcept; + PrimaryKeysResponse success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_primary_keys_result__isset __isset; + + void __set_success(const PrimaryKeysResponse& val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_primary_keys_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_primary_keys_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_primary_keys_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_primary_keys_presult__isset { + _ThriftHiveMetastore_get_primary_keys_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_primary_keys_presult__isset; + +class ThriftHiveMetastore_get_primary_keys_presult { + public: + + + virtual ~ThriftHiveMetastore_get_primary_keys_presult() noexcept; + PrimaryKeysResponse* success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_primary_keys_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_foreign_keys_args__isset { + _ThriftHiveMetastore_get_foreign_keys_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_get_foreign_keys_args__isset; + +class ThriftHiveMetastore_get_foreign_keys_args { + public: + + ThriftHiveMetastore_get_foreign_keys_args(const ThriftHiveMetastore_get_foreign_keys_args&); + ThriftHiveMetastore_get_foreign_keys_args& operator=(const ThriftHiveMetastore_get_foreign_keys_args&); + ThriftHiveMetastore_get_foreign_keys_args() { + } + + virtual ~ThriftHiveMetastore_get_foreign_keys_args() noexcept; + ForeignKeysRequest request; + + _ThriftHiveMetastore_get_foreign_keys_args__isset __isset; + + void __set_request(const ForeignKeysRequest& val); + + bool operator == (const ThriftHiveMetastore_get_foreign_keys_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_foreign_keys_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_foreign_keys_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_foreign_keys_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_foreign_keys_pargs() noexcept; + const ForeignKeysRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_foreign_keys_result__isset { + _ThriftHiveMetastore_get_foreign_keys_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_foreign_keys_result__isset; + +class ThriftHiveMetastore_get_foreign_keys_result { + public: + + ThriftHiveMetastore_get_foreign_keys_result(const ThriftHiveMetastore_get_foreign_keys_result&); + ThriftHiveMetastore_get_foreign_keys_result& operator=(const ThriftHiveMetastore_get_foreign_keys_result&); + ThriftHiveMetastore_get_foreign_keys_result() { + } + + virtual ~ThriftHiveMetastore_get_foreign_keys_result() noexcept; + ForeignKeysResponse success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_foreign_keys_result__isset __isset; + + void __set_success(const ForeignKeysResponse& val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_foreign_keys_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_foreign_keys_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_foreign_keys_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_foreign_keys_presult__isset { + _ThriftHiveMetastore_get_foreign_keys_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_foreign_keys_presult__isset; + +class ThriftHiveMetastore_get_foreign_keys_presult { + public: + + + virtual ~ThriftHiveMetastore_get_foreign_keys_presult() noexcept; + ForeignKeysResponse* success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_foreign_keys_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_unique_constraints_args__isset { + _ThriftHiveMetastore_get_unique_constraints_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_get_unique_constraints_args__isset; + +class ThriftHiveMetastore_get_unique_constraints_args { + public: + + ThriftHiveMetastore_get_unique_constraints_args(const ThriftHiveMetastore_get_unique_constraints_args&); + ThriftHiveMetastore_get_unique_constraints_args& operator=(const ThriftHiveMetastore_get_unique_constraints_args&); + ThriftHiveMetastore_get_unique_constraints_args() { + } + + virtual ~ThriftHiveMetastore_get_unique_constraints_args() noexcept; + UniqueConstraintsRequest request; + + _ThriftHiveMetastore_get_unique_constraints_args__isset __isset; + + void __set_request(const UniqueConstraintsRequest& val); + + bool operator == (const ThriftHiveMetastore_get_unique_constraints_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_unique_constraints_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_unique_constraints_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_unique_constraints_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_unique_constraints_pargs() noexcept; + const UniqueConstraintsRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_unique_constraints_result__isset { + _ThriftHiveMetastore_get_unique_constraints_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_unique_constraints_result__isset; + +class ThriftHiveMetastore_get_unique_constraints_result { + public: + + ThriftHiveMetastore_get_unique_constraints_result(const ThriftHiveMetastore_get_unique_constraints_result&); + ThriftHiveMetastore_get_unique_constraints_result& operator=(const ThriftHiveMetastore_get_unique_constraints_result&); + ThriftHiveMetastore_get_unique_constraints_result() { + } + + virtual ~ThriftHiveMetastore_get_unique_constraints_result() noexcept; + UniqueConstraintsResponse success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_unique_constraints_result__isset __isset; + + void __set_success(const UniqueConstraintsResponse& val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_unique_constraints_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_unique_constraints_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_unique_constraints_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_unique_constraints_presult__isset { + _ThriftHiveMetastore_get_unique_constraints_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_unique_constraints_presult__isset; + +class ThriftHiveMetastore_get_unique_constraints_presult { + public: + + + virtual ~ThriftHiveMetastore_get_unique_constraints_presult() noexcept; + UniqueConstraintsResponse* success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_unique_constraints_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_not_null_constraints_args__isset { + _ThriftHiveMetastore_get_not_null_constraints_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_get_not_null_constraints_args__isset; + +class ThriftHiveMetastore_get_not_null_constraints_args { + public: + + ThriftHiveMetastore_get_not_null_constraints_args(const ThriftHiveMetastore_get_not_null_constraints_args&); + ThriftHiveMetastore_get_not_null_constraints_args& operator=(const ThriftHiveMetastore_get_not_null_constraints_args&); + ThriftHiveMetastore_get_not_null_constraints_args() { + } + + virtual ~ThriftHiveMetastore_get_not_null_constraints_args() noexcept; + NotNullConstraintsRequest request; + + _ThriftHiveMetastore_get_not_null_constraints_args__isset __isset; + + void __set_request(const NotNullConstraintsRequest& val); + + bool operator == (const ThriftHiveMetastore_get_not_null_constraints_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_not_null_constraints_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_not_null_constraints_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_not_null_constraints_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_not_null_constraints_pargs() noexcept; + const NotNullConstraintsRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_not_null_constraints_result__isset { + _ThriftHiveMetastore_get_not_null_constraints_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_not_null_constraints_result__isset; + +class ThriftHiveMetastore_get_not_null_constraints_result { + public: + + ThriftHiveMetastore_get_not_null_constraints_result(const ThriftHiveMetastore_get_not_null_constraints_result&); + ThriftHiveMetastore_get_not_null_constraints_result& operator=(const ThriftHiveMetastore_get_not_null_constraints_result&); + ThriftHiveMetastore_get_not_null_constraints_result() { + } + + virtual ~ThriftHiveMetastore_get_not_null_constraints_result() noexcept; + NotNullConstraintsResponse success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_not_null_constraints_result__isset __isset; + + void __set_success(const NotNullConstraintsResponse& val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_not_null_constraints_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_not_null_constraints_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_not_null_constraints_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_not_null_constraints_presult__isset { + _ThriftHiveMetastore_get_not_null_constraints_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_not_null_constraints_presult__isset; + +class ThriftHiveMetastore_get_not_null_constraints_presult { + public: + + + virtual ~ThriftHiveMetastore_get_not_null_constraints_presult() noexcept; + NotNullConstraintsResponse* success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_not_null_constraints_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_update_table_column_statistics_args__isset { + _ThriftHiveMetastore_update_table_column_statistics_args__isset() : stats_obj(false) {} + bool stats_obj :1; +} _ThriftHiveMetastore_update_table_column_statistics_args__isset; + +class ThriftHiveMetastore_update_table_column_statistics_args { + public: + + ThriftHiveMetastore_update_table_column_statistics_args(const ThriftHiveMetastore_update_table_column_statistics_args&); + ThriftHiveMetastore_update_table_column_statistics_args& operator=(const ThriftHiveMetastore_update_table_column_statistics_args&); + ThriftHiveMetastore_update_table_column_statistics_args() { + } + + virtual ~ThriftHiveMetastore_update_table_column_statistics_args() noexcept; + ColumnStatistics stats_obj; + + _ThriftHiveMetastore_update_table_column_statistics_args__isset __isset; + + void __set_stats_obj(const ColumnStatistics& val); + + bool operator == (const ThriftHiveMetastore_update_table_column_statistics_args & rhs) const + { + if (!(stats_obj == rhs.stats_obj)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_update_table_column_statistics_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_update_table_column_statistics_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_update_table_column_statistics_pargs { + public: + + + virtual ~ThriftHiveMetastore_update_table_column_statistics_pargs() noexcept; + const ColumnStatistics* stats_obj; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_update_table_column_statistics_result__isset { + _ThriftHiveMetastore_update_table_column_statistics_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_update_table_column_statistics_result__isset; + +class ThriftHiveMetastore_update_table_column_statistics_result { + public: + + ThriftHiveMetastore_update_table_column_statistics_result(const ThriftHiveMetastore_update_table_column_statistics_result&); + ThriftHiveMetastore_update_table_column_statistics_result& operator=(const ThriftHiveMetastore_update_table_column_statistics_result&); + ThriftHiveMetastore_update_table_column_statistics_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_update_table_column_statistics_result() noexcept; + bool success; + NoSuchObjectException o1; + InvalidObjectException o2; + MetaException o3; + InvalidInputException o4; + + _ThriftHiveMetastore_update_table_column_statistics_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const InvalidObjectException& val); + + void __set_o3(const MetaException& val); + + void __set_o4(const InvalidInputException& val); + + bool operator == (const ThriftHiveMetastore_update_table_column_statistics_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_update_table_column_statistics_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_update_table_column_statistics_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_update_table_column_statistics_presult__isset { + _ThriftHiveMetastore_update_table_column_statistics_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_update_table_column_statistics_presult__isset; + +class ThriftHiveMetastore_update_table_column_statistics_presult { + public: + + + virtual ~ThriftHiveMetastore_update_table_column_statistics_presult() noexcept; + bool* success; + NoSuchObjectException o1; + InvalidObjectException o2; + MetaException o3; + InvalidInputException o4; + + _ThriftHiveMetastore_update_table_column_statistics_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_update_partition_column_statistics_args__isset { + _ThriftHiveMetastore_update_partition_column_statistics_args__isset() : stats_obj(false) {} + bool stats_obj :1; +} _ThriftHiveMetastore_update_partition_column_statistics_args__isset; + +class ThriftHiveMetastore_update_partition_column_statistics_args { + public: + + ThriftHiveMetastore_update_partition_column_statistics_args(const ThriftHiveMetastore_update_partition_column_statistics_args&); + ThriftHiveMetastore_update_partition_column_statistics_args& operator=(const ThriftHiveMetastore_update_partition_column_statistics_args&); + ThriftHiveMetastore_update_partition_column_statistics_args() { + } + + virtual ~ThriftHiveMetastore_update_partition_column_statistics_args() noexcept; + ColumnStatistics stats_obj; + + _ThriftHiveMetastore_update_partition_column_statistics_args__isset __isset; + + void __set_stats_obj(const ColumnStatistics& val); + + bool operator == (const ThriftHiveMetastore_update_partition_column_statistics_args & rhs) const + { + if (!(stats_obj == rhs.stats_obj)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_update_partition_column_statistics_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_update_partition_column_statistics_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_update_partition_column_statistics_pargs { + public: + + + virtual ~ThriftHiveMetastore_update_partition_column_statistics_pargs() noexcept; + const ColumnStatistics* stats_obj; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_update_partition_column_statistics_result__isset { + _ThriftHiveMetastore_update_partition_column_statistics_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_update_partition_column_statistics_result__isset; + +class ThriftHiveMetastore_update_partition_column_statistics_result { + public: + + ThriftHiveMetastore_update_partition_column_statistics_result(const ThriftHiveMetastore_update_partition_column_statistics_result&); + ThriftHiveMetastore_update_partition_column_statistics_result& operator=(const ThriftHiveMetastore_update_partition_column_statistics_result&); + ThriftHiveMetastore_update_partition_column_statistics_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_update_partition_column_statistics_result() noexcept; + bool success; + NoSuchObjectException o1; + InvalidObjectException o2; + MetaException o3; + InvalidInputException o4; + + _ThriftHiveMetastore_update_partition_column_statistics_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const InvalidObjectException& val); + + void __set_o3(const MetaException& val); + + void __set_o4(const InvalidInputException& val); + + bool operator == (const ThriftHiveMetastore_update_partition_column_statistics_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_update_partition_column_statistics_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_update_partition_column_statistics_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_update_partition_column_statistics_presult__isset { + _ThriftHiveMetastore_update_partition_column_statistics_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_update_partition_column_statistics_presult__isset; + +class ThriftHiveMetastore_update_partition_column_statistics_presult { + public: + + + virtual ~ThriftHiveMetastore_update_partition_column_statistics_presult() noexcept; + bool* success; + NoSuchObjectException o1; + InvalidObjectException o2; + MetaException o3; + InvalidInputException o4; + + _ThriftHiveMetastore_update_partition_column_statistics_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_table_column_statistics_args__isset { + _ThriftHiveMetastore_get_table_column_statistics_args__isset() : db_name(false), tbl_name(false), col_name(false) {} + bool db_name :1; + bool tbl_name :1; + bool col_name :1; +} _ThriftHiveMetastore_get_table_column_statistics_args__isset; + +class ThriftHiveMetastore_get_table_column_statistics_args { + public: + + ThriftHiveMetastore_get_table_column_statistics_args(const ThriftHiveMetastore_get_table_column_statistics_args&); + ThriftHiveMetastore_get_table_column_statistics_args& operator=(const ThriftHiveMetastore_get_table_column_statistics_args&); + ThriftHiveMetastore_get_table_column_statistics_args() : db_name(), tbl_name(), col_name() { + } + + virtual ~ThriftHiveMetastore_get_table_column_statistics_args() noexcept; + std::string db_name; + std::string tbl_name; + std::string col_name; + + _ThriftHiveMetastore_get_table_column_statistics_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_col_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_table_column_statistics_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(col_name == rhs.col_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_table_column_statistics_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_table_column_statistics_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_table_column_statistics_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_table_column_statistics_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::string* col_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_table_column_statistics_result__isset { + _ThriftHiveMetastore_get_table_column_statistics_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_get_table_column_statistics_result__isset; + +class ThriftHiveMetastore_get_table_column_statistics_result { + public: + + ThriftHiveMetastore_get_table_column_statistics_result(const ThriftHiveMetastore_get_table_column_statistics_result&); + ThriftHiveMetastore_get_table_column_statistics_result& operator=(const ThriftHiveMetastore_get_table_column_statistics_result&); + ThriftHiveMetastore_get_table_column_statistics_result() { + } + + virtual ~ThriftHiveMetastore_get_table_column_statistics_result() noexcept; + ColumnStatistics success; + NoSuchObjectException o1; + MetaException o2; + InvalidInputException o3; + InvalidObjectException o4; + + _ThriftHiveMetastore_get_table_column_statistics_result__isset __isset; + + void __set_success(const ColumnStatistics& val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + void __set_o3(const InvalidInputException& val); + + void __set_o4(const InvalidObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_table_column_statistics_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_table_column_statistics_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_table_column_statistics_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_table_column_statistics_presult__isset { + _ThriftHiveMetastore_get_table_column_statistics_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_get_table_column_statistics_presult__isset; + +class ThriftHiveMetastore_get_table_column_statistics_presult { + public: + + + virtual ~ThriftHiveMetastore_get_table_column_statistics_presult() noexcept; + ColumnStatistics* success; + NoSuchObjectException o1; + MetaException o2; + InvalidInputException o3; + InvalidObjectException o4; + + _ThriftHiveMetastore_get_table_column_statistics_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_partition_column_statistics_args__isset { + _ThriftHiveMetastore_get_partition_column_statistics_args__isset() : db_name(false), tbl_name(false), part_name(false), col_name(false) {} + bool db_name :1; + bool tbl_name :1; + bool part_name :1; + bool col_name :1; +} _ThriftHiveMetastore_get_partition_column_statistics_args__isset; + +class ThriftHiveMetastore_get_partition_column_statistics_args { + public: + + ThriftHiveMetastore_get_partition_column_statistics_args(const ThriftHiveMetastore_get_partition_column_statistics_args&); + ThriftHiveMetastore_get_partition_column_statistics_args& operator=(const ThriftHiveMetastore_get_partition_column_statistics_args&); + ThriftHiveMetastore_get_partition_column_statistics_args() : db_name(), tbl_name(), part_name(), col_name() { + } + + virtual ~ThriftHiveMetastore_get_partition_column_statistics_args() noexcept; + std::string db_name; + std::string tbl_name; + std::string part_name; + std::string col_name; + + _ThriftHiveMetastore_get_partition_column_statistics_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_name(const std::string& val); + + void __set_col_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_partition_column_statistics_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_name == rhs.part_name)) + return false; + if (!(col_name == rhs.col_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partition_column_statistics_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partition_column_statistics_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_partition_column_statistics_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_partition_column_statistics_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::string* part_name; + const std::string* col_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partition_column_statistics_result__isset { + _ThriftHiveMetastore_get_partition_column_statistics_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_get_partition_column_statistics_result__isset; + +class ThriftHiveMetastore_get_partition_column_statistics_result { + public: + + ThriftHiveMetastore_get_partition_column_statistics_result(const ThriftHiveMetastore_get_partition_column_statistics_result&); + ThriftHiveMetastore_get_partition_column_statistics_result& operator=(const ThriftHiveMetastore_get_partition_column_statistics_result&); + ThriftHiveMetastore_get_partition_column_statistics_result() { + } + + virtual ~ThriftHiveMetastore_get_partition_column_statistics_result() noexcept; + ColumnStatistics success; + NoSuchObjectException o1; + MetaException o2; + InvalidInputException o3; + InvalidObjectException o4; + + _ThriftHiveMetastore_get_partition_column_statistics_result__isset __isset; + + void __set_success(const ColumnStatistics& val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + void __set_o3(const InvalidInputException& val); + + void __set_o4(const InvalidObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_partition_column_statistics_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partition_column_statistics_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partition_column_statistics_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partition_column_statistics_presult__isset { + _ThriftHiveMetastore_get_partition_column_statistics_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_get_partition_column_statistics_presult__isset; + +class ThriftHiveMetastore_get_partition_column_statistics_presult { + public: + + + virtual ~ThriftHiveMetastore_get_partition_column_statistics_presult() noexcept; + ColumnStatistics* success; + NoSuchObjectException o1; + MetaException o2; + InvalidInputException o3; + InvalidObjectException o4; + + _ThriftHiveMetastore_get_partition_column_statistics_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_table_statistics_req_args__isset { + _ThriftHiveMetastore_get_table_statistics_req_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_get_table_statistics_req_args__isset; + +class ThriftHiveMetastore_get_table_statistics_req_args { + public: + + ThriftHiveMetastore_get_table_statistics_req_args(const ThriftHiveMetastore_get_table_statistics_req_args&); + ThriftHiveMetastore_get_table_statistics_req_args& operator=(const ThriftHiveMetastore_get_table_statistics_req_args&); + ThriftHiveMetastore_get_table_statistics_req_args() { + } + + virtual ~ThriftHiveMetastore_get_table_statistics_req_args() noexcept; + TableStatsRequest request; + + _ThriftHiveMetastore_get_table_statistics_req_args__isset __isset; + + void __set_request(const TableStatsRequest& val); + + bool operator == (const ThriftHiveMetastore_get_table_statistics_req_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_table_statistics_req_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_table_statistics_req_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_table_statistics_req_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_table_statistics_req_pargs() noexcept; + const TableStatsRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_table_statistics_req_result__isset { + _ThriftHiveMetastore_get_table_statistics_req_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_table_statistics_req_result__isset; + +class ThriftHiveMetastore_get_table_statistics_req_result { + public: + + ThriftHiveMetastore_get_table_statistics_req_result(const ThriftHiveMetastore_get_table_statistics_req_result&); + ThriftHiveMetastore_get_table_statistics_req_result& operator=(const ThriftHiveMetastore_get_table_statistics_req_result&); + ThriftHiveMetastore_get_table_statistics_req_result() { + } + + virtual ~ThriftHiveMetastore_get_table_statistics_req_result() noexcept; + TableStatsResult success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_table_statistics_req_result__isset __isset; + + void __set_success(const TableStatsResult& val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_table_statistics_req_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_table_statistics_req_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_table_statistics_req_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_table_statistics_req_presult__isset { + _ThriftHiveMetastore_get_table_statistics_req_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_table_statistics_req_presult__isset; + +class ThriftHiveMetastore_get_table_statistics_req_presult { + public: + + + virtual ~ThriftHiveMetastore_get_table_statistics_req_presult() noexcept; + TableStatsResult* success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_table_statistics_req_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_statistics_req_args__isset { + _ThriftHiveMetastore_get_partitions_statistics_req_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_get_partitions_statistics_req_args__isset; + +class ThriftHiveMetastore_get_partitions_statistics_req_args { + public: + + ThriftHiveMetastore_get_partitions_statistics_req_args(const ThriftHiveMetastore_get_partitions_statistics_req_args&); + ThriftHiveMetastore_get_partitions_statistics_req_args& operator=(const ThriftHiveMetastore_get_partitions_statistics_req_args&); + ThriftHiveMetastore_get_partitions_statistics_req_args() { + } + + virtual ~ThriftHiveMetastore_get_partitions_statistics_req_args() noexcept; + PartitionsStatsRequest request; + + _ThriftHiveMetastore_get_partitions_statistics_req_args__isset __isset; + + void __set_request(const PartitionsStatsRequest& val); + + bool operator == (const ThriftHiveMetastore_get_partitions_statistics_req_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_statistics_req_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_statistics_req_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_partitions_statistics_req_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_statistics_req_pargs() noexcept; + const PartitionsStatsRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_statistics_req_result__isset { + _ThriftHiveMetastore_get_partitions_statistics_req_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_statistics_req_result__isset; + +class ThriftHiveMetastore_get_partitions_statistics_req_result { + public: + + ThriftHiveMetastore_get_partitions_statistics_req_result(const ThriftHiveMetastore_get_partitions_statistics_req_result&); + ThriftHiveMetastore_get_partitions_statistics_req_result& operator=(const ThriftHiveMetastore_get_partitions_statistics_req_result&); + ThriftHiveMetastore_get_partitions_statistics_req_result() { + } + + virtual ~ThriftHiveMetastore_get_partitions_statistics_req_result() noexcept; + PartitionsStatsResult success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_partitions_statistics_req_result__isset __isset; + + void __set_success(const PartitionsStatsResult& val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_partitions_statistics_req_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_partitions_statistics_req_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_partitions_statistics_req_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_partitions_statistics_req_presult__isset { + _ThriftHiveMetastore_get_partitions_statistics_req_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_partitions_statistics_req_presult__isset; + +class ThriftHiveMetastore_get_partitions_statistics_req_presult { + public: + + + virtual ~ThriftHiveMetastore_get_partitions_statistics_req_presult() noexcept; + PartitionsStatsResult* success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_partitions_statistics_req_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_aggr_stats_for_args__isset { + _ThriftHiveMetastore_get_aggr_stats_for_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_get_aggr_stats_for_args__isset; + +class ThriftHiveMetastore_get_aggr_stats_for_args { + public: + + ThriftHiveMetastore_get_aggr_stats_for_args(const ThriftHiveMetastore_get_aggr_stats_for_args&); + ThriftHiveMetastore_get_aggr_stats_for_args& operator=(const ThriftHiveMetastore_get_aggr_stats_for_args&); + ThriftHiveMetastore_get_aggr_stats_for_args() { + } + + virtual ~ThriftHiveMetastore_get_aggr_stats_for_args() noexcept; + PartitionsStatsRequest request; + + _ThriftHiveMetastore_get_aggr_stats_for_args__isset __isset; + + void __set_request(const PartitionsStatsRequest& val); + + bool operator == (const ThriftHiveMetastore_get_aggr_stats_for_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_aggr_stats_for_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_aggr_stats_for_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_aggr_stats_for_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_aggr_stats_for_pargs() noexcept; + const PartitionsStatsRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_aggr_stats_for_result__isset { + _ThriftHiveMetastore_get_aggr_stats_for_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_aggr_stats_for_result__isset; + +class ThriftHiveMetastore_get_aggr_stats_for_result { + public: + + ThriftHiveMetastore_get_aggr_stats_for_result(const ThriftHiveMetastore_get_aggr_stats_for_result&); + ThriftHiveMetastore_get_aggr_stats_for_result& operator=(const ThriftHiveMetastore_get_aggr_stats_for_result&); + ThriftHiveMetastore_get_aggr_stats_for_result() { + } + + virtual ~ThriftHiveMetastore_get_aggr_stats_for_result() noexcept; + AggrStats success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_aggr_stats_for_result__isset __isset; + + void __set_success(const AggrStats& val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_aggr_stats_for_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_aggr_stats_for_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_aggr_stats_for_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_aggr_stats_for_presult__isset { + _ThriftHiveMetastore_get_aggr_stats_for_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_aggr_stats_for_presult__isset; + +class ThriftHiveMetastore_get_aggr_stats_for_presult { + public: + + + virtual ~ThriftHiveMetastore_get_aggr_stats_for_presult() noexcept; + AggrStats* success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_aggr_stats_for_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_set_aggr_stats_for_args__isset { + _ThriftHiveMetastore_set_aggr_stats_for_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_set_aggr_stats_for_args__isset; + +class ThriftHiveMetastore_set_aggr_stats_for_args { + public: + + ThriftHiveMetastore_set_aggr_stats_for_args(const ThriftHiveMetastore_set_aggr_stats_for_args&); + ThriftHiveMetastore_set_aggr_stats_for_args& operator=(const ThriftHiveMetastore_set_aggr_stats_for_args&); + ThriftHiveMetastore_set_aggr_stats_for_args() { + } + + virtual ~ThriftHiveMetastore_set_aggr_stats_for_args() noexcept; + SetPartitionsStatsRequest request; + + _ThriftHiveMetastore_set_aggr_stats_for_args__isset __isset; + + void __set_request(const SetPartitionsStatsRequest& val); + + bool operator == (const ThriftHiveMetastore_set_aggr_stats_for_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_set_aggr_stats_for_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_set_aggr_stats_for_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_set_aggr_stats_for_pargs { + public: + + + virtual ~ThriftHiveMetastore_set_aggr_stats_for_pargs() noexcept; + const SetPartitionsStatsRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_set_aggr_stats_for_result__isset { + _ThriftHiveMetastore_set_aggr_stats_for_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_set_aggr_stats_for_result__isset; + +class ThriftHiveMetastore_set_aggr_stats_for_result { + public: + + ThriftHiveMetastore_set_aggr_stats_for_result(const ThriftHiveMetastore_set_aggr_stats_for_result&); + ThriftHiveMetastore_set_aggr_stats_for_result& operator=(const ThriftHiveMetastore_set_aggr_stats_for_result&); + ThriftHiveMetastore_set_aggr_stats_for_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_set_aggr_stats_for_result() noexcept; + bool success; + NoSuchObjectException o1; + InvalidObjectException o2; + MetaException o3; + InvalidInputException o4; + + _ThriftHiveMetastore_set_aggr_stats_for_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const InvalidObjectException& val); + + void __set_o3(const MetaException& val); + + void __set_o4(const InvalidInputException& val); + + bool operator == (const ThriftHiveMetastore_set_aggr_stats_for_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_set_aggr_stats_for_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_set_aggr_stats_for_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_set_aggr_stats_for_presult__isset { + _ThriftHiveMetastore_set_aggr_stats_for_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_set_aggr_stats_for_presult__isset; + +class ThriftHiveMetastore_set_aggr_stats_for_presult { + public: + + + virtual ~ThriftHiveMetastore_set_aggr_stats_for_presult() noexcept; + bool* success; + NoSuchObjectException o1; + InvalidObjectException o2; + MetaException o3; + InvalidInputException o4; + + _ThriftHiveMetastore_set_aggr_stats_for_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_delete_partition_column_statistics_args__isset { + _ThriftHiveMetastore_delete_partition_column_statistics_args__isset() : db_name(false), tbl_name(false), part_name(false), col_name(false) {} + bool db_name :1; + bool tbl_name :1; + bool part_name :1; + bool col_name :1; +} _ThriftHiveMetastore_delete_partition_column_statistics_args__isset; + +class ThriftHiveMetastore_delete_partition_column_statistics_args { + public: + + ThriftHiveMetastore_delete_partition_column_statistics_args(const ThriftHiveMetastore_delete_partition_column_statistics_args&); + ThriftHiveMetastore_delete_partition_column_statistics_args& operator=(const ThriftHiveMetastore_delete_partition_column_statistics_args&); + ThriftHiveMetastore_delete_partition_column_statistics_args() : db_name(), tbl_name(), part_name(), col_name() { + } + + virtual ~ThriftHiveMetastore_delete_partition_column_statistics_args() noexcept; + std::string db_name; + std::string tbl_name; + std::string part_name; + std::string col_name; + + _ThriftHiveMetastore_delete_partition_column_statistics_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_part_name(const std::string& val); + + void __set_col_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_delete_partition_column_statistics_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(part_name == rhs.part_name)) + return false; + if (!(col_name == rhs.col_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_delete_partition_column_statistics_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_delete_partition_column_statistics_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_delete_partition_column_statistics_pargs { + public: + + + virtual ~ThriftHiveMetastore_delete_partition_column_statistics_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::string* part_name; + const std::string* col_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_delete_partition_column_statistics_result__isset { + _ThriftHiveMetastore_delete_partition_column_statistics_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_delete_partition_column_statistics_result__isset; + +class ThriftHiveMetastore_delete_partition_column_statistics_result { + public: + + ThriftHiveMetastore_delete_partition_column_statistics_result(const ThriftHiveMetastore_delete_partition_column_statistics_result&); + ThriftHiveMetastore_delete_partition_column_statistics_result& operator=(const ThriftHiveMetastore_delete_partition_column_statistics_result&); + ThriftHiveMetastore_delete_partition_column_statistics_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_delete_partition_column_statistics_result() noexcept; + bool success; + NoSuchObjectException o1; + MetaException o2; + InvalidObjectException o3; + InvalidInputException o4; + + _ThriftHiveMetastore_delete_partition_column_statistics_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + void __set_o3(const InvalidObjectException& val); + + void __set_o4(const InvalidInputException& val); + + bool operator == (const ThriftHiveMetastore_delete_partition_column_statistics_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_delete_partition_column_statistics_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_delete_partition_column_statistics_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_delete_partition_column_statistics_presult__isset { + _ThriftHiveMetastore_delete_partition_column_statistics_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_delete_partition_column_statistics_presult__isset; + +class ThriftHiveMetastore_delete_partition_column_statistics_presult { + public: + + + virtual ~ThriftHiveMetastore_delete_partition_column_statistics_presult() noexcept; + bool* success; + NoSuchObjectException o1; + MetaException o2; + InvalidObjectException o3; + InvalidInputException o4; + + _ThriftHiveMetastore_delete_partition_column_statistics_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_delete_table_column_statistics_args__isset { + _ThriftHiveMetastore_delete_table_column_statistics_args__isset() : db_name(false), tbl_name(false), col_name(false) {} + bool db_name :1; + bool tbl_name :1; + bool col_name :1; +} _ThriftHiveMetastore_delete_table_column_statistics_args__isset; + +class ThriftHiveMetastore_delete_table_column_statistics_args { + public: + + ThriftHiveMetastore_delete_table_column_statistics_args(const ThriftHiveMetastore_delete_table_column_statistics_args&); + ThriftHiveMetastore_delete_table_column_statistics_args& operator=(const ThriftHiveMetastore_delete_table_column_statistics_args&); + ThriftHiveMetastore_delete_table_column_statistics_args() : db_name(), tbl_name(), col_name() { + } + + virtual ~ThriftHiveMetastore_delete_table_column_statistics_args() noexcept; + std::string db_name; + std::string tbl_name; + std::string col_name; + + _ThriftHiveMetastore_delete_table_column_statistics_args__isset __isset; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + void __set_col_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_delete_table_column_statistics_args & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + if (!(col_name == rhs.col_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_delete_table_column_statistics_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_delete_table_column_statistics_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_delete_table_column_statistics_pargs { + public: + + + virtual ~ThriftHiveMetastore_delete_table_column_statistics_pargs() noexcept; + const std::string* db_name; + const std::string* tbl_name; + const std::string* col_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_delete_table_column_statistics_result__isset { + _ThriftHiveMetastore_delete_table_column_statistics_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_delete_table_column_statistics_result__isset; + +class ThriftHiveMetastore_delete_table_column_statistics_result { + public: + + ThriftHiveMetastore_delete_table_column_statistics_result(const ThriftHiveMetastore_delete_table_column_statistics_result&); + ThriftHiveMetastore_delete_table_column_statistics_result& operator=(const ThriftHiveMetastore_delete_table_column_statistics_result&); + ThriftHiveMetastore_delete_table_column_statistics_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_delete_table_column_statistics_result() noexcept; + bool success; + NoSuchObjectException o1; + MetaException o2; + InvalidObjectException o3; + InvalidInputException o4; + + _ThriftHiveMetastore_delete_table_column_statistics_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + void __set_o3(const InvalidObjectException& val); + + void __set_o4(const InvalidInputException& val); + + bool operator == (const ThriftHiveMetastore_delete_table_column_statistics_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_delete_table_column_statistics_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_delete_table_column_statistics_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_delete_table_column_statistics_presult__isset { + _ThriftHiveMetastore_delete_table_column_statistics_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_delete_table_column_statistics_presult__isset; + +class ThriftHiveMetastore_delete_table_column_statistics_presult { + public: + + + virtual ~ThriftHiveMetastore_delete_table_column_statistics_presult() noexcept; + bool* success; + NoSuchObjectException o1; + MetaException o2; + InvalidObjectException o3; + InvalidInputException o4; + + _ThriftHiveMetastore_delete_table_column_statistics_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_create_function_args__isset { + _ThriftHiveMetastore_create_function_args__isset() : func(false) {} + bool func :1; +} _ThriftHiveMetastore_create_function_args__isset; + +class ThriftHiveMetastore_create_function_args { + public: + + ThriftHiveMetastore_create_function_args(const ThriftHiveMetastore_create_function_args&); + ThriftHiveMetastore_create_function_args& operator=(const ThriftHiveMetastore_create_function_args&); + ThriftHiveMetastore_create_function_args() { + } + + virtual ~ThriftHiveMetastore_create_function_args() noexcept; + Function func; + + _ThriftHiveMetastore_create_function_args__isset __isset; + + void __set_func(const Function& val); + + bool operator == (const ThriftHiveMetastore_create_function_args & rhs) const + { + if (!(func == rhs.func)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_function_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_function_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_create_function_pargs { + public: + + + virtual ~ThriftHiveMetastore_create_function_pargs() noexcept; + const Function* func; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_function_result__isset { + _ThriftHiveMetastore_create_function_result__isset() : o1(false), o2(false), o3(false), o4(false) {} + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_create_function_result__isset; + +class ThriftHiveMetastore_create_function_result { + public: + + ThriftHiveMetastore_create_function_result(const ThriftHiveMetastore_create_function_result&); + ThriftHiveMetastore_create_function_result& operator=(const ThriftHiveMetastore_create_function_result&); + ThriftHiveMetastore_create_function_result() { + } + + virtual ~ThriftHiveMetastore_create_function_result() noexcept; + AlreadyExistsException o1; + InvalidObjectException o2; + MetaException o3; + NoSuchObjectException o4; + + _ThriftHiveMetastore_create_function_result__isset __isset; + + void __set_o1(const AlreadyExistsException& val); + + void __set_o2(const InvalidObjectException& val); + + void __set_o3(const MetaException& val); + + void __set_o4(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_create_function_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_function_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_function_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_function_presult__isset { + _ThriftHiveMetastore_create_function_presult__isset() : o1(false), o2(false), o3(false), o4(false) {} + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_create_function_presult__isset; + +class ThriftHiveMetastore_create_function_presult { + public: + + + virtual ~ThriftHiveMetastore_create_function_presult() noexcept; + AlreadyExistsException o1; + InvalidObjectException o2; + MetaException o3; + NoSuchObjectException o4; + + _ThriftHiveMetastore_create_function_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_drop_function_args__isset { + _ThriftHiveMetastore_drop_function_args__isset() : dbName(false), funcName(false) {} + bool dbName :1; + bool funcName :1; +} _ThriftHiveMetastore_drop_function_args__isset; + +class ThriftHiveMetastore_drop_function_args { + public: + + ThriftHiveMetastore_drop_function_args(const ThriftHiveMetastore_drop_function_args&); + ThriftHiveMetastore_drop_function_args& operator=(const ThriftHiveMetastore_drop_function_args&); + ThriftHiveMetastore_drop_function_args() : dbName(), funcName() { + } + + virtual ~ThriftHiveMetastore_drop_function_args() noexcept; + std::string dbName; + std::string funcName; + + _ThriftHiveMetastore_drop_function_args__isset __isset; + + void __set_dbName(const std::string& val); + + void __set_funcName(const std::string& val); + + bool operator == (const ThriftHiveMetastore_drop_function_args & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(funcName == rhs.funcName)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_function_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_function_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_drop_function_pargs { + public: + + + virtual ~ThriftHiveMetastore_drop_function_pargs() noexcept; + const std::string* dbName; + const std::string* funcName; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_function_result__isset { + _ThriftHiveMetastore_drop_function_result__isset() : o1(false), o3(false) {} + bool o1 :1; + bool o3 :1; +} _ThriftHiveMetastore_drop_function_result__isset; + +class ThriftHiveMetastore_drop_function_result { + public: + + ThriftHiveMetastore_drop_function_result(const ThriftHiveMetastore_drop_function_result&); + ThriftHiveMetastore_drop_function_result& operator=(const ThriftHiveMetastore_drop_function_result&); + ThriftHiveMetastore_drop_function_result() { + } + + virtual ~ThriftHiveMetastore_drop_function_result() noexcept; + NoSuchObjectException o1; + MetaException o3; + + _ThriftHiveMetastore_drop_function_result__isset __isset; + + void __set_o1(const NoSuchObjectException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_drop_function_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_function_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_function_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_function_presult__isset { + _ThriftHiveMetastore_drop_function_presult__isset() : o1(false), o3(false) {} + bool o1 :1; + bool o3 :1; +} _ThriftHiveMetastore_drop_function_presult__isset; + +class ThriftHiveMetastore_drop_function_presult { + public: + + + virtual ~ThriftHiveMetastore_drop_function_presult() noexcept; + NoSuchObjectException o1; + MetaException o3; + + _ThriftHiveMetastore_drop_function_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_alter_function_args__isset { + _ThriftHiveMetastore_alter_function_args__isset() : dbName(false), funcName(false), newFunc(false) {} + bool dbName :1; + bool funcName :1; + bool newFunc :1; +} _ThriftHiveMetastore_alter_function_args__isset; + +class ThriftHiveMetastore_alter_function_args { + public: + + ThriftHiveMetastore_alter_function_args(const ThriftHiveMetastore_alter_function_args&); + ThriftHiveMetastore_alter_function_args& operator=(const ThriftHiveMetastore_alter_function_args&); + ThriftHiveMetastore_alter_function_args() : dbName(), funcName() { + } + + virtual ~ThriftHiveMetastore_alter_function_args() noexcept; + std::string dbName; + std::string funcName; + Function newFunc; + + _ThriftHiveMetastore_alter_function_args__isset __isset; + + void __set_dbName(const std::string& val); + + void __set_funcName(const std::string& val); + + void __set_newFunc(const Function& val); + + bool operator == (const ThriftHiveMetastore_alter_function_args & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(funcName == rhs.funcName)) + return false; + if (!(newFunc == rhs.newFunc)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_function_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_function_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_alter_function_pargs { + public: + + + virtual ~ThriftHiveMetastore_alter_function_pargs() noexcept; + const std::string* dbName; + const std::string* funcName; + const Function* newFunc; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_function_result__isset { + _ThriftHiveMetastore_alter_function_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_function_result__isset; + +class ThriftHiveMetastore_alter_function_result { + public: + + ThriftHiveMetastore_alter_function_result(const ThriftHiveMetastore_alter_function_result&); + ThriftHiveMetastore_alter_function_result& operator=(const ThriftHiveMetastore_alter_function_result&); + ThriftHiveMetastore_alter_function_result() { + } + + virtual ~ThriftHiveMetastore_alter_function_result() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_function_result__isset __isset; + + void __set_o1(const InvalidOperationException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_alter_function_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_function_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_function_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_function_presult__isset { + _ThriftHiveMetastore_alter_function_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_alter_function_presult__isset; + +class ThriftHiveMetastore_alter_function_presult { + public: + + + virtual ~ThriftHiveMetastore_alter_function_presult() noexcept; + InvalidOperationException o1; + MetaException o2; + + _ThriftHiveMetastore_alter_function_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_functions_args__isset { + _ThriftHiveMetastore_get_functions_args__isset() : dbName(false), pattern(false) {} + bool dbName :1; + bool pattern :1; +} _ThriftHiveMetastore_get_functions_args__isset; + +class ThriftHiveMetastore_get_functions_args { + public: + + ThriftHiveMetastore_get_functions_args(const ThriftHiveMetastore_get_functions_args&); + ThriftHiveMetastore_get_functions_args& operator=(const ThriftHiveMetastore_get_functions_args&); + ThriftHiveMetastore_get_functions_args() : dbName(), pattern() { + } + + virtual ~ThriftHiveMetastore_get_functions_args() noexcept; + std::string dbName; + std::string pattern; + + _ThriftHiveMetastore_get_functions_args__isset __isset; + + void __set_dbName(const std::string& val); + + void __set_pattern(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_functions_args & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(pattern == rhs.pattern)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_functions_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_functions_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_functions_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_functions_pargs() noexcept; + const std::string* dbName; + const std::string* pattern; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_functions_result__isset { + _ThriftHiveMetastore_get_functions_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_functions_result__isset; + +class ThriftHiveMetastore_get_functions_result { + public: + + ThriftHiveMetastore_get_functions_result(const ThriftHiveMetastore_get_functions_result&); + ThriftHiveMetastore_get_functions_result& operator=(const ThriftHiveMetastore_get_functions_result&); + ThriftHiveMetastore_get_functions_result() { + } + + virtual ~ThriftHiveMetastore_get_functions_result() noexcept; + std::vector success; + MetaException o1; + + _ThriftHiveMetastore_get_functions_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_functions_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_functions_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_functions_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_functions_presult__isset { + _ThriftHiveMetastore_get_functions_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_functions_presult__isset; + +class ThriftHiveMetastore_get_functions_presult { + public: + + + virtual ~ThriftHiveMetastore_get_functions_presult() noexcept; + std::vector * success; + MetaException o1; + + _ThriftHiveMetastore_get_functions_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_function_args__isset { + _ThriftHiveMetastore_get_function_args__isset() : dbName(false), funcName(false) {} + bool dbName :1; + bool funcName :1; +} _ThriftHiveMetastore_get_function_args__isset; + +class ThriftHiveMetastore_get_function_args { + public: + + ThriftHiveMetastore_get_function_args(const ThriftHiveMetastore_get_function_args&); + ThriftHiveMetastore_get_function_args& operator=(const ThriftHiveMetastore_get_function_args&); + ThriftHiveMetastore_get_function_args() : dbName(), funcName() { + } + + virtual ~ThriftHiveMetastore_get_function_args() noexcept; + std::string dbName; + std::string funcName; + + _ThriftHiveMetastore_get_function_args__isset __isset; + + void __set_dbName(const std::string& val); + + void __set_funcName(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_function_args & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(funcName == rhs.funcName)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_function_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_function_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_function_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_function_pargs() noexcept; + const std::string* dbName; + const std::string* funcName; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_function_result__isset { + _ThriftHiveMetastore_get_function_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_function_result__isset; + +class ThriftHiveMetastore_get_function_result { + public: + + ThriftHiveMetastore_get_function_result(const ThriftHiveMetastore_get_function_result&); + ThriftHiveMetastore_get_function_result& operator=(const ThriftHiveMetastore_get_function_result&); + ThriftHiveMetastore_get_function_result() { + } + + virtual ~ThriftHiveMetastore_get_function_result() noexcept; + Function success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_function_result__isset __isset; + + void __set_success(const Function& val); + + void __set_o1(const MetaException& val); + + void __set_o2(const NoSuchObjectException& val); + + bool operator == (const ThriftHiveMetastore_get_function_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_function_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_function_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_function_presult__isset { + _ThriftHiveMetastore_get_function_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_function_presult__isset; + +class ThriftHiveMetastore_get_function_presult { + public: + + + virtual ~ThriftHiveMetastore_get_function_presult() noexcept; + Function* success; + MetaException o1; + NoSuchObjectException o2; + + _ThriftHiveMetastore_get_function_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + + +class ThriftHiveMetastore_get_all_functions_args { + public: + + ThriftHiveMetastore_get_all_functions_args(const ThriftHiveMetastore_get_all_functions_args&); + ThriftHiveMetastore_get_all_functions_args& operator=(const ThriftHiveMetastore_get_all_functions_args&); + ThriftHiveMetastore_get_all_functions_args() { + } + + virtual ~ThriftHiveMetastore_get_all_functions_args() noexcept; + + bool operator == (const ThriftHiveMetastore_get_all_functions_args & /* rhs */) const + { + return true; + } + bool operator != (const ThriftHiveMetastore_get_all_functions_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_all_functions_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_all_functions_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_all_functions_pargs() noexcept; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_all_functions_result__isset { + _ThriftHiveMetastore_get_all_functions_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_all_functions_result__isset; + +class ThriftHiveMetastore_get_all_functions_result { + public: + + ThriftHiveMetastore_get_all_functions_result(const ThriftHiveMetastore_get_all_functions_result&); + ThriftHiveMetastore_get_all_functions_result& operator=(const ThriftHiveMetastore_get_all_functions_result&); + ThriftHiveMetastore_get_all_functions_result() { + } + + virtual ~ThriftHiveMetastore_get_all_functions_result() noexcept; + GetAllFunctionsResponse success; + MetaException o1; + + _ThriftHiveMetastore_get_all_functions_result__isset __isset; + + void __set_success(const GetAllFunctionsResponse& val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_all_functions_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_all_functions_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_all_functions_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_all_functions_presult__isset { + _ThriftHiveMetastore_get_all_functions_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_all_functions_presult__isset; + +class ThriftHiveMetastore_get_all_functions_presult { + public: + + + virtual ~ThriftHiveMetastore_get_all_functions_presult() noexcept; + GetAllFunctionsResponse* success; + MetaException o1; + + _ThriftHiveMetastore_get_all_functions_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_create_role_args__isset { + _ThriftHiveMetastore_create_role_args__isset() : role(false) {} + bool role :1; +} _ThriftHiveMetastore_create_role_args__isset; + +class ThriftHiveMetastore_create_role_args { + public: + + ThriftHiveMetastore_create_role_args(const ThriftHiveMetastore_create_role_args&); + ThriftHiveMetastore_create_role_args& operator=(const ThriftHiveMetastore_create_role_args&); + ThriftHiveMetastore_create_role_args() { + } + + virtual ~ThriftHiveMetastore_create_role_args() noexcept; + Role role; + + _ThriftHiveMetastore_create_role_args__isset __isset; + + void __set_role(const Role& val); + + bool operator == (const ThriftHiveMetastore_create_role_args & rhs) const + { + if (!(role == rhs.role)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_role_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_role_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_create_role_pargs { + public: + + + virtual ~ThriftHiveMetastore_create_role_pargs() noexcept; + const Role* role; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_role_result__isset { + _ThriftHiveMetastore_create_role_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_create_role_result__isset; + +class ThriftHiveMetastore_create_role_result { + public: + + ThriftHiveMetastore_create_role_result(const ThriftHiveMetastore_create_role_result&); + ThriftHiveMetastore_create_role_result& operator=(const ThriftHiveMetastore_create_role_result&); + ThriftHiveMetastore_create_role_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_create_role_result() noexcept; + bool success; + MetaException o1; + + _ThriftHiveMetastore_create_role_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_create_role_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_role_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_role_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_role_presult__isset { + _ThriftHiveMetastore_create_role_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_create_role_presult__isset; + +class ThriftHiveMetastore_create_role_presult { + public: + + + virtual ~ThriftHiveMetastore_create_role_presult() noexcept; + bool* success; + MetaException o1; + + _ThriftHiveMetastore_create_role_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_drop_role_args__isset { + _ThriftHiveMetastore_drop_role_args__isset() : role_name(false) {} + bool role_name :1; +} _ThriftHiveMetastore_drop_role_args__isset; + +class ThriftHiveMetastore_drop_role_args { + public: + + ThriftHiveMetastore_drop_role_args(const ThriftHiveMetastore_drop_role_args&); + ThriftHiveMetastore_drop_role_args& operator=(const ThriftHiveMetastore_drop_role_args&); + ThriftHiveMetastore_drop_role_args() : role_name() { + } + + virtual ~ThriftHiveMetastore_drop_role_args() noexcept; + std::string role_name; + + _ThriftHiveMetastore_drop_role_args__isset __isset; + + void __set_role_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_drop_role_args & rhs) const + { + if (!(role_name == rhs.role_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_role_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_role_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_drop_role_pargs { + public: + + + virtual ~ThriftHiveMetastore_drop_role_pargs() noexcept; + const std::string* role_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_role_result__isset { + _ThriftHiveMetastore_drop_role_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_drop_role_result__isset; + +class ThriftHiveMetastore_drop_role_result { + public: + + ThriftHiveMetastore_drop_role_result(const ThriftHiveMetastore_drop_role_result&); + ThriftHiveMetastore_drop_role_result& operator=(const ThriftHiveMetastore_drop_role_result&); + ThriftHiveMetastore_drop_role_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_drop_role_result() noexcept; + bool success; + MetaException o1; + + _ThriftHiveMetastore_drop_role_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_drop_role_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_role_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_role_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_role_presult__isset { + _ThriftHiveMetastore_drop_role_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_drop_role_presult__isset; + +class ThriftHiveMetastore_drop_role_presult { + public: + + + virtual ~ThriftHiveMetastore_drop_role_presult() noexcept; + bool* success; + MetaException o1; + + _ThriftHiveMetastore_drop_role_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + + +class ThriftHiveMetastore_get_role_names_args { + public: + + ThriftHiveMetastore_get_role_names_args(const ThriftHiveMetastore_get_role_names_args&); + ThriftHiveMetastore_get_role_names_args& operator=(const ThriftHiveMetastore_get_role_names_args&); + ThriftHiveMetastore_get_role_names_args() { + } + + virtual ~ThriftHiveMetastore_get_role_names_args() noexcept; + + bool operator == (const ThriftHiveMetastore_get_role_names_args & /* rhs */) const + { + return true; + } + bool operator != (const ThriftHiveMetastore_get_role_names_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_role_names_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_role_names_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_role_names_pargs() noexcept; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_role_names_result__isset { + _ThriftHiveMetastore_get_role_names_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_role_names_result__isset; + +class ThriftHiveMetastore_get_role_names_result { + public: + + ThriftHiveMetastore_get_role_names_result(const ThriftHiveMetastore_get_role_names_result&); + ThriftHiveMetastore_get_role_names_result& operator=(const ThriftHiveMetastore_get_role_names_result&); + ThriftHiveMetastore_get_role_names_result() { + } + + virtual ~ThriftHiveMetastore_get_role_names_result() noexcept; + std::vector success; + MetaException o1; + + _ThriftHiveMetastore_get_role_names_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_role_names_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_role_names_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_role_names_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_role_names_presult__isset { + _ThriftHiveMetastore_get_role_names_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_role_names_presult__isset; + +class ThriftHiveMetastore_get_role_names_presult { + public: + + + virtual ~ThriftHiveMetastore_get_role_names_presult() noexcept; + std::vector * success; + MetaException o1; + + _ThriftHiveMetastore_get_role_names_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_grant_role_args__isset { + _ThriftHiveMetastore_grant_role_args__isset() : role_name(false), principal_name(false), principal_type(false), grantor(false), grantorType(false), grant_option(false) {} + bool role_name :1; + bool principal_name :1; + bool principal_type :1; + bool grantor :1; + bool grantorType :1; + bool grant_option :1; +} _ThriftHiveMetastore_grant_role_args__isset; + +class ThriftHiveMetastore_grant_role_args { + public: + + ThriftHiveMetastore_grant_role_args(const ThriftHiveMetastore_grant_role_args&); + ThriftHiveMetastore_grant_role_args& operator=(const ThriftHiveMetastore_grant_role_args&); + ThriftHiveMetastore_grant_role_args() : role_name(), principal_name(), principal_type((PrincipalType::type)0), grantor(), grantorType((PrincipalType::type)0), grant_option(0) { + } + + virtual ~ThriftHiveMetastore_grant_role_args() noexcept; + std::string role_name; + std::string principal_name; + PrincipalType::type principal_type; + std::string grantor; + PrincipalType::type grantorType; + bool grant_option; + + _ThriftHiveMetastore_grant_role_args__isset __isset; + + void __set_role_name(const std::string& val); + + void __set_principal_name(const std::string& val); + + void __set_principal_type(const PrincipalType::type val); + + void __set_grantor(const std::string& val); + + void __set_grantorType(const PrincipalType::type val); + + void __set_grant_option(const bool val); + + bool operator == (const ThriftHiveMetastore_grant_role_args & rhs) const + { + if (!(role_name == rhs.role_name)) + return false; + if (!(principal_name == rhs.principal_name)) + return false; + if (!(principal_type == rhs.principal_type)) + return false; + if (!(grantor == rhs.grantor)) + return false; + if (!(grantorType == rhs.grantorType)) + return false; + if (!(grant_option == rhs.grant_option)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_grant_role_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_grant_role_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_grant_role_pargs { + public: + + + virtual ~ThriftHiveMetastore_grant_role_pargs() noexcept; + const std::string* role_name; + const std::string* principal_name; + const PrincipalType::type* principal_type; + const std::string* grantor; + const PrincipalType::type* grantorType; + const bool* grant_option; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_grant_role_result__isset { + _ThriftHiveMetastore_grant_role_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_grant_role_result__isset; + +class ThriftHiveMetastore_grant_role_result { + public: + + ThriftHiveMetastore_grant_role_result(const ThriftHiveMetastore_grant_role_result&); + ThriftHiveMetastore_grant_role_result& operator=(const ThriftHiveMetastore_grant_role_result&); + ThriftHiveMetastore_grant_role_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_grant_role_result() noexcept; + bool success; + MetaException o1; + + _ThriftHiveMetastore_grant_role_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_grant_role_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_grant_role_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_grant_role_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_grant_role_presult__isset { + _ThriftHiveMetastore_grant_role_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_grant_role_presult__isset; + +class ThriftHiveMetastore_grant_role_presult { + public: + + + virtual ~ThriftHiveMetastore_grant_role_presult() noexcept; + bool* success; + MetaException o1; + + _ThriftHiveMetastore_grant_role_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_revoke_role_args__isset { + _ThriftHiveMetastore_revoke_role_args__isset() : role_name(false), principal_name(false), principal_type(false) {} + bool role_name :1; + bool principal_name :1; + bool principal_type :1; +} _ThriftHiveMetastore_revoke_role_args__isset; + +class ThriftHiveMetastore_revoke_role_args { + public: + + ThriftHiveMetastore_revoke_role_args(const ThriftHiveMetastore_revoke_role_args&); + ThriftHiveMetastore_revoke_role_args& operator=(const ThriftHiveMetastore_revoke_role_args&); + ThriftHiveMetastore_revoke_role_args() : role_name(), principal_name(), principal_type((PrincipalType::type)0) { + } + + virtual ~ThriftHiveMetastore_revoke_role_args() noexcept; + std::string role_name; + std::string principal_name; + PrincipalType::type principal_type; + + _ThriftHiveMetastore_revoke_role_args__isset __isset; + + void __set_role_name(const std::string& val); + + void __set_principal_name(const std::string& val); + + void __set_principal_type(const PrincipalType::type val); + + bool operator == (const ThriftHiveMetastore_revoke_role_args & rhs) const + { + if (!(role_name == rhs.role_name)) + return false; + if (!(principal_name == rhs.principal_name)) + return false; + if (!(principal_type == rhs.principal_type)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_revoke_role_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_revoke_role_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_revoke_role_pargs { + public: + + + virtual ~ThriftHiveMetastore_revoke_role_pargs() noexcept; + const std::string* role_name; + const std::string* principal_name; + const PrincipalType::type* principal_type; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_revoke_role_result__isset { + _ThriftHiveMetastore_revoke_role_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_revoke_role_result__isset; + +class ThriftHiveMetastore_revoke_role_result { + public: + + ThriftHiveMetastore_revoke_role_result(const ThriftHiveMetastore_revoke_role_result&); + ThriftHiveMetastore_revoke_role_result& operator=(const ThriftHiveMetastore_revoke_role_result&); + ThriftHiveMetastore_revoke_role_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_revoke_role_result() noexcept; + bool success; + MetaException o1; + + _ThriftHiveMetastore_revoke_role_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_revoke_role_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_revoke_role_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_revoke_role_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_revoke_role_presult__isset { + _ThriftHiveMetastore_revoke_role_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_revoke_role_presult__isset; + +class ThriftHiveMetastore_revoke_role_presult { + public: + + + virtual ~ThriftHiveMetastore_revoke_role_presult() noexcept; + bool* success; + MetaException o1; + + _ThriftHiveMetastore_revoke_role_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_list_roles_args__isset { + _ThriftHiveMetastore_list_roles_args__isset() : principal_name(false), principal_type(false) {} + bool principal_name :1; + bool principal_type :1; +} _ThriftHiveMetastore_list_roles_args__isset; + +class ThriftHiveMetastore_list_roles_args { + public: + + ThriftHiveMetastore_list_roles_args(const ThriftHiveMetastore_list_roles_args&); + ThriftHiveMetastore_list_roles_args& operator=(const ThriftHiveMetastore_list_roles_args&); + ThriftHiveMetastore_list_roles_args() : principal_name(), principal_type((PrincipalType::type)0) { + } + + virtual ~ThriftHiveMetastore_list_roles_args() noexcept; + std::string principal_name; + PrincipalType::type principal_type; + + _ThriftHiveMetastore_list_roles_args__isset __isset; + + void __set_principal_name(const std::string& val); + + void __set_principal_type(const PrincipalType::type val); + + bool operator == (const ThriftHiveMetastore_list_roles_args & rhs) const + { + if (!(principal_name == rhs.principal_name)) + return false; + if (!(principal_type == rhs.principal_type)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_list_roles_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_list_roles_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_list_roles_pargs { + public: + + + virtual ~ThriftHiveMetastore_list_roles_pargs() noexcept; + const std::string* principal_name; + const PrincipalType::type* principal_type; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_list_roles_result__isset { + _ThriftHiveMetastore_list_roles_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_list_roles_result__isset; + +class ThriftHiveMetastore_list_roles_result { + public: + + ThriftHiveMetastore_list_roles_result(const ThriftHiveMetastore_list_roles_result&); + ThriftHiveMetastore_list_roles_result& operator=(const ThriftHiveMetastore_list_roles_result&); + ThriftHiveMetastore_list_roles_result() { + } + + virtual ~ThriftHiveMetastore_list_roles_result() noexcept; + std::vector success; + MetaException o1; + + _ThriftHiveMetastore_list_roles_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_list_roles_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_list_roles_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_list_roles_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_list_roles_presult__isset { + _ThriftHiveMetastore_list_roles_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_list_roles_presult__isset; + +class ThriftHiveMetastore_list_roles_presult { + public: + + + virtual ~ThriftHiveMetastore_list_roles_presult() noexcept; + std::vector * success; + MetaException o1; + + _ThriftHiveMetastore_list_roles_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_grant_revoke_role_args__isset { + _ThriftHiveMetastore_grant_revoke_role_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_grant_revoke_role_args__isset; + +class ThriftHiveMetastore_grant_revoke_role_args { + public: + + ThriftHiveMetastore_grant_revoke_role_args(const ThriftHiveMetastore_grant_revoke_role_args&); + ThriftHiveMetastore_grant_revoke_role_args& operator=(const ThriftHiveMetastore_grant_revoke_role_args&); + ThriftHiveMetastore_grant_revoke_role_args() { + } + + virtual ~ThriftHiveMetastore_grant_revoke_role_args() noexcept; + GrantRevokeRoleRequest request; + + _ThriftHiveMetastore_grant_revoke_role_args__isset __isset; + + void __set_request(const GrantRevokeRoleRequest& val); + + bool operator == (const ThriftHiveMetastore_grant_revoke_role_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_grant_revoke_role_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_grant_revoke_role_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_grant_revoke_role_pargs { + public: + + + virtual ~ThriftHiveMetastore_grant_revoke_role_pargs() noexcept; + const GrantRevokeRoleRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_grant_revoke_role_result__isset { + _ThriftHiveMetastore_grant_revoke_role_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_grant_revoke_role_result__isset; + +class ThriftHiveMetastore_grant_revoke_role_result { + public: + + ThriftHiveMetastore_grant_revoke_role_result(const ThriftHiveMetastore_grant_revoke_role_result&); + ThriftHiveMetastore_grant_revoke_role_result& operator=(const ThriftHiveMetastore_grant_revoke_role_result&); + ThriftHiveMetastore_grant_revoke_role_result() { + } + + virtual ~ThriftHiveMetastore_grant_revoke_role_result() noexcept; + GrantRevokeRoleResponse success; + MetaException o1; + + _ThriftHiveMetastore_grant_revoke_role_result__isset __isset; + + void __set_success(const GrantRevokeRoleResponse& val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_grant_revoke_role_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_grant_revoke_role_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_grant_revoke_role_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_grant_revoke_role_presult__isset { + _ThriftHiveMetastore_grant_revoke_role_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_grant_revoke_role_presult__isset; + +class ThriftHiveMetastore_grant_revoke_role_presult { + public: + + + virtual ~ThriftHiveMetastore_grant_revoke_role_presult() noexcept; + GrantRevokeRoleResponse* success; + MetaException o1; + + _ThriftHiveMetastore_grant_revoke_role_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_principals_in_role_args__isset { + _ThriftHiveMetastore_get_principals_in_role_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_get_principals_in_role_args__isset; + +class ThriftHiveMetastore_get_principals_in_role_args { + public: + + ThriftHiveMetastore_get_principals_in_role_args(const ThriftHiveMetastore_get_principals_in_role_args&); + ThriftHiveMetastore_get_principals_in_role_args& operator=(const ThriftHiveMetastore_get_principals_in_role_args&); + ThriftHiveMetastore_get_principals_in_role_args() { + } + + virtual ~ThriftHiveMetastore_get_principals_in_role_args() noexcept; + GetPrincipalsInRoleRequest request; + + _ThriftHiveMetastore_get_principals_in_role_args__isset __isset; + + void __set_request(const GetPrincipalsInRoleRequest& val); + + bool operator == (const ThriftHiveMetastore_get_principals_in_role_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_principals_in_role_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_principals_in_role_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_principals_in_role_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_principals_in_role_pargs() noexcept; + const GetPrincipalsInRoleRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_principals_in_role_result__isset { + _ThriftHiveMetastore_get_principals_in_role_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_principals_in_role_result__isset; + +class ThriftHiveMetastore_get_principals_in_role_result { + public: + + ThriftHiveMetastore_get_principals_in_role_result(const ThriftHiveMetastore_get_principals_in_role_result&); + ThriftHiveMetastore_get_principals_in_role_result& operator=(const ThriftHiveMetastore_get_principals_in_role_result&); + ThriftHiveMetastore_get_principals_in_role_result() { + } + + virtual ~ThriftHiveMetastore_get_principals_in_role_result() noexcept; + GetPrincipalsInRoleResponse success; + MetaException o1; + + _ThriftHiveMetastore_get_principals_in_role_result__isset __isset; + + void __set_success(const GetPrincipalsInRoleResponse& val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_principals_in_role_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_principals_in_role_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_principals_in_role_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_principals_in_role_presult__isset { + _ThriftHiveMetastore_get_principals_in_role_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_principals_in_role_presult__isset; + +class ThriftHiveMetastore_get_principals_in_role_presult { + public: + + + virtual ~ThriftHiveMetastore_get_principals_in_role_presult() noexcept; + GetPrincipalsInRoleResponse* success; + MetaException o1; + + _ThriftHiveMetastore_get_principals_in_role_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_role_grants_for_principal_args__isset { + _ThriftHiveMetastore_get_role_grants_for_principal_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_get_role_grants_for_principal_args__isset; + +class ThriftHiveMetastore_get_role_grants_for_principal_args { + public: + + ThriftHiveMetastore_get_role_grants_for_principal_args(const ThriftHiveMetastore_get_role_grants_for_principal_args&); + ThriftHiveMetastore_get_role_grants_for_principal_args& operator=(const ThriftHiveMetastore_get_role_grants_for_principal_args&); + ThriftHiveMetastore_get_role_grants_for_principal_args() { + } + + virtual ~ThriftHiveMetastore_get_role_grants_for_principal_args() noexcept; + GetRoleGrantsForPrincipalRequest request; + + _ThriftHiveMetastore_get_role_grants_for_principal_args__isset __isset; + + void __set_request(const GetRoleGrantsForPrincipalRequest& val); + + bool operator == (const ThriftHiveMetastore_get_role_grants_for_principal_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_role_grants_for_principal_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_role_grants_for_principal_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_role_grants_for_principal_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_role_grants_for_principal_pargs() noexcept; + const GetRoleGrantsForPrincipalRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_role_grants_for_principal_result__isset { + _ThriftHiveMetastore_get_role_grants_for_principal_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_role_grants_for_principal_result__isset; + +class ThriftHiveMetastore_get_role_grants_for_principal_result { + public: + + ThriftHiveMetastore_get_role_grants_for_principal_result(const ThriftHiveMetastore_get_role_grants_for_principal_result&); + ThriftHiveMetastore_get_role_grants_for_principal_result& operator=(const ThriftHiveMetastore_get_role_grants_for_principal_result&); + ThriftHiveMetastore_get_role_grants_for_principal_result() { + } + + virtual ~ThriftHiveMetastore_get_role_grants_for_principal_result() noexcept; + GetRoleGrantsForPrincipalResponse success; + MetaException o1; + + _ThriftHiveMetastore_get_role_grants_for_principal_result__isset __isset; + + void __set_success(const GetRoleGrantsForPrincipalResponse& val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_role_grants_for_principal_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_role_grants_for_principal_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_role_grants_for_principal_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_role_grants_for_principal_presult__isset { + _ThriftHiveMetastore_get_role_grants_for_principal_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_role_grants_for_principal_presult__isset; + +class ThriftHiveMetastore_get_role_grants_for_principal_presult { + public: + + + virtual ~ThriftHiveMetastore_get_role_grants_for_principal_presult() noexcept; + GetRoleGrantsForPrincipalResponse* success; + MetaException o1; + + _ThriftHiveMetastore_get_role_grants_for_principal_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_privilege_set_args__isset { + _ThriftHiveMetastore_get_privilege_set_args__isset() : hiveObject(false), user_name(false), group_names(false) {} + bool hiveObject :1; + bool user_name :1; + bool group_names :1; +} _ThriftHiveMetastore_get_privilege_set_args__isset; + +class ThriftHiveMetastore_get_privilege_set_args { + public: + + ThriftHiveMetastore_get_privilege_set_args(const ThriftHiveMetastore_get_privilege_set_args&); + ThriftHiveMetastore_get_privilege_set_args& operator=(const ThriftHiveMetastore_get_privilege_set_args&); + ThriftHiveMetastore_get_privilege_set_args() : user_name() { + } + + virtual ~ThriftHiveMetastore_get_privilege_set_args() noexcept; + HiveObjectRef hiveObject; + std::string user_name; + std::vector group_names; + + _ThriftHiveMetastore_get_privilege_set_args__isset __isset; + + void __set_hiveObject(const HiveObjectRef& val); + + void __set_user_name(const std::string& val); + + void __set_group_names(const std::vector & val); + + bool operator == (const ThriftHiveMetastore_get_privilege_set_args & rhs) const + { + if (!(hiveObject == rhs.hiveObject)) + return false; + if (!(user_name == rhs.user_name)) + return false; + if (!(group_names == rhs.group_names)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_privilege_set_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_privilege_set_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_privilege_set_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_privilege_set_pargs() noexcept; + const HiveObjectRef* hiveObject; + const std::string* user_name; + const std::vector * group_names; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_privilege_set_result__isset { + _ThriftHiveMetastore_get_privilege_set_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_privilege_set_result__isset; + +class ThriftHiveMetastore_get_privilege_set_result { + public: + + ThriftHiveMetastore_get_privilege_set_result(const ThriftHiveMetastore_get_privilege_set_result&); + ThriftHiveMetastore_get_privilege_set_result& operator=(const ThriftHiveMetastore_get_privilege_set_result&); + ThriftHiveMetastore_get_privilege_set_result() { + } + + virtual ~ThriftHiveMetastore_get_privilege_set_result() noexcept; + PrincipalPrivilegeSet success; + MetaException o1; + + _ThriftHiveMetastore_get_privilege_set_result__isset __isset; + + void __set_success(const PrincipalPrivilegeSet& val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_privilege_set_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_privilege_set_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_privilege_set_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_privilege_set_presult__isset { + _ThriftHiveMetastore_get_privilege_set_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_privilege_set_presult__isset; + +class ThriftHiveMetastore_get_privilege_set_presult { + public: + + + virtual ~ThriftHiveMetastore_get_privilege_set_presult() noexcept; + PrincipalPrivilegeSet* success; + MetaException o1; + + _ThriftHiveMetastore_get_privilege_set_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_list_privileges_args__isset { + _ThriftHiveMetastore_list_privileges_args__isset() : principal_name(false), principal_type(false), hiveObject(false) {} + bool principal_name :1; + bool principal_type :1; + bool hiveObject :1; +} _ThriftHiveMetastore_list_privileges_args__isset; + +class ThriftHiveMetastore_list_privileges_args { + public: + + ThriftHiveMetastore_list_privileges_args(const ThriftHiveMetastore_list_privileges_args&); + ThriftHiveMetastore_list_privileges_args& operator=(const ThriftHiveMetastore_list_privileges_args&); + ThriftHiveMetastore_list_privileges_args() : principal_name(), principal_type((PrincipalType::type)0) { + } + + virtual ~ThriftHiveMetastore_list_privileges_args() noexcept; + std::string principal_name; + PrincipalType::type principal_type; + HiveObjectRef hiveObject; + + _ThriftHiveMetastore_list_privileges_args__isset __isset; + + void __set_principal_name(const std::string& val); + + void __set_principal_type(const PrincipalType::type val); + + void __set_hiveObject(const HiveObjectRef& val); + + bool operator == (const ThriftHiveMetastore_list_privileges_args & rhs) const + { + if (!(principal_name == rhs.principal_name)) + return false; + if (!(principal_type == rhs.principal_type)) + return false; + if (!(hiveObject == rhs.hiveObject)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_list_privileges_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_list_privileges_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_list_privileges_pargs { + public: + + + virtual ~ThriftHiveMetastore_list_privileges_pargs() noexcept; + const std::string* principal_name; + const PrincipalType::type* principal_type; + const HiveObjectRef* hiveObject; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_list_privileges_result__isset { + _ThriftHiveMetastore_list_privileges_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_list_privileges_result__isset; + +class ThriftHiveMetastore_list_privileges_result { + public: + + ThriftHiveMetastore_list_privileges_result(const ThriftHiveMetastore_list_privileges_result&); + ThriftHiveMetastore_list_privileges_result& operator=(const ThriftHiveMetastore_list_privileges_result&); + ThriftHiveMetastore_list_privileges_result() { + } + + virtual ~ThriftHiveMetastore_list_privileges_result() noexcept; + std::vector success; + MetaException o1; + + _ThriftHiveMetastore_list_privileges_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_list_privileges_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_list_privileges_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_list_privileges_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_list_privileges_presult__isset { + _ThriftHiveMetastore_list_privileges_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_list_privileges_presult__isset; + +class ThriftHiveMetastore_list_privileges_presult { + public: + + + virtual ~ThriftHiveMetastore_list_privileges_presult() noexcept; + std::vector * success; + MetaException o1; + + _ThriftHiveMetastore_list_privileges_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_grant_privileges_args__isset { + _ThriftHiveMetastore_grant_privileges_args__isset() : privileges(false) {} + bool privileges :1; +} _ThriftHiveMetastore_grant_privileges_args__isset; + +class ThriftHiveMetastore_grant_privileges_args { + public: + + ThriftHiveMetastore_grant_privileges_args(const ThriftHiveMetastore_grant_privileges_args&); + ThriftHiveMetastore_grant_privileges_args& operator=(const ThriftHiveMetastore_grant_privileges_args&); + ThriftHiveMetastore_grant_privileges_args() { + } + + virtual ~ThriftHiveMetastore_grant_privileges_args() noexcept; + PrivilegeBag privileges; + + _ThriftHiveMetastore_grant_privileges_args__isset __isset; + + void __set_privileges(const PrivilegeBag& val); + + bool operator == (const ThriftHiveMetastore_grant_privileges_args & rhs) const + { + if (!(privileges == rhs.privileges)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_grant_privileges_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_grant_privileges_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_grant_privileges_pargs { + public: + + + virtual ~ThriftHiveMetastore_grant_privileges_pargs() noexcept; + const PrivilegeBag* privileges; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_grant_privileges_result__isset { + _ThriftHiveMetastore_grant_privileges_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_grant_privileges_result__isset; + +class ThriftHiveMetastore_grant_privileges_result { + public: + + ThriftHiveMetastore_grant_privileges_result(const ThriftHiveMetastore_grant_privileges_result&); + ThriftHiveMetastore_grant_privileges_result& operator=(const ThriftHiveMetastore_grant_privileges_result&); + ThriftHiveMetastore_grant_privileges_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_grant_privileges_result() noexcept; + bool success; + MetaException o1; + + _ThriftHiveMetastore_grant_privileges_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_grant_privileges_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_grant_privileges_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_grant_privileges_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_grant_privileges_presult__isset { + _ThriftHiveMetastore_grant_privileges_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_grant_privileges_presult__isset; + +class ThriftHiveMetastore_grant_privileges_presult { + public: + + + virtual ~ThriftHiveMetastore_grant_privileges_presult() noexcept; + bool* success; + MetaException o1; + + _ThriftHiveMetastore_grant_privileges_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_revoke_privileges_args__isset { + _ThriftHiveMetastore_revoke_privileges_args__isset() : privileges(false) {} + bool privileges :1; +} _ThriftHiveMetastore_revoke_privileges_args__isset; + +class ThriftHiveMetastore_revoke_privileges_args { + public: + + ThriftHiveMetastore_revoke_privileges_args(const ThriftHiveMetastore_revoke_privileges_args&); + ThriftHiveMetastore_revoke_privileges_args& operator=(const ThriftHiveMetastore_revoke_privileges_args&); + ThriftHiveMetastore_revoke_privileges_args() { + } + + virtual ~ThriftHiveMetastore_revoke_privileges_args() noexcept; + PrivilegeBag privileges; + + _ThriftHiveMetastore_revoke_privileges_args__isset __isset; + + void __set_privileges(const PrivilegeBag& val); + + bool operator == (const ThriftHiveMetastore_revoke_privileges_args & rhs) const + { + if (!(privileges == rhs.privileges)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_revoke_privileges_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_revoke_privileges_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_revoke_privileges_pargs { + public: + + + virtual ~ThriftHiveMetastore_revoke_privileges_pargs() noexcept; + const PrivilegeBag* privileges; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_revoke_privileges_result__isset { + _ThriftHiveMetastore_revoke_privileges_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_revoke_privileges_result__isset; + +class ThriftHiveMetastore_revoke_privileges_result { + public: + + ThriftHiveMetastore_revoke_privileges_result(const ThriftHiveMetastore_revoke_privileges_result&); + ThriftHiveMetastore_revoke_privileges_result& operator=(const ThriftHiveMetastore_revoke_privileges_result&); + ThriftHiveMetastore_revoke_privileges_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_revoke_privileges_result() noexcept; + bool success; + MetaException o1; + + _ThriftHiveMetastore_revoke_privileges_result__isset __isset; + + void __set_success(const bool val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_revoke_privileges_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_revoke_privileges_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_revoke_privileges_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_revoke_privileges_presult__isset { + _ThriftHiveMetastore_revoke_privileges_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_revoke_privileges_presult__isset; + +class ThriftHiveMetastore_revoke_privileges_presult { + public: + + + virtual ~ThriftHiveMetastore_revoke_privileges_presult() noexcept; + bool* success; + MetaException o1; + + _ThriftHiveMetastore_revoke_privileges_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_grant_revoke_privileges_args__isset { + _ThriftHiveMetastore_grant_revoke_privileges_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_grant_revoke_privileges_args__isset; + +class ThriftHiveMetastore_grant_revoke_privileges_args { + public: + + ThriftHiveMetastore_grant_revoke_privileges_args(const ThriftHiveMetastore_grant_revoke_privileges_args&); + ThriftHiveMetastore_grant_revoke_privileges_args& operator=(const ThriftHiveMetastore_grant_revoke_privileges_args&); + ThriftHiveMetastore_grant_revoke_privileges_args() { + } + + virtual ~ThriftHiveMetastore_grant_revoke_privileges_args() noexcept; + GrantRevokePrivilegeRequest request; + + _ThriftHiveMetastore_grant_revoke_privileges_args__isset __isset; + + void __set_request(const GrantRevokePrivilegeRequest& val); + + bool operator == (const ThriftHiveMetastore_grant_revoke_privileges_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_grant_revoke_privileges_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_grant_revoke_privileges_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_grant_revoke_privileges_pargs { + public: + + + virtual ~ThriftHiveMetastore_grant_revoke_privileges_pargs() noexcept; + const GrantRevokePrivilegeRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_grant_revoke_privileges_result__isset { + _ThriftHiveMetastore_grant_revoke_privileges_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_grant_revoke_privileges_result__isset; + +class ThriftHiveMetastore_grant_revoke_privileges_result { + public: + + ThriftHiveMetastore_grant_revoke_privileges_result(const ThriftHiveMetastore_grant_revoke_privileges_result&); + ThriftHiveMetastore_grant_revoke_privileges_result& operator=(const ThriftHiveMetastore_grant_revoke_privileges_result&); + ThriftHiveMetastore_grant_revoke_privileges_result() { + } + + virtual ~ThriftHiveMetastore_grant_revoke_privileges_result() noexcept; + GrantRevokePrivilegeResponse success; + MetaException o1; + + _ThriftHiveMetastore_grant_revoke_privileges_result__isset __isset; + + void __set_success(const GrantRevokePrivilegeResponse& val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_grant_revoke_privileges_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_grant_revoke_privileges_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_grant_revoke_privileges_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_grant_revoke_privileges_presult__isset { + _ThriftHiveMetastore_grant_revoke_privileges_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_grant_revoke_privileges_presult__isset; + +class ThriftHiveMetastore_grant_revoke_privileges_presult { + public: + + + virtual ~ThriftHiveMetastore_grant_revoke_privileges_presult() noexcept; + GrantRevokePrivilegeResponse* success; + MetaException o1; + + _ThriftHiveMetastore_grant_revoke_privileges_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_set_ugi_args__isset { + _ThriftHiveMetastore_set_ugi_args__isset() : user_name(false), group_names(false) {} + bool user_name :1; + bool group_names :1; +} _ThriftHiveMetastore_set_ugi_args__isset; + +class ThriftHiveMetastore_set_ugi_args { + public: + + ThriftHiveMetastore_set_ugi_args(const ThriftHiveMetastore_set_ugi_args&); + ThriftHiveMetastore_set_ugi_args& operator=(const ThriftHiveMetastore_set_ugi_args&); + ThriftHiveMetastore_set_ugi_args() : user_name() { + } + + virtual ~ThriftHiveMetastore_set_ugi_args() noexcept; + std::string user_name; + std::vector group_names; + + _ThriftHiveMetastore_set_ugi_args__isset __isset; + + void __set_user_name(const std::string& val); + + void __set_group_names(const std::vector & val); + + bool operator == (const ThriftHiveMetastore_set_ugi_args & rhs) const + { + if (!(user_name == rhs.user_name)) + return false; + if (!(group_names == rhs.group_names)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_set_ugi_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_set_ugi_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_set_ugi_pargs { + public: + + + virtual ~ThriftHiveMetastore_set_ugi_pargs() noexcept; + const std::string* user_name; + const std::vector * group_names; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_set_ugi_result__isset { + _ThriftHiveMetastore_set_ugi_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_set_ugi_result__isset; + +class ThriftHiveMetastore_set_ugi_result { + public: + + ThriftHiveMetastore_set_ugi_result(const ThriftHiveMetastore_set_ugi_result&); + ThriftHiveMetastore_set_ugi_result& operator=(const ThriftHiveMetastore_set_ugi_result&); + ThriftHiveMetastore_set_ugi_result() { + } + + virtual ~ThriftHiveMetastore_set_ugi_result() noexcept; + std::vector success; + MetaException o1; + + _ThriftHiveMetastore_set_ugi_result__isset __isset; + + void __set_success(const std::vector & val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_set_ugi_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_set_ugi_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_set_ugi_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_set_ugi_presult__isset { + _ThriftHiveMetastore_set_ugi_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_set_ugi_presult__isset; + +class ThriftHiveMetastore_set_ugi_presult { + public: + + + virtual ~ThriftHiveMetastore_set_ugi_presult() noexcept; + std::vector * success; + MetaException o1; + + _ThriftHiveMetastore_set_ugi_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_delegation_token_args__isset { + _ThriftHiveMetastore_get_delegation_token_args__isset() : token_owner(false), renewer_kerberos_principal_name(false) {} + bool token_owner :1; + bool renewer_kerberos_principal_name :1; +} _ThriftHiveMetastore_get_delegation_token_args__isset; + +class ThriftHiveMetastore_get_delegation_token_args { + public: + + ThriftHiveMetastore_get_delegation_token_args(const ThriftHiveMetastore_get_delegation_token_args&); + ThriftHiveMetastore_get_delegation_token_args& operator=(const ThriftHiveMetastore_get_delegation_token_args&); + ThriftHiveMetastore_get_delegation_token_args() : token_owner(), renewer_kerberos_principal_name() { + } + + virtual ~ThriftHiveMetastore_get_delegation_token_args() noexcept; + std::string token_owner; + std::string renewer_kerberos_principal_name; + + _ThriftHiveMetastore_get_delegation_token_args__isset __isset; + + void __set_token_owner(const std::string& val); + + void __set_renewer_kerberos_principal_name(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_delegation_token_args & rhs) const + { + if (!(token_owner == rhs.token_owner)) + return false; + if (!(renewer_kerberos_principal_name == rhs.renewer_kerberos_principal_name)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_delegation_token_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_delegation_token_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_delegation_token_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_delegation_token_pargs() noexcept; + const std::string* token_owner; + const std::string* renewer_kerberos_principal_name; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_delegation_token_result__isset { + _ThriftHiveMetastore_get_delegation_token_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_delegation_token_result__isset; + +class ThriftHiveMetastore_get_delegation_token_result { + public: + + ThriftHiveMetastore_get_delegation_token_result(const ThriftHiveMetastore_get_delegation_token_result&); + ThriftHiveMetastore_get_delegation_token_result& operator=(const ThriftHiveMetastore_get_delegation_token_result&); + ThriftHiveMetastore_get_delegation_token_result() : success() { + } + + virtual ~ThriftHiveMetastore_get_delegation_token_result() noexcept; + std::string success; + MetaException o1; + + _ThriftHiveMetastore_get_delegation_token_result__isset __isset; + + void __set_success(const std::string& val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_delegation_token_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_delegation_token_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_delegation_token_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_delegation_token_presult__isset { + _ThriftHiveMetastore_get_delegation_token_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_delegation_token_presult__isset; + +class ThriftHiveMetastore_get_delegation_token_presult { + public: + + + virtual ~ThriftHiveMetastore_get_delegation_token_presult() noexcept; + std::string* success; + MetaException o1; + + _ThriftHiveMetastore_get_delegation_token_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_renew_delegation_token_args__isset { + _ThriftHiveMetastore_renew_delegation_token_args__isset() : token_str_form(false) {} + bool token_str_form :1; +} _ThriftHiveMetastore_renew_delegation_token_args__isset; + +class ThriftHiveMetastore_renew_delegation_token_args { + public: + + ThriftHiveMetastore_renew_delegation_token_args(const ThriftHiveMetastore_renew_delegation_token_args&); + ThriftHiveMetastore_renew_delegation_token_args& operator=(const ThriftHiveMetastore_renew_delegation_token_args&); + ThriftHiveMetastore_renew_delegation_token_args() : token_str_form() { + } + + virtual ~ThriftHiveMetastore_renew_delegation_token_args() noexcept; + std::string token_str_form; + + _ThriftHiveMetastore_renew_delegation_token_args__isset __isset; + + void __set_token_str_form(const std::string& val); + + bool operator == (const ThriftHiveMetastore_renew_delegation_token_args & rhs) const + { + if (!(token_str_form == rhs.token_str_form)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_renew_delegation_token_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_renew_delegation_token_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_renew_delegation_token_pargs { + public: + + + virtual ~ThriftHiveMetastore_renew_delegation_token_pargs() noexcept; + const std::string* token_str_form; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_renew_delegation_token_result__isset { + _ThriftHiveMetastore_renew_delegation_token_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_renew_delegation_token_result__isset; + +class ThriftHiveMetastore_renew_delegation_token_result { + public: + + ThriftHiveMetastore_renew_delegation_token_result(const ThriftHiveMetastore_renew_delegation_token_result&); + ThriftHiveMetastore_renew_delegation_token_result& operator=(const ThriftHiveMetastore_renew_delegation_token_result&); + ThriftHiveMetastore_renew_delegation_token_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_renew_delegation_token_result() noexcept; + int64_t success; + MetaException o1; + + _ThriftHiveMetastore_renew_delegation_token_result__isset __isset; + + void __set_success(const int64_t val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_renew_delegation_token_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_renew_delegation_token_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_renew_delegation_token_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_renew_delegation_token_presult__isset { + _ThriftHiveMetastore_renew_delegation_token_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_renew_delegation_token_presult__isset; + +class ThriftHiveMetastore_renew_delegation_token_presult { + public: + + + virtual ~ThriftHiveMetastore_renew_delegation_token_presult() noexcept; + int64_t* success; + MetaException o1; + + _ThriftHiveMetastore_renew_delegation_token_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_cancel_delegation_token_args__isset { + _ThriftHiveMetastore_cancel_delegation_token_args__isset() : token_str_form(false) {} + bool token_str_form :1; +} _ThriftHiveMetastore_cancel_delegation_token_args__isset; + +class ThriftHiveMetastore_cancel_delegation_token_args { + public: + + ThriftHiveMetastore_cancel_delegation_token_args(const ThriftHiveMetastore_cancel_delegation_token_args&); + ThriftHiveMetastore_cancel_delegation_token_args& operator=(const ThriftHiveMetastore_cancel_delegation_token_args&); + ThriftHiveMetastore_cancel_delegation_token_args() : token_str_form() { + } + + virtual ~ThriftHiveMetastore_cancel_delegation_token_args() noexcept; + std::string token_str_form; + + _ThriftHiveMetastore_cancel_delegation_token_args__isset __isset; + + void __set_token_str_form(const std::string& val); + + bool operator == (const ThriftHiveMetastore_cancel_delegation_token_args & rhs) const + { + if (!(token_str_form == rhs.token_str_form)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_cancel_delegation_token_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_cancel_delegation_token_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_cancel_delegation_token_pargs { + public: + + + virtual ~ThriftHiveMetastore_cancel_delegation_token_pargs() noexcept; + const std::string* token_str_form; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_cancel_delegation_token_result__isset { + _ThriftHiveMetastore_cancel_delegation_token_result__isset() : o1(false) {} + bool o1 :1; +} _ThriftHiveMetastore_cancel_delegation_token_result__isset; + +class ThriftHiveMetastore_cancel_delegation_token_result { + public: + + ThriftHiveMetastore_cancel_delegation_token_result(const ThriftHiveMetastore_cancel_delegation_token_result&); + ThriftHiveMetastore_cancel_delegation_token_result& operator=(const ThriftHiveMetastore_cancel_delegation_token_result&); + ThriftHiveMetastore_cancel_delegation_token_result() { + } + + virtual ~ThriftHiveMetastore_cancel_delegation_token_result() noexcept; + MetaException o1; + + _ThriftHiveMetastore_cancel_delegation_token_result__isset __isset; + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_cancel_delegation_token_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_cancel_delegation_token_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_cancel_delegation_token_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_cancel_delegation_token_presult__isset { + _ThriftHiveMetastore_cancel_delegation_token_presult__isset() : o1(false) {} + bool o1 :1; +} _ThriftHiveMetastore_cancel_delegation_token_presult__isset; + +class ThriftHiveMetastore_cancel_delegation_token_presult { + public: + + + virtual ~ThriftHiveMetastore_cancel_delegation_token_presult() noexcept; + MetaException o1; + + _ThriftHiveMetastore_cancel_delegation_token_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_add_token_args__isset { + _ThriftHiveMetastore_add_token_args__isset() : token_identifier(false), delegation_token(false) {} + bool token_identifier :1; + bool delegation_token :1; +} _ThriftHiveMetastore_add_token_args__isset; + +class ThriftHiveMetastore_add_token_args { + public: + + ThriftHiveMetastore_add_token_args(const ThriftHiveMetastore_add_token_args&); + ThriftHiveMetastore_add_token_args& operator=(const ThriftHiveMetastore_add_token_args&); + ThriftHiveMetastore_add_token_args() : token_identifier(), delegation_token() { + } + + virtual ~ThriftHiveMetastore_add_token_args() noexcept; + std::string token_identifier; + std::string delegation_token; + + _ThriftHiveMetastore_add_token_args__isset __isset; + + void __set_token_identifier(const std::string& val); + + void __set_delegation_token(const std::string& val); + + bool operator == (const ThriftHiveMetastore_add_token_args & rhs) const + { + if (!(token_identifier == rhs.token_identifier)) + return false; + if (!(delegation_token == rhs.delegation_token)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_token_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_token_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_add_token_pargs { + public: + + + virtual ~ThriftHiveMetastore_add_token_pargs() noexcept; + const std::string* token_identifier; + const std::string* delegation_token; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_token_result__isset { + _ThriftHiveMetastore_add_token_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_add_token_result__isset; + +class ThriftHiveMetastore_add_token_result { + public: + + ThriftHiveMetastore_add_token_result(const ThriftHiveMetastore_add_token_result&); + ThriftHiveMetastore_add_token_result& operator=(const ThriftHiveMetastore_add_token_result&); + ThriftHiveMetastore_add_token_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_add_token_result() noexcept; + bool success; + + _ThriftHiveMetastore_add_token_result__isset __isset; + + void __set_success(const bool val); + + bool operator == (const ThriftHiveMetastore_add_token_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_token_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_token_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_token_presult__isset { + _ThriftHiveMetastore_add_token_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_add_token_presult__isset; + +class ThriftHiveMetastore_add_token_presult { + public: + + + virtual ~ThriftHiveMetastore_add_token_presult() noexcept; + bool* success; + + _ThriftHiveMetastore_add_token_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_remove_token_args__isset { + _ThriftHiveMetastore_remove_token_args__isset() : token_identifier(false) {} + bool token_identifier :1; +} _ThriftHiveMetastore_remove_token_args__isset; + +class ThriftHiveMetastore_remove_token_args { + public: + + ThriftHiveMetastore_remove_token_args(const ThriftHiveMetastore_remove_token_args&); + ThriftHiveMetastore_remove_token_args& operator=(const ThriftHiveMetastore_remove_token_args&); + ThriftHiveMetastore_remove_token_args() : token_identifier() { + } + + virtual ~ThriftHiveMetastore_remove_token_args() noexcept; + std::string token_identifier; + + _ThriftHiveMetastore_remove_token_args__isset __isset; + + void __set_token_identifier(const std::string& val); + + bool operator == (const ThriftHiveMetastore_remove_token_args & rhs) const + { + if (!(token_identifier == rhs.token_identifier)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_remove_token_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_remove_token_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_remove_token_pargs { + public: + + + virtual ~ThriftHiveMetastore_remove_token_pargs() noexcept; + const std::string* token_identifier; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_remove_token_result__isset { + _ThriftHiveMetastore_remove_token_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_remove_token_result__isset; + +class ThriftHiveMetastore_remove_token_result { + public: + + ThriftHiveMetastore_remove_token_result(const ThriftHiveMetastore_remove_token_result&); + ThriftHiveMetastore_remove_token_result& operator=(const ThriftHiveMetastore_remove_token_result&); + ThriftHiveMetastore_remove_token_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_remove_token_result() noexcept; + bool success; + + _ThriftHiveMetastore_remove_token_result__isset __isset; + + void __set_success(const bool val); + + bool operator == (const ThriftHiveMetastore_remove_token_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_remove_token_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_remove_token_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_remove_token_presult__isset { + _ThriftHiveMetastore_remove_token_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_remove_token_presult__isset; + +class ThriftHiveMetastore_remove_token_presult { + public: + + + virtual ~ThriftHiveMetastore_remove_token_presult() noexcept; + bool* success; + + _ThriftHiveMetastore_remove_token_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_token_args__isset { + _ThriftHiveMetastore_get_token_args__isset() : token_identifier(false) {} + bool token_identifier :1; +} _ThriftHiveMetastore_get_token_args__isset; + +class ThriftHiveMetastore_get_token_args { + public: + + ThriftHiveMetastore_get_token_args(const ThriftHiveMetastore_get_token_args&); + ThriftHiveMetastore_get_token_args& operator=(const ThriftHiveMetastore_get_token_args&); + ThriftHiveMetastore_get_token_args() : token_identifier() { + } + + virtual ~ThriftHiveMetastore_get_token_args() noexcept; + std::string token_identifier; + + _ThriftHiveMetastore_get_token_args__isset __isset; + + void __set_token_identifier(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_token_args & rhs) const + { + if (!(token_identifier == rhs.token_identifier)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_token_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_token_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_token_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_token_pargs() noexcept; + const std::string* token_identifier; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_token_result__isset { + _ThriftHiveMetastore_get_token_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_token_result__isset; + +class ThriftHiveMetastore_get_token_result { + public: + + ThriftHiveMetastore_get_token_result(const ThriftHiveMetastore_get_token_result&); + ThriftHiveMetastore_get_token_result& operator=(const ThriftHiveMetastore_get_token_result&); + ThriftHiveMetastore_get_token_result() : success() { + } + + virtual ~ThriftHiveMetastore_get_token_result() noexcept; + std::string success; + + _ThriftHiveMetastore_get_token_result__isset __isset; + + void __set_success(const std::string& val); + + bool operator == (const ThriftHiveMetastore_get_token_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_token_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_token_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_token_presult__isset { + _ThriftHiveMetastore_get_token_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_token_presult__isset; + +class ThriftHiveMetastore_get_token_presult { + public: + + + virtual ~ThriftHiveMetastore_get_token_presult() noexcept; + std::string* success; + + _ThriftHiveMetastore_get_token_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + + +class ThriftHiveMetastore_get_all_token_identifiers_args { + public: + + ThriftHiveMetastore_get_all_token_identifiers_args(const ThriftHiveMetastore_get_all_token_identifiers_args&); + ThriftHiveMetastore_get_all_token_identifiers_args& operator=(const ThriftHiveMetastore_get_all_token_identifiers_args&); + ThriftHiveMetastore_get_all_token_identifiers_args() { + } + + virtual ~ThriftHiveMetastore_get_all_token_identifiers_args() noexcept; + + bool operator == (const ThriftHiveMetastore_get_all_token_identifiers_args & /* rhs */) const + { + return true; + } + bool operator != (const ThriftHiveMetastore_get_all_token_identifiers_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_all_token_identifiers_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_all_token_identifiers_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_all_token_identifiers_pargs() noexcept; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_all_token_identifiers_result__isset { + _ThriftHiveMetastore_get_all_token_identifiers_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_all_token_identifiers_result__isset; + +class ThriftHiveMetastore_get_all_token_identifiers_result { + public: + + ThriftHiveMetastore_get_all_token_identifiers_result(const ThriftHiveMetastore_get_all_token_identifiers_result&); + ThriftHiveMetastore_get_all_token_identifiers_result& operator=(const ThriftHiveMetastore_get_all_token_identifiers_result&); + ThriftHiveMetastore_get_all_token_identifiers_result() { + } + + virtual ~ThriftHiveMetastore_get_all_token_identifiers_result() noexcept; + std::vector success; + + _ThriftHiveMetastore_get_all_token_identifiers_result__isset __isset; + + void __set_success(const std::vector & val); + + bool operator == (const ThriftHiveMetastore_get_all_token_identifiers_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_all_token_identifiers_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_all_token_identifiers_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_all_token_identifiers_presult__isset { + _ThriftHiveMetastore_get_all_token_identifiers_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_all_token_identifiers_presult__isset; + +class ThriftHiveMetastore_get_all_token_identifiers_presult { + public: + + + virtual ~ThriftHiveMetastore_get_all_token_identifiers_presult() noexcept; + std::vector * success; + + _ThriftHiveMetastore_get_all_token_identifiers_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_add_master_key_args__isset { + _ThriftHiveMetastore_add_master_key_args__isset() : key(false) {} + bool key :1; +} _ThriftHiveMetastore_add_master_key_args__isset; + +class ThriftHiveMetastore_add_master_key_args { + public: + + ThriftHiveMetastore_add_master_key_args(const ThriftHiveMetastore_add_master_key_args&); + ThriftHiveMetastore_add_master_key_args& operator=(const ThriftHiveMetastore_add_master_key_args&); + ThriftHiveMetastore_add_master_key_args() : key() { + } + + virtual ~ThriftHiveMetastore_add_master_key_args() noexcept; + std::string key; + + _ThriftHiveMetastore_add_master_key_args__isset __isset; + + void __set_key(const std::string& val); + + bool operator == (const ThriftHiveMetastore_add_master_key_args & rhs) const + { + if (!(key == rhs.key)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_master_key_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_master_key_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_add_master_key_pargs { + public: + + + virtual ~ThriftHiveMetastore_add_master_key_pargs() noexcept; + const std::string* key; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_master_key_result__isset { + _ThriftHiveMetastore_add_master_key_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_add_master_key_result__isset; + +class ThriftHiveMetastore_add_master_key_result { + public: + + ThriftHiveMetastore_add_master_key_result(const ThriftHiveMetastore_add_master_key_result&); + ThriftHiveMetastore_add_master_key_result& operator=(const ThriftHiveMetastore_add_master_key_result&); + ThriftHiveMetastore_add_master_key_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_add_master_key_result() noexcept; + int32_t success; + MetaException o1; + + _ThriftHiveMetastore_add_master_key_result__isset __isset; + + void __set_success(const int32_t val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_add_master_key_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_master_key_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_master_key_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_master_key_presult__isset { + _ThriftHiveMetastore_add_master_key_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_add_master_key_presult__isset; + +class ThriftHiveMetastore_add_master_key_presult { + public: + + + virtual ~ThriftHiveMetastore_add_master_key_presult() noexcept; + int32_t* success; + MetaException o1; + + _ThriftHiveMetastore_add_master_key_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_update_master_key_args__isset { + _ThriftHiveMetastore_update_master_key_args__isset() : seq_number(false), key(false) {} + bool seq_number :1; + bool key :1; +} _ThriftHiveMetastore_update_master_key_args__isset; + +class ThriftHiveMetastore_update_master_key_args { + public: + + ThriftHiveMetastore_update_master_key_args(const ThriftHiveMetastore_update_master_key_args&); + ThriftHiveMetastore_update_master_key_args& operator=(const ThriftHiveMetastore_update_master_key_args&); + ThriftHiveMetastore_update_master_key_args() : seq_number(0), key() { + } + + virtual ~ThriftHiveMetastore_update_master_key_args() noexcept; + int32_t seq_number; + std::string key; + + _ThriftHiveMetastore_update_master_key_args__isset __isset; + + void __set_seq_number(const int32_t val); + + void __set_key(const std::string& val); + + bool operator == (const ThriftHiveMetastore_update_master_key_args & rhs) const + { + if (!(seq_number == rhs.seq_number)) + return false; + if (!(key == rhs.key)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_update_master_key_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_update_master_key_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_update_master_key_pargs { + public: + + + virtual ~ThriftHiveMetastore_update_master_key_pargs() noexcept; + const int32_t* seq_number; + const std::string* key; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_update_master_key_result__isset { + _ThriftHiveMetastore_update_master_key_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_update_master_key_result__isset; + +class ThriftHiveMetastore_update_master_key_result { + public: + + ThriftHiveMetastore_update_master_key_result(const ThriftHiveMetastore_update_master_key_result&); + ThriftHiveMetastore_update_master_key_result& operator=(const ThriftHiveMetastore_update_master_key_result&); + ThriftHiveMetastore_update_master_key_result() { + } + + virtual ~ThriftHiveMetastore_update_master_key_result() noexcept; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_update_master_key_result__isset __isset; + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_update_master_key_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_update_master_key_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_update_master_key_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_update_master_key_presult__isset { + _ThriftHiveMetastore_update_master_key_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_update_master_key_presult__isset; + +class ThriftHiveMetastore_update_master_key_presult { + public: + + + virtual ~ThriftHiveMetastore_update_master_key_presult() noexcept; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_update_master_key_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_remove_master_key_args__isset { + _ThriftHiveMetastore_remove_master_key_args__isset() : key_seq(false) {} + bool key_seq :1; +} _ThriftHiveMetastore_remove_master_key_args__isset; + +class ThriftHiveMetastore_remove_master_key_args { + public: + + ThriftHiveMetastore_remove_master_key_args(const ThriftHiveMetastore_remove_master_key_args&); + ThriftHiveMetastore_remove_master_key_args& operator=(const ThriftHiveMetastore_remove_master_key_args&); + ThriftHiveMetastore_remove_master_key_args() : key_seq(0) { + } + + virtual ~ThriftHiveMetastore_remove_master_key_args() noexcept; + int32_t key_seq; + + _ThriftHiveMetastore_remove_master_key_args__isset __isset; + + void __set_key_seq(const int32_t val); + + bool operator == (const ThriftHiveMetastore_remove_master_key_args & rhs) const + { + if (!(key_seq == rhs.key_seq)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_remove_master_key_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_remove_master_key_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_remove_master_key_pargs { + public: + + + virtual ~ThriftHiveMetastore_remove_master_key_pargs() noexcept; + const int32_t* key_seq; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_remove_master_key_result__isset { + _ThriftHiveMetastore_remove_master_key_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_remove_master_key_result__isset; + +class ThriftHiveMetastore_remove_master_key_result { + public: + + ThriftHiveMetastore_remove_master_key_result(const ThriftHiveMetastore_remove_master_key_result&); + ThriftHiveMetastore_remove_master_key_result& operator=(const ThriftHiveMetastore_remove_master_key_result&); + ThriftHiveMetastore_remove_master_key_result() : success(0) { + } + + virtual ~ThriftHiveMetastore_remove_master_key_result() noexcept; + bool success; + + _ThriftHiveMetastore_remove_master_key_result__isset __isset; + + void __set_success(const bool val); + + bool operator == (const ThriftHiveMetastore_remove_master_key_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_remove_master_key_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_remove_master_key_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_remove_master_key_presult__isset { + _ThriftHiveMetastore_remove_master_key_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_remove_master_key_presult__isset; + +class ThriftHiveMetastore_remove_master_key_presult { + public: + + + virtual ~ThriftHiveMetastore_remove_master_key_presult() noexcept; + bool* success; + + _ThriftHiveMetastore_remove_master_key_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + + +class ThriftHiveMetastore_get_master_keys_args { + public: + + ThriftHiveMetastore_get_master_keys_args(const ThriftHiveMetastore_get_master_keys_args&); + ThriftHiveMetastore_get_master_keys_args& operator=(const ThriftHiveMetastore_get_master_keys_args&); + ThriftHiveMetastore_get_master_keys_args() { + } + + virtual ~ThriftHiveMetastore_get_master_keys_args() noexcept; + + bool operator == (const ThriftHiveMetastore_get_master_keys_args & /* rhs */) const + { + return true; + } + bool operator != (const ThriftHiveMetastore_get_master_keys_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_master_keys_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_master_keys_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_master_keys_pargs() noexcept; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_master_keys_result__isset { + _ThriftHiveMetastore_get_master_keys_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_master_keys_result__isset; + +class ThriftHiveMetastore_get_master_keys_result { + public: + + ThriftHiveMetastore_get_master_keys_result(const ThriftHiveMetastore_get_master_keys_result&); + ThriftHiveMetastore_get_master_keys_result& operator=(const ThriftHiveMetastore_get_master_keys_result&); + ThriftHiveMetastore_get_master_keys_result() { + } + + virtual ~ThriftHiveMetastore_get_master_keys_result() noexcept; + std::vector success; + + _ThriftHiveMetastore_get_master_keys_result__isset __isset; + + void __set_success(const std::vector & val); + + bool operator == (const ThriftHiveMetastore_get_master_keys_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_master_keys_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_master_keys_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_master_keys_presult__isset { + _ThriftHiveMetastore_get_master_keys_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_master_keys_presult__isset; + +class ThriftHiveMetastore_get_master_keys_presult { + public: + + + virtual ~ThriftHiveMetastore_get_master_keys_presult() noexcept; + std::vector * success; + + _ThriftHiveMetastore_get_master_keys_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + + +class ThriftHiveMetastore_get_open_txns_args { + public: + + ThriftHiveMetastore_get_open_txns_args(const ThriftHiveMetastore_get_open_txns_args&); + ThriftHiveMetastore_get_open_txns_args& operator=(const ThriftHiveMetastore_get_open_txns_args&); + ThriftHiveMetastore_get_open_txns_args() { + } + + virtual ~ThriftHiveMetastore_get_open_txns_args() noexcept; + + bool operator == (const ThriftHiveMetastore_get_open_txns_args & /* rhs */) const + { + return true; + } + bool operator != (const ThriftHiveMetastore_get_open_txns_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_open_txns_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_open_txns_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_open_txns_pargs() noexcept; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_open_txns_result__isset { + _ThriftHiveMetastore_get_open_txns_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_open_txns_result__isset; + +class ThriftHiveMetastore_get_open_txns_result { + public: + + ThriftHiveMetastore_get_open_txns_result(const ThriftHiveMetastore_get_open_txns_result&); + ThriftHiveMetastore_get_open_txns_result& operator=(const ThriftHiveMetastore_get_open_txns_result&); + ThriftHiveMetastore_get_open_txns_result() { + } + + virtual ~ThriftHiveMetastore_get_open_txns_result() noexcept; + GetOpenTxnsResponse success; + + _ThriftHiveMetastore_get_open_txns_result__isset __isset; + + void __set_success(const GetOpenTxnsResponse& val); + + bool operator == (const ThriftHiveMetastore_get_open_txns_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_open_txns_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_open_txns_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_open_txns_presult__isset { + _ThriftHiveMetastore_get_open_txns_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_open_txns_presult__isset; + +class ThriftHiveMetastore_get_open_txns_presult { + public: + + + virtual ~ThriftHiveMetastore_get_open_txns_presult() noexcept; + GetOpenTxnsResponse* success; + + _ThriftHiveMetastore_get_open_txns_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + + +class ThriftHiveMetastore_get_open_txns_info_args { + public: + + ThriftHiveMetastore_get_open_txns_info_args(const ThriftHiveMetastore_get_open_txns_info_args&); + ThriftHiveMetastore_get_open_txns_info_args& operator=(const ThriftHiveMetastore_get_open_txns_info_args&); + ThriftHiveMetastore_get_open_txns_info_args() { + } + + virtual ~ThriftHiveMetastore_get_open_txns_info_args() noexcept; + + bool operator == (const ThriftHiveMetastore_get_open_txns_info_args & /* rhs */) const + { + return true; + } + bool operator != (const ThriftHiveMetastore_get_open_txns_info_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_open_txns_info_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_open_txns_info_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_open_txns_info_pargs() noexcept; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_open_txns_info_result__isset { + _ThriftHiveMetastore_get_open_txns_info_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_open_txns_info_result__isset; + +class ThriftHiveMetastore_get_open_txns_info_result { + public: + + ThriftHiveMetastore_get_open_txns_info_result(const ThriftHiveMetastore_get_open_txns_info_result&); + ThriftHiveMetastore_get_open_txns_info_result& operator=(const ThriftHiveMetastore_get_open_txns_info_result&); + ThriftHiveMetastore_get_open_txns_info_result() { + } + + virtual ~ThriftHiveMetastore_get_open_txns_info_result() noexcept; + GetOpenTxnsInfoResponse success; + + _ThriftHiveMetastore_get_open_txns_info_result__isset __isset; + + void __set_success(const GetOpenTxnsInfoResponse& val); + + bool operator == (const ThriftHiveMetastore_get_open_txns_info_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_open_txns_info_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_open_txns_info_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_open_txns_info_presult__isset { + _ThriftHiveMetastore_get_open_txns_info_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_open_txns_info_presult__isset; + +class ThriftHiveMetastore_get_open_txns_info_presult { + public: + + + virtual ~ThriftHiveMetastore_get_open_txns_info_presult() noexcept; + GetOpenTxnsInfoResponse* success; + + _ThriftHiveMetastore_get_open_txns_info_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_open_txns_args__isset { + _ThriftHiveMetastore_open_txns_args__isset() : rqst(false) {} + bool rqst :1; +} _ThriftHiveMetastore_open_txns_args__isset; + +class ThriftHiveMetastore_open_txns_args { + public: + + ThriftHiveMetastore_open_txns_args(const ThriftHiveMetastore_open_txns_args&); + ThriftHiveMetastore_open_txns_args& operator=(const ThriftHiveMetastore_open_txns_args&); + ThriftHiveMetastore_open_txns_args() { + } + + virtual ~ThriftHiveMetastore_open_txns_args() noexcept; + OpenTxnRequest rqst; + + _ThriftHiveMetastore_open_txns_args__isset __isset; + + void __set_rqst(const OpenTxnRequest& val); + + bool operator == (const ThriftHiveMetastore_open_txns_args & rhs) const + { + if (!(rqst == rhs.rqst)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_open_txns_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_open_txns_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_open_txns_pargs { + public: + + + virtual ~ThriftHiveMetastore_open_txns_pargs() noexcept; + const OpenTxnRequest* rqst; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_open_txns_result__isset { + _ThriftHiveMetastore_open_txns_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_open_txns_result__isset; + +class ThriftHiveMetastore_open_txns_result { + public: + + ThriftHiveMetastore_open_txns_result(const ThriftHiveMetastore_open_txns_result&); + ThriftHiveMetastore_open_txns_result& operator=(const ThriftHiveMetastore_open_txns_result&); + ThriftHiveMetastore_open_txns_result() { + } + + virtual ~ThriftHiveMetastore_open_txns_result() noexcept; + OpenTxnsResponse success; + + _ThriftHiveMetastore_open_txns_result__isset __isset; + + void __set_success(const OpenTxnsResponse& val); + + bool operator == (const ThriftHiveMetastore_open_txns_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_open_txns_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_open_txns_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_open_txns_presult__isset { + _ThriftHiveMetastore_open_txns_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_open_txns_presult__isset; + +class ThriftHiveMetastore_open_txns_presult { + public: + + + virtual ~ThriftHiveMetastore_open_txns_presult() noexcept; + OpenTxnsResponse* success; + + _ThriftHiveMetastore_open_txns_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_abort_txn_args__isset { + _ThriftHiveMetastore_abort_txn_args__isset() : rqst(false) {} + bool rqst :1; +} _ThriftHiveMetastore_abort_txn_args__isset; + +class ThriftHiveMetastore_abort_txn_args { + public: + + ThriftHiveMetastore_abort_txn_args(const ThriftHiveMetastore_abort_txn_args&); + ThriftHiveMetastore_abort_txn_args& operator=(const ThriftHiveMetastore_abort_txn_args&); + ThriftHiveMetastore_abort_txn_args() { + } + + virtual ~ThriftHiveMetastore_abort_txn_args() noexcept; + AbortTxnRequest rqst; + + _ThriftHiveMetastore_abort_txn_args__isset __isset; + + void __set_rqst(const AbortTxnRequest& val); + + bool operator == (const ThriftHiveMetastore_abort_txn_args & rhs) const + { + if (!(rqst == rhs.rqst)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_abort_txn_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_abort_txn_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_abort_txn_pargs { + public: + + + virtual ~ThriftHiveMetastore_abort_txn_pargs() noexcept; + const AbortTxnRequest* rqst; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_abort_txn_result__isset { + _ThriftHiveMetastore_abort_txn_result__isset() : o1(false) {} + bool o1 :1; +} _ThriftHiveMetastore_abort_txn_result__isset; + +class ThriftHiveMetastore_abort_txn_result { + public: + + ThriftHiveMetastore_abort_txn_result(const ThriftHiveMetastore_abort_txn_result&); + ThriftHiveMetastore_abort_txn_result& operator=(const ThriftHiveMetastore_abort_txn_result&); + ThriftHiveMetastore_abort_txn_result() { + } + + virtual ~ThriftHiveMetastore_abort_txn_result() noexcept; + NoSuchTxnException o1; + + _ThriftHiveMetastore_abort_txn_result__isset __isset; + + void __set_o1(const NoSuchTxnException& val); + + bool operator == (const ThriftHiveMetastore_abort_txn_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_abort_txn_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_abort_txn_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_abort_txn_presult__isset { + _ThriftHiveMetastore_abort_txn_presult__isset() : o1(false) {} + bool o1 :1; +} _ThriftHiveMetastore_abort_txn_presult__isset; + +class ThriftHiveMetastore_abort_txn_presult { + public: + + + virtual ~ThriftHiveMetastore_abort_txn_presult() noexcept; + NoSuchTxnException o1; + + _ThriftHiveMetastore_abort_txn_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_abort_txns_args__isset { + _ThriftHiveMetastore_abort_txns_args__isset() : rqst(false) {} + bool rqst :1; +} _ThriftHiveMetastore_abort_txns_args__isset; + +class ThriftHiveMetastore_abort_txns_args { + public: + + ThriftHiveMetastore_abort_txns_args(const ThriftHiveMetastore_abort_txns_args&); + ThriftHiveMetastore_abort_txns_args& operator=(const ThriftHiveMetastore_abort_txns_args&); + ThriftHiveMetastore_abort_txns_args() { + } + + virtual ~ThriftHiveMetastore_abort_txns_args() noexcept; + AbortTxnsRequest rqst; + + _ThriftHiveMetastore_abort_txns_args__isset __isset; + + void __set_rqst(const AbortTxnsRequest& val); + + bool operator == (const ThriftHiveMetastore_abort_txns_args & rhs) const + { + if (!(rqst == rhs.rqst)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_abort_txns_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_abort_txns_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_abort_txns_pargs { + public: + + + virtual ~ThriftHiveMetastore_abort_txns_pargs() noexcept; + const AbortTxnsRequest* rqst; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_abort_txns_result__isset { + _ThriftHiveMetastore_abort_txns_result__isset() : o1(false) {} + bool o1 :1; +} _ThriftHiveMetastore_abort_txns_result__isset; + +class ThriftHiveMetastore_abort_txns_result { + public: + + ThriftHiveMetastore_abort_txns_result(const ThriftHiveMetastore_abort_txns_result&); + ThriftHiveMetastore_abort_txns_result& operator=(const ThriftHiveMetastore_abort_txns_result&); + ThriftHiveMetastore_abort_txns_result() { + } + + virtual ~ThriftHiveMetastore_abort_txns_result() noexcept; + NoSuchTxnException o1; + + _ThriftHiveMetastore_abort_txns_result__isset __isset; + + void __set_o1(const NoSuchTxnException& val); + + bool operator == (const ThriftHiveMetastore_abort_txns_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_abort_txns_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_abort_txns_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_abort_txns_presult__isset { + _ThriftHiveMetastore_abort_txns_presult__isset() : o1(false) {} + bool o1 :1; +} _ThriftHiveMetastore_abort_txns_presult__isset; + +class ThriftHiveMetastore_abort_txns_presult { + public: + + + virtual ~ThriftHiveMetastore_abort_txns_presult() noexcept; + NoSuchTxnException o1; + + _ThriftHiveMetastore_abort_txns_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_commit_txn_args__isset { + _ThriftHiveMetastore_commit_txn_args__isset() : rqst(false) {} + bool rqst :1; +} _ThriftHiveMetastore_commit_txn_args__isset; + +class ThriftHiveMetastore_commit_txn_args { + public: + + ThriftHiveMetastore_commit_txn_args(const ThriftHiveMetastore_commit_txn_args&); + ThriftHiveMetastore_commit_txn_args& operator=(const ThriftHiveMetastore_commit_txn_args&); + ThriftHiveMetastore_commit_txn_args() { + } + + virtual ~ThriftHiveMetastore_commit_txn_args() noexcept; + CommitTxnRequest rqst; + + _ThriftHiveMetastore_commit_txn_args__isset __isset; + + void __set_rqst(const CommitTxnRequest& val); + + bool operator == (const ThriftHiveMetastore_commit_txn_args & rhs) const + { + if (!(rqst == rhs.rqst)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_commit_txn_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_commit_txn_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_commit_txn_pargs { + public: + + + virtual ~ThriftHiveMetastore_commit_txn_pargs() noexcept; + const CommitTxnRequest* rqst; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_commit_txn_result__isset { + _ThriftHiveMetastore_commit_txn_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_commit_txn_result__isset; + +class ThriftHiveMetastore_commit_txn_result { + public: + + ThriftHiveMetastore_commit_txn_result(const ThriftHiveMetastore_commit_txn_result&); + ThriftHiveMetastore_commit_txn_result& operator=(const ThriftHiveMetastore_commit_txn_result&); + ThriftHiveMetastore_commit_txn_result() { + } + + virtual ~ThriftHiveMetastore_commit_txn_result() noexcept; + NoSuchTxnException o1; + TxnAbortedException o2; + + _ThriftHiveMetastore_commit_txn_result__isset __isset; + + void __set_o1(const NoSuchTxnException& val); + + void __set_o2(const TxnAbortedException& val); + + bool operator == (const ThriftHiveMetastore_commit_txn_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_commit_txn_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_commit_txn_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_commit_txn_presult__isset { + _ThriftHiveMetastore_commit_txn_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_commit_txn_presult__isset; + +class ThriftHiveMetastore_commit_txn_presult { + public: + + + virtual ~ThriftHiveMetastore_commit_txn_presult() noexcept; + NoSuchTxnException o1; + TxnAbortedException o2; + + _ThriftHiveMetastore_commit_txn_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_lock_args__isset { + _ThriftHiveMetastore_lock_args__isset() : rqst(false) {} + bool rqst :1; +} _ThriftHiveMetastore_lock_args__isset; + +class ThriftHiveMetastore_lock_args { + public: + + ThriftHiveMetastore_lock_args(const ThriftHiveMetastore_lock_args&); + ThriftHiveMetastore_lock_args& operator=(const ThriftHiveMetastore_lock_args&); + ThriftHiveMetastore_lock_args() { + } + + virtual ~ThriftHiveMetastore_lock_args() noexcept; + LockRequest rqst; + + _ThriftHiveMetastore_lock_args__isset __isset; + + void __set_rqst(const LockRequest& val); + + bool operator == (const ThriftHiveMetastore_lock_args & rhs) const + { + if (!(rqst == rhs.rqst)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_lock_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_lock_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_lock_pargs { + public: + + + virtual ~ThriftHiveMetastore_lock_pargs() noexcept; + const LockRequest* rqst; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_lock_result__isset { + _ThriftHiveMetastore_lock_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_lock_result__isset; + +class ThriftHiveMetastore_lock_result { + public: + + ThriftHiveMetastore_lock_result(const ThriftHiveMetastore_lock_result&); + ThriftHiveMetastore_lock_result& operator=(const ThriftHiveMetastore_lock_result&); + ThriftHiveMetastore_lock_result() { + } + + virtual ~ThriftHiveMetastore_lock_result() noexcept; + LockResponse success; + NoSuchTxnException o1; + TxnAbortedException o2; + + _ThriftHiveMetastore_lock_result__isset __isset; + + void __set_success(const LockResponse& val); + + void __set_o1(const NoSuchTxnException& val); + + void __set_o2(const TxnAbortedException& val); + + bool operator == (const ThriftHiveMetastore_lock_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_lock_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_lock_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_lock_presult__isset { + _ThriftHiveMetastore_lock_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_lock_presult__isset; + +class ThriftHiveMetastore_lock_presult { + public: + + + virtual ~ThriftHiveMetastore_lock_presult() noexcept; + LockResponse* success; + NoSuchTxnException o1; + TxnAbortedException o2; + + _ThriftHiveMetastore_lock_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_check_lock_args__isset { + _ThriftHiveMetastore_check_lock_args__isset() : rqst(false) {} + bool rqst :1; +} _ThriftHiveMetastore_check_lock_args__isset; + +class ThriftHiveMetastore_check_lock_args { + public: + + ThriftHiveMetastore_check_lock_args(const ThriftHiveMetastore_check_lock_args&); + ThriftHiveMetastore_check_lock_args& operator=(const ThriftHiveMetastore_check_lock_args&); + ThriftHiveMetastore_check_lock_args() { + } + + virtual ~ThriftHiveMetastore_check_lock_args() noexcept; + CheckLockRequest rqst; + + _ThriftHiveMetastore_check_lock_args__isset __isset; + + void __set_rqst(const CheckLockRequest& val); + + bool operator == (const ThriftHiveMetastore_check_lock_args & rhs) const + { + if (!(rqst == rhs.rqst)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_check_lock_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_check_lock_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_check_lock_pargs { + public: + + + virtual ~ThriftHiveMetastore_check_lock_pargs() noexcept; + const CheckLockRequest* rqst; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_check_lock_result__isset { + _ThriftHiveMetastore_check_lock_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_check_lock_result__isset; + +class ThriftHiveMetastore_check_lock_result { + public: + + ThriftHiveMetastore_check_lock_result(const ThriftHiveMetastore_check_lock_result&); + ThriftHiveMetastore_check_lock_result& operator=(const ThriftHiveMetastore_check_lock_result&); + ThriftHiveMetastore_check_lock_result() { + } + + virtual ~ThriftHiveMetastore_check_lock_result() noexcept; + LockResponse success; + NoSuchTxnException o1; + TxnAbortedException o2; + NoSuchLockException o3; + + _ThriftHiveMetastore_check_lock_result__isset __isset; + + void __set_success(const LockResponse& val); + + void __set_o1(const NoSuchTxnException& val); + + void __set_o2(const TxnAbortedException& val); + + void __set_o3(const NoSuchLockException& val); + + bool operator == (const ThriftHiveMetastore_check_lock_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_check_lock_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_check_lock_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_check_lock_presult__isset { + _ThriftHiveMetastore_check_lock_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_check_lock_presult__isset; + +class ThriftHiveMetastore_check_lock_presult { + public: + + + virtual ~ThriftHiveMetastore_check_lock_presult() noexcept; + LockResponse* success; + NoSuchTxnException o1; + TxnAbortedException o2; + NoSuchLockException o3; + + _ThriftHiveMetastore_check_lock_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_unlock_args__isset { + _ThriftHiveMetastore_unlock_args__isset() : rqst(false) {} + bool rqst :1; +} _ThriftHiveMetastore_unlock_args__isset; + +class ThriftHiveMetastore_unlock_args { + public: + + ThriftHiveMetastore_unlock_args(const ThriftHiveMetastore_unlock_args&); + ThriftHiveMetastore_unlock_args& operator=(const ThriftHiveMetastore_unlock_args&); + ThriftHiveMetastore_unlock_args() { + } + + virtual ~ThriftHiveMetastore_unlock_args() noexcept; + UnlockRequest rqst; + + _ThriftHiveMetastore_unlock_args__isset __isset; + + void __set_rqst(const UnlockRequest& val); + + bool operator == (const ThriftHiveMetastore_unlock_args & rhs) const + { + if (!(rqst == rhs.rqst)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_unlock_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_unlock_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_unlock_pargs { + public: + + + virtual ~ThriftHiveMetastore_unlock_pargs() noexcept; + const UnlockRequest* rqst; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_unlock_result__isset { + _ThriftHiveMetastore_unlock_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_unlock_result__isset; + +class ThriftHiveMetastore_unlock_result { + public: + + ThriftHiveMetastore_unlock_result(const ThriftHiveMetastore_unlock_result&); + ThriftHiveMetastore_unlock_result& operator=(const ThriftHiveMetastore_unlock_result&); + ThriftHiveMetastore_unlock_result() { + } + + virtual ~ThriftHiveMetastore_unlock_result() noexcept; + NoSuchLockException o1; + TxnOpenException o2; + + _ThriftHiveMetastore_unlock_result__isset __isset; + + void __set_o1(const NoSuchLockException& val); + + void __set_o2(const TxnOpenException& val); + + bool operator == (const ThriftHiveMetastore_unlock_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_unlock_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_unlock_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_unlock_presult__isset { + _ThriftHiveMetastore_unlock_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_unlock_presult__isset; + +class ThriftHiveMetastore_unlock_presult { + public: + + + virtual ~ThriftHiveMetastore_unlock_presult() noexcept; + NoSuchLockException o1; + TxnOpenException o2; + + _ThriftHiveMetastore_unlock_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_show_locks_args__isset { + _ThriftHiveMetastore_show_locks_args__isset() : rqst(false) {} + bool rqst :1; +} _ThriftHiveMetastore_show_locks_args__isset; + +class ThriftHiveMetastore_show_locks_args { + public: + + ThriftHiveMetastore_show_locks_args(const ThriftHiveMetastore_show_locks_args&); + ThriftHiveMetastore_show_locks_args& operator=(const ThriftHiveMetastore_show_locks_args&); + ThriftHiveMetastore_show_locks_args() { + } + + virtual ~ThriftHiveMetastore_show_locks_args() noexcept; + ShowLocksRequest rqst; + + _ThriftHiveMetastore_show_locks_args__isset __isset; + + void __set_rqst(const ShowLocksRequest& val); + + bool operator == (const ThriftHiveMetastore_show_locks_args & rhs) const + { + if (!(rqst == rhs.rqst)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_show_locks_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_show_locks_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_show_locks_pargs { + public: + + + virtual ~ThriftHiveMetastore_show_locks_pargs() noexcept; + const ShowLocksRequest* rqst; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_show_locks_result__isset { + _ThriftHiveMetastore_show_locks_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_show_locks_result__isset; + +class ThriftHiveMetastore_show_locks_result { + public: + + ThriftHiveMetastore_show_locks_result(const ThriftHiveMetastore_show_locks_result&); + ThriftHiveMetastore_show_locks_result& operator=(const ThriftHiveMetastore_show_locks_result&); + ThriftHiveMetastore_show_locks_result() { + } + + virtual ~ThriftHiveMetastore_show_locks_result() noexcept; + ShowLocksResponse success; + + _ThriftHiveMetastore_show_locks_result__isset __isset; + + void __set_success(const ShowLocksResponse& val); + + bool operator == (const ThriftHiveMetastore_show_locks_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_show_locks_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_show_locks_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_show_locks_presult__isset { + _ThriftHiveMetastore_show_locks_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_show_locks_presult__isset; + +class ThriftHiveMetastore_show_locks_presult { + public: + + + virtual ~ThriftHiveMetastore_show_locks_presult() noexcept; + ShowLocksResponse* success; + + _ThriftHiveMetastore_show_locks_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_heartbeat_args__isset { + _ThriftHiveMetastore_heartbeat_args__isset() : ids(false) {} + bool ids :1; +} _ThriftHiveMetastore_heartbeat_args__isset; + +class ThriftHiveMetastore_heartbeat_args { + public: + + ThriftHiveMetastore_heartbeat_args(const ThriftHiveMetastore_heartbeat_args&); + ThriftHiveMetastore_heartbeat_args& operator=(const ThriftHiveMetastore_heartbeat_args&); + ThriftHiveMetastore_heartbeat_args() { + } + + virtual ~ThriftHiveMetastore_heartbeat_args() noexcept; + HeartbeatRequest ids; + + _ThriftHiveMetastore_heartbeat_args__isset __isset; + + void __set_ids(const HeartbeatRequest& val); + + bool operator == (const ThriftHiveMetastore_heartbeat_args & rhs) const + { + if (!(ids == rhs.ids)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_heartbeat_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_heartbeat_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_heartbeat_pargs { + public: + + + virtual ~ThriftHiveMetastore_heartbeat_pargs() noexcept; + const HeartbeatRequest* ids; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_heartbeat_result__isset { + _ThriftHiveMetastore_heartbeat_result__isset() : o1(false), o2(false), o3(false) {} + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_heartbeat_result__isset; + +class ThriftHiveMetastore_heartbeat_result { + public: + + ThriftHiveMetastore_heartbeat_result(const ThriftHiveMetastore_heartbeat_result&); + ThriftHiveMetastore_heartbeat_result& operator=(const ThriftHiveMetastore_heartbeat_result&); + ThriftHiveMetastore_heartbeat_result() { + } + + virtual ~ThriftHiveMetastore_heartbeat_result() noexcept; + NoSuchLockException o1; + NoSuchTxnException o2; + TxnAbortedException o3; + + _ThriftHiveMetastore_heartbeat_result__isset __isset; + + void __set_o1(const NoSuchLockException& val); + + void __set_o2(const NoSuchTxnException& val); + + void __set_o3(const TxnAbortedException& val); + + bool operator == (const ThriftHiveMetastore_heartbeat_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_heartbeat_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_heartbeat_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_heartbeat_presult__isset { + _ThriftHiveMetastore_heartbeat_presult__isset() : o1(false), o2(false), o3(false) {} + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_heartbeat_presult__isset; + +class ThriftHiveMetastore_heartbeat_presult { + public: + + + virtual ~ThriftHiveMetastore_heartbeat_presult() noexcept; + NoSuchLockException o1; + NoSuchTxnException o2; + TxnAbortedException o3; + + _ThriftHiveMetastore_heartbeat_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_heartbeat_txn_range_args__isset { + _ThriftHiveMetastore_heartbeat_txn_range_args__isset() : txns(false) {} + bool txns :1; +} _ThriftHiveMetastore_heartbeat_txn_range_args__isset; + +class ThriftHiveMetastore_heartbeat_txn_range_args { + public: + + ThriftHiveMetastore_heartbeat_txn_range_args(const ThriftHiveMetastore_heartbeat_txn_range_args&); + ThriftHiveMetastore_heartbeat_txn_range_args& operator=(const ThriftHiveMetastore_heartbeat_txn_range_args&); + ThriftHiveMetastore_heartbeat_txn_range_args() { + } + + virtual ~ThriftHiveMetastore_heartbeat_txn_range_args() noexcept; + HeartbeatTxnRangeRequest txns; + + _ThriftHiveMetastore_heartbeat_txn_range_args__isset __isset; + + void __set_txns(const HeartbeatTxnRangeRequest& val); + + bool operator == (const ThriftHiveMetastore_heartbeat_txn_range_args & rhs) const + { + if (!(txns == rhs.txns)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_heartbeat_txn_range_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_heartbeat_txn_range_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_heartbeat_txn_range_pargs { + public: + + + virtual ~ThriftHiveMetastore_heartbeat_txn_range_pargs() noexcept; + const HeartbeatTxnRangeRequest* txns; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_heartbeat_txn_range_result__isset { + _ThriftHiveMetastore_heartbeat_txn_range_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_heartbeat_txn_range_result__isset; + +class ThriftHiveMetastore_heartbeat_txn_range_result { + public: + + ThriftHiveMetastore_heartbeat_txn_range_result(const ThriftHiveMetastore_heartbeat_txn_range_result&); + ThriftHiveMetastore_heartbeat_txn_range_result& operator=(const ThriftHiveMetastore_heartbeat_txn_range_result&); + ThriftHiveMetastore_heartbeat_txn_range_result() { + } + + virtual ~ThriftHiveMetastore_heartbeat_txn_range_result() noexcept; + HeartbeatTxnRangeResponse success; + + _ThriftHiveMetastore_heartbeat_txn_range_result__isset __isset; + + void __set_success(const HeartbeatTxnRangeResponse& val); + + bool operator == (const ThriftHiveMetastore_heartbeat_txn_range_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_heartbeat_txn_range_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_heartbeat_txn_range_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_heartbeat_txn_range_presult__isset { + _ThriftHiveMetastore_heartbeat_txn_range_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_heartbeat_txn_range_presult__isset; + +class ThriftHiveMetastore_heartbeat_txn_range_presult { + public: + + + virtual ~ThriftHiveMetastore_heartbeat_txn_range_presult() noexcept; + HeartbeatTxnRangeResponse* success; + + _ThriftHiveMetastore_heartbeat_txn_range_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_compact_args__isset { + _ThriftHiveMetastore_compact_args__isset() : rqst(false) {} + bool rqst :1; +} _ThriftHiveMetastore_compact_args__isset; + +class ThriftHiveMetastore_compact_args { + public: + + ThriftHiveMetastore_compact_args(const ThriftHiveMetastore_compact_args&); + ThriftHiveMetastore_compact_args& operator=(const ThriftHiveMetastore_compact_args&); + ThriftHiveMetastore_compact_args() { + } + + virtual ~ThriftHiveMetastore_compact_args() noexcept; + CompactionRequest rqst; + + _ThriftHiveMetastore_compact_args__isset __isset; + + void __set_rqst(const CompactionRequest& val); + + bool operator == (const ThriftHiveMetastore_compact_args & rhs) const + { + if (!(rqst == rhs.rqst)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_compact_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_compact_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_compact_pargs { + public: + + + virtual ~ThriftHiveMetastore_compact_pargs() noexcept; + const CompactionRequest* rqst; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_compact_result { + public: + + ThriftHiveMetastore_compact_result(const ThriftHiveMetastore_compact_result&); + ThriftHiveMetastore_compact_result& operator=(const ThriftHiveMetastore_compact_result&); + ThriftHiveMetastore_compact_result() { + } + + virtual ~ThriftHiveMetastore_compact_result() noexcept; + + bool operator == (const ThriftHiveMetastore_compact_result & /* rhs */) const + { + return true; + } + bool operator != (const ThriftHiveMetastore_compact_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_compact_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_compact_presult { + public: + + + virtual ~ThriftHiveMetastore_compact_presult() noexcept; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_compact2_args__isset { + _ThriftHiveMetastore_compact2_args__isset() : rqst(false) {} + bool rqst :1; +} _ThriftHiveMetastore_compact2_args__isset; + +class ThriftHiveMetastore_compact2_args { + public: + + ThriftHiveMetastore_compact2_args(const ThriftHiveMetastore_compact2_args&); + ThriftHiveMetastore_compact2_args& operator=(const ThriftHiveMetastore_compact2_args&); + ThriftHiveMetastore_compact2_args() { + } + + virtual ~ThriftHiveMetastore_compact2_args() noexcept; + CompactionRequest rqst; + + _ThriftHiveMetastore_compact2_args__isset __isset; + + void __set_rqst(const CompactionRequest& val); + + bool operator == (const ThriftHiveMetastore_compact2_args & rhs) const + { + if (!(rqst == rhs.rqst)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_compact2_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_compact2_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_compact2_pargs { + public: + + + virtual ~ThriftHiveMetastore_compact2_pargs() noexcept; + const CompactionRequest* rqst; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_compact2_result__isset { + _ThriftHiveMetastore_compact2_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_compact2_result__isset; + +class ThriftHiveMetastore_compact2_result { + public: + + ThriftHiveMetastore_compact2_result(const ThriftHiveMetastore_compact2_result&); + ThriftHiveMetastore_compact2_result& operator=(const ThriftHiveMetastore_compact2_result&); + ThriftHiveMetastore_compact2_result() { + } + + virtual ~ThriftHiveMetastore_compact2_result() noexcept; + CompactionResponse success; + + _ThriftHiveMetastore_compact2_result__isset __isset; + + void __set_success(const CompactionResponse& val); + + bool operator == (const ThriftHiveMetastore_compact2_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_compact2_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_compact2_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_compact2_presult__isset { + _ThriftHiveMetastore_compact2_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_compact2_presult__isset; + +class ThriftHiveMetastore_compact2_presult { + public: + + + virtual ~ThriftHiveMetastore_compact2_presult() noexcept; + CompactionResponse* success; + + _ThriftHiveMetastore_compact2_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_show_compact_args__isset { + _ThriftHiveMetastore_show_compact_args__isset() : rqst(false) {} + bool rqst :1; +} _ThriftHiveMetastore_show_compact_args__isset; + +class ThriftHiveMetastore_show_compact_args { + public: + + ThriftHiveMetastore_show_compact_args(const ThriftHiveMetastore_show_compact_args&); + ThriftHiveMetastore_show_compact_args& operator=(const ThriftHiveMetastore_show_compact_args&); + ThriftHiveMetastore_show_compact_args() { + } + + virtual ~ThriftHiveMetastore_show_compact_args() noexcept; + ShowCompactRequest rqst; + + _ThriftHiveMetastore_show_compact_args__isset __isset; + + void __set_rqst(const ShowCompactRequest& val); + + bool operator == (const ThriftHiveMetastore_show_compact_args & rhs) const + { + if (!(rqst == rhs.rqst)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_show_compact_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_show_compact_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_show_compact_pargs { + public: + + + virtual ~ThriftHiveMetastore_show_compact_pargs() noexcept; + const ShowCompactRequest* rqst; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_show_compact_result__isset { + _ThriftHiveMetastore_show_compact_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_show_compact_result__isset; + +class ThriftHiveMetastore_show_compact_result { + public: + + ThriftHiveMetastore_show_compact_result(const ThriftHiveMetastore_show_compact_result&); + ThriftHiveMetastore_show_compact_result& operator=(const ThriftHiveMetastore_show_compact_result&); + ThriftHiveMetastore_show_compact_result() { + } + + virtual ~ThriftHiveMetastore_show_compact_result() noexcept; + ShowCompactResponse success; + + _ThriftHiveMetastore_show_compact_result__isset __isset; + + void __set_success(const ShowCompactResponse& val); + + bool operator == (const ThriftHiveMetastore_show_compact_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_show_compact_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_show_compact_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_show_compact_presult__isset { + _ThriftHiveMetastore_show_compact_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_show_compact_presult__isset; + +class ThriftHiveMetastore_show_compact_presult { + public: + + + virtual ~ThriftHiveMetastore_show_compact_presult() noexcept; + ShowCompactResponse* success; + + _ThriftHiveMetastore_show_compact_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_add_dynamic_partitions_args__isset { + _ThriftHiveMetastore_add_dynamic_partitions_args__isset() : rqst(false) {} + bool rqst :1; +} _ThriftHiveMetastore_add_dynamic_partitions_args__isset; + +class ThriftHiveMetastore_add_dynamic_partitions_args { + public: + + ThriftHiveMetastore_add_dynamic_partitions_args(const ThriftHiveMetastore_add_dynamic_partitions_args&); + ThriftHiveMetastore_add_dynamic_partitions_args& operator=(const ThriftHiveMetastore_add_dynamic_partitions_args&); + ThriftHiveMetastore_add_dynamic_partitions_args() { + } + + virtual ~ThriftHiveMetastore_add_dynamic_partitions_args() noexcept; + AddDynamicPartitions rqst; + + _ThriftHiveMetastore_add_dynamic_partitions_args__isset __isset; + + void __set_rqst(const AddDynamicPartitions& val); + + bool operator == (const ThriftHiveMetastore_add_dynamic_partitions_args & rhs) const + { + if (!(rqst == rhs.rqst)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_dynamic_partitions_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_dynamic_partitions_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_add_dynamic_partitions_pargs { + public: + + + virtual ~ThriftHiveMetastore_add_dynamic_partitions_pargs() noexcept; + const AddDynamicPartitions* rqst; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_dynamic_partitions_result__isset { + _ThriftHiveMetastore_add_dynamic_partitions_result__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_add_dynamic_partitions_result__isset; + +class ThriftHiveMetastore_add_dynamic_partitions_result { + public: + + ThriftHiveMetastore_add_dynamic_partitions_result(const ThriftHiveMetastore_add_dynamic_partitions_result&); + ThriftHiveMetastore_add_dynamic_partitions_result& operator=(const ThriftHiveMetastore_add_dynamic_partitions_result&); + ThriftHiveMetastore_add_dynamic_partitions_result() { + } + + virtual ~ThriftHiveMetastore_add_dynamic_partitions_result() noexcept; + NoSuchTxnException o1; + TxnAbortedException o2; + + _ThriftHiveMetastore_add_dynamic_partitions_result__isset __isset; + + void __set_o1(const NoSuchTxnException& val); + + void __set_o2(const TxnAbortedException& val); + + bool operator == (const ThriftHiveMetastore_add_dynamic_partitions_result & rhs) const + { + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_add_dynamic_partitions_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_add_dynamic_partitions_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_add_dynamic_partitions_presult__isset { + _ThriftHiveMetastore_add_dynamic_partitions_presult__isset() : o1(false), o2(false) {} + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_add_dynamic_partitions_presult__isset; + +class ThriftHiveMetastore_add_dynamic_partitions_presult { + public: + + + virtual ~ThriftHiveMetastore_add_dynamic_partitions_presult() noexcept; + NoSuchTxnException o1; + TxnAbortedException o2; + + _ThriftHiveMetastore_add_dynamic_partitions_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_next_notification_args__isset { + _ThriftHiveMetastore_get_next_notification_args__isset() : rqst(false) {} + bool rqst :1; +} _ThriftHiveMetastore_get_next_notification_args__isset; + +class ThriftHiveMetastore_get_next_notification_args { + public: + + ThriftHiveMetastore_get_next_notification_args(const ThriftHiveMetastore_get_next_notification_args&); + ThriftHiveMetastore_get_next_notification_args& operator=(const ThriftHiveMetastore_get_next_notification_args&); + ThriftHiveMetastore_get_next_notification_args() { + } + + virtual ~ThriftHiveMetastore_get_next_notification_args() noexcept; + NotificationEventRequest rqst; + + _ThriftHiveMetastore_get_next_notification_args__isset __isset; + + void __set_rqst(const NotificationEventRequest& val); + + bool operator == (const ThriftHiveMetastore_get_next_notification_args & rhs) const + { + if (!(rqst == rhs.rqst)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_next_notification_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_next_notification_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_next_notification_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_next_notification_pargs() noexcept; + const NotificationEventRequest* rqst; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_next_notification_result__isset { + _ThriftHiveMetastore_get_next_notification_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_next_notification_result__isset; + +class ThriftHiveMetastore_get_next_notification_result { + public: + + ThriftHiveMetastore_get_next_notification_result(const ThriftHiveMetastore_get_next_notification_result&); + ThriftHiveMetastore_get_next_notification_result& operator=(const ThriftHiveMetastore_get_next_notification_result&); + ThriftHiveMetastore_get_next_notification_result() { + } + + virtual ~ThriftHiveMetastore_get_next_notification_result() noexcept; + NotificationEventResponse success; + + _ThriftHiveMetastore_get_next_notification_result__isset __isset; + + void __set_success(const NotificationEventResponse& val); + + bool operator == (const ThriftHiveMetastore_get_next_notification_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_next_notification_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_next_notification_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_next_notification_presult__isset { + _ThriftHiveMetastore_get_next_notification_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_next_notification_presult__isset; + +class ThriftHiveMetastore_get_next_notification_presult { + public: + + + virtual ~ThriftHiveMetastore_get_next_notification_presult() noexcept; + NotificationEventResponse* success; + + _ThriftHiveMetastore_get_next_notification_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + + +class ThriftHiveMetastore_get_current_notificationEventId_args { + public: + + ThriftHiveMetastore_get_current_notificationEventId_args(const ThriftHiveMetastore_get_current_notificationEventId_args&); + ThriftHiveMetastore_get_current_notificationEventId_args& operator=(const ThriftHiveMetastore_get_current_notificationEventId_args&); + ThriftHiveMetastore_get_current_notificationEventId_args() { + } + + virtual ~ThriftHiveMetastore_get_current_notificationEventId_args() noexcept; + + bool operator == (const ThriftHiveMetastore_get_current_notificationEventId_args & /* rhs */) const + { + return true; + } + bool operator != (const ThriftHiveMetastore_get_current_notificationEventId_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_current_notificationEventId_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_current_notificationEventId_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_current_notificationEventId_pargs() noexcept; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_current_notificationEventId_result__isset { + _ThriftHiveMetastore_get_current_notificationEventId_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_current_notificationEventId_result__isset; + +class ThriftHiveMetastore_get_current_notificationEventId_result { + public: + + ThriftHiveMetastore_get_current_notificationEventId_result(const ThriftHiveMetastore_get_current_notificationEventId_result&); + ThriftHiveMetastore_get_current_notificationEventId_result& operator=(const ThriftHiveMetastore_get_current_notificationEventId_result&); + ThriftHiveMetastore_get_current_notificationEventId_result() { + } + + virtual ~ThriftHiveMetastore_get_current_notificationEventId_result() noexcept; + CurrentNotificationEventId success; + + _ThriftHiveMetastore_get_current_notificationEventId_result__isset __isset; + + void __set_success(const CurrentNotificationEventId& val); + + bool operator == (const ThriftHiveMetastore_get_current_notificationEventId_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_current_notificationEventId_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_current_notificationEventId_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_current_notificationEventId_presult__isset { + _ThriftHiveMetastore_get_current_notificationEventId_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_current_notificationEventId_presult__isset; + +class ThriftHiveMetastore_get_current_notificationEventId_presult { + public: + + + virtual ~ThriftHiveMetastore_get_current_notificationEventId_presult() noexcept; + CurrentNotificationEventId* success; + + _ThriftHiveMetastore_get_current_notificationEventId_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_notification_events_count_args__isset { + _ThriftHiveMetastore_get_notification_events_count_args__isset() : rqst(false) {} + bool rqst :1; +} _ThriftHiveMetastore_get_notification_events_count_args__isset; + +class ThriftHiveMetastore_get_notification_events_count_args { + public: + + ThriftHiveMetastore_get_notification_events_count_args(const ThriftHiveMetastore_get_notification_events_count_args&); + ThriftHiveMetastore_get_notification_events_count_args& operator=(const ThriftHiveMetastore_get_notification_events_count_args&); + ThriftHiveMetastore_get_notification_events_count_args() { + } + + virtual ~ThriftHiveMetastore_get_notification_events_count_args() noexcept; + NotificationEventsCountRequest rqst; + + _ThriftHiveMetastore_get_notification_events_count_args__isset __isset; + + void __set_rqst(const NotificationEventsCountRequest& val); + + bool operator == (const ThriftHiveMetastore_get_notification_events_count_args & rhs) const + { + if (!(rqst == rhs.rqst)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_notification_events_count_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_notification_events_count_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_notification_events_count_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_notification_events_count_pargs() noexcept; + const NotificationEventsCountRequest* rqst; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_notification_events_count_result__isset { + _ThriftHiveMetastore_get_notification_events_count_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_notification_events_count_result__isset; + +class ThriftHiveMetastore_get_notification_events_count_result { + public: + + ThriftHiveMetastore_get_notification_events_count_result(const ThriftHiveMetastore_get_notification_events_count_result&); + ThriftHiveMetastore_get_notification_events_count_result& operator=(const ThriftHiveMetastore_get_notification_events_count_result&); + ThriftHiveMetastore_get_notification_events_count_result() { + } + + virtual ~ThriftHiveMetastore_get_notification_events_count_result() noexcept; + NotificationEventsCountResponse success; + + _ThriftHiveMetastore_get_notification_events_count_result__isset __isset; + + void __set_success(const NotificationEventsCountResponse& val); + + bool operator == (const ThriftHiveMetastore_get_notification_events_count_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_notification_events_count_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_notification_events_count_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_notification_events_count_presult__isset { + _ThriftHiveMetastore_get_notification_events_count_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_notification_events_count_presult__isset; + +class ThriftHiveMetastore_get_notification_events_count_presult { + public: + + + virtual ~ThriftHiveMetastore_get_notification_events_count_presult() noexcept; + NotificationEventsCountResponse* success; + + _ThriftHiveMetastore_get_notification_events_count_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_fire_listener_event_args__isset { + _ThriftHiveMetastore_fire_listener_event_args__isset() : rqst(false) {} + bool rqst :1; +} _ThriftHiveMetastore_fire_listener_event_args__isset; + +class ThriftHiveMetastore_fire_listener_event_args { + public: + + ThriftHiveMetastore_fire_listener_event_args(const ThriftHiveMetastore_fire_listener_event_args&); + ThriftHiveMetastore_fire_listener_event_args& operator=(const ThriftHiveMetastore_fire_listener_event_args&); + ThriftHiveMetastore_fire_listener_event_args() { + } + + virtual ~ThriftHiveMetastore_fire_listener_event_args() noexcept; + FireEventRequest rqst; + + _ThriftHiveMetastore_fire_listener_event_args__isset __isset; + + void __set_rqst(const FireEventRequest& val); + + bool operator == (const ThriftHiveMetastore_fire_listener_event_args & rhs) const + { + if (!(rqst == rhs.rqst)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_fire_listener_event_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_fire_listener_event_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_fire_listener_event_pargs { + public: + + + virtual ~ThriftHiveMetastore_fire_listener_event_pargs() noexcept; + const FireEventRequest* rqst; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_fire_listener_event_result__isset { + _ThriftHiveMetastore_fire_listener_event_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_fire_listener_event_result__isset; + +class ThriftHiveMetastore_fire_listener_event_result { + public: + + ThriftHiveMetastore_fire_listener_event_result(const ThriftHiveMetastore_fire_listener_event_result&); + ThriftHiveMetastore_fire_listener_event_result& operator=(const ThriftHiveMetastore_fire_listener_event_result&); + ThriftHiveMetastore_fire_listener_event_result() { + } + + virtual ~ThriftHiveMetastore_fire_listener_event_result() noexcept; + FireEventResponse success; + + _ThriftHiveMetastore_fire_listener_event_result__isset __isset; + + void __set_success(const FireEventResponse& val); + + bool operator == (const ThriftHiveMetastore_fire_listener_event_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_fire_listener_event_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_fire_listener_event_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_fire_listener_event_presult__isset { + _ThriftHiveMetastore_fire_listener_event_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_fire_listener_event_presult__isset; + +class ThriftHiveMetastore_fire_listener_event_presult { + public: + + + virtual ~ThriftHiveMetastore_fire_listener_event_presult() noexcept; + FireEventResponse* success; + + _ThriftHiveMetastore_fire_listener_event_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + + +class ThriftHiveMetastore_flushCache_args { + public: + + ThriftHiveMetastore_flushCache_args(const ThriftHiveMetastore_flushCache_args&); + ThriftHiveMetastore_flushCache_args& operator=(const ThriftHiveMetastore_flushCache_args&); + ThriftHiveMetastore_flushCache_args() { + } + + virtual ~ThriftHiveMetastore_flushCache_args() noexcept; + + bool operator == (const ThriftHiveMetastore_flushCache_args & /* rhs */) const + { + return true; + } + bool operator != (const ThriftHiveMetastore_flushCache_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_flushCache_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_flushCache_pargs { + public: + + + virtual ~ThriftHiveMetastore_flushCache_pargs() noexcept; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_flushCache_result { + public: + + ThriftHiveMetastore_flushCache_result(const ThriftHiveMetastore_flushCache_result&); + ThriftHiveMetastore_flushCache_result& operator=(const ThriftHiveMetastore_flushCache_result&); + ThriftHiveMetastore_flushCache_result() { + } + + virtual ~ThriftHiveMetastore_flushCache_result() noexcept; + + bool operator == (const ThriftHiveMetastore_flushCache_result & /* rhs */) const + { + return true; + } + bool operator != (const ThriftHiveMetastore_flushCache_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_flushCache_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_flushCache_presult { + public: + + + virtual ~ThriftHiveMetastore_flushCache_presult() noexcept; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_cm_recycle_args__isset { + _ThriftHiveMetastore_cm_recycle_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_cm_recycle_args__isset; + +class ThriftHiveMetastore_cm_recycle_args { + public: + + ThriftHiveMetastore_cm_recycle_args(const ThriftHiveMetastore_cm_recycle_args&); + ThriftHiveMetastore_cm_recycle_args& operator=(const ThriftHiveMetastore_cm_recycle_args&); + ThriftHiveMetastore_cm_recycle_args() { + } + + virtual ~ThriftHiveMetastore_cm_recycle_args() noexcept; + CmRecycleRequest request; + + _ThriftHiveMetastore_cm_recycle_args__isset __isset; + + void __set_request(const CmRecycleRequest& val); + + bool operator == (const ThriftHiveMetastore_cm_recycle_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_cm_recycle_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_cm_recycle_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_cm_recycle_pargs { + public: + + + virtual ~ThriftHiveMetastore_cm_recycle_pargs() noexcept; + const CmRecycleRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_cm_recycle_result__isset { + _ThriftHiveMetastore_cm_recycle_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_cm_recycle_result__isset; + +class ThriftHiveMetastore_cm_recycle_result { + public: + + ThriftHiveMetastore_cm_recycle_result(const ThriftHiveMetastore_cm_recycle_result&); + ThriftHiveMetastore_cm_recycle_result& operator=(const ThriftHiveMetastore_cm_recycle_result&); + ThriftHiveMetastore_cm_recycle_result() { + } + + virtual ~ThriftHiveMetastore_cm_recycle_result() noexcept; + CmRecycleResponse success; + MetaException o1; + + _ThriftHiveMetastore_cm_recycle_result__isset __isset; + + void __set_success(const CmRecycleResponse& val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_cm_recycle_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_cm_recycle_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_cm_recycle_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_cm_recycle_presult__isset { + _ThriftHiveMetastore_cm_recycle_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_cm_recycle_presult__isset; + +class ThriftHiveMetastore_cm_recycle_presult { + public: + + + virtual ~ThriftHiveMetastore_cm_recycle_presult() noexcept; + CmRecycleResponse* success; + MetaException o1; + + _ThriftHiveMetastore_cm_recycle_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_file_metadata_by_expr_args__isset { + _ThriftHiveMetastore_get_file_metadata_by_expr_args__isset() : req(false) {} + bool req :1; +} _ThriftHiveMetastore_get_file_metadata_by_expr_args__isset; + +class ThriftHiveMetastore_get_file_metadata_by_expr_args { + public: + + ThriftHiveMetastore_get_file_metadata_by_expr_args(const ThriftHiveMetastore_get_file_metadata_by_expr_args&); + ThriftHiveMetastore_get_file_metadata_by_expr_args& operator=(const ThriftHiveMetastore_get_file_metadata_by_expr_args&); + ThriftHiveMetastore_get_file_metadata_by_expr_args() { + } + + virtual ~ThriftHiveMetastore_get_file_metadata_by_expr_args() noexcept; + GetFileMetadataByExprRequest req; + + _ThriftHiveMetastore_get_file_metadata_by_expr_args__isset __isset; + + void __set_req(const GetFileMetadataByExprRequest& val); + + bool operator == (const ThriftHiveMetastore_get_file_metadata_by_expr_args & rhs) const + { + if (!(req == rhs.req)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_file_metadata_by_expr_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_file_metadata_by_expr_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_file_metadata_by_expr_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_file_metadata_by_expr_pargs() noexcept; + const GetFileMetadataByExprRequest* req; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_file_metadata_by_expr_result__isset { + _ThriftHiveMetastore_get_file_metadata_by_expr_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_file_metadata_by_expr_result__isset; + +class ThriftHiveMetastore_get_file_metadata_by_expr_result { + public: + + ThriftHiveMetastore_get_file_metadata_by_expr_result(const ThriftHiveMetastore_get_file_metadata_by_expr_result&); + ThriftHiveMetastore_get_file_metadata_by_expr_result& operator=(const ThriftHiveMetastore_get_file_metadata_by_expr_result&); + ThriftHiveMetastore_get_file_metadata_by_expr_result() { + } + + virtual ~ThriftHiveMetastore_get_file_metadata_by_expr_result() noexcept; + GetFileMetadataByExprResult success; + + _ThriftHiveMetastore_get_file_metadata_by_expr_result__isset __isset; + + void __set_success(const GetFileMetadataByExprResult& val); + + bool operator == (const ThriftHiveMetastore_get_file_metadata_by_expr_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_file_metadata_by_expr_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_file_metadata_by_expr_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_file_metadata_by_expr_presult__isset { + _ThriftHiveMetastore_get_file_metadata_by_expr_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_file_metadata_by_expr_presult__isset; + +class ThriftHiveMetastore_get_file_metadata_by_expr_presult { + public: + + + virtual ~ThriftHiveMetastore_get_file_metadata_by_expr_presult() noexcept; + GetFileMetadataByExprResult* success; + + _ThriftHiveMetastore_get_file_metadata_by_expr_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_file_metadata_args__isset { + _ThriftHiveMetastore_get_file_metadata_args__isset() : req(false) {} + bool req :1; +} _ThriftHiveMetastore_get_file_metadata_args__isset; + +class ThriftHiveMetastore_get_file_metadata_args { + public: + + ThriftHiveMetastore_get_file_metadata_args(const ThriftHiveMetastore_get_file_metadata_args&); + ThriftHiveMetastore_get_file_metadata_args& operator=(const ThriftHiveMetastore_get_file_metadata_args&); + ThriftHiveMetastore_get_file_metadata_args() { + } + + virtual ~ThriftHiveMetastore_get_file_metadata_args() noexcept; + GetFileMetadataRequest req; + + _ThriftHiveMetastore_get_file_metadata_args__isset __isset; + + void __set_req(const GetFileMetadataRequest& val); + + bool operator == (const ThriftHiveMetastore_get_file_metadata_args & rhs) const + { + if (!(req == rhs.req)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_file_metadata_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_file_metadata_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_file_metadata_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_file_metadata_pargs() noexcept; + const GetFileMetadataRequest* req; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_file_metadata_result__isset { + _ThriftHiveMetastore_get_file_metadata_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_file_metadata_result__isset; + +class ThriftHiveMetastore_get_file_metadata_result { + public: + + ThriftHiveMetastore_get_file_metadata_result(const ThriftHiveMetastore_get_file_metadata_result&); + ThriftHiveMetastore_get_file_metadata_result& operator=(const ThriftHiveMetastore_get_file_metadata_result&); + ThriftHiveMetastore_get_file_metadata_result() { + } + + virtual ~ThriftHiveMetastore_get_file_metadata_result() noexcept; + GetFileMetadataResult success; + + _ThriftHiveMetastore_get_file_metadata_result__isset __isset; + + void __set_success(const GetFileMetadataResult& val); + + bool operator == (const ThriftHiveMetastore_get_file_metadata_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_file_metadata_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_file_metadata_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_file_metadata_presult__isset { + _ThriftHiveMetastore_get_file_metadata_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_get_file_metadata_presult__isset; + +class ThriftHiveMetastore_get_file_metadata_presult { + public: + + + virtual ~ThriftHiveMetastore_get_file_metadata_presult() noexcept; + GetFileMetadataResult* success; + + _ThriftHiveMetastore_get_file_metadata_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_put_file_metadata_args__isset { + _ThriftHiveMetastore_put_file_metadata_args__isset() : req(false) {} + bool req :1; +} _ThriftHiveMetastore_put_file_metadata_args__isset; + +class ThriftHiveMetastore_put_file_metadata_args { + public: + + ThriftHiveMetastore_put_file_metadata_args(const ThriftHiveMetastore_put_file_metadata_args&); + ThriftHiveMetastore_put_file_metadata_args& operator=(const ThriftHiveMetastore_put_file_metadata_args&); + ThriftHiveMetastore_put_file_metadata_args() { + } + + virtual ~ThriftHiveMetastore_put_file_metadata_args() noexcept; + PutFileMetadataRequest req; + + _ThriftHiveMetastore_put_file_metadata_args__isset __isset; + + void __set_req(const PutFileMetadataRequest& val); + + bool operator == (const ThriftHiveMetastore_put_file_metadata_args & rhs) const + { + if (!(req == rhs.req)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_put_file_metadata_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_put_file_metadata_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_put_file_metadata_pargs { + public: + + + virtual ~ThriftHiveMetastore_put_file_metadata_pargs() noexcept; + const PutFileMetadataRequest* req; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_put_file_metadata_result__isset { + _ThriftHiveMetastore_put_file_metadata_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_put_file_metadata_result__isset; + +class ThriftHiveMetastore_put_file_metadata_result { + public: + + ThriftHiveMetastore_put_file_metadata_result(const ThriftHiveMetastore_put_file_metadata_result&); + ThriftHiveMetastore_put_file_metadata_result& operator=(const ThriftHiveMetastore_put_file_metadata_result&); + ThriftHiveMetastore_put_file_metadata_result() { + } + + virtual ~ThriftHiveMetastore_put_file_metadata_result() noexcept; + PutFileMetadataResult success; + + _ThriftHiveMetastore_put_file_metadata_result__isset __isset; + + void __set_success(const PutFileMetadataResult& val); + + bool operator == (const ThriftHiveMetastore_put_file_metadata_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_put_file_metadata_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_put_file_metadata_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_put_file_metadata_presult__isset { + _ThriftHiveMetastore_put_file_metadata_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_put_file_metadata_presult__isset; + +class ThriftHiveMetastore_put_file_metadata_presult { + public: + + + virtual ~ThriftHiveMetastore_put_file_metadata_presult() noexcept; + PutFileMetadataResult* success; + + _ThriftHiveMetastore_put_file_metadata_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_clear_file_metadata_args__isset { + _ThriftHiveMetastore_clear_file_metadata_args__isset() : req(false) {} + bool req :1; +} _ThriftHiveMetastore_clear_file_metadata_args__isset; + +class ThriftHiveMetastore_clear_file_metadata_args { + public: + + ThriftHiveMetastore_clear_file_metadata_args(const ThriftHiveMetastore_clear_file_metadata_args&); + ThriftHiveMetastore_clear_file_metadata_args& operator=(const ThriftHiveMetastore_clear_file_metadata_args&); + ThriftHiveMetastore_clear_file_metadata_args() { + } + + virtual ~ThriftHiveMetastore_clear_file_metadata_args() noexcept; + ClearFileMetadataRequest req; + + _ThriftHiveMetastore_clear_file_metadata_args__isset __isset; + + void __set_req(const ClearFileMetadataRequest& val); + + bool operator == (const ThriftHiveMetastore_clear_file_metadata_args & rhs) const + { + if (!(req == rhs.req)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_clear_file_metadata_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_clear_file_metadata_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_clear_file_metadata_pargs { + public: + + + virtual ~ThriftHiveMetastore_clear_file_metadata_pargs() noexcept; + const ClearFileMetadataRequest* req; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_clear_file_metadata_result__isset { + _ThriftHiveMetastore_clear_file_metadata_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_clear_file_metadata_result__isset; + +class ThriftHiveMetastore_clear_file_metadata_result { + public: + + ThriftHiveMetastore_clear_file_metadata_result(const ThriftHiveMetastore_clear_file_metadata_result&); + ThriftHiveMetastore_clear_file_metadata_result& operator=(const ThriftHiveMetastore_clear_file_metadata_result&); + ThriftHiveMetastore_clear_file_metadata_result() { + } + + virtual ~ThriftHiveMetastore_clear_file_metadata_result() noexcept; + ClearFileMetadataResult success; + + _ThriftHiveMetastore_clear_file_metadata_result__isset __isset; + + void __set_success(const ClearFileMetadataResult& val); + + bool operator == (const ThriftHiveMetastore_clear_file_metadata_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_clear_file_metadata_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_clear_file_metadata_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_clear_file_metadata_presult__isset { + _ThriftHiveMetastore_clear_file_metadata_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_clear_file_metadata_presult__isset; + +class ThriftHiveMetastore_clear_file_metadata_presult { + public: + + + virtual ~ThriftHiveMetastore_clear_file_metadata_presult() noexcept; + ClearFileMetadataResult* success; + + _ThriftHiveMetastore_clear_file_metadata_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_cache_file_metadata_args__isset { + _ThriftHiveMetastore_cache_file_metadata_args__isset() : req(false) {} + bool req :1; +} _ThriftHiveMetastore_cache_file_metadata_args__isset; + +class ThriftHiveMetastore_cache_file_metadata_args { + public: + + ThriftHiveMetastore_cache_file_metadata_args(const ThriftHiveMetastore_cache_file_metadata_args&); + ThriftHiveMetastore_cache_file_metadata_args& operator=(const ThriftHiveMetastore_cache_file_metadata_args&); + ThriftHiveMetastore_cache_file_metadata_args() { + } + + virtual ~ThriftHiveMetastore_cache_file_metadata_args() noexcept; + CacheFileMetadataRequest req; + + _ThriftHiveMetastore_cache_file_metadata_args__isset __isset; + + void __set_req(const CacheFileMetadataRequest& val); + + bool operator == (const ThriftHiveMetastore_cache_file_metadata_args & rhs) const + { + if (!(req == rhs.req)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_cache_file_metadata_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_cache_file_metadata_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_cache_file_metadata_pargs { + public: + + + virtual ~ThriftHiveMetastore_cache_file_metadata_pargs() noexcept; + const CacheFileMetadataRequest* req; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_cache_file_metadata_result__isset { + _ThriftHiveMetastore_cache_file_metadata_result__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_cache_file_metadata_result__isset; + +class ThriftHiveMetastore_cache_file_metadata_result { + public: + + ThriftHiveMetastore_cache_file_metadata_result(const ThriftHiveMetastore_cache_file_metadata_result&); + ThriftHiveMetastore_cache_file_metadata_result& operator=(const ThriftHiveMetastore_cache_file_metadata_result&); + ThriftHiveMetastore_cache_file_metadata_result() { + } + + virtual ~ThriftHiveMetastore_cache_file_metadata_result() noexcept; + CacheFileMetadataResult success; + + _ThriftHiveMetastore_cache_file_metadata_result__isset __isset; + + void __set_success(const CacheFileMetadataResult& val); + + bool operator == (const ThriftHiveMetastore_cache_file_metadata_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_cache_file_metadata_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_cache_file_metadata_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_cache_file_metadata_presult__isset { + _ThriftHiveMetastore_cache_file_metadata_presult__isset() : success(false) {} + bool success :1; +} _ThriftHiveMetastore_cache_file_metadata_presult__isset; + +class ThriftHiveMetastore_cache_file_metadata_presult { + public: + + + virtual ~ThriftHiveMetastore_cache_file_metadata_presult() noexcept; + CacheFileMetadataResult* success; + + _ThriftHiveMetastore_cache_file_metadata_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + + +class ThriftHiveMetastore_get_metastore_db_uuid_args { + public: + + ThriftHiveMetastore_get_metastore_db_uuid_args(const ThriftHiveMetastore_get_metastore_db_uuid_args&); + ThriftHiveMetastore_get_metastore_db_uuid_args& operator=(const ThriftHiveMetastore_get_metastore_db_uuid_args&); + ThriftHiveMetastore_get_metastore_db_uuid_args() { + } + + virtual ~ThriftHiveMetastore_get_metastore_db_uuid_args() noexcept; + + bool operator == (const ThriftHiveMetastore_get_metastore_db_uuid_args & /* rhs */) const + { + return true; + } + bool operator != (const ThriftHiveMetastore_get_metastore_db_uuid_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_metastore_db_uuid_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_metastore_db_uuid_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_metastore_db_uuid_pargs() noexcept; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_metastore_db_uuid_result__isset { + _ThriftHiveMetastore_get_metastore_db_uuid_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_metastore_db_uuid_result__isset; + +class ThriftHiveMetastore_get_metastore_db_uuid_result { + public: + + ThriftHiveMetastore_get_metastore_db_uuid_result(const ThriftHiveMetastore_get_metastore_db_uuid_result&); + ThriftHiveMetastore_get_metastore_db_uuid_result& operator=(const ThriftHiveMetastore_get_metastore_db_uuid_result&); + ThriftHiveMetastore_get_metastore_db_uuid_result() : success() { + } + + virtual ~ThriftHiveMetastore_get_metastore_db_uuid_result() noexcept; + std::string success; + MetaException o1; + + _ThriftHiveMetastore_get_metastore_db_uuid_result__isset __isset; + + void __set_success(const std::string& val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_metastore_db_uuid_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_metastore_db_uuid_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_metastore_db_uuid_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_metastore_db_uuid_presult__isset { + _ThriftHiveMetastore_get_metastore_db_uuid_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_metastore_db_uuid_presult__isset; + +class ThriftHiveMetastore_get_metastore_db_uuid_presult { + public: + + + virtual ~ThriftHiveMetastore_get_metastore_db_uuid_presult() noexcept; + std::string* success; + MetaException o1; + + _ThriftHiveMetastore_get_metastore_db_uuid_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_create_resource_plan_args__isset { + _ThriftHiveMetastore_create_resource_plan_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_create_resource_plan_args__isset; + +class ThriftHiveMetastore_create_resource_plan_args { + public: + + ThriftHiveMetastore_create_resource_plan_args(const ThriftHiveMetastore_create_resource_plan_args&); + ThriftHiveMetastore_create_resource_plan_args& operator=(const ThriftHiveMetastore_create_resource_plan_args&); + ThriftHiveMetastore_create_resource_plan_args() { + } + + virtual ~ThriftHiveMetastore_create_resource_plan_args() noexcept; + WMCreateResourcePlanRequest request; + + _ThriftHiveMetastore_create_resource_plan_args__isset __isset; + + void __set_request(const WMCreateResourcePlanRequest& val); + + bool operator == (const ThriftHiveMetastore_create_resource_plan_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_resource_plan_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_resource_plan_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_create_resource_plan_pargs { + public: + + + virtual ~ThriftHiveMetastore_create_resource_plan_pargs() noexcept; + const WMCreateResourcePlanRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_resource_plan_result__isset { + _ThriftHiveMetastore_create_resource_plan_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_create_resource_plan_result__isset; + +class ThriftHiveMetastore_create_resource_plan_result { + public: + + ThriftHiveMetastore_create_resource_plan_result(const ThriftHiveMetastore_create_resource_plan_result&); + ThriftHiveMetastore_create_resource_plan_result& operator=(const ThriftHiveMetastore_create_resource_plan_result&); + ThriftHiveMetastore_create_resource_plan_result() { + } + + virtual ~ThriftHiveMetastore_create_resource_plan_result() noexcept; + WMCreateResourcePlanResponse success; + AlreadyExistsException o1; + InvalidObjectException o2; + MetaException o3; + + _ThriftHiveMetastore_create_resource_plan_result__isset __isset; + + void __set_success(const WMCreateResourcePlanResponse& val); + + void __set_o1(const AlreadyExistsException& val); + + void __set_o2(const InvalidObjectException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_create_resource_plan_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_resource_plan_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_resource_plan_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_resource_plan_presult__isset { + _ThriftHiveMetastore_create_resource_plan_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_create_resource_plan_presult__isset; + +class ThriftHiveMetastore_create_resource_plan_presult { + public: + + + virtual ~ThriftHiveMetastore_create_resource_plan_presult() noexcept; + WMCreateResourcePlanResponse* success; + AlreadyExistsException o1; + InvalidObjectException o2; + MetaException o3; + + _ThriftHiveMetastore_create_resource_plan_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_resource_plan_args__isset { + _ThriftHiveMetastore_get_resource_plan_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_get_resource_plan_args__isset; + +class ThriftHiveMetastore_get_resource_plan_args { + public: + + ThriftHiveMetastore_get_resource_plan_args(const ThriftHiveMetastore_get_resource_plan_args&); + ThriftHiveMetastore_get_resource_plan_args& operator=(const ThriftHiveMetastore_get_resource_plan_args&); + ThriftHiveMetastore_get_resource_plan_args() { + } + + virtual ~ThriftHiveMetastore_get_resource_plan_args() noexcept; + WMGetResourcePlanRequest request; + + _ThriftHiveMetastore_get_resource_plan_args__isset __isset; + + void __set_request(const WMGetResourcePlanRequest& val); + + bool operator == (const ThriftHiveMetastore_get_resource_plan_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_resource_plan_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_resource_plan_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_resource_plan_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_resource_plan_pargs() noexcept; + const WMGetResourcePlanRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_resource_plan_result__isset { + _ThriftHiveMetastore_get_resource_plan_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_resource_plan_result__isset; + +class ThriftHiveMetastore_get_resource_plan_result { + public: + + ThriftHiveMetastore_get_resource_plan_result(const ThriftHiveMetastore_get_resource_plan_result&); + ThriftHiveMetastore_get_resource_plan_result& operator=(const ThriftHiveMetastore_get_resource_plan_result&); + ThriftHiveMetastore_get_resource_plan_result() { + } + + virtual ~ThriftHiveMetastore_get_resource_plan_result() noexcept; + WMGetResourcePlanResponse success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_resource_plan_result__isset __isset; + + void __set_success(const WMGetResourcePlanResponse& val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_resource_plan_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_resource_plan_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_resource_plan_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_resource_plan_presult__isset { + _ThriftHiveMetastore_get_resource_plan_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_resource_plan_presult__isset; + +class ThriftHiveMetastore_get_resource_plan_presult { + public: + + + virtual ~ThriftHiveMetastore_get_resource_plan_presult() noexcept; + WMGetResourcePlanResponse* success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_resource_plan_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_active_resource_plan_args__isset { + _ThriftHiveMetastore_get_active_resource_plan_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_get_active_resource_plan_args__isset; + +class ThriftHiveMetastore_get_active_resource_plan_args { + public: + + ThriftHiveMetastore_get_active_resource_plan_args(const ThriftHiveMetastore_get_active_resource_plan_args&); + ThriftHiveMetastore_get_active_resource_plan_args& operator=(const ThriftHiveMetastore_get_active_resource_plan_args&); + ThriftHiveMetastore_get_active_resource_plan_args() { + } + + virtual ~ThriftHiveMetastore_get_active_resource_plan_args() noexcept; + WMGetActiveResourcePlanRequest request; + + _ThriftHiveMetastore_get_active_resource_plan_args__isset __isset; + + void __set_request(const WMGetActiveResourcePlanRequest& val); + + bool operator == (const ThriftHiveMetastore_get_active_resource_plan_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_active_resource_plan_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_active_resource_plan_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_active_resource_plan_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_active_resource_plan_pargs() noexcept; + const WMGetActiveResourcePlanRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_active_resource_plan_result__isset { + _ThriftHiveMetastore_get_active_resource_plan_result__isset() : success(false), o2(false) {} + bool success :1; + bool o2 :1; +} _ThriftHiveMetastore_get_active_resource_plan_result__isset; + +class ThriftHiveMetastore_get_active_resource_plan_result { + public: + + ThriftHiveMetastore_get_active_resource_plan_result(const ThriftHiveMetastore_get_active_resource_plan_result&); + ThriftHiveMetastore_get_active_resource_plan_result& operator=(const ThriftHiveMetastore_get_active_resource_plan_result&); + ThriftHiveMetastore_get_active_resource_plan_result() { + } + + virtual ~ThriftHiveMetastore_get_active_resource_plan_result() noexcept; + WMGetActiveResourcePlanResponse success; + MetaException o2; + + _ThriftHiveMetastore_get_active_resource_plan_result__isset __isset; + + void __set_success(const WMGetActiveResourcePlanResponse& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_active_resource_plan_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_active_resource_plan_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_active_resource_plan_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_active_resource_plan_presult__isset { + _ThriftHiveMetastore_get_active_resource_plan_presult__isset() : success(false), o2(false) {} + bool success :1; + bool o2 :1; +} _ThriftHiveMetastore_get_active_resource_plan_presult__isset; + +class ThriftHiveMetastore_get_active_resource_plan_presult { + public: + + + virtual ~ThriftHiveMetastore_get_active_resource_plan_presult() noexcept; + WMGetActiveResourcePlanResponse* success; + MetaException o2; + + _ThriftHiveMetastore_get_active_resource_plan_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_all_resource_plans_args__isset { + _ThriftHiveMetastore_get_all_resource_plans_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_get_all_resource_plans_args__isset; + +class ThriftHiveMetastore_get_all_resource_plans_args { + public: + + ThriftHiveMetastore_get_all_resource_plans_args(const ThriftHiveMetastore_get_all_resource_plans_args&); + ThriftHiveMetastore_get_all_resource_plans_args& operator=(const ThriftHiveMetastore_get_all_resource_plans_args&); + ThriftHiveMetastore_get_all_resource_plans_args() { + } + + virtual ~ThriftHiveMetastore_get_all_resource_plans_args() noexcept; + WMGetAllResourcePlanRequest request; + + _ThriftHiveMetastore_get_all_resource_plans_args__isset __isset; + + void __set_request(const WMGetAllResourcePlanRequest& val); + + bool operator == (const ThriftHiveMetastore_get_all_resource_plans_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_all_resource_plans_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_all_resource_plans_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_all_resource_plans_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_all_resource_plans_pargs() noexcept; + const WMGetAllResourcePlanRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_all_resource_plans_result__isset { + _ThriftHiveMetastore_get_all_resource_plans_result__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_all_resource_plans_result__isset; + +class ThriftHiveMetastore_get_all_resource_plans_result { + public: + + ThriftHiveMetastore_get_all_resource_plans_result(const ThriftHiveMetastore_get_all_resource_plans_result&); + ThriftHiveMetastore_get_all_resource_plans_result& operator=(const ThriftHiveMetastore_get_all_resource_plans_result&); + ThriftHiveMetastore_get_all_resource_plans_result() { + } + + virtual ~ThriftHiveMetastore_get_all_resource_plans_result() noexcept; + WMGetAllResourcePlanResponse success; + MetaException o1; + + _ThriftHiveMetastore_get_all_resource_plans_result__isset __isset; + + void __set_success(const WMGetAllResourcePlanResponse& val); + + void __set_o1(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_all_resource_plans_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_all_resource_plans_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_all_resource_plans_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_all_resource_plans_presult__isset { + _ThriftHiveMetastore_get_all_resource_plans_presult__isset() : success(false), o1(false) {} + bool success :1; + bool o1 :1; +} _ThriftHiveMetastore_get_all_resource_plans_presult__isset; + +class ThriftHiveMetastore_get_all_resource_plans_presult { + public: + + + virtual ~ThriftHiveMetastore_get_all_resource_plans_presult() noexcept; + WMGetAllResourcePlanResponse* success; + MetaException o1; + + _ThriftHiveMetastore_get_all_resource_plans_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_alter_resource_plan_args__isset { + _ThriftHiveMetastore_alter_resource_plan_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_alter_resource_plan_args__isset; + +class ThriftHiveMetastore_alter_resource_plan_args { + public: + + ThriftHiveMetastore_alter_resource_plan_args(const ThriftHiveMetastore_alter_resource_plan_args&); + ThriftHiveMetastore_alter_resource_plan_args& operator=(const ThriftHiveMetastore_alter_resource_plan_args&); + ThriftHiveMetastore_alter_resource_plan_args() { + } + + virtual ~ThriftHiveMetastore_alter_resource_plan_args() noexcept; + WMAlterResourcePlanRequest request; + + _ThriftHiveMetastore_alter_resource_plan_args__isset __isset; + + void __set_request(const WMAlterResourcePlanRequest& val); + + bool operator == (const ThriftHiveMetastore_alter_resource_plan_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_resource_plan_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_resource_plan_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_alter_resource_plan_pargs { + public: + + + virtual ~ThriftHiveMetastore_alter_resource_plan_pargs() noexcept; + const WMAlterResourcePlanRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_resource_plan_result__isset { + _ThriftHiveMetastore_alter_resource_plan_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_alter_resource_plan_result__isset; + +class ThriftHiveMetastore_alter_resource_plan_result { + public: + + ThriftHiveMetastore_alter_resource_plan_result(const ThriftHiveMetastore_alter_resource_plan_result&); + ThriftHiveMetastore_alter_resource_plan_result& operator=(const ThriftHiveMetastore_alter_resource_plan_result&); + ThriftHiveMetastore_alter_resource_plan_result() { + } + + virtual ~ThriftHiveMetastore_alter_resource_plan_result() noexcept; + WMAlterResourcePlanResponse success; + NoSuchObjectException o1; + InvalidOperationException o2; + MetaException o3; + + _ThriftHiveMetastore_alter_resource_plan_result__isset __isset; + + void __set_success(const WMAlterResourcePlanResponse& val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const InvalidOperationException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_alter_resource_plan_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_resource_plan_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_resource_plan_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_resource_plan_presult__isset { + _ThriftHiveMetastore_alter_resource_plan_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_alter_resource_plan_presult__isset; + +class ThriftHiveMetastore_alter_resource_plan_presult { + public: + + + virtual ~ThriftHiveMetastore_alter_resource_plan_presult() noexcept; + WMAlterResourcePlanResponse* success; + NoSuchObjectException o1; + InvalidOperationException o2; + MetaException o3; + + _ThriftHiveMetastore_alter_resource_plan_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_validate_resource_plan_args__isset { + _ThriftHiveMetastore_validate_resource_plan_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_validate_resource_plan_args__isset; + +class ThriftHiveMetastore_validate_resource_plan_args { + public: + + ThriftHiveMetastore_validate_resource_plan_args(const ThriftHiveMetastore_validate_resource_plan_args&); + ThriftHiveMetastore_validate_resource_plan_args& operator=(const ThriftHiveMetastore_validate_resource_plan_args&); + ThriftHiveMetastore_validate_resource_plan_args() { + } + + virtual ~ThriftHiveMetastore_validate_resource_plan_args() noexcept; + WMValidateResourcePlanRequest request; + + _ThriftHiveMetastore_validate_resource_plan_args__isset __isset; + + void __set_request(const WMValidateResourcePlanRequest& val); + + bool operator == (const ThriftHiveMetastore_validate_resource_plan_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_validate_resource_plan_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_validate_resource_plan_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_validate_resource_plan_pargs { + public: + + + virtual ~ThriftHiveMetastore_validate_resource_plan_pargs() noexcept; + const WMValidateResourcePlanRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_validate_resource_plan_result__isset { + _ThriftHiveMetastore_validate_resource_plan_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_validate_resource_plan_result__isset; + +class ThriftHiveMetastore_validate_resource_plan_result { + public: + + ThriftHiveMetastore_validate_resource_plan_result(const ThriftHiveMetastore_validate_resource_plan_result&); + ThriftHiveMetastore_validate_resource_plan_result& operator=(const ThriftHiveMetastore_validate_resource_plan_result&); + ThriftHiveMetastore_validate_resource_plan_result() { + } + + virtual ~ThriftHiveMetastore_validate_resource_plan_result() noexcept; + WMValidateResourcePlanResponse success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_validate_resource_plan_result__isset __isset; + + void __set_success(const WMValidateResourcePlanResponse& val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_validate_resource_plan_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_validate_resource_plan_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_validate_resource_plan_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_validate_resource_plan_presult__isset { + _ThriftHiveMetastore_validate_resource_plan_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_validate_resource_plan_presult__isset; + +class ThriftHiveMetastore_validate_resource_plan_presult { + public: + + + virtual ~ThriftHiveMetastore_validate_resource_plan_presult() noexcept; + WMValidateResourcePlanResponse* success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_validate_resource_plan_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_drop_resource_plan_args__isset { + _ThriftHiveMetastore_drop_resource_plan_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_drop_resource_plan_args__isset; + +class ThriftHiveMetastore_drop_resource_plan_args { + public: + + ThriftHiveMetastore_drop_resource_plan_args(const ThriftHiveMetastore_drop_resource_plan_args&); + ThriftHiveMetastore_drop_resource_plan_args& operator=(const ThriftHiveMetastore_drop_resource_plan_args&); + ThriftHiveMetastore_drop_resource_plan_args() { + } + + virtual ~ThriftHiveMetastore_drop_resource_plan_args() noexcept; + WMDropResourcePlanRequest request; + + _ThriftHiveMetastore_drop_resource_plan_args__isset __isset; + + void __set_request(const WMDropResourcePlanRequest& val); + + bool operator == (const ThriftHiveMetastore_drop_resource_plan_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_resource_plan_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_resource_plan_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_drop_resource_plan_pargs { + public: + + + virtual ~ThriftHiveMetastore_drop_resource_plan_pargs() noexcept; + const WMDropResourcePlanRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_resource_plan_result__isset { + _ThriftHiveMetastore_drop_resource_plan_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_drop_resource_plan_result__isset; + +class ThriftHiveMetastore_drop_resource_plan_result { + public: + + ThriftHiveMetastore_drop_resource_plan_result(const ThriftHiveMetastore_drop_resource_plan_result&); + ThriftHiveMetastore_drop_resource_plan_result& operator=(const ThriftHiveMetastore_drop_resource_plan_result&); + ThriftHiveMetastore_drop_resource_plan_result() { + } + + virtual ~ThriftHiveMetastore_drop_resource_plan_result() noexcept; + WMDropResourcePlanResponse success; + NoSuchObjectException o1; + InvalidOperationException o2; + MetaException o3; + + _ThriftHiveMetastore_drop_resource_plan_result__isset __isset; + + void __set_success(const WMDropResourcePlanResponse& val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const InvalidOperationException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_drop_resource_plan_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_resource_plan_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_resource_plan_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_resource_plan_presult__isset { + _ThriftHiveMetastore_drop_resource_plan_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_drop_resource_plan_presult__isset; + +class ThriftHiveMetastore_drop_resource_plan_presult { + public: + + + virtual ~ThriftHiveMetastore_drop_resource_plan_presult() noexcept; + WMDropResourcePlanResponse* success; + NoSuchObjectException o1; + InvalidOperationException o2; + MetaException o3; + + _ThriftHiveMetastore_drop_resource_plan_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_create_wm_trigger_args__isset { + _ThriftHiveMetastore_create_wm_trigger_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_create_wm_trigger_args__isset; + +class ThriftHiveMetastore_create_wm_trigger_args { + public: + + ThriftHiveMetastore_create_wm_trigger_args(const ThriftHiveMetastore_create_wm_trigger_args&); + ThriftHiveMetastore_create_wm_trigger_args& operator=(const ThriftHiveMetastore_create_wm_trigger_args&); + ThriftHiveMetastore_create_wm_trigger_args() { + } + + virtual ~ThriftHiveMetastore_create_wm_trigger_args() noexcept; + WMCreateTriggerRequest request; + + _ThriftHiveMetastore_create_wm_trigger_args__isset __isset; + + void __set_request(const WMCreateTriggerRequest& val); + + bool operator == (const ThriftHiveMetastore_create_wm_trigger_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_wm_trigger_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_wm_trigger_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_create_wm_trigger_pargs { + public: + + + virtual ~ThriftHiveMetastore_create_wm_trigger_pargs() noexcept; + const WMCreateTriggerRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_wm_trigger_result__isset { + _ThriftHiveMetastore_create_wm_trigger_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_create_wm_trigger_result__isset; + +class ThriftHiveMetastore_create_wm_trigger_result { + public: + + ThriftHiveMetastore_create_wm_trigger_result(const ThriftHiveMetastore_create_wm_trigger_result&); + ThriftHiveMetastore_create_wm_trigger_result& operator=(const ThriftHiveMetastore_create_wm_trigger_result&); + ThriftHiveMetastore_create_wm_trigger_result() { + } + + virtual ~ThriftHiveMetastore_create_wm_trigger_result() noexcept; + WMCreateTriggerResponse success; + AlreadyExistsException o1; + NoSuchObjectException o2; + InvalidObjectException o3; + MetaException o4; + + _ThriftHiveMetastore_create_wm_trigger_result__isset __isset; + + void __set_success(const WMCreateTriggerResponse& val); + + void __set_o1(const AlreadyExistsException& val); + + void __set_o2(const NoSuchObjectException& val); + + void __set_o3(const InvalidObjectException& val); + + void __set_o4(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_create_wm_trigger_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + if (!(o4 == rhs.o4)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_create_wm_trigger_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_create_wm_trigger_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_create_wm_trigger_presult__isset { + _ThriftHiveMetastore_create_wm_trigger_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; + bool o4 :1; +} _ThriftHiveMetastore_create_wm_trigger_presult__isset; + +class ThriftHiveMetastore_create_wm_trigger_presult { + public: + + + virtual ~ThriftHiveMetastore_create_wm_trigger_presult() noexcept; + WMCreateTriggerResponse* success; + AlreadyExistsException o1; + NoSuchObjectException o2; + InvalidObjectException o3; + MetaException o4; + + _ThriftHiveMetastore_create_wm_trigger_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_alter_wm_trigger_args__isset { + _ThriftHiveMetastore_alter_wm_trigger_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_alter_wm_trigger_args__isset; + +class ThriftHiveMetastore_alter_wm_trigger_args { + public: + + ThriftHiveMetastore_alter_wm_trigger_args(const ThriftHiveMetastore_alter_wm_trigger_args&); + ThriftHiveMetastore_alter_wm_trigger_args& operator=(const ThriftHiveMetastore_alter_wm_trigger_args&); + ThriftHiveMetastore_alter_wm_trigger_args() { + } + + virtual ~ThriftHiveMetastore_alter_wm_trigger_args() noexcept; + WMAlterTriggerRequest request; + + _ThriftHiveMetastore_alter_wm_trigger_args__isset __isset; + + void __set_request(const WMAlterTriggerRequest& val); + + bool operator == (const ThriftHiveMetastore_alter_wm_trigger_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_wm_trigger_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_wm_trigger_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_alter_wm_trigger_pargs { + public: + + + virtual ~ThriftHiveMetastore_alter_wm_trigger_pargs() noexcept; + const WMAlterTriggerRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_wm_trigger_result__isset { + _ThriftHiveMetastore_alter_wm_trigger_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_alter_wm_trigger_result__isset; + +class ThriftHiveMetastore_alter_wm_trigger_result { + public: + + ThriftHiveMetastore_alter_wm_trigger_result(const ThriftHiveMetastore_alter_wm_trigger_result&); + ThriftHiveMetastore_alter_wm_trigger_result& operator=(const ThriftHiveMetastore_alter_wm_trigger_result&); + ThriftHiveMetastore_alter_wm_trigger_result() { + } + + virtual ~ThriftHiveMetastore_alter_wm_trigger_result() noexcept; + WMAlterTriggerResponse success; + NoSuchObjectException o1; + InvalidObjectException o2; + MetaException o3; + + _ThriftHiveMetastore_alter_wm_trigger_result__isset __isset; + + void __set_success(const WMAlterTriggerResponse& val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const InvalidObjectException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_alter_wm_trigger_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_alter_wm_trigger_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_alter_wm_trigger_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_alter_wm_trigger_presult__isset { + _ThriftHiveMetastore_alter_wm_trigger_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_alter_wm_trigger_presult__isset; + +class ThriftHiveMetastore_alter_wm_trigger_presult { + public: + + + virtual ~ThriftHiveMetastore_alter_wm_trigger_presult() noexcept; + WMAlterTriggerResponse* success; + NoSuchObjectException o1; + InvalidObjectException o2; + MetaException o3; + + _ThriftHiveMetastore_alter_wm_trigger_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_drop_wm_trigger_args__isset { + _ThriftHiveMetastore_drop_wm_trigger_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_drop_wm_trigger_args__isset; + +class ThriftHiveMetastore_drop_wm_trigger_args { + public: + + ThriftHiveMetastore_drop_wm_trigger_args(const ThriftHiveMetastore_drop_wm_trigger_args&); + ThriftHiveMetastore_drop_wm_trigger_args& operator=(const ThriftHiveMetastore_drop_wm_trigger_args&); + ThriftHiveMetastore_drop_wm_trigger_args() { + } + + virtual ~ThriftHiveMetastore_drop_wm_trigger_args() noexcept; + WMDropTriggerRequest request; + + _ThriftHiveMetastore_drop_wm_trigger_args__isset __isset; + + void __set_request(const WMDropTriggerRequest& val); + + bool operator == (const ThriftHiveMetastore_drop_wm_trigger_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_wm_trigger_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_wm_trigger_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_drop_wm_trigger_pargs { + public: + + + virtual ~ThriftHiveMetastore_drop_wm_trigger_pargs() noexcept; + const WMDropTriggerRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_wm_trigger_result__isset { + _ThriftHiveMetastore_drop_wm_trigger_result__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_drop_wm_trigger_result__isset; + +class ThriftHiveMetastore_drop_wm_trigger_result { + public: + + ThriftHiveMetastore_drop_wm_trigger_result(const ThriftHiveMetastore_drop_wm_trigger_result&); + ThriftHiveMetastore_drop_wm_trigger_result& operator=(const ThriftHiveMetastore_drop_wm_trigger_result&); + ThriftHiveMetastore_drop_wm_trigger_result() { + } + + virtual ~ThriftHiveMetastore_drop_wm_trigger_result() noexcept; + WMDropTriggerResponse success; + NoSuchObjectException o1; + InvalidOperationException o2; + MetaException o3; + + _ThriftHiveMetastore_drop_wm_trigger_result__isset __isset; + + void __set_success(const WMDropTriggerResponse& val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const InvalidOperationException& val); + + void __set_o3(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_drop_wm_trigger_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + if (!(o3 == rhs.o3)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_drop_wm_trigger_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_drop_wm_trigger_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_drop_wm_trigger_presult__isset { + _ThriftHiveMetastore_drop_wm_trigger_presult__isset() : success(false), o1(false), o2(false), o3(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; + bool o3 :1; +} _ThriftHiveMetastore_drop_wm_trigger_presult__isset; + +class ThriftHiveMetastore_drop_wm_trigger_presult { + public: + + + virtual ~ThriftHiveMetastore_drop_wm_trigger_presult() noexcept; + WMDropTriggerResponse* success; + NoSuchObjectException o1; + InvalidOperationException o2; + MetaException o3; + + _ThriftHiveMetastore_drop_wm_trigger_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_get_triggers_for_resourceplan_args__isset { + _ThriftHiveMetastore_get_triggers_for_resourceplan_args__isset() : request(false) {} + bool request :1; +} _ThriftHiveMetastore_get_triggers_for_resourceplan_args__isset; + +class ThriftHiveMetastore_get_triggers_for_resourceplan_args { + public: + + ThriftHiveMetastore_get_triggers_for_resourceplan_args(const ThriftHiveMetastore_get_triggers_for_resourceplan_args&); + ThriftHiveMetastore_get_triggers_for_resourceplan_args& operator=(const ThriftHiveMetastore_get_triggers_for_resourceplan_args&); + ThriftHiveMetastore_get_triggers_for_resourceplan_args() { + } + + virtual ~ThriftHiveMetastore_get_triggers_for_resourceplan_args() noexcept; + WMGetTriggersForResourePlanRequest request; + + _ThriftHiveMetastore_get_triggers_for_resourceplan_args__isset __isset; + + void __set_request(const WMGetTriggersForResourePlanRequest& val); + + bool operator == (const ThriftHiveMetastore_get_triggers_for_resourceplan_args & rhs) const + { + if (!(request == rhs.request)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_triggers_for_resourceplan_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_triggers_for_resourceplan_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_get_triggers_for_resourceplan_pargs { + public: + + + virtual ~ThriftHiveMetastore_get_triggers_for_resourceplan_pargs() noexcept; + const WMGetTriggersForResourePlanRequest* request; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_triggers_for_resourceplan_result__isset { + _ThriftHiveMetastore_get_triggers_for_resourceplan_result__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_triggers_for_resourceplan_result__isset; + +class ThriftHiveMetastore_get_triggers_for_resourceplan_result { + public: + + ThriftHiveMetastore_get_triggers_for_resourceplan_result(const ThriftHiveMetastore_get_triggers_for_resourceplan_result&); + ThriftHiveMetastore_get_triggers_for_resourceplan_result& operator=(const ThriftHiveMetastore_get_triggers_for_resourceplan_result&); + ThriftHiveMetastore_get_triggers_for_resourceplan_result() { + } + + virtual ~ThriftHiveMetastore_get_triggers_for_resourceplan_result() noexcept; + WMGetTriggersForResourePlanResponse success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_triggers_for_resourceplan_result__isset __isset; + + void __set_success(const WMGetTriggersForResourePlanResponse& val); + + void __set_o1(const NoSuchObjectException& val); + + void __set_o2(const MetaException& val); + + bool operator == (const ThriftHiveMetastore_get_triggers_for_resourceplan_result & rhs) const + { + if (!(success == rhs.success)) + return false; + if (!(o1 == rhs.o1)) + return false; + if (!(o2 == rhs.o2)) + return false; + return true; + } + bool operator != (const ThriftHiveMetastore_get_triggers_for_resourceplan_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ThriftHiveMetastore_get_triggers_for_resourceplan_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _ThriftHiveMetastore_get_triggers_for_resourceplan_presult__isset { + _ThriftHiveMetastore_get_triggers_for_resourceplan_presult__isset() : success(false), o1(false), o2(false) {} + bool success :1; + bool o1 :1; + bool o2 :1; +} _ThriftHiveMetastore_get_triggers_for_resourceplan_presult__isset; + +class ThriftHiveMetastore_get_triggers_for_resourceplan_presult { + public: + + + virtual ~ThriftHiveMetastore_get_triggers_for_resourceplan_presult() noexcept; + WMGetTriggersForResourePlanResponse* success; + NoSuchObjectException o1; + MetaException o2; + + _ThriftHiveMetastore_get_triggers_for_resourceplan_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf { + public: + ThriftHiveMetastoreClient(std::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) { + setProtocol(prot); + } + ThriftHiveMetastoreClient(std::shared_ptr< ::apache::thrift::protocol::TProtocol> iprot, std::shared_ptr< ::apache::thrift::protocol::TProtocol> oprot) { + setProtocol(iprot,oprot); + } + private: + void setProtocol(std::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) { + setProtocol(prot,prot); + } + void setProtocol(std::shared_ptr< ::apache::thrift::protocol::TProtocol> iprot, std::shared_ptr< ::apache::thrift::protocol::TProtocol> oprot) { + piprot_=iprot; + poprot_=oprot; + iprot_ = iprot.get(); + oprot_ = oprot.get(); + } + public: + std::shared_ptr< ::apache::thrift::protocol::TProtocol> getInputProtocol() { + return piprot_; + } + std::shared_ptr< ::apache::thrift::protocol::TProtocol> getOutputProtocol() { + return poprot_; + } + void getMetaConf(std::string& _return, const std::string& key); + void send_getMetaConf(const std::string& key); + void recv_getMetaConf(std::string& _return); + void setMetaConf(const std::string& key, const std::string& value); + void send_setMetaConf(const std::string& key, const std::string& value); + void recv_setMetaConf(); + void create_database(const Database& database); + void send_create_database(const Database& database); + void recv_create_database(); + void get_database(Database& _return, const std::string& name); + void send_get_database(const std::string& name); + void recv_get_database(Database& _return); + void drop_database(const std::string& name, const bool deleteData, const bool cascade); + void send_drop_database(const std::string& name, const bool deleteData, const bool cascade); + void recv_drop_database(); + void get_databases(std::vector & _return, const std::string& pattern); + void send_get_databases(const std::string& pattern); + void recv_get_databases(std::vector & _return); + void get_all_databases(std::vector & _return); + void send_get_all_databases(); + void recv_get_all_databases(std::vector & _return); + void alter_database(const std::string& dbname, const Database& db); + void send_alter_database(const std::string& dbname, const Database& db); + void recv_alter_database(); + void get_type(Type& _return, const std::string& name); + void send_get_type(const std::string& name); + void recv_get_type(Type& _return); + bool create_type(const Type& type); + void send_create_type(const Type& type); + bool recv_create_type(); + bool drop_type(const std::string& type); + void send_drop_type(const std::string& type); + bool recv_drop_type(); + void get_type_all(std::map & _return, const std::string& name); + void send_get_type_all(const std::string& name); + void recv_get_type_all(std::map & _return); + void get_fields(std::vector & _return, const std::string& db_name, const std::string& table_name); + void send_get_fields(const std::string& db_name, const std::string& table_name); + void recv_get_fields(std::vector & _return); + void get_fields_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context); + void send_get_fields_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context); + void recv_get_fields_with_environment_context(std::vector & _return); + void get_schema(std::vector & _return, const std::string& db_name, const std::string& table_name); + void send_get_schema(const std::string& db_name, const std::string& table_name); + void recv_get_schema(std::vector & _return); + void get_schema_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context); + void send_get_schema_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context); + void recv_get_schema_with_environment_context(std::vector & _return); + void create_table(const Table& tbl); + void send_create_table(const Table& tbl); + void recv_create_table(); + void create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context); + void send_create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context); + void recv_create_table_with_environment_context(); + void create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints); + void send_create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints); + void recv_create_table_with_constraints(); + void drop_constraint(const DropConstraintRequest& req); + void send_drop_constraint(const DropConstraintRequest& req); + void recv_drop_constraint(); + void add_primary_key(const AddPrimaryKeyRequest& req); + void send_add_primary_key(const AddPrimaryKeyRequest& req); + void recv_add_primary_key(); + void add_foreign_key(const AddForeignKeyRequest& req); + void send_add_foreign_key(const AddForeignKeyRequest& req); + void recv_add_foreign_key(); + void add_unique_constraint(const AddUniqueConstraintRequest& req); + void send_add_unique_constraint(const AddUniqueConstraintRequest& req); + void recv_add_unique_constraint(); + void add_not_null_constraint(const AddNotNullConstraintRequest& req); + void send_add_not_null_constraint(const AddNotNullConstraintRequest& req); + void recv_add_not_null_constraint(); + void drop_table(const std::string& dbname, const std::string& name, const bool deleteData); + void send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData); + void recv_drop_table(); + void drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context); + void send_drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context); + void recv_drop_table_with_environment_context(); + void truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames); + void send_truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames); + void recv_truncate_table(); + void get_tables(std::vector & _return, const std::string& db_name, const std::string& pattern); + void send_get_tables(const std::string& db_name, const std::string& pattern); + void recv_get_tables(std::vector & _return); + void get_tables_by_type(std::vector & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType); + void send_get_tables_by_type(const std::string& db_name, const std::string& pattern, const std::string& tableType); + void recv_get_tables_by_type(std::vector & _return); + void get_table_meta(std::vector & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types); + void send_get_table_meta(const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types); + void recv_get_table_meta(std::vector & _return); + void get_all_tables(std::vector & _return, const std::string& db_name); + void send_get_all_tables(const std::string& db_name); + void recv_get_all_tables(std::vector & _return); + void get_table(Table& _return, const std::string& dbname, const std::string& tbl_name); + void send_get_table(const std::string& dbname, const std::string& tbl_name); + void recv_get_table(Table& _return); + void get_table_objects_by_name(std::vector
& _return, const std::string& dbname, const std::vector & tbl_names); + void send_get_table_objects_by_name(const std::string& dbname, const std::vector & tbl_names); + void recv_get_table_objects_by_name(std::vector
& _return); + void get_table_req(GetTableResult& _return, const GetTableRequest& req); + void send_get_table_req(const GetTableRequest& req); + void recv_get_table_req(GetTableResult& _return); + void get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req); + void send_get_table_objects_by_name_req(const GetTablesRequest& req); + void recv_get_table_objects_by_name_req(GetTablesResult& _return); + void get_table_names_by_filter(std::vector & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables); + void send_get_table_names_by_filter(const std::string& dbname, const std::string& filter, const int16_t max_tables); + void recv_get_table_names_by_filter(std::vector & _return); + void alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl); + void send_alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl); + void recv_alter_table(); + void alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context); + void send_alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context); + void recv_alter_table_with_environment_context(); + void alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade); + void send_alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade); + void recv_alter_table_with_cascade(); + void add_partition(Partition& _return, const Partition& new_part); + void send_add_partition(const Partition& new_part); + void recv_add_partition(Partition& _return); + void add_partition_with_environment_context(Partition& _return, const Partition& new_part, const EnvironmentContext& environment_context); + void send_add_partition_with_environment_context(const Partition& new_part, const EnvironmentContext& environment_context); + void recv_add_partition_with_environment_context(Partition& _return); + int32_t add_partitions(const std::vector & new_parts); + void send_add_partitions(const std::vector & new_parts); + int32_t recv_add_partitions(); + int32_t add_partitions_pspec(const std::vector & new_parts); + void send_add_partitions_pspec(const std::vector & new_parts); + int32_t recv_add_partitions_pspec(); + void append_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals); + void send_append_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals); + void recv_append_partition(Partition& _return); + void add_partitions_req(AddPartitionsResult& _return, const AddPartitionsRequest& request); + void send_add_partitions_req(const AddPartitionsRequest& request); + void recv_add_partitions_req(AddPartitionsResult& _return); + void append_partition_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context); + void send_append_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context); + void recv_append_partition_with_environment_context(Partition& _return); + void append_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name); + void send_append_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name); + void recv_append_partition_by_name(Partition& _return); + void append_partition_by_name_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context); + void send_append_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context); + void recv_append_partition_by_name_with_environment_context(Partition& _return); + bool drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData); + void send_drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData); + bool recv_drop_partition(); + bool drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context); + void send_drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context); + bool recv_drop_partition_with_environment_context(); + bool drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData); + void send_drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData); + bool recv_drop_partition_by_name(); + bool drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context); + void send_drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context); + bool recv_drop_partition_by_name_with_environment_context(); + void drop_partitions_req(DropPartitionsResult& _return, const DropPartitionsRequest& req); + void send_drop_partitions_req(const DropPartitionsRequest& req); + void recv_drop_partitions_req(DropPartitionsResult& _return); + void get_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals); + void send_get_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals); + void recv_get_partition(Partition& _return); + void exchange_partition(Partition& _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name); + void send_exchange_partition(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name); + void recv_exchange_partition(Partition& _return); + void exchange_partitions(std::vector & _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name); + void send_exchange_partitions(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name); + void recv_exchange_partitions(std::vector & _return); + void get_partition_with_auth(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names); + void send_get_partition_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names); + void recv_get_partition_with_auth(Partition& _return); + void get_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name); + void send_get_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name); + void recv_get_partition_by_name(Partition& _return); + void get_partitions(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts); + void send_get_partitions(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts); + void recv_get_partitions(std::vector & _return); + void get_partitions_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names); + void send_get_partitions_with_auth(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names); + void recv_get_partitions_with_auth(std::vector & _return); + void get_partitions_pspec(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int32_t max_parts); + void send_get_partitions_pspec(const std::string& db_name, const std::string& tbl_name, const int32_t max_parts); + void recv_get_partitions_pspec(std::vector & _return); + void get_partition_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts); + void send_get_partition_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts); + void recv_get_partition_names(std::vector & _return); + void get_partition_values(PartitionValuesResponse& _return, const PartitionValuesRequest& request); + void send_get_partition_values(const PartitionValuesRequest& request); + void recv_get_partition_values(PartitionValuesResponse& _return); + void get_partitions_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts); + void send_get_partitions_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts); + void recv_get_partitions_ps(std::vector & _return); + void get_partitions_ps_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names); + void send_get_partitions_ps_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names); + void recv_get_partitions_ps_with_auth(std::vector & _return); + void get_partition_names_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts); + void send_get_partition_names_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts); + void recv_get_partition_names_ps(std::vector & _return); + void get_partitions_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts); + void send_get_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts); + void recv_get_partitions_by_filter(std::vector & _return); + void get_part_specs_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts); + void send_get_part_specs_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts); + void recv_get_part_specs_by_filter(std::vector & _return); + void get_partitions_by_expr(PartitionsByExprResult& _return, const PartitionsByExprRequest& req); + void send_get_partitions_by_expr(const PartitionsByExprRequest& req); + void recv_get_partitions_by_expr(PartitionsByExprResult& _return); + int32_t get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter); + void send_get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter); + int32_t recv_get_num_partitions_by_filter(); + void get_partitions_by_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & names); + void send_get_partitions_by_names(const std::string& db_name, const std::string& tbl_name, const std::vector & names); + void recv_get_partitions_by_names(std::vector & _return); + void alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part); + void send_alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part); + void recv_alter_partition(); + void alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts); + void send_alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts); + void recv_alter_partitions(); + void alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context); + void send_alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context); + void recv_alter_partitions_with_environment_context(); + void alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context); + void send_alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context); + void recv_alter_partition_with_environment_context(); + void rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part); + void send_rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part); + void recv_rename_partition(); + bool partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception); + void send_partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception); + bool recv_partition_name_has_valid_characters(); + void get_config_value(std::string& _return, const std::string& name, const std::string& defaultValue); + void send_get_config_value(const std::string& name, const std::string& defaultValue); + void recv_get_config_value(std::string& _return); + void partition_name_to_vals(std::vector & _return, const std::string& part_name); + void send_partition_name_to_vals(const std::string& part_name); + void recv_partition_name_to_vals(std::vector & _return); + void partition_name_to_spec(std::map & _return, const std::string& part_name); + void send_partition_name_to_spec(const std::string& part_name); + void recv_partition_name_to_spec(std::map & _return); + void markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType); + void send_markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType); + void recv_markPartitionForEvent(); + bool isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType); + void send_isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType); + bool recv_isPartitionMarkedForEvent(); + void add_index(Index& _return, const Index& new_index, const Table& index_table); + void send_add_index(const Index& new_index, const Table& index_table); + void recv_add_index(Index& _return); + void alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx); + void send_alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx); + void recv_alter_index(); + bool drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData); + void send_drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData); + bool recv_drop_index_by_name(); + void get_index_by_name(Index& _return, const std::string& db_name, const std::string& tbl_name, const std::string& index_name); + void send_get_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name); + void recv_get_index_by_name(Index& _return); + void get_indexes(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes); + void send_get_indexes(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes); + void recv_get_indexes(std::vector & _return); + void get_index_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes); + void send_get_index_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes); + void recv_get_index_names(std::vector & _return); + void get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request); + void send_get_primary_keys(const PrimaryKeysRequest& request); + void recv_get_primary_keys(PrimaryKeysResponse& _return); + void get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request); + void send_get_foreign_keys(const ForeignKeysRequest& request); + void recv_get_foreign_keys(ForeignKeysResponse& _return); + void get_unique_constraints(UniqueConstraintsResponse& _return, const UniqueConstraintsRequest& request); + void send_get_unique_constraints(const UniqueConstraintsRequest& request); + void recv_get_unique_constraints(UniqueConstraintsResponse& _return); + void get_not_null_constraints(NotNullConstraintsResponse& _return, const NotNullConstraintsRequest& request); + void send_get_not_null_constraints(const NotNullConstraintsRequest& request); + void recv_get_not_null_constraints(NotNullConstraintsResponse& _return); + bool update_table_column_statistics(const ColumnStatistics& stats_obj); + void send_update_table_column_statistics(const ColumnStatistics& stats_obj); + bool recv_update_table_column_statistics(); + bool update_partition_column_statistics(const ColumnStatistics& stats_obj); + void send_update_partition_column_statistics(const ColumnStatistics& stats_obj); + bool recv_update_partition_column_statistics(); + void get_table_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& col_name); + void send_get_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name); + void recv_get_table_column_statistics(ColumnStatistics& _return); + void get_partition_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name); + void send_get_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name); + void recv_get_partition_column_statistics(ColumnStatistics& _return); + void get_table_statistics_req(TableStatsResult& _return, const TableStatsRequest& request); + void send_get_table_statistics_req(const TableStatsRequest& request); + void recv_get_table_statistics_req(TableStatsResult& _return); + void get_partitions_statistics_req(PartitionsStatsResult& _return, const PartitionsStatsRequest& request); + void send_get_partitions_statistics_req(const PartitionsStatsRequest& request); + void recv_get_partitions_statistics_req(PartitionsStatsResult& _return); + void get_aggr_stats_for(AggrStats& _return, const PartitionsStatsRequest& request); + void send_get_aggr_stats_for(const PartitionsStatsRequest& request); + void recv_get_aggr_stats_for(AggrStats& _return); + bool set_aggr_stats_for(const SetPartitionsStatsRequest& request); + void send_set_aggr_stats_for(const SetPartitionsStatsRequest& request); + bool recv_set_aggr_stats_for(); + bool delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name); + void send_delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name); + bool recv_delete_partition_column_statistics(); + bool delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name); + void send_delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name); + bool recv_delete_table_column_statistics(); + void create_function(const Function& func); + void send_create_function(const Function& func); + void recv_create_function(); + void drop_function(const std::string& dbName, const std::string& funcName); + void send_drop_function(const std::string& dbName, const std::string& funcName); + void recv_drop_function(); + void alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc); + void send_alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc); + void recv_alter_function(); + void get_functions(std::vector & _return, const std::string& dbName, const std::string& pattern); + void send_get_functions(const std::string& dbName, const std::string& pattern); + void recv_get_functions(std::vector & _return); + void get_function(Function& _return, const std::string& dbName, const std::string& funcName); + void send_get_function(const std::string& dbName, const std::string& funcName); + void recv_get_function(Function& _return); + void get_all_functions(GetAllFunctionsResponse& _return); + void send_get_all_functions(); + void recv_get_all_functions(GetAllFunctionsResponse& _return); + bool create_role(const Role& role); + void send_create_role(const Role& role); + bool recv_create_role(); + bool drop_role(const std::string& role_name); + void send_drop_role(const std::string& role_name); + bool recv_drop_role(); + void get_role_names(std::vector & _return); + void send_get_role_names(); + void recv_get_role_names(std::vector & _return); + bool grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option); + void send_grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option); + bool recv_grant_role(); + bool revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type); + void send_revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type); + bool recv_revoke_role(); + void list_roles(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type); + void send_list_roles(const std::string& principal_name, const PrincipalType::type principal_type); + void recv_list_roles(std::vector & _return); + void grant_revoke_role(GrantRevokeRoleResponse& _return, const GrantRevokeRoleRequest& request); + void send_grant_revoke_role(const GrantRevokeRoleRequest& request); + void recv_grant_revoke_role(GrantRevokeRoleResponse& _return); + void get_principals_in_role(GetPrincipalsInRoleResponse& _return, const GetPrincipalsInRoleRequest& request); + void send_get_principals_in_role(const GetPrincipalsInRoleRequest& request); + void recv_get_principals_in_role(GetPrincipalsInRoleResponse& _return); + void get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return, const GetRoleGrantsForPrincipalRequest& request); + void send_get_role_grants_for_principal(const GetRoleGrantsForPrincipalRequest& request); + void recv_get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return); + void get_privilege_set(PrincipalPrivilegeSet& _return, const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names); + void send_get_privilege_set(const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names); + void recv_get_privilege_set(PrincipalPrivilegeSet& _return); + void list_privileges(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject); + void send_list_privileges(const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject); + void recv_list_privileges(std::vector & _return); + bool grant_privileges(const PrivilegeBag& privileges); + void send_grant_privileges(const PrivilegeBag& privileges); + bool recv_grant_privileges(); + bool revoke_privileges(const PrivilegeBag& privileges); + void send_revoke_privileges(const PrivilegeBag& privileges); + bool recv_revoke_privileges(); + void grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const GrantRevokePrivilegeRequest& request); + void send_grant_revoke_privileges(const GrantRevokePrivilegeRequest& request); + void recv_grant_revoke_privileges(GrantRevokePrivilegeResponse& _return); + void set_ugi(std::vector & _return, const std::string& user_name, const std::vector & group_names); + void send_set_ugi(const std::string& user_name, const std::vector & group_names); + void recv_set_ugi(std::vector & _return); + void get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name); + void send_get_delegation_token(const std::string& token_owner, const std::string& renewer_kerberos_principal_name); + void recv_get_delegation_token(std::string& _return); + int64_t renew_delegation_token(const std::string& token_str_form); + void send_renew_delegation_token(const std::string& token_str_form); + int64_t recv_renew_delegation_token(); + void cancel_delegation_token(const std::string& token_str_form); + void send_cancel_delegation_token(const std::string& token_str_form); + void recv_cancel_delegation_token(); + bool add_token(const std::string& token_identifier, const std::string& delegation_token); + void send_add_token(const std::string& token_identifier, const std::string& delegation_token); + bool recv_add_token(); + bool remove_token(const std::string& token_identifier); + void send_remove_token(const std::string& token_identifier); + bool recv_remove_token(); + void get_token(std::string& _return, const std::string& token_identifier); + void send_get_token(const std::string& token_identifier); + void recv_get_token(std::string& _return); + void get_all_token_identifiers(std::vector & _return); + void send_get_all_token_identifiers(); + void recv_get_all_token_identifiers(std::vector & _return); + int32_t add_master_key(const std::string& key); + void send_add_master_key(const std::string& key); + int32_t recv_add_master_key(); + void update_master_key(const int32_t seq_number, const std::string& key); + void send_update_master_key(const int32_t seq_number, const std::string& key); + void recv_update_master_key(); + bool remove_master_key(const int32_t key_seq); + void send_remove_master_key(const int32_t key_seq); + bool recv_remove_master_key(); + void get_master_keys(std::vector & _return); + void send_get_master_keys(); + void recv_get_master_keys(std::vector & _return); + void get_open_txns(GetOpenTxnsResponse& _return); + void send_get_open_txns(); + void recv_get_open_txns(GetOpenTxnsResponse& _return); + void get_open_txns_info(GetOpenTxnsInfoResponse& _return); + void send_get_open_txns_info(); + void recv_get_open_txns_info(GetOpenTxnsInfoResponse& _return); + void open_txns(OpenTxnsResponse& _return, const OpenTxnRequest& rqst); + void send_open_txns(const OpenTxnRequest& rqst); + void recv_open_txns(OpenTxnsResponse& _return); + void abort_txn(const AbortTxnRequest& rqst); + void send_abort_txn(const AbortTxnRequest& rqst); + void recv_abort_txn(); + void abort_txns(const AbortTxnsRequest& rqst); + void send_abort_txns(const AbortTxnsRequest& rqst); + void recv_abort_txns(); + void commit_txn(const CommitTxnRequest& rqst); + void send_commit_txn(const CommitTxnRequest& rqst); + void recv_commit_txn(); + void lock(LockResponse& _return, const LockRequest& rqst); + void send_lock(const LockRequest& rqst); + void recv_lock(LockResponse& _return); + void check_lock(LockResponse& _return, const CheckLockRequest& rqst); + void send_check_lock(const CheckLockRequest& rqst); + void recv_check_lock(LockResponse& _return); + void unlock(const UnlockRequest& rqst); + void send_unlock(const UnlockRequest& rqst); + void recv_unlock(); + void show_locks(ShowLocksResponse& _return, const ShowLocksRequest& rqst); + void send_show_locks(const ShowLocksRequest& rqst); + void recv_show_locks(ShowLocksResponse& _return); + void heartbeat(const HeartbeatRequest& ids); + void send_heartbeat(const HeartbeatRequest& ids); + void recv_heartbeat(); + void heartbeat_txn_range(HeartbeatTxnRangeResponse& _return, const HeartbeatTxnRangeRequest& txns); + void send_heartbeat_txn_range(const HeartbeatTxnRangeRequest& txns); + void recv_heartbeat_txn_range(HeartbeatTxnRangeResponse& _return); + void compact(const CompactionRequest& rqst); + void send_compact(const CompactionRequest& rqst); + void recv_compact(); + void compact2(CompactionResponse& _return, const CompactionRequest& rqst); + void send_compact2(const CompactionRequest& rqst); + void recv_compact2(CompactionResponse& _return); + void show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst); + void send_show_compact(const ShowCompactRequest& rqst); + void recv_show_compact(ShowCompactResponse& _return); + void add_dynamic_partitions(const AddDynamicPartitions& rqst); + void send_add_dynamic_partitions(const AddDynamicPartitions& rqst); + void recv_add_dynamic_partitions(); + void get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst); + void send_get_next_notification(const NotificationEventRequest& rqst); + void recv_get_next_notification(NotificationEventResponse& _return); + void get_current_notificationEventId(CurrentNotificationEventId& _return); + void send_get_current_notificationEventId(); + void recv_get_current_notificationEventId(CurrentNotificationEventId& _return); + void get_notification_events_count(NotificationEventsCountResponse& _return, const NotificationEventsCountRequest& rqst); + void send_get_notification_events_count(const NotificationEventsCountRequest& rqst); + void recv_get_notification_events_count(NotificationEventsCountResponse& _return); + void fire_listener_event(FireEventResponse& _return, const FireEventRequest& rqst); + void send_fire_listener_event(const FireEventRequest& rqst); + void recv_fire_listener_event(FireEventResponse& _return); + void flushCache(); + void send_flushCache(); + void recv_flushCache(); + void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request); + void send_cm_recycle(const CmRecycleRequest& request); + void recv_cm_recycle(CmRecycleResponse& _return); + void get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req); + void send_get_file_metadata_by_expr(const GetFileMetadataByExprRequest& req); + void recv_get_file_metadata_by_expr(GetFileMetadataByExprResult& _return); + void get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req); + void send_get_file_metadata(const GetFileMetadataRequest& req); + void recv_get_file_metadata(GetFileMetadataResult& _return); + void put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req); + void send_put_file_metadata(const PutFileMetadataRequest& req); + void recv_put_file_metadata(PutFileMetadataResult& _return); + void clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req); + void send_clear_file_metadata(const ClearFileMetadataRequest& req); + void recv_clear_file_metadata(ClearFileMetadataResult& _return); + void cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req); + void send_cache_file_metadata(const CacheFileMetadataRequest& req); + void recv_cache_file_metadata(CacheFileMetadataResult& _return); + void get_metastore_db_uuid(std::string& _return); + void send_get_metastore_db_uuid(); + void recv_get_metastore_db_uuid(std::string& _return); + void create_resource_plan(WMCreateResourcePlanResponse& _return, const WMCreateResourcePlanRequest& request); + void send_create_resource_plan(const WMCreateResourcePlanRequest& request); + void recv_create_resource_plan(WMCreateResourcePlanResponse& _return); + void get_resource_plan(WMGetResourcePlanResponse& _return, const WMGetResourcePlanRequest& request); + void send_get_resource_plan(const WMGetResourcePlanRequest& request); + void recv_get_resource_plan(WMGetResourcePlanResponse& _return); + void get_active_resource_plan(WMGetActiveResourcePlanResponse& _return, const WMGetActiveResourcePlanRequest& request); + void send_get_active_resource_plan(const WMGetActiveResourcePlanRequest& request); + void recv_get_active_resource_plan(WMGetActiveResourcePlanResponse& _return); + void get_all_resource_plans(WMGetAllResourcePlanResponse& _return, const WMGetAllResourcePlanRequest& request); + void send_get_all_resource_plans(const WMGetAllResourcePlanRequest& request); + void recv_get_all_resource_plans(WMGetAllResourcePlanResponse& _return); + void alter_resource_plan(WMAlterResourcePlanResponse& _return, const WMAlterResourcePlanRequest& request); + void send_alter_resource_plan(const WMAlterResourcePlanRequest& request); + void recv_alter_resource_plan(WMAlterResourcePlanResponse& _return); + void validate_resource_plan(WMValidateResourcePlanResponse& _return, const WMValidateResourcePlanRequest& request); + void send_validate_resource_plan(const WMValidateResourcePlanRequest& request); + void recv_validate_resource_plan(WMValidateResourcePlanResponse& _return); + void drop_resource_plan(WMDropResourcePlanResponse& _return, const WMDropResourcePlanRequest& request); + void send_drop_resource_plan(const WMDropResourcePlanRequest& request); + void recv_drop_resource_plan(WMDropResourcePlanResponse& _return); + void create_wm_trigger(WMCreateTriggerResponse& _return, const WMCreateTriggerRequest& request); + void send_create_wm_trigger(const WMCreateTriggerRequest& request); + void recv_create_wm_trigger(WMCreateTriggerResponse& _return); + void alter_wm_trigger(WMAlterTriggerResponse& _return, const WMAlterTriggerRequest& request); + void send_alter_wm_trigger(const WMAlterTriggerRequest& request); + void recv_alter_wm_trigger(WMAlterTriggerResponse& _return); + void drop_wm_trigger(WMDropTriggerResponse& _return, const WMDropTriggerRequest& request); + void send_drop_wm_trigger(const WMDropTriggerRequest& request); + void recv_drop_wm_trigger(WMDropTriggerResponse& _return); + void get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return, const WMGetTriggersForResourePlanRequest& request); + void send_get_triggers_for_resourceplan(const WMGetTriggersForResourePlanRequest& request); + void recv_get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return); + protected: + std::shared_ptr< ::apache::thrift::protocol::TProtocol> piprot_; + std::shared_ptr< ::apache::thrift::protocol::TProtocol> poprot_; + ::apache::thrift::protocol::TProtocol* iprot_; + ::apache::thrift::protocol::TProtocol* oprot_; +}; + +class ThriftHiveMetastoreProcessor : public ::apache::thrift::TDispatchProcessor { + protected: + ::std::shared_ptr iface_; + virtual bool dispatchCall(::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, const std::string& fname, int32_t seqid, void* callContext); + private: + typedef void (ThriftHiveMetastoreProcessor::*ProcessFunction)(int32_t, ::apache::thrift::protocol::TProtocol*, ::apache::thrift::protocol::TProtocol*, void*); + typedef std::map ProcessMap; + ProcessMap processMap_; + void process_getMetaConf(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_setMetaConf(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_create_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_drop_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_databases(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_all_databases(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_alter_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_create_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_drop_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_type_all(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_fields(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_fields_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_schema(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_schema_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_create_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_create_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_create_table_with_constraints(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_drop_constraint(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_add_primary_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_add_foreign_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_add_unique_constraint(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_add_not_null_constraint(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_drop_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_drop_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_truncate_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_tables_by_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_table_meta(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_all_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_table_objects_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_table_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_table_objects_by_name_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_table_names_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_alter_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_alter_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_alter_table_with_cascade(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_add_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_add_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_add_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_add_partitions_pspec(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_append_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_add_partitions_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_append_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_append_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_append_partition_by_name_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_drop_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_drop_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_drop_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_drop_partition_by_name_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_drop_partitions_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_exchange_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_exchange_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_partition_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_partitions_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_partitions_pspec(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_partition_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_partition_values(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_partitions_ps(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_partitions_ps_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_partition_names_ps(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_partitions_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_part_specs_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_partitions_by_expr(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_num_partitions_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_partitions_by_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_alter_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_alter_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_alter_partitions_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_alter_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_rename_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_partition_name_has_valid_characters(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_config_value(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_partition_name_to_vals(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_partition_name_to_spec(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_markPartitionForEvent(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_isPartitionMarkedForEvent(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_add_index(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_alter_index(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_drop_index_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_index_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_indexes(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_index_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_primary_keys(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_foreign_keys(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_unique_constraints(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_not_null_constraints(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_update_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_update_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_table_statistics_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_partitions_statistics_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_aggr_stats_for(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_set_aggr_stats_for(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_delete_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_delete_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_create_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_drop_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_alter_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_functions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_all_functions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_create_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_drop_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_role_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_grant_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_revoke_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_list_roles(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_grant_revoke_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_principals_in_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_role_grants_for_principal(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_privilege_set(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_list_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_grant_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_revoke_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_grant_revoke_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_set_ugi(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_renew_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_cancel_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_add_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_remove_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_all_token_identifiers(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_add_master_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_update_master_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_remove_master_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_master_keys(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_open_txns(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_open_txns_info(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_open_txns(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_abort_txn(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_abort_txns(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_commit_txn(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_lock(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_check_lock(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_unlock(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_show_locks(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_heartbeat(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_heartbeat_txn_range(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_compact(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_compact2(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_show_compact(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_add_dynamic_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_next_notification(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_current_notificationEventId(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_notification_events_count(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_fire_listener_event(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_flushCache(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_cm_recycle(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_file_metadata_by_expr(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_put_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_clear_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_cache_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_metastore_db_uuid(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_create_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_active_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_all_resource_plans(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_alter_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_validate_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_drop_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_create_wm_trigger(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_alter_wm_trigger(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_drop_wm_trigger(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_get_triggers_for_resourceplan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + public: + ThriftHiveMetastoreProcessor(::std::shared_ptr iface) : + iface_(iface) { + processMap_["getMetaConf"] = &ThriftHiveMetastoreProcessor::process_getMetaConf; + processMap_["setMetaConf"] = &ThriftHiveMetastoreProcessor::process_setMetaConf; + processMap_["create_database"] = &ThriftHiveMetastoreProcessor::process_create_database; + processMap_["get_database"] = &ThriftHiveMetastoreProcessor::process_get_database; + processMap_["drop_database"] = &ThriftHiveMetastoreProcessor::process_drop_database; + processMap_["get_databases"] = &ThriftHiveMetastoreProcessor::process_get_databases; + processMap_["get_all_databases"] = &ThriftHiveMetastoreProcessor::process_get_all_databases; + processMap_["alter_database"] = &ThriftHiveMetastoreProcessor::process_alter_database; + processMap_["get_type"] = &ThriftHiveMetastoreProcessor::process_get_type; + processMap_["create_type"] = &ThriftHiveMetastoreProcessor::process_create_type; + processMap_["drop_type"] = &ThriftHiveMetastoreProcessor::process_drop_type; + processMap_["get_type_all"] = &ThriftHiveMetastoreProcessor::process_get_type_all; + processMap_["get_fields"] = &ThriftHiveMetastoreProcessor::process_get_fields; + processMap_["get_fields_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_get_fields_with_environment_context; + processMap_["get_schema"] = &ThriftHiveMetastoreProcessor::process_get_schema; + processMap_["get_schema_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_get_schema_with_environment_context; + processMap_["create_table"] = &ThriftHiveMetastoreProcessor::process_create_table; + processMap_["create_table_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_create_table_with_environment_context; + processMap_["create_table_with_constraints"] = &ThriftHiveMetastoreProcessor::process_create_table_with_constraints; + processMap_["drop_constraint"] = &ThriftHiveMetastoreProcessor::process_drop_constraint; + processMap_["add_primary_key"] = &ThriftHiveMetastoreProcessor::process_add_primary_key; + processMap_["add_foreign_key"] = &ThriftHiveMetastoreProcessor::process_add_foreign_key; + processMap_["add_unique_constraint"] = &ThriftHiveMetastoreProcessor::process_add_unique_constraint; + processMap_["add_not_null_constraint"] = &ThriftHiveMetastoreProcessor::process_add_not_null_constraint; + processMap_["drop_table"] = &ThriftHiveMetastoreProcessor::process_drop_table; + processMap_["drop_table_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_drop_table_with_environment_context; + processMap_["truncate_table"] = &ThriftHiveMetastoreProcessor::process_truncate_table; + processMap_["get_tables"] = &ThriftHiveMetastoreProcessor::process_get_tables; + processMap_["get_tables_by_type"] = &ThriftHiveMetastoreProcessor::process_get_tables_by_type; + processMap_["get_table_meta"] = &ThriftHiveMetastoreProcessor::process_get_table_meta; + processMap_["get_all_tables"] = &ThriftHiveMetastoreProcessor::process_get_all_tables; + processMap_["get_table"] = &ThriftHiveMetastoreProcessor::process_get_table; + processMap_["get_table_objects_by_name"] = &ThriftHiveMetastoreProcessor::process_get_table_objects_by_name; + processMap_["get_table_req"] = &ThriftHiveMetastoreProcessor::process_get_table_req; + processMap_["get_table_objects_by_name_req"] = &ThriftHiveMetastoreProcessor::process_get_table_objects_by_name_req; + processMap_["get_table_names_by_filter"] = &ThriftHiveMetastoreProcessor::process_get_table_names_by_filter; + processMap_["alter_table"] = &ThriftHiveMetastoreProcessor::process_alter_table; + processMap_["alter_table_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_alter_table_with_environment_context; + processMap_["alter_table_with_cascade"] = &ThriftHiveMetastoreProcessor::process_alter_table_with_cascade; + processMap_["add_partition"] = &ThriftHiveMetastoreProcessor::process_add_partition; + processMap_["add_partition_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_add_partition_with_environment_context; + processMap_["add_partitions"] = &ThriftHiveMetastoreProcessor::process_add_partitions; + processMap_["add_partitions_pspec"] = &ThriftHiveMetastoreProcessor::process_add_partitions_pspec; + processMap_["append_partition"] = &ThriftHiveMetastoreProcessor::process_append_partition; + processMap_["add_partitions_req"] = &ThriftHiveMetastoreProcessor::process_add_partitions_req; + processMap_["append_partition_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_append_partition_with_environment_context; + processMap_["append_partition_by_name"] = &ThriftHiveMetastoreProcessor::process_append_partition_by_name; + processMap_["append_partition_by_name_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_append_partition_by_name_with_environment_context; + processMap_["drop_partition"] = &ThriftHiveMetastoreProcessor::process_drop_partition; + processMap_["drop_partition_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_drop_partition_with_environment_context; + processMap_["drop_partition_by_name"] = &ThriftHiveMetastoreProcessor::process_drop_partition_by_name; + processMap_["drop_partition_by_name_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_drop_partition_by_name_with_environment_context; + processMap_["drop_partitions_req"] = &ThriftHiveMetastoreProcessor::process_drop_partitions_req; + processMap_["get_partition"] = &ThriftHiveMetastoreProcessor::process_get_partition; + processMap_["exchange_partition"] = &ThriftHiveMetastoreProcessor::process_exchange_partition; + processMap_["exchange_partitions"] = &ThriftHiveMetastoreProcessor::process_exchange_partitions; + processMap_["get_partition_with_auth"] = &ThriftHiveMetastoreProcessor::process_get_partition_with_auth; + processMap_["get_partition_by_name"] = &ThriftHiveMetastoreProcessor::process_get_partition_by_name; + processMap_["get_partitions"] = &ThriftHiveMetastoreProcessor::process_get_partitions; + processMap_["get_partitions_with_auth"] = &ThriftHiveMetastoreProcessor::process_get_partitions_with_auth; + processMap_["get_partitions_pspec"] = &ThriftHiveMetastoreProcessor::process_get_partitions_pspec; + processMap_["get_partition_names"] = &ThriftHiveMetastoreProcessor::process_get_partition_names; + processMap_["get_partition_values"] = &ThriftHiveMetastoreProcessor::process_get_partition_values; + processMap_["get_partitions_ps"] = &ThriftHiveMetastoreProcessor::process_get_partitions_ps; + processMap_["get_partitions_ps_with_auth"] = &ThriftHiveMetastoreProcessor::process_get_partitions_ps_with_auth; + processMap_["get_partition_names_ps"] = &ThriftHiveMetastoreProcessor::process_get_partition_names_ps; + processMap_["get_partitions_by_filter"] = &ThriftHiveMetastoreProcessor::process_get_partitions_by_filter; + processMap_["get_part_specs_by_filter"] = &ThriftHiveMetastoreProcessor::process_get_part_specs_by_filter; + processMap_["get_partitions_by_expr"] = &ThriftHiveMetastoreProcessor::process_get_partitions_by_expr; + processMap_["get_num_partitions_by_filter"] = &ThriftHiveMetastoreProcessor::process_get_num_partitions_by_filter; + processMap_["get_partitions_by_names"] = &ThriftHiveMetastoreProcessor::process_get_partitions_by_names; + processMap_["alter_partition"] = &ThriftHiveMetastoreProcessor::process_alter_partition; + processMap_["alter_partitions"] = &ThriftHiveMetastoreProcessor::process_alter_partitions; + processMap_["alter_partitions_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_alter_partitions_with_environment_context; + processMap_["alter_partition_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_alter_partition_with_environment_context; + processMap_["rename_partition"] = &ThriftHiveMetastoreProcessor::process_rename_partition; + processMap_["partition_name_has_valid_characters"] = &ThriftHiveMetastoreProcessor::process_partition_name_has_valid_characters; + processMap_["get_config_value"] = &ThriftHiveMetastoreProcessor::process_get_config_value; + processMap_["partition_name_to_vals"] = &ThriftHiveMetastoreProcessor::process_partition_name_to_vals; + processMap_["partition_name_to_spec"] = &ThriftHiveMetastoreProcessor::process_partition_name_to_spec; + processMap_["markPartitionForEvent"] = &ThriftHiveMetastoreProcessor::process_markPartitionForEvent; + processMap_["isPartitionMarkedForEvent"] = &ThriftHiveMetastoreProcessor::process_isPartitionMarkedForEvent; + processMap_["add_index"] = &ThriftHiveMetastoreProcessor::process_add_index; + processMap_["alter_index"] = &ThriftHiveMetastoreProcessor::process_alter_index; + processMap_["drop_index_by_name"] = &ThriftHiveMetastoreProcessor::process_drop_index_by_name; + processMap_["get_index_by_name"] = &ThriftHiveMetastoreProcessor::process_get_index_by_name; + processMap_["get_indexes"] = &ThriftHiveMetastoreProcessor::process_get_indexes; + processMap_["get_index_names"] = &ThriftHiveMetastoreProcessor::process_get_index_names; + processMap_["get_primary_keys"] = &ThriftHiveMetastoreProcessor::process_get_primary_keys; + processMap_["get_foreign_keys"] = &ThriftHiveMetastoreProcessor::process_get_foreign_keys; + processMap_["get_unique_constraints"] = &ThriftHiveMetastoreProcessor::process_get_unique_constraints; + processMap_["get_not_null_constraints"] = &ThriftHiveMetastoreProcessor::process_get_not_null_constraints; + processMap_["update_table_column_statistics"] = &ThriftHiveMetastoreProcessor::process_update_table_column_statistics; + processMap_["update_partition_column_statistics"] = &ThriftHiveMetastoreProcessor::process_update_partition_column_statistics; + processMap_["get_table_column_statistics"] = &ThriftHiveMetastoreProcessor::process_get_table_column_statistics; + processMap_["get_partition_column_statistics"] = &ThriftHiveMetastoreProcessor::process_get_partition_column_statistics; + processMap_["get_table_statistics_req"] = &ThriftHiveMetastoreProcessor::process_get_table_statistics_req; + processMap_["get_partitions_statistics_req"] = &ThriftHiveMetastoreProcessor::process_get_partitions_statistics_req; + processMap_["get_aggr_stats_for"] = &ThriftHiveMetastoreProcessor::process_get_aggr_stats_for; + processMap_["set_aggr_stats_for"] = &ThriftHiveMetastoreProcessor::process_set_aggr_stats_for; + processMap_["delete_partition_column_statistics"] = &ThriftHiveMetastoreProcessor::process_delete_partition_column_statistics; + processMap_["delete_table_column_statistics"] = &ThriftHiveMetastoreProcessor::process_delete_table_column_statistics; + processMap_["create_function"] = &ThriftHiveMetastoreProcessor::process_create_function; + processMap_["drop_function"] = &ThriftHiveMetastoreProcessor::process_drop_function; + processMap_["alter_function"] = &ThriftHiveMetastoreProcessor::process_alter_function; + processMap_["get_functions"] = &ThriftHiveMetastoreProcessor::process_get_functions; + processMap_["get_function"] = &ThriftHiveMetastoreProcessor::process_get_function; + processMap_["get_all_functions"] = &ThriftHiveMetastoreProcessor::process_get_all_functions; + processMap_["create_role"] = &ThriftHiveMetastoreProcessor::process_create_role; + processMap_["drop_role"] = &ThriftHiveMetastoreProcessor::process_drop_role; + processMap_["get_role_names"] = &ThriftHiveMetastoreProcessor::process_get_role_names; + processMap_["grant_role"] = &ThriftHiveMetastoreProcessor::process_grant_role; + processMap_["revoke_role"] = &ThriftHiveMetastoreProcessor::process_revoke_role; + processMap_["list_roles"] = &ThriftHiveMetastoreProcessor::process_list_roles; + processMap_["grant_revoke_role"] = &ThriftHiveMetastoreProcessor::process_grant_revoke_role; + processMap_["get_principals_in_role"] = &ThriftHiveMetastoreProcessor::process_get_principals_in_role; + processMap_["get_role_grants_for_principal"] = &ThriftHiveMetastoreProcessor::process_get_role_grants_for_principal; + processMap_["get_privilege_set"] = &ThriftHiveMetastoreProcessor::process_get_privilege_set; + processMap_["list_privileges"] = &ThriftHiveMetastoreProcessor::process_list_privileges; + processMap_["grant_privileges"] = &ThriftHiveMetastoreProcessor::process_grant_privileges; + processMap_["revoke_privileges"] = &ThriftHiveMetastoreProcessor::process_revoke_privileges; + processMap_["grant_revoke_privileges"] = &ThriftHiveMetastoreProcessor::process_grant_revoke_privileges; + processMap_["set_ugi"] = &ThriftHiveMetastoreProcessor::process_set_ugi; + processMap_["get_delegation_token"] = &ThriftHiveMetastoreProcessor::process_get_delegation_token; + processMap_["renew_delegation_token"] = &ThriftHiveMetastoreProcessor::process_renew_delegation_token; + processMap_["cancel_delegation_token"] = &ThriftHiveMetastoreProcessor::process_cancel_delegation_token; + processMap_["add_token"] = &ThriftHiveMetastoreProcessor::process_add_token; + processMap_["remove_token"] = &ThriftHiveMetastoreProcessor::process_remove_token; + processMap_["get_token"] = &ThriftHiveMetastoreProcessor::process_get_token; + processMap_["get_all_token_identifiers"] = &ThriftHiveMetastoreProcessor::process_get_all_token_identifiers; + processMap_["add_master_key"] = &ThriftHiveMetastoreProcessor::process_add_master_key; + processMap_["update_master_key"] = &ThriftHiveMetastoreProcessor::process_update_master_key; + processMap_["remove_master_key"] = &ThriftHiveMetastoreProcessor::process_remove_master_key; + processMap_["get_master_keys"] = &ThriftHiveMetastoreProcessor::process_get_master_keys; + processMap_["get_open_txns"] = &ThriftHiveMetastoreProcessor::process_get_open_txns; + processMap_["get_open_txns_info"] = &ThriftHiveMetastoreProcessor::process_get_open_txns_info; + processMap_["open_txns"] = &ThriftHiveMetastoreProcessor::process_open_txns; + processMap_["abort_txn"] = &ThriftHiveMetastoreProcessor::process_abort_txn; + processMap_["abort_txns"] = &ThriftHiveMetastoreProcessor::process_abort_txns; + processMap_["commit_txn"] = &ThriftHiveMetastoreProcessor::process_commit_txn; + processMap_["lock"] = &ThriftHiveMetastoreProcessor::process_lock; + processMap_["check_lock"] = &ThriftHiveMetastoreProcessor::process_check_lock; + processMap_["unlock"] = &ThriftHiveMetastoreProcessor::process_unlock; + processMap_["show_locks"] = &ThriftHiveMetastoreProcessor::process_show_locks; + processMap_["heartbeat"] = &ThriftHiveMetastoreProcessor::process_heartbeat; + processMap_["heartbeat_txn_range"] = &ThriftHiveMetastoreProcessor::process_heartbeat_txn_range; + processMap_["compact"] = &ThriftHiveMetastoreProcessor::process_compact; + processMap_["compact2"] = &ThriftHiveMetastoreProcessor::process_compact2; + processMap_["show_compact"] = &ThriftHiveMetastoreProcessor::process_show_compact; + processMap_["add_dynamic_partitions"] = &ThriftHiveMetastoreProcessor::process_add_dynamic_partitions; + processMap_["get_next_notification"] = &ThriftHiveMetastoreProcessor::process_get_next_notification; + processMap_["get_current_notificationEventId"] = &ThriftHiveMetastoreProcessor::process_get_current_notificationEventId; + processMap_["get_notification_events_count"] = &ThriftHiveMetastoreProcessor::process_get_notification_events_count; + processMap_["fire_listener_event"] = &ThriftHiveMetastoreProcessor::process_fire_listener_event; + processMap_["flushCache"] = &ThriftHiveMetastoreProcessor::process_flushCache; + processMap_["cm_recycle"] = &ThriftHiveMetastoreProcessor::process_cm_recycle; + processMap_["get_file_metadata_by_expr"] = &ThriftHiveMetastoreProcessor::process_get_file_metadata_by_expr; + processMap_["get_file_metadata"] = &ThriftHiveMetastoreProcessor::process_get_file_metadata; + processMap_["put_file_metadata"] = &ThriftHiveMetastoreProcessor::process_put_file_metadata; + processMap_["clear_file_metadata"] = &ThriftHiveMetastoreProcessor::process_clear_file_metadata; + processMap_["cache_file_metadata"] = &ThriftHiveMetastoreProcessor::process_cache_file_metadata; + processMap_["get_metastore_db_uuid"] = &ThriftHiveMetastoreProcessor::process_get_metastore_db_uuid; + processMap_["create_resource_plan"] = &ThriftHiveMetastoreProcessor::process_create_resource_plan; + processMap_["get_resource_plan"] = &ThriftHiveMetastoreProcessor::process_get_resource_plan; + processMap_["get_active_resource_plan"] = &ThriftHiveMetastoreProcessor::process_get_active_resource_plan; + processMap_["get_all_resource_plans"] = &ThriftHiveMetastoreProcessor::process_get_all_resource_plans; + processMap_["alter_resource_plan"] = &ThriftHiveMetastoreProcessor::process_alter_resource_plan; + processMap_["validate_resource_plan"] = &ThriftHiveMetastoreProcessor::process_validate_resource_plan; + processMap_["drop_resource_plan"] = &ThriftHiveMetastoreProcessor::process_drop_resource_plan; + processMap_["create_wm_trigger"] = &ThriftHiveMetastoreProcessor::process_create_wm_trigger; + processMap_["alter_wm_trigger"] = &ThriftHiveMetastoreProcessor::process_alter_wm_trigger; + processMap_["drop_wm_trigger"] = &ThriftHiveMetastoreProcessor::process_drop_wm_trigger; + processMap_["get_triggers_for_resourceplan"] = &ThriftHiveMetastoreProcessor::process_get_triggers_for_resourceplan; + } + + virtual ~ThriftHiveMetastoreProcessor() {} +}; + +class ThriftHiveMetastoreProcessorFactory : public ::apache::thrift::TProcessorFactory { + public: + ThriftHiveMetastoreProcessorFactory(const ::std::shared_ptr< ThriftHiveMetastoreIfFactory >& handlerFactory) : + handlerFactory_(handlerFactory) {} + + ::std::shared_ptr< ::apache::thrift::TProcessor > getProcessor(const ::apache::thrift::TConnectionInfo& connInfo); + + protected: + ::std::shared_ptr< ThriftHiveMetastoreIfFactory > handlerFactory_; +}; + +class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf { + public: + ThriftHiveMetastoreMultiface(std::vector >& ifaces) : ifaces_(ifaces) { + } + virtual ~ThriftHiveMetastoreMultiface() {} + protected: + std::vector > ifaces_; + ThriftHiveMetastoreMultiface() {} + void add(::std::shared_ptr iface) { + ifaces_.push_back(iface); + } + public: + void getMetaConf(std::string& _return, const std::string& key) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->getMetaConf(_return, key); + } + ifaces_[i]->getMetaConf(_return, key); + return; + } + + void setMetaConf(const std::string& key, const std::string& value) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->setMetaConf(key, value); + } + ifaces_[i]->setMetaConf(key, value); + } + + void create_database(const Database& database) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->create_database(database); + } + ifaces_[i]->create_database(database); + } + + void get_database(Database& _return, const std::string& name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_database(_return, name); + } + ifaces_[i]->get_database(_return, name); + return; + } + + void drop_database(const std::string& name, const bool deleteData, const bool cascade) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->drop_database(name, deleteData, cascade); + } + ifaces_[i]->drop_database(name, deleteData, cascade); + } + + void get_databases(std::vector & _return, const std::string& pattern) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_databases(_return, pattern); + } + ifaces_[i]->get_databases(_return, pattern); + return; + } + + void get_all_databases(std::vector & _return) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_all_databases(_return); + } + ifaces_[i]->get_all_databases(_return); + return; + } + + void alter_database(const std::string& dbname, const Database& db) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->alter_database(dbname, db); + } + ifaces_[i]->alter_database(dbname, db); + } + + void get_type(Type& _return, const std::string& name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_type(_return, name); + } + ifaces_[i]->get_type(_return, name); + return; + } + + bool create_type(const Type& type) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->create_type(type); + } + return ifaces_[i]->create_type(type); + } + + bool drop_type(const std::string& type) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->drop_type(type); + } + return ifaces_[i]->drop_type(type); + } + + void get_type_all(std::map & _return, const std::string& name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_type_all(_return, name); + } + ifaces_[i]->get_type_all(_return, name); + return; + } + + void get_fields(std::vector & _return, const std::string& db_name, const std::string& table_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_fields(_return, db_name, table_name); + } + ifaces_[i]->get_fields(_return, db_name, table_name); + return; + } + + void get_fields_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_fields_with_environment_context(_return, db_name, table_name, environment_context); + } + ifaces_[i]->get_fields_with_environment_context(_return, db_name, table_name, environment_context); + return; + } + + void get_schema(std::vector & _return, const std::string& db_name, const std::string& table_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_schema(_return, db_name, table_name); + } + ifaces_[i]->get_schema(_return, db_name, table_name); + return; + } + + void get_schema_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_schema_with_environment_context(_return, db_name, table_name, environment_context); + } + ifaces_[i]->get_schema_with_environment_context(_return, db_name, table_name, environment_context); + return; + } + + void create_table(const Table& tbl) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->create_table(tbl); + } + ifaces_[i]->create_table(tbl); + } + + void create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->create_table_with_environment_context(tbl, environment_context); + } + ifaces_[i]->create_table_with_environment_context(tbl, environment_context); + } + + void create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints); + } + ifaces_[i]->create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints); + } + + void drop_constraint(const DropConstraintRequest& req) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->drop_constraint(req); + } + ifaces_[i]->drop_constraint(req); + } + + void add_primary_key(const AddPrimaryKeyRequest& req) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->add_primary_key(req); + } + ifaces_[i]->add_primary_key(req); + } + + void add_foreign_key(const AddForeignKeyRequest& req) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->add_foreign_key(req); + } + ifaces_[i]->add_foreign_key(req); + } + + void add_unique_constraint(const AddUniqueConstraintRequest& req) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->add_unique_constraint(req); + } + ifaces_[i]->add_unique_constraint(req); + } + + void add_not_null_constraint(const AddNotNullConstraintRequest& req) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->add_not_null_constraint(req); + } + ifaces_[i]->add_not_null_constraint(req); + } + + void drop_table(const std::string& dbname, const std::string& name, const bool deleteData) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->drop_table(dbname, name, deleteData); + } + ifaces_[i]->drop_table(dbname, name, deleteData); + } + + void drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->drop_table_with_environment_context(dbname, name, deleteData, environment_context); + } + ifaces_[i]->drop_table_with_environment_context(dbname, name, deleteData, environment_context); + } + + void truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->truncate_table(dbName, tableName, partNames); + } + ifaces_[i]->truncate_table(dbName, tableName, partNames); + } + + void get_tables(std::vector & _return, const std::string& db_name, const std::string& pattern) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_tables(_return, db_name, pattern); + } + ifaces_[i]->get_tables(_return, db_name, pattern); + return; + } + + void get_tables_by_type(std::vector & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_tables_by_type(_return, db_name, pattern, tableType); + } + ifaces_[i]->get_tables_by_type(_return, db_name, pattern, tableType); + return; + } + + void get_table_meta(std::vector & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_table_meta(_return, db_patterns, tbl_patterns, tbl_types); + } + ifaces_[i]->get_table_meta(_return, db_patterns, tbl_patterns, tbl_types); + return; + } + + void get_all_tables(std::vector & _return, const std::string& db_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_all_tables(_return, db_name); + } + ifaces_[i]->get_all_tables(_return, db_name); + return; + } + + void get_table(Table& _return, const std::string& dbname, const std::string& tbl_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_table(_return, dbname, tbl_name); + } + ifaces_[i]->get_table(_return, dbname, tbl_name); + return; + } + + void get_table_objects_by_name(std::vector
& _return, const std::string& dbname, const std::vector & tbl_names) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_table_objects_by_name(_return, dbname, tbl_names); + } + ifaces_[i]->get_table_objects_by_name(_return, dbname, tbl_names); + return; + } + + void get_table_req(GetTableResult& _return, const GetTableRequest& req) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_table_req(_return, req); + } + ifaces_[i]->get_table_req(_return, req); + return; + } + + void get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_table_objects_by_name_req(_return, req); + } + ifaces_[i]->get_table_objects_by_name_req(_return, req); + return; + } + + void get_table_names_by_filter(std::vector & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_table_names_by_filter(_return, dbname, filter, max_tables); + } + ifaces_[i]->get_table_names_by_filter(_return, dbname, filter, max_tables); + return; + } + + void alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->alter_table(dbname, tbl_name, new_tbl); + } + ifaces_[i]->alter_table(dbname, tbl_name, new_tbl); + } + + void alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->alter_table_with_environment_context(dbname, tbl_name, new_tbl, environment_context); + } + ifaces_[i]->alter_table_with_environment_context(dbname, tbl_name, new_tbl, environment_context); + } + + void alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->alter_table_with_cascade(dbname, tbl_name, new_tbl, cascade); + } + ifaces_[i]->alter_table_with_cascade(dbname, tbl_name, new_tbl, cascade); + } + + void add_partition(Partition& _return, const Partition& new_part) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->add_partition(_return, new_part); + } + ifaces_[i]->add_partition(_return, new_part); + return; + } + + void add_partition_with_environment_context(Partition& _return, const Partition& new_part, const EnvironmentContext& environment_context) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->add_partition_with_environment_context(_return, new_part, environment_context); + } + ifaces_[i]->add_partition_with_environment_context(_return, new_part, environment_context); + return; + } + + int32_t add_partitions(const std::vector & new_parts) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->add_partitions(new_parts); + } + return ifaces_[i]->add_partitions(new_parts); + } + + int32_t add_partitions_pspec(const std::vector & new_parts) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->add_partitions_pspec(new_parts); + } + return ifaces_[i]->add_partitions_pspec(new_parts); + } + + void append_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->append_partition(_return, db_name, tbl_name, part_vals); + } + ifaces_[i]->append_partition(_return, db_name, tbl_name, part_vals); + return; + } + + void add_partitions_req(AddPartitionsResult& _return, const AddPartitionsRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->add_partitions_req(_return, request); + } + ifaces_[i]->add_partitions_req(_return, request); + return; + } + + void append_partition_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->append_partition_with_environment_context(_return, db_name, tbl_name, part_vals, environment_context); + } + ifaces_[i]->append_partition_with_environment_context(_return, db_name, tbl_name, part_vals, environment_context); + return; + } + + void append_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->append_partition_by_name(_return, db_name, tbl_name, part_name); + } + ifaces_[i]->append_partition_by_name(_return, db_name, tbl_name, part_name); + return; + } + + void append_partition_by_name_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->append_partition_by_name_with_environment_context(_return, db_name, tbl_name, part_name, environment_context); + } + ifaces_[i]->append_partition_by_name_with_environment_context(_return, db_name, tbl_name, part_name, environment_context); + return; + } + + bool drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->drop_partition(db_name, tbl_name, part_vals, deleteData); + } + return ifaces_[i]->drop_partition(db_name, tbl_name, part_vals, deleteData); + } + + bool drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->drop_partition_with_environment_context(db_name, tbl_name, part_vals, deleteData, environment_context); + } + return ifaces_[i]->drop_partition_with_environment_context(db_name, tbl_name, part_vals, deleteData, environment_context); + } + + bool drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->drop_partition_by_name(db_name, tbl_name, part_name, deleteData); + } + return ifaces_[i]->drop_partition_by_name(db_name, tbl_name, part_name, deleteData); + } + + bool drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->drop_partition_by_name_with_environment_context(db_name, tbl_name, part_name, deleteData, environment_context); + } + return ifaces_[i]->drop_partition_by_name_with_environment_context(db_name, tbl_name, part_name, deleteData, environment_context); + } + + void drop_partitions_req(DropPartitionsResult& _return, const DropPartitionsRequest& req) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->drop_partitions_req(_return, req); + } + ifaces_[i]->drop_partitions_req(_return, req); + return; + } + + void get_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_partition(_return, db_name, tbl_name, part_vals); + } + ifaces_[i]->get_partition(_return, db_name, tbl_name, part_vals); + return; + } + + void exchange_partition(Partition& _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->exchange_partition(_return, partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); + } + ifaces_[i]->exchange_partition(_return, partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); + return; + } + + void exchange_partitions(std::vector & _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->exchange_partitions(_return, partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); + } + ifaces_[i]->exchange_partitions(_return, partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); + return; + } + + void get_partition_with_auth(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_partition_with_auth(_return, db_name, tbl_name, part_vals, user_name, group_names); + } + ifaces_[i]->get_partition_with_auth(_return, db_name, tbl_name, part_vals, user_name, group_names); + return; + } + + void get_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_partition_by_name(_return, db_name, tbl_name, part_name); + } + ifaces_[i]->get_partition_by_name(_return, db_name, tbl_name, part_name); + return; + } + + void get_partitions(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_partitions(_return, db_name, tbl_name, max_parts); + } + ifaces_[i]->get_partitions(_return, db_name, tbl_name, max_parts); + return; + } + + void get_partitions_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_partitions_with_auth(_return, db_name, tbl_name, max_parts, user_name, group_names); + } + ifaces_[i]->get_partitions_with_auth(_return, db_name, tbl_name, max_parts, user_name, group_names); + return; + } + + void get_partitions_pspec(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int32_t max_parts) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_partitions_pspec(_return, db_name, tbl_name, max_parts); + } + ifaces_[i]->get_partitions_pspec(_return, db_name, tbl_name, max_parts); + return; + } + + void get_partition_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_partition_names(_return, db_name, tbl_name, max_parts); + } + ifaces_[i]->get_partition_names(_return, db_name, tbl_name, max_parts); + return; + } + + void get_partition_values(PartitionValuesResponse& _return, const PartitionValuesRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_partition_values(_return, request); + } + ifaces_[i]->get_partition_values(_return, request); + return; + } + + void get_partitions_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_partitions_ps(_return, db_name, tbl_name, part_vals, max_parts); + } + ifaces_[i]->get_partitions_ps(_return, db_name, tbl_name, part_vals, max_parts); + return; + } + + void get_partitions_ps_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_partitions_ps_with_auth(_return, db_name, tbl_name, part_vals, max_parts, user_name, group_names); + } + ifaces_[i]->get_partitions_ps_with_auth(_return, db_name, tbl_name, part_vals, max_parts, user_name, group_names); + return; + } + + void get_partition_names_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_partition_names_ps(_return, db_name, tbl_name, part_vals, max_parts); + } + ifaces_[i]->get_partition_names_ps(_return, db_name, tbl_name, part_vals, max_parts); + return; + } + + void get_partitions_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_partitions_by_filter(_return, db_name, tbl_name, filter, max_parts); + } + ifaces_[i]->get_partitions_by_filter(_return, db_name, tbl_name, filter, max_parts); + return; + } + + void get_part_specs_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_part_specs_by_filter(_return, db_name, tbl_name, filter, max_parts); + } + ifaces_[i]->get_part_specs_by_filter(_return, db_name, tbl_name, filter, max_parts); + return; + } + + void get_partitions_by_expr(PartitionsByExprResult& _return, const PartitionsByExprRequest& req) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_partitions_by_expr(_return, req); + } + ifaces_[i]->get_partitions_by_expr(_return, req); + return; + } + + int32_t get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_num_partitions_by_filter(db_name, tbl_name, filter); + } + return ifaces_[i]->get_num_partitions_by_filter(db_name, tbl_name, filter); + } + + void get_partitions_by_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & names) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_partitions_by_names(_return, db_name, tbl_name, names); + } + ifaces_[i]->get_partitions_by_names(_return, db_name, tbl_name, names); + return; + } + + void alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->alter_partition(db_name, tbl_name, new_part); + } + ifaces_[i]->alter_partition(db_name, tbl_name, new_part); + } + + void alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->alter_partitions(db_name, tbl_name, new_parts); + } + ifaces_[i]->alter_partitions(db_name, tbl_name, new_parts); + } + + void alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->alter_partitions_with_environment_context(db_name, tbl_name, new_parts, environment_context); + } + ifaces_[i]->alter_partitions_with_environment_context(db_name, tbl_name, new_parts, environment_context); + } + + void alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->alter_partition_with_environment_context(db_name, tbl_name, new_part, environment_context); + } + ifaces_[i]->alter_partition_with_environment_context(db_name, tbl_name, new_part, environment_context); + } + + void rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->rename_partition(db_name, tbl_name, part_vals, new_part); + } + ifaces_[i]->rename_partition(db_name, tbl_name, part_vals, new_part); + } + + bool partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->partition_name_has_valid_characters(part_vals, throw_exception); + } + return ifaces_[i]->partition_name_has_valid_characters(part_vals, throw_exception); + } + + void get_config_value(std::string& _return, const std::string& name, const std::string& defaultValue) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_config_value(_return, name, defaultValue); + } + ifaces_[i]->get_config_value(_return, name, defaultValue); + return; + } + + void partition_name_to_vals(std::vector & _return, const std::string& part_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->partition_name_to_vals(_return, part_name); + } + ifaces_[i]->partition_name_to_vals(_return, part_name); + return; + } + + void partition_name_to_spec(std::map & _return, const std::string& part_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->partition_name_to_spec(_return, part_name); + } + ifaces_[i]->partition_name_to_spec(_return, part_name); + return; + } + + void markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->markPartitionForEvent(db_name, tbl_name, part_vals, eventType); + } + ifaces_[i]->markPartitionForEvent(db_name, tbl_name, part_vals, eventType); + } + + bool isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->isPartitionMarkedForEvent(db_name, tbl_name, part_vals, eventType); + } + return ifaces_[i]->isPartitionMarkedForEvent(db_name, tbl_name, part_vals, eventType); + } + + void add_index(Index& _return, const Index& new_index, const Table& index_table) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->add_index(_return, new_index, index_table); + } + ifaces_[i]->add_index(_return, new_index, index_table); + return; + } + + void alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->alter_index(dbname, base_tbl_name, idx_name, new_idx); + } + ifaces_[i]->alter_index(dbname, base_tbl_name, idx_name, new_idx); + } + + bool drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->drop_index_by_name(db_name, tbl_name, index_name, deleteData); + } + return ifaces_[i]->drop_index_by_name(db_name, tbl_name, index_name, deleteData); + } + + void get_index_by_name(Index& _return, const std::string& db_name, const std::string& tbl_name, const std::string& index_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_index_by_name(_return, db_name, tbl_name, index_name); + } + ifaces_[i]->get_index_by_name(_return, db_name, tbl_name, index_name); + return; + } + + void get_indexes(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_indexes(_return, db_name, tbl_name, max_indexes); + } + ifaces_[i]->get_indexes(_return, db_name, tbl_name, max_indexes); + return; + } + + void get_index_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_index_names(_return, db_name, tbl_name, max_indexes); + } + ifaces_[i]->get_index_names(_return, db_name, tbl_name, max_indexes); + return; + } + + void get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_primary_keys(_return, request); + } + ifaces_[i]->get_primary_keys(_return, request); + return; + } + + void get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_foreign_keys(_return, request); + } + ifaces_[i]->get_foreign_keys(_return, request); + return; + } + + void get_unique_constraints(UniqueConstraintsResponse& _return, const UniqueConstraintsRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_unique_constraints(_return, request); + } + ifaces_[i]->get_unique_constraints(_return, request); + return; + } + + void get_not_null_constraints(NotNullConstraintsResponse& _return, const NotNullConstraintsRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_not_null_constraints(_return, request); + } + ifaces_[i]->get_not_null_constraints(_return, request); + return; + } + + bool update_table_column_statistics(const ColumnStatistics& stats_obj) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->update_table_column_statistics(stats_obj); + } + return ifaces_[i]->update_table_column_statistics(stats_obj); + } + + bool update_partition_column_statistics(const ColumnStatistics& stats_obj) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->update_partition_column_statistics(stats_obj); + } + return ifaces_[i]->update_partition_column_statistics(stats_obj); + } + + void get_table_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& col_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_table_column_statistics(_return, db_name, tbl_name, col_name); + } + ifaces_[i]->get_table_column_statistics(_return, db_name, tbl_name, col_name); + return; + } + + void get_partition_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_partition_column_statistics(_return, db_name, tbl_name, part_name, col_name); + } + ifaces_[i]->get_partition_column_statistics(_return, db_name, tbl_name, part_name, col_name); + return; + } + + void get_table_statistics_req(TableStatsResult& _return, const TableStatsRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_table_statistics_req(_return, request); + } + ifaces_[i]->get_table_statistics_req(_return, request); + return; + } + + void get_partitions_statistics_req(PartitionsStatsResult& _return, const PartitionsStatsRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_partitions_statistics_req(_return, request); + } + ifaces_[i]->get_partitions_statistics_req(_return, request); + return; + } + + void get_aggr_stats_for(AggrStats& _return, const PartitionsStatsRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_aggr_stats_for(_return, request); + } + ifaces_[i]->get_aggr_stats_for(_return, request); + return; + } + + bool set_aggr_stats_for(const SetPartitionsStatsRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->set_aggr_stats_for(request); + } + return ifaces_[i]->set_aggr_stats_for(request); + } + + bool delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->delete_partition_column_statistics(db_name, tbl_name, part_name, col_name); + } + return ifaces_[i]->delete_partition_column_statistics(db_name, tbl_name, part_name, col_name); + } + + bool delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->delete_table_column_statistics(db_name, tbl_name, col_name); + } + return ifaces_[i]->delete_table_column_statistics(db_name, tbl_name, col_name); + } + + void create_function(const Function& func) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->create_function(func); + } + ifaces_[i]->create_function(func); + } + + void drop_function(const std::string& dbName, const std::string& funcName) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->drop_function(dbName, funcName); + } + ifaces_[i]->drop_function(dbName, funcName); + } + + void alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->alter_function(dbName, funcName, newFunc); + } + ifaces_[i]->alter_function(dbName, funcName, newFunc); + } + + void get_functions(std::vector & _return, const std::string& dbName, const std::string& pattern) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_functions(_return, dbName, pattern); + } + ifaces_[i]->get_functions(_return, dbName, pattern); + return; + } + + void get_function(Function& _return, const std::string& dbName, const std::string& funcName) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_function(_return, dbName, funcName); + } + ifaces_[i]->get_function(_return, dbName, funcName); + return; + } + + void get_all_functions(GetAllFunctionsResponse& _return) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_all_functions(_return); + } + ifaces_[i]->get_all_functions(_return); + return; + } + + bool create_role(const Role& role) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->create_role(role); + } + return ifaces_[i]->create_role(role); + } + + bool drop_role(const std::string& role_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->drop_role(role_name); + } + return ifaces_[i]->drop_role(role_name); + } + + void get_role_names(std::vector & _return) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_role_names(_return); + } + ifaces_[i]->get_role_names(_return); + return; + } + + bool grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->grant_role(role_name, principal_name, principal_type, grantor, grantorType, grant_option); + } + return ifaces_[i]->grant_role(role_name, principal_name, principal_type, grantor, grantorType, grant_option); + } + + bool revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->revoke_role(role_name, principal_name, principal_type); + } + return ifaces_[i]->revoke_role(role_name, principal_name, principal_type); + } + + void list_roles(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->list_roles(_return, principal_name, principal_type); + } + ifaces_[i]->list_roles(_return, principal_name, principal_type); + return; + } + + void grant_revoke_role(GrantRevokeRoleResponse& _return, const GrantRevokeRoleRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->grant_revoke_role(_return, request); + } + ifaces_[i]->grant_revoke_role(_return, request); + return; + } + + void get_principals_in_role(GetPrincipalsInRoleResponse& _return, const GetPrincipalsInRoleRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_principals_in_role(_return, request); + } + ifaces_[i]->get_principals_in_role(_return, request); + return; + } + + void get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return, const GetRoleGrantsForPrincipalRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_role_grants_for_principal(_return, request); + } + ifaces_[i]->get_role_grants_for_principal(_return, request); + return; + } + + void get_privilege_set(PrincipalPrivilegeSet& _return, const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_privilege_set(_return, hiveObject, user_name, group_names); + } + ifaces_[i]->get_privilege_set(_return, hiveObject, user_name, group_names); + return; + } + + void list_privileges(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->list_privileges(_return, principal_name, principal_type, hiveObject); + } + ifaces_[i]->list_privileges(_return, principal_name, principal_type, hiveObject); + return; + } + + bool grant_privileges(const PrivilegeBag& privileges) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->grant_privileges(privileges); + } + return ifaces_[i]->grant_privileges(privileges); + } + + bool revoke_privileges(const PrivilegeBag& privileges) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->revoke_privileges(privileges); + } + return ifaces_[i]->revoke_privileges(privileges); + } + + void grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const GrantRevokePrivilegeRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->grant_revoke_privileges(_return, request); + } + ifaces_[i]->grant_revoke_privileges(_return, request); + return; + } + + void set_ugi(std::vector & _return, const std::string& user_name, const std::vector & group_names) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->set_ugi(_return, user_name, group_names); + } + ifaces_[i]->set_ugi(_return, user_name, group_names); + return; + } + + void get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_delegation_token(_return, token_owner, renewer_kerberos_principal_name); + } + ifaces_[i]->get_delegation_token(_return, token_owner, renewer_kerberos_principal_name); + return; + } + + int64_t renew_delegation_token(const std::string& token_str_form) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->renew_delegation_token(token_str_form); + } + return ifaces_[i]->renew_delegation_token(token_str_form); + } + + void cancel_delegation_token(const std::string& token_str_form) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->cancel_delegation_token(token_str_form); + } + ifaces_[i]->cancel_delegation_token(token_str_form); + } + + bool add_token(const std::string& token_identifier, const std::string& delegation_token) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->add_token(token_identifier, delegation_token); + } + return ifaces_[i]->add_token(token_identifier, delegation_token); + } + + bool remove_token(const std::string& token_identifier) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->remove_token(token_identifier); + } + return ifaces_[i]->remove_token(token_identifier); + } + + void get_token(std::string& _return, const std::string& token_identifier) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_token(_return, token_identifier); + } + ifaces_[i]->get_token(_return, token_identifier); + return; + } + + void get_all_token_identifiers(std::vector & _return) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_all_token_identifiers(_return); + } + ifaces_[i]->get_all_token_identifiers(_return); + return; + } + + int32_t add_master_key(const std::string& key) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->add_master_key(key); + } + return ifaces_[i]->add_master_key(key); + } + + void update_master_key(const int32_t seq_number, const std::string& key) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->update_master_key(seq_number, key); + } + ifaces_[i]->update_master_key(seq_number, key); + } + + bool remove_master_key(const int32_t key_seq) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->remove_master_key(key_seq); + } + return ifaces_[i]->remove_master_key(key_seq); + } + + void get_master_keys(std::vector & _return) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_master_keys(_return); + } + ifaces_[i]->get_master_keys(_return); + return; + } + + void get_open_txns(GetOpenTxnsResponse& _return) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_open_txns(_return); + } + ifaces_[i]->get_open_txns(_return); + return; + } + + void get_open_txns_info(GetOpenTxnsInfoResponse& _return) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_open_txns_info(_return); + } + ifaces_[i]->get_open_txns_info(_return); + return; + } + + void open_txns(OpenTxnsResponse& _return, const OpenTxnRequest& rqst) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->open_txns(_return, rqst); + } + ifaces_[i]->open_txns(_return, rqst); + return; + } + + void abort_txn(const AbortTxnRequest& rqst) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->abort_txn(rqst); + } + ifaces_[i]->abort_txn(rqst); + } + + void abort_txns(const AbortTxnsRequest& rqst) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->abort_txns(rqst); + } + ifaces_[i]->abort_txns(rqst); + } + + void commit_txn(const CommitTxnRequest& rqst) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->commit_txn(rqst); + } + ifaces_[i]->commit_txn(rqst); + } + + void lock(LockResponse& _return, const LockRequest& rqst) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->lock(_return, rqst); + } + ifaces_[i]->lock(_return, rqst); + return; + } + + void check_lock(LockResponse& _return, const CheckLockRequest& rqst) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->check_lock(_return, rqst); + } + ifaces_[i]->check_lock(_return, rqst); + return; + } + + void unlock(const UnlockRequest& rqst) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->unlock(rqst); + } + ifaces_[i]->unlock(rqst); + } + + void show_locks(ShowLocksResponse& _return, const ShowLocksRequest& rqst) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->show_locks(_return, rqst); + } + ifaces_[i]->show_locks(_return, rqst); + return; + } + + void heartbeat(const HeartbeatRequest& ids) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->heartbeat(ids); + } + ifaces_[i]->heartbeat(ids); + } + + void heartbeat_txn_range(HeartbeatTxnRangeResponse& _return, const HeartbeatTxnRangeRequest& txns) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->heartbeat_txn_range(_return, txns); + } + ifaces_[i]->heartbeat_txn_range(_return, txns); + return; + } + + void compact(const CompactionRequest& rqst) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->compact(rqst); + } + ifaces_[i]->compact(rqst); + } + + void compact2(CompactionResponse& _return, const CompactionRequest& rqst) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->compact2(_return, rqst); + } + ifaces_[i]->compact2(_return, rqst); + return; + } + + void show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->show_compact(_return, rqst); + } + ifaces_[i]->show_compact(_return, rqst); + return; + } + + void add_dynamic_partitions(const AddDynamicPartitions& rqst) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->add_dynamic_partitions(rqst); + } + ifaces_[i]->add_dynamic_partitions(rqst); + } + + void get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_next_notification(_return, rqst); + } + ifaces_[i]->get_next_notification(_return, rqst); + return; + } + + void get_current_notificationEventId(CurrentNotificationEventId& _return) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_current_notificationEventId(_return); + } + ifaces_[i]->get_current_notificationEventId(_return); + return; + } + + void get_notification_events_count(NotificationEventsCountResponse& _return, const NotificationEventsCountRequest& rqst) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_notification_events_count(_return, rqst); + } + ifaces_[i]->get_notification_events_count(_return, rqst); + return; + } + + void fire_listener_event(FireEventResponse& _return, const FireEventRequest& rqst) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->fire_listener_event(_return, rqst); + } + ifaces_[i]->fire_listener_event(_return, rqst); + return; + } + + void flushCache() { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->flushCache(); + } + ifaces_[i]->flushCache(); + } + + void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->cm_recycle(_return, request); + } + ifaces_[i]->cm_recycle(_return, request); + return; + } + + void get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_file_metadata_by_expr(_return, req); + } + ifaces_[i]->get_file_metadata_by_expr(_return, req); + return; + } + + void get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_file_metadata(_return, req); + } + ifaces_[i]->get_file_metadata(_return, req); + return; + } + + void put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->put_file_metadata(_return, req); + } + ifaces_[i]->put_file_metadata(_return, req); + return; + } + + void clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->clear_file_metadata(_return, req); + } + ifaces_[i]->clear_file_metadata(_return, req); + return; + } + + void cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->cache_file_metadata(_return, req); + } + ifaces_[i]->cache_file_metadata(_return, req); + return; + } + + void get_metastore_db_uuid(std::string& _return) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_metastore_db_uuid(_return); + } + ifaces_[i]->get_metastore_db_uuid(_return); + return; + } + + void create_resource_plan(WMCreateResourcePlanResponse& _return, const WMCreateResourcePlanRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->create_resource_plan(_return, request); + } + ifaces_[i]->create_resource_plan(_return, request); + return; + } + + void get_resource_plan(WMGetResourcePlanResponse& _return, const WMGetResourcePlanRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_resource_plan(_return, request); + } + ifaces_[i]->get_resource_plan(_return, request); + return; + } + + void get_active_resource_plan(WMGetActiveResourcePlanResponse& _return, const WMGetActiveResourcePlanRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_active_resource_plan(_return, request); + } + ifaces_[i]->get_active_resource_plan(_return, request); + return; + } + + void get_all_resource_plans(WMGetAllResourcePlanResponse& _return, const WMGetAllResourcePlanRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_all_resource_plans(_return, request); + } + ifaces_[i]->get_all_resource_plans(_return, request); + return; + } + + void alter_resource_plan(WMAlterResourcePlanResponse& _return, const WMAlterResourcePlanRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->alter_resource_plan(_return, request); + } + ifaces_[i]->alter_resource_plan(_return, request); + return; + } + + void validate_resource_plan(WMValidateResourcePlanResponse& _return, const WMValidateResourcePlanRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->validate_resource_plan(_return, request); + } + ifaces_[i]->validate_resource_plan(_return, request); + return; + } + + void drop_resource_plan(WMDropResourcePlanResponse& _return, const WMDropResourcePlanRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->drop_resource_plan(_return, request); + } + ifaces_[i]->drop_resource_plan(_return, request); + return; + } + + void create_wm_trigger(WMCreateTriggerResponse& _return, const WMCreateTriggerRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->create_wm_trigger(_return, request); + } + ifaces_[i]->create_wm_trigger(_return, request); + return; + } + + void alter_wm_trigger(WMAlterTriggerResponse& _return, const WMAlterTriggerRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->alter_wm_trigger(_return, request); + } + ifaces_[i]->alter_wm_trigger(_return, request); + return; + } + + void drop_wm_trigger(WMDropTriggerResponse& _return, const WMDropTriggerRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->drop_wm_trigger(_return, request); + } + ifaces_[i]->drop_wm_trigger(_return, request); + return; + } + + void get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return, const WMGetTriggersForResourePlanRequest& request) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->get_triggers_for_resourceplan(_return, request); + } + ifaces_[i]->get_triggers_for_resourceplan(_return, request); + return; + } + +}; + +// The 'concurrent' client is a thread safe client that correctly handles +// out of order responses. It is slower than the regular client, so should +// only be used when you need to share a connection among multiple threads +class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf { + public: + ThriftHiveMetastoreConcurrentClient(std::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) { + setProtocol(prot); + } + ThriftHiveMetastoreConcurrentClient(std::shared_ptr< ::apache::thrift::protocol::TProtocol> iprot, std::shared_ptr< ::apache::thrift::protocol::TProtocol> oprot) { + setProtocol(iprot,oprot); + } + private: + void setProtocol(std::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) { + setProtocol(prot,prot); + } + void setProtocol(std::shared_ptr< ::apache::thrift::protocol::TProtocol> iprot, std::shared_ptr< ::apache::thrift::protocol::TProtocol> oprot) { + piprot_=iprot; + poprot_=oprot; + iprot_ = iprot.get(); + oprot_ = oprot.get(); + } + public: + std::shared_ptr< ::apache::thrift::protocol::TProtocol> getInputProtocol() { + return piprot_; + } + std::shared_ptr< ::apache::thrift::protocol::TProtocol> getOutputProtocol() { + return poprot_; + } + void getMetaConf(std::string& _return, const std::string& key); + int32_t send_getMetaConf(const std::string& key); + void recv_getMetaConf(std::string& _return, const int32_t seqid); + void setMetaConf(const std::string& key, const std::string& value); + int32_t send_setMetaConf(const std::string& key, const std::string& value); + void recv_setMetaConf(const int32_t seqid); + void create_database(const Database& database); + int32_t send_create_database(const Database& database); + void recv_create_database(const int32_t seqid); + void get_database(Database& _return, const std::string& name); + int32_t send_get_database(const std::string& name); + void recv_get_database(Database& _return, const int32_t seqid); + void drop_database(const std::string& name, const bool deleteData, const bool cascade); + int32_t send_drop_database(const std::string& name, const bool deleteData, const bool cascade); + void recv_drop_database(const int32_t seqid); + void get_databases(std::vector & _return, const std::string& pattern); + int32_t send_get_databases(const std::string& pattern); + void recv_get_databases(std::vector & _return, const int32_t seqid); + void get_all_databases(std::vector & _return); + int32_t send_get_all_databases(); + void recv_get_all_databases(std::vector & _return, const int32_t seqid); + void alter_database(const std::string& dbname, const Database& db); + int32_t send_alter_database(const std::string& dbname, const Database& db); + void recv_alter_database(const int32_t seqid); + void get_type(Type& _return, const std::string& name); + int32_t send_get_type(const std::string& name); + void recv_get_type(Type& _return, const int32_t seqid); + bool create_type(const Type& type); + int32_t send_create_type(const Type& type); + bool recv_create_type(const int32_t seqid); + bool drop_type(const std::string& type); + int32_t send_drop_type(const std::string& type); + bool recv_drop_type(const int32_t seqid); + void get_type_all(std::map & _return, const std::string& name); + int32_t send_get_type_all(const std::string& name); + void recv_get_type_all(std::map & _return, const int32_t seqid); + void get_fields(std::vector & _return, const std::string& db_name, const std::string& table_name); + int32_t send_get_fields(const std::string& db_name, const std::string& table_name); + void recv_get_fields(std::vector & _return, const int32_t seqid); + void get_fields_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context); + int32_t send_get_fields_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context); + void recv_get_fields_with_environment_context(std::vector & _return, const int32_t seqid); + void get_schema(std::vector & _return, const std::string& db_name, const std::string& table_name); + int32_t send_get_schema(const std::string& db_name, const std::string& table_name); + void recv_get_schema(std::vector & _return, const int32_t seqid); + void get_schema_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context); + int32_t send_get_schema_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context); + void recv_get_schema_with_environment_context(std::vector & _return, const int32_t seqid); + void create_table(const Table& tbl); + int32_t send_create_table(const Table& tbl); + void recv_create_table(const int32_t seqid); + void create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context); + int32_t send_create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context); + void recv_create_table_with_environment_context(const int32_t seqid); + void create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints); + int32_t send_create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints); + void recv_create_table_with_constraints(const int32_t seqid); + void drop_constraint(const DropConstraintRequest& req); + int32_t send_drop_constraint(const DropConstraintRequest& req); + void recv_drop_constraint(const int32_t seqid); + void add_primary_key(const AddPrimaryKeyRequest& req); + int32_t send_add_primary_key(const AddPrimaryKeyRequest& req); + void recv_add_primary_key(const int32_t seqid); + void add_foreign_key(const AddForeignKeyRequest& req); + int32_t send_add_foreign_key(const AddForeignKeyRequest& req); + void recv_add_foreign_key(const int32_t seqid); + void add_unique_constraint(const AddUniqueConstraintRequest& req); + int32_t send_add_unique_constraint(const AddUniqueConstraintRequest& req); + void recv_add_unique_constraint(const int32_t seqid); + void add_not_null_constraint(const AddNotNullConstraintRequest& req); + int32_t send_add_not_null_constraint(const AddNotNullConstraintRequest& req); + void recv_add_not_null_constraint(const int32_t seqid); + void drop_table(const std::string& dbname, const std::string& name, const bool deleteData); + int32_t send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData); + void recv_drop_table(const int32_t seqid); + void drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context); + int32_t send_drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context); + void recv_drop_table_with_environment_context(const int32_t seqid); + void truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames); + int32_t send_truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames); + void recv_truncate_table(const int32_t seqid); + void get_tables(std::vector & _return, const std::string& db_name, const std::string& pattern); + int32_t send_get_tables(const std::string& db_name, const std::string& pattern); + void recv_get_tables(std::vector & _return, const int32_t seqid); + void get_tables_by_type(std::vector & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType); + int32_t send_get_tables_by_type(const std::string& db_name, const std::string& pattern, const std::string& tableType); + void recv_get_tables_by_type(std::vector & _return, const int32_t seqid); + void get_table_meta(std::vector & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types); + int32_t send_get_table_meta(const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types); + void recv_get_table_meta(std::vector & _return, const int32_t seqid); + void get_all_tables(std::vector & _return, const std::string& db_name); + int32_t send_get_all_tables(const std::string& db_name); + void recv_get_all_tables(std::vector & _return, const int32_t seqid); + void get_table(Table& _return, const std::string& dbname, const std::string& tbl_name); + int32_t send_get_table(const std::string& dbname, const std::string& tbl_name); + void recv_get_table(Table& _return, const int32_t seqid); + void get_table_objects_by_name(std::vector
& _return, const std::string& dbname, const std::vector & tbl_names); + int32_t send_get_table_objects_by_name(const std::string& dbname, const std::vector & tbl_names); + void recv_get_table_objects_by_name(std::vector
& _return, const int32_t seqid); + void get_table_req(GetTableResult& _return, const GetTableRequest& req); + int32_t send_get_table_req(const GetTableRequest& req); + void recv_get_table_req(GetTableResult& _return, const int32_t seqid); + void get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req); + int32_t send_get_table_objects_by_name_req(const GetTablesRequest& req); + void recv_get_table_objects_by_name_req(GetTablesResult& _return, const int32_t seqid); + void get_table_names_by_filter(std::vector & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables); + int32_t send_get_table_names_by_filter(const std::string& dbname, const std::string& filter, const int16_t max_tables); + void recv_get_table_names_by_filter(std::vector & _return, const int32_t seqid); + void alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl); + int32_t send_alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl); + void recv_alter_table(const int32_t seqid); + void alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context); + int32_t send_alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context); + void recv_alter_table_with_environment_context(const int32_t seqid); + void alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade); + int32_t send_alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade); + void recv_alter_table_with_cascade(const int32_t seqid); + void add_partition(Partition& _return, const Partition& new_part); + int32_t send_add_partition(const Partition& new_part); + void recv_add_partition(Partition& _return, const int32_t seqid); + void add_partition_with_environment_context(Partition& _return, const Partition& new_part, const EnvironmentContext& environment_context); + int32_t send_add_partition_with_environment_context(const Partition& new_part, const EnvironmentContext& environment_context); + void recv_add_partition_with_environment_context(Partition& _return, const int32_t seqid); + int32_t add_partitions(const std::vector & new_parts); + int32_t send_add_partitions(const std::vector & new_parts); + int32_t recv_add_partitions(const int32_t seqid); + int32_t add_partitions_pspec(const std::vector & new_parts); + int32_t send_add_partitions_pspec(const std::vector & new_parts); + int32_t recv_add_partitions_pspec(const int32_t seqid); + void append_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals); + int32_t send_append_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals); + void recv_append_partition(Partition& _return, const int32_t seqid); + void add_partitions_req(AddPartitionsResult& _return, const AddPartitionsRequest& request); + int32_t send_add_partitions_req(const AddPartitionsRequest& request); + void recv_add_partitions_req(AddPartitionsResult& _return, const int32_t seqid); + void append_partition_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context); + int32_t send_append_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context); + void recv_append_partition_with_environment_context(Partition& _return, const int32_t seqid); + void append_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name); + int32_t send_append_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name); + void recv_append_partition_by_name(Partition& _return, const int32_t seqid); + void append_partition_by_name_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context); + int32_t send_append_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context); + void recv_append_partition_by_name_with_environment_context(Partition& _return, const int32_t seqid); + bool drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData); + int32_t send_drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData); + bool recv_drop_partition(const int32_t seqid); + bool drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context); + int32_t send_drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context); + bool recv_drop_partition_with_environment_context(const int32_t seqid); + bool drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData); + int32_t send_drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData); + bool recv_drop_partition_by_name(const int32_t seqid); + bool drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context); + int32_t send_drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context); + bool recv_drop_partition_by_name_with_environment_context(const int32_t seqid); + void drop_partitions_req(DropPartitionsResult& _return, const DropPartitionsRequest& req); + int32_t send_drop_partitions_req(const DropPartitionsRequest& req); + void recv_drop_partitions_req(DropPartitionsResult& _return, const int32_t seqid); + void get_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals); + int32_t send_get_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals); + void recv_get_partition(Partition& _return, const int32_t seqid); + void exchange_partition(Partition& _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name); + int32_t send_exchange_partition(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name); + void recv_exchange_partition(Partition& _return, const int32_t seqid); + void exchange_partitions(std::vector & _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name); + int32_t send_exchange_partitions(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name); + void recv_exchange_partitions(std::vector & _return, const int32_t seqid); + void get_partition_with_auth(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names); + int32_t send_get_partition_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names); + void recv_get_partition_with_auth(Partition& _return, const int32_t seqid); + void get_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name); + int32_t send_get_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name); + void recv_get_partition_by_name(Partition& _return, const int32_t seqid); + void get_partitions(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts); + int32_t send_get_partitions(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts); + void recv_get_partitions(std::vector & _return, const int32_t seqid); + void get_partitions_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names); + int32_t send_get_partitions_with_auth(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names); + void recv_get_partitions_with_auth(std::vector & _return, const int32_t seqid); + void get_partitions_pspec(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int32_t max_parts); + int32_t send_get_partitions_pspec(const std::string& db_name, const std::string& tbl_name, const int32_t max_parts); + void recv_get_partitions_pspec(std::vector & _return, const int32_t seqid); + void get_partition_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts); + int32_t send_get_partition_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts); + void recv_get_partition_names(std::vector & _return, const int32_t seqid); + void get_partition_values(PartitionValuesResponse& _return, const PartitionValuesRequest& request); + int32_t send_get_partition_values(const PartitionValuesRequest& request); + void recv_get_partition_values(PartitionValuesResponse& _return, const int32_t seqid); + void get_partitions_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts); + int32_t send_get_partitions_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts); + void recv_get_partitions_ps(std::vector & _return, const int32_t seqid); + void get_partitions_ps_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names); + int32_t send_get_partitions_ps_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names); + void recv_get_partitions_ps_with_auth(std::vector & _return, const int32_t seqid); + void get_partition_names_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts); + int32_t send_get_partition_names_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts); + void recv_get_partition_names_ps(std::vector & _return, const int32_t seqid); + void get_partitions_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts); + int32_t send_get_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts); + void recv_get_partitions_by_filter(std::vector & _return, const int32_t seqid); + void get_part_specs_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts); + int32_t send_get_part_specs_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts); + void recv_get_part_specs_by_filter(std::vector & _return, const int32_t seqid); + void get_partitions_by_expr(PartitionsByExprResult& _return, const PartitionsByExprRequest& req); + int32_t send_get_partitions_by_expr(const PartitionsByExprRequest& req); + void recv_get_partitions_by_expr(PartitionsByExprResult& _return, const int32_t seqid); + int32_t get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter); + int32_t send_get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter); + int32_t recv_get_num_partitions_by_filter(const int32_t seqid); + void get_partitions_by_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & names); + int32_t send_get_partitions_by_names(const std::string& db_name, const std::string& tbl_name, const std::vector & names); + void recv_get_partitions_by_names(std::vector & _return, const int32_t seqid); + void alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part); + int32_t send_alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part); + void recv_alter_partition(const int32_t seqid); + void alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts); + int32_t send_alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts); + void recv_alter_partitions(const int32_t seqid); + void alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context); + int32_t send_alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context); + void recv_alter_partitions_with_environment_context(const int32_t seqid); + void alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context); + int32_t send_alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context); + void recv_alter_partition_with_environment_context(const int32_t seqid); + void rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part); + int32_t send_rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part); + void recv_rename_partition(const int32_t seqid); + bool partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception); + int32_t send_partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception); + bool recv_partition_name_has_valid_characters(const int32_t seqid); + void get_config_value(std::string& _return, const std::string& name, const std::string& defaultValue); + int32_t send_get_config_value(const std::string& name, const std::string& defaultValue); + void recv_get_config_value(std::string& _return, const int32_t seqid); + void partition_name_to_vals(std::vector & _return, const std::string& part_name); + int32_t send_partition_name_to_vals(const std::string& part_name); + void recv_partition_name_to_vals(std::vector & _return, const int32_t seqid); + void partition_name_to_spec(std::map & _return, const std::string& part_name); + int32_t send_partition_name_to_spec(const std::string& part_name); + void recv_partition_name_to_spec(std::map & _return, const int32_t seqid); + void markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType); + int32_t send_markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType); + void recv_markPartitionForEvent(const int32_t seqid); + bool isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType); + int32_t send_isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType); + bool recv_isPartitionMarkedForEvent(const int32_t seqid); + void add_index(Index& _return, const Index& new_index, const Table& index_table); + int32_t send_add_index(const Index& new_index, const Table& index_table); + void recv_add_index(Index& _return, const int32_t seqid); + void alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx); + int32_t send_alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx); + void recv_alter_index(const int32_t seqid); + bool drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData); + int32_t send_drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData); + bool recv_drop_index_by_name(const int32_t seqid); + void get_index_by_name(Index& _return, const std::string& db_name, const std::string& tbl_name, const std::string& index_name); + int32_t send_get_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name); + void recv_get_index_by_name(Index& _return, const int32_t seqid); + void get_indexes(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes); + int32_t send_get_indexes(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes); + void recv_get_indexes(std::vector & _return, const int32_t seqid); + void get_index_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes); + int32_t send_get_index_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes); + void recv_get_index_names(std::vector & _return, const int32_t seqid); + void get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request); + int32_t send_get_primary_keys(const PrimaryKeysRequest& request); + void recv_get_primary_keys(PrimaryKeysResponse& _return, const int32_t seqid); + void get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request); + int32_t send_get_foreign_keys(const ForeignKeysRequest& request); + void recv_get_foreign_keys(ForeignKeysResponse& _return, const int32_t seqid); + void get_unique_constraints(UniqueConstraintsResponse& _return, const UniqueConstraintsRequest& request); + int32_t send_get_unique_constraints(const UniqueConstraintsRequest& request); + void recv_get_unique_constraints(UniqueConstraintsResponse& _return, const int32_t seqid); + void get_not_null_constraints(NotNullConstraintsResponse& _return, const NotNullConstraintsRequest& request); + int32_t send_get_not_null_constraints(const NotNullConstraintsRequest& request); + void recv_get_not_null_constraints(NotNullConstraintsResponse& _return, const int32_t seqid); + bool update_table_column_statistics(const ColumnStatistics& stats_obj); + int32_t send_update_table_column_statistics(const ColumnStatistics& stats_obj); + bool recv_update_table_column_statistics(const int32_t seqid); + bool update_partition_column_statistics(const ColumnStatistics& stats_obj); + int32_t send_update_partition_column_statistics(const ColumnStatistics& stats_obj); + bool recv_update_partition_column_statistics(const int32_t seqid); + void get_table_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& col_name); + int32_t send_get_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name); + void recv_get_table_column_statistics(ColumnStatistics& _return, const int32_t seqid); + void get_partition_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name); + int32_t send_get_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name); + void recv_get_partition_column_statistics(ColumnStatistics& _return, const int32_t seqid); + void get_table_statistics_req(TableStatsResult& _return, const TableStatsRequest& request); + int32_t send_get_table_statistics_req(const TableStatsRequest& request); + void recv_get_table_statistics_req(TableStatsResult& _return, const int32_t seqid); + void get_partitions_statistics_req(PartitionsStatsResult& _return, const PartitionsStatsRequest& request); + int32_t send_get_partitions_statistics_req(const PartitionsStatsRequest& request); + void recv_get_partitions_statistics_req(PartitionsStatsResult& _return, const int32_t seqid); + void get_aggr_stats_for(AggrStats& _return, const PartitionsStatsRequest& request); + int32_t send_get_aggr_stats_for(const PartitionsStatsRequest& request); + void recv_get_aggr_stats_for(AggrStats& _return, const int32_t seqid); + bool set_aggr_stats_for(const SetPartitionsStatsRequest& request); + int32_t send_set_aggr_stats_for(const SetPartitionsStatsRequest& request); + bool recv_set_aggr_stats_for(const int32_t seqid); + bool delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name); + int32_t send_delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name); + bool recv_delete_partition_column_statistics(const int32_t seqid); + bool delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name); + int32_t send_delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name); + bool recv_delete_table_column_statistics(const int32_t seqid); + void create_function(const Function& func); + int32_t send_create_function(const Function& func); + void recv_create_function(const int32_t seqid); + void drop_function(const std::string& dbName, const std::string& funcName); + int32_t send_drop_function(const std::string& dbName, const std::string& funcName); + void recv_drop_function(const int32_t seqid); + void alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc); + int32_t send_alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc); + void recv_alter_function(const int32_t seqid); + void get_functions(std::vector & _return, const std::string& dbName, const std::string& pattern); + int32_t send_get_functions(const std::string& dbName, const std::string& pattern); + void recv_get_functions(std::vector & _return, const int32_t seqid); + void get_function(Function& _return, const std::string& dbName, const std::string& funcName); + int32_t send_get_function(const std::string& dbName, const std::string& funcName); + void recv_get_function(Function& _return, const int32_t seqid); + void get_all_functions(GetAllFunctionsResponse& _return); + int32_t send_get_all_functions(); + void recv_get_all_functions(GetAllFunctionsResponse& _return, const int32_t seqid); + bool create_role(const Role& role); + int32_t send_create_role(const Role& role); + bool recv_create_role(const int32_t seqid); + bool drop_role(const std::string& role_name); + int32_t send_drop_role(const std::string& role_name); + bool recv_drop_role(const int32_t seqid); + void get_role_names(std::vector & _return); + int32_t send_get_role_names(); + void recv_get_role_names(std::vector & _return, const int32_t seqid); + bool grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option); + int32_t send_grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option); + bool recv_grant_role(const int32_t seqid); + bool revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type); + int32_t send_revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type); + bool recv_revoke_role(const int32_t seqid); + void list_roles(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type); + int32_t send_list_roles(const std::string& principal_name, const PrincipalType::type principal_type); + void recv_list_roles(std::vector & _return, const int32_t seqid); + void grant_revoke_role(GrantRevokeRoleResponse& _return, const GrantRevokeRoleRequest& request); + int32_t send_grant_revoke_role(const GrantRevokeRoleRequest& request); + void recv_grant_revoke_role(GrantRevokeRoleResponse& _return, const int32_t seqid); + void get_principals_in_role(GetPrincipalsInRoleResponse& _return, const GetPrincipalsInRoleRequest& request); + int32_t send_get_principals_in_role(const GetPrincipalsInRoleRequest& request); + void recv_get_principals_in_role(GetPrincipalsInRoleResponse& _return, const int32_t seqid); + void get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return, const GetRoleGrantsForPrincipalRequest& request); + int32_t send_get_role_grants_for_principal(const GetRoleGrantsForPrincipalRequest& request); + void recv_get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return, const int32_t seqid); + void get_privilege_set(PrincipalPrivilegeSet& _return, const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names); + int32_t send_get_privilege_set(const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names); + void recv_get_privilege_set(PrincipalPrivilegeSet& _return, const int32_t seqid); + void list_privileges(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject); + int32_t send_list_privileges(const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject); + void recv_list_privileges(std::vector & _return, const int32_t seqid); + bool grant_privileges(const PrivilegeBag& privileges); + int32_t send_grant_privileges(const PrivilegeBag& privileges); + bool recv_grant_privileges(const int32_t seqid); + bool revoke_privileges(const PrivilegeBag& privileges); + int32_t send_revoke_privileges(const PrivilegeBag& privileges); + bool recv_revoke_privileges(const int32_t seqid); + void grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const GrantRevokePrivilegeRequest& request); + int32_t send_grant_revoke_privileges(const GrantRevokePrivilegeRequest& request); + void recv_grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const int32_t seqid); + void set_ugi(std::vector & _return, const std::string& user_name, const std::vector & group_names); + int32_t send_set_ugi(const std::string& user_name, const std::vector & group_names); + void recv_set_ugi(std::vector & _return, const int32_t seqid); + void get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name); + int32_t send_get_delegation_token(const std::string& token_owner, const std::string& renewer_kerberos_principal_name); + void recv_get_delegation_token(std::string& _return, const int32_t seqid); + int64_t renew_delegation_token(const std::string& token_str_form); + int32_t send_renew_delegation_token(const std::string& token_str_form); + int64_t recv_renew_delegation_token(const int32_t seqid); + void cancel_delegation_token(const std::string& token_str_form); + int32_t send_cancel_delegation_token(const std::string& token_str_form); + void recv_cancel_delegation_token(const int32_t seqid); + bool add_token(const std::string& token_identifier, const std::string& delegation_token); + int32_t send_add_token(const std::string& token_identifier, const std::string& delegation_token); + bool recv_add_token(const int32_t seqid); + bool remove_token(const std::string& token_identifier); + int32_t send_remove_token(const std::string& token_identifier); + bool recv_remove_token(const int32_t seqid); + void get_token(std::string& _return, const std::string& token_identifier); + int32_t send_get_token(const std::string& token_identifier); + void recv_get_token(std::string& _return, const int32_t seqid); + void get_all_token_identifiers(std::vector & _return); + int32_t send_get_all_token_identifiers(); + void recv_get_all_token_identifiers(std::vector & _return, const int32_t seqid); + int32_t add_master_key(const std::string& key); + int32_t send_add_master_key(const std::string& key); + int32_t recv_add_master_key(const int32_t seqid); + void update_master_key(const int32_t seq_number, const std::string& key); + int32_t send_update_master_key(const int32_t seq_number, const std::string& key); + void recv_update_master_key(const int32_t seqid); + bool remove_master_key(const int32_t key_seq); + int32_t send_remove_master_key(const int32_t key_seq); + bool recv_remove_master_key(const int32_t seqid); + void get_master_keys(std::vector & _return); + int32_t send_get_master_keys(); + void recv_get_master_keys(std::vector & _return, const int32_t seqid); + void get_open_txns(GetOpenTxnsResponse& _return); + int32_t send_get_open_txns(); + void recv_get_open_txns(GetOpenTxnsResponse& _return, const int32_t seqid); + void get_open_txns_info(GetOpenTxnsInfoResponse& _return); + int32_t send_get_open_txns_info(); + void recv_get_open_txns_info(GetOpenTxnsInfoResponse& _return, const int32_t seqid); + void open_txns(OpenTxnsResponse& _return, const OpenTxnRequest& rqst); + int32_t send_open_txns(const OpenTxnRequest& rqst); + void recv_open_txns(OpenTxnsResponse& _return, const int32_t seqid); + void abort_txn(const AbortTxnRequest& rqst); + int32_t send_abort_txn(const AbortTxnRequest& rqst); + void recv_abort_txn(const int32_t seqid); + void abort_txns(const AbortTxnsRequest& rqst); + int32_t send_abort_txns(const AbortTxnsRequest& rqst); + void recv_abort_txns(const int32_t seqid); + void commit_txn(const CommitTxnRequest& rqst); + int32_t send_commit_txn(const CommitTxnRequest& rqst); + void recv_commit_txn(const int32_t seqid); + void lock(LockResponse& _return, const LockRequest& rqst); + int32_t send_lock(const LockRequest& rqst); + void recv_lock(LockResponse& _return, const int32_t seqid); + void check_lock(LockResponse& _return, const CheckLockRequest& rqst); + int32_t send_check_lock(const CheckLockRequest& rqst); + void recv_check_lock(LockResponse& _return, const int32_t seqid); + void unlock(const UnlockRequest& rqst); + int32_t send_unlock(const UnlockRequest& rqst); + void recv_unlock(const int32_t seqid); + void show_locks(ShowLocksResponse& _return, const ShowLocksRequest& rqst); + int32_t send_show_locks(const ShowLocksRequest& rqst); + void recv_show_locks(ShowLocksResponse& _return, const int32_t seqid); + void heartbeat(const HeartbeatRequest& ids); + int32_t send_heartbeat(const HeartbeatRequest& ids); + void recv_heartbeat(const int32_t seqid); + void heartbeat_txn_range(HeartbeatTxnRangeResponse& _return, const HeartbeatTxnRangeRequest& txns); + int32_t send_heartbeat_txn_range(const HeartbeatTxnRangeRequest& txns); + void recv_heartbeat_txn_range(HeartbeatTxnRangeResponse& _return, const int32_t seqid); + void compact(const CompactionRequest& rqst); + int32_t send_compact(const CompactionRequest& rqst); + void recv_compact(const int32_t seqid); + void compact2(CompactionResponse& _return, const CompactionRequest& rqst); + int32_t send_compact2(const CompactionRequest& rqst); + void recv_compact2(CompactionResponse& _return, const int32_t seqid); + void show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst); + int32_t send_show_compact(const ShowCompactRequest& rqst); + void recv_show_compact(ShowCompactResponse& _return, const int32_t seqid); + void add_dynamic_partitions(const AddDynamicPartitions& rqst); + int32_t send_add_dynamic_partitions(const AddDynamicPartitions& rqst); + void recv_add_dynamic_partitions(const int32_t seqid); + void get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst); + int32_t send_get_next_notification(const NotificationEventRequest& rqst); + void recv_get_next_notification(NotificationEventResponse& _return, const int32_t seqid); + void get_current_notificationEventId(CurrentNotificationEventId& _return); + int32_t send_get_current_notificationEventId(); + void recv_get_current_notificationEventId(CurrentNotificationEventId& _return, const int32_t seqid); + void get_notification_events_count(NotificationEventsCountResponse& _return, const NotificationEventsCountRequest& rqst); + int32_t send_get_notification_events_count(const NotificationEventsCountRequest& rqst); + void recv_get_notification_events_count(NotificationEventsCountResponse& _return, const int32_t seqid); + void fire_listener_event(FireEventResponse& _return, const FireEventRequest& rqst); + int32_t send_fire_listener_event(const FireEventRequest& rqst); + void recv_fire_listener_event(FireEventResponse& _return, const int32_t seqid); + void flushCache(); + int32_t send_flushCache(); + void recv_flushCache(const int32_t seqid); + void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request); + int32_t send_cm_recycle(const CmRecycleRequest& request); + void recv_cm_recycle(CmRecycleResponse& _return, const int32_t seqid); + void get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req); + int32_t send_get_file_metadata_by_expr(const GetFileMetadataByExprRequest& req); + void recv_get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const int32_t seqid); + void get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req); + int32_t send_get_file_metadata(const GetFileMetadataRequest& req); + void recv_get_file_metadata(GetFileMetadataResult& _return, const int32_t seqid); + void put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req); + int32_t send_put_file_metadata(const PutFileMetadataRequest& req); + void recv_put_file_metadata(PutFileMetadataResult& _return, const int32_t seqid); + void clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req); + int32_t send_clear_file_metadata(const ClearFileMetadataRequest& req); + void recv_clear_file_metadata(ClearFileMetadataResult& _return, const int32_t seqid); + void cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req); + int32_t send_cache_file_metadata(const CacheFileMetadataRequest& req); + void recv_cache_file_metadata(CacheFileMetadataResult& _return, const int32_t seqid); + void get_metastore_db_uuid(std::string& _return); + int32_t send_get_metastore_db_uuid(); + void recv_get_metastore_db_uuid(std::string& _return, const int32_t seqid); + void create_resource_plan(WMCreateResourcePlanResponse& _return, const WMCreateResourcePlanRequest& request); + int32_t send_create_resource_plan(const WMCreateResourcePlanRequest& request); + void recv_create_resource_plan(WMCreateResourcePlanResponse& _return, const int32_t seqid); + void get_resource_plan(WMGetResourcePlanResponse& _return, const WMGetResourcePlanRequest& request); + int32_t send_get_resource_plan(const WMGetResourcePlanRequest& request); + void recv_get_resource_plan(WMGetResourcePlanResponse& _return, const int32_t seqid); + void get_active_resource_plan(WMGetActiveResourcePlanResponse& _return, const WMGetActiveResourcePlanRequest& request); + int32_t send_get_active_resource_plan(const WMGetActiveResourcePlanRequest& request); + void recv_get_active_resource_plan(WMGetActiveResourcePlanResponse& _return, const int32_t seqid); + void get_all_resource_plans(WMGetAllResourcePlanResponse& _return, const WMGetAllResourcePlanRequest& request); + int32_t send_get_all_resource_plans(const WMGetAllResourcePlanRequest& request); + void recv_get_all_resource_plans(WMGetAllResourcePlanResponse& _return, const int32_t seqid); + void alter_resource_plan(WMAlterResourcePlanResponse& _return, const WMAlterResourcePlanRequest& request); + int32_t send_alter_resource_plan(const WMAlterResourcePlanRequest& request); + void recv_alter_resource_plan(WMAlterResourcePlanResponse& _return, const int32_t seqid); + void validate_resource_plan(WMValidateResourcePlanResponse& _return, const WMValidateResourcePlanRequest& request); + int32_t send_validate_resource_plan(const WMValidateResourcePlanRequest& request); + void recv_validate_resource_plan(WMValidateResourcePlanResponse& _return, const int32_t seqid); + void drop_resource_plan(WMDropResourcePlanResponse& _return, const WMDropResourcePlanRequest& request); + int32_t send_drop_resource_plan(const WMDropResourcePlanRequest& request); + void recv_drop_resource_plan(WMDropResourcePlanResponse& _return, const int32_t seqid); + void create_wm_trigger(WMCreateTriggerResponse& _return, const WMCreateTriggerRequest& request); + int32_t send_create_wm_trigger(const WMCreateTriggerRequest& request); + void recv_create_wm_trigger(WMCreateTriggerResponse& _return, const int32_t seqid); + void alter_wm_trigger(WMAlterTriggerResponse& _return, const WMAlterTriggerRequest& request); + int32_t send_alter_wm_trigger(const WMAlterTriggerRequest& request); + void recv_alter_wm_trigger(WMAlterTriggerResponse& _return, const int32_t seqid); + void drop_wm_trigger(WMDropTriggerResponse& _return, const WMDropTriggerRequest& request); + int32_t send_drop_wm_trigger(const WMDropTriggerRequest& request); + void recv_drop_wm_trigger(WMDropTriggerResponse& _return, const int32_t seqid); + void get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return, const WMGetTriggersForResourePlanRequest& request); + int32_t send_get_triggers_for_resourceplan(const WMGetTriggersForResourePlanRequest& request); + void recv_get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return, const int32_t seqid); + protected: + std::shared_ptr< ::apache::thrift::protocol::TProtocol> piprot_; + std::shared_ptr< ::apache::thrift::protocol::TProtocol> poprot_; + ::apache::thrift::protocol::TProtocol* iprot_; + ::apache::thrift::protocol::TProtocol* oprot_; + ::apache::thrift::async::TConcurrentClientSyncInfo sync_; +}; + +#ifdef _MSC_VER + #pragma warning( pop ) +#endif + +}}} // namespace + +#endif diff --git a/contrib/hive-metastore/hive_metastore_constants.cpp b/contrib/hive-metastore/hive_metastore_constants.cpp new file mode 100644 index 00000000000..517d771efce --- /dev/null +++ b/contrib/hive-metastore/hive_metastore_constants.cpp @@ -0,0 +1,65 @@ +/** + * Autogenerated by Thrift Compiler () + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +#include "hive_metastore_constants.h" + +namespace Apache { namespace Hadoop { namespace Hive { + +const hive_metastoreConstants g_hive_metastore_constants; + +hive_metastoreConstants::hive_metastoreConstants() { + DDL_TIME = "transient_lastDdlTime"; + + HIVE_FILTER_FIELD_OWNER = "hive_filter_field_owner__"; + + HIVE_FILTER_FIELD_PARAMS = "hive_filter_field_params__"; + + HIVE_FILTER_FIELD_LAST_ACCESS = "hive_filter_field_last_access__"; + + IS_ARCHIVED = "is_archived"; + + ORIGINAL_LOCATION = "original_location"; + + IS_IMMUTABLE = "immutable"; + + META_TABLE_COLUMNS = "columns"; + + META_TABLE_COLUMN_TYPES = "columns.types"; + + BUCKET_FIELD_NAME = "bucket_field_name"; + + BUCKET_COUNT = "bucket_count"; + + FIELD_TO_DIMENSION = "field_to_dimension"; + + META_TABLE_NAME = "name"; + + META_TABLE_DB = "db"; + + META_TABLE_LOCATION = "location"; + + META_TABLE_SERDE = "serde"; + + META_TABLE_PARTITION_COLUMNS = "partition_columns"; + + META_TABLE_PARTITION_COLUMN_TYPES = "partition_columns.types"; + + FILE_INPUT_FORMAT = "file.inputformat"; + + FILE_OUTPUT_FORMAT = "file.outputformat"; + + META_TABLE_STORAGE = "storage_handler"; + + TABLE_IS_TRANSACTIONAL = "transactional"; + + TABLE_NO_AUTO_COMPACT = "no_auto_compaction"; + + TABLE_TRANSACTIONAL_PROPERTIES = "transactional_properties"; + +} + +}}} // namespace + diff --git a/contrib/hive-metastore/hive_metastore_constants.h b/contrib/hive-metastore/hive_metastore_constants.h new file mode 100644 index 00000000000..68e8bea72e2 --- /dev/null +++ b/contrib/hive-metastore/hive_metastore_constants.h @@ -0,0 +1,48 @@ +/** + * Autogenerated by Thrift Compiler () + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +#ifndef hive_metastore_CONSTANTS_H +#define hive_metastore_CONSTANTS_H + +#include "hive_metastore_types.h" + +namespace Apache { namespace Hadoop { namespace Hive { + +class hive_metastoreConstants { + public: + hive_metastoreConstants(); + + std::string DDL_TIME; + std::string HIVE_FILTER_FIELD_OWNER; + std::string HIVE_FILTER_FIELD_PARAMS; + std::string HIVE_FILTER_FIELD_LAST_ACCESS; + std::string IS_ARCHIVED; + std::string ORIGINAL_LOCATION; + std::string IS_IMMUTABLE; + std::string META_TABLE_COLUMNS; + std::string META_TABLE_COLUMN_TYPES; + std::string BUCKET_FIELD_NAME; + std::string BUCKET_COUNT; + std::string FIELD_TO_DIMENSION; + std::string META_TABLE_NAME; + std::string META_TABLE_DB; + std::string META_TABLE_LOCATION; + std::string META_TABLE_SERDE; + std::string META_TABLE_PARTITION_COLUMNS; + std::string META_TABLE_PARTITION_COLUMN_TYPES; + std::string FILE_INPUT_FORMAT; + std::string FILE_OUTPUT_FORMAT; + std::string META_TABLE_STORAGE; + std::string TABLE_IS_TRANSACTIONAL; + std::string TABLE_NO_AUTO_COMPACT; + std::string TABLE_TRANSACTIONAL_PROPERTIES; +}; + +extern const hive_metastoreConstants g_hive_metastore_constants; + +}}} // namespace + +#endif diff --git a/contrib/hive-metastore/hive_metastore_types.cpp b/contrib/hive-metastore/hive_metastore_types.cpp new file mode 100644 index 00000000000..e4d820f1863 --- /dev/null +++ b/contrib/hive-metastore/hive_metastore_types.cpp @@ -0,0 +1,26465 @@ +/** + * Autogenerated by Thrift Compiler () + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +#include "hive_metastore_types.h" + +#include +#include + +#include + +namespace Apache { namespace Hadoop { namespace Hive { + +int _kHiveObjectTypeValues[] = { + HiveObjectType::GLOBAL, + HiveObjectType::DATABASE, + HiveObjectType::TABLE, + HiveObjectType::PARTITION, + HiveObjectType::COLUMN +}; +const char* _kHiveObjectTypeNames[] = { + "GLOBAL", + "DATABASE", + "TABLE", + "PARTITION", + "COLUMN" +}; +const std::map _HiveObjectType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(5, _kHiveObjectTypeValues, _kHiveObjectTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); + +std::ostream& operator<<(std::ostream& out, const HiveObjectType::type& val) { + std::map::const_iterator it = _HiveObjectType_VALUES_TO_NAMES.find(val); + if (it != _HiveObjectType_VALUES_TO_NAMES.end()) { + out << it->second; + } else { + out << static_cast(val); + } + return out; +} + +int _kPrincipalTypeValues[] = { + PrincipalType::USER, + PrincipalType::ROLE, + PrincipalType::GROUP +}; +const char* _kPrincipalTypeNames[] = { + "USER", + "ROLE", + "GROUP" +}; +const std::map _PrincipalType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kPrincipalTypeValues, _kPrincipalTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); + +std::ostream& operator<<(std::ostream& out, const PrincipalType::type& val) { + std::map::const_iterator it = _PrincipalType_VALUES_TO_NAMES.find(val); + if (it != _PrincipalType_VALUES_TO_NAMES.end()) { + out << it->second; + } else { + out << static_cast(val); + } + return out; +} + +int _kPartitionEventTypeValues[] = { + PartitionEventType::LOAD_DONE +}; +const char* _kPartitionEventTypeNames[] = { + "LOAD_DONE" +}; +const std::map _PartitionEventType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(1, _kPartitionEventTypeValues, _kPartitionEventTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); + +std::ostream& operator<<(std::ostream& out, const PartitionEventType::type& val) { + std::map::const_iterator it = _PartitionEventType_VALUES_TO_NAMES.find(val); + if (it != _PartitionEventType_VALUES_TO_NAMES.end()) { + out << it->second; + } else { + out << static_cast(val); + } + return out; +} + +int _kTxnStateValues[] = { + TxnState::COMMITTED, + TxnState::ABORTED, + TxnState::OPEN +}; +const char* _kTxnStateNames[] = { + "COMMITTED", + "ABORTED", + "OPEN" +}; +const std::map _TxnState_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kTxnStateValues, _kTxnStateNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); + +std::ostream& operator<<(std::ostream& out, const TxnState::type& val) { + std::map::const_iterator it = _TxnState_VALUES_TO_NAMES.find(val); + if (it != _TxnState_VALUES_TO_NAMES.end()) { + out << it->second; + } else { + out << static_cast(val); + } + return out; +} + +int _kLockLevelValues[] = { + LockLevel::DB, + LockLevel::TABLE, + LockLevel::PARTITION +}; +const char* _kLockLevelNames[] = { + "DB", + "TABLE", + "PARTITION" +}; +const std::map _LockLevel_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kLockLevelValues, _kLockLevelNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); + +std::ostream& operator<<(std::ostream& out, const LockLevel::type& val) { + std::map::const_iterator it = _LockLevel_VALUES_TO_NAMES.find(val); + if (it != _LockLevel_VALUES_TO_NAMES.end()) { + out << it->second; + } else { + out << static_cast(val); + } + return out; +} + +int _kLockStateValues[] = { + LockState::ACQUIRED, + LockState::WAITING, + LockState::ABORT, + LockState::NOT_ACQUIRED +}; +const char* _kLockStateNames[] = { + "ACQUIRED", + "WAITING", + "ABORT", + "NOT_ACQUIRED" +}; +const std::map _LockState_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(4, _kLockStateValues, _kLockStateNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); + +std::ostream& operator<<(std::ostream& out, const LockState::type& val) { + std::map::const_iterator it = _LockState_VALUES_TO_NAMES.find(val); + if (it != _LockState_VALUES_TO_NAMES.end()) { + out << it->second; + } else { + out << static_cast(val); + } + return out; +} + +int _kLockTypeValues[] = { + LockType::SHARED_READ, + LockType::SHARED_WRITE, + LockType::EXCLUSIVE +}; +const char* _kLockTypeNames[] = { + "SHARED_READ", + "SHARED_WRITE", + "EXCLUSIVE" +}; +const std::map _LockType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kLockTypeValues, _kLockTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); + +std::ostream& operator<<(std::ostream& out, const LockType::type& val) { + std::map::const_iterator it = _LockType_VALUES_TO_NAMES.find(val); + if (it != _LockType_VALUES_TO_NAMES.end()) { + out << it->second; + } else { + out << static_cast(val); + } + return out; +} + +int _kCompactionTypeValues[] = { + CompactionType::MINOR, + CompactionType::MAJOR +}; +const char* _kCompactionTypeNames[] = { + "MINOR", + "MAJOR" +}; +const std::map _CompactionType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(2, _kCompactionTypeValues, _kCompactionTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); + +std::ostream& operator<<(std::ostream& out, const CompactionType::type& val) { + std::map::const_iterator it = _CompactionType_VALUES_TO_NAMES.find(val); + if (it != _CompactionType_VALUES_TO_NAMES.end()) { + out << it->second; + } else { + out << static_cast(val); + } + return out; +} + +int _kGrantRevokeTypeValues[] = { + GrantRevokeType::GRANT, + GrantRevokeType::REVOKE +}; +const char* _kGrantRevokeTypeNames[] = { + "GRANT", + "REVOKE" +}; +const std::map _GrantRevokeType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(2, _kGrantRevokeTypeValues, _kGrantRevokeTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); + +std::ostream& operator<<(std::ostream& out, const GrantRevokeType::type& val) { + std::map::const_iterator it = _GrantRevokeType_VALUES_TO_NAMES.find(val); + if (it != _GrantRevokeType_VALUES_TO_NAMES.end()) { + out << it->second; + } else { + out << static_cast(val); + } + return out; +} + +int _kDataOperationTypeValues[] = { + DataOperationType::SELECT, + DataOperationType::INSERT, + DataOperationType::UPDATE, + DataOperationType::DELETE, + DataOperationType::UNSET, + DataOperationType::NO_TXN +}; +const char* _kDataOperationTypeNames[] = { + "SELECT", + "INSERT", + "UPDATE", + "DELETE", + "UNSET", + "NO_TXN" +}; +const std::map _DataOperationType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(6, _kDataOperationTypeValues, _kDataOperationTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); + +std::ostream& operator<<(std::ostream& out, const DataOperationType::type& val) { + std::map::const_iterator it = _DataOperationType_VALUES_TO_NAMES.find(val); + if (it != _DataOperationType_VALUES_TO_NAMES.end()) { + out << it->second; + } else { + out << static_cast(val); + } + return out; +} + +int _kEventRequestTypeValues[] = { + EventRequestType::INSERT, + EventRequestType::UPDATE, + EventRequestType::DELETE +}; +const char* _kEventRequestTypeNames[] = { + "INSERT", + "UPDATE", + "DELETE" +}; +const std::map _EventRequestType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kEventRequestTypeValues, _kEventRequestTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); + +std::ostream& operator<<(std::ostream& out, const EventRequestType::type& val) { + std::map::const_iterator it = _EventRequestType_VALUES_TO_NAMES.find(val); + if (it != _EventRequestType_VALUES_TO_NAMES.end()) { + out << it->second; + } else { + out << static_cast(val); + } + return out; +} + +int _kFunctionTypeValues[] = { + FunctionType::JAVA +}; +const char* _kFunctionTypeNames[] = { + "JAVA" +}; +const std::map _FunctionType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(1, _kFunctionTypeValues, _kFunctionTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); + +std::ostream& operator<<(std::ostream& out, const FunctionType::type& val) { + std::map::const_iterator it = _FunctionType_VALUES_TO_NAMES.find(val); + if (it != _FunctionType_VALUES_TO_NAMES.end()) { + out << it->second; + } else { + out << static_cast(val); + } + return out; +} + +int _kResourceTypeValues[] = { + ResourceType::JAR, + ResourceType::FILE, + ResourceType::ARCHIVE +}; +const char* _kResourceTypeNames[] = { + "JAR", + "FILE", + "ARCHIVE" +}; +const std::map _ResourceType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kResourceTypeValues, _kResourceTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); + +std::ostream& operator<<(std::ostream& out, const ResourceType::type& val) { + std::map::const_iterator it = _ResourceType_VALUES_TO_NAMES.find(val); + if (it != _ResourceType_VALUES_TO_NAMES.end()) { + out << it->second; + } else { + out << static_cast(val); + } + return out; +} + +int _kFileMetadataExprTypeValues[] = { + FileMetadataExprType::ORC_SARG +}; +const char* _kFileMetadataExprTypeNames[] = { + "ORC_SARG" +}; +const std::map _FileMetadataExprType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(1, _kFileMetadataExprTypeValues, _kFileMetadataExprTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); + +std::ostream& operator<<(std::ostream& out, const FileMetadataExprType::type& val) { + std::map::const_iterator it = _FileMetadataExprType_VALUES_TO_NAMES.find(val); + if (it != _FileMetadataExprType_VALUES_TO_NAMES.end()) { + out << it->second; + } else { + out << static_cast(val); + } + return out; +} + +int _kClientCapabilityValues[] = { + ClientCapability::TEST_CAPABILITY, + ClientCapability::INSERT_ONLY_TABLES +}; +const char* _kClientCapabilityNames[] = { + "TEST_CAPABILITY", + "INSERT_ONLY_TABLES" +}; +const std::map _ClientCapability_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(2, _kClientCapabilityValues, _kClientCapabilityNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); + +std::ostream& operator<<(std::ostream& out, const ClientCapability::type& val) { + std::map::const_iterator it = _ClientCapability_VALUES_TO_NAMES.find(val); + if (it != _ClientCapability_VALUES_TO_NAMES.end()) { + out << it->second; + } else { + out << static_cast(val); + } + return out; +} + +int _kWMResourcePlanStatusValues[] = { + WMResourcePlanStatus::ACTIVE, + WMResourcePlanStatus::ENABLED, + WMResourcePlanStatus::DISABLED +}; +const char* _kWMResourcePlanStatusNames[] = { + "ACTIVE", + "ENABLED", + "DISABLED" +}; +const std::map _WMResourcePlanStatus_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kWMResourcePlanStatusValues, _kWMResourcePlanStatusNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); + +std::ostream& operator<<(std::ostream& out, const WMResourcePlanStatus::type& val) { + std::map::const_iterator it = _WMResourcePlanStatus_VALUES_TO_NAMES.find(val); + if (it != _WMResourcePlanStatus_VALUES_TO_NAMES.end()) { + out << it->second; + } else { + out << static_cast(val); + } + return out; +} + + +Version::~Version() noexcept { +} + + +void Version::__set_version(const std::string& val) { + this->version = val; +} + +void Version::__set_comments(const std::string& val) { + this->comments = val; +} +std::ostream& operator<<(std::ostream& out, const Version& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t Version::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->version); + this->__isset.version = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->comments); + this->__isset.comments = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t Version::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("Version"); + + xfer += oprot->writeFieldBegin("version", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->version); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("comments", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->comments); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(Version &a, Version &b) { + using ::std::swap; + swap(a.version, b.version); + swap(a.comments, b.comments); + swap(a.__isset, b.__isset); +} + +Version::Version(const Version& other0) { + version = other0.version; + comments = other0.comments; + __isset = other0.__isset; +} +Version& Version::operator=(const Version& other1) { + version = other1.version; + comments = other1.comments; + __isset = other1.__isset; + return *this; +} +void Version::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "Version("; + out << "version=" << to_string(version); + out << ", " << "comments=" << to_string(comments); + out << ")"; +} + + +FieldSchema::~FieldSchema() noexcept { +} + + +void FieldSchema::__set_name(const std::string& val) { + this->name = val; +} + +void FieldSchema::__set_type(const std::string& val) { + this->type = val; +} + +void FieldSchema::__set_comment(const std::string& val) { + this->comment = val; +} +std::ostream& operator<<(std::ostream& out, const FieldSchema& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t FieldSchema::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->name); + this->__isset.name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->type); + this->__isset.type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->comment); + this->__isset.comment = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t FieldSchema::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("FieldSchema"); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->type); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("comment", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->comment); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(FieldSchema &a, FieldSchema &b) { + using ::std::swap; + swap(a.name, b.name); + swap(a.type, b.type); + swap(a.comment, b.comment); + swap(a.__isset, b.__isset); +} + +FieldSchema::FieldSchema(const FieldSchema& other2) { + name = other2.name; + type = other2.type; + comment = other2.comment; + __isset = other2.__isset; +} +FieldSchema& FieldSchema::operator=(const FieldSchema& other3) { + name = other3.name; + type = other3.type; + comment = other3.comment; + __isset = other3.__isset; + return *this; +} +void FieldSchema::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "FieldSchema("; + out << "name=" << to_string(name); + out << ", " << "type=" << to_string(type); + out << ", " << "comment=" << to_string(comment); + out << ")"; +} + + +SQLPrimaryKey::~SQLPrimaryKey() noexcept { +} + + +void SQLPrimaryKey::__set_table_db(const std::string& val) { + this->table_db = val; +} + +void SQLPrimaryKey::__set_table_name(const std::string& val) { + this->table_name = val; +} + +void SQLPrimaryKey::__set_column_name(const std::string& val) { + this->column_name = val; +} + +void SQLPrimaryKey::__set_key_seq(const int32_t val) { + this->key_seq = val; +} + +void SQLPrimaryKey::__set_pk_name(const std::string& val) { + this->pk_name = val; +} + +void SQLPrimaryKey::__set_enable_cstr(const bool val) { + this->enable_cstr = val; +} + +void SQLPrimaryKey::__set_validate_cstr(const bool val) { + this->validate_cstr = val; +} + +void SQLPrimaryKey::__set_rely_cstr(const bool val) { + this->rely_cstr = val; +} +std::ostream& operator<<(std::ostream& out, const SQLPrimaryKey& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t SQLPrimaryKey::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->table_db); + this->__isset.table_db = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->table_name); + this->__isset.table_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->column_name); + this->__isset.column_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->key_seq); + this->__isset.key_seq = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->pk_name); + this->__isset.pk_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->enable_cstr); + this->__isset.enable_cstr = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->validate_cstr); + this->__isset.validate_cstr = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 8: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->rely_cstr); + this->__isset.rely_cstr = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t SQLPrimaryKey::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("SQLPrimaryKey"); + + xfer += oprot->writeFieldBegin("table_db", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->table_db); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->table_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("column_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->column_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("key_seq", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32(this->key_seq); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("pk_name", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->pk_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 6); + xfer += oprot->writeBool(this->enable_cstr); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 7); + xfer += oprot->writeBool(this->validate_cstr); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 8); + xfer += oprot->writeBool(this->rely_cstr); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(SQLPrimaryKey &a, SQLPrimaryKey &b) { + using ::std::swap; + swap(a.table_db, b.table_db); + swap(a.table_name, b.table_name); + swap(a.column_name, b.column_name); + swap(a.key_seq, b.key_seq); + swap(a.pk_name, b.pk_name); + swap(a.enable_cstr, b.enable_cstr); + swap(a.validate_cstr, b.validate_cstr); + swap(a.rely_cstr, b.rely_cstr); + swap(a.__isset, b.__isset); +} + +SQLPrimaryKey::SQLPrimaryKey(const SQLPrimaryKey& other4) { + table_db = other4.table_db; + table_name = other4.table_name; + column_name = other4.column_name; + key_seq = other4.key_seq; + pk_name = other4.pk_name; + enable_cstr = other4.enable_cstr; + validate_cstr = other4.validate_cstr; + rely_cstr = other4.rely_cstr; + __isset = other4.__isset; +} +SQLPrimaryKey& SQLPrimaryKey::operator=(const SQLPrimaryKey& other5) { + table_db = other5.table_db; + table_name = other5.table_name; + column_name = other5.column_name; + key_seq = other5.key_seq; + pk_name = other5.pk_name; + enable_cstr = other5.enable_cstr; + validate_cstr = other5.validate_cstr; + rely_cstr = other5.rely_cstr; + __isset = other5.__isset; + return *this; +} +void SQLPrimaryKey::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "SQLPrimaryKey("; + out << "table_db=" << to_string(table_db); + out << ", " << "table_name=" << to_string(table_name); + out << ", " << "column_name=" << to_string(column_name); + out << ", " << "key_seq=" << to_string(key_seq); + out << ", " << "pk_name=" << to_string(pk_name); + out << ", " << "enable_cstr=" << to_string(enable_cstr); + out << ", " << "validate_cstr=" << to_string(validate_cstr); + out << ", " << "rely_cstr=" << to_string(rely_cstr); + out << ")"; +} + + +SQLForeignKey::~SQLForeignKey() noexcept { +} + + +void SQLForeignKey::__set_pktable_db(const std::string& val) { + this->pktable_db = val; +} + +void SQLForeignKey::__set_pktable_name(const std::string& val) { + this->pktable_name = val; +} + +void SQLForeignKey::__set_pkcolumn_name(const std::string& val) { + this->pkcolumn_name = val; +} + +void SQLForeignKey::__set_fktable_db(const std::string& val) { + this->fktable_db = val; +} + +void SQLForeignKey::__set_fktable_name(const std::string& val) { + this->fktable_name = val; +} + +void SQLForeignKey::__set_fkcolumn_name(const std::string& val) { + this->fkcolumn_name = val; +} + +void SQLForeignKey::__set_key_seq(const int32_t val) { + this->key_seq = val; +} + +void SQLForeignKey::__set_update_rule(const int32_t val) { + this->update_rule = val; +} + +void SQLForeignKey::__set_delete_rule(const int32_t val) { + this->delete_rule = val; +} + +void SQLForeignKey::__set_fk_name(const std::string& val) { + this->fk_name = val; +} + +void SQLForeignKey::__set_pk_name(const std::string& val) { + this->pk_name = val; +} + +void SQLForeignKey::__set_enable_cstr(const bool val) { + this->enable_cstr = val; +} + +void SQLForeignKey::__set_validate_cstr(const bool val) { + this->validate_cstr = val; +} + +void SQLForeignKey::__set_rely_cstr(const bool val) { + this->rely_cstr = val; +} +std::ostream& operator<<(std::ostream& out, const SQLForeignKey& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t SQLForeignKey::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->pktable_db); + this->__isset.pktable_db = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->pktable_name); + this->__isset.pktable_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->pkcolumn_name); + this->__isset.pkcolumn_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->fktable_db); + this->__isset.fktable_db = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->fktable_name); + this->__isset.fktable_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->fkcolumn_name); + this->__isset.fkcolumn_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->key_seq); + this->__isset.key_seq = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 8: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->update_rule); + this->__isset.update_rule = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 9: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->delete_rule); + this->__isset.delete_rule = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 10: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->fk_name); + this->__isset.fk_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 11: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->pk_name); + this->__isset.pk_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 12: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->enable_cstr); + this->__isset.enable_cstr = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 13: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->validate_cstr); + this->__isset.validate_cstr = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 14: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->rely_cstr); + this->__isset.rely_cstr = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t SQLForeignKey::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("SQLForeignKey"); + + xfer += oprot->writeFieldBegin("pktable_db", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->pktable_db); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("pktable_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->pktable_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("pkcolumn_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->pkcolumn_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("fktable_db", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->fktable_db); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("fktable_name", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->fktable_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("fkcolumn_name", ::apache::thrift::protocol::T_STRING, 6); + xfer += oprot->writeString(this->fkcolumn_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("key_seq", ::apache::thrift::protocol::T_I32, 7); + xfer += oprot->writeI32(this->key_seq); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("update_rule", ::apache::thrift::protocol::T_I32, 8); + xfer += oprot->writeI32(this->update_rule); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("delete_rule", ::apache::thrift::protocol::T_I32, 9); + xfer += oprot->writeI32(this->delete_rule); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("fk_name", ::apache::thrift::protocol::T_STRING, 10); + xfer += oprot->writeString(this->fk_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("pk_name", ::apache::thrift::protocol::T_STRING, 11); + xfer += oprot->writeString(this->pk_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 12); + xfer += oprot->writeBool(this->enable_cstr); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 13); + xfer += oprot->writeBool(this->validate_cstr); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 14); + xfer += oprot->writeBool(this->rely_cstr); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(SQLForeignKey &a, SQLForeignKey &b) { + using ::std::swap; + swap(a.pktable_db, b.pktable_db); + swap(a.pktable_name, b.pktable_name); + swap(a.pkcolumn_name, b.pkcolumn_name); + swap(a.fktable_db, b.fktable_db); + swap(a.fktable_name, b.fktable_name); + swap(a.fkcolumn_name, b.fkcolumn_name); + swap(a.key_seq, b.key_seq); + swap(a.update_rule, b.update_rule); + swap(a.delete_rule, b.delete_rule); + swap(a.fk_name, b.fk_name); + swap(a.pk_name, b.pk_name); + swap(a.enable_cstr, b.enable_cstr); + swap(a.validate_cstr, b.validate_cstr); + swap(a.rely_cstr, b.rely_cstr); + swap(a.__isset, b.__isset); +} + +SQLForeignKey::SQLForeignKey(const SQLForeignKey& other6) { + pktable_db = other6.pktable_db; + pktable_name = other6.pktable_name; + pkcolumn_name = other6.pkcolumn_name; + fktable_db = other6.fktable_db; + fktable_name = other6.fktable_name; + fkcolumn_name = other6.fkcolumn_name; + key_seq = other6.key_seq; + update_rule = other6.update_rule; + delete_rule = other6.delete_rule; + fk_name = other6.fk_name; + pk_name = other6.pk_name; + enable_cstr = other6.enable_cstr; + validate_cstr = other6.validate_cstr; + rely_cstr = other6.rely_cstr; + __isset = other6.__isset; +} +SQLForeignKey& SQLForeignKey::operator=(const SQLForeignKey& other7) { + pktable_db = other7.pktable_db; + pktable_name = other7.pktable_name; + pkcolumn_name = other7.pkcolumn_name; + fktable_db = other7.fktable_db; + fktable_name = other7.fktable_name; + fkcolumn_name = other7.fkcolumn_name; + key_seq = other7.key_seq; + update_rule = other7.update_rule; + delete_rule = other7.delete_rule; + fk_name = other7.fk_name; + pk_name = other7.pk_name; + enable_cstr = other7.enable_cstr; + validate_cstr = other7.validate_cstr; + rely_cstr = other7.rely_cstr; + __isset = other7.__isset; + return *this; +} +void SQLForeignKey::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "SQLForeignKey("; + out << "pktable_db=" << to_string(pktable_db); + out << ", " << "pktable_name=" << to_string(pktable_name); + out << ", " << "pkcolumn_name=" << to_string(pkcolumn_name); + out << ", " << "fktable_db=" << to_string(fktable_db); + out << ", " << "fktable_name=" << to_string(fktable_name); + out << ", " << "fkcolumn_name=" << to_string(fkcolumn_name); + out << ", " << "key_seq=" << to_string(key_seq); + out << ", " << "update_rule=" << to_string(update_rule); + out << ", " << "delete_rule=" << to_string(delete_rule); + out << ", " << "fk_name=" << to_string(fk_name); + out << ", " << "pk_name=" << to_string(pk_name); + out << ", " << "enable_cstr=" << to_string(enable_cstr); + out << ", " << "validate_cstr=" << to_string(validate_cstr); + out << ", " << "rely_cstr=" << to_string(rely_cstr); + out << ")"; +} + + +SQLUniqueConstraint::~SQLUniqueConstraint() noexcept { +} + + +void SQLUniqueConstraint::__set_table_db(const std::string& val) { + this->table_db = val; +} + +void SQLUniqueConstraint::__set_table_name(const std::string& val) { + this->table_name = val; +} + +void SQLUniqueConstraint::__set_column_name(const std::string& val) { + this->column_name = val; +} + +void SQLUniqueConstraint::__set_key_seq(const int32_t val) { + this->key_seq = val; +} + +void SQLUniqueConstraint::__set_uk_name(const std::string& val) { + this->uk_name = val; +} + +void SQLUniqueConstraint::__set_enable_cstr(const bool val) { + this->enable_cstr = val; +} + +void SQLUniqueConstraint::__set_validate_cstr(const bool val) { + this->validate_cstr = val; +} + +void SQLUniqueConstraint::__set_rely_cstr(const bool val) { + this->rely_cstr = val; +} +std::ostream& operator<<(std::ostream& out, const SQLUniqueConstraint& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t SQLUniqueConstraint::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->table_db); + this->__isset.table_db = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->table_name); + this->__isset.table_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->column_name); + this->__isset.column_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->key_seq); + this->__isset.key_seq = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->uk_name); + this->__isset.uk_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->enable_cstr); + this->__isset.enable_cstr = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->validate_cstr); + this->__isset.validate_cstr = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 8: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->rely_cstr); + this->__isset.rely_cstr = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t SQLUniqueConstraint::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("SQLUniqueConstraint"); + + xfer += oprot->writeFieldBegin("table_db", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->table_db); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->table_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("column_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->column_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("key_seq", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32(this->key_seq); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("uk_name", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->uk_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 6); + xfer += oprot->writeBool(this->enable_cstr); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 7); + xfer += oprot->writeBool(this->validate_cstr); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 8); + xfer += oprot->writeBool(this->rely_cstr); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(SQLUniqueConstraint &a, SQLUniqueConstraint &b) { + using ::std::swap; + swap(a.table_db, b.table_db); + swap(a.table_name, b.table_name); + swap(a.column_name, b.column_name); + swap(a.key_seq, b.key_seq); + swap(a.uk_name, b.uk_name); + swap(a.enable_cstr, b.enable_cstr); + swap(a.validate_cstr, b.validate_cstr); + swap(a.rely_cstr, b.rely_cstr); + swap(a.__isset, b.__isset); +} + +SQLUniqueConstraint::SQLUniqueConstraint(const SQLUniqueConstraint& other8) { + table_db = other8.table_db; + table_name = other8.table_name; + column_name = other8.column_name; + key_seq = other8.key_seq; + uk_name = other8.uk_name; + enable_cstr = other8.enable_cstr; + validate_cstr = other8.validate_cstr; + rely_cstr = other8.rely_cstr; + __isset = other8.__isset; +} +SQLUniqueConstraint& SQLUniqueConstraint::operator=(const SQLUniqueConstraint& other9) { + table_db = other9.table_db; + table_name = other9.table_name; + column_name = other9.column_name; + key_seq = other9.key_seq; + uk_name = other9.uk_name; + enable_cstr = other9.enable_cstr; + validate_cstr = other9.validate_cstr; + rely_cstr = other9.rely_cstr; + __isset = other9.__isset; + return *this; +} +void SQLUniqueConstraint::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "SQLUniqueConstraint("; + out << "table_db=" << to_string(table_db); + out << ", " << "table_name=" << to_string(table_name); + out << ", " << "column_name=" << to_string(column_name); + out << ", " << "key_seq=" << to_string(key_seq); + out << ", " << "uk_name=" << to_string(uk_name); + out << ", " << "enable_cstr=" << to_string(enable_cstr); + out << ", " << "validate_cstr=" << to_string(validate_cstr); + out << ", " << "rely_cstr=" << to_string(rely_cstr); + out << ")"; +} + + +SQLNotNullConstraint::~SQLNotNullConstraint() noexcept { +} + + +void SQLNotNullConstraint::__set_table_db(const std::string& val) { + this->table_db = val; +} + +void SQLNotNullConstraint::__set_table_name(const std::string& val) { + this->table_name = val; +} + +void SQLNotNullConstraint::__set_column_name(const std::string& val) { + this->column_name = val; +} + +void SQLNotNullConstraint::__set_nn_name(const std::string& val) { + this->nn_name = val; +} + +void SQLNotNullConstraint::__set_enable_cstr(const bool val) { + this->enable_cstr = val; +} + +void SQLNotNullConstraint::__set_validate_cstr(const bool val) { + this->validate_cstr = val; +} + +void SQLNotNullConstraint::__set_rely_cstr(const bool val) { + this->rely_cstr = val; +} +std::ostream& operator<<(std::ostream& out, const SQLNotNullConstraint& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t SQLNotNullConstraint::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->table_db); + this->__isset.table_db = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->table_name); + this->__isset.table_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->column_name); + this->__isset.column_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->nn_name); + this->__isset.nn_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->enable_cstr); + this->__isset.enable_cstr = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->validate_cstr); + this->__isset.validate_cstr = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->rely_cstr); + this->__isset.rely_cstr = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t SQLNotNullConstraint::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("SQLNotNullConstraint"); + + xfer += oprot->writeFieldBegin("table_db", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->table_db); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->table_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("column_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->column_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("nn_name", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->nn_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 5); + xfer += oprot->writeBool(this->enable_cstr); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 6); + xfer += oprot->writeBool(this->validate_cstr); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 7); + xfer += oprot->writeBool(this->rely_cstr); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(SQLNotNullConstraint &a, SQLNotNullConstraint &b) { + using ::std::swap; + swap(a.table_db, b.table_db); + swap(a.table_name, b.table_name); + swap(a.column_name, b.column_name); + swap(a.nn_name, b.nn_name); + swap(a.enable_cstr, b.enable_cstr); + swap(a.validate_cstr, b.validate_cstr); + swap(a.rely_cstr, b.rely_cstr); + swap(a.__isset, b.__isset); +} + +SQLNotNullConstraint::SQLNotNullConstraint(const SQLNotNullConstraint& other10) { + table_db = other10.table_db; + table_name = other10.table_name; + column_name = other10.column_name; + nn_name = other10.nn_name; + enable_cstr = other10.enable_cstr; + validate_cstr = other10.validate_cstr; + rely_cstr = other10.rely_cstr; + __isset = other10.__isset; +} +SQLNotNullConstraint& SQLNotNullConstraint::operator=(const SQLNotNullConstraint& other11) { + table_db = other11.table_db; + table_name = other11.table_name; + column_name = other11.column_name; + nn_name = other11.nn_name; + enable_cstr = other11.enable_cstr; + validate_cstr = other11.validate_cstr; + rely_cstr = other11.rely_cstr; + __isset = other11.__isset; + return *this; +} +void SQLNotNullConstraint::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "SQLNotNullConstraint("; + out << "table_db=" << to_string(table_db); + out << ", " << "table_name=" << to_string(table_name); + out << ", " << "column_name=" << to_string(column_name); + out << ", " << "nn_name=" << to_string(nn_name); + out << ", " << "enable_cstr=" << to_string(enable_cstr); + out << ", " << "validate_cstr=" << to_string(validate_cstr); + out << ", " << "rely_cstr=" << to_string(rely_cstr); + out << ")"; +} + + +Type::~Type() noexcept { +} + + +void Type::__set_name(const std::string& val) { + this->name = val; +} + +void Type::__set_type1(const std::string& val) { + this->type1 = val; +__isset.type1 = true; +} + +void Type::__set_type2(const std::string& val) { + this->type2 = val; +__isset.type2 = true; +} + +void Type::__set_fields(const std::vector & val) { + this->fields = val; +__isset.fields = true; +} +std::ostream& operator<<(std::ostream& out, const Type& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t Type::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->name); + this->__isset.name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->type1); + this->__isset.type1 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->type2); + this->__isset.type2 = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->fields.clear(); + uint32_t _size12; + ::apache::thrift::protocol::TType _etype15; + xfer += iprot->readListBegin(_etype15, _size12); + this->fields.resize(_size12); + uint32_t _i16; + for (_i16 = 0; _i16 < _size12; ++_i16) + { + xfer += this->fields[_i16].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.fields = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t Type::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("Type"); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->name); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.type1) { + xfer += oprot->writeFieldBegin("type1", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->type1); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.type2) { + xfer += oprot->writeFieldBegin("type2", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->type2); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.fields) { + xfer += oprot->writeFieldBegin("fields", ::apache::thrift::protocol::T_LIST, 4); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->fields.size())); + std::vector ::const_iterator _iter17; + for (_iter17 = this->fields.begin(); _iter17 != this->fields.end(); ++_iter17) + { + xfer += (*_iter17).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(Type &a, Type &b) { + using ::std::swap; + swap(a.name, b.name); + swap(a.type1, b.type1); + swap(a.type2, b.type2); + swap(a.fields, b.fields); + swap(a.__isset, b.__isset); +} + +Type::Type(const Type& other18) { + name = other18.name; + type1 = other18.type1; + type2 = other18.type2; + fields = other18.fields; + __isset = other18.__isset; +} +Type& Type::operator=(const Type& other19) { + name = other19.name; + type1 = other19.type1; + type2 = other19.type2; + fields = other19.fields; + __isset = other19.__isset; + return *this; +} +void Type::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "Type("; + out << "name=" << to_string(name); + out << ", " << "type1="; (__isset.type1 ? (out << to_string(type1)) : (out << "")); + out << ", " << "type2="; (__isset.type2 ? (out << to_string(type2)) : (out << "")); + out << ", " << "fields="; (__isset.fields ? (out << to_string(fields)) : (out << "")); + out << ")"; +} + + +HiveObjectRef::~HiveObjectRef() noexcept { +} + + +void HiveObjectRef::__set_objectType(const HiveObjectType::type val) { + this->objectType = val; +} + +void HiveObjectRef::__set_dbName(const std::string& val) { + this->dbName = val; +} + +void HiveObjectRef::__set_objectName(const std::string& val) { + this->objectName = val; +} + +void HiveObjectRef::__set_partValues(const std::vector & val) { + this->partValues = val; +} + +void HiveObjectRef::__set_columnName(const std::string& val) { + this->columnName = val; +} +std::ostream& operator<<(std::ostream& out, const HiveObjectRef& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t HiveObjectRef::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast20; + xfer += iprot->readI32(ecast20); + this->objectType = (HiveObjectType::type)ecast20; + this->__isset.objectType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->objectName); + this->__isset.objectName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->partValues.clear(); + uint32_t _size21; + ::apache::thrift::protocol::TType _etype24; + xfer += iprot->readListBegin(_etype24, _size21); + this->partValues.resize(_size21); + uint32_t _i25; + for (_i25 = 0; _i25 < _size21; ++_i25) + { + xfer += iprot->readString(this->partValues[_i25]); + } + xfer += iprot->readListEnd(); + } + this->__isset.partValues = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->columnName); + this->__isset.columnName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t HiveObjectRef::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("HiveObjectRef"); + + xfer += oprot->writeFieldBegin("objectType", ::apache::thrift::protocol::T_I32, 1); + xfer += oprot->writeI32((int32_t)this->objectType); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("objectName", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->objectName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("partValues", ::apache::thrift::protocol::T_LIST, 4); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partValues.size())); + std::vector ::const_iterator _iter26; + for (_iter26 = this->partValues.begin(); _iter26 != this->partValues.end(); ++_iter26) + { + xfer += oprot->writeString((*_iter26)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("columnName", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->columnName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(HiveObjectRef &a, HiveObjectRef &b) { + using ::std::swap; + swap(a.objectType, b.objectType); + swap(a.dbName, b.dbName); + swap(a.objectName, b.objectName); + swap(a.partValues, b.partValues); + swap(a.columnName, b.columnName); + swap(a.__isset, b.__isset); +} + +HiveObjectRef::HiveObjectRef(const HiveObjectRef& other27) { + objectType = other27.objectType; + dbName = other27.dbName; + objectName = other27.objectName; + partValues = other27.partValues; + columnName = other27.columnName; + __isset = other27.__isset; +} +HiveObjectRef& HiveObjectRef::operator=(const HiveObjectRef& other28) { + objectType = other28.objectType; + dbName = other28.dbName; + objectName = other28.objectName; + partValues = other28.partValues; + columnName = other28.columnName; + __isset = other28.__isset; + return *this; +} +void HiveObjectRef::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "HiveObjectRef("; + out << "objectType=" << to_string(objectType); + out << ", " << "dbName=" << to_string(dbName); + out << ", " << "objectName=" << to_string(objectName); + out << ", " << "partValues=" << to_string(partValues); + out << ", " << "columnName=" << to_string(columnName); + out << ")"; +} + + +PrivilegeGrantInfo::~PrivilegeGrantInfo() noexcept { +} + + +void PrivilegeGrantInfo::__set_privilege(const std::string& val) { + this->privilege = val; +} + +void PrivilegeGrantInfo::__set_createTime(const int32_t val) { + this->createTime = val; +} + +void PrivilegeGrantInfo::__set_grantor(const std::string& val) { + this->grantor = val; +} + +void PrivilegeGrantInfo::__set_grantorType(const PrincipalType::type val) { + this->grantorType = val; +} + +void PrivilegeGrantInfo::__set_grantOption(const bool val) { + this->grantOption = val; +} +std::ostream& operator<<(std::ostream& out, const PrivilegeGrantInfo& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PrivilegeGrantInfo::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->privilege); + this->__isset.privilege = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->createTime); + this->__isset.createTime = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->grantor); + this->__isset.grantor = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast29; + xfer += iprot->readI32(ecast29); + this->grantorType = (PrincipalType::type)ecast29; + this->__isset.grantorType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->grantOption); + this->__isset.grantOption = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t PrivilegeGrantInfo::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PrivilegeGrantInfo"); + + xfer += oprot->writeFieldBegin("privilege", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->privilege); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32(this->createTime); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grantor", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->grantor); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grantorType", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32((int32_t)this->grantorType); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grantOption", ::apache::thrift::protocol::T_BOOL, 5); + xfer += oprot->writeBool(this->grantOption); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PrivilegeGrantInfo &a, PrivilegeGrantInfo &b) { + using ::std::swap; + swap(a.privilege, b.privilege); + swap(a.createTime, b.createTime); + swap(a.grantor, b.grantor); + swap(a.grantorType, b.grantorType); + swap(a.grantOption, b.grantOption); + swap(a.__isset, b.__isset); +} + +PrivilegeGrantInfo::PrivilegeGrantInfo(const PrivilegeGrantInfo& other30) { + privilege = other30.privilege; + createTime = other30.createTime; + grantor = other30.grantor; + grantorType = other30.grantorType; + grantOption = other30.grantOption; + __isset = other30.__isset; +} +PrivilegeGrantInfo& PrivilegeGrantInfo::operator=(const PrivilegeGrantInfo& other31) { + privilege = other31.privilege; + createTime = other31.createTime; + grantor = other31.grantor; + grantorType = other31.grantorType; + grantOption = other31.grantOption; + __isset = other31.__isset; + return *this; +} +void PrivilegeGrantInfo::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PrivilegeGrantInfo("; + out << "privilege=" << to_string(privilege); + out << ", " << "createTime=" << to_string(createTime); + out << ", " << "grantor=" << to_string(grantor); + out << ", " << "grantorType=" << to_string(grantorType); + out << ", " << "grantOption=" << to_string(grantOption); + out << ")"; +} + + +HiveObjectPrivilege::~HiveObjectPrivilege() noexcept { +} + + +void HiveObjectPrivilege::__set_hiveObject(const HiveObjectRef& val) { + this->hiveObject = val; +} + +void HiveObjectPrivilege::__set_principalName(const std::string& val) { + this->principalName = val; +} + +void HiveObjectPrivilege::__set_principalType(const PrincipalType::type val) { + this->principalType = val; +} + +void HiveObjectPrivilege::__set_grantInfo(const PrivilegeGrantInfo& val) { + this->grantInfo = val; +} +std::ostream& operator<<(std::ostream& out, const HiveObjectPrivilege& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t HiveObjectPrivilege::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->hiveObject.read(iprot); + this->__isset.hiveObject = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->principalName); + this->__isset.principalName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast32; + xfer += iprot->readI32(ecast32); + this->principalType = (PrincipalType::type)ecast32; + this->__isset.principalType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->grantInfo.read(iprot); + this->__isset.grantInfo = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t HiveObjectPrivilege::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("HiveObjectPrivilege"); + + xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->hiveObject.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principalName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->principalName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principalType", ::apache::thrift::protocol::T_I32, 3); + xfer += oprot->writeI32((int32_t)this->principalType); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grantInfo", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->grantInfo.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(HiveObjectPrivilege &a, HiveObjectPrivilege &b) { + using ::std::swap; + swap(a.hiveObject, b.hiveObject); + swap(a.principalName, b.principalName); + swap(a.principalType, b.principalType); + swap(a.grantInfo, b.grantInfo); + swap(a.__isset, b.__isset); +} + +HiveObjectPrivilege::HiveObjectPrivilege(const HiveObjectPrivilege& other33) { + hiveObject = other33.hiveObject; + principalName = other33.principalName; + principalType = other33.principalType; + grantInfo = other33.grantInfo; + __isset = other33.__isset; +} +HiveObjectPrivilege& HiveObjectPrivilege::operator=(const HiveObjectPrivilege& other34) { + hiveObject = other34.hiveObject; + principalName = other34.principalName; + principalType = other34.principalType; + grantInfo = other34.grantInfo; + __isset = other34.__isset; + return *this; +} +void HiveObjectPrivilege::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "HiveObjectPrivilege("; + out << "hiveObject=" << to_string(hiveObject); + out << ", " << "principalName=" << to_string(principalName); + out << ", " << "principalType=" << to_string(principalType); + out << ", " << "grantInfo=" << to_string(grantInfo); + out << ")"; +} + + +PrivilegeBag::~PrivilegeBag() noexcept { +} + + +void PrivilegeBag::__set_privileges(const std::vector & val) { + this->privileges = val; +} +std::ostream& operator<<(std::ostream& out, const PrivilegeBag& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PrivilegeBag::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->privileges.clear(); + uint32_t _size35; + ::apache::thrift::protocol::TType _etype38; + xfer += iprot->readListBegin(_etype38, _size35); + this->privileges.resize(_size35); + uint32_t _i39; + for (_i39 = 0; _i39 < _size35; ++_i39) + { + xfer += this->privileges[_i39].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.privileges = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t PrivilegeBag::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PrivilegeBag"); + + xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->privileges.size())); + std::vector ::const_iterator _iter40; + for (_iter40 = this->privileges.begin(); _iter40 != this->privileges.end(); ++_iter40) + { + xfer += (*_iter40).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PrivilegeBag &a, PrivilegeBag &b) { + using ::std::swap; + swap(a.privileges, b.privileges); + swap(a.__isset, b.__isset); +} + +PrivilegeBag::PrivilegeBag(const PrivilegeBag& other41) { + privileges = other41.privileges; + __isset = other41.__isset; +} +PrivilegeBag& PrivilegeBag::operator=(const PrivilegeBag& other42) { + privileges = other42.privileges; + __isset = other42.__isset; + return *this; +} +void PrivilegeBag::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PrivilegeBag("; + out << "privileges=" << to_string(privileges); + out << ")"; +} + + +PrincipalPrivilegeSet::~PrincipalPrivilegeSet() noexcept { +} + + +void PrincipalPrivilegeSet::__set_userPrivileges(const std::map > & val) { + this->userPrivileges = val; +} + +void PrincipalPrivilegeSet::__set_groupPrivileges(const std::map > & val) { + this->groupPrivileges = val; +} + +void PrincipalPrivilegeSet::__set_rolePrivileges(const std::map > & val) { + this->rolePrivileges = val; +} +std::ostream& operator<<(std::ostream& out, const PrincipalPrivilegeSet& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PrincipalPrivilegeSet::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->userPrivileges.clear(); + uint32_t _size43; + ::apache::thrift::protocol::TType _ktype44; + ::apache::thrift::protocol::TType _vtype45; + xfer += iprot->readMapBegin(_ktype44, _vtype45, _size43); + uint32_t _i47; + for (_i47 = 0; _i47 < _size43; ++_i47) + { + std::string _key48; + xfer += iprot->readString(_key48); + std::vector & _val49 = this->userPrivileges[_key48]; + { + _val49.clear(); + uint32_t _size50; + ::apache::thrift::protocol::TType _etype53; + xfer += iprot->readListBegin(_etype53, _size50); + _val49.resize(_size50); + uint32_t _i54; + for (_i54 = 0; _i54 < _size50; ++_i54) + { + xfer += _val49[_i54].read(iprot); + } + xfer += iprot->readListEnd(); + } + } + xfer += iprot->readMapEnd(); + } + this->__isset.userPrivileges = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->groupPrivileges.clear(); + uint32_t _size55; + ::apache::thrift::protocol::TType _ktype56; + ::apache::thrift::protocol::TType _vtype57; + xfer += iprot->readMapBegin(_ktype56, _vtype57, _size55); + uint32_t _i59; + for (_i59 = 0; _i59 < _size55; ++_i59) + { + std::string _key60; + xfer += iprot->readString(_key60); + std::vector & _val61 = this->groupPrivileges[_key60]; + { + _val61.clear(); + uint32_t _size62; + ::apache::thrift::protocol::TType _etype65; + xfer += iprot->readListBegin(_etype65, _size62); + _val61.resize(_size62); + uint32_t _i66; + for (_i66 = 0; _i66 < _size62; ++_i66) + { + xfer += _val61[_i66].read(iprot); + } + xfer += iprot->readListEnd(); + } + } + xfer += iprot->readMapEnd(); + } + this->__isset.groupPrivileges = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->rolePrivileges.clear(); + uint32_t _size67; + ::apache::thrift::protocol::TType _ktype68; + ::apache::thrift::protocol::TType _vtype69; + xfer += iprot->readMapBegin(_ktype68, _vtype69, _size67); + uint32_t _i71; + for (_i71 = 0; _i71 < _size67; ++_i71) + { + std::string _key72; + xfer += iprot->readString(_key72); + std::vector & _val73 = this->rolePrivileges[_key72]; + { + _val73.clear(); + uint32_t _size74; + ::apache::thrift::protocol::TType _etype77; + xfer += iprot->readListBegin(_etype77, _size74); + _val73.resize(_size74); + uint32_t _i78; + for (_i78 = 0; _i78 < _size74; ++_i78) + { + xfer += _val73[_i78].read(iprot); + } + xfer += iprot->readListEnd(); + } + } + xfer += iprot->readMapEnd(); + } + this->__isset.rolePrivileges = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t PrincipalPrivilegeSet::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PrincipalPrivilegeSet"); + + xfer += oprot->writeFieldBegin("userPrivileges", ::apache::thrift::protocol::T_MAP, 1); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast(this->userPrivileges.size())); + std::map > ::const_iterator _iter79; + for (_iter79 = this->userPrivileges.begin(); _iter79 != this->userPrivileges.end(); ++_iter79) + { + xfer += oprot->writeString(_iter79->first); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(_iter79->second.size())); + std::vector ::const_iterator _iter80; + for (_iter80 = _iter79->second.begin(); _iter80 != _iter79->second.end(); ++_iter80) + { + xfer += (*_iter80).write(oprot); + } + xfer += oprot->writeListEnd(); + } + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("groupPrivileges", ::apache::thrift::protocol::T_MAP, 2); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast(this->groupPrivileges.size())); + std::map > ::const_iterator _iter81; + for (_iter81 = this->groupPrivileges.begin(); _iter81 != this->groupPrivileges.end(); ++_iter81) + { + xfer += oprot->writeString(_iter81->first); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(_iter81->second.size())); + std::vector ::const_iterator _iter82; + for (_iter82 = _iter81->second.begin(); _iter82 != _iter81->second.end(); ++_iter82) + { + xfer += (*_iter82).write(oprot); + } + xfer += oprot->writeListEnd(); + } + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("rolePrivileges", ::apache::thrift::protocol::T_MAP, 3); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast(this->rolePrivileges.size())); + std::map > ::const_iterator _iter83; + for (_iter83 = this->rolePrivileges.begin(); _iter83 != this->rolePrivileges.end(); ++_iter83) + { + xfer += oprot->writeString(_iter83->first); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(_iter83->second.size())); + std::vector ::const_iterator _iter84; + for (_iter84 = _iter83->second.begin(); _iter84 != _iter83->second.end(); ++_iter84) + { + xfer += (*_iter84).write(oprot); + } + xfer += oprot->writeListEnd(); + } + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PrincipalPrivilegeSet &a, PrincipalPrivilegeSet &b) { + using ::std::swap; + swap(a.userPrivileges, b.userPrivileges); + swap(a.groupPrivileges, b.groupPrivileges); + swap(a.rolePrivileges, b.rolePrivileges); + swap(a.__isset, b.__isset); +} + +PrincipalPrivilegeSet::PrincipalPrivilegeSet(const PrincipalPrivilegeSet& other85) { + userPrivileges = other85.userPrivileges; + groupPrivileges = other85.groupPrivileges; + rolePrivileges = other85.rolePrivileges; + __isset = other85.__isset; +} +PrincipalPrivilegeSet& PrincipalPrivilegeSet::operator=(const PrincipalPrivilegeSet& other86) { + userPrivileges = other86.userPrivileges; + groupPrivileges = other86.groupPrivileges; + rolePrivileges = other86.rolePrivileges; + __isset = other86.__isset; + return *this; +} +void PrincipalPrivilegeSet::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PrincipalPrivilegeSet("; + out << "userPrivileges=" << to_string(userPrivileges); + out << ", " << "groupPrivileges=" << to_string(groupPrivileges); + out << ", " << "rolePrivileges=" << to_string(rolePrivileges); + out << ")"; +} + + +GrantRevokePrivilegeRequest::~GrantRevokePrivilegeRequest() noexcept { +} + + +void GrantRevokePrivilegeRequest::__set_requestType(const GrantRevokeType::type val) { + this->requestType = val; +} + +void GrantRevokePrivilegeRequest::__set_privileges(const PrivilegeBag& val) { + this->privileges = val; +} + +void GrantRevokePrivilegeRequest::__set_revokeGrantOption(const bool val) { + this->revokeGrantOption = val; +__isset.revokeGrantOption = true; +} +std::ostream& operator<<(std::ostream& out, const GrantRevokePrivilegeRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GrantRevokePrivilegeRequest::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast87; + xfer += iprot->readI32(ecast87); + this->requestType = (GrantRevokeType::type)ecast87; + this->__isset.requestType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->privileges.read(iprot); + this->__isset.privileges = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->revokeGrantOption); + this->__isset.revokeGrantOption = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t GrantRevokePrivilegeRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GrantRevokePrivilegeRequest"); + + xfer += oprot->writeFieldBegin("requestType", ::apache::thrift::protocol::T_I32, 1); + xfer += oprot->writeI32((int32_t)this->requestType); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->privileges.write(oprot); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.revokeGrantOption) { + xfer += oprot->writeFieldBegin("revokeGrantOption", ::apache::thrift::protocol::T_BOOL, 3); + xfer += oprot->writeBool(this->revokeGrantOption); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GrantRevokePrivilegeRequest &a, GrantRevokePrivilegeRequest &b) { + using ::std::swap; + swap(a.requestType, b.requestType); + swap(a.privileges, b.privileges); + swap(a.revokeGrantOption, b.revokeGrantOption); + swap(a.__isset, b.__isset); +} + +GrantRevokePrivilegeRequest::GrantRevokePrivilegeRequest(const GrantRevokePrivilegeRequest& other88) { + requestType = other88.requestType; + privileges = other88.privileges; + revokeGrantOption = other88.revokeGrantOption; + __isset = other88.__isset; +} +GrantRevokePrivilegeRequest& GrantRevokePrivilegeRequest::operator=(const GrantRevokePrivilegeRequest& other89) { + requestType = other89.requestType; + privileges = other89.privileges; + revokeGrantOption = other89.revokeGrantOption; + __isset = other89.__isset; + return *this; +} +void GrantRevokePrivilegeRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GrantRevokePrivilegeRequest("; + out << "requestType=" << to_string(requestType); + out << ", " << "privileges=" << to_string(privileges); + out << ", " << "revokeGrantOption="; (__isset.revokeGrantOption ? (out << to_string(revokeGrantOption)) : (out << "")); + out << ")"; +} + + +GrantRevokePrivilegeResponse::~GrantRevokePrivilegeResponse() noexcept { +} + + +void GrantRevokePrivilegeResponse::__set_success(const bool val) { + this->success = val; +__isset.success = true; +} +std::ostream& operator<<(std::ostream& out, const GrantRevokePrivilegeResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GrantRevokePrivilegeResponse::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t GrantRevokePrivilegeResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GrantRevokePrivilegeResponse"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 1); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GrantRevokePrivilegeResponse &a, GrantRevokePrivilegeResponse &b) { + using ::std::swap; + swap(a.success, b.success); + swap(a.__isset, b.__isset); +} + +GrantRevokePrivilegeResponse::GrantRevokePrivilegeResponse(const GrantRevokePrivilegeResponse& other90) { + success = other90.success; + __isset = other90.__isset; +} +GrantRevokePrivilegeResponse& GrantRevokePrivilegeResponse::operator=(const GrantRevokePrivilegeResponse& other91) { + success = other91.success; + __isset = other91.__isset; + return *this; +} +void GrantRevokePrivilegeResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GrantRevokePrivilegeResponse("; + out << "success="; (__isset.success ? (out << to_string(success)) : (out << "")); + out << ")"; +} + + +Role::~Role() noexcept { +} + + +void Role::__set_roleName(const std::string& val) { + this->roleName = val; +} + +void Role::__set_createTime(const int32_t val) { + this->createTime = val; +} + +void Role::__set_ownerName(const std::string& val) { + this->ownerName = val; +} +std::ostream& operator<<(std::ostream& out, const Role& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t Role::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->roleName); + this->__isset.roleName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->createTime); + this->__isset.createTime = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->ownerName); + this->__isset.ownerName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t Role::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("Role"); + + xfer += oprot->writeFieldBegin("roleName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->roleName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32(this->createTime); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("ownerName", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->ownerName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(Role &a, Role &b) { + using ::std::swap; + swap(a.roleName, b.roleName); + swap(a.createTime, b.createTime); + swap(a.ownerName, b.ownerName); + swap(a.__isset, b.__isset); +} + +Role::Role(const Role& other92) { + roleName = other92.roleName; + createTime = other92.createTime; + ownerName = other92.ownerName; + __isset = other92.__isset; +} +Role& Role::operator=(const Role& other93) { + roleName = other93.roleName; + createTime = other93.createTime; + ownerName = other93.ownerName; + __isset = other93.__isset; + return *this; +} +void Role::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "Role("; + out << "roleName=" << to_string(roleName); + out << ", " << "createTime=" << to_string(createTime); + out << ", " << "ownerName=" << to_string(ownerName); + out << ")"; +} + + +RolePrincipalGrant::~RolePrincipalGrant() noexcept { +} + + +void RolePrincipalGrant::__set_roleName(const std::string& val) { + this->roleName = val; +} + +void RolePrincipalGrant::__set_principalName(const std::string& val) { + this->principalName = val; +} + +void RolePrincipalGrant::__set_principalType(const PrincipalType::type val) { + this->principalType = val; +} + +void RolePrincipalGrant::__set_grantOption(const bool val) { + this->grantOption = val; +} + +void RolePrincipalGrant::__set_grantTime(const int32_t val) { + this->grantTime = val; +} + +void RolePrincipalGrant::__set_grantorName(const std::string& val) { + this->grantorName = val; +} + +void RolePrincipalGrant::__set_grantorPrincipalType(const PrincipalType::type val) { + this->grantorPrincipalType = val; +} +std::ostream& operator<<(std::ostream& out, const RolePrincipalGrant& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t RolePrincipalGrant::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->roleName); + this->__isset.roleName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->principalName); + this->__isset.principalName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast94; + xfer += iprot->readI32(ecast94); + this->principalType = (PrincipalType::type)ecast94; + this->__isset.principalType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->grantOption); + this->__isset.grantOption = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->grantTime); + this->__isset.grantTime = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->grantorName); + this->__isset.grantorName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast95; + xfer += iprot->readI32(ecast95); + this->grantorPrincipalType = (PrincipalType::type)ecast95; + this->__isset.grantorPrincipalType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t RolePrincipalGrant::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("RolePrincipalGrant"); + + xfer += oprot->writeFieldBegin("roleName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->roleName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principalName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->principalName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principalType", ::apache::thrift::protocol::T_I32, 3); + xfer += oprot->writeI32((int32_t)this->principalType); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grantOption", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool(this->grantOption); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grantTime", ::apache::thrift::protocol::T_I32, 5); + xfer += oprot->writeI32(this->grantTime); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grantorName", ::apache::thrift::protocol::T_STRING, 6); + xfer += oprot->writeString(this->grantorName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("grantorPrincipalType", ::apache::thrift::protocol::T_I32, 7); + xfer += oprot->writeI32((int32_t)this->grantorPrincipalType); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(RolePrincipalGrant &a, RolePrincipalGrant &b) { + using ::std::swap; + swap(a.roleName, b.roleName); + swap(a.principalName, b.principalName); + swap(a.principalType, b.principalType); + swap(a.grantOption, b.grantOption); + swap(a.grantTime, b.grantTime); + swap(a.grantorName, b.grantorName); + swap(a.grantorPrincipalType, b.grantorPrincipalType); + swap(a.__isset, b.__isset); +} + +RolePrincipalGrant::RolePrincipalGrant(const RolePrincipalGrant& other96) { + roleName = other96.roleName; + principalName = other96.principalName; + principalType = other96.principalType; + grantOption = other96.grantOption; + grantTime = other96.grantTime; + grantorName = other96.grantorName; + grantorPrincipalType = other96.grantorPrincipalType; + __isset = other96.__isset; +} +RolePrincipalGrant& RolePrincipalGrant::operator=(const RolePrincipalGrant& other97) { + roleName = other97.roleName; + principalName = other97.principalName; + principalType = other97.principalType; + grantOption = other97.grantOption; + grantTime = other97.grantTime; + grantorName = other97.grantorName; + grantorPrincipalType = other97.grantorPrincipalType; + __isset = other97.__isset; + return *this; +} +void RolePrincipalGrant::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "RolePrincipalGrant("; + out << "roleName=" << to_string(roleName); + out << ", " << "principalName=" << to_string(principalName); + out << ", " << "principalType=" << to_string(principalType); + out << ", " << "grantOption=" << to_string(grantOption); + out << ", " << "grantTime=" << to_string(grantTime); + out << ", " << "grantorName=" << to_string(grantorName); + out << ", " << "grantorPrincipalType=" << to_string(grantorPrincipalType); + out << ")"; +} + + +GetRoleGrantsForPrincipalRequest::~GetRoleGrantsForPrincipalRequest() noexcept { +} + + +void GetRoleGrantsForPrincipalRequest::__set_principal_name(const std::string& val) { + this->principal_name = val; +} + +void GetRoleGrantsForPrincipalRequest::__set_principal_type(const PrincipalType::type val) { + this->principal_type = val; +} +std::ostream& operator<<(std::ostream& out, const GetRoleGrantsForPrincipalRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GetRoleGrantsForPrincipalRequest::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_principal_name = false; + bool isset_principal_type = 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_STRING) { + xfer += iprot->readString(this->principal_name); + isset_principal_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast98; + xfer += iprot->readI32(ecast98); + this->principal_type = (PrincipalType::type)ecast98; + isset_principal_type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_principal_name) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_principal_type) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t GetRoleGrantsForPrincipalRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GetRoleGrantsForPrincipalRequest"); + + xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->principal_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32((int32_t)this->principal_type); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GetRoleGrantsForPrincipalRequest &a, GetRoleGrantsForPrincipalRequest &b) { + using ::std::swap; + swap(a.principal_name, b.principal_name); + swap(a.principal_type, b.principal_type); +} + +GetRoleGrantsForPrincipalRequest::GetRoleGrantsForPrincipalRequest(const GetRoleGrantsForPrincipalRequest& other99) { + principal_name = other99.principal_name; + principal_type = other99.principal_type; +} +GetRoleGrantsForPrincipalRequest& GetRoleGrantsForPrincipalRequest::operator=(const GetRoleGrantsForPrincipalRequest& other100) { + principal_name = other100.principal_name; + principal_type = other100.principal_type; + return *this; +} +void GetRoleGrantsForPrincipalRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GetRoleGrantsForPrincipalRequest("; + out << "principal_name=" << to_string(principal_name); + out << ", " << "principal_type=" << to_string(principal_type); + out << ")"; +} + + +GetRoleGrantsForPrincipalResponse::~GetRoleGrantsForPrincipalResponse() noexcept { +} + + +void GetRoleGrantsForPrincipalResponse::__set_principalGrants(const std::vector & val) { + this->principalGrants = val; +} +std::ostream& operator<<(std::ostream& out, const GetRoleGrantsForPrincipalResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GetRoleGrantsForPrincipalResponse::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_principalGrants = 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_LIST) { + { + this->principalGrants.clear(); + uint32_t _size101; + ::apache::thrift::protocol::TType _etype104; + xfer += iprot->readListBegin(_etype104, _size101); + this->principalGrants.resize(_size101); + uint32_t _i105; + for (_i105 = 0; _i105 < _size101; ++_i105) + { + xfer += this->principalGrants[_i105].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_principalGrants = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_principalGrants) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t GetRoleGrantsForPrincipalResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GetRoleGrantsForPrincipalResponse"); + + xfer += oprot->writeFieldBegin("principalGrants", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->principalGrants.size())); + std::vector ::const_iterator _iter106; + for (_iter106 = this->principalGrants.begin(); _iter106 != this->principalGrants.end(); ++_iter106) + { + xfer += (*_iter106).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GetRoleGrantsForPrincipalResponse &a, GetRoleGrantsForPrincipalResponse &b) { + using ::std::swap; + swap(a.principalGrants, b.principalGrants); +} + +GetRoleGrantsForPrincipalResponse::GetRoleGrantsForPrincipalResponse(const GetRoleGrantsForPrincipalResponse& other107) { + principalGrants = other107.principalGrants; +} +GetRoleGrantsForPrincipalResponse& GetRoleGrantsForPrincipalResponse::operator=(const GetRoleGrantsForPrincipalResponse& other108) { + principalGrants = other108.principalGrants; + return *this; +} +void GetRoleGrantsForPrincipalResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GetRoleGrantsForPrincipalResponse("; + out << "principalGrants=" << to_string(principalGrants); + out << ")"; +} + + +GetPrincipalsInRoleRequest::~GetPrincipalsInRoleRequest() noexcept { +} + + +void GetPrincipalsInRoleRequest::__set_roleName(const std::string& val) { + this->roleName = val; +} +std::ostream& operator<<(std::ostream& out, const GetPrincipalsInRoleRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GetPrincipalsInRoleRequest::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_roleName = 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_STRING) { + xfer += iprot->readString(this->roleName); + isset_roleName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_roleName) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t GetPrincipalsInRoleRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GetPrincipalsInRoleRequest"); + + xfer += oprot->writeFieldBegin("roleName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->roleName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GetPrincipalsInRoleRequest &a, GetPrincipalsInRoleRequest &b) { + using ::std::swap; + swap(a.roleName, b.roleName); +} + +GetPrincipalsInRoleRequest::GetPrincipalsInRoleRequest(const GetPrincipalsInRoleRequest& other109) { + roleName = other109.roleName; +} +GetPrincipalsInRoleRequest& GetPrincipalsInRoleRequest::operator=(const GetPrincipalsInRoleRequest& other110) { + roleName = other110.roleName; + return *this; +} +void GetPrincipalsInRoleRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GetPrincipalsInRoleRequest("; + out << "roleName=" << to_string(roleName); + out << ")"; +} + + +GetPrincipalsInRoleResponse::~GetPrincipalsInRoleResponse() noexcept { +} + + +void GetPrincipalsInRoleResponse::__set_principalGrants(const std::vector & val) { + this->principalGrants = val; +} +std::ostream& operator<<(std::ostream& out, const GetPrincipalsInRoleResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GetPrincipalsInRoleResponse::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_principalGrants = 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_LIST) { + { + this->principalGrants.clear(); + uint32_t _size111; + ::apache::thrift::protocol::TType _etype114; + xfer += iprot->readListBegin(_etype114, _size111); + this->principalGrants.resize(_size111); + uint32_t _i115; + for (_i115 = 0; _i115 < _size111; ++_i115) + { + xfer += this->principalGrants[_i115].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_principalGrants = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_principalGrants) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t GetPrincipalsInRoleResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GetPrincipalsInRoleResponse"); + + xfer += oprot->writeFieldBegin("principalGrants", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->principalGrants.size())); + std::vector ::const_iterator _iter116; + for (_iter116 = this->principalGrants.begin(); _iter116 != this->principalGrants.end(); ++_iter116) + { + xfer += (*_iter116).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GetPrincipalsInRoleResponse &a, GetPrincipalsInRoleResponse &b) { + using ::std::swap; + swap(a.principalGrants, b.principalGrants); +} + +GetPrincipalsInRoleResponse::GetPrincipalsInRoleResponse(const GetPrincipalsInRoleResponse& other117) { + principalGrants = other117.principalGrants; +} +GetPrincipalsInRoleResponse& GetPrincipalsInRoleResponse::operator=(const GetPrincipalsInRoleResponse& other118) { + principalGrants = other118.principalGrants; + return *this; +} +void GetPrincipalsInRoleResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GetPrincipalsInRoleResponse("; + out << "principalGrants=" << to_string(principalGrants); + out << ")"; +} + + +GrantRevokeRoleRequest::~GrantRevokeRoleRequest() noexcept { +} + + +void GrantRevokeRoleRequest::__set_requestType(const GrantRevokeType::type val) { + this->requestType = val; +} + +void GrantRevokeRoleRequest::__set_roleName(const std::string& val) { + this->roleName = val; +} + +void GrantRevokeRoleRequest::__set_principalName(const std::string& val) { + this->principalName = val; +} + +void GrantRevokeRoleRequest::__set_principalType(const PrincipalType::type val) { + this->principalType = val; +} + +void GrantRevokeRoleRequest::__set_grantor(const std::string& val) { + this->grantor = val; +__isset.grantor = true; +} + +void GrantRevokeRoleRequest::__set_grantorType(const PrincipalType::type val) { + this->grantorType = val; +__isset.grantorType = true; +} + +void GrantRevokeRoleRequest::__set_grantOption(const bool val) { + this->grantOption = val; +__isset.grantOption = true; +} +std::ostream& operator<<(std::ostream& out, const GrantRevokeRoleRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GrantRevokeRoleRequest::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast119; + xfer += iprot->readI32(ecast119); + this->requestType = (GrantRevokeType::type)ecast119; + this->__isset.requestType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->roleName); + this->__isset.roleName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->principalName); + this->__isset.principalName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast120; + xfer += iprot->readI32(ecast120); + this->principalType = (PrincipalType::type)ecast120; + this->__isset.principalType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->grantor); + this->__isset.grantor = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast121; + xfer += iprot->readI32(ecast121); + this->grantorType = (PrincipalType::type)ecast121; + this->__isset.grantorType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->grantOption); + this->__isset.grantOption = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t GrantRevokeRoleRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GrantRevokeRoleRequest"); + + xfer += oprot->writeFieldBegin("requestType", ::apache::thrift::protocol::T_I32, 1); + xfer += oprot->writeI32((int32_t)this->requestType); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("roleName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->roleName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principalName", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->principalName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("principalType", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32((int32_t)this->principalType); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.grantor) { + xfer += oprot->writeFieldBegin("grantor", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->grantor); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.grantorType) { + xfer += oprot->writeFieldBegin("grantorType", ::apache::thrift::protocol::T_I32, 6); + xfer += oprot->writeI32((int32_t)this->grantorType); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.grantOption) { + xfer += oprot->writeFieldBegin("grantOption", ::apache::thrift::protocol::T_BOOL, 7); + xfer += oprot->writeBool(this->grantOption); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GrantRevokeRoleRequest &a, GrantRevokeRoleRequest &b) { + using ::std::swap; + swap(a.requestType, b.requestType); + swap(a.roleName, b.roleName); + swap(a.principalName, b.principalName); + swap(a.principalType, b.principalType); + swap(a.grantor, b.grantor); + swap(a.grantorType, b.grantorType); + swap(a.grantOption, b.grantOption); + swap(a.__isset, b.__isset); +} + +GrantRevokeRoleRequest::GrantRevokeRoleRequest(const GrantRevokeRoleRequest& other122) { + requestType = other122.requestType; + roleName = other122.roleName; + principalName = other122.principalName; + principalType = other122.principalType; + grantor = other122.grantor; + grantorType = other122.grantorType; + grantOption = other122.grantOption; + __isset = other122.__isset; +} +GrantRevokeRoleRequest& GrantRevokeRoleRequest::operator=(const GrantRevokeRoleRequest& other123) { + requestType = other123.requestType; + roleName = other123.roleName; + principalName = other123.principalName; + principalType = other123.principalType; + grantor = other123.grantor; + grantorType = other123.grantorType; + grantOption = other123.grantOption; + __isset = other123.__isset; + return *this; +} +void GrantRevokeRoleRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GrantRevokeRoleRequest("; + out << "requestType=" << to_string(requestType); + out << ", " << "roleName=" << to_string(roleName); + out << ", " << "principalName=" << to_string(principalName); + out << ", " << "principalType=" << to_string(principalType); + out << ", " << "grantor="; (__isset.grantor ? (out << to_string(grantor)) : (out << "")); + out << ", " << "grantorType="; (__isset.grantorType ? (out << to_string(grantorType)) : (out << "")); + out << ", " << "grantOption="; (__isset.grantOption ? (out << to_string(grantOption)) : (out << "")); + out << ")"; +} + + +GrantRevokeRoleResponse::~GrantRevokeRoleResponse() noexcept { +} + + +void GrantRevokeRoleResponse::__set_success(const bool val) { + this->success = val; +__isset.success = true; +} +std::ostream& operator<<(std::ostream& out, const GrantRevokeRoleResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GrantRevokeRoleResponse::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->success); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t GrantRevokeRoleResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GrantRevokeRoleResponse"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 1); + xfer += oprot->writeBool(this->success); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GrantRevokeRoleResponse &a, GrantRevokeRoleResponse &b) { + using ::std::swap; + swap(a.success, b.success); + swap(a.__isset, b.__isset); +} + +GrantRevokeRoleResponse::GrantRevokeRoleResponse(const GrantRevokeRoleResponse& other124) { + success = other124.success; + __isset = other124.__isset; +} +GrantRevokeRoleResponse& GrantRevokeRoleResponse::operator=(const GrantRevokeRoleResponse& other125) { + success = other125.success; + __isset = other125.__isset; + return *this; +} +void GrantRevokeRoleResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GrantRevokeRoleResponse("; + out << "success="; (__isset.success ? (out << to_string(success)) : (out << "")); + out << ")"; +} + + +Database::~Database() noexcept { +} + + +void Database::__set_name(const std::string& val) { + this->name = val; +} + +void Database::__set_description(const std::string& val) { + this->description = val; +} + +void Database::__set_locationUri(const std::string& val) { + this->locationUri = val; +} + +void Database::__set_parameters(const std::map & val) { + this->parameters = val; +} + +void Database::__set_privileges(const PrincipalPrivilegeSet& val) { + this->privileges = val; +__isset.privileges = true; +} + +void Database::__set_ownerName(const std::string& val) { + this->ownerName = val; +__isset.ownerName = true; +} + +void Database::__set_ownerType(const PrincipalType::type val) { + this->ownerType = val; +__isset.ownerType = true; +} +std::ostream& operator<<(std::ostream& out, const Database& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t Database::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->name); + this->__isset.name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->description); + this->__isset.description = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->locationUri); + this->__isset.locationUri = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->parameters.clear(); + uint32_t _size126; + ::apache::thrift::protocol::TType _ktype127; + ::apache::thrift::protocol::TType _vtype128; + xfer += iprot->readMapBegin(_ktype127, _vtype128, _size126); + uint32_t _i130; + for (_i130 = 0; _i130 < _size126; ++_i130) + { + std::string _key131; + xfer += iprot->readString(_key131); + std::string& _val132 = this->parameters[_key131]; + xfer += iprot->readString(_val132); + } + xfer += iprot->readMapEnd(); + } + this->__isset.parameters = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->privileges.read(iprot); + this->__isset.privileges = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->ownerName); + this->__isset.ownerName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast133; + xfer += iprot->readI32(ecast133); + this->ownerType = (PrincipalType::type)ecast133; + this->__isset.ownerType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t Database::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("Database"); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("description", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->description); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("locationUri", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->locationUri); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 4); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->parameters.size())); + std::map ::const_iterator _iter134; + for (_iter134 = this->parameters.begin(); _iter134 != this->parameters.end(); ++_iter134) + { + xfer += oprot->writeString(_iter134->first); + xfer += oprot->writeString(_iter134->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + if (this->__isset.privileges) { + xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 5); + xfer += this->privileges.write(oprot); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.ownerName) { + xfer += oprot->writeFieldBegin("ownerName", ::apache::thrift::protocol::T_STRING, 6); + xfer += oprot->writeString(this->ownerName); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.ownerType) { + xfer += oprot->writeFieldBegin("ownerType", ::apache::thrift::protocol::T_I32, 7); + xfer += oprot->writeI32((int32_t)this->ownerType); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(Database &a, Database &b) { + using ::std::swap; + swap(a.name, b.name); + swap(a.description, b.description); + swap(a.locationUri, b.locationUri); + swap(a.parameters, b.parameters); + swap(a.privileges, b.privileges); + swap(a.ownerName, b.ownerName); + swap(a.ownerType, b.ownerType); + swap(a.__isset, b.__isset); +} + +Database::Database(const Database& other135) { + name = other135.name; + description = other135.description; + locationUri = other135.locationUri; + parameters = other135.parameters; + privileges = other135.privileges; + ownerName = other135.ownerName; + ownerType = other135.ownerType; + __isset = other135.__isset; +} +Database& Database::operator=(const Database& other136) { + name = other136.name; + description = other136.description; + locationUri = other136.locationUri; + parameters = other136.parameters; + privileges = other136.privileges; + ownerName = other136.ownerName; + ownerType = other136.ownerType; + __isset = other136.__isset; + return *this; +} +void Database::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "Database("; + out << "name=" << to_string(name); + out << ", " << "description=" << to_string(description); + out << ", " << "locationUri=" << to_string(locationUri); + out << ", " << "parameters=" << to_string(parameters); + out << ", " << "privileges="; (__isset.privileges ? (out << to_string(privileges)) : (out << "")); + out << ", " << "ownerName="; (__isset.ownerName ? (out << to_string(ownerName)) : (out << "")); + out << ", " << "ownerType="; (__isset.ownerType ? (out << to_string(ownerType)) : (out << "")); + out << ")"; +} + + +SerDeInfo::~SerDeInfo() noexcept { +} + + +void SerDeInfo::__set_name(const std::string& val) { + this->name = val; +} + +void SerDeInfo::__set_serializationLib(const std::string& val) { + this->serializationLib = val; +} + +void SerDeInfo::__set_parameters(const std::map & val) { + this->parameters = val; +} +std::ostream& operator<<(std::ostream& out, const SerDeInfo& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t SerDeInfo::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->name); + this->__isset.name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->serializationLib); + this->__isset.serializationLib = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->parameters.clear(); + uint32_t _size137; + ::apache::thrift::protocol::TType _ktype138; + ::apache::thrift::protocol::TType _vtype139; + xfer += iprot->readMapBegin(_ktype138, _vtype139, _size137); + uint32_t _i141; + for (_i141 = 0; _i141 < _size137; ++_i141) + { + std::string _key142; + xfer += iprot->readString(_key142); + std::string& _val143 = this->parameters[_key142]; + xfer += iprot->readString(_val143); + } + xfer += iprot->readMapEnd(); + } + this->__isset.parameters = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t SerDeInfo::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("SerDeInfo"); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("serializationLib", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->serializationLib); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 3); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->parameters.size())); + std::map ::const_iterator _iter144; + for (_iter144 = this->parameters.begin(); _iter144 != this->parameters.end(); ++_iter144) + { + xfer += oprot->writeString(_iter144->first); + xfer += oprot->writeString(_iter144->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(SerDeInfo &a, SerDeInfo &b) { + using ::std::swap; + swap(a.name, b.name); + swap(a.serializationLib, b.serializationLib); + swap(a.parameters, b.parameters); + swap(a.__isset, b.__isset); +} + +SerDeInfo::SerDeInfo(const SerDeInfo& other145) { + name = other145.name; + serializationLib = other145.serializationLib; + parameters = other145.parameters; + __isset = other145.__isset; +} +SerDeInfo& SerDeInfo::operator=(const SerDeInfo& other146) { + name = other146.name; + serializationLib = other146.serializationLib; + parameters = other146.parameters; + __isset = other146.__isset; + return *this; +} +void SerDeInfo::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "SerDeInfo("; + out << "name=" << to_string(name); + out << ", " << "serializationLib=" << to_string(serializationLib); + out << ", " << "parameters=" << to_string(parameters); + out << ")"; +} + + +Order::~Order() noexcept { +} + + +void Order::__set_col(const std::string& val) { + this->col = val; +} + +void Order::__set_order(const int32_t val) { + this->order = val; +} +std::ostream& operator<<(std::ostream& out, const Order& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t Order::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->col); + this->__isset.col = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->order); + this->__isset.order = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t Order::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("Order"); + + xfer += oprot->writeFieldBegin("col", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->col); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("order", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32(this->order); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(Order &a, Order &b) { + using ::std::swap; + swap(a.col, b.col); + swap(a.order, b.order); + swap(a.__isset, b.__isset); +} + +Order::Order(const Order& other147) { + col = other147.col; + order = other147.order; + __isset = other147.__isset; +} +Order& Order::operator=(const Order& other148) { + col = other148.col; + order = other148.order; + __isset = other148.__isset; + return *this; +} +void Order::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "Order("; + out << "col=" << to_string(col); + out << ", " << "order=" << to_string(order); + out << ")"; +} + + +SkewedInfo::~SkewedInfo() noexcept { +} + + +void SkewedInfo::__set_skewedColNames(const std::vector & val) { + this->skewedColNames = val; +} + +void SkewedInfo::__set_skewedColValues(const std::vector > & val) { + this->skewedColValues = val; +} + +void SkewedInfo::__set_skewedColValueLocationMaps(const std::map & val) { + this->skewedColValueLocationMaps = val; +} +std::ostream& operator<<(std::ostream& out, const SkewedInfo& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t SkewedInfo::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->skewedColNames.clear(); + uint32_t _size149; + ::apache::thrift::protocol::TType _etype152; + xfer += iprot->readListBegin(_etype152, _size149); + this->skewedColNames.resize(_size149); + uint32_t _i153; + for (_i153 = 0; _i153 < _size149; ++_i153) + { + xfer += iprot->readString(this->skewedColNames[_i153]); + } + xfer += iprot->readListEnd(); + } + this->__isset.skewedColNames = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->skewedColValues.clear(); + uint32_t _size154; + ::apache::thrift::protocol::TType _etype157; + xfer += iprot->readListBegin(_etype157, _size154); + this->skewedColValues.resize(_size154); + uint32_t _i158; + for (_i158 = 0; _i158 < _size154; ++_i158) + { + { + this->skewedColValues[_i158].clear(); + uint32_t _size159; + ::apache::thrift::protocol::TType _etype162; + xfer += iprot->readListBegin(_etype162, _size159); + this->skewedColValues[_i158].resize(_size159); + uint32_t _i163; + for (_i163 = 0; _i163 < _size159; ++_i163) + { + xfer += iprot->readString(this->skewedColValues[_i158][_i163]); + } + xfer += iprot->readListEnd(); + } + } + xfer += iprot->readListEnd(); + } + this->__isset.skewedColValues = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->skewedColValueLocationMaps.clear(); + uint32_t _size164; + ::apache::thrift::protocol::TType _ktype165; + ::apache::thrift::protocol::TType _vtype166; + xfer += iprot->readMapBegin(_ktype165, _vtype166, _size164); + uint32_t _i168; + for (_i168 = 0; _i168 < _size164; ++_i168) + { + std::string _key169; + xfer += iprot->readString(_key169); + std::string& _val170 = this->skewedColValueLocationMaps[_key169]; + xfer += iprot->readString(_val170); + } + xfer += iprot->readMapEnd(); + } + this->__isset.skewedColValueLocationMaps = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t SkewedInfo::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("SkewedInfo"); + + xfer += oprot->writeFieldBegin("skewedColNames", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->skewedColNames.size())); + std::vector ::const_iterator _iter171; + for (_iter171 = this->skewedColNames.begin(); _iter171 != this->skewedColNames.end(); ++_iter171) + { + xfer += oprot->writeString((*_iter171)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("skewedColValues", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_LIST, static_cast(this->skewedColValues.size())); + std::vector > ::const_iterator _iter172; + for (_iter172 = this->skewedColValues.begin(); _iter172 != this->skewedColValues.end(); ++_iter172) + { + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*_iter172).size())); + std::vector ::const_iterator _iter173; + for (_iter173 = (*_iter172).begin(); _iter173 != (*_iter172).end(); ++_iter173) + { + xfer += oprot->writeString((*_iter173)); + } + xfer += oprot->writeListEnd(); + } + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("skewedColValueLocationMaps", ::apache::thrift::protocol::T_MAP, 3); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->skewedColValueLocationMaps.size())); + std::map ::const_iterator _iter174; + for (_iter174 = this->skewedColValueLocationMaps.begin(); _iter174 != this->skewedColValueLocationMaps.end(); ++_iter174) + { + xfer += oprot->writeString(_iter174->first); + xfer += oprot->writeString(_iter174->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(SkewedInfo &a, SkewedInfo &b) { + using ::std::swap; + swap(a.skewedColNames, b.skewedColNames); + swap(a.skewedColValues, b.skewedColValues); + swap(a.skewedColValueLocationMaps, b.skewedColValueLocationMaps); + swap(a.__isset, b.__isset); +} + +SkewedInfo::SkewedInfo(const SkewedInfo& other175) { + skewedColNames = other175.skewedColNames; + skewedColValues = other175.skewedColValues; + skewedColValueLocationMaps = other175.skewedColValueLocationMaps; + __isset = other175.__isset; +} +SkewedInfo& SkewedInfo::operator=(const SkewedInfo& other176) { + skewedColNames = other176.skewedColNames; + skewedColValues = other176.skewedColValues; + skewedColValueLocationMaps = other176.skewedColValueLocationMaps; + __isset = other176.__isset; + return *this; +} +void SkewedInfo::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "SkewedInfo("; + out << "skewedColNames=" << to_string(skewedColNames); + out << ", " << "skewedColValues=" << to_string(skewedColValues); + out << ", " << "skewedColValueLocationMaps=" << to_string(skewedColValueLocationMaps); + out << ")"; +} + + +StorageDescriptor::~StorageDescriptor() noexcept { +} + + +void StorageDescriptor::__set_cols(const std::vector & val) { + this->cols = val; +} + +void StorageDescriptor::__set_location(const std::string& val) { + this->location = val; +} + +void StorageDescriptor::__set_inputFormat(const std::string& val) { + this->inputFormat = val; +} + +void StorageDescriptor::__set_outputFormat(const std::string& val) { + this->outputFormat = val; +} + +void StorageDescriptor::__set_compressed(const bool val) { + this->compressed = val; +} + +void StorageDescriptor::__set_numBuckets(const int32_t val) { + this->numBuckets = val; +} + +void StorageDescriptor::__set_serdeInfo(const SerDeInfo& val) { + this->serdeInfo = val; +} + +void StorageDescriptor::__set_bucketCols(const std::vector & val) { + this->bucketCols = val; +} + +void StorageDescriptor::__set_sortCols(const std::vector & val) { + this->sortCols = val; +} + +void StorageDescriptor::__set_parameters(const std::map & val) { + this->parameters = val; +} + +void StorageDescriptor::__set_skewedInfo(const SkewedInfo& val) { + this->skewedInfo = val; +__isset.skewedInfo = true; +} + +void StorageDescriptor::__set_storedAsSubDirectories(const bool val) { + this->storedAsSubDirectories = val; +__isset.storedAsSubDirectories = true; +} +std::ostream& operator<<(std::ostream& out, const StorageDescriptor& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t StorageDescriptor::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->cols.clear(); + uint32_t _size177; + ::apache::thrift::protocol::TType _etype180; + xfer += iprot->readListBegin(_etype180, _size177); + this->cols.resize(_size177); + uint32_t _i181; + for (_i181 = 0; _i181 < _size177; ++_i181) + { + xfer += this->cols[_i181].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.cols = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->location); + this->__isset.location = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->inputFormat); + this->__isset.inputFormat = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->outputFormat); + this->__isset.outputFormat = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->compressed); + this->__isset.compressed = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->numBuckets); + this->__isset.numBuckets = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->serdeInfo.read(iprot); + this->__isset.serdeInfo = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 8: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->bucketCols.clear(); + uint32_t _size182; + ::apache::thrift::protocol::TType _etype185; + xfer += iprot->readListBegin(_etype185, _size182); + this->bucketCols.resize(_size182); + uint32_t _i186; + for (_i186 = 0; _i186 < _size182; ++_i186) + { + xfer += iprot->readString(this->bucketCols[_i186]); + } + xfer += iprot->readListEnd(); + } + this->__isset.bucketCols = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 9: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->sortCols.clear(); + uint32_t _size187; + ::apache::thrift::protocol::TType _etype190; + xfer += iprot->readListBegin(_etype190, _size187); + this->sortCols.resize(_size187); + uint32_t _i191; + for (_i191 = 0; _i191 < _size187; ++_i191) + { + xfer += this->sortCols[_i191].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.sortCols = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 10: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->parameters.clear(); + uint32_t _size192; + ::apache::thrift::protocol::TType _ktype193; + ::apache::thrift::protocol::TType _vtype194; + xfer += iprot->readMapBegin(_ktype193, _vtype194, _size192); + uint32_t _i196; + for (_i196 = 0; _i196 < _size192; ++_i196) + { + std::string _key197; + xfer += iprot->readString(_key197); + std::string& _val198 = this->parameters[_key197]; + xfer += iprot->readString(_val198); + } + xfer += iprot->readMapEnd(); + } + this->__isset.parameters = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 11: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->skewedInfo.read(iprot); + this->__isset.skewedInfo = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 12: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->storedAsSubDirectories); + this->__isset.storedAsSubDirectories = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t StorageDescriptor::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("StorageDescriptor"); + + xfer += oprot->writeFieldBegin("cols", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->cols.size())); + std::vector ::const_iterator _iter199; + for (_iter199 = this->cols.begin(); _iter199 != this->cols.end(); ++_iter199) + { + xfer += (*_iter199).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("location", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->location); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("inputFormat", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->inputFormat); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("outputFormat", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->outputFormat); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("compressed", ::apache::thrift::protocol::T_BOOL, 5); + xfer += oprot->writeBool(this->compressed); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("numBuckets", ::apache::thrift::protocol::T_I32, 6); + xfer += oprot->writeI32(this->numBuckets); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("serdeInfo", ::apache::thrift::protocol::T_STRUCT, 7); + xfer += this->serdeInfo.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("bucketCols", ::apache::thrift::protocol::T_LIST, 8); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->bucketCols.size())); + std::vector ::const_iterator _iter200; + for (_iter200 = this->bucketCols.begin(); _iter200 != this->bucketCols.end(); ++_iter200) + { + xfer += oprot->writeString((*_iter200)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("sortCols", ::apache::thrift::protocol::T_LIST, 9); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->sortCols.size())); + std::vector ::const_iterator _iter201; + for (_iter201 = this->sortCols.begin(); _iter201 != this->sortCols.end(); ++_iter201) + { + xfer += (*_iter201).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 10); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->parameters.size())); + std::map ::const_iterator _iter202; + for (_iter202 = this->parameters.begin(); _iter202 != this->parameters.end(); ++_iter202) + { + xfer += oprot->writeString(_iter202->first); + xfer += oprot->writeString(_iter202->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + if (this->__isset.skewedInfo) { + xfer += oprot->writeFieldBegin("skewedInfo", ::apache::thrift::protocol::T_STRUCT, 11); + xfer += this->skewedInfo.write(oprot); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.storedAsSubDirectories) { + xfer += oprot->writeFieldBegin("storedAsSubDirectories", ::apache::thrift::protocol::T_BOOL, 12); + xfer += oprot->writeBool(this->storedAsSubDirectories); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(StorageDescriptor &a, StorageDescriptor &b) { + using ::std::swap; + swap(a.cols, b.cols); + swap(a.location, b.location); + swap(a.inputFormat, b.inputFormat); + swap(a.outputFormat, b.outputFormat); + swap(a.compressed, b.compressed); + swap(a.numBuckets, b.numBuckets); + swap(a.serdeInfo, b.serdeInfo); + swap(a.bucketCols, b.bucketCols); + swap(a.sortCols, b.sortCols); + swap(a.parameters, b.parameters); + swap(a.skewedInfo, b.skewedInfo); + swap(a.storedAsSubDirectories, b.storedAsSubDirectories); + swap(a.__isset, b.__isset); +} + +StorageDescriptor::StorageDescriptor(const StorageDescriptor& other203) { + cols = other203.cols; + location = other203.location; + inputFormat = other203.inputFormat; + outputFormat = other203.outputFormat; + compressed = other203.compressed; + numBuckets = other203.numBuckets; + serdeInfo = other203.serdeInfo; + bucketCols = other203.bucketCols; + sortCols = other203.sortCols; + parameters = other203.parameters; + skewedInfo = other203.skewedInfo; + storedAsSubDirectories = other203.storedAsSubDirectories; + __isset = other203.__isset; +} +StorageDescriptor& StorageDescriptor::operator=(const StorageDescriptor& other204) { + cols = other204.cols; + location = other204.location; + inputFormat = other204.inputFormat; + outputFormat = other204.outputFormat; + compressed = other204.compressed; + numBuckets = other204.numBuckets; + serdeInfo = other204.serdeInfo; + bucketCols = other204.bucketCols; + sortCols = other204.sortCols; + parameters = other204.parameters; + skewedInfo = other204.skewedInfo; + storedAsSubDirectories = other204.storedAsSubDirectories; + __isset = other204.__isset; + return *this; +} +void StorageDescriptor::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "StorageDescriptor("; + out << "cols=" << to_string(cols); + out << ", " << "location=" << to_string(location); + out << ", " << "inputFormat=" << to_string(inputFormat); + out << ", " << "outputFormat=" << to_string(outputFormat); + out << ", " << "compressed=" << to_string(compressed); + out << ", " << "numBuckets=" << to_string(numBuckets); + out << ", " << "serdeInfo=" << to_string(serdeInfo); + out << ", " << "bucketCols=" << to_string(bucketCols); + out << ", " << "sortCols=" << to_string(sortCols); + out << ", " << "parameters=" << to_string(parameters); + out << ", " << "skewedInfo="; (__isset.skewedInfo ? (out << to_string(skewedInfo)) : (out << "")); + out << ", " << "storedAsSubDirectories="; (__isset.storedAsSubDirectories ? (out << to_string(storedAsSubDirectories)) : (out << "")); + out << ")"; +} + + +Table::~Table() noexcept { +} + + +void Table::__set_tableName(const std::string& val) { + this->tableName = val; +} + +void Table::__set_dbName(const std::string& val) { + this->dbName = val; +} + +void Table::__set_owner(const std::string& val) { + this->owner = val; +} + +void Table::__set_createTime(const int32_t val) { + this->createTime = val; +} + +void Table::__set_lastAccessTime(const int32_t val) { + this->lastAccessTime = val; +} + +void Table::__set_retention(const int32_t val) { + this->retention = val; +} + +void Table::__set_sd(const StorageDescriptor& val) { + this->sd = val; +} + +void Table::__set_partitionKeys(const std::vector & val) { + this->partitionKeys = val; +} + +void Table::__set_parameters(const std::map & val) { + this->parameters = val; +} + +void Table::__set_viewOriginalText(const std::string& val) { + this->viewOriginalText = val; +} + +void Table::__set_viewExpandedText(const std::string& val) { + this->viewExpandedText = val; +} + +void Table::__set_tableType(const std::string& val) { + this->tableType = val; +} + +void Table::__set_privileges(const PrincipalPrivilegeSet& val) { + this->privileges = val; +__isset.privileges = true; +} + +void Table::__set_temporary(const bool val) { + this->temporary = val; +__isset.temporary = true; +} + +void Table::__set_rewriteEnabled(const bool val) { + this->rewriteEnabled = val; +__isset.rewriteEnabled = true; +} +std::ostream& operator<<(std::ostream& out, const Table& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t Table::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tableName); + this->__isset.tableName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->owner); + this->__isset.owner = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->createTime); + this->__isset.createTime = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->lastAccessTime); + this->__isset.lastAccessTime = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->retention); + this->__isset.retention = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->sd.read(iprot); + this->__isset.sd = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 8: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->partitionKeys.clear(); + uint32_t _size205; + ::apache::thrift::protocol::TType _etype208; + xfer += iprot->readListBegin(_etype208, _size205); + this->partitionKeys.resize(_size205); + uint32_t _i209; + for (_i209 = 0; _i209 < _size205; ++_i209) + { + xfer += this->partitionKeys[_i209].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.partitionKeys = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 9: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->parameters.clear(); + uint32_t _size210; + ::apache::thrift::protocol::TType _ktype211; + ::apache::thrift::protocol::TType _vtype212; + xfer += iprot->readMapBegin(_ktype211, _vtype212, _size210); + uint32_t _i214; + for (_i214 = 0; _i214 < _size210; ++_i214) + { + std::string _key215; + xfer += iprot->readString(_key215); + std::string& _val216 = this->parameters[_key215]; + xfer += iprot->readString(_val216); + } + xfer += iprot->readMapEnd(); + } + this->__isset.parameters = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 10: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->viewOriginalText); + this->__isset.viewOriginalText = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 11: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->viewExpandedText); + this->__isset.viewExpandedText = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 12: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tableType); + this->__isset.tableType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 13: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->privileges.read(iprot); + this->__isset.privileges = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 14: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->temporary); + this->__isset.temporary = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 15: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->rewriteEnabled); + this->__isset.rewriteEnabled = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t Table::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("Table"); + + xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->tableName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("owner", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->owner); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32(this->createTime); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("lastAccessTime", ::apache::thrift::protocol::T_I32, 5); + xfer += oprot->writeI32(this->lastAccessTime); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("retention", ::apache::thrift::protocol::T_I32, 6); + xfer += oprot->writeI32(this->retention); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("sd", ::apache::thrift::protocol::T_STRUCT, 7); + xfer += this->sd.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("partitionKeys", ::apache::thrift::protocol::T_LIST, 8); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitionKeys.size())); + std::vector ::const_iterator _iter217; + for (_iter217 = this->partitionKeys.begin(); _iter217 != this->partitionKeys.end(); ++_iter217) + { + xfer += (*_iter217).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 9); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->parameters.size())); + std::map ::const_iterator _iter218; + for (_iter218 = this->parameters.begin(); _iter218 != this->parameters.end(); ++_iter218) + { + xfer += oprot->writeString(_iter218->first); + xfer += oprot->writeString(_iter218->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("viewOriginalText", ::apache::thrift::protocol::T_STRING, 10); + xfer += oprot->writeString(this->viewOriginalText); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("viewExpandedText", ::apache::thrift::protocol::T_STRING, 11); + xfer += oprot->writeString(this->viewExpandedText); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 12); + xfer += oprot->writeString(this->tableType); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.privileges) { + xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 13); + xfer += this->privileges.write(oprot); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.temporary) { + xfer += oprot->writeFieldBegin("temporary", ::apache::thrift::protocol::T_BOOL, 14); + xfer += oprot->writeBool(this->temporary); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.rewriteEnabled) { + xfer += oprot->writeFieldBegin("rewriteEnabled", ::apache::thrift::protocol::T_BOOL, 15); + xfer += oprot->writeBool(this->rewriteEnabled); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(Table &a, Table &b) { + using ::std::swap; + swap(a.tableName, b.tableName); + swap(a.dbName, b.dbName); + swap(a.owner, b.owner); + swap(a.createTime, b.createTime); + swap(a.lastAccessTime, b.lastAccessTime); + swap(a.retention, b.retention); + swap(a.sd, b.sd); + swap(a.partitionKeys, b.partitionKeys); + swap(a.parameters, b.parameters); + swap(a.viewOriginalText, b.viewOriginalText); + swap(a.viewExpandedText, b.viewExpandedText); + swap(a.tableType, b.tableType); + swap(a.privileges, b.privileges); + swap(a.temporary, b.temporary); + swap(a.rewriteEnabled, b.rewriteEnabled); + swap(a.__isset, b.__isset); +} + +Table::Table(const Table& other219) { + tableName = other219.tableName; + dbName = other219.dbName; + owner = other219.owner; + createTime = other219.createTime; + lastAccessTime = other219.lastAccessTime; + retention = other219.retention; + sd = other219.sd; + partitionKeys = other219.partitionKeys; + parameters = other219.parameters; + viewOriginalText = other219.viewOriginalText; + viewExpandedText = other219.viewExpandedText; + tableType = other219.tableType; + privileges = other219.privileges; + temporary = other219.temporary; + rewriteEnabled = other219.rewriteEnabled; + __isset = other219.__isset; +} +Table& Table::operator=(const Table& other220) { + tableName = other220.tableName; + dbName = other220.dbName; + owner = other220.owner; + createTime = other220.createTime; + lastAccessTime = other220.lastAccessTime; + retention = other220.retention; + sd = other220.sd; + partitionKeys = other220.partitionKeys; + parameters = other220.parameters; + viewOriginalText = other220.viewOriginalText; + viewExpandedText = other220.viewExpandedText; + tableType = other220.tableType; + privileges = other220.privileges; + temporary = other220.temporary; + rewriteEnabled = other220.rewriteEnabled; + __isset = other220.__isset; + return *this; +} +void Table::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "Table("; + out << "tableName=" << to_string(tableName); + out << ", " << "dbName=" << to_string(dbName); + out << ", " << "owner=" << to_string(owner); + out << ", " << "createTime=" << to_string(createTime); + out << ", " << "lastAccessTime=" << to_string(lastAccessTime); + out << ", " << "retention=" << to_string(retention); + out << ", " << "sd=" << to_string(sd); + out << ", " << "partitionKeys=" << to_string(partitionKeys); + out << ", " << "parameters=" << to_string(parameters); + out << ", " << "viewOriginalText=" << to_string(viewOriginalText); + out << ", " << "viewExpandedText=" << to_string(viewExpandedText); + out << ", " << "tableType=" << to_string(tableType); + out << ", " << "privileges="; (__isset.privileges ? (out << to_string(privileges)) : (out << "")); + out << ", " << "temporary="; (__isset.temporary ? (out << to_string(temporary)) : (out << "")); + out << ", " << "rewriteEnabled="; (__isset.rewriteEnabled ? (out << to_string(rewriteEnabled)) : (out << "")); + out << ")"; +} + + +Partition::~Partition() noexcept { +} + + +void Partition::__set_values(const std::vector & val) { + this->values = val; +} + +void Partition::__set_dbName(const std::string& val) { + this->dbName = val; +} + +void Partition::__set_tableName(const std::string& val) { + this->tableName = val; +} + +void Partition::__set_createTime(const int32_t val) { + this->createTime = val; +} + +void Partition::__set_lastAccessTime(const int32_t val) { + this->lastAccessTime = val; +} + +void Partition::__set_sd(const StorageDescriptor& val) { + this->sd = val; +} + +void Partition::__set_parameters(const std::map & val) { + this->parameters = val; +} + +void Partition::__set_privileges(const PrincipalPrivilegeSet& val) { + this->privileges = val; +__isset.privileges = true; +} +std::ostream& operator<<(std::ostream& out, const Partition& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t Partition::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->values.clear(); + uint32_t _size221; + ::apache::thrift::protocol::TType _etype224; + xfer += iprot->readListBegin(_etype224, _size221); + this->values.resize(_size221); + uint32_t _i225; + for (_i225 = 0; _i225 < _size221; ++_i225) + { + xfer += iprot->readString(this->values[_i225]); + } + xfer += iprot->readListEnd(); + } + this->__isset.values = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tableName); + this->__isset.tableName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->createTime); + this->__isset.createTime = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->lastAccessTime); + this->__isset.lastAccessTime = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->sd.read(iprot); + this->__isset.sd = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->parameters.clear(); + uint32_t _size226; + ::apache::thrift::protocol::TType _ktype227; + ::apache::thrift::protocol::TType _vtype228; + xfer += iprot->readMapBegin(_ktype227, _vtype228, _size226); + uint32_t _i230; + for (_i230 = 0; _i230 < _size226; ++_i230) + { + std::string _key231; + xfer += iprot->readString(_key231); + std::string& _val232 = this->parameters[_key231]; + xfer += iprot->readString(_val232); + } + xfer += iprot->readMapEnd(); + } + this->__isset.parameters = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 8: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->privileges.read(iprot); + this->__isset.privileges = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t Partition::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("Partition"); + + xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->values.size())); + std::vector ::const_iterator _iter233; + for (_iter233 = this->values.begin(); _iter233 != this->values.end(); ++_iter233) + { + xfer += oprot->writeString((*_iter233)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->tableName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32(this->createTime); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("lastAccessTime", ::apache::thrift::protocol::T_I32, 5); + xfer += oprot->writeI32(this->lastAccessTime); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("sd", ::apache::thrift::protocol::T_STRUCT, 6); + xfer += this->sd.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 7); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->parameters.size())); + std::map ::const_iterator _iter234; + for (_iter234 = this->parameters.begin(); _iter234 != this->parameters.end(); ++_iter234) + { + xfer += oprot->writeString(_iter234->first); + xfer += oprot->writeString(_iter234->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + if (this->__isset.privileges) { + xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 8); + xfer += this->privileges.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(Partition &a, Partition &b) { + using ::std::swap; + swap(a.values, b.values); + swap(a.dbName, b.dbName); + swap(a.tableName, b.tableName); + swap(a.createTime, b.createTime); + swap(a.lastAccessTime, b.lastAccessTime); + swap(a.sd, b.sd); + swap(a.parameters, b.parameters); + swap(a.privileges, b.privileges); + swap(a.__isset, b.__isset); +} + +Partition::Partition(const Partition& other235) { + values = other235.values; + dbName = other235.dbName; + tableName = other235.tableName; + createTime = other235.createTime; + lastAccessTime = other235.lastAccessTime; + sd = other235.sd; + parameters = other235.parameters; + privileges = other235.privileges; + __isset = other235.__isset; +} +Partition& Partition::operator=(const Partition& other236) { + values = other236.values; + dbName = other236.dbName; + tableName = other236.tableName; + createTime = other236.createTime; + lastAccessTime = other236.lastAccessTime; + sd = other236.sd; + parameters = other236.parameters; + privileges = other236.privileges; + __isset = other236.__isset; + return *this; +} +void Partition::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "Partition("; + out << "values=" << to_string(values); + out << ", " << "dbName=" << to_string(dbName); + out << ", " << "tableName=" << to_string(tableName); + out << ", " << "createTime=" << to_string(createTime); + out << ", " << "lastAccessTime=" << to_string(lastAccessTime); + out << ", " << "sd=" << to_string(sd); + out << ", " << "parameters=" << to_string(parameters); + out << ", " << "privileges="; (__isset.privileges ? (out << to_string(privileges)) : (out << "")); + out << ")"; +} + + +PartitionWithoutSD::~PartitionWithoutSD() noexcept { +} + + +void PartitionWithoutSD::__set_values(const std::vector & val) { + this->values = val; +} + +void PartitionWithoutSD::__set_createTime(const int32_t val) { + this->createTime = val; +} + +void PartitionWithoutSD::__set_lastAccessTime(const int32_t val) { + this->lastAccessTime = val; +} + +void PartitionWithoutSD::__set_relativePath(const std::string& val) { + this->relativePath = val; +} + +void PartitionWithoutSD::__set_parameters(const std::map & val) { + this->parameters = val; +} + +void PartitionWithoutSD::__set_privileges(const PrincipalPrivilegeSet& val) { + this->privileges = val; +__isset.privileges = true; +} +std::ostream& operator<<(std::ostream& out, const PartitionWithoutSD& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PartitionWithoutSD::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->values.clear(); + uint32_t _size237; + ::apache::thrift::protocol::TType _etype240; + xfer += iprot->readListBegin(_etype240, _size237); + this->values.resize(_size237); + uint32_t _i241; + for (_i241 = 0; _i241 < _size237; ++_i241) + { + xfer += iprot->readString(this->values[_i241]); + } + xfer += iprot->readListEnd(); + } + this->__isset.values = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->createTime); + this->__isset.createTime = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->lastAccessTime); + this->__isset.lastAccessTime = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->relativePath); + this->__isset.relativePath = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->parameters.clear(); + uint32_t _size242; + ::apache::thrift::protocol::TType _ktype243; + ::apache::thrift::protocol::TType _vtype244; + xfer += iprot->readMapBegin(_ktype243, _vtype244, _size242); + uint32_t _i246; + for (_i246 = 0; _i246 < _size242; ++_i246) + { + std::string _key247; + xfer += iprot->readString(_key247); + std::string& _val248 = this->parameters[_key247]; + xfer += iprot->readString(_val248); + } + xfer += iprot->readMapEnd(); + } + this->__isset.parameters = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->privileges.read(iprot); + this->__isset.privileges = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t PartitionWithoutSD::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PartitionWithoutSD"); + + xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->values.size())); + std::vector ::const_iterator _iter249; + for (_iter249 = this->values.begin(); _iter249 != this->values.end(); ++_iter249) + { + xfer += oprot->writeString((*_iter249)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32(this->createTime); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("lastAccessTime", ::apache::thrift::protocol::T_I32, 3); + xfer += oprot->writeI32(this->lastAccessTime); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("relativePath", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->relativePath); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 5); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->parameters.size())); + std::map ::const_iterator _iter250; + for (_iter250 = this->parameters.begin(); _iter250 != this->parameters.end(); ++_iter250) + { + xfer += oprot->writeString(_iter250->first); + xfer += oprot->writeString(_iter250->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + if (this->__isset.privileges) { + xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 6); + xfer += this->privileges.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PartitionWithoutSD &a, PartitionWithoutSD &b) { + using ::std::swap; + swap(a.values, b.values); + swap(a.createTime, b.createTime); + swap(a.lastAccessTime, b.lastAccessTime); + swap(a.relativePath, b.relativePath); + swap(a.parameters, b.parameters); + swap(a.privileges, b.privileges); + swap(a.__isset, b.__isset); +} + +PartitionWithoutSD::PartitionWithoutSD(const PartitionWithoutSD& other251) { + values = other251.values; + createTime = other251.createTime; + lastAccessTime = other251.lastAccessTime; + relativePath = other251.relativePath; + parameters = other251.parameters; + privileges = other251.privileges; + __isset = other251.__isset; +} +PartitionWithoutSD& PartitionWithoutSD::operator=(const PartitionWithoutSD& other252) { + values = other252.values; + createTime = other252.createTime; + lastAccessTime = other252.lastAccessTime; + relativePath = other252.relativePath; + parameters = other252.parameters; + privileges = other252.privileges; + __isset = other252.__isset; + return *this; +} +void PartitionWithoutSD::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PartitionWithoutSD("; + out << "values=" << to_string(values); + out << ", " << "createTime=" << to_string(createTime); + out << ", " << "lastAccessTime=" << to_string(lastAccessTime); + out << ", " << "relativePath=" << to_string(relativePath); + out << ", " << "parameters=" << to_string(parameters); + out << ", " << "privileges="; (__isset.privileges ? (out << to_string(privileges)) : (out << "")); + out << ")"; +} + + +PartitionSpecWithSharedSD::~PartitionSpecWithSharedSD() noexcept { +} + + +void PartitionSpecWithSharedSD::__set_partitions(const std::vector & val) { + this->partitions = val; +} + +void PartitionSpecWithSharedSD::__set_sd(const StorageDescriptor& val) { + this->sd = val; +} +std::ostream& operator<<(std::ostream& out, const PartitionSpecWithSharedSD& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PartitionSpecWithSharedSD::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->partitions.clear(); + uint32_t _size253; + ::apache::thrift::protocol::TType _etype256; + xfer += iprot->readListBegin(_etype256, _size253); + this->partitions.resize(_size253); + uint32_t _i257; + for (_i257 = 0; _i257 < _size253; ++_i257) + { + xfer += this->partitions[_i257].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.partitions = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->sd.read(iprot); + this->__isset.sd = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t PartitionSpecWithSharedSD::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PartitionSpecWithSharedSD"); + + xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitions.size())); + std::vector ::const_iterator _iter258; + for (_iter258 = this->partitions.begin(); _iter258 != this->partitions.end(); ++_iter258) + { + xfer += (*_iter258).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("sd", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->sd.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PartitionSpecWithSharedSD &a, PartitionSpecWithSharedSD &b) { + using ::std::swap; + swap(a.partitions, b.partitions); + swap(a.sd, b.sd); + swap(a.__isset, b.__isset); +} + +PartitionSpecWithSharedSD::PartitionSpecWithSharedSD(const PartitionSpecWithSharedSD& other259) { + partitions = other259.partitions; + sd = other259.sd; + __isset = other259.__isset; +} +PartitionSpecWithSharedSD& PartitionSpecWithSharedSD::operator=(const PartitionSpecWithSharedSD& other260) { + partitions = other260.partitions; + sd = other260.sd; + __isset = other260.__isset; + return *this; +} +void PartitionSpecWithSharedSD::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PartitionSpecWithSharedSD("; + out << "partitions=" << to_string(partitions); + out << ", " << "sd=" << to_string(sd); + out << ")"; +} + + +PartitionListComposingSpec::~PartitionListComposingSpec() noexcept { +} + + +void PartitionListComposingSpec::__set_partitions(const std::vector & val) { + this->partitions = val; +} +std::ostream& operator<<(std::ostream& out, const PartitionListComposingSpec& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PartitionListComposingSpec::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->partitions.clear(); + uint32_t _size261; + ::apache::thrift::protocol::TType _etype264; + xfer += iprot->readListBegin(_etype264, _size261); + this->partitions.resize(_size261); + uint32_t _i265; + for (_i265 = 0; _i265 < _size261; ++_i265) + { + xfer += this->partitions[_i265].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.partitions = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t PartitionListComposingSpec::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PartitionListComposingSpec"); + + xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitions.size())); + std::vector ::const_iterator _iter266; + for (_iter266 = this->partitions.begin(); _iter266 != this->partitions.end(); ++_iter266) + { + xfer += (*_iter266).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PartitionListComposingSpec &a, PartitionListComposingSpec &b) { + using ::std::swap; + swap(a.partitions, b.partitions); + swap(a.__isset, b.__isset); +} + +PartitionListComposingSpec::PartitionListComposingSpec(const PartitionListComposingSpec& other267) { + partitions = other267.partitions; + __isset = other267.__isset; +} +PartitionListComposingSpec& PartitionListComposingSpec::operator=(const PartitionListComposingSpec& other268) { + partitions = other268.partitions; + __isset = other268.__isset; + return *this; +} +void PartitionListComposingSpec::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PartitionListComposingSpec("; + out << "partitions=" << to_string(partitions); + out << ")"; +} + + +PartitionSpec::~PartitionSpec() noexcept { +} + + +void PartitionSpec::__set_dbName(const std::string& val) { + this->dbName = val; +} + +void PartitionSpec::__set_tableName(const std::string& val) { + this->tableName = val; +} + +void PartitionSpec::__set_rootPath(const std::string& val) { + this->rootPath = val; +} + +void PartitionSpec::__set_sharedSDPartitionSpec(const PartitionSpecWithSharedSD& val) { + this->sharedSDPartitionSpec = val; +__isset.sharedSDPartitionSpec = true; +} + +void PartitionSpec::__set_partitionList(const PartitionListComposingSpec& val) { + this->partitionList = val; +__isset.partitionList = true; +} +std::ostream& operator<<(std::ostream& out, const PartitionSpec& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PartitionSpec::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tableName); + this->__isset.tableName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->rootPath); + this->__isset.rootPath = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->sharedSDPartitionSpec.read(iprot); + this->__isset.sharedSDPartitionSpec = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->partitionList.read(iprot); + this->__isset.partitionList = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t PartitionSpec::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PartitionSpec"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tableName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("rootPath", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->rootPath); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.sharedSDPartitionSpec) { + xfer += oprot->writeFieldBegin("sharedSDPartitionSpec", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->sharedSDPartitionSpec.write(oprot); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.partitionList) { + xfer += oprot->writeFieldBegin("partitionList", ::apache::thrift::protocol::T_STRUCT, 5); + xfer += this->partitionList.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PartitionSpec &a, PartitionSpec &b) { + using ::std::swap; + swap(a.dbName, b.dbName); + swap(a.tableName, b.tableName); + swap(a.rootPath, b.rootPath); + swap(a.sharedSDPartitionSpec, b.sharedSDPartitionSpec); + swap(a.partitionList, b.partitionList); + swap(a.__isset, b.__isset); +} + +PartitionSpec::PartitionSpec(const PartitionSpec& other269) { + dbName = other269.dbName; + tableName = other269.tableName; + rootPath = other269.rootPath; + sharedSDPartitionSpec = other269.sharedSDPartitionSpec; + partitionList = other269.partitionList; + __isset = other269.__isset; +} +PartitionSpec& PartitionSpec::operator=(const PartitionSpec& other270) { + dbName = other270.dbName; + tableName = other270.tableName; + rootPath = other270.rootPath; + sharedSDPartitionSpec = other270.sharedSDPartitionSpec; + partitionList = other270.partitionList; + __isset = other270.__isset; + return *this; +} +void PartitionSpec::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PartitionSpec("; + out << "dbName=" << to_string(dbName); + out << ", " << "tableName=" << to_string(tableName); + out << ", " << "rootPath=" << to_string(rootPath); + out << ", " << "sharedSDPartitionSpec="; (__isset.sharedSDPartitionSpec ? (out << to_string(sharedSDPartitionSpec)) : (out << "")); + out << ", " << "partitionList="; (__isset.partitionList ? (out << to_string(partitionList)) : (out << "")); + out << ")"; +} + + +Index::~Index() noexcept { +} + + +void Index::__set_indexName(const std::string& val) { + this->indexName = val; +} + +void Index::__set_indexHandlerClass(const std::string& val) { + this->indexHandlerClass = val; +} + +void Index::__set_dbName(const std::string& val) { + this->dbName = val; +} + +void Index::__set_origTableName(const std::string& val) { + this->origTableName = val; +} + +void Index::__set_createTime(const int32_t val) { + this->createTime = val; +} + +void Index::__set_lastAccessTime(const int32_t val) { + this->lastAccessTime = val; +} + +void Index::__set_indexTableName(const std::string& val) { + this->indexTableName = val; +} + +void Index::__set_sd(const StorageDescriptor& val) { + this->sd = val; +} + +void Index::__set_parameters(const std::map & val) { + this->parameters = val; +} + +void Index::__set_deferredRebuild(const bool val) { + this->deferredRebuild = val; +} +std::ostream& operator<<(std::ostream& out, const Index& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t Index::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->indexName); + this->__isset.indexName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->indexHandlerClass); + this->__isset.indexHandlerClass = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->origTableName); + this->__isset.origTableName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->createTime); + this->__isset.createTime = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->lastAccessTime); + this->__isset.lastAccessTime = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->indexTableName); + this->__isset.indexTableName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 8: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->sd.read(iprot); + this->__isset.sd = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 9: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->parameters.clear(); + uint32_t _size271; + ::apache::thrift::protocol::TType _ktype272; + ::apache::thrift::protocol::TType _vtype273; + xfer += iprot->readMapBegin(_ktype272, _vtype273, _size271); + uint32_t _i275; + for (_i275 = 0; _i275 < _size271; ++_i275) + { + std::string _key276; + xfer += iprot->readString(_key276); + std::string& _val277 = this->parameters[_key276]; + xfer += iprot->readString(_val277); + } + xfer += iprot->readMapEnd(); + } + this->__isset.parameters = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 10: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->deferredRebuild); + this->__isset.deferredRebuild = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t Index::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("Index"); + + xfer += oprot->writeFieldBegin("indexName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->indexName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("indexHandlerClass", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->indexHandlerClass); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("origTableName", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->origTableName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 5); + xfer += oprot->writeI32(this->createTime); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("lastAccessTime", ::apache::thrift::protocol::T_I32, 6); + xfer += oprot->writeI32(this->lastAccessTime); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("indexTableName", ::apache::thrift::protocol::T_STRING, 7); + xfer += oprot->writeString(this->indexTableName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("sd", ::apache::thrift::protocol::T_STRUCT, 8); + xfer += this->sd.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 9); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->parameters.size())); + std::map ::const_iterator _iter278; + for (_iter278 = this->parameters.begin(); _iter278 != this->parameters.end(); ++_iter278) + { + xfer += oprot->writeString(_iter278->first); + xfer += oprot->writeString(_iter278->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("deferredRebuild", ::apache::thrift::protocol::T_BOOL, 10); + xfer += oprot->writeBool(this->deferredRebuild); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(Index &a, Index &b) { + using ::std::swap; + swap(a.indexName, b.indexName); + swap(a.indexHandlerClass, b.indexHandlerClass); + swap(a.dbName, b.dbName); + swap(a.origTableName, b.origTableName); + swap(a.createTime, b.createTime); + swap(a.lastAccessTime, b.lastAccessTime); + swap(a.indexTableName, b.indexTableName); + swap(a.sd, b.sd); + swap(a.parameters, b.parameters); + swap(a.deferredRebuild, b.deferredRebuild); + swap(a.__isset, b.__isset); +} + +Index::Index(const Index& other279) { + indexName = other279.indexName; + indexHandlerClass = other279.indexHandlerClass; + dbName = other279.dbName; + origTableName = other279.origTableName; + createTime = other279.createTime; + lastAccessTime = other279.lastAccessTime; + indexTableName = other279.indexTableName; + sd = other279.sd; + parameters = other279.parameters; + deferredRebuild = other279.deferredRebuild; + __isset = other279.__isset; +} +Index& Index::operator=(const Index& other280) { + indexName = other280.indexName; + indexHandlerClass = other280.indexHandlerClass; + dbName = other280.dbName; + origTableName = other280.origTableName; + createTime = other280.createTime; + lastAccessTime = other280.lastAccessTime; + indexTableName = other280.indexTableName; + sd = other280.sd; + parameters = other280.parameters; + deferredRebuild = other280.deferredRebuild; + __isset = other280.__isset; + return *this; +} +void Index::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "Index("; + out << "indexName=" << to_string(indexName); + out << ", " << "indexHandlerClass=" << to_string(indexHandlerClass); + out << ", " << "dbName=" << to_string(dbName); + out << ", " << "origTableName=" << to_string(origTableName); + out << ", " << "createTime=" << to_string(createTime); + out << ", " << "lastAccessTime=" << to_string(lastAccessTime); + out << ", " << "indexTableName=" << to_string(indexTableName); + out << ", " << "sd=" << to_string(sd); + out << ", " << "parameters=" << to_string(parameters); + out << ", " << "deferredRebuild=" << to_string(deferredRebuild); + out << ")"; +} + + +BooleanColumnStatsData::~BooleanColumnStatsData() noexcept { +} + + +void BooleanColumnStatsData::__set_numTrues(const int64_t val) { + this->numTrues = val; +} + +void BooleanColumnStatsData::__set_numFalses(const int64_t val) { + this->numFalses = val; +} + +void BooleanColumnStatsData::__set_numNulls(const int64_t val) { + this->numNulls = val; +} + +void BooleanColumnStatsData::__set_bitVectors(const std::string& val) { + this->bitVectors = val; +__isset.bitVectors = true; +} +std::ostream& operator<<(std::ostream& out, const BooleanColumnStatsData& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t BooleanColumnStatsData::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_numTrues = false; + bool isset_numFalses = false; + bool isset_numNulls = 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_I64) { + xfer += iprot->readI64(this->numTrues); + isset_numTrues = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->numFalses); + isset_numFalses = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->numNulls); + isset_numNulls = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readBinary(this->bitVectors); + this->__isset.bitVectors = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_numTrues) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_numFalses) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_numNulls) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t BooleanColumnStatsData::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("BooleanColumnStatsData"); + + xfer += oprot->writeFieldBegin("numTrues", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->numTrues); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("numFalses", ::apache::thrift::protocol::T_I64, 2); + xfer += oprot->writeI64(this->numFalses); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("numNulls", ::apache::thrift::protocol::T_I64, 3); + xfer += oprot->writeI64(this->numNulls); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.bitVectors) { + xfer += oprot->writeFieldBegin("bitVectors", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeBinary(this->bitVectors); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(BooleanColumnStatsData &a, BooleanColumnStatsData &b) { + using ::std::swap; + swap(a.numTrues, b.numTrues); + swap(a.numFalses, b.numFalses); + swap(a.numNulls, b.numNulls); + swap(a.bitVectors, b.bitVectors); + swap(a.__isset, b.__isset); +} + +BooleanColumnStatsData::BooleanColumnStatsData(const BooleanColumnStatsData& other281) { + numTrues = other281.numTrues; + numFalses = other281.numFalses; + numNulls = other281.numNulls; + bitVectors = other281.bitVectors; + __isset = other281.__isset; +} +BooleanColumnStatsData& BooleanColumnStatsData::operator=(const BooleanColumnStatsData& other282) { + numTrues = other282.numTrues; + numFalses = other282.numFalses; + numNulls = other282.numNulls; + bitVectors = other282.bitVectors; + __isset = other282.__isset; + return *this; +} +void BooleanColumnStatsData::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "BooleanColumnStatsData("; + out << "numTrues=" << to_string(numTrues); + out << ", " << "numFalses=" << to_string(numFalses); + out << ", " << "numNulls=" << to_string(numNulls); + out << ", " << "bitVectors="; (__isset.bitVectors ? (out << to_string(bitVectors)) : (out << "")); + out << ")"; +} + + +DoubleColumnStatsData::~DoubleColumnStatsData() noexcept { +} + + +void DoubleColumnStatsData::__set_lowValue(const double val) { + this->lowValue = val; +__isset.lowValue = true; +} + +void DoubleColumnStatsData::__set_highValue(const double val) { + this->highValue = val; +__isset.highValue = true; +} + +void DoubleColumnStatsData::__set_numNulls(const int64_t val) { + this->numNulls = val; +} + +void DoubleColumnStatsData::__set_numDVs(const int64_t val) { + this->numDVs = val; +} + +void DoubleColumnStatsData::__set_bitVectors(const std::string& val) { + this->bitVectors = val; +__isset.bitVectors = true; +} +std::ostream& operator<<(std::ostream& out, const DoubleColumnStatsData& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t DoubleColumnStatsData::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_numNulls = false; + bool isset_numDVs = 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_DOUBLE) { + xfer += iprot->readDouble(this->lowValue); + this->__isset.lowValue = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_DOUBLE) { + xfer += iprot->readDouble(this->highValue); + this->__isset.highValue = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->numNulls); + isset_numNulls = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->numDVs); + isset_numDVs = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readBinary(this->bitVectors); + this->__isset.bitVectors = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_numNulls) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_numDVs) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t DoubleColumnStatsData::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("DoubleColumnStatsData"); + + if (this->__isset.lowValue) { + xfer += oprot->writeFieldBegin("lowValue", ::apache::thrift::protocol::T_DOUBLE, 1); + xfer += oprot->writeDouble(this->lowValue); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.highValue) { + xfer += oprot->writeFieldBegin("highValue", ::apache::thrift::protocol::T_DOUBLE, 2); + xfer += oprot->writeDouble(this->highValue); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldBegin("numNulls", ::apache::thrift::protocol::T_I64, 3); + xfer += oprot->writeI64(this->numNulls); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("numDVs", ::apache::thrift::protocol::T_I64, 4); + xfer += oprot->writeI64(this->numDVs); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.bitVectors) { + xfer += oprot->writeFieldBegin("bitVectors", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeBinary(this->bitVectors); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(DoubleColumnStatsData &a, DoubleColumnStatsData &b) { + using ::std::swap; + swap(a.lowValue, b.lowValue); + swap(a.highValue, b.highValue); + swap(a.numNulls, b.numNulls); + swap(a.numDVs, b.numDVs); + swap(a.bitVectors, b.bitVectors); + swap(a.__isset, b.__isset); +} + +DoubleColumnStatsData::DoubleColumnStatsData(const DoubleColumnStatsData& other283) { + lowValue = other283.lowValue; + highValue = other283.highValue; + numNulls = other283.numNulls; + numDVs = other283.numDVs; + bitVectors = other283.bitVectors; + __isset = other283.__isset; +} +DoubleColumnStatsData& DoubleColumnStatsData::operator=(const DoubleColumnStatsData& other284) { + lowValue = other284.lowValue; + highValue = other284.highValue; + numNulls = other284.numNulls; + numDVs = other284.numDVs; + bitVectors = other284.bitVectors; + __isset = other284.__isset; + return *this; +} +void DoubleColumnStatsData::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "DoubleColumnStatsData("; + out << "lowValue="; (__isset.lowValue ? (out << to_string(lowValue)) : (out << "")); + out << ", " << "highValue="; (__isset.highValue ? (out << to_string(highValue)) : (out << "")); + out << ", " << "numNulls=" << to_string(numNulls); + out << ", " << "numDVs=" << to_string(numDVs); + out << ", " << "bitVectors="; (__isset.bitVectors ? (out << to_string(bitVectors)) : (out << "")); + out << ")"; +} + + +LongColumnStatsData::~LongColumnStatsData() noexcept { +} + + +void LongColumnStatsData::__set_lowValue(const int64_t val) { + this->lowValue = val; +__isset.lowValue = true; +} + +void LongColumnStatsData::__set_highValue(const int64_t val) { + this->highValue = val; +__isset.highValue = true; +} + +void LongColumnStatsData::__set_numNulls(const int64_t val) { + this->numNulls = val; +} + +void LongColumnStatsData::__set_numDVs(const int64_t val) { + this->numDVs = val; +} + +void LongColumnStatsData::__set_bitVectors(const std::string& val) { + this->bitVectors = val; +__isset.bitVectors = true; +} +std::ostream& operator<<(std::ostream& out, const LongColumnStatsData& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t LongColumnStatsData::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_numNulls = false; + bool isset_numDVs = 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_I64) { + xfer += iprot->readI64(this->lowValue); + this->__isset.lowValue = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->highValue); + this->__isset.highValue = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->numNulls); + isset_numNulls = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->numDVs); + isset_numDVs = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readBinary(this->bitVectors); + this->__isset.bitVectors = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_numNulls) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_numDVs) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t LongColumnStatsData::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("LongColumnStatsData"); + + if (this->__isset.lowValue) { + xfer += oprot->writeFieldBegin("lowValue", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->lowValue); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.highValue) { + xfer += oprot->writeFieldBegin("highValue", ::apache::thrift::protocol::T_I64, 2); + xfer += oprot->writeI64(this->highValue); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldBegin("numNulls", ::apache::thrift::protocol::T_I64, 3); + xfer += oprot->writeI64(this->numNulls); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("numDVs", ::apache::thrift::protocol::T_I64, 4); + xfer += oprot->writeI64(this->numDVs); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.bitVectors) { + xfer += oprot->writeFieldBegin("bitVectors", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeBinary(this->bitVectors); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(LongColumnStatsData &a, LongColumnStatsData &b) { + using ::std::swap; + swap(a.lowValue, b.lowValue); + swap(a.highValue, b.highValue); + swap(a.numNulls, b.numNulls); + swap(a.numDVs, b.numDVs); + swap(a.bitVectors, b.bitVectors); + swap(a.__isset, b.__isset); +} + +LongColumnStatsData::LongColumnStatsData(const LongColumnStatsData& other285) { + lowValue = other285.lowValue; + highValue = other285.highValue; + numNulls = other285.numNulls; + numDVs = other285.numDVs; + bitVectors = other285.bitVectors; + __isset = other285.__isset; +} +LongColumnStatsData& LongColumnStatsData::operator=(const LongColumnStatsData& other286) { + lowValue = other286.lowValue; + highValue = other286.highValue; + numNulls = other286.numNulls; + numDVs = other286.numDVs; + bitVectors = other286.bitVectors; + __isset = other286.__isset; + return *this; +} +void LongColumnStatsData::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "LongColumnStatsData("; + out << "lowValue="; (__isset.lowValue ? (out << to_string(lowValue)) : (out << "")); + out << ", " << "highValue="; (__isset.highValue ? (out << to_string(highValue)) : (out << "")); + out << ", " << "numNulls=" << to_string(numNulls); + out << ", " << "numDVs=" << to_string(numDVs); + out << ", " << "bitVectors="; (__isset.bitVectors ? (out << to_string(bitVectors)) : (out << "")); + out << ")"; +} + + +StringColumnStatsData::~StringColumnStatsData() noexcept { +} + + +void StringColumnStatsData::__set_maxColLen(const int64_t val) { + this->maxColLen = val; +} + +void StringColumnStatsData::__set_avgColLen(const double val) { + this->avgColLen = val; +} + +void StringColumnStatsData::__set_numNulls(const int64_t val) { + this->numNulls = val; +} + +void StringColumnStatsData::__set_numDVs(const int64_t val) { + this->numDVs = val; +} + +void StringColumnStatsData::__set_bitVectors(const std::string& val) { + this->bitVectors = val; +__isset.bitVectors = true; +} +std::ostream& operator<<(std::ostream& out, const StringColumnStatsData& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t StringColumnStatsData::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_maxColLen = false; + bool isset_avgColLen = false; + bool isset_numNulls = false; + bool isset_numDVs = 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_I64) { + xfer += iprot->readI64(this->maxColLen); + isset_maxColLen = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_DOUBLE) { + xfer += iprot->readDouble(this->avgColLen); + isset_avgColLen = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->numNulls); + isset_numNulls = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->numDVs); + isset_numDVs = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readBinary(this->bitVectors); + this->__isset.bitVectors = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_maxColLen) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_avgColLen) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_numNulls) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_numDVs) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t StringColumnStatsData::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("StringColumnStatsData"); + + xfer += oprot->writeFieldBegin("maxColLen", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->maxColLen); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("avgColLen", ::apache::thrift::protocol::T_DOUBLE, 2); + xfer += oprot->writeDouble(this->avgColLen); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("numNulls", ::apache::thrift::protocol::T_I64, 3); + xfer += oprot->writeI64(this->numNulls); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("numDVs", ::apache::thrift::protocol::T_I64, 4); + xfer += oprot->writeI64(this->numDVs); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.bitVectors) { + xfer += oprot->writeFieldBegin("bitVectors", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeBinary(this->bitVectors); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(StringColumnStatsData &a, StringColumnStatsData &b) { + using ::std::swap; + swap(a.maxColLen, b.maxColLen); + swap(a.avgColLen, b.avgColLen); + swap(a.numNulls, b.numNulls); + swap(a.numDVs, b.numDVs); + swap(a.bitVectors, b.bitVectors); + swap(a.__isset, b.__isset); +} + +StringColumnStatsData::StringColumnStatsData(const StringColumnStatsData& other287) { + maxColLen = other287.maxColLen; + avgColLen = other287.avgColLen; + numNulls = other287.numNulls; + numDVs = other287.numDVs; + bitVectors = other287.bitVectors; + __isset = other287.__isset; +} +StringColumnStatsData& StringColumnStatsData::operator=(const StringColumnStatsData& other288) { + maxColLen = other288.maxColLen; + avgColLen = other288.avgColLen; + numNulls = other288.numNulls; + numDVs = other288.numDVs; + bitVectors = other288.bitVectors; + __isset = other288.__isset; + return *this; +} +void StringColumnStatsData::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "StringColumnStatsData("; + out << "maxColLen=" << to_string(maxColLen); + out << ", " << "avgColLen=" << to_string(avgColLen); + out << ", " << "numNulls=" << to_string(numNulls); + out << ", " << "numDVs=" << to_string(numDVs); + out << ", " << "bitVectors="; (__isset.bitVectors ? (out << to_string(bitVectors)) : (out << "")); + out << ")"; +} + + +BinaryColumnStatsData::~BinaryColumnStatsData() noexcept { +} + + +void BinaryColumnStatsData::__set_maxColLen(const int64_t val) { + this->maxColLen = val; +} + +void BinaryColumnStatsData::__set_avgColLen(const double val) { + this->avgColLen = val; +} + +void BinaryColumnStatsData::__set_numNulls(const int64_t val) { + this->numNulls = val; +} + +void BinaryColumnStatsData::__set_bitVectors(const std::string& val) { + this->bitVectors = val; +__isset.bitVectors = true; +} +std::ostream& operator<<(std::ostream& out, const BinaryColumnStatsData& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t BinaryColumnStatsData::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_maxColLen = false; + bool isset_avgColLen = false; + bool isset_numNulls = 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_I64) { + xfer += iprot->readI64(this->maxColLen); + isset_maxColLen = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_DOUBLE) { + xfer += iprot->readDouble(this->avgColLen); + isset_avgColLen = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->numNulls); + isset_numNulls = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readBinary(this->bitVectors); + this->__isset.bitVectors = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_maxColLen) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_avgColLen) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_numNulls) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t BinaryColumnStatsData::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("BinaryColumnStatsData"); + + xfer += oprot->writeFieldBegin("maxColLen", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->maxColLen); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("avgColLen", ::apache::thrift::protocol::T_DOUBLE, 2); + xfer += oprot->writeDouble(this->avgColLen); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("numNulls", ::apache::thrift::protocol::T_I64, 3); + xfer += oprot->writeI64(this->numNulls); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.bitVectors) { + xfer += oprot->writeFieldBegin("bitVectors", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeBinary(this->bitVectors); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(BinaryColumnStatsData &a, BinaryColumnStatsData &b) { + using ::std::swap; + swap(a.maxColLen, b.maxColLen); + swap(a.avgColLen, b.avgColLen); + swap(a.numNulls, b.numNulls); + swap(a.bitVectors, b.bitVectors); + swap(a.__isset, b.__isset); +} + +BinaryColumnStatsData::BinaryColumnStatsData(const BinaryColumnStatsData& other289) { + maxColLen = other289.maxColLen; + avgColLen = other289.avgColLen; + numNulls = other289.numNulls; + bitVectors = other289.bitVectors; + __isset = other289.__isset; +} +BinaryColumnStatsData& BinaryColumnStatsData::operator=(const BinaryColumnStatsData& other290) { + maxColLen = other290.maxColLen; + avgColLen = other290.avgColLen; + numNulls = other290.numNulls; + bitVectors = other290.bitVectors; + __isset = other290.__isset; + return *this; +} +void BinaryColumnStatsData::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "BinaryColumnStatsData("; + out << "maxColLen=" << to_string(maxColLen); + out << ", " << "avgColLen=" << to_string(avgColLen); + out << ", " << "numNulls=" << to_string(numNulls); + out << ", " << "bitVectors="; (__isset.bitVectors ? (out << to_string(bitVectors)) : (out << "")); + out << ")"; +} + + +Decimal::~Decimal() noexcept { +} + + +void Decimal::__set_unscaled(const std::string& val) { + this->unscaled = val; +} + +void Decimal::__set_scale(const int16_t val) { + this->scale = val; +} +std::ostream& operator<<(std::ostream& out, const Decimal& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t Decimal::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_unscaled = false; + bool isset_scale = 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_STRING) { + xfer += iprot->readBinary(this->unscaled); + isset_unscaled = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I16) { + xfer += iprot->readI16(this->scale); + isset_scale = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_unscaled) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_scale) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t Decimal::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("Decimal"); + + xfer += oprot->writeFieldBegin("unscaled", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeBinary(this->unscaled); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("scale", ::apache::thrift::protocol::T_I16, 3); + xfer += oprot->writeI16(this->scale); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(Decimal &a, Decimal &b) { + using ::std::swap; + swap(a.unscaled, b.unscaled); + swap(a.scale, b.scale); +} + +Decimal::Decimal(const Decimal& other291) { + unscaled = other291.unscaled; + scale = other291.scale; +} +Decimal& Decimal::operator=(const Decimal& other292) { + unscaled = other292.unscaled; + scale = other292.scale; + return *this; +} +void Decimal::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "Decimal("; + out << "unscaled=" << to_string(unscaled); + out << ", " << "scale=" << to_string(scale); + out << ")"; +} + + +DecimalColumnStatsData::~DecimalColumnStatsData() noexcept { +} + + +void DecimalColumnStatsData::__set_lowValue(const Decimal& val) { + this->lowValue = val; +__isset.lowValue = true; +} + +void DecimalColumnStatsData::__set_highValue(const Decimal& val) { + this->highValue = val; +__isset.highValue = true; +} + +void DecimalColumnStatsData::__set_numNulls(const int64_t val) { + this->numNulls = val; +} + +void DecimalColumnStatsData::__set_numDVs(const int64_t val) { + this->numDVs = val; +} + +void DecimalColumnStatsData::__set_bitVectors(const std::string& val) { + this->bitVectors = val; +__isset.bitVectors = true; +} +std::ostream& operator<<(std::ostream& out, const DecimalColumnStatsData& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t DecimalColumnStatsData::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_numNulls = false; + bool isset_numDVs = 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_STRUCT) { + xfer += this->lowValue.read(iprot); + this->__isset.lowValue = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->highValue.read(iprot); + this->__isset.highValue = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->numNulls); + isset_numNulls = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->numDVs); + isset_numDVs = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readBinary(this->bitVectors); + this->__isset.bitVectors = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_numNulls) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_numDVs) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t DecimalColumnStatsData::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("DecimalColumnStatsData"); + + if (this->__isset.lowValue) { + xfer += oprot->writeFieldBegin("lowValue", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->lowValue.write(oprot); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.highValue) { + xfer += oprot->writeFieldBegin("highValue", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->highValue.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldBegin("numNulls", ::apache::thrift::protocol::T_I64, 3); + xfer += oprot->writeI64(this->numNulls); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("numDVs", ::apache::thrift::protocol::T_I64, 4); + xfer += oprot->writeI64(this->numDVs); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.bitVectors) { + xfer += oprot->writeFieldBegin("bitVectors", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeBinary(this->bitVectors); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(DecimalColumnStatsData &a, DecimalColumnStatsData &b) { + using ::std::swap; + swap(a.lowValue, b.lowValue); + swap(a.highValue, b.highValue); + swap(a.numNulls, b.numNulls); + swap(a.numDVs, b.numDVs); + swap(a.bitVectors, b.bitVectors); + swap(a.__isset, b.__isset); +} + +DecimalColumnStatsData::DecimalColumnStatsData(const DecimalColumnStatsData& other293) { + lowValue = other293.lowValue; + highValue = other293.highValue; + numNulls = other293.numNulls; + numDVs = other293.numDVs; + bitVectors = other293.bitVectors; + __isset = other293.__isset; +} +DecimalColumnStatsData& DecimalColumnStatsData::operator=(const DecimalColumnStatsData& other294) { + lowValue = other294.lowValue; + highValue = other294.highValue; + numNulls = other294.numNulls; + numDVs = other294.numDVs; + bitVectors = other294.bitVectors; + __isset = other294.__isset; + return *this; +} +void DecimalColumnStatsData::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "DecimalColumnStatsData("; + out << "lowValue="; (__isset.lowValue ? (out << to_string(lowValue)) : (out << "")); + out << ", " << "highValue="; (__isset.highValue ? (out << to_string(highValue)) : (out << "")); + out << ", " << "numNulls=" << to_string(numNulls); + out << ", " << "numDVs=" << to_string(numDVs); + out << ", " << "bitVectors="; (__isset.bitVectors ? (out << to_string(bitVectors)) : (out << "")); + out << ")"; +} + + +Date::~Date() noexcept { +} + + +void Date::__set_daysSinceEpoch(const int64_t val) { + this->daysSinceEpoch = val; +} +std::ostream& operator<<(std::ostream& out, const Date& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t Date::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_daysSinceEpoch = 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_I64) { + xfer += iprot->readI64(this->daysSinceEpoch); + isset_daysSinceEpoch = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_daysSinceEpoch) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t Date::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("Date"); + + xfer += oprot->writeFieldBegin("daysSinceEpoch", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->daysSinceEpoch); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(Date &a, Date &b) { + using ::std::swap; + swap(a.daysSinceEpoch, b.daysSinceEpoch); +} + +Date::Date(const Date& other295) { + daysSinceEpoch = other295.daysSinceEpoch; +} +Date& Date::operator=(const Date& other296) { + daysSinceEpoch = other296.daysSinceEpoch; + return *this; +} +void Date::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "Date("; + out << "daysSinceEpoch=" << to_string(daysSinceEpoch); + out << ")"; +} + + +DateColumnStatsData::~DateColumnStatsData() noexcept { +} + + +void DateColumnStatsData::__set_lowValue(const Date& val) { + this->lowValue = val; +__isset.lowValue = true; +} + +void DateColumnStatsData::__set_highValue(const Date& val) { + this->highValue = val; +__isset.highValue = true; +} + +void DateColumnStatsData::__set_numNulls(const int64_t val) { + this->numNulls = val; +} + +void DateColumnStatsData::__set_numDVs(const int64_t val) { + this->numDVs = val; +} + +void DateColumnStatsData::__set_bitVectors(const std::string& val) { + this->bitVectors = val; +__isset.bitVectors = true; +} +std::ostream& operator<<(std::ostream& out, const DateColumnStatsData& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t DateColumnStatsData::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_numNulls = false; + bool isset_numDVs = 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_STRUCT) { + xfer += this->lowValue.read(iprot); + this->__isset.lowValue = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->highValue.read(iprot); + this->__isset.highValue = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->numNulls); + isset_numNulls = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->numDVs); + isset_numDVs = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readBinary(this->bitVectors); + this->__isset.bitVectors = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_numNulls) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_numDVs) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t DateColumnStatsData::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("DateColumnStatsData"); + + if (this->__isset.lowValue) { + xfer += oprot->writeFieldBegin("lowValue", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->lowValue.write(oprot); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.highValue) { + xfer += oprot->writeFieldBegin("highValue", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->highValue.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldBegin("numNulls", ::apache::thrift::protocol::T_I64, 3); + xfer += oprot->writeI64(this->numNulls); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("numDVs", ::apache::thrift::protocol::T_I64, 4); + xfer += oprot->writeI64(this->numDVs); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.bitVectors) { + xfer += oprot->writeFieldBegin("bitVectors", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeBinary(this->bitVectors); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(DateColumnStatsData &a, DateColumnStatsData &b) { + using ::std::swap; + swap(a.lowValue, b.lowValue); + swap(a.highValue, b.highValue); + swap(a.numNulls, b.numNulls); + swap(a.numDVs, b.numDVs); + swap(a.bitVectors, b.bitVectors); + swap(a.__isset, b.__isset); +} + +DateColumnStatsData::DateColumnStatsData(const DateColumnStatsData& other297) { + lowValue = other297.lowValue; + highValue = other297.highValue; + numNulls = other297.numNulls; + numDVs = other297.numDVs; + bitVectors = other297.bitVectors; + __isset = other297.__isset; +} +DateColumnStatsData& DateColumnStatsData::operator=(const DateColumnStatsData& other298) { + lowValue = other298.lowValue; + highValue = other298.highValue; + numNulls = other298.numNulls; + numDVs = other298.numDVs; + bitVectors = other298.bitVectors; + __isset = other298.__isset; + return *this; +} +void DateColumnStatsData::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "DateColumnStatsData("; + out << "lowValue="; (__isset.lowValue ? (out << to_string(lowValue)) : (out << "")); + out << ", " << "highValue="; (__isset.highValue ? (out << to_string(highValue)) : (out << "")); + out << ", " << "numNulls=" << to_string(numNulls); + out << ", " << "numDVs=" << to_string(numDVs); + out << ", " << "bitVectors="; (__isset.bitVectors ? (out << to_string(bitVectors)) : (out << "")); + out << ")"; +} + + +ColumnStatisticsData::~ColumnStatisticsData() noexcept { +} + + +void ColumnStatisticsData::__set_booleanStats(const BooleanColumnStatsData& val) { + this->booleanStats = val; +__isset.booleanStats = true; +} + +void ColumnStatisticsData::__set_longStats(const LongColumnStatsData& val) { + this->longStats = val; +__isset.longStats = true; +} + +void ColumnStatisticsData::__set_doubleStats(const DoubleColumnStatsData& val) { + this->doubleStats = val; +__isset.doubleStats = true; +} + +void ColumnStatisticsData::__set_stringStats(const StringColumnStatsData& val) { + this->stringStats = val; +__isset.stringStats = true; +} + +void ColumnStatisticsData::__set_binaryStats(const BinaryColumnStatsData& val) { + this->binaryStats = val; +__isset.binaryStats = true; +} + +void ColumnStatisticsData::__set_decimalStats(const DecimalColumnStatsData& val) { + this->decimalStats = val; +__isset.decimalStats = true; +} + +void ColumnStatisticsData::__set_dateStats(const DateColumnStatsData& val) { + this->dateStats = val; +__isset.dateStats = true; +} +std::ostream& operator<<(std::ostream& out, const ColumnStatisticsData& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t ColumnStatisticsData::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->booleanStats.read(iprot); + this->__isset.booleanStats = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->longStats.read(iprot); + this->__isset.longStats = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->doubleStats.read(iprot); + this->__isset.doubleStats = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->stringStats.read(iprot); + this->__isset.stringStats = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->binaryStats.read(iprot); + this->__isset.binaryStats = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->decimalStats.read(iprot); + this->__isset.decimalStats = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->dateStats.read(iprot); + this->__isset.dateStats = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ColumnStatisticsData::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ColumnStatisticsData"); + + if (this->__isset.booleanStats) { + xfer += oprot->writeFieldBegin("booleanStats", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->booleanStats.write(oprot); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.longStats) { + xfer += oprot->writeFieldBegin("longStats", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->longStats.write(oprot); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.doubleStats) { + xfer += oprot->writeFieldBegin("doubleStats", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->doubleStats.write(oprot); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.stringStats) { + xfer += oprot->writeFieldBegin("stringStats", ::apache::thrift::protocol::T_STRUCT, 4); + xfer += this->stringStats.write(oprot); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.binaryStats) { + xfer += oprot->writeFieldBegin("binaryStats", ::apache::thrift::protocol::T_STRUCT, 5); + xfer += this->binaryStats.write(oprot); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.decimalStats) { + xfer += oprot->writeFieldBegin("decimalStats", ::apache::thrift::protocol::T_STRUCT, 6); + xfer += this->decimalStats.write(oprot); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.dateStats) { + xfer += oprot->writeFieldBegin("dateStats", ::apache::thrift::protocol::T_STRUCT, 7); + xfer += this->dateStats.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ColumnStatisticsData &a, ColumnStatisticsData &b) { + using ::std::swap; + swap(a.booleanStats, b.booleanStats); + swap(a.longStats, b.longStats); + swap(a.doubleStats, b.doubleStats); + swap(a.stringStats, b.stringStats); + swap(a.binaryStats, b.binaryStats); + swap(a.decimalStats, b.decimalStats); + swap(a.dateStats, b.dateStats); + swap(a.__isset, b.__isset); +} + +ColumnStatisticsData::ColumnStatisticsData(const ColumnStatisticsData& other299) { + booleanStats = other299.booleanStats; + longStats = other299.longStats; + doubleStats = other299.doubleStats; + stringStats = other299.stringStats; + binaryStats = other299.binaryStats; + decimalStats = other299.decimalStats; + dateStats = other299.dateStats; + __isset = other299.__isset; +} +ColumnStatisticsData& ColumnStatisticsData::operator=(const ColumnStatisticsData& other300) { + booleanStats = other300.booleanStats; + longStats = other300.longStats; + doubleStats = other300.doubleStats; + stringStats = other300.stringStats; + binaryStats = other300.binaryStats; + decimalStats = other300.decimalStats; + dateStats = other300.dateStats; + __isset = other300.__isset; + return *this; +} +void ColumnStatisticsData::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "ColumnStatisticsData("; + out << "booleanStats="; (__isset.booleanStats ? (out << to_string(booleanStats)) : (out << "")); + out << ", " << "longStats="; (__isset.longStats ? (out << to_string(longStats)) : (out << "")); + out << ", " << "doubleStats="; (__isset.doubleStats ? (out << to_string(doubleStats)) : (out << "")); + out << ", " << "stringStats="; (__isset.stringStats ? (out << to_string(stringStats)) : (out << "")); + out << ", " << "binaryStats="; (__isset.binaryStats ? (out << to_string(binaryStats)) : (out << "")); + out << ", " << "decimalStats="; (__isset.decimalStats ? (out << to_string(decimalStats)) : (out << "")); + out << ", " << "dateStats="; (__isset.dateStats ? (out << to_string(dateStats)) : (out << "")); + out << ")"; +} + + +ColumnStatisticsObj::~ColumnStatisticsObj() noexcept { +} + + +void ColumnStatisticsObj::__set_colName(const std::string& val) { + this->colName = val; +} + +void ColumnStatisticsObj::__set_colType(const std::string& val) { + this->colType = val; +} + +void ColumnStatisticsObj::__set_statsData(const ColumnStatisticsData& val) { + this->statsData = val; +} +std::ostream& operator<<(std::ostream& out, const ColumnStatisticsObj& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t ColumnStatisticsObj::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_colName = false; + bool isset_colType = false; + bool isset_statsData = 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_STRING) { + xfer += iprot->readString(this->colName); + isset_colName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->colType); + isset_colType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->statsData.read(iprot); + isset_statsData = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_colName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_colType) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_statsData) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t ColumnStatisticsObj::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ColumnStatisticsObj"); + + xfer += oprot->writeFieldBegin("colName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->colName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("colType", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->colType); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("statsData", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->statsData.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ColumnStatisticsObj &a, ColumnStatisticsObj &b) { + using ::std::swap; + swap(a.colName, b.colName); + swap(a.colType, b.colType); + swap(a.statsData, b.statsData); +} + +ColumnStatisticsObj::ColumnStatisticsObj(const ColumnStatisticsObj& other301) { + colName = other301.colName; + colType = other301.colType; + statsData = other301.statsData; +} +ColumnStatisticsObj& ColumnStatisticsObj::operator=(const ColumnStatisticsObj& other302) { + colName = other302.colName; + colType = other302.colType; + statsData = other302.statsData; + return *this; +} +void ColumnStatisticsObj::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "ColumnStatisticsObj("; + out << "colName=" << to_string(colName); + out << ", " << "colType=" << to_string(colType); + out << ", " << "statsData=" << to_string(statsData); + out << ")"; +} + + +ColumnStatisticsDesc::~ColumnStatisticsDesc() noexcept { +} + + +void ColumnStatisticsDesc::__set_isTblLevel(const bool val) { + this->isTblLevel = val; +} + +void ColumnStatisticsDesc::__set_dbName(const std::string& val) { + this->dbName = val; +} + +void ColumnStatisticsDesc::__set_tableName(const std::string& val) { + this->tableName = val; +} + +void ColumnStatisticsDesc::__set_partName(const std::string& val) { + this->partName = val; +__isset.partName = true; +} + +void ColumnStatisticsDesc::__set_lastAnalyzed(const int64_t val) { + this->lastAnalyzed = val; +__isset.lastAnalyzed = true; +} +std::ostream& operator<<(std::ostream& out, const ColumnStatisticsDesc& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t ColumnStatisticsDesc::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_isTblLevel = false; + bool isset_dbName = false; + bool isset_tableName = 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_BOOL) { + xfer += iprot->readBool(this->isTblLevel); + isset_isTblLevel = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbName); + isset_dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tableName); + isset_tableName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->partName); + this->__isset.partName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->lastAnalyzed); + this->__isset.lastAnalyzed = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_isTblLevel) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_dbName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tableName) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t ColumnStatisticsDesc::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ColumnStatisticsDesc"); + + xfer += oprot->writeFieldBegin("isTblLevel", ::apache::thrift::protocol::T_BOOL, 1); + xfer += oprot->writeBool(this->isTblLevel); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->tableName); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.partName) { + xfer += oprot->writeFieldBegin("partName", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->partName); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.lastAnalyzed) { + xfer += oprot->writeFieldBegin("lastAnalyzed", ::apache::thrift::protocol::T_I64, 5); + xfer += oprot->writeI64(this->lastAnalyzed); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ColumnStatisticsDesc &a, ColumnStatisticsDesc &b) { + using ::std::swap; + swap(a.isTblLevel, b.isTblLevel); + swap(a.dbName, b.dbName); + swap(a.tableName, b.tableName); + swap(a.partName, b.partName); + swap(a.lastAnalyzed, b.lastAnalyzed); + swap(a.__isset, b.__isset); +} + +ColumnStatisticsDesc::ColumnStatisticsDesc(const ColumnStatisticsDesc& other303) { + isTblLevel = other303.isTblLevel; + dbName = other303.dbName; + tableName = other303.tableName; + partName = other303.partName; + lastAnalyzed = other303.lastAnalyzed; + __isset = other303.__isset; +} +ColumnStatisticsDesc& ColumnStatisticsDesc::operator=(const ColumnStatisticsDesc& other304) { + isTblLevel = other304.isTblLevel; + dbName = other304.dbName; + tableName = other304.tableName; + partName = other304.partName; + lastAnalyzed = other304.lastAnalyzed; + __isset = other304.__isset; + return *this; +} +void ColumnStatisticsDesc::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "ColumnStatisticsDesc("; + out << "isTblLevel=" << to_string(isTblLevel); + out << ", " << "dbName=" << to_string(dbName); + out << ", " << "tableName=" << to_string(tableName); + out << ", " << "partName="; (__isset.partName ? (out << to_string(partName)) : (out << "")); + out << ", " << "lastAnalyzed="; (__isset.lastAnalyzed ? (out << to_string(lastAnalyzed)) : (out << "")); + out << ")"; +} + + +ColumnStatistics::~ColumnStatistics() noexcept { +} + + +void ColumnStatistics::__set_statsDesc(const ColumnStatisticsDesc& val) { + this->statsDesc = val; +} + +void ColumnStatistics::__set_statsObj(const std::vector & val) { + this->statsObj = val; +} +std::ostream& operator<<(std::ostream& out, const ColumnStatistics& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t ColumnStatistics::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_statsDesc = false; + bool isset_statsObj = 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_STRUCT) { + xfer += this->statsDesc.read(iprot); + isset_statsDesc = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->statsObj.clear(); + uint32_t _size305; + ::apache::thrift::protocol::TType _etype308; + xfer += iprot->readListBegin(_etype308, _size305); + this->statsObj.resize(_size305); + uint32_t _i309; + for (_i309 = 0; _i309 < _size305; ++_i309) + { + xfer += this->statsObj[_i309].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_statsObj = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_statsDesc) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_statsObj) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t ColumnStatistics::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ColumnStatistics"); + + xfer += oprot->writeFieldBegin("statsDesc", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->statsDesc.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("statsObj", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->statsObj.size())); + std::vector ::const_iterator _iter310; + for (_iter310 = this->statsObj.begin(); _iter310 != this->statsObj.end(); ++_iter310) + { + xfer += (*_iter310).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ColumnStatistics &a, ColumnStatistics &b) { + using ::std::swap; + swap(a.statsDesc, b.statsDesc); + swap(a.statsObj, b.statsObj); +} + +ColumnStatistics::ColumnStatistics(const ColumnStatistics& other311) { + statsDesc = other311.statsDesc; + statsObj = other311.statsObj; +} +ColumnStatistics& ColumnStatistics::operator=(const ColumnStatistics& other312) { + statsDesc = other312.statsDesc; + statsObj = other312.statsObj; + return *this; +} +void ColumnStatistics::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "ColumnStatistics("; + out << "statsDesc=" << to_string(statsDesc); + out << ", " << "statsObj=" << to_string(statsObj); + out << ")"; +} + + +AggrStats::~AggrStats() noexcept { +} + + +void AggrStats::__set_colStats(const std::vector & val) { + this->colStats = val; +} + +void AggrStats::__set_partsFound(const int64_t val) { + this->partsFound = val; +} +std::ostream& operator<<(std::ostream& out, const AggrStats& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t AggrStats::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_colStats = false; + bool isset_partsFound = 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_LIST) { + { + this->colStats.clear(); + uint32_t _size313; + ::apache::thrift::protocol::TType _etype316; + xfer += iprot->readListBegin(_etype316, _size313); + this->colStats.resize(_size313); + uint32_t _i317; + for (_i317 = 0; _i317 < _size313; ++_i317) + { + xfer += this->colStats[_i317].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_colStats = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->partsFound); + isset_partsFound = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_colStats) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_partsFound) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t AggrStats::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("AggrStats"); + + xfer += oprot->writeFieldBegin("colStats", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->colStats.size())); + std::vector ::const_iterator _iter318; + for (_iter318 = this->colStats.begin(); _iter318 != this->colStats.end(); ++_iter318) + { + xfer += (*_iter318).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("partsFound", ::apache::thrift::protocol::T_I64, 2); + xfer += oprot->writeI64(this->partsFound); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(AggrStats &a, AggrStats &b) { + using ::std::swap; + swap(a.colStats, b.colStats); + swap(a.partsFound, b.partsFound); +} + +AggrStats::AggrStats(const AggrStats& other319) { + colStats = other319.colStats; + partsFound = other319.partsFound; +} +AggrStats& AggrStats::operator=(const AggrStats& other320) { + colStats = other320.colStats; + partsFound = other320.partsFound; + return *this; +} +void AggrStats::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "AggrStats("; + out << "colStats=" << to_string(colStats); + out << ", " << "partsFound=" << to_string(partsFound); + out << ")"; +} + + +SetPartitionsStatsRequest::~SetPartitionsStatsRequest() noexcept { +} + + +void SetPartitionsStatsRequest::__set_colStats(const std::vector & val) { + this->colStats = val; +} + +void SetPartitionsStatsRequest::__set_needMerge(const bool val) { + this->needMerge = val; +__isset.needMerge = true; +} +std::ostream& operator<<(std::ostream& out, const SetPartitionsStatsRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t SetPartitionsStatsRequest::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_colStats = 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_LIST) { + { + this->colStats.clear(); + uint32_t _size321; + ::apache::thrift::protocol::TType _etype324; + xfer += iprot->readListBegin(_etype324, _size321); + this->colStats.resize(_size321); + uint32_t _i325; + for (_i325 = 0; _i325 < _size321; ++_i325) + { + xfer += this->colStats[_i325].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_colStats = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->needMerge); + this->__isset.needMerge = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_colStats) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t SetPartitionsStatsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("SetPartitionsStatsRequest"); + + xfer += oprot->writeFieldBegin("colStats", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->colStats.size())); + std::vector ::const_iterator _iter326; + for (_iter326 = this->colStats.begin(); _iter326 != this->colStats.end(); ++_iter326) + { + xfer += (*_iter326).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + if (this->__isset.needMerge) { + xfer += oprot->writeFieldBegin("needMerge", ::apache::thrift::protocol::T_BOOL, 2); + xfer += oprot->writeBool(this->needMerge); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(SetPartitionsStatsRequest &a, SetPartitionsStatsRequest &b) { + using ::std::swap; + swap(a.colStats, b.colStats); + swap(a.needMerge, b.needMerge); + swap(a.__isset, b.__isset); +} + +SetPartitionsStatsRequest::SetPartitionsStatsRequest(const SetPartitionsStatsRequest& other327) { + colStats = other327.colStats; + needMerge = other327.needMerge; + __isset = other327.__isset; +} +SetPartitionsStatsRequest& SetPartitionsStatsRequest::operator=(const SetPartitionsStatsRequest& other328) { + colStats = other328.colStats; + needMerge = other328.needMerge; + __isset = other328.__isset; + return *this; +} +void SetPartitionsStatsRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "SetPartitionsStatsRequest("; + out << "colStats=" << to_string(colStats); + out << ", " << "needMerge="; (__isset.needMerge ? (out << to_string(needMerge)) : (out << "")); + out << ")"; +} + + +Schema::~Schema() noexcept { +} + + +void Schema::__set_fieldSchemas(const std::vector & val) { + this->fieldSchemas = val; +} + +void Schema::__set_properties(const std::map & val) { + this->properties = val; +} +std::ostream& operator<<(std::ostream& out, const Schema& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t Schema::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->fieldSchemas.clear(); + uint32_t _size329; + ::apache::thrift::protocol::TType _etype332; + xfer += iprot->readListBegin(_etype332, _size329); + this->fieldSchemas.resize(_size329); + uint32_t _i333; + for (_i333 = 0; _i333 < _size329; ++_i333) + { + xfer += this->fieldSchemas[_i333].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.fieldSchemas = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->properties.clear(); + uint32_t _size334; + ::apache::thrift::protocol::TType _ktype335; + ::apache::thrift::protocol::TType _vtype336; + xfer += iprot->readMapBegin(_ktype335, _vtype336, _size334); + uint32_t _i338; + for (_i338 = 0; _i338 < _size334; ++_i338) + { + std::string _key339; + xfer += iprot->readString(_key339); + std::string& _val340 = this->properties[_key339]; + xfer += iprot->readString(_val340); + } + xfer += iprot->readMapEnd(); + } + this->__isset.properties = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t Schema::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("Schema"); + + xfer += oprot->writeFieldBegin("fieldSchemas", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->fieldSchemas.size())); + std::vector ::const_iterator _iter341; + for (_iter341 = this->fieldSchemas.begin(); _iter341 != this->fieldSchemas.end(); ++_iter341) + { + xfer += (*_iter341).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 2); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->properties.size())); + std::map ::const_iterator _iter342; + for (_iter342 = this->properties.begin(); _iter342 != this->properties.end(); ++_iter342) + { + xfer += oprot->writeString(_iter342->first); + xfer += oprot->writeString(_iter342->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(Schema &a, Schema &b) { + using ::std::swap; + swap(a.fieldSchemas, b.fieldSchemas); + swap(a.properties, b.properties); + swap(a.__isset, b.__isset); +} + +Schema::Schema(const Schema& other343) { + fieldSchemas = other343.fieldSchemas; + properties = other343.properties; + __isset = other343.__isset; +} +Schema& Schema::operator=(const Schema& other344) { + fieldSchemas = other344.fieldSchemas; + properties = other344.properties; + __isset = other344.__isset; + return *this; +} +void Schema::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "Schema("; + out << "fieldSchemas=" << to_string(fieldSchemas); + out << ", " << "properties=" << to_string(properties); + out << ")"; +} + + +EnvironmentContext::~EnvironmentContext() noexcept { +} + + +void EnvironmentContext::__set_properties(const std::map & val) { + this->properties = val; +} +std::ostream& operator<<(std::ostream& out, const EnvironmentContext& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t EnvironmentContext::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->properties.clear(); + uint32_t _size345; + ::apache::thrift::protocol::TType _ktype346; + ::apache::thrift::protocol::TType _vtype347; + xfer += iprot->readMapBegin(_ktype346, _vtype347, _size345); + uint32_t _i349; + for (_i349 = 0; _i349 < _size345; ++_i349) + { + std::string _key350; + xfer += iprot->readString(_key350); + std::string& _val351 = this->properties[_key350]; + xfer += iprot->readString(_val351); + } + xfer += iprot->readMapEnd(); + } + this->__isset.properties = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t EnvironmentContext::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("EnvironmentContext"); + + xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 1); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->properties.size())); + std::map ::const_iterator _iter352; + for (_iter352 = this->properties.begin(); _iter352 != this->properties.end(); ++_iter352) + { + xfer += oprot->writeString(_iter352->first); + xfer += oprot->writeString(_iter352->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(EnvironmentContext &a, EnvironmentContext &b) { + using ::std::swap; + swap(a.properties, b.properties); + swap(a.__isset, b.__isset); +} + +EnvironmentContext::EnvironmentContext(const EnvironmentContext& other353) { + properties = other353.properties; + __isset = other353.__isset; +} +EnvironmentContext& EnvironmentContext::operator=(const EnvironmentContext& other354) { + properties = other354.properties; + __isset = other354.__isset; + return *this; +} +void EnvironmentContext::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "EnvironmentContext("; + out << "properties=" << to_string(properties); + out << ")"; +} + + +PrimaryKeysRequest::~PrimaryKeysRequest() noexcept { +} + + +void PrimaryKeysRequest::__set_db_name(const std::string& val) { + this->db_name = val; +} + +void PrimaryKeysRequest::__set_tbl_name(const std::string& val) { + this->tbl_name = val; +} +std::ostream& operator<<(std::ostream& out, const PrimaryKeysRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PrimaryKeysRequest::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_db_name = false; + bool isset_tbl_name = 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_STRING) { + xfer += iprot->readString(this->db_name); + isset_db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + isset_tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_db_name) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tbl_name) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t PrimaryKeysRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PrimaryKeysRequest"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PrimaryKeysRequest &a, PrimaryKeysRequest &b) { + using ::std::swap; + swap(a.db_name, b.db_name); + swap(a.tbl_name, b.tbl_name); +} + +PrimaryKeysRequest::PrimaryKeysRequest(const PrimaryKeysRequest& other355) { + db_name = other355.db_name; + tbl_name = other355.tbl_name; +} +PrimaryKeysRequest& PrimaryKeysRequest::operator=(const PrimaryKeysRequest& other356) { + db_name = other356.db_name; + tbl_name = other356.tbl_name; + return *this; +} +void PrimaryKeysRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PrimaryKeysRequest("; + out << "db_name=" << to_string(db_name); + out << ", " << "tbl_name=" << to_string(tbl_name); + out << ")"; +} + + +PrimaryKeysResponse::~PrimaryKeysResponse() noexcept { +} + + +void PrimaryKeysResponse::__set_primaryKeys(const std::vector & val) { + this->primaryKeys = val; +} +std::ostream& operator<<(std::ostream& out, const PrimaryKeysResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PrimaryKeysResponse::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_primaryKeys = 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_LIST) { + { + this->primaryKeys.clear(); + uint32_t _size357; + ::apache::thrift::protocol::TType _etype360; + xfer += iprot->readListBegin(_etype360, _size357); + this->primaryKeys.resize(_size357); + uint32_t _i361; + for (_i361 = 0; _i361 < _size357; ++_i361) + { + xfer += this->primaryKeys[_i361].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_primaryKeys = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_primaryKeys) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t PrimaryKeysResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PrimaryKeysResponse"); + + xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->primaryKeys.size())); + std::vector ::const_iterator _iter362; + for (_iter362 = this->primaryKeys.begin(); _iter362 != this->primaryKeys.end(); ++_iter362) + { + xfer += (*_iter362).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PrimaryKeysResponse &a, PrimaryKeysResponse &b) { + using ::std::swap; + swap(a.primaryKeys, b.primaryKeys); +} + +PrimaryKeysResponse::PrimaryKeysResponse(const PrimaryKeysResponse& other363) { + primaryKeys = other363.primaryKeys; +} +PrimaryKeysResponse& PrimaryKeysResponse::operator=(const PrimaryKeysResponse& other364) { + primaryKeys = other364.primaryKeys; + return *this; +} +void PrimaryKeysResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PrimaryKeysResponse("; + out << "primaryKeys=" << to_string(primaryKeys); + out << ")"; +} + + +ForeignKeysRequest::~ForeignKeysRequest() noexcept { +} + + +void ForeignKeysRequest::__set_parent_db_name(const std::string& val) { + this->parent_db_name = val; +} + +void ForeignKeysRequest::__set_parent_tbl_name(const std::string& val) { + this->parent_tbl_name = val; +} + +void ForeignKeysRequest::__set_foreign_db_name(const std::string& val) { + this->foreign_db_name = val; +} + +void ForeignKeysRequest::__set_foreign_tbl_name(const std::string& val) { + this->foreign_tbl_name = val; +} +std::ostream& operator<<(std::ostream& out, const ForeignKeysRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t ForeignKeysRequest::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->parent_db_name); + this->__isset.parent_db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->parent_tbl_name); + this->__isset.parent_tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->foreign_db_name); + this->__isset.foreign_db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->foreign_tbl_name); + this->__isset.foreign_tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ForeignKeysRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ForeignKeysRequest"); + + xfer += oprot->writeFieldBegin("parent_db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->parent_db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("parent_tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->parent_tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("foreign_db_name", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->foreign_db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("foreign_tbl_name", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->foreign_tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ForeignKeysRequest &a, ForeignKeysRequest &b) { + using ::std::swap; + swap(a.parent_db_name, b.parent_db_name); + swap(a.parent_tbl_name, b.parent_tbl_name); + swap(a.foreign_db_name, b.foreign_db_name); + swap(a.foreign_tbl_name, b.foreign_tbl_name); + swap(a.__isset, b.__isset); +} + +ForeignKeysRequest::ForeignKeysRequest(const ForeignKeysRequest& other365) { + parent_db_name = other365.parent_db_name; + parent_tbl_name = other365.parent_tbl_name; + foreign_db_name = other365.foreign_db_name; + foreign_tbl_name = other365.foreign_tbl_name; + __isset = other365.__isset; +} +ForeignKeysRequest& ForeignKeysRequest::operator=(const ForeignKeysRequest& other366) { + parent_db_name = other366.parent_db_name; + parent_tbl_name = other366.parent_tbl_name; + foreign_db_name = other366.foreign_db_name; + foreign_tbl_name = other366.foreign_tbl_name; + __isset = other366.__isset; + return *this; +} +void ForeignKeysRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "ForeignKeysRequest("; + out << "parent_db_name=" << to_string(parent_db_name); + out << ", " << "parent_tbl_name=" << to_string(parent_tbl_name); + out << ", " << "foreign_db_name=" << to_string(foreign_db_name); + out << ", " << "foreign_tbl_name=" << to_string(foreign_tbl_name); + out << ")"; +} + + +ForeignKeysResponse::~ForeignKeysResponse() noexcept { +} + + +void ForeignKeysResponse::__set_foreignKeys(const std::vector & val) { + this->foreignKeys = val; +} +std::ostream& operator<<(std::ostream& out, const ForeignKeysResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t ForeignKeysResponse::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_foreignKeys = 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_LIST) { + { + this->foreignKeys.clear(); + uint32_t _size367; + ::apache::thrift::protocol::TType _etype370; + xfer += iprot->readListBegin(_etype370, _size367); + this->foreignKeys.resize(_size367); + uint32_t _i371; + for (_i371 = 0; _i371 < _size367; ++_i371) + { + xfer += this->foreignKeys[_i371].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_foreignKeys = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_foreignKeys) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t ForeignKeysResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ForeignKeysResponse"); + + xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->foreignKeys.size())); + std::vector ::const_iterator _iter372; + for (_iter372 = this->foreignKeys.begin(); _iter372 != this->foreignKeys.end(); ++_iter372) + { + xfer += (*_iter372).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ForeignKeysResponse &a, ForeignKeysResponse &b) { + using ::std::swap; + swap(a.foreignKeys, b.foreignKeys); +} + +ForeignKeysResponse::ForeignKeysResponse(const ForeignKeysResponse& other373) { + foreignKeys = other373.foreignKeys; +} +ForeignKeysResponse& ForeignKeysResponse::operator=(const ForeignKeysResponse& other374) { + foreignKeys = other374.foreignKeys; + return *this; +} +void ForeignKeysResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "ForeignKeysResponse("; + out << "foreignKeys=" << to_string(foreignKeys); + out << ")"; +} + + +UniqueConstraintsRequest::~UniqueConstraintsRequest() noexcept { +} + + +void UniqueConstraintsRequest::__set_db_name(const std::string& val) { + this->db_name = val; +} + +void UniqueConstraintsRequest::__set_tbl_name(const std::string& val) { + this->tbl_name = val; +} +std::ostream& operator<<(std::ostream& out, const UniqueConstraintsRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t UniqueConstraintsRequest::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_db_name = false; + bool isset_tbl_name = 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_STRING) { + xfer += iprot->readString(this->db_name); + isset_db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + isset_tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_db_name) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tbl_name) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t UniqueConstraintsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("UniqueConstraintsRequest"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(UniqueConstraintsRequest &a, UniqueConstraintsRequest &b) { + using ::std::swap; + swap(a.db_name, b.db_name); + swap(a.tbl_name, b.tbl_name); +} + +UniqueConstraintsRequest::UniqueConstraintsRequest(const UniqueConstraintsRequest& other375) { + db_name = other375.db_name; + tbl_name = other375.tbl_name; +} +UniqueConstraintsRequest& UniqueConstraintsRequest::operator=(const UniqueConstraintsRequest& other376) { + db_name = other376.db_name; + tbl_name = other376.tbl_name; + return *this; +} +void UniqueConstraintsRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "UniqueConstraintsRequest("; + out << "db_name=" << to_string(db_name); + out << ", " << "tbl_name=" << to_string(tbl_name); + out << ")"; +} + + +UniqueConstraintsResponse::~UniqueConstraintsResponse() noexcept { +} + + +void UniqueConstraintsResponse::__set_uniqueConstraints(const std::vector & val) { + this->uniqueConstraints = val; +} +std::ostream& operator<<(std::ostream& out, const UniqueConstraintsResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t UniqueConstraintsResponse::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_uniqueConstraints = 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_LIST) { + { + this->uniqueConstraints.clear(); + uint32_t _size377; + ::apache::thrift::protocol::TType _etype380; + xfer += iprot->readListBegin(_etype380, _size377); + this->uniqueConstraints.resize(_size377); + uint32_t _i381; + for (_i381 = 0; _i381 < _size377; ++_i381) + { + xfer += this->uniqueConstraints[_i381].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_uniqueConstraints = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_uniqueConstraints) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t UniqueConstraintsResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("UniqueConstraintsResponse"); + + xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->uniqueConstraints.size())); + std::vector ::const_iterator _iter382; + for (_iter382 = this->uniqueConstraints.begin(); _iter382 != this->uniqueConstraints.end(); ++_iter382) + { + xfer += (*_iter382).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(UniqueConstraintsResponse &a, UniqueConstraintsResponse &b) { + using ::std::swap; + swap(a.uniqueConstraints, b.uniqueConstraints); +} + +UniqueConstraintsResponse::UniqueConstraintsResponse(const UniqueConstraintsResponse& other383) { + uniqueConstraints = other383.uniqueConstraints; +} +UniqueConstraintsResponse& UniqueConstraintsResponse::operator=(const UniqueConstraintsResponse& other384) { + uniqueConstraints = other384.uniqueConstraints; + return *this; +} +void UniqueConstraintsResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "UniqueConstraintsResponse("; + out << "uniqueConstraints=" << to_string(uniqueConstraints); + out << ")"; +} + + +NotNullConstraintsRequest::~NotNullConstraintsRequest() noexcept { +} + + +void NotNullConstraintsRequest::__set_db_name(const std::string& val) { + this->db_name = val; +} + +void NotNullConstraintsRequest::__set_tbl_name(const std::string& val) { + this->tbl_name = val; +} +std::ostream& operator<<(std::ostream& out, const NotNullConstraintsRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t NotNullConstraintsRequest::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_db_name = false; + bool isset_tbl_name = 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_STRING) { + xfer += iprot->readString(this->db_name); + isset_db_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tbl_name); + isset_tbl_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_db_name) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tbl_name) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t NotNullConstraintsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("NotNullConstraintsRequest"); + + xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->db_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tbl_name); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(NotNullConstraintsRequest &a, NotNullConstraintsRequest &b) { + using ::std::swap; + swap(a.db_name, b.db_name); + swap(a.tbl_name, b.tbl_name); +} + +NotNullConstraintsRequest::NotNullConstraintsRequest(const NotNullConstraintsRequest& other385) { + db_name = other385.db_name; + tbl_name = other385.tbl_name; +} +NotNullConstraintsRequest& NotNullConstraintsRequest::operator=(const NotNullConstraintsRequest& other386) { + db_name = other386.db_name; + tbl_name = other386.tbl_name; + return *this; +} +void NotNullConstraintsRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "NotNullConstraintsRequest("; + out << "db_name=" << to_string(db_name); + out << ", " << "tbl_name=" << to_string(tbl_name); + out << ")"; +} + + +NotNullConstraintsResponse::~NotNullConstraintsResponse() noexcept { +} + + +void NotNullConstraintsResponse::__set_notNullConstraints(const std::vector & val) { + this->notNullConstraints = val; +} +std::ostream& operator<<(std::ostream& out, const NotNullConstraintsResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t NotNullConstraintsResponse::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_notNullConstraints = 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_LIST) { + { + this->notNullConstraints.clear(); + uint32_t _size387; + ::apache::thrift::protocol::TType _etype390; + xfer += iprot->readListBegin(_etype390, _size387); + this->notNullConstraints.resize(_size387); + uint32_t _i391; + for (_i391 = 0; _i391 < _size387; ++_i391) + { + xfer += this->notNullConstraints[_i391].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_notNullConstraints = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_notNullConstraints) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t NotNullConstraintsResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("NotNullConstraintsResponse"); + + xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->notNullConstraints.size())); + std::vector ::const_iterator _iter392; + for (_iter392 = this->notNullConstraints.begin(); _iter392 != this->notNullConstraints.end(); ++_iter392) + { + xfer += (*_iter392).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(NotNullConstraintsResponse &a, NotNullConstraintsResponse &b) { + using ::std::swap; + swap(a.notNullConstraints, b.notNullConstraints); +} + +NotNullConstraintsResponse::NotNullConstraintsResponse(const NotNullConstraintsResponse& other393) { + notNullConstraints = other393.notNullConstraints; +} +NotNullConstraintsResponse& NotNullConstraintsResponse::operator=(const NotNullConstraintsResponse& other394) { + notNullConstraints = other394.notNullConstraints; + return *this; +} +void NotNullConstraintsResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "NotNullConstraintsResponse("; + out << "notNullConstraints=" << to_string(notNullConstraints); + out << ")"; +} + + +DropConstraintRequest::~DropConstraintRequest() noexcept { +} + + +void DropConstraintRequest::__set_dbname(const std::string& val) { + this->dbname = val; +} + +void DropConstraintRequest::__set_tablename(const std::string& val) { + this->tablename = val; +} + +void DropConstraintRequest::__set_constraintname(const std::string& val) { + this->constraintname = val; +} +std::ostream& operator<<(std::ostream& out, const DropConstraintRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t DropConstraintRequest::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_dbname = false; + bool isset_tablename = false; + bool isset_constraintname = 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_STRING) { + xfer += iprot->readString(this->dbname); + isset_dbname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tablename); + isset_tablename = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->constraintname); + isset_constraintname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_dbname) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tablename) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_constraintname) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t DropConstraintRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("DropConstraintRequest"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbname); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tablename", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tablename); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("constraintname", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->constraintname); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(DropConstraintRequest &a, DropConstraintRequest &b) { + using ::std::swap; + swap(a.dbname, b.dbname); + swap(a.tablename, b.tablename); + swap(a.constraintname, b.constraintname); +} + +DropConstraintRequest::DropConstraintRequest(const DropConstraintRequest& other395) { + dbname = other395.dbname; + tablename = other395.tablename; + constraintname = other395.constraintname; +} +DropConstraintRequest& DropConstraintRequest::operator=(const DropConstraintRequest& other396) { + dbname = other396.dbname; + tablename = other396.tablename; + constraintname = other396.constraintname; + return *this; +} +void DropConstraintRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "DropConstraintRequest("; + out << "dbname=" << to_string(dbname); + out << ", " << "tablename=" << to_string(tablename); + out << ", " << "constraintname=" << to_string(constraintname); + out << ")"; +} + + +AddPrimaryKeyRequest::~AddPrimaryKeyRequest() noexcept { +} + + +void AddPrimaryKeyRequest::__set_primaryKeyCols(const std::vector & val) { + this->primaryKeyCols = val; +} +std::ostream& operator<<(std::ostream& out, const AddPrimaryKeyRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t AddPrimaryKeyRequest::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_primaryKeyCols = 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_LIST) { + { + this->primaryKeyCols.clear(); + uint32_t _size397; + ::apache::thrift::protocol::TType _etype400; + xfer += iprot->readListBegin(_etype400, _size397); + this->primaryKeyCols.resize(_size397); + uint32_t _i401; + for (_i401 = 0; _i401 < _size397; ++_i401) + { + xfer += this->primaryKeyCols[_i401].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_primaryKeyCols = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_primaryKeyCols) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t AddPrimaryKeyRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("AddPrimaryKeyRequest"); + + xfer += oprot->writeFieldBegin("primaryKeyCols", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->primaryKeyCols.size())); + std::vector ::const_iterator _iter402; + for (_iter402 = this->primaryKeyCols.begin(); _iter402 != this->primaryKeyCols.end(); ++_iter402) + { + xfer += (*_iter402).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(AddPrimaryKeyRequest &a, AddPrimaryKeyRequest &b) { + using ::std::swap; + swap(a.primaryKeyCols, b.primaryKeyCols); +} + +AddPrimaryKeyRequest::AddPrimaryKeyRequest(const AddPrimaryKeyRequest& other403) { + primaryKeyCols = other403.primaryKeyCols; +} +AddPrimaryKeyRequest& AddPrimaryKeyRequest::operator=(const AddPrimaryKeyRequest& other404) { + primaryKeyCols = other404.primaryKeyCols; + return *this; +} +void AddPrimaryKeyRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "AddPrimaryKeyRequest("; + out << "primaryKeyCols=" << to_string(primaryKeyCols); + out << ")"; +} + + +AddForeignKeyRequest::~AddForeignKeyRequest() noexcept { +} + + +void AddForeignKeyRequest::__set_foreignKeyCols(const std::vector & val) { + this->foreignKeyCols = val; +} +std::ostream& operator<<(std::ostream& out, const AddForeignKeyRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t AddForeignKeyRequest::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_foreignKeyCols = 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_LIST) { + { + this->foreignKeyCols.clear(); + uint32_t _size405; + ::apache::thrift::protocol::TType _etype408; + xfer += iprot->readListBegin(_etype408, _size405); + this->foreignKeyCols.resize(_size405); + uint32_t _i409; + for (_i409 = 0; _i409 < _size405; ++_i409) + { + xfer += this->foreignKeyCols[_i409].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_foreignKeyCols = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_foreignKeyCols) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t AddForeignKeyRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("AddForeignKeyRequest"); + + xfer += oprot->writeFieldBegin("foreignKeyCols", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->foreignKeyCols.size())); + std::vector ::const_iterator _iter410; + for (_iter410 = this->foreignKeyCols.begin(); _iter410 != this->foreignKeyCols.end(); ++_iter410) + { + xfer += (*_iter410).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(AddForeignKeyRequest &a, AddForeignKeyRequest &b) { + using ::std::swap; + swap(a.foreignKeyCols, b.foreignKeyCols); +} + +AddForeignKeyRequest::AddForeignKeyRequest(const AddForeignKeyRequest& other411) { + foreignKeyCols = other411.foreignKeyCols; +} +AddForeignKeyRequest& AddForeignKeyRequest::operator=(const AddForeignKeyRequest& other412) { + foreignKeyCols = other412.foreignKeyCols; + return *this; +} +void AddForeignKeyRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "AddForeignKeyRequest("; + out << "foreignKeyCols=" << to_string(foreignKeyCols); + out << ")"; +} + + +AddUniqueConstraintRequest::~AddUniqueConstraintRequest() noexcept { +} + + +void AddUniqueConstraintRequest::__set_uniqueConstraintCols(const std::vector & val) { + this->uniqueConstraintCols = val; +} +std::ostream& operator<<(std::ostream& out, const AddUniqueConstraintRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t AddUniqueConstraintRequest::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_uniqueConstraintCols = 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_LIST) { + { + this->uniqueConstraintCols.clear(); + uint32_t _size413; + ::apache::thrift::protocol::TType _etype416; + xfer += iprot->readListBegin(_etype416, _size413); + this->uniqueConstraintCols.resize(_size413); + uint32_t _i417; + for (_i417 = 0; _i417 < _size413; ++_i417) + { + xfer += this->uniqueConstraintCols[_i417].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_uniqueConstraintCols = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_uniqueConstraintCols) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t AddUniqueConstraintRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("AddUniqueConstraintRequest"); + + xfer += oprot->writeFieldBegin("uniqueConstraintCols", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->uniqueConstraintCols.size())); + std::vector ::const_iterator _iter418; + for (_iter418 = this->uniqueConstraintCols.begin(); _iter418 != this->uniqueConstraintCols.end(); ++_iter418) + { + xfer += (*_iter418).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(AddUniqueConstraintRequest &a, AddUniqueConstraintRequest &b) { + using ::std::swap; + swap(a.uniqueConstraintCols, b.uniqueConstraintCols); +} + +AddUniqueConstraintRequest::AddUniqueConstraintRequest(const AddUniqueConstraintRequest& other419) { + uniqueConstraintCols = other419.uniqueConstraintCols; +} +AddUniqueConstraintRequest& AddUniqueConstraintRequest::operator=(const AddUniqueConstraintRequest& other420) { + uniqueConstraintCols = other420.uniqueConstraintCols; + return *this; +} +void AddUniqueConstraintRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "AddUniqueConstraintRequest("; + out << "uniqueConstraintCols=" << to_string(uniqueConstraintCols); + out << ")"; +} + + +AddNotNullConstraintRequest::~AddNotNullConstraintRequest() noexcept { +} + + +void AddNotNullConstraintRequest::__set_notNullConstraintCols(const std::vector & val) { + this->notNullConstraintCols = val; +} +std::ostream& operator<<(std::ostream& out, const AddNotNullConstraintRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t AddNotNullConstraintRequest::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_notNullConstraintCols = 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_LIST) { + { + this->notNullConstraintCols.clear(); + uint32_t _size421; + ::apache::thrift::protocol::TType _etype424; + xfer += iprot->readListBegin(_etype424, _size421); + this->notNullConstraintCols.resize(_size421); + uint32_t _i425; + for (_i425 = 0; _i425 < _size421; ++_i425) + { + xfer += this->notNullConstraintCols[_i425].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_notNullConstraintCols = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_notNullConstraintCols) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t AddNotNullConstraintRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("AddNotNullConstraintRequest"); + + xfer += oprot->writeFieldBegin("notNullConstraintCols", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->notNullConstraintCols.size())); + std::vector ::const_iterator _iter426; + for (_iter426 = this->notNullConstraintCols.begin(); _iter426 != this->notNullConstraintCols.end(); ++_iter426) + { + xfer += (*_iter426).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(AddNotNullConstraintRequest &a, AddNotNullConstraintRequest &b) { + using ::std::swap; + swap(a.notNullConstraintCols, b.notNullConstraintCols); +} + +AddNotNullConstraintRequest::AddNotNullConstraintRequest(const AddNotNullConstraintRequest& other427) { + notNullConstraintCols = other427.notNullConstraintCols; +} +AddNotNullConstraintRequest& AddNotNullConstraintRequest::operator=(const AddNotNullConstraintRequest& other428) { + notNullConstraintCols = other428.notNullConstraintCols; + return *this; +} +void AddNotNullConstraintRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "AddNotNullConstraintRequest("; + out << "notNullConstraintCols=" << to_string(notNullConstraintCols); + out << ")"; +} + + +PartitionsByExprResult::~PartitionsByExprResult() noexcept { +} + + +void PartitionsByExprResult::__set_partitions(const std::vector & val) { + this->partitions = val; +} + +void PartitionsByExprResult::__set_hasUnknownPartitions(const bool val) { + this->hasUnknownPartitions = val; +} +std::ostream& operator<<(std::ostream& out, const PartitionsByExprResult& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PartitionsByExprResult::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_partitions = false; + bool isset_hasUnknownPartitions = 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_LIST) { + { + this->partitions.clear(); + uint32_t _size429; + ::apache::thrift::protocol::TType _etype432; + xfer += iprot->readListBegin(_etype432, _size429); + this->partitions.resize(_size429); + uint32_t _i433; + for (_i433 = 0; _i433 < _size429; ++_i433) + { + xfer += this->partitions[_i433].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_partitions = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->hasUnknownPartitions); + isset_hasUnknownPartitions = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_partitions) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_hasUnknownPartitions) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t PartitionsByExprResult::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PartitionsByExprResult"); + + xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitions.size())); + std::vector ::const_iterator _iter434; + for (_iter434 = this->partitions.begin(); _iter434 != this->partitions.end(); ++_iter434) + { + xfer += (*_iter434).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("hasUnknownPartitions", ::apache::thrift::protocol::T_BOOL, 2); + xfer += oprot->writeBool(this->hasUnknownPartitions); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PartitionsByExprResult &a, PartitionsByExprResult &b) { + using ::std::swap; + swap(a.partitions, b.partitions); + swap(a.hasUnknownPartitions, b.hasUnknownPartitions); +} + +PartitionsByExprResult::PartitionsByExprResult(const PartitionsByExprResult& other435) { + partitions = other435.partitions; + hasUnknownPartitions = other435.hasUnknownPartitions; +} +PartitionsByExprResult& PartitionsByExprResult::operator=(const PartitionsByExprResult& other436) { + partitions = other436.partitions; + hasUnknownPartitions = other436.hasUnknownPartitions; + return *this; +} +void PartitionsByExprResult::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PartitionsByExprResult("; + out << "partitions=" << to_string(partitions); + out << ", " << "hasUnknownPartitions=" << to_string(hasUnknownPartitions); + out << ")"; +} + + +PartitionsByExprRequest::~PartitionsByExprRequest() noexcept { +} + + +void PartitionsByExprRequest::__set_dbName(const std::string& val) { + this->dbName = val; +} + +void PartitionsByExprRequest::__set_tblName(const std::string& val) { + this->tblName = val; +} + +void PartitionsByExprRequest::__set_expr(const std::string& val) { + this->expr = val; +} + +void PartitionsByExprRequest::__set_defaultPartitionName(const std::string& val) { + this->defaultPartitionName = val; +__isset.defaultPartitionName = true; +} + +void PartitionsByExprRequest::__set_maxParts(const int16_t val) { + this->maxParts = val; +__isset.maxParts = true; +} +std::ostream& operator<<(std::ostream& out, const PartitionsByExprRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PartitionsByExprRequest::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_dbName = false; + bool isset_tblName = false; + bool isset_expr = 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_STRING) { + xfer += iprot->readString(this->dbName); + isset_dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tblName); + isset_tblName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readBinary(this->expr); + isset_expr = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->defaultPartitionName); + this->__isset.defaultPartitionName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_I16) { + xfer += iprot->readI16(this->maxParts); + this->__isset.maxParts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_dbName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tblName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_expr) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t PartitionsByExprRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PartitionsByExprRequest"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tblName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("expr", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeBinary(this->expr); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.defaultPartitionName) { + xfer += oprot->writeFieldBegin("defaultPartitionName", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->defaultPartitionName); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.maxParts) { + xfer += oprot->writeFieldBegin("maxParts", ::apache::thrift::protocol::T_I16, 5); + xfer += oprot->writeI16(this->maxParts); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PartitionsByExprRequest &a, PartitionsByExprRequest &b) { + using ::std::swap; + swap(a.dbName, b.dbName); + swap(a.tblName, b.tblName); + swap(a.expr, b.expr); + swap(a.defaultPartitionName, b.defaultPartitionName); + swap(a.maxParts, b.maxParts); + swap(a.__isset, b.__isset); +} + +PartitionsByExprRequest::PartitionsByExprRequest(const PartitionsByExprRequest& other437) { + dbName = other437.dbName; + tblName = other437.tblName; + expr = other437.expr; + defaultPartitionName = other437.defaultPartitionName; + maxParts = other437.maxParts; + __isset = other437.__isset; +} +PartitionsByExprRequest& PartitionsByExprRequest::operator=(const PartitionsByExprRequest& other438) { + dbName = other438.dbName; + tblName = other438.tblName; + expr = other438.expr; + defaultPartitionName = other438.defaultPartitionName; + maxParts = other438.maxParts; + __isset = other438.__isset; + return *this; +} +void PartitionsByExprRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PartitionsByExprRequest("; + out << "dbName=" << to_string(dbName); + out << ", " << "tblName=" << to_string(tblName); + out << ", " << "expr=" << to_string(expr); + out << ", " << "defaultPartitionName="; (__isset.defaultPartitionName ? (out << to_string(defaultPartitionName)) : (out << "")); + out << ", " << "maxParts="; (__isset.maxParts ? (out << to_string(maxParts)) : (out << "")); + out << ")"; +} + + +TableStatsResult::~TableStatsResult() noexcept { +} + + +void TableStatsResult::__set_tableStats(const std::vector & val) { + this->tableStats = val; +} +std::ostream& operator<<(std::ostream& out, const TableStatsResult& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t TableStatsResult::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_tableStats = 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_LIST) { + { + this->tableStats.clear(); + uint32_t _size439; + ::apache::thrift::protocol::TType _etype442; + xfer += iprot->readListBegin(_etype442, _size439); + this->tableStats.resize(_size439); + uint32_t _i443; + for (_i443 = 0; _i443 < _size439; ++_i443) + { + xfer += this->tableStats[_i443].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_tableStats = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_tableStats) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t TableStatsResult::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("TableStatsResult"); + + xfer += oprot->writeFieldBegin("tableStats", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->tableStats.size())); + std::vector ::const_iterator _iter444; + for (_iter444 = this->tableStats.begin(); _iter444 != this->tableStats.end(); ++_iter444) + { + xfer += (*_iter444).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(TableStatsResult &a, TableStatsResult &b) { + using ::std::swap; + swap(a.tableStats, b.tableStats); +} + +TableStatsResult::TableStatsResult(const TableStatsResult& other445) { + tableStats = other445.tableStats; +} +TableStatsResult& TableStatsResult::operator=(const TableStatsResult& other446) { + tableStats = other446.tableStats; + return *this; +} +void TableStatsResult::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "TableStatsResult("; + out << "tableStats=" << to_string(tableStats); + out << ")"; +} + + +PartitionsStatsResult::~PartitionsStatsResult() noexcept { +} + + +void PartitionsStatsResult::__set_partStats(const std::map > & val) { + this->partStats = val; +} +std::ostream& operator<<(std::ostream& out, const PartitionsStatsResult& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PartitionsStatsResult::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_partStats = 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_MAP) { + { + this->partStats.clear(); + uint32_t _size447; + ::apache::thrift::protocol::TType _ktype448; + ::apache::thrift::protocol::TType _vtype449; + xfer += iprot->readMapBegin(_ktype448, _vtype449, _size447); + uint32_t _i451; + for (_i451 = 0; _i451 < _size447; ++_i451) + { + std::string _key452; + xfer += iprot->readString(_key452); + std::vector & _val453 = this->partStats[_key452]; + { + _val453.clear(); + uint32_t _size454; + ::apache::thrift::protocol::TType _etype457; + xfer += iprot->readListBegin(_etype457, _size454); + _val453.resize(_size454); + uint32_t _i458; + for (_i458 = 0; _i458 < _size454; ++_i458) + { + xfer += _val453[_i458].read(iprot); + } + xfer += iprot->readListEnd(); + } + } + xfer += iprot->readMapEnd(); + } + isset_partStats = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_partStats) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t PartitionsStatsResult::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PartitionsStatsResult"); + + xfer += oprot->writeFieldBegin("partStats", ::apache::thrift::protocol::T_MAP, 1); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast(this->partStats.size())); + std::map > ::const_iterator _iter459; + for (_iter459 = this->partStats.begin(); _iter459 != this->partStats.end(); ++_iter459) + { + xfer += oprot->writeString(_iter459->first); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(_iter459->second.size())); + std::vector ::const_iterator _iter460; + for (_iter460 = _iter459->second.begin(); _iter460 != _iter459->second.end(); ++_iter460) + { + xfer += (*_iter460).write(oprot); + } + xfer += oprot->writeListEnd(); + } + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PartitionsStatsResult &a, PartitionsStatsResult &b) { + using ::std::swap; + swap(a.partStats, b.partStats); +} + +PartitionsStatsResult::PartitionsStatsResult(const PartitionsStatsResult& other461) { + partStats = other461.partStats; +} +PartitionsStatsResult& PartitionsStatsResult::operator=(const PartitionsStatsResult& other462) { + partStats = other462.partStats; + return *this; +} +void PartitionsStatsResult::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PartitionsStatsResult("; + out << "partStats=" << to_string(partStats); + out << ")"; +} + + +TableStatsRequest::~TableStatsRequest() noexcept { +} + + +void TableStatsRequest::__set_dbName(const std::string& val) { + this->dbName = val; +} + +void TableStatsRequest::__set_tblName(const std::string& val) { + this->tblName = val; +} + +void TableStatsRequest::__set_colNames(const std::vector & val) { + this->colNames = val; +} +std::ostream& operator<<(std::ostream& out, const TableStatsRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t TableStatsRequest::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_dbName = false; + bool isset_tblName = false; + bool isset_colNames = 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_STRING) { + xfer += iprot->readString(this->dbName); + isset_dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tblName); + isset_tblName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->colNames.clear(); + uint32_t _size463; + ::apache::thrift::protocol::TType _etype466; + xfer += iprot->readListBegin(_etype466, _size463); + this->colNames.resize(_size463); + uint32_t _i467; + for (_i467 = 0; _i467 < _size463; ++_i467) + { + xfer += iprot->readString(this->colNames[_i467]); + } + xfer += iprot->readListEnd(); + } + isset_colNames = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_dbName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tblName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_colNames) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t TableStatsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("TableStatsRequest"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tblName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("colNames", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->colNames.size())); + std::vector ::const_iterator _iter468; + for (_iter468 = this->colNames.begin(); _iter468 != this->colNames.end(); ++_iter468) + { + xfer += oprot->writeString((*_iter468)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(TableStatsRequest &a, TableStatsRequest &b) { + using ::std::swap; + swap(a.dbName, b.dbName); + swap(a.tblName, b.tblName); + swap(a.colNames, b.colNames); +} + +TableStatsRequest::TableStatsRequest(const TableStatsRequest& other469) { + dbName = other469.dbName; + tblName = other469.tblName; + colNames = other469.colNames; +} +TableStatsRequest& TableStatsRequest::operator=(const TableStatsRequest& other470) { + dbName = other470.dbName; + tblName = other470.tblName; + colNames = other470.colNames; + return *this; +} +void TableStatsRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "TableStatsRequest("; + out << "dbName=" << to_string(dbName); + out << ", " << "tblName=" << to_string(tblName); + out << ", " << "colNames=" << to_string(colNames); + out << ")"; +} + + +PartitionsStatsRequest::~PartitionsStatsRequest() noexcept { +} + + +void PartitionsStatsRequest::__set_dbName(const std::string& val) { + this->dbName = val; +} + +void PartitionsStatsRequest::__set_tblName(const std::string& val) { + this->tblName = val; +} + +void PartitionsStatsRequest::__set_colNames(const std::vector & val) { + this->colNames = val; +} + +void PartitionsStatsRequest::__set_partNames(const std::vector & val) { + this->partNames = val; +} +std::ostream& operator<<(std::ostream& out, const PartitionsStatsRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PartitionsStatsRequest::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_dbName = false; + bool isset_tblName = false; + bool isset_colNames = false; + bool isset_partNames = 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_STRING) { + xfer += iprot->readString(this->dbName); + isset_dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tblName); + isset_tblName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->colNames.clear(); + uint32_t _size471; + ::apache::thrift::protocol::TType _etype474; + xfer += iprot->readListBegin(_etype474, _size471); + this->colNames.resize(_size471); + uint32_t _i475; + for (_i475 = 0; _i475 < _size471; ++_i475) + { + xfer += iprot->readString(this->colNames[_i475]); + } + xfer += iprot->readListEnd(); + } + isset_colNames = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->partNames.clear(); + uint32_t _size476; + ::apache::thrift::protocol::TType _etype479; + xfer += iprot->readListBegin(_etype479, _size476); + this->partNames.resize(_size476); + uint32_t _i480; + for (_i480 = 0; _i480 < _size476; ++_i480) + { + xfer += iprot->readString(this->partNames[_i480]); + } + xfer += iprot->readListEnd(); + } + isset_partNames = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_dbName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tblName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_colNames) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_partNames) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t PartitionsStatsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PartitionsStatsRequest"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tblName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("colNames", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->colNames.size())); + std::vector ::const_iterator _iter481; + for (_iter481 = this->colNames.begin(); _iter481 != this->colNames.end(); ++_iter481) + { + xfer += oprot->writeString((*_iter481)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 4); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partNames.size())); + std::vector ::const_iterator _iter482; + for (_iter482 = this->partNames.begin(); _iter482 != this->partNames.end(); ++_iter482) + { + xfer += oprot->writeString((*_iter482)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PartitionsStatsRequest &a, PartitionsStatsRequest &b) { + using ::std::swap; + swap(a.dbName, b.dbName); + swap(a.tblName, b.tblName); + swap(a.colNames, b.colNames); + swap(a.partNames, b.partNames); +} + +PartitionsStatsRequest::PartitionsStatsRequest(const PartitionsStatsRequest& other483) { + dbName = other483.dbName; + tblName = other483.tblName; + colNames = other483.colNames; + partNames = other483.partNames; +} +PartitionsStatsRequest& PartitionsStatsRequest::operator=(const PartitionsStatsRequest& other484) { + dbName = other484.dbName; + tblName = other484.tblName; + colNames = other484.colNames; + partNames = other484.partNames; + return *this; +} +void PartitionsStatsRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PartitionsStatsRequest("; + out << "dbName=" << to_string(dbName); + out << ", " << "tblName=" << to_string(tblName); + out << ", " << "colNames=" << to_string(colNames); + out << ", " << "partNames=" << to_string(partNames); + out << ")"; +} + + +AddPartitionsResult::~AddPartitionsResult() noexcept { +} + + +void AddPartitionsResult::__set_partitions(const std::vector & val) { + this->partitions = val; +__isset.partitions = true; +} +std::ostream& operator<<(std::ostream& out, const AddPartitionsResult& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t AddPartitionsResult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->partitions.clear(); + uint32_t _size485; + ::apache::thrift::protocol::TType _etype488; + xfer += iprot->readListBegin(_etype488, _size485); + this->partitions.resize(_size485); + uint32_t _i489; + for (_i489 = 0; _i489 < _size485; ++_i489) + { + xfer += this->partitions[_i489].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.partitions = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t AddPartitionsResult::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("AddPartitionsResult"); + + if (this->__isset.partitions) { + xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitions.size())); + std::vector ::const_iterator _iter490; + for (_iter490 = this->partitions.begin(); _iter490 != this->partitions.end(); ++_iter490) + { + xfer += (*_iter490).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(AddPartitionsResult &a, AddPartitionsResult &b) { + using ::std::swap; + swap(a.partitions, b.partitions); + swap(a.__isset, b.__isset); +} + +AddPartitionsResult::AddPartitionsResult(const AddPartitionsResult& other491) { + partitions = other491.partitions; + __isset = other491.__isset; +} +AddPartitionsResult& AddPartitionsResult::operator=(const AddPartitionsResult& other492) { + partitions = other492.partitions; + __isset = other492.__isset; + return *this; +} +void AddPartitionsResult::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "AddPartitionsResult("; + out << "partitions="; (__isset.partitions ? (out << to_string(partitions)) : (out << "")); + out << ")"; +} + + +AddPartitionsRequest::~AddPartitionsRequest() noexcept { +} + + +void AddPartitionsRequest::__set_dbName(const std::string& val) { + this->dbName = val; +} + +void AddPartitionsRequest::__set_tblName(const std::string& val) { + this->tblName = val; +} + +void AddPartitionsRequest::__set_parts(const std::vector & val) { + this->parts = val; +} + +void AddPartitionsRequest::__set_ifNotExists(const bool val) { + this->ifNotExists = val; +} + +void AddPartitionsRequest::__set_needResult(const bool val) { + this->needResult = val; +__isset.needResult = true; +} +std::ostream& operator<<(std::ostream& out, const AddPartitionsRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t AddPartitionsRequest::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_dbName = false; + bool isset_tblName = false; + bool isset_parts = false; + bool isset_ifNotExists = 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_STRING) { + xfer += iprot->readString(this->dbName); + isset_dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tblName); + isset_tblName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->parts.clear(); + uint32_t _size493; + ::apache::thrift::protocol::TType _etype496; + xfer += iprot->readListBegin(_etype496, _size493); + this->parts.resize(_size493); + uint32_t _i497; + for (_i497 = 0; _i497 < _size493; ++_i497) + { + xfer += this->parts[_i497].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_parts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->ifNotExists); + isset_ifNotExists = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->needResult); + this->__isset.needResult = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_dbName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tblName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_parts) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_ifNotExists) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t AddPartitionsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("AddPartitionsRequest"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tblName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("parts", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->parts.size())); + std::vector ::const_iterator _iter498; + for (_iter498 = this->parts.begin(); _iter498 != this->parts.end(); ++_iter498) + { + xfer += (*_iter498).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("ifNotExists", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool(this->ifNotExists); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.needResult) { + xfer += oprot->writeFieldBegin("needResult", ::apache::thrift::protocol::T_BOOL, 5); + xfer += oprot->writeBool(this->needResult); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(AddPartitionsRequest &a, AddPartitionsRequest &b) { + using ::std::swap; + swap(a.dbName, b.dbName); + swap(a.tblName, b.tblName); + swap(a.parts, b.parts); + swap(a.ifNotExists, b.ifNotExists); + swap(a.needResult, b.needResult); + swap(a.__isset, b.__isset); +} + +AddPartitionsRequest::AddPartitionsRequest(const AddPartitionsRequest& other499) { + dbName = other499.dbName; + tblName = other499.tblName; + parts = other499.parts; + ifNotExists = other499.ifNotExists; + needResult = other499.needResult; + __isset = other499.__isset; +} +AddPartitionsRequest& AddPartitionsRequest::operator=(const AddPartitionsRequest& other500) { + dbName = other500.dbName; + tblName = other500.tblName; + parts = other500.parts; + ifNotExists = other500.ifNotExists; + needResult = other500.needResult; + __isset = other500.__isset; + return *this; +} +void AddPartitionsRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "AddPartitionsRequest("; + out << "dbName=" << to_string(dbName); + out << ", " << "tblName=" << to_string(tblName); + out << ", " << "parts=" << to_string(parts); + out << ", " << "ifNotExists=" << to_string(ifNotExists); + out << ", " << "needResult="; (__isset.needResult ? (out << to_string(needResult)) : (out << "")); + out << ")"; +} + + +DropPartitionsResult::~DropPartitionsResult() noexcept { +} + + +void DropPartitionsResult::__set_partitions(const std::vector & val) { + this->partitions = val; +__isset.partitions = true; +} +std::ostream& operator<<(std::ostream& out, const DropPartitionsResult& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t DropPartitionsResult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->partitions.clear(); + uint32_t _size501; + ::apache::thrift::protocol::TType _etype504; + xfer += iprot->readListBegin(_etype504, _size501); + this->partitions.resize(_size501); + uint32_t _i505; + for (_i505 = 0; _i505 < _size501; ++_i505) + { + xfer += this->partitions[_i505].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.partitions = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t DropPartitionsResult::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("DropPartitionsResult"); + + if (this->__isset.partitions) { + xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitions.size())); + std::vector ::const_iterator _iter506; + for (_iter506 = this->partitions.begin(); _iter506 != this->partitions.end(); ++_iter506) + { + xfer += (*_iter506).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(DropPartitionsResult &a, DropPartitionsResult &b) { + using ::std::swap; + swap(a.partitions, b.partitions); + swap(a.__isset, b.__isset); +} + +DropPartitionsResult::DropPartitionsResult(const DropPartitionsResult& other507) { + partitions = other507.partitions; + __isset = other507.__isset; +} +DropPartitionsResult& DropPartitionsResult::operator=(const DropPartitionsResult& other508) { + partitions = other508.partitions; + __isset = other508.__isset; + return *this; +} +void DropPartitionsResult::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "DropPartitionsResult("; + out << "partitions="; (__isset.partitions ? (out << to_string(partitions)) : (out << "")); + out << ")"; +} + + +DropPartitionsExpr::~DropPartitionsExpr() noexcept { +} + + +void DropPartitionsExpr::__set_expr(const std::string& val) { + this->expr = val; +} + +void DropPartitionsExpr::__set_partArchiveLevel(const int32_t val) { + this->partArchiveLevel = val; +__isset.partArchiveLevel = true; +} +std::ostream& operator<<(std::ostream& out, const DropPartitionsExpr& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t DropPartitionsExpr::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_expr = 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_STRING) { + xfer += iprot->readBinary(this->expr); + isset_expr = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->partArchiveLevel); + this->__isset.partArchiveLevel = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_expr) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t DropPartitionsExpr::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("DropPartitionsExpr"); + + xfer += oprot->writeFieldBegin("expr", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeBinary(this->expr); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.partArchiveLevel) { + xfer += oprot->writeFieldBegin("partArchiveLevel", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32(this->partArchiveLevel); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(DropPartitionsExpr &a, DropPartitionsExpr &b) { + using ::std::swap; + swap(a.expr, b.expr); + swap(a.partArchiveLevel, b.partArchiveLevel); + swap(a.__isset, b.__isset); +} + +DropPartitionsExpr::DropPartitionsExpr(const DropPartitionsExpr& other509) { + expr = other509.expr; + partArchiveLevel = other509.partArchiveLevel; + __isset = other509.__isset; +} +DropPartitionsExpr& DropPartitionsExpr::operator=(const DropPartitionsExpr& other510) { + expr = other510.expr; + partArchiveLevel = other510.partArchiveLevel; + __isset = other510.__isset; + return *this; +} +void DropPartitionsExpr::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "DropPartitionsExpr("; + out << "expr=" << to_string(expr); + out << ", " << "partArchiveLevel="; (__isset.partArchiveLevel ? (out << to_string(partArchiveLevel)) : (out << "")); + out << ")"; +} + + +RequestPartsSpec::~RequestPartsSpec() noexcept { +} + + +void RequestPartsSpec::__set_names(const std::vector & val) { + this->names = val; +__isset.names = true; +} + +void RequestPartsSpec::__set_exprs(const std::vector & val) { + this->exprs = val; +__isset.exprs = true; +} +std::ostream& operator<<(std::ostream& out, const RequestPartsSpec& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t RequestPartsSpec::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->names.clear(); + uint32_t _size511; + ::apache::thrift::protocol::TType _etype514; + xfer += iprot->readListBegin(_etype514, _size511); + this->names.resize(_size511); + uint32_t _i515; + for (_i515 = 0; _i515 < _size511; ++_i515) + { + xfer += iprot->readString(this->names[_i515]); + } + xfer += iprot->readListEnd(); + } + this->__isset.names = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->exprs.clear(); + uint32_t _size516; + ::apache::thrift::protocol::TType _etype519; + xfer += iprot->readListBegin(_etype519, _size516); + this->exprs.resize(_size516); + uint32_t _i520; + for (_i520 = 0; _i520 < _size516; ++_i520) + { + xfer += this->exprs[_i520].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.exprs = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t RequestPartsSpec::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("RequestPartsSpec"); + + if (this->__isset.names) { + xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->names.size())); + std::vector ::const_iterator _iter521; + for (_iter521 = this->names.begin(); _iter521 != this->names.end(); ++_iter521) + { + xfer += oprot->writeString((*_iter521)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.exprs) { + xfer += oprot->writeFieldBegin("exprs", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->exprs.size())); + std::vector ::const_iterator _iter522; + for (_iter522 = this->exprs.begin(); _iter522 != this->exprs.end(); ++_iter522) + { + xfer += (*_iter522).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(RequestPartsSpec &a, RequestPartsSpec &b) { + using ::std::swap; + swap(a.names, b.names); + swap(a.exprs, b.exprs); + swap(a.__isset, b.__isset); +} + +RequestPartsSpec::RequestPartsSpec(const RequestPartsSpec& other523) { + names = other523.names; + exprs = other523.exprs; + __isset = other523.__isset; +} +RequestPartsSpec& RequestPartsSpec::operator=(const RequestPartsSpec& other524) { + names = other524.names; + exprs = other524.exprs; + __isset = other524.__isset; + return *this; +} +void RequestPartsSpec::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "RequestPartsSpec("; + out << "names="; (__isset.names ? (out << to_string(names)) : (out << "")); + out << ", " << "exprs="; (__isset.exprs ? (out << to_string(exprs)) : (out << "")); + out << ")"; +} + + +DropPartitionsRequest::~DropPartitionsRequest() noexcept { +} + + +void DropPartitionsRequest::__set_dbName(const std::string& val) { + this->dbName = val; +} + +void DropPartitionsRequest::__set_tblName(const std::string& val) { + this->tblName = val; +} + +void DropPartitionsRequest::__set_parts(const RequestPartsSpec& val) { + this->parts = val; +} + +void DropPartitionsRequest::__set_deleteData(const bool val) { + this->deleteData = val; +__isset.deleteData = true; +} + +void DropPartitionsRequest::__set_ifExists(const bool val) { + this->ifExists = val; +__isset.ifExists = true; +} + +void DropPartitionsRequest::__set_ignoreProtection(const bool val) { + this->ignoreProtection = val; +__isset.ignoreProtection = true; +} + +void DropPartitionsRequest::__set_environmentContext(const EnvironmentContext& val) { + this->environmentContext = val; +__isset.environmentContext = true; +} + +void DropPartitionsRequest::__set_needResult(const bool val) { + this->needResult = val; +__isset.needResult = true; +} +std::ostream& operator<<(std::ostream& out, const DropPartitionsRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t DropPartitionsRequest::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_dbName = false; + bool isset_tblName = false; + bool isset_parts = 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_STRING) { + xfer += iprot->readString(this->dbName); + isset_dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tblName); + isset_tblName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->parts.read(iprot); + isset_parts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->deleteData); + this->__isset.deleteData = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->ifExists); + this->__isset.ifExists = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->ignoreProtection); + this->__isset.ignoreProtection = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->environmentContext.read(iprot); + this->__isset.environmentContext = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 8: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->needResult); + this->__isset.needResult = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_dbName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tblName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_parts) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t DropPartitionsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("DropPartitionsRequest"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tblName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("parts", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->parts.write(oprot); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.deleteData) { + xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool(this->deleteData); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.ifExists) { + xfer += oprot->writeFieldBegin("ifExists", ::apache::thrift::protocol::T_BOOL, 5); + xfer += oprot->writeBool(this->ifExists); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.ignoreProtection) { + xfer += oprot->writeFieldBegin("ignoreProtection", ::apache::thrift::protocol::T_BOOL, 6); + xfer += oprot->writeBool(this->ignoreProtection); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.environmentContext) { + xfer += oprot->writeFieldBegin("environmentContext", ::apache::thrift::protocol::T_STRUCT, 7); + xfer += this->environmentContext.write(oprot); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.needResult) { + xfer += oprot->writeFieldBegin("needResult", ::apache::thrift::protocol::T_BOOL, 8); + xfer += oprot->writeBool(this->needResult); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(DropPartitionsRequest &a, DropPartitionsRequest &b) { + using ::std::swap; + swap(a.dbName, b.dbName); + swap(a.tblName, b.tblName); + swap(a.parts, b.parts); + swap(a.deleteData, b.deleteData); + swap(a.ifExists, b.ifExists); + swap(a.ignoreProtection, b.ignoreProtection); + swap(a.environmentContext, b.environmentContext); + swap(a.needResult, b.needResult); + swap(a.__isset, b.__isset); +} + +DropPartitionsRequest::DropPartitionsRequest(const DropPartitionsRequest& other525) { + dbName = other525.dbName; + tblName = other525.tblName; + parts = other525.parts; + deleteData = other525.deleteData; + ifExists = other525.ifExists; + ignoreProtection = other525.ignoreProtection; + environmentContext = other525.environmentContext; + needResult = other525.needResult; + __isset = other525.__isset; +} +DropPartitionsRequest& DropPartitionsRequest::operator=(const DropPartitionsRequest& other526) { + dbName = other526.dbName; + tblName = other526.tblName; + parts = other526.parts; + deleteData = other526.deleteData; + ifExists = other526.ifExists; + ignoreProtection = other526.ignoreProtection; + environmentContext = other526.environmentContext; + needResult = other526.needResult; + __isset = other526.__isset; + return *this; +} +void DropPartitionsRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "DropPartitionsRequest("; + out << "dbName=" << to_string(dbName); + out << ", " << "tblName=" << to_string(tblName); + out << ", " << "parts=" << to_string(parts); + out << ", " << "deleteData="; (__isset.deleteData ? (out << to_string(deleteData)) : (out << "")); + out << ", " << "ifExists="; (__isset.ifExists ? (out << to_string(ifExists)) : (out << "")); + out << ", " << "ignoreProtection="; (__isset.ignoreProtection ? (out << to_string(ignoreProtection)) : (out << "")); + out << ", " << "environmentContext="; (__isset.environmentContext ? (out << to_string(environmentContext)) : (out << "")); + out << ", " << "needResult="; (__isset.needResult ? (out << to_string(needResult)) : (out << "")); + out << ")"; +} + + +PartitionValuesRequest::~PartitionValuesRequest() noexcept { +} + + +void PartitionValuesRequest::__set_dbName(const std::string& val) { + this->dbName = val; +} + +void PartitionValuesRequest::__set_tblName(const std::string& val) { + this->tblName = val; +} + +void PartitionValuesRequest::__set_partitionKeys(const std::vector & val) { + this->partitionKeys = val; +} + +void PartitionValuesRequest::__set_applyDistinct(const bool val) { + this->applyDistinct = val; +__isset.applyDistinct = true; +} + +void PartitionValuesRequest::__set_filter(const std::string& val) { + this->filter = val; +__isset.filter = true; +} + +void PartitionValuesRequest::__set_partitionOrder(const std::vector & val) { + this->partitionOrder = val; +__isset.partitionOrder = true; +} + +void PartitionValuesRequest::__set_ascending(const bool val) { + this->ascending = val; +__isset.ascending = true; +} + +void PartitionValuesRequest::__set_maxParts(const int64_t val) { + this->maxParts = val; +__isset.maxParts = true; +} +std::ostream& operator<<(std::ostream& out, const PartitionValuesRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PartitionValuesRequest::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_dbName = false; + bool isset_tblName = false; + bool isset_partitionKeys = 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_STRING) { + xfer += iprot->readString(this->dbName); + isset_dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tblName); + isset_tblName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->partitionKeys.clear(); + uint32_t _size527; + ::apache::thrift::protocol::TType _etype530; + xfer += iprot->readListBegin(_etype530, _size527); + this->partitionKeys.resize(_size527); + uint32_t _i531; + for (_i531 = 0; _i531 < _size527; ++_i531) + { + xfer += this->partitionKeys[_i531].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_partitionKeys = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->applyDistinct); + this->__isset.applyDistinct = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->filter); + this->__isset.filter = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->partitionOrder.clear(); + uint32_t _size532; + ::apache::thrift::protocol::TType _etype535; + xfer += iprot->readListBegin(_etype535, _size532); + this->partitionOrder.resize(_size532); + uint32_t _i536; + for (_i536 = 0; _i536 < _size532; ++_i536) + { + xfer += this->partitionOrder[_i536].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.partitionOrder = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->ascending); + this->__isset.ascending = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 8: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->maxParts); + this->__isset.maxParts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_dbName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tblName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_partitionKeys) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t PartitionValuesRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PartitionValuesRequest"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tblName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("partitionKeys", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitionKeys.size())); + std::vector ::const_iterator _iter537; + for (_iter537 = this->partitionKeys.begin(); _iter537 != this->partitionKeys.end(); ++_iter537) + { + xfer += (*_iter537).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + if (this->__isset.applyDistinct) { + xfer += oprot->writeFieldBegin("applyDistinct", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool(this->applyDistinct); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.filter) { + xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->filter); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.partitionOrder) { + xfer += oprot->writeFieldBegin("partitionOrder", ::apache::thrift::protocol::T_LIST, 6); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitionOrder.size())); + std::vector ::const_iterator _iter538; + for (_iter538 = this->partitionOrder.begin(); _iter538 != this->partitionOrder.end(); ++_iter538) + { + xfer += (*_iter538).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.ascending) { + xfer += oprot->writeFieldBegin("ascending", ::apache::thrift::protocol::T_BOOL, 7); + xfer += oprot->writeBool(this->ascending); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.maxParts) { + xfer += oprot->writeFieldBegin("maxParts", ::apache::thrift::protocol::T_I64, 8); + xfer += oprot->writeI64(this->maxParts); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PartitionValuesRequest &a, PartitionValuesRequest &b) { + using ::std::swap; + swap(a.dbName, b.dbName); + swap(a.tblName, b.tblName); + swap(a.partitionKeys, b.partitionKeys); + swap(a.applyDistinct, b.applyDistinct); + swap(a.filter, b.filter); + swap(a.partitionOrder, b.partitionOrder); + swap(a.ascending, b.ascending); + swap(a.maxParts, b.maxParts); + swap(a.__isset, b.__isset); +} + +PartitionValuesRequest::PartitionValuesRequest(const PartitionValuesRequest& other539) { + dbName = other539.dbName; + tblName = other539.tblName; + partitionKeys = other539.partitionKeys; + applyDistinct = other539.applyDistinct; + filter = other539.filter; + partitionOrder = other539.partitionOrder; + ascending = other539.ascending; + maxParts = other539.maxParts; + __isset = other539.__isset; +} +PartitionValuesRequest& PartitionValuesRequest::operator=(const PartitionValuesRequest& other540) { + dbName = other540.dbName; + tblName = other540.tblName; + partitionKeys = other540.partitionKeys; + applyDistinct = other540.applyDistinct; + filter = other540.filter; + partitionOrder = other540.partitionOrder; + ascending = other540.ascending; + maxParts = other540.maxParts; + __isset = other540.__isset; + return *this; +} +void PartitionValuesRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PartitionValuesRequest("; + out << "dbName=" << to_string(dbName); + out << ", " << "tblName=" << to_string(tblName); + out << ", " << "partitionKeys=" << to_string(partitionKeys); + out << ", " << "applyDistinct="; (__isset.applyDistinct ? (out << to_string(applyDistinct)) : (out << "")); + out << ", " << "filter="; (__isset.filter ? (out << to_string(filter)) : (out << "")); + out << ", " << "partitionOrder="; (__isset.partitionOrder ? (out << to_string(partitionOrder)) : (out << "")); + out << ", " << "ascending="; (__isset.ascending ? (out << to_string(ascending)) : (out << "")); + out << ", " << "maxParts="; (__isset.maxParts ? (out << to_string(maxParts)) : (out << "")); + out << ")"; +} + + +PartitionValuesRow::~PartitionValuesRow() noexcept { +} + + +void PartitionValuesRow::__set_row(const std::vector & val) { + this->row = val; +} +std::ostream& operator<<(std::ostream& out, const PartitionValuesRow& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PartitionValuesRow::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_row = 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_LIST) { + { + this->row.clear(); + uint32_t _size541; + ::apache::thrift::protocol::TType _etype544; + xfer += iprot->readListBegin(_etype544, _size541); + this->row.resize(_size541); + uint32_t _i545; + for (_i545 = 0; _i545 < _size541; ++_i545) + { + xfer += iprot->readString(this->row[_i545]); + } + xfer += iprot->readListEnd(); + } + isset_row = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_row) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t PartitionValuesRow::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PartitionValuesRow"); + + xfer += oprot->writeFieldBegin("row", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->row.size())); + std::vector ::const_iterator _iter546; + for (_iter546 = this->row.begin(); _iter546 != this->row.end(); ++_iter546) + { + xfer += oprot->writeString((*_iter546)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PartitionValuesRow &a, PartitionValuesRow &b) { + using ::std::swap; + swap(a.row, b.row); +} + +PartitionValuesRow::PartitionValuesRow(const PartitionValuesRow& other547) { + row = other547.row; +} +PartitionValuesRow& PartitionValuesRow::operator=(const PartitionValuesRow& other548) { + row = other548.row; + return *this; +} +void PartitionValuesRow::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PartitionValuesRow("; + out << "row=" << to_string(row); + out << ")"; +} + + +PartitionValuesResponse::~PartitionValuesResponse() noexcept { +} + + +void PartitionValuesResponse::__set_partitionValues(const std::vector & val) { + this->partitionValues = val; +} +std::ostream& operator<<(std::ostream& out, const PartitionValuesResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PartitionValuesResponse::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_partitionValues = 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_LIST) { + { + this->partitionValues.clear(); + uint32_t _size549; + ::apache::thrift::protocol::TType _etype552; + xfer += iprot->readListBegin(_etype552, _size549); + this->partitionValues.resize(_size549); + uint32_t _i553; + for (_i553 = 0; _i553 < _size549; ++_i553) + { + xfer += this->partitionValues[_i553].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_partitionValues = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_partitionValues) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t PartitionValuesResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PartitionValuesResponse"); + + xfer += oprot->writeFieldBegin("partitionValues", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitionValues.size())); + std::vector ::const_iterator _iter554; + for (_iter554 = this->partitionValues.begin(); _iter554 != this->partitionValues.end(); ++_iter554) + { + xfer += (*_iter554).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PartitionValuesResponse &a, PartitionValuesResponse &b) { + using ::std::swap; + swap(a.partitionValues, b.partitionValues); +} + +PartitionValuesResponse::PartitionValuesResponse(const PartitionValuesResponse& other555) { + partitionValues = other555.partitionValues; +} +PartitionValuesResponse& PartitionValuesResponse::operator=(const PartitionValuesResponse& other556) { + partitionValues = other556.partitionValues; + return *this; +} +void PartitionValuesResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PartitionValuesResponse("; + out << "partitionValues=" << to_string(partitionValues); + out << ")"; +} + + +ResourceUri::~ResourceUri() noexcept { +} + + +void ResourceUri::__set_resourceType(const ResourceType::type val) { + this->resourceType = val; +} + +void ResourceUri::__set_uri(const std::string& val) { + this->uri = val; +} +std::ostream& operator<<(std::ostream& out, const ResourceUri& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t ResourceUri::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast557; + xfer += iprot->readI32(ecast557); + this->resourceType = (ResourceType::type)ecast557; + this->__isset.resourceType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->uri); + this->__isset.uri = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ResourceUri::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ResourceUri"); + + xfer += oprot->writeFieldBegin("resourceType", ::apache::thrift::protocol::T_I32, 1); + xfer += oprot->writeI32((int32_t)this->resourceType); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("uri", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->uri); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ResourceUri &a, ResourceUri &b) { + using ::std::swap; + swap(a.resourceType, b.resourceType); + swap(a.uri, b.uri); + swap(a.__isset, b.__isset); +} + +ResourceUri::ResourceUri(const ResourceUri& other558) { + resourceType = other558.resourceType; + uri = other558.uri; + __isset = other558.__isset; +} +ResourceUri& ResourceUri::operator=(const ResourceUri& other559) { + resourceType = other559.resourceType; + uri = other559.uri; + __isset = other559.__isset; + return *this; +} +void ResourceUri::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "ResourceUri("; + out << "resourceType=" << to_string(resourceType); + out << ", " << "uri=" << to_string(uri); + out << ")"; +} + + +Function::~Function() noexcept { +} + + +void Function::__set_functionName(const std::string& val) { + this->functionName = val; +} + +void Function::__set_dbName(const std::string& val) { + this->dbName = val; +} + +void Function::__set_className(const std::string& val) { + this->className = val; +} + +void Function::__set_ownerName(const std::string& val) { + this->ownerName = val; +} + +void Function::__set_ownerType(const PrincipalType::type val) { + this->ownerType = val; +} + +void Function::__set_createTime(const int32_t val) { + this->createTime = val; +} + +void Function::__set_functionType(const FunctionType::type val) { + this->functionType = val; +} + +void Function::__set_resourceUris(const std::vector & val) { + this->resourceUris = val; +} +std::ostream& operator<<(std::ostream& out, const Function& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t Function::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->functionName); + this->__isset.functionName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->className); + this->__isset.className = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->ownerName); + this->__isset.ownerName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast560; + xfer += iprot->readI32(ecast560); + this->ownerType = (PrincipalType::type)ecast560; + this->__isset.ownerType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->createTime); + this->__isset.createTime = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast561; + xfer += iprot->readI32(ecast561); + this->functionType = (FunctionType::type)ecast561; + this->__isset.functionType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 8: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->resourceUris.clear(); + uint32_t _size562; + ::apache::thrift::protocol::TType _etype565; + xfer += iprot->readListBegin(_etype565, _size562); + this->resourceUris.resize(_size562); + uint32_t _i566; + for (_i566 = 0; _i566 < _size562; ++_i566) + { + xfer += this->resourceUris[_i566].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.resourceUris = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t Function::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("Function"); + + xfer += oprot->writeFieldBegin("functionName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->functionName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("className", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->className); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("ownerName", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->ownerName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("ownerType", ::apache::thrift::protocol::T_I32, 5); + xfer += oprot->writeI32((int32_t)this->ownerType); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 6); + xfer += oprot->writeI32(this->createTime); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("functionType", ::apache::thrift::protocol::T_I32, 7); + xfer += oprot->writeI32((int32_t)this->functionType); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("resourceUris", ::apache::thrift::protocol::T_LIST, 8); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->resourceUris.size())); + std::vector ::const_iterator _iter567; + for (_iter567 = this->resourceUris.begin(); _iter567 != this->resourceUris.end(); ++_iter567) + { + xfer += (*_iter567).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(Function &a, Function &b) { + using ::std::swap; + swap(a.functionName, b.functionName); + swap(a.dbName, b.dbName); + swap(a.className, b.className); + swap(a.ownerName, b.ownerName); + swap(a.ownerType, b.ownerType); + swap(a.createTime, b.createTime); + swap(a.functionType, b.functionType); + swap(a.resourceUris, b.resourceUris); + swap(a.__isset, b.__isset); +} + +Function::Function(const Function& other568) { + functionName = other568.functionName; + dbName = other568.dbName; + className = other568.className; + ownerName = other568.ownerName; + ownerType = other568.ownerType; + createTime = other568.createTime; + functionType = other568.functionType; + resourceUris = other568.resourceUris; + __isset = other568.__isset; +} +Function& Function::operator=(const Function& other569) { + functionName = other569.functionName; + dbName = other569.dbName; + className = other569.className; + ownerName = other569.ownerName; + ownerType = other569.ownerType; + createTime = other569.createTime; + functionType = other569.functionType; + resourceUris = other569.resourceUris; + __isset = other569.__isset; + return *this; +} +void Function::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "Function("; + out << "functionName=" << to_string(functionName); + out << ", " << "dbName=" << to_string(dbName); + out << ", " << "className=" << to_string(className); + out << ", " << "ownerName=" << to_string(ownerName); + out << ", " << "ownerType=" << to_string(ownerType); + out << ", " << "createTime=" << to_string(createTime); + out << ", " << "functionType=" << to_string(functionType); + out << ", " << "resourceUris=" << to_string(resourceUris); + out << ")"; +} + + +TxnInfo::~TxnInfo() noexcept { +} + + +void TxnInfo::__set_id(const int64_t val) { + this->id = val; +} + +void TxnInfo::__set_state(const TxnState::type val) { + this->state = val; +} + +void TxnInfo::__set_user(const std::string& val) { + this->user = val; +} + +void TxnInfo::__set_hostname(const std::string& val) { + this->hostname = val; +} + +void TxnInfo::__set_agentInfo(const std::string& val) { + this->agentInfo = val; +__isset.agentInfo = true; +} + +void TxnInfo::__set_heartbeatCount(const int32_t val) { + this->heartbeatCount = val; +__isset.heartbeatCount = true; +} + +void TxnInfo::__set_metaInfo(const std::string& val) { + this->metaInfo = val; +__isset.metaInfo = true; +} + +void TxnInfo::__set_startedTime(const int64_t val) { + this->startedTime = val; +__isset.startedTime = true; +} + +void TxnInfo::__set_lastHeartbeatTime(const int64_t val) { + this->lastHeartbeatTime = val; +__isset.lastHeartbeatTime = true; +} +std::ostream& operator<<(std::ostream& out, const TxnInfo& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t TxnInfo::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_id = false; + bool isset_state = false; + bool isset_user = false; + bool isset_hostname = 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_I64) { + xfer += iprot->readI64(this->id); + isset_id = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast570; + xfer += iprot->readI32(ecast570); + this->state = (TxnState::type)ecast570; + isset_state = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->user); + isset_user = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->hostname); + isset_hostname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->agentInfo); + this->__isset.agentInfo = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->heartbeatCount); + this->__isset.heartbeatCount = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->metaInfo); + this->__isset.metaInfo = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 8: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->startedTime); + this->__isset.startedTime = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 9: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->lastHeartbeatTime); + this->__isset.lastHeartbeatTime = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_id) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_state) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_user) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_hostname) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t TxnInfo::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("TxnInfo"); + + xfer += oprot->writeFieldBegin("id", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->id); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("state", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32((int32_t)this->state); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("user", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->user); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("hostname", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->hostname); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.agentInfo) { + xfer += oprot->writeFieldBegin("agentInfo", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->agentInfo); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.heartbeatCount) { + xfer += oprot->writeFieldBegin("heartbeatCount", ::apache::thrift::protocol::T_I32, 6); + xfer += oprot->writeI32(this->heartbeatCount); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.metaInfo) { + xfer += oprot->writeFieldBegin("metaInfo", ::apache::thrift::protocol::T_STRING, 7); + xfer += oprot->writeString(this->metaInfo); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.startedTime) { + xfer += oprot->writeFieldBegin("startedTime", ::apache::thrift::protocol::T_I64, 8); + xfer += oprot->writeI64(this->startedTime); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.lastHeartbeatTime) { + xfer += oprot->writeFieldBegin("lastHeartbeatTime", ::apache::thrift::protocol::T_I64, 9); + xfer += oprot->writeI64(this->lastHeartbeatTime); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(TxnInfo &a, TxnInfo &b) { + using ::std::swap; + swap(a.id, b.id); + swap(a.state, b.state); + swap(a.user, b.user); + swap(a.hostname, b.hostname); + swap(a.agentInfo, b.agentInfo); + swap(a.heartbeatCount, b.heartbeatCount); + swap(a.metaInfo, b.metaInfo); + swap(a.startedTime, b.startedTime); + swap(a.lastHeartbeatTime, b.lastHeartbeatTime); + swap(a.__isset, b.__isset); +} + +TxnInfo::TxnInfo(const TxnInfo& other571) { + id = other571.id; + state = other571.state; + user = other571.user; + hostname = other571.hostname; + agentInfo = other571.agentInfo; + heartbeatCount = other571.heartbeatCount; + metaInfo = other571.metaInfo; + startedTime = other571.startedTime; + lastHeartbeatTime = other571.lastHeartbeatTime; + __isset = other571.__isset; +} +TxnInfo& TxnInfo::operator=(const TxnInfo& other572) { + id = other572.id; + state = other572.state; + user = other572.user; + hostname = other572.hostname; + agentInfo = other572.agentInfo; + heartbeatCount = other572.heartbeatCount; + metaInfo = other572.metaInfo; + startedTime = other572.startedTime; + lastHeartbeatTime = other572.lastHeartbeatTime; + __isset = other572.__isset; + return *this; +} +void TxnInfo::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "TxnInfo("; + out << "id=" << to_string(id); + out << ", " << "state=" << to_string(state); + out << ", " << "user=" << to_string(user); + out << ", " << "hostname=" << to_string(hostname); + out << ", " << "agentInfo="; (__isset.agentInfo ? (out << to_string(agentInfo)) : (out << "")); + out << ", " << "heartbeatCount="; (__isset.heartbeatCount ? (out << to_string(heartbeatCount)) : (out << "")); + out << ", " << "metaInfo="; (__isset.metaInfo ? (out << to_string(metaInfo)) : (out << "")); + out << ", " << "startedTime="; (__isset.startedTime ? (out << to_string(startedTime)) : (out << "")); + out << ", " << "lastHeartbeatTime="; (__isset.lastHeartbeatTime ? (out << to_string(lastHeartbeatTime)) : (out << "")); + out << ")"; +} + + +GetOpenTxnsInfoResponse::~GetOpenTxnsInfoResponse() noexcept { +} + + +void GetOpenTxnsInfoResponse::__set_txn_high_water_mark(const int64_t val) { + this->txn_high_water_mark = val; +} + +void GetOpenTxnsInfoResponse::__set_open_txns(const std::vector & val) { + this->open_txns = val; +} +std::ostream& operator<<(std::ostream& out, const GetOpenTxnsInfoResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GetOpenTxnsInfoResponse::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_txn_high_water_mark = false; + bool isset_open_txns = 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_I64) { + xfer += iprot->readI64(this->txn_high_water_mark); + isset_txn_high_water_mark = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->open_txns.clear(); + uint32_t _size573; + ::apache::thrift::protocol::TType _etype576; + xfer += iprot->readListBegin(_etype576, _size573); + this->open_txns.resize(_size573); + uint32_t _i577; + for (_i577 = 0; _i577 < _size573; ++_i577) + { + xfer += this->open_txns[_i577].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_open_txns = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_txn_high_water_mark) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_open_txns) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t GetOpenTxnsInfoResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GetOpenTxnsInfoResponse"); + + xfer += oprot->writeFieldBegin("txn_high_water_mark", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->txn_high_water_mark); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("open_txns", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->open_txns.size())); + std::vector ::const_iterator _iter578; + for (_iter578 = this->open_txns.begin(); _iter578 != this->open_txns.end(); ++_iter578) + { + xfer += (*_iter578).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GetOpenTxnsInfoResponse &a, GetOpenTxnsInfoResponse &b) { + using ::std::swap; + swap(a.txn_high_water_mark, b.txn_high_water_mark); + swap(a.open_txns, b.open_txns); +} + +GetOpenTxnsInfoResponse::GetOpenTxnsInfoResponse(const GetOpenTxnsInfoResponse& other579) { + txn_high_water_mark = other579.txn_high_water_mark; + open_txns = other579.open_txns; +} +GetOpenTxnsInfoResponse& GetOpenTxnsInfoResponse::operator=(const GetOpenTxnsInfoResponse& other580) { + txn_high_water_mark = other580.txn_high_water_mark; + open_txns = other580.open_txns; + return *this; +} +void GetOpenTxnsInfoResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GetOpenTxnsInfoResponse("; + out << "txn_high_water_mark=" << to_string(txn_high_water_mark); + out << ", " << "open_txns=" << to_string(open_txns); + out << ")"; +} + + +GetOpenTxnsResponse::~GetOpenTxnsResponse() noexcept { +} + + +void GetOpenTxnsResponse::__set_txn_high_water_mark(const int64_t val) { + this->txn_high_water_mark = val; +} + +void GetOpenTxnsResponse::__set_open_txns(const std::vector & val) { + this->open_txns = val; +} + +void GetOpenTxnsResponse::__set_min_open_txn(const int64_t val) { + this->min_open_txn = val; +__isset.min_open_txn = true; +} + +void GetOpenTxnsResponse::__set_abortedBits(const std::string& val) { + this->abortedBits = val; +} +std::ostream& operator<<(std::ostream& out, const GetOpenTxnsResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GetOpenTxnsResponse::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_txn_high_water_mark = false; + bool isset_open_txns = false; + bool isset_abortedBits = 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_I64) { + xfer += iprot->readI64(this->txn_high_water_mark); + isset_txn_high_water_mark = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->open_txns.clear(); + uint32_t _size581; + ::apache::thrift::protocol::TType _etype584; + xfer += iprot->readListBegin(_etype584, _size581); + this->open_txns.resize(_size581); + uint32_t _i585; + for (_i585 = 0; _i585 < _size581; ++_i585) + { + xfer += iprot->readI64(this->open_txns[_i585]); + } + xfer += iprot->readListEnd(); + } + isset_open_txns = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->min_open_txn); + this->__isset.min_open_txn = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readBinary(this->abortedBits); + isset_abortedBits = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_txn_high_water_mark) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_open_txns) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_abortedBits) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t GetOpenTxnsResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GetOpenTxnsResponse"); + + xfer += oprot->writeFieldBegin("txn_high_water_mark", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->txn_high_water_mark); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("open_txns", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->open_txns.size())); + std::vector ::const_iterator _iter586; + for (_iter586 = this->open_txns.begin(); _iter586 != this->open_txns.end(); ++_iter586) + { + xfer += oprot->writeI64((*_iter586)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + if (this->__isset.min_open_txn) { + xfer += oprot->writeFieldBegin("min_open_txn", ::apache::thrift::protocol::T_I64, 3); + xfer += oprot->writeI64(this->min_open_txn); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldBegin("abortedBits", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeBinary(this->abortedBits); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GetOpenTxnsResponse &a, GetOpenTxnsResponse &b) { + using ::std::swap; + swap(a.txn_high_water_mark, b.txn_high_water_mark); + swap(a.open_txns, b.open_txns); + swap(a.min_open_txn, b.min_open_txn); + swap(a.abortedBits, b.abortedBits); + swap(a.__isset, b.__isset); +} + +GetOpenTxnsResponse::GetOpenTxnsResponse(const GetOpenTxnsResponse& other587) { + txn_high_water_mark = other587.txn_high_water_mark; + open_txns = other587.open_txns; + min_open_txn = other587.min_open_txn; + abortedBits = other587.abortedBits; + __isset = other587.__isset; +} +GetOpenTxnsResponse& GetOpenTxnsResponse::operator=(const GetOpenTxnsResponse& other588) { + txn_high_water_mark = other588.txn_high_water_mark; + open_txns = other588.open_txns; + min_open_txn = other588.min_open_txn; + abortedBits = other588.abortedBits; + __isset = other588.__isset; + return *this; +} +void GetOpenTxnsResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GetOpenTxnsResponse("; + out << "txn_high_water_mark=" << to_string(txn_high_water_mark); + out << ", " << "open_txns=" << to_string(open_txns); + out << ", " << "min_open_txn="; (__isset.min_open_txn ? (out << to_string(min_open_txn)) : (out << "")); + out << ", " << "abortedBits=" << to_string(abortedBits); + out << ")"; +} + + +OpenTxnRequest::~OpenTxnRequest() noexcept { +} + + +void OpenTxnRequest::__set_num_txns(const int32_t val) { + this->num_txns = val; +} + +void OpenTxnRequest::__set_user(const std::string& val) { + this->user = val; +} + +void OpenTxnRequest::__set_hostname(const std::string& val) { + this->hostname = val; +} + +void OpenTxnRequest::__set_agentInfo(const std::string& val) { + this->agentInfo = val; +__isset.agentInfo = true; +} +std::ostream& operator<<(std::ostream& out, const OpenTxnRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t OpenTxnRequest::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_num_txns = false; + bool isset_user = false; + bool isset_hostname = 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->num_txns); + isset_num_txns = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->user); + isset_user = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->hostname); + isset_hostname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->agentInfo); + this->__isset.agentInfo = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_num_txns) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_user) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_hostname) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t OpenTxnRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("OpenTxnRequest"); + + xfer += oprot->writeFieldBegin("num_txns", ::apache::thrift::protocol::T_I32, 1); + xfer += oprot->writeI32(this->num_txns); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("user", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->user); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("hostname", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->hostname); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.agentInfo) { + xfer += oprot->writeFieldBegin("agentInfo", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->agentInfo); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(OpenTxnRequest &a, OpenTxnRequest &b) { + using ::std::swap; + swap(a.num_txns, b.num_txns); + swap(a.user, b.user); + swap(a.hostname, b.hostname); + swap(a.agentInfo, b.agentInfo); + swap(a.__isset, b.__isset); +} + +OpenTxnRequest::OpenTxnRequest(const OpenTxnRequest& other589) { + num_txns = other589.num_txns; + user = other589.user; + hostname = other589.hostname; + agentInfo = other589.agentInfo; + __isset = other589.__isset; +} +OpenTxnRequest& OpenTxnRequest::operator=(const OpenTxnRequest& other590) { + num_txns = other590.num_txns; + user = other590.user; + hostname = other590.hostname; + agentInfo = other590.agentInfo; + __isset = other590.__isset; + return *this; +} +void OpenTxnRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "OpenTxnRequest("; + out << "num_txns=" << to_string(num_txns); + out << ", " << "user=" << to_string(user); + out << ", " << "hostname=" << to_string(hostname); + out << ", " << "agentInfo="; (__isset.agentInfo ? (out << to_string(agentInfo)) : (out << "")); + out << ")"; +} + + +OpenTxnsResponse::~OpenTxnsResponse() noexcept { +} + + +void OpenTxnsResponse::__set_txn_ids(const std::vector & val) { + this->txn_ids = val; +} +std::ostream& operator<<(std::ostream& out, const OpenTxnsResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t OpenTxnsResponse::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_txn_ids = 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_LIST) { + { + this->txn_ids.clear(); + uint32_t _size591; + ::apache::thrift::protocol::TType _etype594; + xfer += iprot->readListBegin(_etype594, _size591); + this->txn_ids.resize(_size591); + uint32_t _i595; + for (_i595 = 0; _i595 < _size591; ++_i595) + { + xfer += iprot->readI64(this->txn_ids[_i595]); + } + xfer += iprot->readListEnd(); + } + isset_txn_ids = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_txn_ids) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t OpenTxnsResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("OpenTxnsResponse"); + + xfer += oprot->writeFieldBegin("txn_ids", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->txn_ids.size())); + std::vector ::const_iterator _iter596; + for (_iter596 = this->txn_ids.begin(); _iter596 != this->txn_ids.end(); ++_iter596) + { + xfer += oprot->writeI64((*_iter596)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(OpenTxnsResponse &a, OpenTxnsResponse &b) { + using ::std::swap; + swap(a.txn_ids, b.txn_ids); +} + +OpenTxnsResponse::OpenTxnsResponse(const OpenTxnsResponse& other597) { + txn_ids = other597.txn_ids; +} +OpenTxnsResponse& OpenTxnsResponse::operator=(const OpenTxnsResponse& other598) { + txn_ids = other598.txn_ids; + return *this; +} +void OpenTxnsResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "OpenTxnsResponse("; + out << "txn_ids=" << to_string(txn_ids); + out << ")"; +} + + +AbortTxnRequest::~AbortTxnRequest() noexcept { +} + + +void AbortTxnRequest::__set_txnid(const int64_t val) { + this->txnid = val; +} +std::ostream& operator<<(std::ostream& out, const AbortTxnRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t AbortTxnRequest::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_txnid = 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_I64) { + xfer += iprot->readI64(this->txnid); + isset_txnid = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_txnid) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t AbortTxnRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("AbortTxnRequest"); + + xfer += oprot->writeFieldBegin("txnid", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->txnid); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(AbortTxnRequest &a, AbortTxnRequest &b) { + using ::std::swap; + swap(a.txnid, b.txnid); +} + +AbortTxnRequest::AbortTxnRequest(const AbortTxnRequest& other599) { + txnid = other599.txnid; +} +AbortTxnRequest& AbortTxnRequest::operator=(const AbortTxnRequest& other600) { + txnid = other600.txnid; + return *this; +} +void AbortTxnRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "AbortTxnRequest("; + out << "txnid=" << to_string(txnid); + out << ")"; +} + + +AbortTxnsRequest::~AbortTxnsRequest() noexcept { +} + + +void AbortTxnsRequest::__set_txn_ids(const std::vector & val) { + this->txn_ids = val; +} +std::ostream& operator<<(std::ostream& out, const AbortTxnsRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t AbortTxnsRequest::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_txn_ids = 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_LIST) { + { + this->txn_ids.clear(); + uint32_t _size601; + ::apache::thrift::protocol::TType _etype604; + xfer += iprot->readListBegin(_etype604, _size601); + this->txn_ids.resize(_size601); + uint32_t _i605; + for (_i605 = 0; _i605 < _size601; ++_i605) + { + xfer += iprot->readI64(this->txn_ids[_i605]); + } + xfer += iprot->readListEnd(); + } + isset_txn_ids = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_txn_ids) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t AbortTxnsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("AbortTxnsRequest"); + + xfer += oprot->writeFieldBegin("txn_ids", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->txn_ids.size())); + std::vector ::const_iterator _iter606; + for (_iter606 = this->txn_ids.begin(); _iter606 != this->txn_ids.end(); ++_iter606) + { + xfer += oprot->writeI64((*_iter606)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(AbortTxnsRequest &a, AbortTxnsRequest &b) { + using ::std::swap; + swap(a.txn_ids, b.txn_ids); +} + +AbortTxnsRequest::AbortTxnsRequest(const AbortTxnsRequest& other607) { + txn_ids = other607.txn_ids; +} +AbortTxnsRequest& AbortTxnsRequest::operator=(const AbortTxnsRequest& other608) { + txn_ids = other608.txn_ids; + return *this; +} +void AbortTxnsRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "AbortTxnsRequest("; + out << "txn_ids=" << to_string(txn_ids); + out << ")"; +} + + +CommitTxnRequest::~CommitTxnRequest() noexcept { +} + + +void CommitTxnRequest::__set_txnid(const int64_t val) { + this->txnid = val; +} +std::ostream& operator<<(std::ostream& out, const CommitTxnRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t CommitTxnRequest::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_txnid = 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_I64) { + xfer += iprot->readI64(this->txnid); + isset_txnid = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_txnid) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t CommitTxnRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("CommitTxnRequest"); + + xfer += oprot->writeFieldBegin("txnid", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->txnid); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(CommitTxnRequest &a, CommitTxnRequest &b) { + using ::std::swap; + swap(a.txnid, b.txnid); +} + +CommitTxnRequest::CommitTxnRequest(const CommitTxnRequest& other609) { + txnid = other609.txnid; +} +CommitTxnRequest& CommitTxnRequest::operator=(const CommitTxnRequest& other610) { + txnid = other610.txnid; + return *this; +} +void CommitTxnRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "CommitTxnRequest("; + out << "txnid=" << to_string(txnid); + out << ")"; +} + + +LockComponent::~LockComponent() noexcept { +} + + +void LockComponent::__set_type(const LockType::type val) { + this->type = val; +} + +void LockComponent::__set_level(const LockLevel::type val) { + this->level = val; +} + +void LockComponent::__set_dbname(const std::string& val) { + this->dbname = val; +} + +void LockComponent::__set_tablename(const std::string& val) { + this->tablename = val; +__isset.tablename = true; +} + +void LockComponent::__set_partitionname(const std::string& val) { + this->partitionname = val; +__isset.partitionname = true; +} + +void LockComponent::__set_operationType(const DataOperationType::type val) { + this->operationType = val; +__isset.operationType = true; +} + +void LockComponent::__set_isAcid(const bool val) { + this->isAcid = val; +__isset.isAcid = true; +} + +void LockComponent::__set_isDynamicPartitionWrite(const bool val) { + this->isDynamicPartitionWrite = val; +__isset.isDynamicPartitionWrite = true; +} +std::ostream& operator<<(std::ostream& out, const LockComponent& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t LockComponent::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_type = false; + bool isset_level = false; + bool isset_dbname = 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) { + int32_t ecast611; + xfer += iprot->readI32(ecast611); + this->type = (LockType::type)ecast611; + isset_type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast612; + xfer += iprot->readI32(ecast612); + this->level = (LockLevel::type)ecast612; + isset_level = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbname); + isset_dbname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tablename); + this->__isset.tablename = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->partitionname); + this->__isset.partitionname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast613; + xfer += iprot->readI32(ecast613); + this->operationType = (DataOperationType::type)ecast613; + this->__isset.operationType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->isAcid); + this->__isset.isAcid = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 8: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->isDynamicPartitionWrite); + this->__isset.isDynamicPartitionWrite = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_type) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_level) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_dbname) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t LockComponent::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("LockComponent"); + + xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_I32, 1); + xfer += oprot->writeI32((int32_t)this->type); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("level", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32((int32_t)this->level); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->dbname); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.tablename) { + xfer += oprot->writeFieldBegin("tablename", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->tablename); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.partitionname) { + xfer += oprot->writeFieldBegin("partitionname", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->partitionname); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.operationType) { + xfer += oprot->writeFieldBegin("operationType", ::apache::thrift::protocol::T_I32, 6); + xfer += oprot->writeI32((int32_t)this->operationType); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.isAcid) { + xfer += oprot->writeFieldBegin("isAcid", ::apache::thrift::protocol::T_BOOL, 7); + xfer += oprot->writeBool(this->isAcid); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.isDynamicPartitionWrite) { + xfer += oprot->writeFieldBegin("isDynamicPartitionWrite", ::apache::thrift::protocol::T_BOOL, 8); + xfer += oprot->writeBool(this->isDynamicPartitionWrite); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(LockComponent &a, LockComponent &b) { + using ::std::swap; + swap(a.type, b.type); + swap(a.level, b.level); + swap(a.dbname, b.dbname); + swap(a.tablename, b.tablename); + swap(a.partitionname, b.partitionname); + swap(a.operationType, b.operationType); + swap(a.isAcid, b.isAcid); + swap(a.isDynamicPartitionWrite, b.isDynamicPartitionWrite); + swap(a.__isset, b.__isset); +} + +LockComponent::LockComponent(const LockComponent& other614) { + type = other614.type; + level = other614.level; + dbname = other614.dbname; + tablename = other614.tablename; + partitionname = other614.partitionname; + operationType = other614.operationType; + isAcid = other614.isAcid; + isDynamicPartitionWrite = other614.isDynamicPartitionWrite; + __isset = other614.__isset; +} +LockComponent& LockComponent::operator=(const LockComponent& other615) { + type = other615.type; + level = other615.level; + dbname = other615.dbname; + tablename = other615.tablename; + partitionname = other615.partitionname; + operationType = other615.operationType; + isAcid = other615.isAcid; + isDynamicPartitionWrite = other615.isDynamicPartitionWrite; + __isset = other615.__isset; + return *this; +} +void LockComponent::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "LockComponent("; + out << "type=" << to_string(type); + out << ", " << "level=" << to_string(level); + out << ", " << "dbname=" << to_string(dbname); + out << ", " << "tablename="; (__isset.tablename ? (out << to_string(tablename)) : (out << "")); + out << ", " << "partitionname="; (__isset.partitionname ? (out << to_string(partitionname)) : (out << "")); + out << ", " << "operationType="; (__isset.operationType ? (out << to_string(operationType)) : (out << "")); + out << ", " << "isAcid="; (__isset.isAcid ? (out << to_string(isAcid)) : (out << "")); + out << ", " << "isDynamicPartitionWrite="; (__isset.isDynamicPartitionWrite ? (out << to_string(isDynamicPartitionWrite)) : (out << "")); + out << ")"; +} + + +LockRequest::~LockRequest() noexcept { +} + + +void LockRequest::__set_component(const std::vector & val) { + this->component = val; +} + +void LockRequest::__set_txnid(const int64_t val) { + this->txnid = val; +__isset.txnid = true; +} + +void LockRequest::__set_user(const std::string& val) { + this->user = val; +} + +void LockRequest::__set_hostname(const std::string& val) { + this->hostname = val; +} + +void LockRequest::__set_agentInfo(const std::string& val) { + this->agentInfo = val; +__isset.agentInfo = true; +} +std::ostream& operator<<(std::ostream& out, const LockRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t LockRequest::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_component = false; + bool isset_user = false; + bool isset_hostname = 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_LIST) { + { + this->component.clear(); + uint32_t _size616; + ::apache::thrift::protocol::TType _etype619; + xfer += iprot->readListBegin(_etype619, _size616); + this->component.resize(_size616); + uint32_t _i620; + for (_i620 = 0; _i620 < _size616; ++_i620) + { + xfer += this->component[_i620].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_component = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->txnid); + this->__isset.txnid = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->user); + isset_user = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->hostname); + isset_hostname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->agentInfo); + this->__isset.agentInfo = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_component) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_user) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_hostname) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t LockRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("LockRequest"); + + xfer += oprot->writeFieldBegin("component", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->component.size())); + std::vector ::const_iterator _iter621; + for (_iter621 = this->component.begin(); _iter621 != this->component.end(); ++_iter621) + { + xfer += (*_iter621).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + if (this->__isset.txnid) { + xfer += oprot->writeFieldBegin("txnid", ::apache::thrift::protocol::T_I64, 2); + xfer += oprot->writeI64(this->txnid); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldBegin("user", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->user); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("hostname", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->hostname); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.agentInfo) { + xfer += oprot->writeFieldBegin("agentInfo", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->agentInfo); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(LockRequest &a, LockRequest &b) { + using ::std::swap; + swap(a.component, b.component); + swap(a.txnid, b.txnid); + swap(a.user, b.user); + swap(a.hostname, b.hostname); + swap(a.agentInfo, b.agentInfo); + swap(a.__isset, b.__isset); +} + +LockRequest::LockRequest(const LockRequest& other622) { + component = other622.component; + txnid = other622.txnid; + user = other622.user; + hostname = other622.hostname; + agentInfo = other622.agentInfo; + __isset = other622.__isset; +} +LockRequest& LockRequest::operator=(const LockRequest& other623) { + component = other623.component; + txnid = other623.txnid; + user = other623.user; + hostname = other623.hostname; + agentInfo = other623.agentInfo; + __isset = other623.__isset; + return *this; +} +void LockRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "LockRequest("; + out << "component=" << to_string(component); + out << ", " << "txnid="; (__isset.txnid ? (out << to_string(txnid)) : (out << "")); + out << ", " << "user=" << to_string(user); + out << ", " << "hostname=" << to_string(hostname); + out << ", " << "agentInfo="; (__isset.agentInfo ? (out << to_string(agentInfo)) : (out << "")); + out << ")"; +} + + +LockResponse::~LockResponse() noexcept { +} + + +void LockResponse::__set_lockid(const int64_t val) { + this->lockid = val; +} + +void LockResponse::__set_state(const LockState::type val) { + this->state = val; +} +std::ostream& operator<<(std::ostream& out, const LockResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t LockResponse::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_lockid = false; + bool isset_state = 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_I64) { + xfer += iprot->readI64(this->lockid); + isset_lockid = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast624; + xfer += iprot->readI32(ecast624); + this->state = (LockState::type)ecast624; + isset_state = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_lockid) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_state) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t LockResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("LockResponse"); + + xfer += oprot->writeFieldBegin("lockid", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->lockid); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("state", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32((int32_t)this->state); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(LockResponse &a, LockResponse &b) { + using ::std::swap; + swap(a.lockid, b.lockid); + swap(a.state, b.state); +} + +LockResponse::LockResponse(const LockResponse& other625) { + lockid = other625.lockid; + state = other625.state; +} +LockResponse& LockResponse::operator=(const LockResponse& other626) { + lockid = other626.lockid; + state = other626.state; + return *this; +} +void LockResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "LockResponse("; + out << "lockid=" << to_string(lockid); + out << ", " << "state=" << to_string(state); + out << ")"; +} + + +CheckLockRequest::~CheckLockRequest() noexcept { +} + + +void CheckLockRequest::__set_lockid(const int64_t val) { + this->lockid = val; +} + +void CheckLockRequest::__set_txnid(const int64_t val) { + this->txnid = val; +__isset.txnid = true; +} + +void CheckLockRequest::__set_elapsed_ms(const int64_t val) { + this->elapsed_ms = val; +__isset.elapsed_ms = true; +} +std::ostream& operator<<(std::ostream& out, const CheckLockRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t CheckLockRequest::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_lockid = 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_I64) { + xfer += iprot->readI64(this->lockid); + isset_lockid = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->txnid); + this->__isset.txnid = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->elapsed_ms); + this->__isset.elapsed_ms = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_lockid) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t CheckLockRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("CheckLockRequest"); + + xfer += oprot->writeFieldBegin("lockid", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->lockid); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.txnid) { + xfer += oprot->writeFieldBegin("txnid", ::apache::thrift::protocol::T_I64, 2); + xfer += oprot->writeI64(this->txnid); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.elapsed_ms) { + xfer += oprot->writeFieldBegin("elapsed_ms", ::apache::thrift::protocol::T_I64, 3); + xfer += oprot->writeI64(this->elapsed_ms); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(CheckLockRequest &a, CheckLockRequest &b) { + using ::std::swap; + swap(a.lockid, b.lockid); + swap(a.txnid, b.txnid); + swap(a.elapsed_ms, b.elapsed_ms); + swap(a.__isset, b.__isset); +} + +CheckLockRequest::CheckLockRequest(const CheckLockRequest& other627) { + lockid = other627.lockid; + txnid = other627.txnid; + elapsed_ms = other627.elapsed_ms; + __isset = other627.__isset; +} +CheckLockRequest& CheckLockRequest::operator=(const CheckLockRequest& other628) { + lockid = other628.lockid; + txnid = other628.txnid; + elapsed_ms = other628.elapsed_ms; + __isset = other628.__isset; + return *this; +} +void CheckLockRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "CheckLockRequest("; + out << "lockid=" << to_string(lockid); + out << ", " << "txnid="; (__isset.txnid ? (out << to_string(txnid)) : (out << "")); + out << ", " << "elapsed_ms="; (__isset.elapsed_ms ? (out << to_string(elapsed_ms)) : (out << "")); + out << ")"; +} + + +UnlockRequest::~UnlockRequest() noexcept { +} + + +void UnlockRequest::__set_lockid(const int64_t val) { + this->lockid = val; +} +std::ostream& operator<<(std::ostream& out, const UnlockRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t UnlockRequest::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_lockid = 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_I64) { + xfer += iprot->readI64(this->lockid); + isset_lockid = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_lockid) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t UnlockRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("UnlockRequest"); + + xfer += oprot->writeFieldBegin("lockid", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->lockid); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(UnlockRequest &a, UnlockRequest &b) { + using ::std::swap; + swap(a.lockid, b.lockid); +} + +UnlockRequest::UnlockRequest(const UnlockRequest& other629) { + lockid = other629.lockid; +} +UnlockRequest& UnlockRequest::operator=(const UnlockRequest& other630) { + lockid = other630.lockid; + return *this; +} +void UnlockRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "UnlockRequest("; + out << "lockid=" << to_string(lockid); + out << ")"; +} + + +ShowLocksRequest::~ShowLocksRequest() noexcept { +} + + +void ShowLocksRequest::__set_dbname(const std::string& val) { + this->dbname = val; +__isset.dbname = true; +} + +void ShowLocksRequest::__set_tablename(const std::string& val) { + this->tablename = val; +__isset.tablename = true; +} + +void ShowLocksRequest::__set_partname(const std::string& val) { + this->partname = val; +__isset.partname = true; +} + +void ShowLocksRequest::__set_isExtended(const bool val) { + this->isExtended = val; +__isset.isExtended = true; +} +std::ostream& operator<<(std::ostream& out, const ShowLocksRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t ShowLocksRequest::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbname); + this->__isset.dbname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tablename); + this->__isset.tablename = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->partname); + this->__isset.partname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->isExtended); + this->__isset.isExtended = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ShowLocksRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ShowLocksRequest"); + + if (this->__isset.dbname) { + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbname); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.tablename) { + xfer += oprot->writeFieldBegin("tablename", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tablename); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.partname) { + xfer += oprot->writeFieldBegin("partname", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->partname); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.isExtended) { + xfer += oprot->writeFieldBegin("isExtended", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool(this->isExtended); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ShowLocksRequest &a, ShowLocksRequest &b) { + using ::std::swap; + swap(a.dbname, b.dbname); + swap(a.tablename, b.tablename); + swap(a.partname, b.partname); + swap(a.isExtended, b.isExtended); + swap(a.__isset, b.__isset); +} + +ShowLocksRequest::ShowLocksRequest(const ShowLocksRequest& other631) { + dbname = other631.dbname; + tablename = other631.tablename; + partname = other631.partname; + isExtended = other631.isExtended; + __isset = other631.__isset; +} +ShowLocksRequest& ShowLocksRequest::operator=(const ShowLocksRequest& other632) { + dbname = other632.dbname; + tablename = other632.tablename; + partname = other632.partname; + isExtended = other632.isExtended; + __isset = other632.__isset; + return *this; +} +void ShowLocksRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "ShowLocksRequest("; + out << "dbname="; (__isset.dbname ? (out << to_string(dbname)) : (out << "")); + out << ", " << "tablename="; (__isset.tablename ? (out << to_string(tablename)) : (out << "")); + out << ", " << "partname="; (__isset.partname ? (out << to_string(partname)) : (out << "")); + out << ", " << "isExtended="; (__isset.isExtended ? (out << to_string(isExtended)) : (out << "")); + out << ")"; +} + + +ShowLocksResponseElement::~ShowLocksResponseElement() noexcept { +} + + +void ShowLocksResponseElement::__set_lockid(const int64_t val) { + this->lockid = val; +} + +void ShowLocksResponseElement::__set_dbname(const std::string& val) { + this->dbname = val; +} + +void ShowLocksResponseElement::__set_tablename(const std::string& val) { + this->tablename = val; +__isset.tablename = true; +} + +void ShowLocksResponseElement::__set_partname(const std::string& val) { + this->partname = val; +__isset.partname = true; +} + +void ShowLocksResponseElement::__set_state(const LockState::type val) { + this->state = val; +} + +void ShowLocksResponseElement::__set_type(const LockType::type val) { + this->type = val; +} + +void ShowLocksResponseElement::__set_txnid(const int64_t val) { + this->txnid = val; +__isset.txnid = true; +} + +void ShowLocksResponseElement::__set_lastheartbeat(const int64_t val) { + this->lastheartbeat = val; +} + +void ShowLocksResponseElement::__set_acquiredat(const int64_t val) { + this->acquiredat = val; +__isset.acquiredat = true; +} + +void ShowLocksResponseElement::__set_user(const std::string& val) { + this->user = val; +} + +void ShowLocksResponseElement::__set_hostname(const std::string& val) { + this->hostname = val; +} + +void ShowLocksResponseElement::__set_heartbeatCount(const int32_t val) { + this->heartbeatCount = val; +__isset.heartbeatCount = true; +} + +void ShowLocksResponseElement::__set_agentInfo(const std::string& val) { + this->agentInfo = val; +__isset.agentInfo = true; +} + +void ShowLocksResponseElement::__set_blockedByExtId(const int64_t val) { + this->blockedByExtId = val; +__isset.blockedByExtId = true; +} + +void ShowLocksResponseElement::__set_blockedByIntId(const int64_t val) { + this->blockedByIntId = val; +__isset.blockedByIntId = true; +} + +void ShowLocksResponseElement::__set_lockIdInternal(const int64_t val) { + this->lockIdInternal = val; +__isset.lockIdInternal = true; +} +std::ostream& operator<<(std::ostream& out, const ShowLocksResponseElement& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t ShowLocksResponseElement::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_lockid = false; + bool isset_dbname = false; + bool isset_state = false; + bool isset_type = false; + bool isset_lastheartbeat = false; + bool isset_user = false; + bool isset_hostname = 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_I64) { + xfer += iprot->readI64(this->lockid); + isset_lockid = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbname); + isset_dbname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tablename); + this->__isset.tablename = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->partname); + this->__isset.partname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast633; + xfer += iprot->readI32(ecast633); + this->state = (LockState::type)ecast633; + isset_state = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast634; + xfer += iprot->readI32(ecast634); + this->type = (LockType::type)ecast634; + isset_type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->txnid); + this->__isset.txnid = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 8: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->lastheartbeat); + isset_lastheartbeat = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 9: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->acquiredat); + this->__isset.acquiredat = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 10: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->user); + isset_user = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 11: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->hostname); + isset_hostname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 12: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->heartbeatCount); + this->__isset.heartbeatCount = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 13: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->agentInfo); + this->__isset.agentInfo = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 14: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->blockedByExtId); + this->__isset.blockedByExtId = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 15: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->blockedByIntId); + this->__isset.blockedByIntId = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 16: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->lockIdInternal); + this->__isset.lockIdInternal = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_lockid) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_dbname) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_state) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_type) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_lastheartbeat) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_user) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_hostname) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t ShowLocksResponseElement::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ShowLocksResponseElement"); + + xfer += oprot->writeFieldBegin("lockid", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->lockid); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->dbname); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.tablename) { + xfer += oprot->writeFieldBegin("tablename", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->tablename); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.partname) { + xfer += oprot->writeFieldBegin("partname", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->partname); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldBegin("state", ::apache::thrift::protocol::T_I32, 5); + xfer += oprot->writeI32((int32_t)this->state); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_I32, 6); + xfer += oprot->writeI32((int32_t)this->type); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.txnid) { + xfer += oprot->writeFieldBegin("txnid", ::apache::thrift::protocol::T_I64, 7); + xfer += oprot->writeI64(this->txnid); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldBegin("lastheartbeat", ::apache::thrift::protocol::T_I64, 8); + xfer += oprot->writeI64(this->lastheartbeat); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.acquiredat) { + xfer += oprot->writeFieldBegin("acquiredat", ::apache::thrift::protocol::T_I64, 9); + xfer += oprot->writeI64(this->acquiredat); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldBegin("user", ::apache::thrift::protocol::T_STRING, 10); + xfer += oprot->writeString(this->user); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("hostname", ::apache::thrift::protocol::T_STRING, 11); + xfer += oprot->writeString(this->hostname); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.heartbeatCount) { + xfer += oprot->writeFieldBegin("heartbeatCount", ::apache::thrift::protocol::T_I32, 12); + xfer += oprot->writeI32(this->heartbeatCount); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.agentInfo) { + xfer += oprot->writeFieldBegin("agentInfo", ::apache::thrift::protocol::T_STRING, 13); + xfer += oprot->writeString(this->agentInfo); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.blockedByExtId) { + xfer += oprot->writeFieldBegin("blockedByExtId", ::apache::thrift::protocol::T_I64, 14); + xfer += oprot->writeI64(this->blockedByExtId); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.blockedByIntId) { + xfer += oprot->writeFieldBegin("blockedByIntId", ::apache::thrift::protocol::T_I64, 15); + xfer += oprot->writeI64(this->blockedByIntId); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.lockIdInternal) { + xfer += oprot->writeFieldBegin("lockIdInternal", ::apache::thrift::protocol::T_I64, 16); + xfer += oprot->writeI64(this->lockIdInternal); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ShowLocksResponseElement &a, ShowLocksResponseElement &b) { + using ::std::swap; + swap(a.lockid, b.lockid); + swap(a.dbname, b.dbname); + swap(a.tablename, b.tablename); + swap(a.partname, b.partname); + swap(a.state, b.state); + swap(a.type, b.type); + swap(a.txnid, b.txnid); + swap(a.lastheartbeat, b.lastheartbeat); + swap(a.acquiredat, b.acquiredat); + swap(a.user, b.user); + swap(a.hostname, b.hostname); + swap(a.heartbeatCount, b.heartbeatCount); + swap(a.agentInfo, b.agentInfo); + swap(a.blockedByExtId, b.blockedByExtId); + swap(a.blockedByIntId, b.blockedByIntId); + swap(a.lockIdInternal, b.lockIdInternal); + swap(a.__isset, b.__isset); +} + +ShowLocksResponseElement::ShowLocksResponseElement(const ShowLocksResponseElement& other635) { + lockid = other635.lockid; + dbname = other635.dbname; + tablename = other635.tablename; + partname = other635.partname; + state = other635.state; + type = other635.type; + txnid = other635.txnid; + lastheartbeat = other635.lastheartbeat; + acquiredat = other635.acquiredat; + user = other635.user; + hostname = other635.hostname; + heartbeatCount = other635.heartbeatCount; + agentInfo = other635.agentInfo; + blockedByExtId = other635.blockedByExtId; + blockedByIntId = other635.blockedByIntId; + lockIdInternal = other635.lockIdInternal; + __isset = other635.__isset; +} +ShowLocksResponseElement& ShowLocksResponseElement::operator=(const ShowLocksResponseElement& other636) { + lockid = other636.lockid; + dbname = other636.dbname; + tablename = other636.tablename; + partname = other636.partname; + state = other636.state; + type = other636.type; + txnid = other636.txnid; + lastheartbeat = other636.lastheartbeat; + acquiredat = other636.acquiredat; + user = other636.user; + hostname = other636.hostname; + heartbeatCount = other636.heartbeatCount; + agentInfo = other636.agentInfo; + blockedByExtId = other636.blockedByExtId; + blockedByIntId = other636.blockedByIntId; + lockIdInternal = other636.lockIdInternal; + __isset = other636.__isset; + return *this; +} +void ShowLocksResponseElement::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "ShowLocksResponseElement("; + out << "lockid=" << to_string(lockid); + out << ", " << "dbname=" << to_string(dbname); + out << ", " << "tablename="; (__isset.tablename ? (out << to_string(tablename)) : (out << "")); + out << ", " << "partname="; (__isset.partname ? (out << to_string(partname)) : (out << "")); + out << ", " << "state=" << to_string(state); + out << ", " << "type=" << to_string(type); + out << ", " << "txnid="; (__isset.txnid ? (out << to_string(txnid)) : (out << "")); + out << ", " << "lastheartbeat=" << to_string(lastheartbeat); + out << ", " << "acquiredat="; (__isset.acquiredat ? (out << to_string(acquiredat)) : (out << "")); + out << ", " << "user=" << to_string(user); + out << ", " << "hostname=" << to_string(hostname); + out << ", " << "heartbeatCount="; (__isset.heartbeatCount ? (out << to_string(heartbeatCount)) : (out << "")); + out << ", " << "agentInfo="; (__isset.agentInfo ? (out << to_string(agentInfo)) : (out << "")); + out << ", " << "blockedByExtId="; (__isset.blockedByExtId ? (out << to_string(blockedByExtId)) : (out << "")); + out << ", " << "blockedByIntId="; (__isset.blockedByIntId ? (out << to_string(blockedByIntId)) : (out << "")); + out << ", " << "lockIdInternal="; (__isset.lockIdInternal ? (out << to_string(lockIdInternal)) : (out << "")); + out << ")"; +} + + +ShowLocksResponse::~ShowLocksResponse() noexcept { +} + + +void ShowLocksResponse::__set_locks(const std::vector & val) { + this->locks = val; +} +std::ostream& operator<<(std::ostream& out, const ShowLocksResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t ShowLocksResponse::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->locks.clear(); + uint32_t _size637; + ::apache::thrift::protocol::TType _etype640; + xfer += iprot->readListBegin(_etype640, _size637); + this->locks.resize(_size637); + uint32_t _i641; + for (_i641 = 0; _i641 < _size637; ++_i641) + { + xfer += this->locks[_i641].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.locks = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ShowLocksResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ShowLocksResponse"); + + xfer += oprot->writeFieldBegin("locks", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->locks.size())); + std::vector ::const_iterator _iter642; + for (_iter642 = this->locks.begin(); _iter642 != this->locks.end(); ++_iter642) + { + xfer += (*_iter642).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ShowLocksResponse &a, ShowLocksResponse &b) { + using ::std::swap; + swap(a.locks, b.locks); + swap(a.__isset, b.__isset); +} + +ShowLocksResponse::ShowLocksResponse(const ShowLocksResponse& other643) { + locks = other643.locks; + __isset = other643.__isset; +} +ShowLocksResponse& ShowLocksResponse::operator=(const ShowLocksResponse& other644) { + locks = other644.locks; + __isset = other644.__isset; + return *this; +} +void ShowLocksResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "ShowLocksResponse("; + out << "locks=" << to_string(locks); + out << ")"; +} + + +HeartbeatRequest::~HeartbeatRequest() noexcept { +} + + +void HeartbeatRequest::__set_lockid(const int64_t val) { + this->lockid = val; +__isset.lockid = true; +} + +void HeartbeatRequest::__set_txnid(const int64_t val) { + this->txnid = val; +__isset.txnid = true; +} +std::ostream& operator<<(std::ostream& out, const HeartbeatRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t HeartbeatRequest::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->lockid); + this->__isset.lockid = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->txnid); + this->__isset.txnid = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t HeartbeatRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("HeartbeatRequest"); + + if (this->__isset.lockid) { + xfer += oprot->writeFieldBegin("lockid", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->lockid); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.txnid) { + xfer += oprot->writeFieldBegin("txnid", ::apache::thrift::protocol::T_I64, 2); + xfer += oprot->writeI64(this->txnid); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(HeartbeatRequest &a, HeartbeatRequest &b) { + using ::std::swap; + swap(a.lockid, b.lockid); + swap(a.txnid, b.txnid); + swap(a.__isset, b.__isset); +} + +HeartbeatRequest::HeartbeatRequest(const HeartbeatRequest& other645) { + lockid = other645.lockid; + txnid = other645.txnid; + __isset = other645.__isset; +} +HeartbeatRequest& HeartbeatRequest::operator=(const HeartbeatRequest& other646) { + lockid = other646.lockid; + txnid = other646.txnid; + __isset = other646.__isset; + return *this; +} +void HeartbeatRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "HeartbeatRequest("; + out << "lockid="; (__isset.lockid ? (out << to_string(lockid)) : (out << "")); + out << ", " << "txnid="; (__isset.txnid ? (out << to_string(txnid)) : (out << "")); + out << ")"; +} + + +HeartbeatTxnRangeRequest::~HeartbeatTxnRangeRequest() noexcept { +} + + +void HeartbeatTxnRangeRequest::__set_min(const int64_t val) { + this->min = val; +} + +void HeartbeatTxnRangeRequest::__set_max(const int64_t val) { + this->max = val; +} +std::ostream& operator<<(std::ostream& out, const HeartbeatTxnRangeRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t HeartbeatTxnRangeRequest::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_min = false; + bool isset_max = 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_I64) { + xfer += iprot->readI64(this->min); + isset_min = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->max); + isset_max = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_min) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_max) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t HeartbeatTxnRangeRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("HeartbeatTxnRangeRequest"); + + xfer += oprot->writeFieldBegin("min", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->min); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("max", ::apache::thrift::protocol::T_I64, 2); + xfer += oprot->writeI64(this->max); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(HeartbeatTxnRangeRequest &a, HeartbeatTxnRangeRequest &b) { + using ::std::swap; + swap(a.min, b.min); + swap(a.max, b.max); +} + +HeartbeatTxnRangeRequest::HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest& other647) { + min = other647.min; + max = other647.max; +} +HeartbeatTxnRangeRequest& HeartbeatTxnRangeRequest::operator=(const HeartbeatTxnRangeRequest& other648) { + min = other648.min; + max = other648.max; + return *this; +} +void HeartbeatTxnRangeRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "HeartbeatTxnRangeRequest("; + out << "min=" << to_string(min); + out << ", " << "max=" << to_string(max); + out << ")"; +} + + +HeartbeatTxnRangeResponse::~HeartbeatTxnRangeResponse() noexcept { +} + + +void HeartbeatTxnRangeResponse::__set_aborted(const std::set & val) { + this->aborted = val; +} + +void HeartbeatTxnRangeResponse::__set_nosuch(const std::set & val) { + this->nosuch = val; +} +std::ostream& operator<<(std::ostream& out, const HeartbeatTxnRangeResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t HeartbeatTxnRangeResponse::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_aborted = false; + bool isset_nosuch = 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_SET) { + { + this->aborted.clear(); + uint32_t _size649; + ::apache::thrift::protocol::TType _etype652; + xfer += iprot->readSetBegin(_etype652, _size649); + uint32_t _i653; + for (_i653 = 0; _i653 < _size649; ++_i653) + { + int64_t _elem654; + xfer += iprot->readI64(_elem654); + this->aborted.insert(_elem654); + } + xfer += iprot->readSetEnd(); + } + isset_aborted = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_SET) { + { + this->nosuch.clear(); + uint32_t _size655; + ::apache::thrift::protocol::TType _etype658; + xfer += iprot->readSetBegin(_etype658, _size655); + uint32_t _i659; + for (_i659 = 0; _i659 < _size655; ++_i659) + { + int64_t _elem660; + xfer += iprot->readI64(_elem660); + this->nosuch.insert(_elem660); + } + xfer += iprot->readSetEnd(); + } + isset_nosuch = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_aborted) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_nosuch) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t HeartbeatTxnRangeResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("HeartbeatTxnRangeResponse"); + + xfer += oprot->writeFieldBegin("aborted", ::apache::thrift::protocol::T_SET, 1); + { + xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast(this->aborted.size())); + std::set ::const_iterator _iter661; + for (_iter661 = this->aborted.begin(); _iter661 != this->aborted.end(); ++_iter661) + { + xfer += oprot->writeI64((*_iter661)); + } + xfer += oprot->writeSetEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("nosuch", ::apache::thrift::protocol::T_SET, 2); + { + xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast(this->nosuch.size())); + std::set ::const_iterator _iter662; + for (_iter662 = this->nosuch.begin(); _iter662 != this->nosuch.end(); ++_iter662) + { + xfer += oprot->writeI64((*_iter662)); + } + xfer += oprot->writeSetEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(HeartbeatTxnRangeResponse &a, HeartbeatTxnRangeResponse &b) { + using ::std::swap; + swap(a.aborted, b.aborted); + swap(a.nosuch, b.nosuch); +} + +HeartbeatTxnRangeResponse::HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse& other663) { + aborted = other663.aborted; + nosuch = other663.nosuch; +} +HeartbeatTxnRangeResponse& HeartbeatTxnRangeResponse::operator=(const HeartbeatTxnRangeResponse& other664) { + aborted = other664.aborted; + nosuch = other664.nosuch; + return *this; +} +void HeartbeatTxnRangeResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "HeartbeatTxnRangeResponse("; + out << "aborted=" << to_string(aborted); + out << ", " << "nosuch=" << to_string(nosuch); + out << ")"; +} + + +CompactionRequest::~CompactionRequest() noexcept { +} + + +void CompactionRequest::__set_dbname(const std::string& val) { + this->dbname = val; +} + +void CompactionRequest::__set_tablename(const std::string& val) { + this->tablename = val; +} + +void CompactionRequest::__set_partitionname(const std::string& val) { + this->partitionname = val; +__isset.partitionname = true; +} + +void CompactionRequest::__set_type(const CompactionType::type val) { + this->type = val; +} + +void CompactionRequest::__set_runas(const std::string& val) { + this->runas = val; +__isset.runas = true; +} + +void CompactionRequest::__set_properties(const std::map & val) { + this->properties = val; +__isset.properties = true; +} +std::ostream& operator<<(std::ostream& out, const CompactionRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t CompactionRequest::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_dbname = false; + bool isset_tablename = false; + bool isset_type = 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_STRING) { + xfer += iprot->readString(this->dbname); + isset_dbname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tablename); + isset_tablename = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->partitionname); + this->__isset.partitionname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast665; + xfer += iprot->readI32(ecast665); + this->type = (CompactionType::type)ecast665; + isset_type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->runas); + this->__isset.runas = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_MAP) { + { + this->properties.clear(); + uint32_t _size666; + ::apache::thrift::protocol::TType _ktype667; + ::apache::thrift::protocol::TType _vtype668; + xfer += iprot->readMapBegin(_ktype667, _vtype668, _size666); + uint32_t _i670; + for (_i670 = 0; _i670 < _size666; ++_i670) + { + std::string _key671; + xfer += iprot->readString(_key671); + std::string& _val672 = this->properties[_key671]; + xfer += iprot->readString(_val672); + } + xfer += iprot->readMapEnd(); + } + this->__isset.properties = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_dbname) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tablename) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_type) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t CompactionRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("CompactionRequest"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbname); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tablename", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tablename); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.partitionname) { + xfer += oprot->writeFieldBegin("partitionname", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->partitionname); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32((int32_t)this->type); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.runas) { + xfer += oprot->writeFieldBegin("runas", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->runas); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.properties) { + xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 6); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->properties.size())); + std::map ::const_iterator _iter673; + for (_iter673 = this->properties.begin(); _iter673 != this->properties.end(); ++_iter673) + { + xfer += oprot->writeString(_iter673->first); + xfer += oprot->writeString(_iter673->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(CompactionRequest &a, CompactionRequest &b) { + using ::std::swap; + swap(a.dbname, b.dbname); + swap(a.tablename, b.tablename); + swap(a.partitionname, b.partitionname); + swap(a.type, b.type); + swap(a.runas, b.runas); + swap(a.properties, b.properties); + swap(a.__isset, b.__isset); +} + +CompactionRequest::CompactionRequest(const CompactionRequest& other674) { + dbname = other674.dbname; + tablename = other674.tablename; + partitionname = other674.partitionname; + type = other674.type; + runas = other674.runas; + properties = other674.properties; + __isset = other674.__isset; +} +CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other675) { + dbname = other675.dbname; + tablename = other675.tablename; + partitionname = other675.partitionname; + type = other675.type; + runas = other675.runas; + properties = other675.properties; + __isset = other675.__isset; + return *this; +} +void CompactionRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "CompactionRequest("; + out << "dbname=" << to_string(dbname); + out << ", " << "tablename=" << to_string(tablename); + out << ", " << "partitionname="; (__isset.partitionname ? (out << to_string(partitionname)) : (out << "")); + out << ", " << "type=" << to_string(type); + out << ", " << "runas="; (__isset.runas ? (out << to_string(runas)) : (out << "")); + out << ", " << "properties="; (__isset.properties ? (out << to_string(properties)) : (out << "")); + out << ")"; +} + + +CompactionResponse::~CompactionResponse() noexcept { +} + + +void CompactionResponse::__set_id(const int64_t val) { + this->id = val; +} + +void CompactionResponse::__set_state(const std::string& val) { + this->state = val; +} + +void CompactionResponse::__set_accepted(const bool val) { + this->accepted = val; +} +std::ostream& operator<<(std::ostream& out, const CompactionResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t CompactionResponse::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_id = false; + bool isset_state = false; + bool isset_accepted = 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_I64) { + xfer += iprot->readI64(this->id); + isset_id = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->state); + isset_state = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->accepted); + isset_accepted = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_id) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_state) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_accepted) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t CompactionResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("CompactionResponse"); + + xfer += oprot->writeFieldBegin("id", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->id); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("state", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->state); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("accepted", ::apache::thrift::protocol::T_BOOL, 3); + xfer += oprot->writeBool(this->accepted); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(CompactionResponse &a, CompactionResponse &b) { + using ::std::swap; + swap(a.id, b.id); + swap(a.state, b.state); + swap(a.accepted, b.accepted); +} + +CompactionResponse::CompactionResponse(const CompactionResponse& other676) { + id = other676.id; + state = other676.state; + accepted = other676.accepted; +} +CompactionResponse& CompactionResponse::operator=(const CompactionResponse& other677) { + id = other677.id; + state = other677.state; + accepted = other677.accepted; + return *this; +} +void CompactionResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "CompactionResponse("; + out << "id=" << to_string(id); + out << ", " << "state=" << to_string(state); + out << ", " << "accepted=" << to_string(accepted); + out << ")"; +} + + +ShowCompactRequest::~ShowCompactRequest() noexcept { +} + +std::ostream& operator<<(std::ostream& out, const ShowCompactRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t ShowCompactRequest::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 ShowCompactRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ShowCompactRequest"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ShowCompactRequest &a, ShowCompactRequest &b) { + using ::std::swap; + (void) a; + (void) b; +} + +ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other678) { + (void) other678; +} +ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other679) { + (void) other679; + return *this; +} +void ShowCompactRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "ShowCompactRequest("; + out << ")"; +} + + +ShowCompactResponseElement::~ShowCompactResponseElement() noexcept { +} + + +void ShowCompactResponseElement::__set_dbname(const std::string& val) { + this->dbname = val; +} + +void ShowCompactResponseElement::__set_tablename(const std::string& val) { + this->tablename = val; +} + +void ShowCompactResponseElement::__set_partitionname(const std::string& val) { + this->partitionname = val; +__isset.partitionname = true; +} + +void ShowCompactResponseElement::__set_type(const CompactionType::type val) { + this->type = val; +} + +void ShowCompactResponseElement::__set_state(const std::string& val) { + this->state = val; +} + +void ShowCompactResponseElement::__set_workerid(const std::string& val) { + this->workerid = val; +__isset.workerid = true; +} + +void ShowCompactResponseElement::__set_start(const int64_t val) { + this->start = val; +__isset.start = true; +} + +void ShowCompactResponseElement::__set_runAs(const std::string& val) { + this->runAs = val; +__isset.runAs = true; +} + +void ShowCompactResponseElement::__set_hightestTxnId(const int64_t val) { + this->hightestTxnId = val; +__isset.hightestTxnId = true; +} + +void ShowCompactResponseElement::__set_metaInfo(const std::string& val) { + this->metaInfo = val; +__isset.metaInfo = true; +} + +void ShowCompactResponseElement::__set_endTime(const int64_t val) { + this->endTime = val; +__isset.endTime = true; +} + +void ShowCompactResponseElement::__set_hadoopJobId(const std::string& val) { + this->hadoopJobId = val; +__isset.hadoopJobId = true; +} + +void ShowCompactResponseElement::__set_id(const int64_t val) { + this->id = val; +__isset.id = true; +} +std::ostream& operator<<(std::ostream& out, const ShowCompactResponseElement& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t ShowCompactResponseElement::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_dbname = false; + bool isset_tablename = false; + bool isset_type = false; + bool isset_state = 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_STRING) { + xfer += iprot->readString(this->dbname); + isset_dbname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tablename); + isset_tablename = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->partitionname); + this->__isset.partitionname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast680; + xfer += iprot->readI32(ecast680); + this->type = (CompactionType::type)ecast680; + isset_type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->state); + isset_state = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->workerid); + this->__isset.workerid = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->start); + this->__isset.start = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 8: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->runAs); + this->__isset.runAs = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 9: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->hightestTxnId); + this->__isset.hightestTxnId = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 10: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->metaInfo); + this->__isset.metaInfo = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 11: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->endTime); + this->__isset.endTime = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 12: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->hadoopJobId); + this->__isset.hadoopJobId = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 13: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->id); + this->__isset.id = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_dbname) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tablename) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_type) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_state) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t ShowCompactResponseElement::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ShowCompactResponseElement"); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbname); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tablename", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tablename); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.partitionname) { + xfer += oprot->writeFieldBegin("partitionname", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->partitionname); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32((int32_t)this->type); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("state", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->state); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.workerid) { + xfer += oprot->writeFieldBegin("workerid", ::apache::thrift::protocol::T_STRING, 6); + xfer += oprot->writeString(this->workerid); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.start) { + xfer += oprot->writeFieldBegin("start", ::apache::thrift::protocol::T_I64, 7); + xfer += oprot->writeI64(this->start); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.runAs) { + xfer += oprot->writeFieldBegin("runAs", ::apache::thrift::protocol::T_STRING, 8); + xfer += oprot->writeString(this->runAs); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.hightestTxnId) { + xfer += oprot->writeFieldBegin("hightestTxnId", ::apache::thrift::protocol::T_I64, 9); + xfer += oprot->writeI64(this->hightestTxnId); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.metaInfo) { + xfer += oprot->writeFieldBegin("metaInfo", ::apache::thrift::protocol::T_STRING, 10); + xfer += oprot->writeString(this->metaInfo); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.endTime) { + xfer += oprot->writeFieldBegin("endTime", ::apache::thrift::protocol::T_I64, 11); + xfer += oprot->writeI64(this->endTime); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.hadoopJobId) { + xfer += oprot->writeFieldBegin("hadoopJobId", ::apache::thrift::protocol::T_STRING, 12); + xfer += oprot->writeString(this->hadoopJobId); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.id) { + xfer += oprot->writeFieldBegin("id", ::apache::thrift::protocol::T_I64, 13); + xfer += oprot->writeI64(this->id); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ShowCompactResponseElement &a, ShowCompactResponseElement &b) { + using ::std::swap; + swap(a.dbname, b.dbname); + swap(a.tablename, b.tablename); + swap(a.partitionname, b.partitionname); + swap(a.type, b.type); + swap(a.state, b.state); + swap(a.workerid, b.workerid); + swap(a.start, b.start); + swap(a.runAs, b.runAs); + swap(a.hightestTxnId, b.hightestTxnId); + swap(a.metaInfo, b.metaInfo); + swap(a.endTime, b.endTime); + swap(a.hadoopJobId, b.hadoopJobId); + swap(a.id, b.id); + swap(a.__isset, b.__isset); +} + +ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other681) { + dbname = other681.dbname; + tablename = other681.tablename; + partitionname = other681.partitionname; + type = other681.type; + state = other681.state; + workerid = other681.workerid; + start = other681.start; + runAs = other681.runAs; + hightestTxnId = other681.hightestTxnId; + metaInfo = other681.metaInfo; + endTime = other681.endTime; + hadoopJobId = other681.hadoopJobId; + id = other681.id; + __isset = other681.__isset; +} +ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other682) { + dbname = other682.dbname; + tablename = other682.tablename; + partitionname = other682.partitionname; + type = other682.type; + state = other682.state; + workerid = other682.workerid; + start = other682.start; + runAs = other682.runAs; + hightestTxnId = other682.hightestTxnId; + metaInfo = other682.metaInfo; + endTime = other682.endTime; + hadoopJobId = other682.hadoopJobId; + id = other682.id; + __isset = other682.__isset; + return *this; +} +void ShowCompactResponseElement::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "ShowCompactResponseElement("; + out << "dbname=" << to_string(dbname); + out << ", " << "tablename=" << to_string(tablename); + out << ", " << "partitionname="; (__isset.partitionname ? (out << to_string(partitionname)) : (out << "")); + out << ", " << "type=" << to_string(type); + out << ", " << "state=" << to_string(state); + out << ", " << "workerid="; (__isset.workerid ? (out << to_string(workerid)) : (out << "")); + out << ", " << "start="; (__isset.start ? (out << to_string(start)) : (out << "")); + out << ", " << "runAs="; (__isset.runAs ? (out << to_string(runAs)) : (out << "")); + out << ", " << "hightestTxnId="; (__isset.hightestTxnId ? (out << to_string(hightestTxnId)) : (out << "")); + out << ", " << "metaInfo="; (__isset.metaInfo ? (out << to_string(metaInfo)) : (out << "")); + out << ", " << "endTime="; (__isset.endTime ? (out << to_string(endTime)) : (out << "")); + out << ", " << "hadoopJobId="; (__isset.hadoopJobId ? (out << to_string(hadoopJobId)) : (out << "")); + out << ", " << "id="; (__isset.id ? (out << to_string(id)) : (out << "")); + out << ")"; +} + + +ShowCompactResponse::~ShowCompactResponse() noexcept { +} + + +void ShowCompactResponse::__set_compacts(const std::vector & val) { + this->compacts = val; +} +std::ostream& operator<<(std::ostream& out, const ShowCompactResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t ShowCompactResponse::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_compacts = 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_LIST) { + { + this->compacts.clear(); + uint32_t _size683; + ::apache::thrift::protocol::TType _etype686; + xfer += iprot->readListBegin(_etype686, _size683); + this->compacts.resize(_size683); + uint32_t _i687; + for (_i687 = 0; _i687 < _size683; ++_i687) + { + xfer += this->compacts[_i687].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_compacts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_compacts) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t ShowCompactResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ShowCompactResponse"); + + xfer += oprot->writeFieldBegin("compacts", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->compacts.size())); + std::vector ::const_iterator _iter688; + for (_iter688 = this->compacts.begin(); _iter688 != this->compacts.end(); ++_iter688) + { + xfer += (*_iter688).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ShowCompactResponse &a, ShowCompactResponse &b) { + using ::std::swap; + swap(a.compacts, b.compacts); +} + +ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other689) { + compacts = other689.compacts; +} +ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other690) { + compacts = other690.compacts; + return *this; +} +void ShowCompactResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "ShowCompactResponse("; + out << "compacts=" << to_string(compacts); + out << ")"; +} + + +AddDynamicPartitions::~AddDynamicPartitions() noexcept { +} + + +void AddDynamicPartitions::__set_txnid(const int64_t val) { + this->txnid = val; +} + +void AddDynamicPartitions::__set_dbname(const std::string& val) { + this->dbname = val; +} + +void AddDynamicPartitions::__set_tablename(const std::string& val) { + this->tablename = val; +} + +void AddDynamicPartitions::__set_partitionnames(const std::vector & val) { + this->partitionnames = val; +} + +void AddDynamicPartitions::__set_operationType(const DataOperationType::type val) { + this->operationType = val; +__isset.operationType = true; +} +std::ostream& operator<<(std::ostream& out, const AddDynamicPartitions& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t AddDynamicPartitions::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_txnid = false; + bool isset_dbname = false; + bool isset_tablename = false; + bool isset_partitionnames = 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_I64) { + xfer += iprot->readI64(this->txnid); + isset_txnid = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbname); + isset_dbname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tablename); + isset_tablename = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->partitionnames.clear(); + uint32_t _size691; + ::apache::thrift::protocol::TType _etype694; + xfer += iprot->readListBegin(_etype694, _size691); + this->partitionnames.resize(_size691); + uint32_t _i695; + for (_i695 = 0; _i695 < _size691; ++_i695) + { + xfer += iprot->readString(this->partitionnames[_i695]); + } + xfer += iprot->readListEnd(); + } + isset_partitionnames = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast696; + xfer += iprot->readI32(ecast696); + this->operationType = (DataOperationType::type)ecast696; + this->__isset.operationType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_txnid) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_dbname) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tablename) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_partitionnames) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t AddDynamicPartitions::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("AddDynamicPartitions"); + + xfer += oprot->writeFieldBegin("txnid", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->txnid); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->dbname); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tablename", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->tablename); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("partitionnames", ::apache::thrift::protocol::T_LIST, 4); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partitionnames.size())); + std::vector ::const_iterator _iter697; + for (_iter697 = this->partitionnames.begin(); _iter697 != this->partitionnames.end(); ++_iter697) + { + xfer += oprot->writeString((*_iter697)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + if (this->__isset.operationType) { + xfer += oprot->writeFieldBegin("operationType", ::apache::thrift::protocol::T_I32, 5); + xfer += oprot->writeI32((int32_t)this->operationType); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(AddDynamicPartitions &a, AddDynamicPartitions &b) { + using ::std::swap; + swap(a.txnid, b.txnid); + swap(a.dbname, b.dbname); + swap(a.tablename, b.tablename); + swap(a.partitionnames, b.partitionnames); + swap(a.operationType, b.operationType); + swap(a.__isset, b.__isset); +} + +AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other698) { + txnid = other698.txnid; + dbname = other698.dbname; + tablename = other698.tablename; + partitionnames = other698.partitionnames; + operationType = other698.operationType; + __isset = other698.__isset; +} +AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other699) { + txnid = other699.txnid; + dbname = other699.dbname; + tablename = other699.tablename; + partitionnames = other699.partitionnames; + operationType = other699.operationType; + __isset = other699.__isset; + return *this; +} +void AddDynamicPartitions::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "AddDynamicPartitions("; + out << "txnid=" << to_string(txnid); + out << ", " << "dbname=" << to_string(dbname); + out << ", " << "tablename=" << to_string(tablename); + out << ", " << "partitionnames=" << to_string(partitionnames); + out << ", " << "operationType="; (__isset.operationType ? (out << to_string(operationType)) : (out << "")); + out << ")"; +} + + +NotificationEventRequest::~NotificationEventRequest() noexcept { +} + + +void NotificationEventRequest::__set_lastEvent(const int64_t val) { + this->lastEvent = val; +} + +void NotificationEventRequest::__set_maxEvents(const int32_t val) { + this->maxEvents = val; +__isset.maxEvents = true; +} +std::ostream& operator<<(std::ostream& out, const NotificationEventRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t NotificationEventRequest::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_lastEvent = 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_I64) { + xfer += iprot->readI64(this->lastEvent); + isset_lastEvent = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->maxEvents); + this->__isset.maxEvents = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_lastEvent) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t NotificationEventRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("NotificationEventRequest"); + + xfer += oprot->writeFieldBegin("lastEvent", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->lastEvent); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.maxEvents) { + xfer += oprot->writeFieldBegin("maxEvents", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32(this->maxEvents); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(NotificationEventRequest &a, NotificationEventRequest &b) { + using ::std::swap; + swap(a.lastEvent, b.lastEvent); + swap(a.maxEvents, b.maxEvents); + swap(a.__isset, b.__isset); +} + +NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other700) { + lastEvent = other700.lastEvent; + maxEvents = other700.maxEvents; + __isset = other700.__isset; +} +NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other701) { + lastEvent = other701.lastEvent; + maxEvents = other701.maxEvents; + __isset = other701.__isset; + return *this; +} +void NotificationEventRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "NotificationEventRequest("; + out << "lastEvent=" << to_string(lastEvent); + out << ", " << "maxEvents="; (__isset.maxEvents ? (out << to_string(maxEvents)) : (out << "")); + out << ")"; +} + + +NotificationEvent::~NotificationEvent() noexcept { +} + + +void NotificationEvent::__set_eventId(const int64_t val) { + this->eventId = val; +} + +void NotificationEvent::__set_eventTime(const int32_t val) { + this->eventTime = val; +} + +void NotificationEvent::__set_eventType(const std::string& val) { + this->eventType = val; +} + +void NotificationEvent::__set_dbName(const std::string& val) { + this->dbName = val; +__isset.dbName = true; +} + +void NotificationEvent::__set_tableName(const std::string& val) { + this->tableName = val; +__isset.tableName = true; +} + +void NotificationEvent::__set_message(const std::string& val) { + this->message = val; +} + +void NotificationEvent::__set_messageFormat(const std::string& val) { + this->messageFormat = val; +__isset.messageFormat = true; +} +std::ostream& operator<<(std::ostream& out, const NotificationEvent& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t NotificationEvent::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_eventId = false; + bool isset_eventTime = false; + bool isset_eventType = false; + bool isset_message = 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_I64) { + xfer += iprot->readI64(this->eventId); + isset_eventId = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->eventTime); + isset_eventTime = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->eventType); + isset_eventType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tableName); + this->__isset.tableName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->message); + isset_message = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->messageFormat); + this->__isset.messageFormat = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_eventId) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_eventTime) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_eventType) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_message) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t NotificationEvent::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("NotificationEvent"); + + xfer += oprot->writeFieldBegin("eventId", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->eventId); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("eventTime", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32(this->eventTime); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("eventType", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->eventType); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.dbName) { + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.tableName) { + xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->tableName); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 6); + xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.messageFormat) { + xfer += oprot->writeFieldBegin("messageFormat", ::apache::thrift::protocol::T_STRING, 7); + xfer += oprot->writeString(this->messageFormat); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(NotificationEvent &a, NotificationEvent &b) { + using ::std::swap; + swap(a.eventId, b.eventId); + swap(a.eventTime, b.eventTime); + swap(a.eventType, b.eventType); + swap(a.dbName, b.dbName); + swap(a.tableName, b.tableName); + swap(a.message, b.message); + swap(a.messageFormat, b.messageFormat); + swap(a.__isset, b.__isset); +} + +NotificationEvent::NotificationEvent(const NotificationEvent& other702) { + eventId = other702.eventId; + eventTime = other702.eventTime; + eventType = other702.eventType; + dbName = other702.dbName; + tableName = other702.tableName; + message = other702.message; + messageFormat = other702.messageFormat; + __isset = other702.__isset; +} +NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other703) { + eventId = other703.eventId; + eventTime = other703.eventTime; + eventType = other703.eventType; + dbName = other703.dbName; + tableName = other703.tableName; + message = other703.message; + messageFormat = other703.messageFormat; + __isset = other703.__isset; + return *this; +} +void NotificationEvent::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "NotificationEvent("; + out << "eventId=" << to_string(eventId); + out << ", " << "eventTime=" << to_string(eventTime); + out << ", " << "eventType=" << to_string(eventType); + out << ", " << "dbName="; (__isset.dbName ? (out << to_string(dbName)) : (out << "")); + out << ", " << "tableName="; (__isset.tableName ? (out << to_string(tableName)) : (out << "")); + out << ", " << "message=" << to_string(message); + out << ", " << "messageFormat="; (__isset.messageFormat ? (out << to_string(messageFormat)) : (out << "")); + out << ")"; +} + + +NotificationEventResponse::~NotificationEventResponse() noexcept { +} + + +void NotificationEventResponse::__set_events(const std::vector & val) { + this->events = val; +} +std::ostream& operator<<(std::ostream& out, const NotificationEventResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t NotificationEventResponse::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_events = 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_LIST) { + { + this->events.clear(); + uint32_t _size704; + ::apache::thrift::protocol::TType _etype707; + xfer += iprot->readListBegin(_etype707, _size704); + this->events.resize(_size704); + uint32_t _i708; + for (_i708 = 0; _i708 < _size704; ++_i708) + { + xfer += this->events[_i708].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_events = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_events) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t NotificationEventResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("NotificationEventResponse"); + + xfer += oprot->writeFieldBegin("events", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->events.size())); + std::vector ::const_iterator _iter709; + for (_iter709 = this->events.begin(); _iter709 != this->events.end(); ++_iter709) + { + xfer += (*_iter709).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(NotificationEventResponse &a, NotificationEventResponse &b) { + using ::std::swap; + swap(a.events, b.events); +} + +NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other710) { + events = other710.events; +} +NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other711) { + events = other711.events; + return *this; +} +void NotificationEventResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "NotificationEventResponse("; + out << "events=" << to_string(events); + out << ")"; +} + + +CurrentNotificationEventId::~CurrentNotificationEventId() noexcept { +} + + +void CurrentNotificationEventId::__set_eventId(const int64_t val) { + this->eventId = val; +} +std::ostream& operator<<(std::ostream& out, const CurrentNotificationEventId& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t CurrentNotificationEventId::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_eventId = 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_I64) { + xfer += iprot->readI64(this->eventId); + isset_eventId = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_eventId) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t CurrentNotificationEventId::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("CurrentNotificationEventId"); + + xfer += oprot->writeFieldBegin("eventId", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->eventId); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(CurrentNotificationEventId &a, CurrentNotificationEventId &b) { + using ::std::swap; + swap(a.eventId, b.eventId); +} + +CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other712) { + eventId = other712.eventId; +} +CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other713) { + eventId = other713.eventId; + return *this; +} +void CurrentNotificationEventId::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "CurrentNotificationEventId("; + out << "eventId=" << to_string(eventId); + out << ")"; +} + + +NotificationEventsCountRequest::~NotificationEventsCountRequest() noexcept { +} + + +void NotificationEventsCountRequest::__set_fromEventId(const int64_t val) { + this->fromEventId = val; +} + +void NotificationEventsCountRequest::__set_dbName(const std::string& val) { + this->dbName = val; +} +std::ostream& operator<<(std::ostream& out, const NotificationEventsCountRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t NotificationEventsCountRequest::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_fromEventId = false; + bool isset_dbName = 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_I64) { + xfer += iprot->readI64(this->fromEventId); + isset_fromEventId = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbName); + isset_dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_fromEventId) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_dbName) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t NotificationEventsCountRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("NotificationEventsCountRequest"); + + xfer += oprot->writeFieldBegin("fromEventId", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->fromEventId); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(NotificationEventsCountRequest &a, NotificationEventsCountRequest &b) { + using ::std::swap; + swap(a.fromEventId, b.fromEventId); + swap(a.dbName, b.dbName); +} + +NotificationEventsCountRequest::NotificationEventsCountRequest(const NotificationEventsCountRequest& other714) { + fromEventId = other714.fromEventId; + dbName = other714.dbName; +} +NotificationEventsCountRequest& NotificationEventsCountRequest::operator=(const NotificationEventsCountRequest& other715) { + fromEventId = other715.fromEventId; + dbName = other715.dbName; + return *this; +} +void NotificationEventsCountRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "NotificationEventsCountRequest("; + out << "fromEventId=" << to_string(fromEventId); + out << ", " << "dbName=" << to_string(dbName); + out << ")"; +} + + +NotificationEventsCountResponse::~NotificationEventsCountResponse() noexcept { +} + + +void NotificationEventsCountResponse::__set_eventsCount(const int64_t val) { + this->eventsCount = val; +} +std::ostream& operator<<(std::ostream& out, const NotificationEventsCountResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t NotificationEventsCountResponse::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_eventsCount = 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_I64) { + xfer += iprot->readI64(this->eventsCount); + isset_eventsCount = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_eventsCount) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t NotificationEventsCountResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("NotificationEventsCountResponse"); + + xfer += oprot->writeFieldBegin("eventsCount", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->eventsCount); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(NotificationEventsCountResponse &a, NotificationEventsCountResponse &b) { + using ::std::swap; + swap(a.eventsCount, b.eventsCount); +} + +NotificationEventsCountResponse::NotificationEventsCountResponse(const NotificationEventsCountResponse& other716) { + eventsCount = other716.eventsCount; +} +NotificationEventsCountResponse& NotificationEventsCountResponse::operator=(const NotificationEventsCountResponse& other717) { + eventsCount = other717.eventsCount; + return *this; +} +void NotificationEventsCountResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "NotificationEventsCountResponse("; + out << "eventsCount=" << to_string(eventsCount); + out << ")"; +} + + +InsertEventRequestData::~InsertEventRequestData() noexcept { +} + + +void InsertEventRequestData::__set_replace(const bool val) { + this->replace = val; +__isset.replace = true; +} + +void InsertEventRequestData::__set_filesAdded(const std::vector & val) { + this->filesAdded = val; +} + +void InsertEventRequestData::__set_filesAddedChecksum(const std::vector & val) { + this->filesAddedChecksum = val; +__isset.filesAddedChecksum = true; +} +std::ostream& operator<<(std::ostream& out, const InsertEventRequestData& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t InsertEventRequestData::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_filesAdded = 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_BOOL) { + xfer += iprot->readBool(this->replace); + this->__isset.replace = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->filesAdded.clear(); + uint32_t _size718; + ::apache::thrift::protocol::TType _etype721; + xfer += iprot->readListBegin(_etype721, _size718); + this->filesAdded.resize(_size718); + uint32_t _i722; + for (_i722 = 0; _i722 < _size718; ++_i722) + { + xfer += iprot->readString(this->filesAdded[_i722]); + } + xfer += iprot->readListEnd(); + } + isset_filesAdded = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->filesAddedChecksum.clear(); + uint32_t _size723; + ::apache::thrift::protocol::TType _etype726; + xfer += iprot->readListBegin(_etype726, _size723); + this->filesAddedChecksum.resize(_size723); + uint32_t _i727; + for (_i727 = 0; _i727 < _size723; ++_i727) + { + xfer += iprot->readString(this->filesAddedChecksum[_i727]); + } + xfer += iprot->readListEnd(); + } + this->__isset.filesAddedChecksum = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_filesAdded) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("InsertEventRequestData"); + + if (this->__isset.replace) { + xfer += oprot->writeFieldBegin("replace", ::apache::thrift::protocol::T_BOOL, 1); + xfer += oprot->writeBool(this->replace); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldBegin("filesAdded", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->filesAdded.size())); + std::vector ::const_iterator _iter728; + for (_iter728 = this->filesAdded.begin(); _iter728 != this->filesAdded.end(); ++_iter728) + { + xfer += oprot->writeString((*_iter728)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + if (this->__isset.filesAddedChecksum) { + xfer += oprot->writeFieldBegin("filesAddedChecksum", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->filesAddedChecksum.size())); + std::vector ::const_iterator _iter729; + for (_iter729 = this->filesAddedChecksum.begin(); _iter729 != this->filesAddedChecksum.end(); ++_iter729) + { + xfer += oprot->writeString((*_iter729)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(InsertEventRequestData &a, InsertEventRequestData &b) { + using ::std::swap; + swap(a.replace, b.replace); + swap(a.filesAdded, b.filesAdded); + swap(a.filesAddedChecksum, b.filesAddedChecksum); + swap(a.__isset, b.__isset); +} + +InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other730) { + replace = other730.replace; + filesAdded = other730.filesAdded; + filesAddedChecksum = other730.filesAddedChecksum; + __isset = other730.__isset; +} +InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other731) { + replace = other731.replace; + filesAdded = other731.filesAdded; + filesAddedChecksum = other731.filesAddedChecksum; + __isset = other731.__isset; + return *this; +} +void InsertEventRequestData::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "InsertEventRequestData("; + out << "replace="; (__isset.replace ? (out << to_string(replace)) : (out << "")); + out << ", " << "filesAdded=" << to_string(filesAdded); + out << ", " << "filesAddedChecksum="; (__isset.filesAddedChecksum ? (out << to_string(filesAddedChecksum)) : (out << "")); + out << ")"; +} + + +FireEventRequestData::~FireEventRequestData() noexcept { +} + + +void FireEventRequestData::__set_insertData(const InsertEventRequestData& val) { + this->insertData = val; +__isset.insertData = true; +} +std::ostream& operator<<(std::ostream& out, const FireEventRequestData& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t FireEventRequestData::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->insertData.read(iprot); + this->__isset.insertData = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t FireEventRequestData::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("FireEventRequestData"); + + if (this->__isset.insertData) { + xfer += oprot->writeFieldBegin("insertData", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->insertData.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(FireEventRequestData &a, FireEventRequestData &b) { + using ::std::swap; + swap(a.insertData, b.insertData); + swap(a.__isset, b.__isset); +} + +FireEventRequestData::FireEventRequestData(const FireEventRequestData& other732) { + insertData = other732.insertData; + __isset = other732.__isset; +} +FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other733) { + insertData = other733.insertData; + __isset = other733.__isset; + return *this; +} +void FireEventRequestData::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "FireEventRequestData("; + out << "insertData="; (__isset.insertData ? (out << to_string(insertData)) : (out << "")); + out << ")"; +} + + +FireEventRequest::~FireEventRequest() noexcept { +} + + +void FireEventRequest::__set_successful(const bool val) { + this->successful = val; +} + +void FireEventRequest::__set_data(const FireEventRequestData& val) { + this->data = val; +} + +void FireEventRequest::__set_dbName(const std::string& val) { + this->dbName = val; +__isset.dbName = true; +} + +void FireEventRequest::__set_tableName(const std::string& val) { + this->tableName = val; +__isset.tableName = true; +} + +void FireEventRequest::__set_partitionVals(const std::vector & val) { + this->partitionVals = val; +__isset.partitionVals = true; +} +std::ostream& operator<<(std::ostream& out, const FireEventRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t FireEventRequest::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_successful = false; + bool isset_data = 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_BOOL) { + xfer += iprot->readBool(this->successful); + isset_successful = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->data.read(iprot); + isset_data = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->dbName); + this->__isset.dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tableName); + this->__isset.tableName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->partitionVals.clear(); + uint32_t _size734; + ::apache::thrift::protocol::TType _etype737; + xfer += iprot->readListBegin(_etype737, _size734); + this->partitionVals.resize(_size734); + uint32_t _i738; + for (_i738 = 0; _i738 < _size734; ++_i738) + { + xfer += iprot->readString(this->partitionVals[_i738]); + } + xfer += iprot->readListEnd(); + } + this->__isset.partitionVals = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_successful) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_data) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t FireEventRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("FireEventRequest"); + + xfer += oprot->writeFieldBegin("successful", ::apache::thrift::protocol::T_BOOL, 1); + xfer += oprot->writeBool(this->successful); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("data", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->data.write(oprot); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.dbName) { + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.tableName) { + xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->tableName); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.partitionVals) { + xfer += oprot->writeFieldBegin("partitionVals", ::apache::thrift::protocol::T_LIST, 5); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partitionVals.size())); + std::vector ::const_iterator _iter739; + for (_iter739 = this->partitionVals.begin(); _iter739 != this->partitionVals.end(); ++_iter739) + { + xfer += oprot->writeString((*_iter739)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(FireEventRequest &a, FireEventRequest &b) { + using ::std::swap; + swap(a.successful, b.successful); + swap(a.data, b.data); + swap(a.dbName, b.dbName); + swap(a.tableName, b.tableName); + swap(a.partitionVals, b.partitionVals); + swap(a.__isset, b.__isset); +} + +FireEventRequest::FireEventRequest(const FireEventRequest& other740) { + successful = other740.successful; + data = other740.data; + dbName = other740.dbName; + tableName = other740.tableName; + partitionVals = other740.partitionVals; + __isset = other740.__isset; +} +FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other741) { + successful = other741.successful; + data = other741.data; + dbName = other741.dbName; + tableName = other741.tableName; + partitionVals = other741.partitionVals; + __isset = other741.__isset; + return *this; +} +void FireEventRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "FireEventRequest("; + out << "successful=" << to_string(successful); + out << ", " << "data=" << to_string(data); + out << ", " << "dbName="; (__isset.dbName ? (out << to_string(dbName)) : (out << "")); + out << ", " << "tableName="; (__isset.tableName ? (out << to_string(tableName)) : (out << "")); + out << ", " << "partitionVals="; (__isset.partitionVals ? (out << to_string(partitionVals)) : (out << "")); + out << ")"; +} + + +FireEventResponse::~FireEventResponse() noexcept { +} + +std::ostream& operator<<(std::ostream& out, const FireEventResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t FireEventResponse::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 FireEventResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("FireEventResponse"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(FireEventResponse &a, FireEventResponse &b) { + using ::std::swap; + (void) a; + (void) b; +} + +FireEventResponse::FireEventResponse(const FireEventResponse& other742) { + (void) other742; +} +FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other743) { + (void) other743; + return *this; +} +void FireEventResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "FireEventResponse("; + out << ")"; +} + + +MetadataPpdResult::~MetadataPpdResult() noexcept { +} + + +void MetadataPpdResult::__set_metadata(const std::string& val) { + this->metadata = val; +__isset.metadata = true; +} + +void MetadataPpdResult::__set_includeBitset(const std::string& val) { + this->includeBitset = val; +__isset.includeBitset = true; +} +std::ostream& operator<<(std::ostream& out, const MetadataPpdResult& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t MetadataPpdResult::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readBinary(this->metadata); + this->__isset.metadata = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readBinary(this->includeBitset); + this->__isset.includeBitset = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t MetadataPpdResult::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("MetadataPpdResult"); + + if (this->__isset.metadata) { + xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeBinary(this->metadata); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.includeBitset) { + xfer += oprot->writeFieldBegin("includeBitset", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeBinary(this->includeBitset); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(MetadataPpdResult &a, MetadataPpdResult &b) { + using ::std::swap; + swap(a.metadata, b.metadata); + swap(a.includeBitset, b.includeBitset); + swap(a.__isset, b.__isset); +} + +MetadataPpdResult::MetadataPpdResult(const MetadataPpdResult& other744) { + metadata = other744.metadata; + includeBitset = other744.includeBitset; + __isset = other744.__isset; +} +MetadataPpdResult& MetadataPpdResult::operator=(const MetadataPpdResult& other745) { + metadata = other745.metadata; + includeBitset = other745.includeBitset; + __isset = other745.__isset; + return *this; +} +void MetadataPpdResult::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "MetadataPpdResult("; + out << "metadata="; (__isset.metadata ? (out << to_string(metadata)) : (out << "")); + out << ", " << "includeBitset="; (__isset.includeBitset ? (out << to_string(includeBitset)) : (out << "")); + out << ")"; +} + + +GetFileMetadataByExprResult::~GetFileMetadataByExprResult() noexcept { +} + + +void GetFileMetadataByExprResult::__set_metadata(const std::map & val) { + this->metadata = val; +} + +void GetFileMetadataByExprResult::__set_isSupported(const bool val) { + this->isSupported = val; +} +std::ostream& operator<<(std::ostream& out, const GetFileMetadataByExprResult& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GetFileMetadataByExprResult::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_metadata = false; + bool isset_isSupported = 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_MAP) { + { + this->metadata.clear(); + uint32_t _size746; + ::apache::thrift::protocol::TType _ktype747; + ::apache::thrift::protocol::TType _vtype748; + xfer += iprot->readMapBegin(_ktype747, _vtype748, _size746); + uint32_t _i750; + for (_i750 = 0; _i750 < _size746; ++_i750) + { + int64_t _key751; + xfer += iprot->readI64(_key751); + MetadataPpdResult& _val752 = this->metadata[_key751]; + xfer += _val752.read(iprot); + } + xfer += iprot->readMapEnd(); + } + isset_metadata = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->isSupported); + isset_isSupported = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_metadata) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_isSupported) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t GetFileMetadataByExprResult::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GetFileMetadataByExprResult"); + + xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_MAP, 1); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_I64, ::apache::thrift::protocol::T_STRUCT, static_cast(this->metadata.size())); + std::map ::const_iterator _iter753; + for (_iter753 = this->metadata.begin(); _iter753 != this->metadata.end(); ++_iter753) + { + xfer += oprot->writeI64(_iter753->first); + xfer += _iter753->second.write(oprot); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("isSupported", ::apache::thrift::protocol::T_BOOL, 2); + xfer += oprot->writeBool(this->isSupported); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GetFileMetadataByExprResult &a, GetFileMetadataByExprResult &b) { + using ::std::swap; + swap(a.metadata, b.metadata); + swap(a.isSupported, b.isSupported); +} + +GetFileMetadataByExprResult::GetFileMetadataByExprResult(const GetFileMetadataByExprResult& other754) { + metadata = other754.metadata; + isSupported = other754.isSupported; +} +GetFileMetadataByExprResult& GetFileMetadataByExprResult::operator=(const GetFileMetadataByExprResult& other755) { + metadata = other755.metadata; + isSupported = other755.isSupported; + return *this; +} +void GetFileMetadataByExprResult::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GetFileMetadataByExprResult("; + out << "metadata=" << to_string(metadata); + out << ", " << "isSupported=" << to_string(isSupported); + out << ")"; +} + + +GetFileMetadataByExprRequest::~GetFileMetadataByExprRequest() noexcept { +} + + +void GetFileMetadataByExprRequest::__set_fileIds(const std::vector & val) { + this->fileIds = val; +} + +void GetFileMetadataByExprRequest::__set_expr(const std::string& val) { + this->expr = val; +} + +void GetFileMetadataByExprRequest::__set_doGetFooters(const bool val) { + this->doGetFooters = val; +__isset.doGetFooters = true; +} + +void GetFileMetadataByExprRequest::__set_type(const FileMetadataExprType::type val) { + this->type = val; +__isset.type = true; +} +std::ostream& operator<<(std::ostream& out, const GetFileMetadataByExprRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GetFileMetadataByExprRequest::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_fileIds = false; + bool isset_expr = 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_LIST) { + { + this->fileIds.clear(); + uint32_t _size756; + ::apache::thrift::protocol::TType _etype759; + xfer += iprot->readListBegin(_etype759, _size756); + this->fileIds.resize(_size756); + uint32_t _i760; + for (_i760 = 0; _i760 < _size756; ++_i760) + { + xfer += iprot->readI64(this->fileIds[_i760]); + } + xfer += iprot->readListEnd(); + } + isset_fileIds = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readBinary(this->expr); + isset_expr = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->doGetFooters); + this->__isset.doGetFooters = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast761; + xfer += iprot->readI32(ecast761); + this->type = (FileMetadataExprType::type)ecast761; + this->__isset.type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_fileIds) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_expr) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t GetFileMetadataByExprRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GetFileMetadataByExprRequest"); + + xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->fileIds.size())); + std::vector ::const_iterator _iter762; + for (_iter762 = this->fileIds.begin(); _iter762 != this->fileIds.end(); ++_iter762) + { + xfer += oprot->writeI64((*_iter762)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("expr", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeBinary(this->expr); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.doGetFooters) { + xfer += oprot->writeFieldBegin("doGetFooters", ::apache::thrift::protocol::T_BOOL, 3); + xfer += oprot->writeBool(this->doGetFooters); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.type) { + xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32((int32_t)this->type); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GetFileMetadataByExprRequest &a, GetFileMetadataByExprRequest &b) { + using ::std::swap; + swap(a.fileIds, b.fileIds); + swap(a.expr, b.expr); + swap(a.doGetFooters, b.doGetFooters); + swap(a.type, b.type); + swap(a.__isset, b.__isset); +} + +GetFileMetadataByExprRequest::GetFileMetadataByExprRequest(const GetFileMetadataByExprRequest& other763) { + fileIds = other763.fileIds; + expr = other763.expr; + doGetFooters = other763.doGetFooters; + type = other763.type; + __isset = other763.__isset; +} +GetFileMetadataByExprRequest& GetFileMetadataByExprRequest::operator=(const GetFileMetadataByExprRequest& other764) { + fileIds = other764.fileIds; + expr = other764.expr; + doGetFooters = other764.doGetFooters; + type = other764.type; + __isset = other764.__isset; + return *this; +} +void GetFileMetadataByExprRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GetFileMetadataByExprRequest("; + out << "fileIds=" << to_string(fileIds); + out << ", " << "expr=" << to_string(expr); + out << ", " << "doGetFooters="; (__isset.doGetFooters ? (out << to_string(doGetFooters)) : (out << "")); + out << ", " << "type="; (__isset.type ? (out << to_string(type)) : (out << "")); + out << ")"; +} + + +GetFileMetadataResult::~GetFileMetadataResult() noexcept { +} + + +void GetFileMetadataResult::__set_metadata(const std::map & val) { + this->metadata = val; +} + +void GetFileMetadataResult::__set_isSupported(const bool val) { + this->isSupported = val; +} +std::ostream& operator<<(std::ostream& out, const GetFileMetadataResult& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GetFileMetadataResult::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_metadata = false; + bool isset_isSupported = 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_MAP) { + { + this->metadata.clear(); + uint32_t _size765; + ::apache::thrift::protocol::TType _ktype766; + ::apache::thrift::protocol::TType _vtype767; + xfer += iprot->readMapBegin(_ktype766, _vtype767, _size765); + uint32_t _i769; + for (_i769 = 0; _i769 < _size765; ++_i769) + { + int64_t _key770; + xfer += iprot->readI64(_key770); + std::string& _val771 = this->metadata[_key770]; + xfer += iprot->readBinary(_val771); + } + xfer += iprot->readMapEnd(); + } + isset_metadata = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->isSupported); + isset_isSupported = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_metadata) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_isSupported) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t GetFileMetadataResult::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GetFileMetadataResult"); + + xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_MAP, 1); + { + xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_I64, ::apache::thrift::protocol::T_STRING, static_cast(this->metadata.size())); + std::map ::const_iterator _iter772; + for (_iter772 = this->metadata.begin(); _iter772 != this->metadata.end(); ++_iter772) + { + xfer += oprot->writeI64(_iter772->first); + xfer += oprot->writeBinary(_iter772->second); + } + xfer += oprot->writeMapEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("isSupported", ::apache::thrift::protocol::T_BOOL, 2); + xfer += oprot->writeBool(this->isSupported); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GetFileMetadataResult &a, GetFileMetadataResult &b) { + using ::std::swap; + swap(a.metadata, b.metadata); + swap(a.isSupported, b.isSupported); +} + +GetFileMetadataResult::GetFileMetadataResult(const GetFileMetadataResult& other773) { + metadata = other773.metadata; + isSupported = other773.isSupported; +} +GetFileMetadataResult& GetFileMetadataResult::operator=(const GetFileMetadataResult& other774) { + metadata = other774.metadata; + isSupported = other774.isSupported; + return *this; +} +void GetFileMetadataResult::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GetFileMetadataResult("; + out << "metadata=" << to_string(metadata); + out << ", " << "isSupported=" << to_string(isSupported); + out << ")"; +} + + +GetFileMetadataRequest::~GetFileMetadataRequest() noexcept { +} + + +void GetFileMetadataRequest::__set_fileIds(const std::vector & val) { + this->fileIds = val; +} +std::ostream& operator<<(std::ostream& out, const GetFileMetadataRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GetFileMetadataRequest::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_fileIds = 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_LIST) { + { + this->fileIds.clear(); + uint32_t _size775; + ::apache::thrift::protocol::TType _etype778; + xfer += iprot->readListBegin(_etype778, _size775); + this->fileIds.resize(_size775); + uint32_t _i779; + for (_i779 = 0; _i779 < _size775; ++_i779) + { + xfer += iprot->readI64(this->fileIds[_i779]); + } + xfer += iprot->readListEnd(); + } + isset_fileIds = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_fileIds) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t GetFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GetFileMetadataRequest"); + + xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->fileIds.size())); + std::vector ::const_iterator _iter780; + for (_iter780 = this->fileIds.begin(); _iter780 != this->fileIds.end(); ++_iter780) + { + xfer += oprot->writeI64((*_iter780)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GetFileMetadataRequest &a, GetFileMetadataRequest &b) { + using ::std::swap; + swap(a.fileIds, b.fileIds); +} + +GetFileMetadataRequest::GetFileMetadataRequest(const GetFileMetadataRequest& other781) { + fileIds = other781.fileIds; +} +GetFileMetadataRequest& GetFileMetadataRequest::operator=(const GetFileMetadataRequest& other782) { + fileIds = other782.fileIds; + return *this; +} +void GetFileMetadataRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GetFileMetadataRequest("; + out << "fileIds=" << to_string(fileIds); + out << ")"; +} + + +PutFileMetadataResult::~PutFileMetadataResult() noexcept { +} + +std::ostream& operator<<(std::ostream& out, const PutFileMetadataResult& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PutFileMetadataResult::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 PutFileMetadataResult::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PutFileMetadataResult"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PutFileMetadataResult &a, PutFileMetadataResult &b) { + using ::std::swap; + (void) a; + (void) b; +} + +PutFileMetadataResult::PutFileMetadataResult(const PutFileMetadataResult& other783) { + (void) other783; +} +PutFileMetadataResult& PutFileMetadataResult::operator=(const PutFileMetadataResult& other784) { + (void) other784; + return *this; +} +void PutFileMetadataResult::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PutFileMetadataResult("; + out << ")"; +} + + +PutFileMetadataRequest::~PutFileMetadataRequest() noexcept { +} + + +void PutFileMetadataRequest::__set_fileIds(const std::vector & val) { + this->fileIds = val; +} + +void PutFileMetadataRequest::__set_metadata(const std::vector & val) { + this->metadata = val; +} + +void PutFileMetadataRequest::__set_type(const FileMetadataExprType::type val) { + this->type = val; +__isset.type = true; +} +std::ostream& operator<<(std::ostream& out, const PutFileMetadataRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t PutFileMetadataRequest::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_fileIds = false; + bool isset_metadata = 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_LIST) { + { + this->fileIds.clear(); + uint32_t _size785; + ::apache::thrift::protocol::TType _etype788; + xfer += iprot->readListBegin(_etype788, _size785); + this->fileIds.resize(_size785); + uint32_t _i789; + for (_i789 = 0; _i789 < _size785; ++_i789) + { + xfer += iprot->readI64(this->fileIds[_i789]); + } + xfer += iprot->readListEnd(); + } + isset_fileIds = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->metadata.clear(); + uint32_t _size790; + ::apache::thrift::protocol::TType _etype793; + xfer += iprot->readListBegin(_etype793, _size790); + this->metadata.resize(_size790); + uint32_t _i794; + for (_i794 = 0; _i794 < _size790; ++_i794) + { + xfer += iprot->readBinary(this->metadata[_i794]); + } + xfer += iprot->readListEnd(); + } + isset_metadata = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast795; + xfer += iprot->readI32(ecast795); + this->type = (FileMetadataExprType::type)ecast795; + this->__isset.type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_fileIds) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_metadata) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t PutFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("PutFileMetadataRequest"); + + xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->fileIds.size())); + std::vector ::const_iterator _iter796; + for (_iter796 = this->fileIds.begin(); _iter796 != this->fileIds.end(); ++_iter796) + { + xfer += oprot->writeI64((*_iter796)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->metadata.size())); + std::vector ::const_iterator _iter797; + for (_iter797 = this->metadata.begin(); _iter797 != this->metadata.end(); ++_iter797) + { + xfer += oprot->writeBinary((*_iter797)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + if (this->__isset.type) { + xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_I32, 3); + xfer += oprot->writeI32((int32_t)this->type); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(PutFileMetadataRequest &a, PutFileMetadataRequest &b) { + using ::std::swap; + swap(a.fileIds, b.fileIds); + swap(a.metadata, b.metadata); + swap(a.type, b.type); + swap(a.__isset, b.__isset); +} + +PutFileMetadataRequest::PutFileMetadataRequest(const PutFileMetadataRequest& other798) { + fileIds = other798.fileIds; + metadata = other798.metadata; + type = other798.type; + __isset = other798.__isset; +} +PutFileMetadataRequest& PutFileMetadataRequest::operator=(const PutFileMetadataRequest& other799) { + fileIds = other799.fileIds; + metadata = other799.metadata; + type = other799.type; + __isset = other799.__isset; + return *this; +} +void PutFileMetadataRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "PutFileMetadataRequest("; + out << "fileIds=" << to_string(fileIds); + out << ", " << "metadata=" << to_string(metadata); + out << ", " << "type="; (__isset.type ? (out << to_string(type)) : (out << "")); + out << ")"; +} + + +ClearFileMetadataResult::~ClearFileMetadataResult() noexcept { +} + +std::ostream& operator<<(std::ostream& out, const ClearFileMetadataResult& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t ClearFileMetadataResult::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 ClearFileMetadataResult::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ClearFileMetadataResult"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ClearFileMetadataResult &a, ClearFileMetadataResult &b) { + using ::std::swap; + (void) a; + (void) b; +} + +ClearFileMetadataResult::ClearFileMetadataResult(const ClearFileMetadataResult& other800) { + (void) other800; +} +ClearFileMetadataResult& ClearFileMetadataResult::operator=(const ClearFileMetadataResult& other801) { + (void) other801; + return *this; +} +void ClearFileMetadataResult::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "ClearFileMetadataResult("; + out << ")"; +} + + +ClearFileMetadataRequest::~ClearFileMetadataRequest() noexcept { +} + + +void ClearFileMetadataRequest::__set_fileIds(const std::vector & val) { + this->fileIds = val; +} +std::ostream& operator<<(std::ostream& out, const ClearFileMetadataRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t ClearFileMetadataRequest::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_fileIds = 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_LIST) { + { + this->fileIds.clear(); + uint32_t _size802; + ::apache::thrift::protocol::TType _etype805; + xfer += iprot->readListBegin(_etype805, _size802); + this->fileIds.resize(_size802); + uint32_t _i806; + for (_i806 = 0; _i806 < _size802; ++_i806) + { + xfer += iprot->readI64(this->fileIds[_i806]); + } + xfer += iprot->readListEnd(); + } + isset_fileIds = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_fileIds) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t ClearFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ClearFileMetadataRequest"); + + xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->fileIds.size())); + std::vector ::const_iterator _iter807; + for (_iter807 = this->fileIds.begin(); _iter807 != this->fileIds.end(); ++_iter807) + { + xfer += oprot->writeI64((*_iter807)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ClearFileMetadataRequest &a, ClearFileMetadataRequest &b) { + using ::std::swap; + swap(a.fileIds, b.fileIds); +} + +ClearFileMetadataRequest::ClearFileMetadataRequest(const ClearFileMetadataRequest& other808) { + fileIds = other808.fileIds; +} +ClearFileMetadataRequest& ClearFileMetadataRequest::operator=(const ClearFileMetadataRequest& other809) { + fileIds = other809.fileIds; + return *this; +} +void ClearFileMetadataRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "ClearFileMetadataRequest("; + out << "fileIds=" << to_string(fileIds); + out << ")"; +} + + +CacheFileMetadataResult::~CacheFileMetadataResult() noexcept { +} + + +void CacheFileMetadataResult::__set_isSupported(const bool val) { + this->isSupported = val; +} +std::ostream& operator<<(std::ostream& out, const CacheFileMetadataResult& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t CacheFileMetadataResult::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_isSupported = 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_BOOL) { + xfer += iprot->readBool(this->isSupported); + isset_isSupported = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_isSupported) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t CacheFileMetadataResult::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("CacheFileMetadataResult"); + + xfer += oprot->writeFieldBegin("isSupported", ::apache::thrift::protocol::T_BOOL, 1); + xfer += oprot->writeBool(this->isSupported); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(CacheFileMetadataResult &a, CacheFileMetadataResult &b) { + using ::std::swap; + swap(a.isSupported, b.isSupported); +} + +CacheFileMetadataResult::CacheFileMetadataResult(const CacheFileMetadataResult& other810) { + isSupported = other810.isSupported; +} +CacheFileMetadataResult& CacheFileMetadataResult::operator=(const CacheFileMetadataResult& other811) { + isSupported = other811.isSupported; + return *this; +} +void CacheFileMetadataResult::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "CacheFileMetadataResult("; + out << "isSupported=" << to_string(isSupported); + out << ")"; +} + + +CacheFileMetadataRequest::~CacheFileMetadataRequest() noexcept { +} + + +void CacheFileMetadataRequest::__set_dbName(const std::string& val) { + this->dbName = val; +} + +void CacheFileMetadataRequest::__set_tblName(const std::string& val) { + this->tblName = val; +} + +void CacheFileMetadataRequest::__set_partName(const std::string& val) { + this->partName = val; +__isset.partName = true; +} + +void CacheFileMetadataRequest::__set_isAllParts(const bool val) { + this->isAllParts = val; +__isset.isAllParts = true; +} +std::ostream& operator<<(std::ostream& out, const CacheFileMetadataRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t CacheFileMetadataRequest::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_dbName = false; + bool isset_tblName = 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_STRING) { + xfer += iprot->readString(this->dbName); + isset_dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tblName); + isset_tblName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->partName); + this->__isset.partName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->isAllParts); + this->__isset.isAllParts = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_dbName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tblName) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t CacheFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("CacheFileMetadataRequest"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tblName); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.partName) { + xfer += oprot->writeFieldBegin("partName", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->partName); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.isAllParts) { + xfer += oprot->writeFieldBegin("isAllParts", ::apache::thrift::protocol::T_BOOL, 4); + xfer += oprot->writeBool(this->isAllParts); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(CacheFileMetadataRequest &a, CacheFileMetadataRequest &b) { + using ::std::swap; + swap(a.dbName, b.dbName); + swap(a.tblName, b.tblName); + swap(a.partName, b.partName); + swap(a.isAllParts, b.isAllParts); + swap(a.__isset, b.__isset); +} + +CacheFileMetadataRequest::CacheFileMetadataRequest(const CacheFileMetadataRequest& other812) { + dbName = other812.dbName; + tblName = other812.tblName; + partName = other812.partName; + isAllParts = other812.isAllParts; + __isset = other812.__isset; +} +CacheFileMetadataRequest& CacheFileMetadataRequest::operator=(const CacheFileMetadataRequest& other813) { + dbName = other813.dbName; + tblName = other813.tblName; + partName = other813.partName; + isAllParts = other813.isAllParts; + __isset = other813.__isset; + return *this; +} +void CacheFileMetadataRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "CacheFileMetadataRequest("; + out << "dbName=" << to_string(dbName); + out << ", " << "tblName=" << to_string(tblName); + out << ", " << "partName="; (__isset.partName ? (out << to_string(partName)) : (out << "")); + out << ", " << "isAllParts="; (__isset.isAllParts ? (out << to_string(isAllParts)) : (out << "")); + out << ")"; +} + + +GetAllFunctionsResponse::~GetAllFunctionsResponse() noexcept { +} + + +void GetAllFunctionsResponse::__set_functions(const std::vector & val) { + this->functions = val; +__isset.functions = true; +} +std::ostream& operator<<(std::ostream& out, const GetAllFunctionsResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GetAllFunctionsResponse::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->functions.clear(); + uint32_t _size814; + ::apache::thrift::protocol::TType _etype817; + xfer += iprot->readListBegin(_etype817, _size814); + this->functions.resize(_size814); + uint32_t _i818; + for (_i818 = 0; _i818 < _size814; ++_i818) + { + xfer += this->functions[_i818].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.functions = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t GetAllFunctionsResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GetAllFunctionsResponse"); + + if (this->__isset.functions) { + xfer += oprot->writeFieldBegin("functions", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->functions.size())); + std::vector ::const_iterator _iter819; + for (_iter819 = this->functions.begin(); _iter819 != this->functions.end(); ++_iter819) + { + xfer += (*_iter819).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GetAllFunctionsResponse &a, GetAllFunctionsResponse &b) { + using ::std::swap; + swap(a.functions, b.functions); + swap(a.__isset, b.__isset); +} + +GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other820) { + functions = other820.functions; + __isset = other820.__isset; +} +GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other821) { + functions = other821.functions; + __isset = other821.__isset; + return *this; +} +void GetAllFunctionsResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GetAllFunctionsResponse("; + out << "functions="; (__isset.functions ? (out << to_string(functions)) : (out << "")); + out << ")"; +} + + +ClientCapabilities::~ClientCapabilities() noexcept { +} + + +void ClientCapabilities::__set_values(const std::vector & val) { + this->values = val; +} +std::ostream& operator<<(std::ostream& out, const ClientCapabilities& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t ClientCapabilities::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_values = 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_LIST) { + { + this->values.clear(); + uint32_t _size822; + ::apache::thrift::protocol::TType _etype825; + xfer += iprot->readListBegin(_etype825, _size822); + this->values.resize(_size822); + uint32_t _i826; + for (_i826 = 0; _i826 < _size822; ++_i826) + { + int32_t ecast827; + xfer += iprot->readI32(ecast827); + this->values[_i826] = (ClientCapability::type)ecast827; + } + xfer += iprot->readListEnd(); + } + isset_values = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_values) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t ClientCapabilities::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ClientCapabilities"); + + xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I32, static_cast(this->values.size())); + std::vector ::const_iterator _iter828; + for (_iter828 = this->values.begin(); _iter828 != this->values.end(); ++_iter828) + { + xfer += oprot->writeI32((int32_t)(*_iter828)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ClientCapabilities &a, ClientCapabilities &b) { + using ::std::swap; + swap(a.values, b.values); +} + +ClientCapabilities::ClientCapabilities(const ClientCapabilities& other829) { + values = other829.values; +} +ClientCapabilities& ClientCapabilities::operator=(const ClientCapabilities& other830) { + values = other830.values; + return *this; +} +void ClientCapabilities::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "ClientCapabilities("; + out << "values=" << to_string(values); + out << ")"; +} + + +GetTableRequest::~GetTableRequest() noexcept { +} + + +void GetTableRequest::__set_dbName(const std::string& val) { + this->dbName = val; +} + +void GetTableRequest::__set_tblName(const std::string& val) { + this->tblName = val; +} + +void GetTableRequest::__set_capabilities(const ClientCapabilities& val) { + this->capabilities = val; +__isset.capabilities = true; +} +std::ostream& operator<<(std::ostream& out, const GetTableRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GetTableRequest::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_dbName = false; + bool isset_tblName = 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_STRING) { + xfer += iprot->readString(this->dbName); + isset_dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tblName); + isset_tblName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->capabilities.read(iprot); + this->__isset.capabilities = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_dbName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tblName) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t GetTableRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GetTableRequest"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tblName); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.capabilities) { + xfer += oprot->writeFieldBegin("capabilities", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->capabilities.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GetTableRequest &a, GetTableRequest &b) { + using ::std::swap; + swap(a.dbName, b.dbName); + swap(a.tblName, b.tblName); + swap(a.capabilities, b.capabilities); + swap(a.__isset, b.__isset); +} + +GetTableRequest::GetTableRequest(const GetTableRequest& other831) { + dbName = other831.dbName; + tblName = other831.tblName; + capabilities = other831.capabilities; + __isset = other831.__isset; +} +GetTableRequest& GetTableRequest::operator=(const GetTableRequest& other832) { + dbName = other832.dbName; + tblName = other832.tblName; + capabilities = other832.capabilities; + __isset = other832.__isset; + return *this; +} +void GetTableRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GetTableRequest("; + out << "dbName=" << to_string(dbName); + out << ", " << "tblName=" << to_string(tblName); + out << ", " << "capabilities="; (__isset.capabilities ? (out << to_string(capabilities)) : (out << "")); + out << ")"; +} + + +GetTableResult::~GetTableResult() noexcept { +} + + +void GetTableResult::__set_table(const Table& val) { + this->table = val; +} +std::ostream& operator<<(std::ostream& out, const GetTableResult& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GetTableResult::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_table = 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_STRUCT) { + xfer += this->table.read(iprot); + isset_table = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_table) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t GetTableResult::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GetTableResult"); + + xfer += oprot->writeFieldBegin("table", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->table.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GetTableResult &a, GetTableResult &b) { + using ::std::swap; + swap(a.table, b.table); +} + +GetTableResult::GetTableResult(const GetTableResult& other833) { + table = other833.table; +} +GetTableResult& GetTableResult::operator=(const GetTableResult& other834) { + table = other834.table; + return *this; +} +void GetTableResult::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GetTableResult("; + out << "table=" << to_string(table); + out << ")"; +} + + +GetTablesRequest::~GetTablesRequest() noexcept { +} + + +void GetTablesRequest::__set_dbName(const std::string& val) { + this->dbName = val; +} + +void GetTablesRequest::__set_tblNames(const std::vector & val) { + this->tblNames = val; +__isset.tblNames = true; +} + +void GetTablesRequest::__set_capabilities(const ClientCapabilities& val) { + this->capabilities = val; +__isset.capabilities = true; +} +std::ostream& operator<<(std::ostream& out, const GetTablesRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GetTablesRequest::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_dbName = 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_STRING) { + xfer += iprot->readString(this->dbName); + isset_dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->tblNames.clear(); + uint32_t _size835; + ::apache::thrift::protocol::TType _etype838; + xfer += iprot->readListBegin(_etype838, _size835); + this->tblNames.resize(_size835); + uint32_t _i839; + for (_i839 = 0; _i839 < _size835; ++_i839) + { + xfer += iprot->readString(this->tblNames[_i839]); + } + xfer += iprot->readListEnd(); + } + this->__isset.tblNames = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->capabilities.read(iprot); + this->__isset.capabilities = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_dbName) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t GetTablesRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GetTablesRequest"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.tblNames) { + xfer += oprot->writeFieldBegin("tblNames", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->tblNames.size())); + std::vector ::const_iterator _iter840; + for (_iter840 = this->tblNames.begin(); _iter840 != this->tblNames.end(); ++_iter840) + { + xfer += oprot->writeString((*_iter840)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.capabilities) { + xfer += oprot->writeFieldBegin("capabilities", ::apache::thrift::protocol::T_STRUCT, 3); + xfer += this->capabilities.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GetTablesRequest &a, GetTablesRequest &b) { + using ::std::swap; + swap(a.dbName, b.dbName); + swap(a.tblNames, b.tblNames); + swap(a.capabilities, b.capabilities); + swap(a.__isset, b.__isset); +} + +GetTablesRequest::GetTablesRequest(const GetTablesRequest& other841) { + dbName = other841.dbName; + tblNames = other841.tblNames; + capabilities = other841.capabilities; + __isset = other841.__isset; +} +GetTablesRequest& GetTablesRequest::operator=(const GetTablesRequest& other842) { + dbName = other842.dbName; + tblNames = other842.tblNames; + capabilities = other842.capabilities; + __isset = other842.__isset; + return *this; +} +void GetTablesRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GetTablesRequest("; + out << "dbName=" << to_string(dbName); + out << ", " << "tblNames="; (__isset.tblNames ? (out << to_string(tblNames)) : (out << "")); + out << ", " << "capabilities="; (__isset.capabilities ? (out << to_string(capabilities)) : (out << "")); + out << ")"; +} + + +GetTablesResult::~GetTablesResult() noexcept { +} + + +void GetTablesResult::__set_tables(const std::vector
& val) { + this->tables = val; +} +std::ostream& operator<<(std::ostream& out, const GetTablesResult& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t GetTablesResult::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_tables = 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_LIST) { + { + this->tables.clear(); + uint32_t _size843; + ::apache::thrift::protocol::TType _etype846; + xfer += iprot->readListBegin(_etype846, _size843); + this->tables.resize(_size843); + uint32_t _i847; + for (_i847 = 0; _i847 < _size843; ++_i847) + { + xfer += this->tables[_i847].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_tables = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_tables) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t GetTablesResult::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("GetTablesResult"); + + xfer += oprot->writeFieldBegin("tables", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->tables.size())); + std::vector
::const_iterator _iter848; + for (_iter848 = this->tables.begin(); _iter848 != this->tables.end(); ++_iter848) + { + xfer += (*_iter848).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(GetTablesResult &a, GetTablesResult &b) { + using ::std::swap; + swap(a.tables, b.tables); +} + +GetTablesResult::GetTablesResult(const GetTablesResult& other849) { + tables = other849.tables; +} +GetTablesResult& GetTablesResult::operator=(const GetTablesResult& other850) { + tables = other850.tables; + return *this; +} +void GetTablesResult::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "GetTablesResult("; + out << "tables=" << to_string(tables); + out << ")"; +} + + +CmRecycleRequest::~CmRecycleRequest() noexcept { +} + + +void CmRecycleRequest::__set_dataPath(const std::string& val) { + this->dataPath = val; +} + +void CmRecycleRequest::__set_purge(const bool val) { + this->purge = val; +} +std::ostream& operator<<(std::ostream& out, const CmRecycleRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t CmRecycleRequest::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_dataPath = false; + bool isset_purge = 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_STRING) { + xfer += iprot->readString(this->dataPath); + isset_dataPath = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->purge); + isset_purge = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_dataPath) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_purge) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t CmRecycleRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("CmRecycleRequest"); + + xfer += oprot->writeFieldBegin("dataPath", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dataPath); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("purge", ::apache::thrift::protocol::T_BOOL, 2); + xfer += oprot->writeBool(this->purge); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(CmRecycleRequest &a, CmRecycleRequest &b) { + using ::std::swap; + swap(a.dataPath, b.dataPath); + swap(a.purge, b.purge); +} + +CmRecycleRequest::CmRecycleRequest(const CmRecycleRequest& other851) { + dataPath = other851.dataPath; + purge = other851.purge; +} +CmRecycleRequest& CmRecycleRequest::operator=(const CmRecycleRequest& other852) { + dataPath = other852.dataPath; + purge = other852.purge; + return *this; +} +void CmRecycleRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "CmRecycleRequest("; + out << "dataPath=" << to_string(dataPath); + out << ", " << "purge=" << to_string(purge); + out << ")"; +} + + +CmRecycleResponse::~CmRecycleResponse() noexcept { +} + +std::ostream& operator<<(std::ostream& out, const CmRecycleResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t CmRecycleResponse::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 CmRecycleResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("CmRecycleResponse"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(CmRecycleResponse &a, CmRecycleResponse &b) { + using ::std::swap; + (void) a; + (void) b; +} + +CmRecycleResponse::CmRecycleResponse(const CmRecycleResponse& other853) { + (void) other853; +} +CmRecycleResponse& CmRecycleResponse::operator=(const CmRecycleResponse& other854) { + (void) other854; + return *this; +} +void CmRecycleResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "CmRecycleResponse("; + out << ")"; +} + + +TableMeta::~TableMeta() noexcept { +} + + +void TableMeta::__set_dbName(const std::string& val) { + this->dbName = val; +} + +void TableMeta::__set_tableName(const std::string& val) { + this->tableName = val; +} + +void TableMeta::__set_tableType(const std::string& val) { + this->tableType = val; +} + +void TableMeta::__set_comments(const std::string& val) { + this->comments = val; +__isset.comments = true; +} +std::ostream& operator<<(std::ostream& out, const TableMeta& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t TableMeta::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_dbName = false; + bool isset_tableName = false; + bool isset_tableType = 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_STRING) { + xfer += iprot->readString(this->dbName); + isset_dbName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tableName); + isset_tableName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->tableType); + isset_tableType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->comments); + this->__isset.comments = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_dbName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tableName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_tableType) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t TableMeta::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("TableMeta"); + + xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->dbName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->tableName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->tableType); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.comments) { + xfer += oprot->writeFieldBegin("comments", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->comments); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(TableMeta &a, TableMeta &b) { + using ::std::swap; + swap(a.dbName, b.dbName); + swap(a.tableName, b.tableName); + swap(a.tableType, b.tableType); + swap(a.comments, b.comments); + swap(a.__isset, b.__isset); +} + +TableMeta::TableMeta(const TableMeta& other855) { + dbName = other855.dbName; + tableName = other855.tableName; + tableType = other855.tableType; + comments = other855.comments; + __isset = other855.__isset; +} +TableMeta& TableMeta::operator=(const TableMeta& other856) { + dbName = other856.dbName; + tableName = other856.tableName; + tableType = other856.tableType; + comments = other856.comments; + __isset = other856.__isset; + return *this; +} +void TableMeta::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "TableMeta("; + out << "dbName=" << to_string(dbName); + out << ", " << "tableName=" << to_string(tableName); + out << ", " << "tableType=" << to_string(tableType); + out << ", " << "comments="; (__isset.comments ? (out << to_string(comments)) : (out << "")); + out << ")"; +} + + +WMResourcePlan::~WMResourcePlan() noexcept { +} + + +void WMResourcePlan::__set_name(const std::string& val) { + this->name = val; +} + +void WMResourcePlan::__set_status(const WMResourcePlanStatus::type val) { + this->status = val; +__isset.status = true; +} + +void WMResourcePlan::__set_queryParallelism(const int32_t val) { + this->queryParallelism = val; +__isset.queryParallelism = true; +} + +void WMResourcePlan::__set_defaultPoolPath(const std::string& val) { + this->defaultPoolPath = val; +__isset.defaultPoolPath = true; +} +std::ostream& operator<<(std::ostream& out, const WMResourcePlan& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMResourcePlan::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_name = 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_STRING) { + xfer += iprot->readString(this->name); + isset_name = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast857; + xfer += iprot->readI32(ecast857); + this->status = (WMResourcePlanStatus::type)ecast857; + this->__isset.status = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->queryParallelism); + this->__isset.queryParallelism = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->defaultPoolPath); + this->__isset.defaultPoolPath = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_name) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t WMResourcePlan::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMResourcePlan"); + + xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->name); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.status) { + xfer += oprot->writeFieldBegin("status", ::apache::thrift::protocol::T_I32, 2); + xfer += oprot->writeI32((int32_t)this->status); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.queryParallelism) { + xfer += oprot->writeFieldBegin("queryParallelism", ::apache::thrift::protocol::T_I32, 3); + xfer += oprot->writeI32(this->queryParallelism); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.defaultPoolPath) { + xfer += oprot->writeFieldBegin("defaultPoolPath", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->defaultPoolPath); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMResourcePlan &a, WMResourcePlan &b) { + using ::std::swap; + swap(a.name, b.name); + swap(a.status, b.status); + swap(a.queryParallelism, b.queryParallelism); + swap(a.defaultPoolPath, b.defaultPoolPath); + swap(a.__isset, b.__isset); +} + +WMResourcePlan::WMResourcePlan(const WMResourcePlan& other858) { + name = other858.name; + status = other858.status; + queryParallelism = other858.queryParallelism; + defaultPoolPath = other858.defaultPoolPath; + __isset = other858.__isset; +} +WMResourcePlan& WMResourcePlan::operator=(const WMResourcePlan& other859) { + name = other859.name; + status = other859.status; + queryParallelism = other859.queryParallelism; + defaultPoolPath = other859.defaultPoolPath; + __isset = other859.__isset; + return *this; +} +void WMResourcePlan::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMResourcePlan("; + out << "name=" << to_string(name); + out << ", " << "status="; (__isset.status ? (out << to_string(status)) : (out << "")); + out << ", " << "queryParallelism="; (__isset.queryParallelism ? (out << to_string(queryParallelism)) : (out << "")); + out << ", " << "defaultPoolPath="; (__isset.defaultPoolPath ? (out << to_string(defaultPoolPath)) : (out << "")); + out << ")"; +} + + +WMPool::~WMPool() noexcept { +} + + +void WMPool::__set_resourcePlanName(const std::string& val) { + this->resourcePlanName = val; +} + +void WMPool::__set_poolPath(const std::string& val) { + this->poolPath = val; +} + +void WMPool::__set_allocFraction(const double val) { + this->allocFraction = val; +__isset.allocFraction = true; +} + +void WMPool::__set_queryParallelism(const int32_t val) { + this->queryParallelism = val; +__isset.queryParallelism = true; +} + +void WMPool::__set_schedulingPolicy(const std::string& val) { + this->schedulingPolicy = val; +__isset.schedulingPolicy = true; +} +std::ostream& operator<<(std::ostream& out, const WMPool& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMPool::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_resourcePlanName = false; + bool isset_poolPath = 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_STRING) { + xfer += iprot->readString(this->resourcePlanName); + isset_resourcePlanName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->poolPath); + isset_poolPath = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_DOUBLE) { + xfer += iprot->readDouble(this->allocFraction); + this->__isset.allocFraction = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->queryParallelism); + this->__isset.queryParallelism = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->schedulingPolicy); + this->__isset.schedulingPolicy = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_resourcePlanName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_poolPath) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t WMPool::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMPool"); + + xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->resourcePlanName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("poolPath", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->poolPath); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.allocFraction) { + xfer += oprot->writeFieldBegin("allocFraction", ::apache::thrift::protocol::T_DOUBLE, 3); + xfer += oprot->writeDouble(this->allocFraction); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.queryParallelism) { + xfer += oprot->writeFieldBegin("queryParallelism", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32(this->queryParallelism); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.schedulingPolicy) { + xfer += oprot->writeFieldBegin("schedulingPolicy", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->schedulingPolicy); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMPool &a, WMPool &b) { + using ::std::swap; + swap(a.resourcePlanName, b.resourcePlanName); + swap(a.poolPath, b.poolPath); + swap(a.allocFraction, b.allocFraction); + swap(a.queryParallelism, b.queryParallelism); + swap(a.schedulingPolicy, b.schedulingPolicy); + swap(a.__isset, b.__isset); +} + +WMPool::WMPool(const WMPool& other860) { + resourcePlanName = other860.resourcePlanName; + poolPath = other860.poolPath; + allocFraction = other860.allocFraction; + queryParallelism = other860.queryParallelism; + schedulingPolicy = other860.schedulingPolicy; + __isset = other860.__isset; +} +WMPool& WMPool::operator=(const WMPool& other861) { + resourcePlanName = other861.resourcePlanName; + poolPath = other861.poolPath; + allocFraction = other861.allocFraction; + queryParallelism = other861.queryParallelism; + schedulingPolicy = other861.schedulingPolicy; + __isset = other861.__isset; + return *this; +} +void WMPool::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMPool("; + out << "resourcePlanName=" << to_string(resourcePlanName); + out << ", " << "poolPath=" << to_string(poolPath); + out << ", " << "allocFraction="; (__isset.allocFraction ? (out << to_string(allocFraction)) : (out << "")); + out << ", " << "queryParallelism="; (__isset.queryParallelism ? (out << to_string(queryParallelism)) : (out << "")); + out << ", " << "schedulingPolicy="; (__isset.schedulingPolicy ? (out << to_string(schedulingPolicy)) : (out << "")); + out << ")"; +} + + +WMTrigger::~WMTrigger() noexcept { +} + + +void WMTrigger::__set_resourcePlanName(const std::string& val) { + this->resourcePlanName = val; +} + +void WMTrigger::__set_triggerName(const std::string& val) { + this->triggerName = val; +} + +void WMTrigger::__set_triggerExpression(const std::string& val) { + this->triggerExpression = val; +__isset.triggerExpression = true; +} + +void WMTrigger::__set_actionExpression(const std::string& val) { + this->actionExpression = val; +__isset.actionExpression = true; +} +std::ostream& operator<<(std::ostream& out, const WMTrigger& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMTrigger::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_resourcePlanName = false; + bool isset_triggerName = 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_STRING) { + xfer += iprot->readString(this->resourcePlanName); + isset_resourcePlanName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->triggerName); + isset_triggerName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->triggerExpression); + this->__isset.triggerExpression = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->actionExpression); + this->__isset.actionExpression = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_resourcePlanName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_triggerName) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t WMTrigger::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMTrigger"); + + xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->resourcePlanName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("triggerName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->triggerName); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.triggerExpression) { + xfer += oprot->writeFieldBegin("triggerExpression", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->triggerExpression); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.actionExpression) { + xfer += oprot->writeFieldBegin("actionExpression", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->actionExpression); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMTrigger &a, WMTrigger &b) { + using ::std::swap; + swap(a.resourcePlanName, b.resourcePlanName); + swap(a.triggerName, b.triggerName); + swap(a.triggerExpression, b.triggerExpression); + swap(a.actionExpression, b.actionExpression); + swap(a.__isset, b.__isset); +} + +WMTrigger::WMTrigger(const WMTrigger& other862) { + resourcePlanName = other862.resourcePlanName; + triggerName = other862.triggerName; + triggerExpression = other862.triggerExpression; + actionExpression = other862.actionExpression; + __isset = other862.__isset; +} +WMTrigger& WMTrigger::operator=(const WMTrigger& other863) { + resourcePlanName = other863.resourcePlanName; + triggerName = other863.triggerName; + triggerExpression = other863.triggerExpression; + actionExpression = other863.actionExpression; + __isset = other863.__isset; + return *this; +} +void WMTrigger::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMTrigger("; + out << "resourcePlanName=" << to_string(resourcePlanName); + out << ", " << "triggerName=" << to_string(triggerName); + out << ", " << "triggerExpression="; (__isset.triggerExpression ? (out << to_string(triggerExpression)) : (out << "")); + out << ", " << "actionExpression="; (__isset.actionExpression ? (out << to_string(actionExpression)) : (out << "")); + out << ")"; +} + + +WMMapping::~WMMapping() noexcept { +} + + +void WMMapping::__set_resourcePlanName(const std::string& val) { + this->resourcePlanName = val; +} + +void WMMapping::__set_entityType(const std::string& val) { + this->entityType = val; +} + +void WMMapping::__set_entityName(const std::string& val) { + this->entityName = val; +} + +void WMMapping::__set_poolName(const std::string& val) { + this->poolName = val; +__isset.poolName = true; +} + +void WMMapping::__set_ordering(const int32_t val) { + this->ordering = val; +__isset.ordering = true; +} +std::ostream& operator<<(std::ostream& out, const WMMapping& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMMapping::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_resourcePlanName = false; + bool isset_entityType = false; + bool isset_entityName = 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_STRING) { + xfer += iprot->readString(this->resourcePlanName); + isset_resourcePlanName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->entityType); + isset_entityType = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->entityName); + isset_entityName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->poolName); + this->__isset.poolName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->ordering); + this->__isset.ordering = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_resourcePlanName) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_entityType) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_entityName) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t WMMapping::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMMapping"); + + xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->resourcePlanName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("entityType", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->entityType); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("entityName", ::apache::thrift::protocol::T_STRING, 3); + xfer += oprot->writeString(this->entityName); + xfer += oprot->writeFieldEnd(); + + if (this->__isset.poolName) { + xfer += oprot->writeFieldBegin("poolName", ::apache::thrift::protocol::T_STRING, 4); + xfer += oprot->writeString(this->poolName); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.ordering) { + xfer += oprot->writeFieldBegin("ordering", ::apache::thrift::protocol::T_I32, 5); + xfer += oprot->writeI32(this->ordering); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMMapping &a, WMMapping &b) { + using ::std::swap; + swap(a.resourcePlanName, b.resourcePlanName); + swap(a.entityType, b.entityType); + swap(a.entityName, b.entityName); + swap(a.poolName, b.poolName); + swap(a.ordering, b.ordering); + swap(a.__isset, b.__isset); +} + +WMMapping::WMMapping(const WMMapping& other864) { + resourcePlanName = other864.resourcePlanName; + entityType = other864.entityType; + entityName = other864.entityName; + poolName = other864.poolName; + ordering = other864.ordering; + __isset = other864.__isset; +} +WMMapping& WMMapping::operator=(const WMMapping& other865) { + resourcePlanName = other865.resourcePlanName; + entityType = other865.entityType; + entityName = other865.entityName; + poolName = other865.poolName; + ordering = other865.ordering; + __isset = other865.__isset; + return *this; +} +void WMMapping::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMMapping("; + out << "resourcePlanName=" << to_string(resourcePlanName); + out << ", " << "entityType=" << to_string(entityType); + out << ", " << "entityName=" << to_string(entityName); + out << ", " << "poolName="; (__isset.poolName ? (out << to_string(poolName)) : (out << "")); + out << ", " << "ordering="; (__isset.ordering ? (out << to_string(ordering)) : (out << "")); + out << ")"; +} + + +WMPoolTrigger::~WMPoolTrigger() noexcept { +} + + +void WMPoolTrigger::__set_pool(const std::string& val) { + this->pool = val; +} + +void WMPoolTrigger::__set_trigger(const std::string& val) { + this->trigger = val; +} +std::ostream& operator<<(std::ostream& out, const WMPoolTrigger& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMPoolTrigger::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_pool = false; + bool isset_trigger = 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_STRING) { + xfer += iprot->readString(this->pool); + isset_pool = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->trigger); + isset_trigger = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_pool) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_trigger) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t WMPoolTrigger::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMPoolTrigger"); + + xfer += oprot->writeFieldBegin("pool", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->pool); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("trigger", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->trigger); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMPoolTrigger &a, WMPoolTrigger &b) { + using ::std::swap; + swap(a.pool, b.pool); + swap(a.trigger, b.trigger); +} + +WMPoolTrigger::WMPoolTrigger(const WMPoolTrigger& other866) { + pool = other866.pool; + trigger = other866.trigger; +} +WMPoolTrigger& WMPoolTrigger::operator=(const WMPoolTrigger& other867) { + pool = other867.pool; + trigger = other867.trigger; + return *this; +} +void WMPoolTrigger::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMPoolTrigger("; + out << "pool=" << to_string(pool); + out << ", " << "trigger=" << to_string(trigger); + out << ")"; +} + + +WMFullResourcePlan::~WMFullResourcePlan() noexcept { +} + + +void WMFullResourcePlan::__set_plan(const WMResourcePlan& val) { + this->plan = val; +} + +void WMFullResourcePlan::__set_pools(const std::vector & val) { + this->pools = val; +} + +void WMFullResourcePlan::__set_mappings(const std::vector & val) { + this->mappings = val; +__isset.mappings = true; +} + +void WMFullResourcePlan::__set_triggers(const std::vector & val) { + this->triggers = val; +__isset.triggers = true; +} + +void WMFullResourcePlan::__set_poolTriggers(const std::vector & val) { + this->poolTriggers = val; +__isset.poolTriggers = true; +} +std::ostream& operator<<(std::ostream& out, const WMFullResourcePlan& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMFullResourcePlan::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_plan = false; + bool isset_pools = 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_STRUCT) { + xfer += this->plan.read(iprot); + isset_plan = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->pools.clear(); + uint32_t _size868; + ::apache::thrift::protocol::TType _etype871; + xfer += iprot->readListBegin(_etype871, _size868); + this->pools.resize(_size868); + uint32_t _i872; + for (_i872 = 0; _i872 < _size868; ++_i872) + { + xfer += this->pools[_i872].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_pools = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->mappings.clear(); + uint32_t _size873; + ::apache::thrift::protocol::TType _etype876; + xfer += iprot->readListBegin(_etype876, _size873); + this->mappings.resize(_size873); + uint32_t _i877; + for (_i877 = 0; _i877 < _size873; ++_i877) + { + xfer += this->mappings[_i877].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.mappings = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->triggers.clear(); + uint32_t _size878; + ::apache::thrift::protocol::TType _etype881; + xfer += iprot->readListBegin(_etype881, _size878); + this->triggers.resize(_size878); + uint32_t _i882; + for (_i882 = 0; _i882 < _size878; ++_i882) + { + xfer += this->triggers[_i882].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.triggers = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->poolTriggers.clear(); + uint32_t _size883; + ::apache::thrift::protocol::TType _etype886; + xfer += iprot->readListBegin(_etype886, _size883); + this->poolTriggers.resize(_size883); + uint32_t _i887; + for (_i887 = 0; _i887 < _size883; ++_i887) + { + xfer += this->poolTriggers[_i887].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.poolTriggers = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_plan) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_pools) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t WMFullResourcePlan::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMFullResourcePlan"); + + xfer += oprot->writeFieldBegin("plan", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->plan.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("pools", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->pools.size())); + std::vector ::const_iterator _iter888; + for (_iter888 = this->pools.begin(); _iter888 != this->pools.end(); ++_iter888) + { + xfer += (*_iter888).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + if (this->__isset.mappings) { + xfer += oprot->writeFieldBegin("mappings", ::apache::thrift::protocol::T_LIST, 3); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->mappings.size())); + std::vector ::const_iterator _iter889; + for (_iter889 = this->mappings.begin(); _iter889 != this->mappings.end(); ++_iter889) + { + xfer += (*_iter889).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.triggers) { + xfer += oprot->writeFieldBegin("triggers", ::apache::thrift::protocol::T_LIST, 4); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->triggers.size())); + std::vector ::const_iterator _iter890; + for (_iter890 = this->triggers.begin(); _iter890 != this->triggers.end(); ++_iter890) + { + xfer += (*_iter890).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.poolTriggers) { + xfer += oprot->writeFieldBegin("poolTriggers", ::apache::thrift::protocol::T_LIST, 5); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->poolTriggers.size())); + std::vector ::const_iterator _iter891; + for (_iter891 = this->poolTriggers.begin(); _iter891 != this->poolTriggers.end(); ++_iter891) + { + xfer += (*_iter891).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMFullResourcePlan &a, WMFullResourcePlan &b) { + using ::std::swap; + swap(a.plan, b.plan); + swap(a.pools, b.pools); + swap(a.mappings, b.mappings); + swap(a.triggers, b.triggers); + swap(a.poolTriggers, b.poolTriggers); + swap(a.__isset, b.__isset); +} + +WMFullResourcePlan::WMFullResourcePlan(const WMFullResourcePlan& other892) { + plan = other892.plan; + pools = other892.pools; + mappings = other892.mappings; + triggers = other892.triggers; + poolTriggers = other892.poolTriggers; + __isset = other892.__isset; +} +WMFullResourcePlan& WMFullResourcePlan::operator=(const WMFullResourcePlan& other893) { + plan = other893.plan; + pools = other893.pools; + mappings = other893.mappings; + triggers = other893.triggers; + poolTriggers = other893.poolTriggers; + __isset = other893.__isset; + return *this; +} +void WMFullResourcePlan::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMFullResourcePlan("; + out << "plan=" << to_string(plan); + out << ", " << "pools=" << to_string(pools); + out << ", " << "mappings="; (__isset.mappings ? (out << to_string(mappings)) : (out << "")); + out << ", " << "triggers="; (__isset.triggers ? (out << to_string(triggers)) : (out << "")); + out << ", " << "poolTriggers="; (__isset.poolTriggers ? (out << to_string(poolTriggers)) : (out << "")); + out << ")"; +} + + +WMCreateResourcePlanRequest::~WMCreateResourcePlanRequest() noexcept { +} + + +void WMCreateResourcePlanRequest::__set_resourcePlan(const WMResourcePlan& val) { + this->resourcePlan = val; +__isset.resourcePlan = true; +} +std::ostream& operator<<(std::ostream& out, const WMCreateResourcePlanRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMCreateResourcePlanRequest::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->resourcePlan.read(iprot); + this->__isset.resourcePlan = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t WMCreateResourcePlanRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMCreateResourcePlanRequest"); + + if (this->__isset.resourcePlan) { + xfer += oprot->writeFieldBegin("resourcePlan", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->resourcePlan.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMCreateResourcePlanRequest &a, WMCreateResourcePlanRequest &b) { + using ::std::swap; + swap(a.resourcePlan, b.resourcePlan); + swap(a.__isset, b.__isset); +} + +WMCreateResourcePlanRequest::WMCreateResourcePlanRequest(const WMCreateResourcePlanRequest& other894) { + resourcePlan = other894.resourcePlan; + __isset = other894.__isset; +} +WMCreateResourcePlanRequest& WMCreateResourcePlanRequest::operator=(const WMCreateResourcePlanRequest& other895) { + resourcePlan = other895.resourcePlan; + __isset = other895.__isset; + return *this; +} +void WMCreateResourcePlanRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMCreateResourcePlanRequest("; + out << "resourcePlan="; (__isset.resourcePlan ? (out << to_string(resourcePlan)) : (out << "")); + out << ")"; +} + + +WMCreateResourcePlanResponse::~WMCreateResourcePlanResponse() noexcept { +} + +std::ostream& operator<<(std::ostream& out, const WMCreateResourcePlanResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMCreateResourcePlanResponse::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 WMCreateResourcePlanResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMCreateResourcePlanResponse"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMCreateResourcePlanResponse &a, WMCreateResourcePlanResponse &b) { + using ::std::swap; + (void) a; + (void) b; +} + +WMCreateResourcePlanResponse::WMCreateResourcePlanResponse(const WMCreateResourcePlanResponse& other896) { + (void) other896; +} +WMCreateResourcePlanResponse& WMCreateResourcePlanResponse::operator=(const WMCreateResourcePlanResponse& other897) { + (void) other897; + return *this; +} +void WMCreateResourcePlanResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMCreateResourcePlanResponse("; + out << ")"; +} + + +WMGetActiveResourcePlanRequest::~WMGetActiveResourcePlanRequest() noexcept { +} + +std::ostream& operator<<(std::ostream& out, const WMGetActiveResourcePlanRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMGetActiveResourcePlanRequest::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 WMGetActiveResourcePlanRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMGetActiveResourcePlanRequest"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMGetActiveResourcePlanRequest &a, WMGetActiveResourcePlanRequest &b) { + using ::std::swap; + (void) a; + (void) b; +} + +WMGetActiveResourcePlanRequest::WMGetActiveResourcePlanRequest(const WMGetActiveResourcePlanRequest& other898) { + (void) other898; +} +WMGetActiveResourcePlanRequest& WMGetActiveResourcePlanRequest::operator=(const WMGetActiveResourcePlanRequest& other899) { + (void) other899; + return *this; +} +void WMGetActiveResourcePlanRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMGetActiveResourcePlanRequest("; + out << ")"; +} + + +WMGetActiveResourcePlanResponse::~WMGetActiveResourcePlanResponse() noexcept { +} + + +void WMGetActiveResourcePlanResponse::__set_resourcePlan(const WMFullResourcePlan& val) { + this->resourcePlan = val; +__isset.resourcePlan = true; +} +std::ostream& operator<<(std::ostream& out, const WMGetActiveResourcePlanResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMGetActiveResourcePlanResponse::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->resourcePlan.read(iprot); + this->__isset.resourcePlan = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t WMGetActiveResourcePlanResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMGetActiveResourcePlanResponse"); + + if (this->__isset.resourcePlan) { + xfer += oprot->writeFieldBegin("resourcePlan", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->resourcePlan.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMGetActiveResourcePlanResponse &a, WMGetActiveResourcePlanResponse &b) { + using ::std::swap; + swap(a.resourcePlan, b.resourcePlan); + swap(a.__isset, b.__isset); +} + +WMGetActiveResourcePlanResponse::WMGetActiveResourcePlanResponse(const WMGetActiveResourcePlanResponse& other900) { + resourcePlan = other900.resourcePlan; + __isset = other900.__isset; +} +WMGetActiveResourcePlanResponse& WMGetActiveResourcePlanResponse::operator=(const WMGetActiveResourcePlanResponse& other901) { + resourcePlan = other901.resourcePlan; + __isset = other901.__isset; + return *this; +} +void WMGetActiveResourcePlanResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMGetActiveResourcePlanResponse("; + out << "resourcePlan="; (__isset.resourcePlan ? (out << to_string(resourcePlan)) : (out << "")); + out << ")"; +} + + +WMGetResourcePlanRequest::~WMGetResourcePlanRequest() noexcept { +} + + +void WMGetResourcePlanRequest::__set_resourcePlanName(const std::string& val) { + this->resourcePlanName = val; +__isset.resourcePlanName = true; +} +std::ostream& operator<<(std::ostream& out, const WMGetResourcePlanRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMGetResourcePlanRequest::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->resourcePlanName); + this->__isset.resourcePlanName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t WMGetResourcePlanRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMGetResourcePlanRequest"); + + if (this->__isset.resourcePlanName) { + xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->resourcePlanName); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMGetResourcePlanRequest &a, WMGetResourcePlanRequest &b) { + using ::std::swap; + swap(a.resourcePlanName, b.resourcePlanName); + swap(a.__isset, b.__isset); +} + +WMGetResourcePlanRequest::WMGetResourcePlanRequest(const WMGetResourcePlanRequest& other902) { + resourcePlanName = other902.resourcePlanName; + __isset = other902.__isset; +} +WMGetResourcePlanRequest& WMGetResourcePlanRequest::operator=(const WMGetResourcePlanRequest& other903) { + resourcePlanName = other903.resourcePlanName; + __isset = other903.__isset; + return *this; +} +void WMGetResourcePlanRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMGetResourcePlanRequest("; + out << "resourcePlanName="; (__isset.resourcePlanName ? (out << to_string(resourcePlanName)) : (out << "")); + out << ")"; +} + + +WMGetResourcePlanResponse::~WMGetResourcePlanResponse() noexcept { +} + + +void WMGetResourcePlanResponse::__set_resourcePlan(const WMResourcePlan& val) { + this->resourcePlan = val; +__isset.resourcePlan = true; +} +std::ostream& operator<<(std::ostream& out, const WMGetResourcePlanResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMGetResourcePlanResponse::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->resourcePlan.read(iprot); + this->__isset.resourcePlan = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t WMGetResourcePlanResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMGetResourcePlanResponse"); + + if (this->__isset.resourcePlan) { + xfer += oprot->writeFieldBegin("resourcePlan", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->resourcePlan.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMGetResourcePlanResponse &a, WMGetResourcePlanResponse &b) { + using ::std::swap; + swap(a.resourcePlan, b.resourcePlan); + swap(a.__isset, b.__isset); +} + +WMGetResourcePlanResponse::WMGetResourcePlanResponse(const WMGetResourcePlanResponse& other904) { + resourcePlan = other904.resourcePlan; + __isset = other904.__isset; +} +WMGetResourcePlanResponse& WMGetResourcePlanResponse::operator=(const WMGetResourcePlanResponse& other905) { + resourcePlan = other905.resourcePlan; + __isset = other905.__isset; + return *this; +} +void WMGetResourcePlanResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMGetResourcePlanResponse("; + out << "resourcePlan="; (__isset.resourcePlan ? (out << to_string(resourcePlan)) : (out << "")); + out << ")"; +} + + +WMGetAllResourcePlanRequest::~WMGetAllResourcePlanRequest() noexcept { +} + +std::ostream& operator<<(std::ostream& out, const WMGetAllResourcePlanRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMGetAllResourcePlanRequest::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 WMGetAllResourcePlanRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMGetAllResourcePlanRequest"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMGetAllResourcePlanRequest &a, WMGetAllResourcePlanRequest &b) { + using ::std::swap; + (void) a; + (void) b; +} + +WMGetAllResourcePlanRequest::WMGetAllResourcePlanRequest(const WMGetAllResourcePlanRequest& other906) { + (void) other906; +} +WMGetAllResourcePlanRequest& WMGetAllResourcePlanRequest::operator=(const WMGetAllResourcePlanRequest& other907) { + (void) other907; + return *this; +} +void WMGetAllResourcePlanRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMGetAllResourcePlanRequest("; + out << ")"; +} + + +WMGetAllResourcePlanResponse::~WMGetAllResourcePlanResponse() noexcept { +} + + +void WMGetAllResourcePlanResponse::__set_resourcePlans(const std::vector & val) { + this->resourcePlans = val; +__isset.resourcePlans = true; +} +std::ostream& operator<<(std::ostream& out, const WMGetAllResourcePlanResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMGetAllResourcePlanResponse::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->resourcePlans.clear(); + uint32_t _size908; + ::apache::thrift::protocol::TType _etype911; + xfer += iprot->readListBegin(_etype911, _size908); + this->resourcePlans.resize(_size908); + uint32_t _i912; + for (_i912 = 0; _i912 < _size908; ++_i912) + { + xfer += this->resourcePlans[_i912].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.resourcePlans = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t WMGetAllResourcePlanResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMGetAllResourcePlanResponse"); + + if (this->__isset.resourcePlans) { + xfer += oprot->writeFieldBegin("resourcePlans", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->resourcePlans.size())); + std::vector ::const_iterator _iter913; + for (_iter913 = this->resourcePlans.begin(); _iter913 != this->resourcePlans.end(); ++_iter913) + { + xfer += (*_iter913).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMGetAllResourcePlanResponse &a, WMGetAllResourcePlanResponse &b) { + using ::std::swap; + swap(a.resourcePlans, b.resourcePlans); + swap(a.__isset, b.__isset); +} + +WMGetAllResourcePlanResponse::WMGetAllResourcePlanResponse(const WMGetAllResourcePlanResponse& other914) { + resourcePlans = other914.resourcePlans; + __isset = other914.__isset; +} +WMGetAllResourcePlanResponse& WMGetAllResourcePlanResponse::operator=(const WMGetAllResourcePlanResponse& other915) { + resourcePlans = other915.resourcePlans; + __isset = other915.__isset; + return *this; +} +void WMGetAllResourcePlanResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMGetAllResourcePlanResponse("; + out << "resourcePlans="; (__isset.resourcePlans ? (out << to_string(resourcePlans)) : (out << "")); + out << ")"; +} + + +WMAlterResourcePlanRequest::~WMAlterResourcePlanRequest() noexcept { +} + + +void WMAlterResourcePlanRequest::__set_resourcePlanName(const std::string& val) { + this->resourcePlanName = val; +__isset.resourcePlanName = true; +} + +void WMAlterResourcePlanRequest::__set_resourcePlan(const WMResourcePlan& val) { + this->resourcePlan = val; +__isset.resourcePlan = true; +} + +void WMAlterResourcePlanRequest::__set_isEnableAndActivate(const bool val) { + this->isEnableAndActivate = val; +__isset.isEnableAndActivate = true; +} +std::ostream& operator<<(std::ostream& out, const WMAlterResourcePlanRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMAlterResourcePlanRequest::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->resourcePlanName); + this->__isset.resourcePlanName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->resourcePlan.read(iprot); + this->__isset.resourcePlan = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->isEnableAndActivate); + this->__isset.isEnableAndActivate = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t WMAlterResourcePlanRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMAlterResourcePlanRequest"); + + if (this->__isset.resourcePlanName) { + xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->resourcePlanName); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.resourcePlan) { + xfer += oprot->writeFieldBegin("resourcePlan", ::apache::thrift::protocol::T_STRUCT, 2); + xfer += this->resourcePlan.write(oprot); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.isEnableAndActivate) { + xfer += oprot->writeFieldBegin("isEnableAndActivate", ::apache::thrift::protocol::T_BOOL, 3); + xfer += oprot->writeBool(this->isEnableAndActivate); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMAlterResourcePlanRequest &a, WMAlterResourcePlanRequest &b) { + using ::std::swap; + swap(a.resourcePlanName, b.resourcePlanName); + swap(a.resourcePlan, b.resourcePlan); + swap(a.isEnableAndActivate, b.isEnableAndActivate); + swap(a.__isset, b.__isset); +} + +WMAlterResourcePlanRequest::WMAlterResourcePlanRequest(const WMAlterResourcePlanRequest& other916) { + resourcePlanName = other916.resourcePlanName; + resourcePlan = other916.resourcePlan; + isEnableAndActivate = other916.isEnableAndActivate; + __isset = other916.__isset; +} +WMAlterResourcePlanRequest& WMAlterResourcePlanRequest::operator=(const WMAlterResourcePlanRequest& other917) { + resourcePlanName = other917.resourcePlanName; + resourcePlan = other917.resourcePlan; + isEnableAndActivate = other917.isEnableAndActivate; + __isset = other917.__isset; + return *this; +} +void WMAlterResourcePlanRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMAlterResourcePlanRequest("; + out << "resourcePlanName="; (__isset.resourcePlanName ? (out << to_string(resourcePlanName)) : (out << "")); + out << ", " << "resourcePlan="; (__isset.resourcePlan ? (out << to_string(resourcePlan)) : (out << "")); + out << ", " << "isEnableAndActivate="; (__isset.isEnableAndActivate ? (out << to_string(isEnableAndActivate)) : (out << "")); + out << ")"; +} + + +WMAlterResourcePlanResponse::~WMAlterResourcePlanResponse() noexcept { +} + + +void WMAlterResourcePlanResponse::__set_fullResourcePlan(const WMFullResourcePlan& val) { + this->fullResourcePlan = val; +__isset.fullResourcePlan = true; +} +std::ostream& operator<<(std::ostream& out, const WMAlterResourcePlanResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMAlterResourcePlanResponse::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->fullResourcePlan.read(iprot); + this->__isset.fullResourcePlan = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t WMAlterResourcePlanResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMAlterResourcePlanResponse"); + + if (this->__isset.fullResourcePlan) { + xfer += oprot->writeFieldBegin("fullResourcePlan", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->fullResourcePlan.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMAlterResourcePlanResponse &a, WMAlterResourcePlanResponse &b) { + using ::std::swap; + swap(a.fullResourcePlan, b.fullResourcePlan); + swap(a.__isset, b.__isset); +} + +WMAlterResourcePlanResponse::WMAlterResourcePlanResponse(const WMAlterResourcePlanResponse& other918) { + fullResourcePlan = other918.fullResourcePlan; + __isset = other918.__isset; +} +WMAlterResourcePlanResponse& WMAlterResourcePlanResponse::operator=(const WMAlterResourcePlanResponse& other919) { + fullResourcePlan = other919.fullResourcePlan; + __isset = other919.__isset; + return *this; +} +void WMAlterResourcePlanResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMAlterResourcePlanResponse("; + out << "fullResourcePlan="; (__isset.fullResourcePlan ? (out << to_string(fullResourcePlan)) : (out << "")); + out << ")"; +} + + +WMValidateResourcePlanRequest::~WMValidateResourcePlanRequest() noexcept { +} + + +void WMValidateResourcePlanRequest::__set_resourcePlanName(const std::string& val) { + this->resourcePlanName = val; +__isset.resourcePlanName = true; +} +std::ostream& operator<<(std::ostream& out, const WMValidateResourcePlanRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMValidateResourcePlanRequest::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->resourcePlanName); + this->__isset.resourcePlanName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t WMValidateResourcePlanRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMValidateResourcePlanRequest"); + + if (this->__isset.resourcePlanName) { + xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->resourcePlanName); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMValidateResourcePlanRequest &a, WMValidateResourcePlanRequest &b) { + using ::std::swap; + swap(a.resourcePlanName, b.resourcePlanName); + swap(a.__isset, b.__isset); +} + +WMValidateResourcePlanRequest::WMValidateResourcePlanRequest(const WMValidateResourcePlanRequest& other920) { + resourcePlanName = other920.resourcePlanName; + __isset = other920.__isset; +} +WMValidateResourcePlanRequest& WMValidateResourcePlanRequest::operator=(const WMValidateResourcePlanRequest& other921) { + resourcePlanName = other921.resourcePlanName; + __isset = other921.__isset; + return *this; +} +void WMValidateResourcePlanRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMValidateResourcePlanRequest("; + out << "resourcePlanName="; (__isset.resourcePlanName ? (out << to_string(resourcePlanName)) : (out << "")); + out << ")"; +} + + +WMValidateResourcePlanResponse::~WMValidateResourcePlanResponse() noexcept { +} + + +void WMValidateResourcePlanResponse::__set_isValid(const bool val) { + this->isValid = val; +__isset.isValid = true; +} +std::ostream& operator<<(std::ostream& out, const WMValidateResourcePlanResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMValidateResourcePlanResponse::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_BOOL) { + xfer += iprot->readBool(this->isValid); + this->__isset.isValid = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t WMValidateResourcePlanResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMValidateResourcePlanResponse"); + + if (this->__isset.isValid) { + xfer += oprot->writeFieldBegin("isValid", ::apache::thrift::protocol::T_BOOL, 1); + xfer += oprot->writeBool(this->isValid); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMValidateResourcePlanResponse &a, WMValidateResourcePlanResponse &b) { + using ::std::swap; + swap(a.isValid, b.isValid); + swap(a.__isset, b.__isset); +} + +WMValidateResourcePlanResponse::WMValidateResourcePlanResponse(const WMValidateResourcePlanResponse& other922) { + isValid = other922.isValid; + __isset = other922.__isset; +} +WMValidateResourcePlanResponse& WMValidateResourcePlanResponse::operator=(const WMValidateResourcePlanResponse& other923) { + isValid = other923.isValid; + __isset = other923.__isset; + return *this; +} +void WMValidateResourcePlanResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMValidateResourcePlanResponse("; + out << "isValid="; (__isset.isValid ? (out << to_string(isValid)) : (out << "")); + out << ")"; +} + + +WMDropResourcePlanRequest::~WMDropResourcePlanRequest() noexcept { +} + + +void WMDropResourcePlanRequest::__set_resourcePlanName(const std::string& val) { + this->resourcePlanName = val; +__isset.resourcePlanName = true; +} +std::ostream& operator<<(std::ostream& out, const WMDropResourcePlanRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMDropResourcePlanRequest::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->resourcePlanName); + this->__isset.resourcePlanName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t WMDropResourcePlanRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMDropResourcePlanRequest"); + + if (this->__isset.resourcePlanName) { + xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->resourcePlanName); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMDropResourcePlanRequest &a, WMDropResourcePlanRequest &b) { + using ::std::swap; + swap(a.resourcePlanName, b.resourcePlanName); + swap(a.__isset, b.__isset); +} + +WMDropResourcePlanRequest::WMDropResourcePlanRequest(const WMDropResourcePlanRequest& other924) { + resourcePlanName = other924.resourcePlanName; + __isset = other924.__isset; +} +WMDropResourcePlanRequest& WMDropResourcePlanRequest::operator=(const WMDropResourcePlanRequest& other925) { + resourcePlanName = other925.resourcePlanName; + __isset = other925.__isset; + return *this; +} +void WMDropResourcePlanRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMDropResourcePlanRequest("; + out << "resourcePlanName="; (__isset.resourcePlanName ? (out << to_string(resourcePlanName)) : (out << "")); + out << ")"; +} + + +WMDropResourcePlanResponse::~WMDropResourcePlanResponse() noexcept { +} + +std::ostream& operator<<(std::ostream& out, const WMDropResourcePlanResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMDropResourcePlanResponse::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 WMDropResourcePlanResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMDropResourcePlanResponse"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMDropResourcePlanResponse &a, WMDropResourcePlanResponse &b) { + using ::std::swap; + (void) a; + (void) b; +} + +WMDropResourcePlanResponse::WMDropResourcePlanResponse(const WMDropResourcePlanResponse& other926) { + (void) other926; +} +WMDropResourcePlanResponse& WMDropResourcePlanResponse::operator=(const WMDropResourcePlanResponse& other927) { + (void) other927; + return *this; +} +void WMDropResourcePlanResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMDropResourcePlanResponse("; + out << ")"; +} + + +WMCreateTriggerRequest::~WMCreateTriggerRequest() noexcept { +} + + +void WMCreateTriggerRequest::__set_trigger(const WMTrigger& val) { + this->trigger = val; +__isset.trigger = true; +} +std::ostream& operator<<(std::ostream& out, const WMCreateTriggerRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMCreateTriggerRequest::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->trigger.read(iprot); + this->__isset.trigger = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t WMCreateTriggerRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMCreateTriggerRequest"); + + if (this->__isset.trigger) { + xfer += oprot->writeFieldBegin("trigger", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->trigger.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMCreateTriggerRequest &a, WMCreateTriggerRequest &b) { + using ::std::swap; + swap(a.trigger, b.trigger); + swap(a.__isset, b.__isset); +} + +WMCreateTriggerRequest::WMCreateTriggerRequest(const WMCreateTriggerRequest& other928) { + trigger = other928.trigger; + __isset = other928.__isset; +} +WMCreateTriggerRequest& WMCreateTriggerRequest::operator=(const WMCreateTriggerRequest& other929) { + trigger = other929.trigger; + __isset = other929.__isset; + return *this; +} +void WMCreateTriggerRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMCreateTriggerRequest("; + out << "trigger="; (__isset.trigger ? (out << to_string(trigger)) : (out << "")); + out << ")"; +} + + +WMCreateTriggerResponse::~WMCreateTriggerResponse() noexcept { +} + +std::ostream& operator<<(std::ostream& out, const WMCreateTriggerResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMCreateTriggerResponse::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 WMCreateTriggerResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMCreateTriggerResponse"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMCreateTriggerResponse &a, WMCreateTriggerResponse &b) { + using ::std::swap; + (void) a; + (void) b; +} + +WMCreateTriggerResponse::WMCreateTriggerResponse(const WMCreateTriggerResponse& other930) { + (void) other930; +} +WMCreateTriggerResponse& WMCreateTriggerResponse::operator=(const WMCreateTriggerResponse& other931) { + (void) other931; + return *this; +} +void WMCreateTriggerResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMCreateTriggerResponse("; + out << ")"; +} + + +WMAlterTriggerRequest::~WMAlterTriggerRequest() noexcept { +} + + +void WMAlterTriggerRequest::__set_trigger(const WMTrigger& val) { + this->trigger = val; +__isset.trigger = true; +} +std::ostream& operator<<(std::ostream& out, const WMAlterTriggerRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMAlterTriggerRequest::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->trigger.read(iprot); + this->__isset.trigger = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t WMAlterTriggerRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMAlterTriggerRequest"); + + if (this->__isset.trigger) { + xfer += oprot->writeFieldBegin("trigger", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->trigger.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMAlterTriggerRequest &a, WMAlterTriggerRequest &b) { + using ::std::swap; + swap(a.trigger, b.trigger); + swap(a.__isset, b.__isset); +} + +WMAlterTriggerRequest::WMAlterTriggerRequest(const WMAlterTriggerRequest& other932) { + trigger = other932.trigger; + __isset = other932.__isset; +} +WMAlterTriggerRequest& WMAlterTriggerRequest::operator=(const WMAlterTriggerRequest& other933) { + trigger = other933.trigger; + __isset = other933.__isset; + return *this; +} +void WMAlterTriggerRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMAlterTriggerRequest("; + out << "trigger="; (__isset.trigger ? (out << to_string(trigger)) : (out << "")); + out << ")"; +} + + +WMAlterTriggerResponse::~WMAlterTriggerResponse() noexcept { +} + +std::ostream& operator<<(std::ostream& out, const WMAlterTriggerResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMAlterTriggerResponse::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 WMAlterTriggerResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMAlterTriggerResponse"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMAlterTriggerResponse &a, WMAlterTriggerResponse &b) { + using ::std::swap; + (void) a; + (void) b; +} + +WMAlterTriggerResponse::WMAlterTriggerResponse(const WMAlterTriggerResponse& other934) { + (void) other934; +} +WMAlterTriggerResponse& WMAlterTriggerResponse::operator=(const WMAlterTriggerResponse& other935) { + (void) other935; + return *this; +} +void WMAlterTriggerResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMAlterTriggerResponse("; + out << ")"; +} + + +WMDropTriggerRequest::~WMDropTriggerRequest() noexcept { +} + + +void WMDropTriggerRequest::__set_resourcePlanName(const std::string& val) { + this->resourcePlanName = val; +__isset.resourcePlanName = true; +} + +void WMDropTriggerRequest::__set_triggerName(const std::string& val) { + this->triggerName = val; +__isset.triggerName = true; +} +std::ostream& operator<<(std::ostream& out, const WMDropTriggerRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMDropTriggerRequest::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->resourcePlanName); + this->__isset.resourcePlanName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->triggerName); + this->__isset.triggerName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t WMDropTriggerRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMDropTriggerRequest"); + + if (this->__isset.resourcePlanName) { + xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->resourcePlanName); + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.triggerName) { + xfer += oprot->writeFieldBegin("triggerName", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeString(this->triggerName); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMDropTriggerRequest &a, WMDropTriggerRequest &b) { + using ::std::swap; + swap(a.resourcePlanName, b.resourcePlanName); + swap(a.triggerName, b.triggerName); + swap(a.__isset, b.__isset); +} + +WMDropTriggerRequest::WMDropTriggerRequest(const WMDropTriggerRequest& other936) { + resourcePlanName = other936.resourcePlanName; + triggerName = other936.triggerName; + __isset = other936.__isset; +} +WMDropTriggerRequest& WMDropTriggerRequest::operator=(const WMDropTriggerRequest& other937) { + resourcePlanName = other937.resourcePlanName; + triggerName = other937.triggerName; + __isset = other937.__isset; + return *this; +} +void WMDropTriggerRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMDropTriggerRequest("; + out << "resourcePlanName="; (__isset.resourcePlanName ? (out << to_string(resourcePlanName)) : (out << "")); + out << ", " << "triggerName="; (__isset.triggerName ? (out << to_string(triggerName)) : (out << "")); + out << ")"; +} + + +WMDropTriggerResponse::~WMDropTriggerResponse() noexcept { +} + +std::ostream& operator<<(std::ostream& out, const WMDropTriggerResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMDropTriggerResponse::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 WMDropTriggerResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMDropTriggerResponse"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMDropTriggerResponse &a, WMDropTriggerResponse &b) { + using ::std::swap; + (void) a; + (void) b; +} + +WMDropTriggerResponse::WMDropTriggerResponse(const WMDropTriggerResponse& other938) { + (void) other938; +} +WMDropTriggerResponse& WMDropTriggerResponse::operator=(const WMDropTriggerResponse& other939) { + (void) other939; + return *this; +} +void WMDropTriggerResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMDropTriggerResponse("; + out << ")"; +} + + +WMGetTriggersForResourePlanRequest::~WMGetTriggersForResourePlanRequest() noexcept { +} + + +void WMGetTriggersForResourePlanRequest::__set_resourcePlanName(const std::string& val) { + this->resourcePlanName = val; +__isset.resourcePlanName = true; +} +std::ostream& operator<<(std::ostream& out, const WMGetTriggersForResourePlanRequest& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMGetTriggersForResourePlanRequest::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->resourcePlanName); + this->__isset.resourcePlanName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t WMGetTriggersForResourePlanRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMGetTriggersForResourePlanRequest"); + + if (this->__isset.resourcePlanName) { + xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->resourcePlanName); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMGetTriggersForResourePlanRequest &a, WMGetTriggersForResourePlanRequest &b) { + using ::std::swap; + swap(a.resourcePlanName, b.resourcePlanName); + swap(a.__isset, b.__isset); +} + +WMGetTriggersForResourePlanRequest::WMGetTriggersForResourePlanRequest(const WMGetTriggersForResourePlanRequest& other940) { + resourcePlanName = other940.resourcePlanName; + __isset = other940.__isset; +} +WMGetTriggersForResourePlanRequest& WMGetTriggersForResourePlanRequest::operator=(const WMGetTriggersForResourePlanRequest& other941) { + resourcePlanName = other941.resourcePlanName; + __isset = other941.__isset; + return *this; +} +void WMGetTriggersForResourePlanRequest::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMGetTriggersForResourePlanRequest("; + out << "resourcePlanName="; (__isset.resourcePlanName ? (out << to_string(resourcePlanName)) : (out << "")); + out << ")"; +} + + +WMGetTriggersForResourePlanResponse::~WMGetTriggersForResourePlanResponse() noexcept { +} + + +void WMGetTriggersForResourePlanResponse::__set_triggers(const std::vector & val) { + this->triggers = val; +__isset.triggers = true; +} +std::ostream& operator<<(std::ostream& out, const WMGetTriggersForResourePlanResponse& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t WMGetTriggersForResourePlanResponse::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->triggers.clear(); + uint32_t _size942; + ::apache::thrift::protocol::TType _etype945; + xfer += iprot->readListBegin(_etype945, _size942); + this->triggers.resize(_size942); + uint32_t _i946; + for (_i946 = 0; _i946 < _size942; ++_i946) + { + xfer += this->triggers[_i946].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.triggers = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t WMGetTriggersForResourePlanResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("WMGetTriggersForResourePlanResponse"); + + if (this->__isset.triggers) { + xfer += oprot->writeFieldBegin("triggers", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->triggers.size())); + std::vector ::const_iterator _iter947; + for (_iter947 = this->triggers.begin(); _iter947 != this->triggers.end(); ++_iter947) + { + xfer += (*_iter947).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(WMGetTriggersForResourePlanResponse &a, WMGetTriggersForResourePlanResponse &b) { + using ::std::swap; + swap(a.triggers, b.triggers); + swap(a.__isset, b.__isset); +} + +WMGetTriggersForResourePlanResponse::WMGetTriggersForResourePlanResponse(const WMGetTriggersForResourePlanResponse& other948) { + triggers = other948.triggers; + __isset = other948.__isset; +} +WMGetTriggersForResourePlanResponse& WMGetTriggersForResourePlanResponse::operator=(const WMGetTriggersForResourePlanResponse& other949) { + triggers = other949.triggers; + __isset = other949.__isset; + return *this; +} +void WMGetTriggersForResourePlanResponse::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "WMGetTriggersForResourePlanResponse("; + out << "triggers="; (__isset.triggers ? (out << to_string(triggers)) : (out << "")); + out << ")"; +} + + +MetaException::~MetaException() noexcept { +} + + +void MetaException::__set_message(const std::string& val) { + this->message = val; +} +std::ostream& operator<<(std::ostream& out, const MetaException& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t MetaException::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->message); + this->__isset.message = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t MetaException::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("MetaException"); + + xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(MetaException &a, MetaException &b) { + using ::std::swap; + swap(a.message, b.message); + swap(a.__isset, b.__isset); +} + +MetaException::MetaException(const MetaException& other950) : TException() { + message = other950.message; + __isset = other950.__isset; +} +MetaException& MetaException::operator=(const MetaException& other951) { + message = other951.message; + __isset = other951.__isset; + return *this; +} +void MetaException::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "MetaException("; + out << "message=" << to_string(message); + out << ")"; +} + +const char* MetaException::what() const noexcept { + try { + std::stringstream ss; + ss << "TException - service has thrown: " << *this; + this->thriftTExceptionMessageHolder_ = ss.str(); + return this->thriftTExceptionMessageHolder_.c_str(); + } catch (const std::exception&) { + return "TException - service has thrown: MetaException"; + } +} + + +UnknownTableException::~UnknownTableException() noexcept { +} + + +void UnknownTableException::__set_message(const std::string& val) { + this->message = val; +} +std::ostream& operator<<(std::ostream& out, const UnknownTableException& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t UnknownTableException::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->message); + this->__isset.message = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t UnknownTableException::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("UnknownTableException"); + + xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(UnknownTableException &a, UnknownTableException &b) { + using ::std::swap; + swap(a.message, b.message); + swap(a.__isset, b.__isset); +} + +UnknownTableException::UnknownTableException(const UnknownTableException& other952) : TException() { + message = other952.message; + __isset = other952.__isset; +} +UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other953) { + message = other953.message; + __isset = other953.__isset; + return *this; +} +void UnknownTableException::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "UnknownTableException("; + out << "message=" << to_string(message); + out << ")"; +} + +const char* UnknownTableException::what() const noexcept { + try { + std::stringstream ss; + ss << "TException - service has thrown: " << *this; + this->thriftTExceptionMessageHolder_ = ss.str(); + return this->thriftTExceptionMessageHolder_.c_str(); + } catch (const std::exception&) { + return "TException - service has thrown: UnknownTableException"; + } +} + + +UnknownDBException::~UnknownDBException() noexcept { +} + + +void UnknownDBException::__set_message(const std::string& val) { + this->message = val; +} +std::ostream& operator<<(std::ostream& out, const UnknownDBException& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t UnknownDBException::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->message); + this->__isset.message = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t UnknownDBException::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("UnknownDBException"); + + xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(UnknownDBException &a, UnknownDBException &b) { + using ::std::swap; + swap(a.message, b.message); + swap(a.__isset, b.__isset); +} + +UnknownDBException::UnknownDBException(const UnknownDBException& other954) : TException() { + message = other954.message; + __isset = other954.__isset; +} +UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other955) { + message = other955.message; + __isset = other955.__isset; + return *this; +} +void UnknownDBException::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "UnknownDBException("; + out << "message=" << to_string(message); + out << ")"; +} + +const char* UnknownDBException::what() const noexcept { + try { + std::stringstream ss; + ss << "TException - service has thrown: " << *this; + this->thriftTExceptionMessageHolder_ = ss.str(); + return this->thriftTExceptionMessageHolder_.c_str(); + } catch (const std::exception&) { + return "TException - service has thrown: UnknownDBException"; + } +} + + +AlreadyExistsException::~AlreadyExistsException() noexcept { +} + + +void AlreadyExistsException::__set_message(const std::string& val) { + this->message = val; +} +std::ostream& operator<<(std::ostream& out, const AlreadyExistsException& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t AlreadyExistsException::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->message); + this->__isset.message = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t AlreadyExistsException::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("AlreadyExistsException"); + + xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(AlreadyExistsException &a, AlreadyExistsException &b) { + using ::std::swap; + swap(a.message, b.message); + swap(a.__isset, b.__isset); +} + +AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other956) : TException() { + message = other956.message; + __isset = other956.__isset; +} +AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other957) { + message = other957.message; + __isset = other957.__isset; + return *this; +} +void AlreadyExistsException::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "AlreadyExistsException("; + out << "message=" << to_string(message); + out << ")"; +} + +const char* AlreadyExistsException::what() const noexcept { + try { + std::stringstream ss; + ss << "TException - service has thrown: " << *this; + this->thriftTExceptionMessageHolder_ = ss.str(); + return this->thriftTExceptionMessageHolder_.c_str(); + } catch (const std::exception&) { + return "TException - service has thrown: AlreadyExistsException"; + } +} + + +InvalidPartitionException::~InvalidPartitionException() noexcept { +} + + +void InvalidPartitionException::__set_message(const std::string& val) { + this->message = val; +} +std::ostream& operator<<(std::ostream& out, const InvalidPartitionException& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t InvalidPartitionException::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->message); + this->__isset.message = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t InvalidPartitionException::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("InvalidPartitionException"); + + xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(InvalidPartitionException &a, InvalidPartitionException &b) { + using ::std::swap; + swap(a.message, b.message); + swap(a.__isset, b.__isset); +} + +InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other958) : TException() { + message = other958.message; + __isset = other958.__isset; +} +InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other959) { + message = other959.message; + __isset = other959.__isset; + return *this; +} +void InvalidPartitionException::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "InvalidPartitionException("; + out << "message=" << to_string(message); + out << ")"; +} + +const char* InvalidPartitionException::what() const noexcept { + try { + std::stringstream ss; + ss << "TException - service has thrown: " << *this; + this->thriftTExceptionMessageHolder_ = ss.str(); + return this->thriftTExceptionMessageHolder_.c_str(); + } catch (const std::exception&) { + return "TException - service has thrown: InvalidPartitionException"; + } +} + + +UnknownPartitionException::~UnknownPartitionException() noexcept { +} + + +void UnknownPartitionException::__set_message(const std::string& val) { + this->message = val; +} +std::ostream& operator<<(std::ostream& out, const UnknownPartitionException& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t UnknownPartitionException::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->message); + this->__isset.message = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t UnknownPartitionException::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("UnknownPartitionException"); + + xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(UnknownPartitionException &a, UnknownPartitionException &b) { + using ::std::swap; + swap(a.message, b.message); + swap(a.__isset, b.__isset); +} + +UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other960) : TException() { + message = other960.message; + __isset = other960.__isset; +} +UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other961) { + message = other961.message; + __isset = other961.__isset; + return *this; +} +void UnknownPartitionException::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "UnknownPartitionException("; + out << "message=" << to_string(message); + out << ")"; +} + +const char* UnknownPartitionException::what() const noexcept { + try { + std::stringstream ss; + ss << "TException - service has thrown: " << *this; + this->thriftTExceptionMessageHolder_ = ss.str(); + return this->thriftTExceptionMessageHolder_.c_str(); + } catch (const std::exception&) { + return "TException - service has thrown: UnknownPartitionException"; + } +} + + +InvalidObjectException::~InvalidObjectException() noexcept { +} + + +void InvalidObjectException::__set_message(const std::string& val) { + this->message = val; +} +std::ostream& operator<<(std::ostream& out, const InvalidObjectException& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t InvalidObjectException::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->message); + this->__isset.message = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t InvalidObjectException::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("InvalidObjectException"); + + xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(InvalidObjectException &a, InvalidObjectException &b) { + using ::std::swap; + swap(a.message, b.message); + swap(a.__isset, b.__isset); +} + +InvalidObjectException::InvalidObjectException(const InvalidObjectException& other962) : TException() { + message = other962.message; + __isset = other962.__isset; +} +InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other963) { + message = other963.message; + __isset = other963.__isset; + return *this; +} +void InvalidObjectException::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "InvalidObjectException("; + out << "message=" << to_string(message); + out << ")"; +} + +const char* InvalidObjectException::what() const noexcept { + try { + std::stringstream ss; + ss << "TException - service has thrown: " << *this; + this->thriftTExceptionMessageHolder_ = ss.str(); + return this->thriftTExceptionMessageHolder_.c_str(); + } catch (const std::exception&) { + return "TException - service has thrown: InvalidObjectException"; + } +} + + +NoSuchObjectException::~NoSuchObjectException() noexcept { +} + + +void NoSuchObjectException::__set_message(const std::string& val) { + this->message = val; +} +std::ostream& operator<<(std::ostream& out, const NoSuchObjectException& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t NoSuchObjectException::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->message); + this->__isset.message = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t NoSuchObjectException::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("NoSuchObjectException"); + + xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(NoSuchObjectException &a, NoSuchObjectException &b) { + using ::std::swap; + swap(a.message, b.message); + swap(a.__isset, b.__isset); +} + +NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other964) : TException() { + message = other964.message; + __isset = other964.__isset; +} +NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other965) { + message = other965.message; + __isset = other965.__isset; + return *this; +} +void NoSuchObjectException::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "NoSuchObjectException("; + out << "message=" << to_string(message); + out << ")"; +} + +const char* NoSuchObjectException::what() const noexcept { + try { + std::stringstream ss; + ss << "TException - service has thrown: " << *this; + this->thriftTExceptionMessageHolder_ = ss.str(); + return this->thriftTExceptionMessageHolder_.c_str(); + } catch (const std::exception&) { + return "TException - service has thrown: NoSuchObjectException"; + } +} + + +IndexAlreadyExistsException::~IndexAlreadyExistsException() noexcept { +} + + +void IndexAlreadyExistsException::__set_message(const std::string& val) { + this->message = val; +} +std::ostream& operator<<(std::ostream& out, const IndexAlreadyExistsException& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t IndexAlreadyExistsException::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->message); + this->__isset.message = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t IndexAlreadyExistsException::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("IndexAlreadyExistsException"); + + xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b) { + using ::std::swap; + swap(a.message, b.message); + swap(a.__isset, b.__isset); +} + +IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other966) : TException() { + message = other966.message; + __isset = other966.__isset; +} +IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other967) { + message = other967.message; + __isset = other967.__isset; + return *this; +} +void IndexAlreadyExistsException::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "IndexAlreadyExistsException("; + out << "message=" << to_string(message); + out << ")"; +} + +const char* IndexAlreadyExistsException::what() const noexcept { + try { + std::stringstream ss; + ss << "TException - service has thrown: " << *this; + this->thriftTExceptionMessageHolder_ = ss.str(); + return this->thriftTExceptionMessageHolder_.c_str(); + } catch (const std::exception&) { + return "TException - service has thrown: IndexAlreadyExistsException"; + } +} + + +InvalidOperationException::~InvalidOperationException() noexcept { +} + + +void InvalidOperationException::__set_message(const std::string& val) { + this->message = val; +} +std::ostream& operator<<(std::ostream& out, const InvalidOperationException& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t InvalidOperationException::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->message); + this->__isset.message = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t InvalidOperationException::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("InvalidOperationException"); + + xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(InvalidOperationException &a, InvalidOperationException &b) { + using ::std::swap; + swap(a.message, b.message); + swap(a.__isset, b.__isset); +} + +InvalidOperationException::InvalidOperationException(const InvalidOperationException& other968) : TException() { + message = other968.message; + __isset = other968.__isset; +} +InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other969) { + message = other969.message; + __isset = other969.__isset; + return *this; +} +void InvalidOperationException::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "InvalidOperationException("; + out << "message=" << to_string(message); + out << ")"; +} + +const char* InvalidOperationException::what() const noexcept { + try { + std::stringstream ss; + ss << "TException - service has thrown: " << *this; + this->thriftTExceptionMessageHolder_ = ss.str(); + return this->thriftTExceptionMessageHolder_.c_str(); + } catch (const std::exception&) { + return "TException - service has thrown: InvalidOperationException"; + } +} + + +ConfigValSecurityException::~ConfigValSecurityException() noexcept { +} + + +void ConfigValSecurityException::__set_message(const std::string& val) { + this->message = val; +} +std::ostream& operator<<(std::ostream& out, const ConfigValSecurityException& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t ConfigValSecurityException::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->message); + this->__isset.message = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t ConfigValSecurityException::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("ConfigValSecurityException"); + + xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) { + using ::std::swap; + swap(a.message, b.message); + swap(a.__isset, b.__isset); +} + +ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other970) : TException() { + message = other970.message; + __isset = other970.__isset; +} +ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other971) { + message = other971.message; + __isset = other971.__isset; + return *this; +} +void ConfigValSecurityException::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "ConfigValSecurityException("; + out << "message=" << to_string(message); + out << ")"; +} + +const char* ConfigValSecurityException::what() const noexcept { + try { + std::stringstream ss; + ss << "TException - service has thrown: " << *this; + this->thriftTExceptionMessageHolder_ = ss.str(); + return this->thriftTExceptionMessageHolder_.c_str(); + } catch (const std::exception&) { + return "TException - service has thrown: ConfigValSecurityException"; + } +} + + +InvalidInputException::~InvalidInputException() noexcept { +} + + +void InvalidInputException::__set_message(const std::string& val) { + this->message = val; +} +std::ostream& operator<<(std::ostream& out, const InvalidInputException& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t InvalidInputException::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->message); + this->__isset.message = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t InvalidInputException::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("InvalidInputException"); + + xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(InvalidInputException &a, InvalidInputException &b) { + using ::std::swap; + swap(a.message, b.message); + swap(a.__isset, b.__isset); +} + +InvalidInputException::InvalidInputException(const InvalidInputException& other972) : TException() { + message = other972.message; + __isset = other972.__isset; +} +InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other973) { + message = other973.message; + __isset = other973.__isset; + return *this; +} +void InvalidInputException::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "InvalidInputException("; + out << "message=" << to_string(message); + out << ")"; +} + +const char* InvalidInputException::what() const noexcept { + try { + std::stringstream ss; + ss << "TException - service has thrown: " << *this; + this->thriftTExceptionMessageHolder_ = ss.str(); + return this->thriftTExceptionMessageHolder_.c_str(); + } catch (const std::exception&) { + return "TException - service has thrown: InvalidInputException"; + } +} + + +NoSuchTxnException::~NoSuchTxnException() noexcept { +} + + +void NoSuchTxnException::__set_message(const std::string& val) { + this->message = val; +} +std::ostream& operator<<(std::ostream& out, const NoSuchTxnException& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t NoSuchTxnException::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->message); + this->__isset.message = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t NoSuchTxnException::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("NoSuchTxnException"); + + xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(NoSuchTxnException &a, NoSuchTxnException &b) { + using ::std::swap; + swap(a.message, b.message); + swap(a.__isset, b.__isset); +} + +NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other974) : TException() { + message = other974.message; + __isset = other974.__isset; +} +NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other975) { + message = other975.message; + __isset = other975.__isset; + return *this; +} +void NoSuchTxnException::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "NoSuchTxnException("; + out << "message=" << to_string(message); + out << ")"; +} + +const char* NoSuchTxnException::what() const noexcept { + try { + std::stringstream ss; + ss << "TException - service has thrown: " << *this; + this->thriftTExceptionMessageHolder_ = ss.str(); + return this->thriftTExceptionMessageHolder_.c_str(); + } catch (const std::exception&) { + return "TException - service has thrown: NoSuchTxnException"; + } +} + + +TxnAbortedException::~TxnAbortedException() noexcept { +} + + +void TxnAbortedException::__set_message(const std::string& val) { + this->message = val; +} +std::ostream& operator<<(std::ostream& out, const TxnAbortedException& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t TxnAbortedException::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->message); + this->__isset.message = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t TxnAbortedException::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("TxnAbortedException"); + + xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(TxnAbortedException &a, TxnAbortedException &b) { + using ::std::swap; + swap(a.message, b.message); + swap(a.__isset, b.__isset); +} + +TxnAbortedException::TxnAbortedException(const TxnAbortedException& other976) : TException() { + message = other976.message; + __isset = other976.__isset; +} +TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other977) { + message = other977.message; + __isset = other977.__isset; + return *this; +} +void TxnAbortedException::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "TxnAbortedException("; + out << "message=" << to_string(message); + out << ")"; +} + +const char* TxnAbortedException::what() const noexcept { + try { + std::stringstream ss; + ss << "TException - service has thrown: " << *this; + this->thriftTExceptionMessageHolder_ = ss.str(); + return this->thriftTExceptionMessageHolder_.c_str(); + } catch (const std::exception&) { + return "TException - service has thrown: TxnAbortedException"; + } +} + + +TxnOpenException::~TxnOpenException() noexcept { +} + + +void TxnOpenException::__set_message(const std::string& val) { + this->message = val; +} +std::ostream& operator<<(std::ostream& out, const TxnOpenException& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t TxnOpenException::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->message); + this->__isset.message = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t TxnOpenException::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("TxnOpenException"); + + xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(TxnOpenException &a, TxnOpenException &b) { + using ::std::swap; + swap(a.message, b.message); + swap(a.__isset, b.__isset); +} + +TxnOpenException::TxnOpenException(const TxnOpenException& other978) : TException() { + message = other978.message; + __isset = other978.__isset; +} +TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other979) { + message = other979.message; + __isset = other979.__isset; + return *this; +} +void TxnOpenException::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "TxnOpenException("; + out << "message=" << to_string(message); + out << ")"; +} + +const char* TxnOpenException::what() const noexcept { + try { + std::stringstream ss; + ss << "TException - service has thrown: " << *this; + this->thriftTExceptionMessageHolder_ = ss.str(); + return this->thriftTExceptionMessageHolder_.c_str(); + } catch (const std::exception&) { + return "TException - service has thrown: TxnOpenException"; + } +} + + +NoSuchLockException::~NoSuchLockException() noexcept { +} + + +void NoSuchLockException::__set_message(const std::string& val) { + this->message = val; +} +std::ostream& operator<<(std::ostream& out, const NoSuchLockException& obj) +{ + obj.printTo(out); + return out; +} + + +uint32_t NoSuchLockException::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; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->message); + this->__isset.message = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t NoSuchLockException::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("NoSuchLockException"); + + xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(NoSuchLockException &a, NoSuchLockException &b) { + using ::std::swap; + swap(a.message, b.message); + swap(a.__isset, b.__isset); +} + +NoSuchLockException::NoSuchLockException(const NoSuchLockException& other980) : TException() { + message = other980.message; + __isset = other980.__isset; +} +NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other981) { + message = other981.message; + __isset = other981.__isset; + return *this; +} +void NoSuchLockException::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "NoSuchLockException("; + out << "message=" << to_string(message); + out << ")"; +} + +const char* NoSuchLockException::what() const noexcept { + try { + std::stringstream ss; + ss << "TException - service has thrown: " << *this; + this->thriftTExceptionMessageHolder_ = ss.str(); + return this->thriftTExceptionMessageHolder_.c_str(); + } catch (const std::exception&) { + return "TException - service has thrown: NoSuchLockException"; + } +} + +}}} // namespace diff --git a/contrib/hive-metastore/hive_metastore_types.h b/contrib/hive-metastore/hive_metastore_types.h new file mode 100644 index 00000000000..ceb44329028 --- /dev/null +++ b/contrib/hive-metastore/hive_metastore_types.h @@ -0,0 +1,10002 @@ +/** + * Autogenerated by Thrift Compiler () + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +#ifndef hive_metastore_TYPES_H +#define hive_metastore_TYPES_H + +#include + +#include +#include +#include +#include +#include + +#include +#include + + +namespace Apache { namespace Hadoop { namespace Hive { + +struct HiveObjectType { + enum type { + GLOBAL = 1, + DATABASE = 2, + TABLE = 3, + PARTITION = 4, + COLUMN = 5 + }; +}; + +extern const std::map _HiveObjectType_VALUES_TO_NAMES; + +std::ostream& operator<<(std::ostream& out, const HiveObjectType::type& val); + +struct PrincipalType { + enum type { + USER = 1, + ROLE = 2, + GROUP = 3 + }; +}; + +extern const std::map _PrincipalType_VALUES_TO_NAMES; + +std::ostream& operator<<(std::ostream& out, const PrincipalType::type& val); + +struct PartitionEventType { + enum type { + LOAD_DONE = 1 + }; +}; + +extern const std::map _PartitionEventType_VALUES_TO_NAMES; + +std::ostream& operator<<(std::ostream& out, const PartitionEventType::type& val); + +struct TxnState { + enum type { + COMMITTED = 1, + ABORTED = 2, + OPEN = 3 + }; +}; + +extern const std::map _TxnState_VALUES_TO_NAMES; + +std::ostream& operator<<(std::ostream& out, const TxnState::type& val); + +struct LockLevel { + enum type { + DB = 1, + TABLE = 2, + PARTITION = 3 + }; +}; + +extern const std::map _LockLevel_VALUES_TO_NAMES; + +std::ostream& operator<<(std::ostream& out, const LockLevel::type& val); + +struct LockState { + enum type { + ACQUIRED = 1, + WAITING = 2, + ABORT = 3, + NOT_ACQUIRED = 4 + }; +}; + +extern const std::map _LockState_VALUES_TO_NAMES; + +std::ostream& operator<<(std::ostream& out, const LockState::type& val); + +struct LockType { + enum type { + SHARED_READ = 1, + SHARED_WRITE = 2, + EXCLUSIVE = 3 + }; +}; + +extern const std::map _LockType_VALUES_TO_NAMES; + +std::ostream& operator<<(std::ostream& out, const LockType::type& val); + +struct CompactionType { + enum type { + MINOR = 1, + MAJOR = 2 + }; +}; + +extern const std::map _CompactionType_VALUES_TO_NAMES; + +std::ostream& operator<<(std::ostream& out, const CompactionType::type& val); + +struct GrantRevokeType { + enum type { + GRANT = 1, + REVOKE = 2 + }; +}; + +extern const std::map _GrantRevokeType_VALUES_TO_NAMES; + +std::ostream& operator<<(std::ostream& out, const GrantRevokeType::type& val); + +struct DataOperationType { + enum type { + SELECT = 1, + INSERT = 2, + UPDATE = 3, + DELETE = 4, + UNSET = 5, + NO_TXN = 6 + }; +}; + +extern const std::map _DataOperationType_VALUES_TO_NAMES; + +std::ostream& operator<<(std::ostream& out, const DataOperationType::type& val); + +struct EventRequestType { + enum type { + INSERT = 1, + UPDATE = 2, + DELETE = 3 + }; +}; + +extern const std::map _EventRequestType_VALUES_TO_NAMES; + +std::ostream& operator<<(std::ostream& out, const EventRequestType::type& val); + +struct FunctionType { + enum type { + JAVA = 1 + }; +}; + +extern const std::map _FunctionType_VALUES_TO_NAMES; + +std::ostream& operator<<(std::ostream& out, const FunctionType::type& val); + +struct ResourceType { + enum type { + JAR = 1, + FILE = 2, + ARCHIVE = 3 + }; +}; + +extern const std::map _ResourceType_VALUES_TO_NAMES; + +std::ostream& operator<<(std::ostream& out, const ResourceType::type& val); + +struct FileMetadataExprType { + enum type { + ORC_SARG = 1 + }; +}; + +extern const std::map _FileMetadataExprType_VALUES_TO_NAMES; + +std::ostream& operator<<(std::ostream& out, const FileMetadataExprType::type& val); + +struct ClientCapability { + enum type { + TEST_CAPABILITY = 1, + INSERT_ONLY_TABLES = 2 + }; +}; + +extern const std::map _ClientCapability_VALUES_TO_NAMES; + +std::ostream& operator<<(std::ostream& out, const ClientCapability::type& val); + +struct WMResourcePlanStatus { + enum type { + ACTIVE = 1, + ENABLED = 2, + DISABLED = 3 + }; +}; + +extern const std::map _WMResourcePlanStatus_VALUES_TO_NAMES; + +std::ostream& operator<<(std::ostream& out, const WMResourcePlanStatus::type& val); + +class Version; + +class FieldSchema; + +class SQLPrimaryKey; + +class SQLForeignKey; + +class SQLUniqueConstraint; + +class SQLNotNullConstraint; + +class Type; + +class HiveObjectRef; + +class PrivilegeGrantInfo; + +class HiveObjectPrivilege; + +class PrivilegeBag; + +class PrincipalPrivilegeSet; + +class GrantRevokePrivilegeRequest; + +class GrantRevokePrivilegeResponse; + +class Role; + +class RolePrincipalGrant; + +class GetRoleGrantsForPrincipalRequest; + +class GetRoleGrantsForPrincipalResponse; + +class GetPrincipalsInRoleRequest; + +class GetPrincipalsInRoleResponse; + +class GrantRevokeRoleRequest; + +class GrantRevokeRoleResponse; + +class Database; + +class SerDeInfo; + +class Order; + +class SkewedInfo; + +class StorageDescriptor; + +class Table; + +class Partition; + +class PartitionWithoutSD; + +class PartitionSpecWithSharedSD; + +class PartitionListComposingSpec; + +class PartitionSpec; + +class Index; + +class BooleanColumnStatsData; + +class DoubleColumnStatsData; + +class LongColumnStatsData; + +class StringColumnStatsData; + +class BinaryColumnStatsData; + +class Decimal; + +class DecimalColumnStatsData; + +class Date; + +class DateColumnStatsData; + +class ColumnStatisticsData; + +class ColumnStatisticsObj; + +class ColumnStatisticsDesc; + +class ColumnStatistics; + +class AggrStats; + +class SetPartitionsStatsRequest; + +class Schema; + +class EnvironmentContext; + +class PrimaryKeysRequest; + +class PrimaryKeysResponse; + +class ForeignKeysRequest; + +class ForeignKeysResponse; + +class UniqueConstraintsRequest; + +class UniqueConstraintsResponse; + +class NotNullConstraintsRequest; + +class NotNullConstraintsResponse; + +class DropConstraintRequest; + +class AddPrimaryKeyRequest; + +class AddForeignKeyRequest; + +class AddUniqueConstraintRequest; + +class AddNotNullConstraintRequest; + +class PartitionsByExprResult; + +class PartitionsByExprRequest; + +class TableStatsResult; + +class PartitionsStatsResult; + +class TableStatsRequest; + +class PartitionsStatsRequest; + +class AddPartitionsResult; + +class AddPartitionsRequest; + +class DropPartitionsResult; + +class DropPartitionsExpr; + +class RequestPartsSpec; + +class DropPartitionsRequest; + +class PartitionValuesRequest; + +class PartitionValuesRow; + +class PartitionValuesResponse; + +class ResourceUri; + +class Function; + +class TxnInfo; + +class GetOpenTxnsInfoResponse; + +class GetOpenTxnsResponse; + +class OpenTxnRequest; + +class OpenTxnsResponse; + +class AbortTxnRequest; + +class AbortTxnsRequest; + +class CommitTxnRequest; + +class LockComponent; + +class LockRequest; + +class LockResponse; + +class CheckLockRequest; + +class UnlockRequest; + +class ShowLocksRequest; + +class ShowLocksResponseElement; + +class ShowLocksResponse; + +class HeartbeatRequest; + +class HeartbeatTxnRangeRequest; + +class HeartbeatTxnRangeResponse; + +class CompactionRequest; + +class CompactionResponse; + +class ShowCompactRequest; + +class ShowCompactResponseElement; + +class ShowCompactResponse; + +class AddDynamicPartitions; + +class NotificationEventRequest; + +class NotificationEvent; + +class NotificationEventResponse; + +class CurrentNotificationEventId; + +class NotificationEventsCountRequest; + +class NotificationEventsCountResponse; + +class InsertEventRequestData; + +class FireEventRequestData; + +class FireEventRequest; + +class FireEventResponse; + +class MetadataPpdResult; + +class GetFileMetadataByExprResult; + +class GetFileMetadataByExprRequest; + +class GetFileMetadataResult; + +class GetFileMetadataRequest; + +class PutFileMetadataResult; + +class PutFileMetadataRequest; + +class ClearFileMetadataResult; + +class ClearFileMetadataRequest; + +class CacheFileMetadataResult; + +class CacheFileMetadataRequest; + +class GetAllFunctionsResponse; + +class ClientCapabilities; + +class GetTableRequest; + +class GetTableResult; + +class GetTablesRequest; + +class GetTablesResult; + +class CmRecycleRequest; + +class CmRecycleResponse; + +class TableMeta; + +class WMResourcePlan; + +class WMPool; + +class WMTrigger; + +class WMMapping; + +class WMPoolTrigger; + +class WMFullResourcePlan; + +class WMCreateResourcePlanRequest; + +class WMCreateResourcePlanResponse; + +class WMGetActiveResourcePlanRequest; + +class WMGetActiveResourcePlanResponse; + +class WMGetResourcePlanRequest; + +class WMGetResourcePlanResponse; + +class WMGetAllResourcePlanRequest; + +class WMGetAllResourcePlanResponse; + +class WMAlterResourcePlanRequest; + +class WMAlterResourcePlanResponse; + +class WMValidateResourcePlanRequest; + +class WMValidateResourcePlanResponse; + +class WMDropResourcePlanRequest; + +class WMDropResourcePlanResponse; + +class WMCreateTriggerRequest; + +class WMCreateTriggerResponse; + +class WMAlterTriggerRequest; + +class WMAlterTriggerResponse; + +class WMDropTriggerRequest; + +class WMDropTriggerResponse; + +class WMGetTriggersForResourePlanRequest; + +class WMGetTriggersForResourePlanResponse; + +class MetaException; + +class UnknownTableException; + +class UnknownDBException; + +class AlreadyExistsException; + +class InvalidPartitionException; + +class UnknownPartitionException; + +class InvalidObjectException; + +class NoSuchObjectException; + +class IndexAlreadyExistsException; + +class InvalidOperationException; + +class ConfigValSecurityException; + +class InvalidInputException; + +class NoSuchTxnException; + +class TxnAbortedException; + +class TxnOpenException; + +class NoSuchLockException; + +typedef struct _Version__isset { + _Version__isset() : version(false), comments(false) {} + bool version :1; + bool comments :1; +} _Version__isset; + +class Version : public virtual ::apache::thrift::TBase { + public: + + Version(const Version&); + Version& operator=(const Version&); + Version() : version(), comments() { + } + + virtual ~Version() noexcept; + std::string version; + std::string comments; + + _Version__isset __isset; + + void __set_version(const std::string& val); + + void __set_comments(const std::string& val); + + bool operator == (const Version & rhs) const + { + if (!(version == rhs.version)) + return false; + if (!(comments == rhs.comments)) + return false; + return true; + } + bool operator != (const Version &rhs) const { + return !(*this == rhs); + } + + bool operator < (const Version & ) 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(Version &a, Version &b); + +std::ostream& operator<<(std::ostream& out, const Version& obj); + +typedef struct _FieldSchema__isset { + _FieldSchema__isset() : name(false), type(false), comment(false) {} + bool name :1; + bool type :1; + bool comment :1; +} _FieldSchema__isset; + +class FieldSchema : public virtual ::apache::thrift::TBase { + public: + + FieldSchema(const FieldSchema&); + FieldSchema& operator=(const FieldSchema&); + FieldSchema() : name(), type(), comment() { + } + + virtual ~FieldSchema() noexcept; + std::string name; + std::string type; + std::string comment; + + _FieldSchema__isset __isset; + + void __set_name(const std::string& val); + + void __set_type(const std::string& val); + + void __set_comment(const std::string& val); + + bool operator == (const FieldSchema & rhs) const + { + if (!(name == rhs.name)) + return false; + if (!(type == rhs.type)) + return false; + if (!(comment == rhs.comment)) + return false; + return true; + } + bool operator != (const FieldSchema &rhs) const { + return !(*this == rhs); + } + + bool operator < (const FieldSchema & ) 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(FieldSchema &a, FieldSchema &b); + +std::ostream& operator<<(std::ostream& out, const FieldSchema& obj); + +typedef struct _SQLPrimaryKey__isset { + _SQLPrimaryKey__isset() : table_db(false), table_name(false), column_name(false), key_seq(false), pk_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {} + bool table_db :1; + bool table_name :1; + bool column_name :1; + bool key_seq :1; + bool pk_name :1; + bool enable_cstr :1; + bool validate_cstr :1; + bool rely_cstr :1; +} _SQLPrimaryKey__isset; + +class SQLPrimaryKey : public virtual ::apache::thrift::TBase { + public: + + SQLPrimaryKey(const SQLPrimaryKey&); + SQLPrimaryKey& operator=(const SQLPrimaryKey&); + SQLPrimaryKey() : table_db(), table_name(), column_name(), key_seq(0), pk_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) { + } + + virtual ~SQLPrimaryKey() noexcept; + std::string table_db; + std::string table_name; + std::string column_name; + int32_t key_seq; + std::string pk_name; + bool enable_cstr; + bool validate_cstr; + bool rely_cstr; + + _SQLPrimaryKey__isset __isset; + + void __set_table_db(const std::string& val); + + void __set_table_name(const std::string& val); + + void __set_column_name(const std::string& val); + + void __set_key_seq(const int32_t val); + + void __set_pk_name(const std::string& val); + + void __set_enable_cstr(const bool val); + + void __set_validate_cstr(const bool val); + + void __set_rely_cstr(const bool val); + + bool operator == (const SQLPrimaryKey & rhs) const + { + if (!(table_db == rhs.table_db)) + return false; + if (!(table_name == rhs.table_name)) + return false; + if (!(column_name == rhs.column_name)) + return false; + if (!(key_seq == rhs.key_seq)) + return false; + if (!(pk_name == rhs.pk_name)) + return false; + if (!(enable_cstr == rhs.enable_cstr)) + return false; + if (!(validate_cstr == rhs.validate_cstr)) + return false; + if (!(rely_cstr == rhs.rely_cstr)) + return false; + return true; + } + bool operator != (const SQLPrimaryKey &rhs) const { + return !(*this == rhs); + } + + bool operator < (const SQLPrimaryKey & ) 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(SQLPrimaryKey &a, SQLPrimaryKey &b); + +std::ostream& operator<<(std::ostream& out, const SQLPrimaryKey& obj); + +typedef struct _SQLForeignKey__isset { + _SQLForeignKey__isset() : pktable_db(false), pktable_name(false), pkcolumn_name(false), fktable_db(false), fktable_name(false), fkcolumn_name(false), key_seq(false), update_rule(false), delete_rule(false), fk_name(false), pk_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {} + bool pktable_db :1; + bool pktable_name :1; + bool pkcolumn_name :1; + bool fktable_db :1; + bool fktable_name :1; + bool fkcolumn_name :1; + bool key_seq :1; + bool update_rule :1; + bool delete_rule :1; + bool fk_name :1; + bool pk_name :1; + bool enable_cstr :1; + bool validate_cstr :1; + bool rely_cstr :1; +} _SQLForeignKey__isset; + +class SQLForeignKey : public virtual ::apache::thrift::TBase { + public: + + SQLForeignKey(const SQLForeignKey&); + SQLForeignKey& operator=(const SQLForeignKey&); + SQLForeignKey() : pktable_db(), pktable_name(), pkcolumn_name(), fktable_db(), fktable_name(), fkcolumn_name(), key_seq(0), update_rule(0), delete_rule(0), fk_name(), pk_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) { + } + + virtual ~SQLForeignKey() noexcept; + std::string pktable_db; + std::string pktable_name; + std::string pkcolumn_name; + std::string fktable_db; + std::string fktable_name; + std::string fkcolumn_name; + int32_t key_seq; + int32_t update_rule; + int32_t delete_rule; + std::string fk_name; + std::string pk_name; + bool enable_cstr; + bool validate_cstr; + bool rely_cstr; + + _SQLForeignKey__isset __isset; + + void __set_pktable_db(const std::string& val); + + void __set_pktable_name(const std::string& val); + + void __set_pkcolumn_name(const std::string& val); + + void __set_fktable_db(const std::string& val); + + void __set_fktable_name(const std::string& val); + + void __set_fkcolumn_name(const std::string& val); + + void __set_key_seq(const int32_t val); + + void __set_update_rule(const int32_t val); + + void __set_delete_rule(const int32_t val); + + void __set_fk_name(const std::string& val); + + void __set_pk_name(const std::string& val); + + void __set_enable_cstr(const bool val); + + void __set_validate_cstr(const bool val); + + void __set_rely_cstr(const bool val); + + bool operator == (const SQLForeignKey & rhs) const + { + if (!(pktable_db == rhs.pktable_db)) + return false; + if (!(pktable_name == rhs.pktable_name)) + return false; + if (!(pkcolumn_name == rhs.pkcolumn_name)) + return false; + if (!(fktable_db == rhs.fktable_db)) + return false; + if (!(fktable_name == rhs.fktable_name)) + return false; + if (!(fkcolumn_name == rhs.fkcolumn_name)) + return false; + if (!(key_seq == rhs.key_seq)) + return false; + if (!(update_rule == rhs.update_rule)) + return false; + if (!(delete_rule == rhs.delete_rule)) + return false; + if (!(fk_name == rhs.fk_name)) + return false; + if (!(pk_name == rhs.pk_name)) + return false; + if (!(enable_cstr == rhs.enable_cstr)) + return false; + if (!(validate_cstr == rhs.validate_cstr)) + return false; + if (!(rely_cstr == rhs.rely_cstr)) + return false; + return true; + } + bool operator != (const SQLForeignKey &rhs) const { + return !(*this == rhs); + } + + bool operator < (const SQLForeignKey & ) 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(SQLForeignKey &a, SQLForeignKey &b); + +std::ostream& operator<<(std::ostream& out, const SQLForeignKey& obj); + +typedef struct _SQLUniqueConstraint__isset { + _SQLUniqueConstraint__isset() : table_db(false), table_name(false), column_name(false), key_seq(false), uk_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {} + bool table_db :1; + bool table_name :1; + bool column_name :1; + bool key_seq :1; + bool uk_name :1; + bool enable_cstr :1; + bool validate_cstr :1; + bool rely_cstr :1; +} _SQLUniqueConstraint__isset; + +class SQLUniqueConstraint : public virtual ::apache::thrift::TBase { + public: + + SQLUniqueConstraint(const SQLUniqueConstraint&); + SQLUniqueConstraint& operator=(const SQLUniqueConstraint&); + SQLUniqueConstraint() : table_db(), table_name(), column_name(), key_seq(0), uk_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) { + } + + virtual ~SQLUniqueConstraint() noexcept; + std::string table_db; + std::string table_name; + std::string column_name; + int32_t key_seq; + std::string uk_name; + bool enable_cstr; + bool validate_cstr; + bool rely_cstr; + + _SQLUniqueConstraint__isset __isset; + + void __set_table_db(const std::string& val); + + void __set_table_name(const std::string& val); + + void __set_column_name(const std::string& val); + + void __set_key_seq(const int32_t val); + + void __set_uk_name(const std::string& val); + + void __set_enable_cstr(const bool val); + + void __set_validate_cstr(const bool val); + + void __set_rely_cstr(const bool val); + + bool operator == (const SQLUniqueConstraint & rhs) const + { + if (!(table_db == rhs.table_db)) + return false; + if (!(table_name == rhs.table_name)) + return false; + if (!(column_name == rhs.column_name)) + return false; + if (!(key_seq == rhs.key_seq)) + return false; + if (!(uk_name == rhs.uk_name)) + return false; + if (!(enable_cstr == rhs.enable_cstr)) + return false; + if (!(validate_cstr == rhs.validate_cstr)) + return false; + if (!(rely_cstr == rhs.rely_cstr)) + return false; + return true; + } + bool operator != (const SQLUniqueConstraint &rhs) const { + return !(*this == rhs); + } + + bool operator < (const SQLUniqueConstraint & ) 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(SQLUniqueConstraint &a, SQLUniqueConstraint &b); + +std::ostream& operator<<(std::ostream& out, const SQLUniqueConstraint& obj); + +typedef struct _SQLNotNullConstraint__isset { + _SQLNotNullConstraint__isset() : table_db(false), table_name(false), column_name(false), nn_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {} + bool table_db :1; + bool table_name :1; + bool column_name :1; + bool nn_name :1; + bool enable_cstr :1; + bool validate_cstr :1; + bool rely_cstr :1; +} _SQLNotNullConstraint__isset; + +class SQLNotNullConstraint : public virtual ::apache::thrift::TBase { + public: + + SQLNotNullConstraint(const SQLNotNullConstraint&); + SQLNotNullConstraint& operator=(const SQLNotNullConstraint&); + SQLNotNullConstraint() : table_db(), table_name(), column_name(), nn_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) { + } + + virtual ~SQLNotNullConstraint() noexcept; + std::string table_db; + std::string table_name; + std::string column_name; + std::string nn_name; + bool enable_cstr; + bool validate_cstr; + bool rely_cstr; + + _SQLNotNullConstraint__isset __isset; + + void __set_table_db(const std::string& val); + + void __set_table_name(const std::string& val); + + void __set_column_name(const std::string& val); + + void __set_nn_name(const std::string& val); + + void __set_enable_cstr(const bool val); + + void __set_validate_cstr(const bool val); + + void __set_rely_cstr(const bool val); + + bool operator == (const SQLNotNullConstraint & rhs) const + { + if (!(table_db == rhs.table_db)) + return false; + if (!(table_name == rhs.table_name)) + return false; + if (!(column_name == rhs.column_name)) + return false; + if (!(nn_name == rhs.nn_name)) + return false; + if (!(enable_cstr == rhs.enable_cstr)) + return false; + if (!(validate_cstr == rhs.validate_cstr)) + return false; + if (!(rely_cstr == rhs.rely_cstr)) + return false; + return true; + } + bool operator != (const SQLNotNullConstraint &rhs) const { + return !(*this == rhs); + } + + bool operator < (const SQLNotNullConstraint & ) 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(SQLNotNullConstraint &a, SQLNotNullConstraint &b); + +std::ostream& operator<<(std::ostream& out, const SQLNotNullConstraint& obj); + +typedef struct _Type__isset { + _Type__isset() : name(false), type1(false), type2(false), fields(false) {} + bool name :1; + bool type1 :1; + bool type2 :1; + bool fields :1; +} _Type__isset; + +class Type : public virtual ::apache::thrift::TBase { + public: + + Type(const Type&); + Type& operator=(const Type&); + Type() : name(), type1(), type2() { + } + + virtual ~Type() noexcept; + std::string name; + std::string type1; + std::string type2; + std::vector fields; + + _Type__isset __isset; + + void __set_name(const std::string& val); + + void __set_type1(const std::string& val); + + void __set_type2(const std::string& val); + + void __set_fields(const std::vector & val); + + bool operator == (const Type & rhs) const + { + if (!(name == rhs.name)) + return false; + if (__isset.type1 != rhs.__isset.type1) + return false; + else if (__isset.type1 && !(type1 == rhs.type1)) + return false; + if (__isset.type2 != rhs.__isset.type2) + return false; + else if (__isset.type2 && !(type2 == rhs.type2)) + return false; + if (__isset.fields != rhs.__isset.fields) + return false; + else if (__isset.fields && !(fields == rhs.fields)) + return false; + return true; + } + bool operator != (const Type &rhs) const { + return !(*this == rhs); + } + + bool operator < (const Type & ) 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(Type &a, Type &b); + +std::ostream& operator<<(std::ostream& out, const Type& obj); + +typedef struct _HiveObjectRef__isset { + _HiveObjectRef__isset() : objectType(false), dbName(false), objectName(false), partValues(false), columnName(false) {} + bool objectType :1; + bool dbName :1; + bool objectName :1; + bool partValues :1; + bool columnName :1; +} _HiveObjectRef__isset; + +class HiveObjectRef : public virtual ::apache::thrift::TBase { + public: + + HiveObjectRef(const HiveObjectRef&); + HiveObjectRef& operator=(const HiveObjectRef&); + HiveObjectRef() : objectType((HiveObjectType::type)0), dbName(), objectName(), columnName() { + } + + virtual ~HiveObjectRef() noexcept; + HiveObjectType::type objectType; + std::string dbName; + std::string objectName; + std::vector partValues; + std::string columnName; + + _HiveObjectRef__isset __isset; + + void __set_objectType(const HiveObjectType::type val); + + void __set_dbName(const std::string& val); + + void __set_objectName(const std::string& val); + + void __set_partValues(const std::vector & val); + + void __set_columnName(const std::string& val); + + bool operator == (const HiveObjectRef & rhs) const + { + if (!(objectType == rhs.objectType)) + return false; + if (!(dbName == rhs.dbName)) + return false; + if (!(objectName == rhs.objectName)) + return false; + if (!(partValues == rhs.partValues)) + return false; + if (!(columnName == rhs.columnName)) + return false; + return true; + } + bool operator != (const HiveObjectRef &rhs) const { + return !(*this == rhs); + } + + bool operator < (const HiveObjectRef & ) 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(HiveObjectRef &a, HiveObjectRef &b); + +std::ostream& operator<<(std::ostream& out, const HiveObjectRef& obj); + +typedef struct _PrivilegeGrantInfo__isset { + _PrivilegeGrantInfo__isset() : privilege(false), createTime(false), grantor(false), grantorType(false), grantOption(false) {} + bool privilege :1; + bool createTime :1; + bool grantor :1; + bool grantorType :1; + bool grantOption :1; +} _PrivilegeGrantInfo__isset; + +class PrivilegeGrantInfo : public virtual ::apache::thrift::TBase { + public: + + PrivilegeGrantInfo(const PrivilegeGrantInfo&); + PrivilegeGrantInfo& operator=(const PrivilegeGrantInfo&); + PrivilegeGrantInfo() : privilege(), createTime(0), grantor(), grantorType((PrincipalType::type)0), grantOption(0) { + } + + virtual ~PrivilegeGrantInfo() noexcept; + std::string privilege; + int32_t createTime; + std::string grantor; + PrincipalType::type grantorType; + bool grantOption; + + _PrivilegeGrantInfo__isset __isset; + + void __set_privilege(const std::string& val); + + void __set_createTime(const int32_t val); + + void __set_grantor(const std::string& val); + + void __set_grantorType(const PrincipalType::type val); + + void __set_grantOption(const bool val); + + bool operator == (const PrivilegeGrantInfo & rhs) const + { + if (!(privilege == rhs.privilege)) + return false; + if (!(createTime == rhs.createTime)) + return false; + if (!(grantor == rhs.grantor)) + return false; + if (!(grantorType == rhs.grantorType)) + return false; + if (!(grantOption == rhs.grantOption)) + return false; + return true; + } + bool operator != (const PrivilegeGrantInfo &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PrivilegeGrantInfo & ) 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(PrivilegeGrantInfo &a, PrivilegeGrantInfo &b); + +std::ostream& operator<<(std::ostream& out, const PrivilegeGrantInfo& obj); + +typedef struct _HiveObjectPrivilege__isset { + _HiveObjectPrivilege__isset() : hiveObject(false), principalName(false), principalType(false), grantInfo(false) {} + bool hiveObject :1; + bool principalName :1; + bool principalType :1; + bool grantInfo :1; +} _HiveObjectPrivilege__isset; + +class HiveObjectPrivilege : public virtual ::apache::thrift::TBase { + public: + + HiveObjectPrivilege(const HiveObjectPrivilege&); + HiveObjectPrivilege& operator=(const HiveObjectPrivilege&); + HiveObjectPrivilege() : principalName(), principalType((PrincipalType::type)0) { + } + + virtual ~HiveObjectPrivilege() noexcept; + HiveObjectRef hiveObject; + std::string principalName; + PrincipalType::type principalType; + PrivilegeGrantInfo grantInfo; + + _HiveObjectPrivilege__isset __isset; + + void __set_hiveObject(const HiveObjectRef& val); + + void __set_principalName(const std::string& val); + + void __set_principalType(const PrincipalType::type val); + + void __set_grantInfo(const PrivilegeGrantInfo& val); + + bool operator == (const HiveObjectPrivilege & rhs) const + { + if (!(hiveObject == rhs.hiveObject)) + return false; + if (!(principalName == rhs.principalName)) + return false; + if (!(principalType == rhs.principalType)) + return false; + if (!(grantInfo == rhs.grantInfo)) + return false; + return true; + } + bool operator != (const HiveObjectPrivilege &rhs) const { + return !(*this == rhs); + } + + bool operator < (const HiveObjectPrivilege & ) 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(HiveObjectPrivilege &a, HiveObjectPrivilege &b); + +std::ostream& operator<<(std::ostream& out, const HiveObjectPrivilege& obj); + +typedef struct _PrivilegeBag__isset { + _PrivilegeBag__isset() : privileges(false) {} + bool privileges :1; +} _PrivilegeBag__isset; + +class PrivilegeBag : public virtual ::apache::thrift::TBase { + public: + + PrivilegeBag(const PrivilegeBag&); + PrivilegeBag& operator=(const PrivilegeBag&); + PrivilegeBag() { + } + + virtual ~PrivilegeBag() noexcept; + std::vector privileges; + + _PrivilegeBag__isset __isset; + + void __set_privileges(const std::vector & val); + + bool operator == (const PrivilegeBag & rhs) const + { + if (!(privileges == rhs.privileges)) + return false; + return true; + } + bool operator != (const PrivilegeBag &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PrivilegeBag & ) 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(PrivilegeBag &a, PrivilegeBag &b); + +std::ostream& operator<<(std::ostream& out, const PrivilegeBag& obj); + +typedef struct _PrincipalPrivilegeSet__isset { + _PrincipalPrivilegeSet__isset() : userPrivileges(false), groupPrivileges(false), rolePrivileges(false) {} + bool userPrivileges :1; + bool groupPrivileges :1; + bool rolePrivileges :1; +} _PrincipalPrivilegeSet__isset; + +class PrincipalPrivilegeSet : public virtual ::apache::thrift::TBase { + public: + + PrincipalPrivilegeSet(const PrincipalPrivilegeSet&); + PrincipalPrivilegeSet& operator=(const PrincipalPrivilegeSet&); + PrincipalPrivilegeSet() { + } + + virtual ~PrincipalPrivilegeSet() noexcept; + std::map > userPrivileges; + std::map > groupPrivileges; + std::map > rolePrivileges; + + _PrincipalPrivilegeSet__isset __isset; + + void __set_userPrivileges(const std::map > & val); + + void __set_groupPrivileges(const std::map > & val); + + void __set_rolePrivileges(const std::map > & val); + + bool operator == (const PrincipalPrivilegeSet & rhs) const + { + if (!(userPrivileges == rhs.userPrivileges)) + return false; + if (!(groupPrivileges == rhs.groupPrivileges)) + return false; + if (!(rolePrivileges == rhs.rolePrivileges)) + return false; + return true; + } + bool operator != (const PrincipalPrivilegeSet &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PrincipalPrivilegeSet & ) 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(PrincipalPrivilegeSet &a, PrincipalPrivilegeSet &b); + +std::ostream& operator<<(std::ostream& out, const PrincipalPrivilegeSet& obj); + +typedef struct _GrantRevokePrivilegeRequest__isset { + _GrantRevokePrivilegeRequest__isset() : requestType(false), privileges(false), revokeGrantOption(false) {} + bool requestType :1; + bool privileges :1; + bool revokeGrantOption :1; +} _GrantRevokePrivilegeRequest__isset; + +class GrantRevokePrivilegeRequest : public virtual ::apache::thrift::TBase { + public: + + GrantRevokePrivilegeRequest(const GrantRevokePrivilegeRequest&); + GrantRevokePrivilegeRequest& operator=(const GrantRevokePrivilegeRequest&); + GrantRevokePrivilegeRequest() : requestType((GrantRevokeType::type)0), revokeGrantOption(0) { + } + + virtual ~GrantRevokePrivilegeRequest() noexcept; + GrantRevokeType::type requestType; + PrivilegeBag privileges; + bool revokeGrantOption; + + _GrantRevokePrivilegeRequest__isset __isset; + + void __set_requestType(const GrantRevokeType::type val); + + void __set_privileges(const PrivilegeBag& val); + + void __set_revokeGrantOption(const bool val); + + bool operator == (const GrantRevokePrivilegeRequest & rhs) const + { + if (!(requestType == rhs.requestType)) + return false; + if (!(privileges == rhs.privileges)) + return false; + if (__isset.revokeGrantOption != rhs.__isset.revokeGrantOption) + return false; + else if (__isset.revokeGrantOption && !(revokeGrantOption == rhs.revokeGrantOption)) + return false; + return true; + } + bool operator != (const GrantRevokePrivilegeRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GrantRevokePrivilegeRequest & ) 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(GrantRevokePrivilegeRequest &a, GrantRevokePrivilegeRequest &b); + +std::ostream& operator<<(std::ostream& out, const GrantRevokePrivilegeRequest& obj); + +typedef struct _GrantRevokePrivilegeResponse__isset { + _GrantRevokePrivilegeResponse__isset() : success(false) {} + bool success :1; +} _GrantRevokePrivilegeResponse__isset; + +class GrantRevokePrivilegeResponse : public virtual ::apache::thrift::TBase { + public: + + GrantRevokePrivilegeResponse(const GrantRevokePrivilegeResponse&); + GrantRevokePrivilegeResponse& operator=(const GrantRevokePrivilegeResponse&); + GrantRevokePrivilegeResponse() : success(0) { + } + + virtual ~GrantRevokePrivilegeResponse() noexcept; + bool success; + + _GrantRevokePrivilegeResponse__isset __isset; + + void __set_success(const bool val); + + bool operator == (const GrantRevokePrivilegeResponse & rhs) const + { + if (__isset.success != rhs.__isset.success) + return false; + else if (__isset.success && !(success == rhs.success)) + return false; + return true; + } + bool operator != (const GrantRevokePrivilegeResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GrantRevokePrivilegeResponse & ) 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(GrantRevokePrivilegeResponse &a, GrantRevokePrivilegeResponse &b); + +std::ostream& operator<<(std::ostream& out, const GrantRevokePrivilegeResponse& obj); + +typedef struct _Role__isset { + _Role__isset() : roleName(false), createTime(false), ownerName(false) {} + bool roleName :1; + bool createTime :1; + bool ownerName :1; +} _Role__isset; + +class Role : public virtual ::apache::thrift::TBase { + public: + + Role(const Role&); + Role& operator=(const Role&); + Role() : roleName(), createTime(0), ownerName() { + } + + virtual ~Role() noexcept; + std::string roleName; + int32_t createTime; + std::string ownerName; + + _Role__isset __isset; + + void __set_roleName(const std::string& val); + + void __set_createTime(const int32_t val); + + void __set_ownerName(const std::string& val); + + bool operator == (const Role & rhs) const + { + if (!(roleName == rhs.roleName)) + return false; + if (!(createTime == rhs.createTime)) + return false; + if (!(ownerName == rhs.ownerName)) + return false; + return true; + } + bool operator != (const Role &rhs) const { + return !(*this == rhs); + } + + bool operator < (const Role & ) 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(Role &a, Role &b); + +std::ostream& operator<<(std::ostream& out, const Role& obj); + +typedef struct _RolePrincipalGrant__isset { + _RolePrincipalGrant__isset() : roleName(false), principalName(false), principalType(false), grantOption(false), grantTime(false), grantorName(false), grantorPrincipalType(false) {} + bool roleName :1; + bool principalName :1; + bool principalType :1; + bool grantOption :1; + bool grantTime :1; + bool grantorName :1; + bool grantorPrincipalType :1; +} _RolePrincipalGrant__isset; + +class RolePrincipalGrant : public virtual ::apache::thrift::TBase { + public: + + RolePrincipalGrant(const RolePrincipalGrant&); + RolePrincipalGrant& operator=(const RolePrincipalGrant&); + RolePrincipalGrant() : roleName(), principalName(), principalType((PrincipalType::type)0), grantOption(0), grantTime(0), grantorName(), grantorPrincipalType((PrincipalType::type)0) { + } + + virtual ~RolePrincipalGrant() noexcept; + std::string roleName; + std::string principalName; + PrincipalType::type principalType; + bool grantOption; + int32_t grantTime; + std::string grantorName; + PrincipalType::type grantorPrincipalType; + + _RolePrincipalGrant__isset __isset; + + void __set_roleName(const std::string& val); + + void __set_principalName(const std::string& val); + + void __set_principalType(const PrincipalType::type val); + + void __set_grantOption(const bool val); + + void __set_grantTime(const int32_t val); + + void __set_grantorName(const std::string& val); + + void __set_grantorPrincipalType(const PrincipalType::type val); + + bool operator == (const RolePrincipalGrant & rhs) const + { + if (!(roleName == rhs.roleName)) + return false; + if (!(principalName == rhs.principalName)) + return false; + if (!(principalType == rhs.principalType)) + return false; + if (!(grantOption == rhs.grantOption)) + return false; + if (!(grantTime == rhs.grantTime)) + return false; + if (!(grantorName == rhs.grantorName)) + return false; + if (!(grantorPrincipalType == rhs.grantorPrincipalType)) + return false; + return true; + } + bool operator != (const RolePrincipalGrant &rhs) const { + return !(*this == rhs); + } + + bool operator < (const RolePrincipalGrant & ) 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(RolePrincipalGrant &a, RolePrincipalGrant &b); + +std::ostream& operator<<(std::ostream& out, const RolePrincipalGrant& obj); + + +class GetRoleGrantsForPrincipalRequest : public virtual ::apache::thrift::TBase { + public: + + GetRoleGrantsForPrincipalRequest(const GetRoleGrantsForPrincipalRequest&); + GetRoleGrantsForPrincipalRequest& operator=(const GetRoleGrantsForPrincipalRequest&); + GetRoleGrantsForPrincipalRequest() : principal_name(), principal_type((PrincipalType::type)0) { + } + + virtual ~GetRoleGrantsForPrincipalRequest() noexcept; + std::string principal_name; + PrincipalType::type principal_type; + + void __set_principal_name(const std::string& val); + + void __set_principal_type(const PrincipalType::type val); + + bool operator == (const GetRoleGrantsForPrincipalRequest & rhs) const + { + if (!(principal_name == rhs.principal_name)) + return false; + if (!(principal_type == rhs.principal_type)) + return false; + return true; + } + bool operator != (const GetRoleGrantsForPrincipalRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GetRoleGrantsForPrincipalRequest & ) 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(GetRoleGrantsForPrincipalRequest &a, GetRoleGrantsForPrincipalRequest &b); + +std::ostream& operator<<(std::ostream& out, const GetRoleGrantsForPrincipalRequest& obj); + + +class GetRoleGrantsForPrincipalResponse : public virtual ::apache::thrift::TBase { + public: + + GetRoleGrantsForPrincipalResponse(const GetRoleGrantsForPrincipalResponse&); + GetRoleGrantsForPrincipalResponse& operator=(const GetRoleGrantsForPrincipalResponse&); + GetRoleGrantsForPrincipalResponse() { + } + + virtual ~GetRoleGrantsForPrincipalResponse() noexcept; + std::vector principalGrants; + + void __set_principalGrants(const std::vector & val); + + bool operator == (const GetRoleGrantsForPrincipalResponse & rhs) const + { + if (!(principalGrants == rhs.principalGrants)) + return false; + return true; + } + bool operator != (const GetRoleGrantsForPrincipalResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GetRoleGrantsForPrincipalResponse & ) 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(GetRoleGrantsForPrincipalResponse &a, GetRoleGrantsForPrincipalResponse &b); + +std::ostream& operator<<(std::ostream& out, const GetRoleGrantsForPrincipalResponse& obj); + + +class GetPrincipalsInRoleRequest : public virtual ::apache::thrift::TBase { + public: + + GetPrincipalsInRoleRequest(const GetPrincipalsInRoleRequest&); + GetPrincipalsInRoleRequest& operator=(const GetPrincipalsInRoleRequest&); + GetPrincipalsInRoleRequest() : roleName() { + } + + virtual ~GetPrincipalsInRoleRequest() noexcept; + std::string roleName; + + void __set_roleName(const std::string& val); + + bool operator == (const GetPrincipalsInRoleRequest & rhs) const + { + if (!(roleName == rhs.roleName)) + return false; + return true; + } + bool operator != (const GetPrincipalsInRoleRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GetPrincipalsInRoleRequest & ) 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(GetPrincipalsInRoleRequest &a, GetPrincipalsInRoleRequest &b); + +std::ostream& operator<<(std::ostream& out, const GetPrincipalsInRoleRequest& obj); + + +class GetPrincipalsInRoleResponse : public virtual ::apache::thrift::TBase { + public: + + GetPrincipalsInRoleResponse(const GetPrincipalsInRoleResponse&); + GetPrincipalsInRoleResponse& operator=(const GetPrincipalsInRoleResponse&); + GetPrincipalsInRoleResponse() { + } + + virtual ~GetPrincipalsInRoleResponse() noexcept; + std::vector principalGrants; + + void __set_principalGrants(const std::vector & val); + + bool operator == (const GetPrincipalsInRoleResponse & rhs) const + { + if (!(principalGrants == rhs.principalGrants)) + return false; + return true; + } + bool operator != (const GetPrincipalsInRoleResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GetPrincipalsInRoleResponse & ) 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(GetPrincipalsInRoleResponse &a, GetPrincipalsInRoleResponse &b); + +std::ostream& operator<<(std::ostream& out, const GetPrincipalsInRoleResponse& obj); + +typedef struct _GrantRevokeRoleRequest__isset { + _GrantRevokeRoleRequest__isset() : requestType(false), roleName(false), principalName(false), principalType(false), grantor(false), grantorType(false), grantOption(false) {} + bool requestType :1; + bool roleName :1; + bool principalName :1; + bool principalType :1; + bool grantor :1; + bool grantorType :1; + bool grantOption :1; +} _GrantRevokeRoleRequest__isset; + +class GrantRevokeRoleRequest : public virtual ::apache::thrift::TBase { + public: + + GrantRevokeRoleRequest(const GrantRevokeRoleRequest&); + GrantRevokeRoleRequest& operator=(const GrantRevokeRoleRequest&); + GrantRevokeRoleRequest() : requestType((GrantRevokeType::type)0), roleName(), principalName(), principalType((PrincipalType::type)0), grantor(), grantorType((PrincipalType::type)0), grantOption(0) { + } + + virtual ~GrantRevokeRoleRequest() noexcept; + GrantRevokeType::type requestType; + std::string roleName; + std::string principalName; + PrincipalType::type principalType; + std::string grantor; + PrincipalType::type grantorType; + bool grantOption; + + _GrantRevokeRoleRequest__isset __isset; + + void __set_requestType(const GrantRevokeType::type val); + + void __set_roleName(const std::string& val); + + void __set_principalName(const std::string& val); + + void __set_principalType(const PrincipalType::type val); + + void __set_grantor(const std::string& val); + + void __set_grantorType(const PrincipalType::type val); + + void __set_grantOption(const bool val); + + bool operator == (const GrantRevokeRoleRequest & rhs) const + { + if (!(requestType == rhs.requestType)) + return false; + if (!(roleName == rhs.roleName)) + return false; + if (!(principalName == rhs.principalName)) + return false; + if (!(principalType == rhs.principalType)) + return false; + if (__isset.grantor != rhs.__isset.grantor) + return false; + else if (__isset.grantor && !(grantor == rhs.grantor)) + return false; + if (__isset.grantorType != rhs.__isset.grantorType) + return false; + else if (__isset.grantorType && !(grantorType == rhs.grantorType)) + return false; + if (__isset.grantOption != rhs.__isset.grantOption) + return false; + else if (__isset.grantOption && !(grantOption == rhs.grantOption)) + return false; + return true; + } + bool operator != (const GrantRevokeRoleRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GrantRevokeRoleRequest & ) 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(GrantRevokeRoleRequest &a, GrantRevokeRoleRequest &b); + +std::ostream& operator<<(std::ostream& out, const GrantRevokeRoleRequest& obj); + +typedef struct _GrantRevokeRoleResponse__isset { + _GrantRevokeRoleResponse__isset() : success(false) {} + bool success :1; +} _GrantRevokeRoleResponse__isset; + +class GrantRevokeRoleResponse : public virtual ::apache::thrift::TBase { + public: + + GrantRevokeRoleResponse(const GrantRevokeRoleResponse&); + GrantRevokeRoleResponse& operator=(const GrantRevokeRoleResponse&); + GrantRevokeRoleResponse() : success(0) { + } + + virtual ~GrantRevokeRoleResponse() noexcept; + bool success; + + _GrantRevokeRoleResponse__isset __isset; + + void __set_success(const bool val); + + bool operator == (const GrantRevokeRoleResponse & rhs) const + { + if (__isset.success != rhs.__isset.success) + return false; + else if (__isset.success && !(success == rhs.success)) + return false; + return true; + } + bool operator != (const GrantRevokeRoleResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GrantRevokeRoleResponse & ) 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(GrantRevokeRoleResponse &a, GrantRevokeRoleResponse &b); + +std::ostream& operator<<(std::ostream& out, const GrantRevokeRoleResponse& obj); + +typedef struct _Database__isset { + _Database__isset() : name(false), description(false), locationUri(false), parameters(false), privileges(false), ownerName(false), ownerType(false) {} + bool name :1; + bool description :1; + bool locationUri :1; + bool parameters :1; + bool privileges :1; + bool ownerName :1; + bool ownerType :1; +} _Database__isset; + +class Database : public virtual ::apache::thrift::TBase { + public: + + Database(const Database&); + Database& operator=(const Database&); + Database() : name(), description(), locationUri(), ownerName(), ownerType((PrincipalType::type)0) { + } + + virtual ~Database() noexcept; + std::string name; + std::string description; + std::string locationUri; + std::map parameters; + PrincipalPrivilegeSet privileges; + std::string ownerName; + PrincipalType::type ownerType; + + _Database__isset __isset; + + void __set_name(const std::string& val); + + void __set_description(const std::string& val); + + void __set_locationUri(const std::string& val); + + void __set_parameters(const std::map & val); + + void __set_privileges(const PrincipalPrivilegeSet& val); + + void __set_ownerName(const std::string& val); + + void __set_ownerType(const PrincipalType::type val); + + bool operator == (const Database & rhs) const + { + if (!(name == rhs.name)) + return false; + if (!(description == rhs.description)) + return false; + if (!(locationUri == rhs.locationUri)) + return false; + if (!(parameters == rhs.parameters)) + return false; + if (__isset.privileges != rhs.__isset.privileges) + return false; + else if (__isset.privileges && !(privileges == rhs.privileges)) + return false; + if (__isset.ownerName != rhs.__isset.ownerName) + return false; + else if (__isset.ownerName && !(ownerName == rhs.ownerName)) + return false; + if (__isset.ownerType != rhs.__isset.ownerType) + return false; + else if (__isset.ownerType && !(ownerType == rhs.ownerType)) + return false; + return true; + } + bool operator != (const Database &rhs) const { + return !(*this == rhs); + } + + bool operator < (const Database & ) 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(Database &a, Database &b); + +std::ostream& operator<<(std::ostream& out, const Database& obj); + +typedef struct _SerDeInfo__isset { + _SerDeInfo__isset() : name(false), serializationLib(false), parameters(false) {} + bool name :1; + bool serializationLib :1; + bool parameters :1; +} _SerDeInfo__isset; + +class SerDeInfo : public virtual ::apache::thrift::TBase { + public: + + SerDeInfo(const SerDeInfo&); + SerDeInfo& operator=(const SerDeInfo&); + SerDeInfo() : name(), serializationLib() { + } + + virtual ~SerDeInfo() noexcept; + std::string name; + std::string serializationLib; + std::map parameters; + + _SerDeInfo__isset __isset; + + void __set_name(const std::string& val); + + void __set_serializationLib(const std::string& val); + + void __set_parameters(const std::map & val); + + bool operator == (const SerDeInfo & rhs) const + { + if (!(name == rhs.name)) + return false; + if (!(serializationLib == rhs.serializationLib)) + return false; + if (!(parameters == rhs.parameters)) + return false; + return true; + } + bool operator != (const SerDeInfo &rhs) const { + return !(*this == rhs); + } + + bool operator < (const SerDeInfo & ) 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(SerDeInfo &a, SerDeInfo &b); + +std::ostream& operator<<(std::ostream& out, const SerDeInfo& obj); + +typedef struct _Order__isset { + _Order__isset() : col(false), order(false) {} + bool col :1; + bool order :1; +} _Order__isset; + +class Order : public virtual ::apache::thrift::TBase { + public: + + Order(const Order&); + Order& operator=(const Order&); + Order() : col(), order(0) { + } + + virtual ~Order() noexcept; + std::string col; + int32_t order; + + _Order__isset __isset; + + void __set_col(const std::string& val); + + void __set_order(const int32_t val); + + bool operator == (const Order & rhs) const + { + if (!(col == rhs.col)) + return false; + if (!(order == rhs.order)) + return false; + return true; + } + bool operator != (const Order &rhs) const { + return !(*this == rhs); + } + + bool operator < (const Order & ) 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(Order &a, Order &b); + +std::ostream& operator<<(std::ostream& out, const Order& obj); + +typedef struct _SkewedInfo__isset { + _SkewedInfo__isset() : skewedColNames(false), skewedColValues(false), skewedColValueLocationMaps(false) {} + bool skewedColNames :1; + bool skewedColValues :1; + bool skewedColValueLocationMaps :1; +} _SkewedInfo__isset; + +class SkewedInfo : public virtual ::apache::thrift::TBase { + public: + + SkewedInfo(const SkewedInfo&); + SkewedInfo& operator=(const SkewedInfo&); + SkewedInfo() { + } + + virtual ~SkewedInfo() noexcept; + std::vector skewedColNames; + std::vector > skewedColValues; + std::map skewedColValueLocationMaps; + + _SkewedInfo__isset __isset; + + void __set_skewedColNames(const std::vector & val); + + void __set_skewedColValues(const std::vector > & val); + + void __set_skewedColValueLocationMaps(const std::map & val); + + bool operator == (const SkewedInfo & rhs) const + { + if (!(skewedColNames == rhs.skewedColNames)) + return false; + if (!(skewedColValues == rhs.skewedColValues)) + return false; + if (!(skewedColValueLocationMaps == rhs.skewedColValueLocationMaps)) + return false; + return true; + } + bool operator != (const SkewedInfo &rhs) const { + return !(*this == rhs); + } + + bool operator < (const SkewedInfo & ) 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(SkewedInfo &a, SkewedInfo &b); + +std::ostream& operator<<(std::ostream& out, const SkewedInfo& obj); + +typedef struct _StorageDescriptor__isset { + _StorageDescriptor__isset() : cols(false), location(false), inputFormat(false), outputFormat(false), compressed(false), numBuckets(false), serdeInfo(false), bucketCols(false), sortCols(false), parameters(false), skewedInfo(false), storedAsSubDirectories(false) {} + bool cols :1; + bool location :1; + bool inputFormat :1; + bool outputFormat :1; + bool compressed :1; + bool numBuckets :1; + bool serdeInfo :1; + bool bucketCols :1; + bool sortCols :1; + bool parameters :1; + bool skewedInfo :1; + bool storedAsSubDirectories :1; +} _StorageDescriptor__isset; + +class StorageDescriptor : public virtual ::apache::thrift::TBase { + public: + + StorageDescriptor(const StorageDescriptor&); + StorageDescriptor& operator=(const StorageDescriptor&); + StorageDescriptor() : location(), inputFormat(), outputFormat(), compressed(0), numBuckets(0), storedAsSubDirectories(0) { + } + + virtual ~StorageDescriptor() noexcept; + std::vector cols; + std::string location; + std::string inputFormat; + std::string outputFormat; + bool compressed; + int32_t numBuckets; + SerDeInfo serdeInfo; + std::vector bucketCols; + std::vector sortCols; + std::map parameters; + SkewedInfo skewedInfo; + bool storedAsSubDirectories; + + _StorageDescriptor__isset __isset; + + void __set_cols(const std::vector & val); + + void __set_location(const std::string& val); + + void __set_inputFormat(const std::string& val); + + void __set_outputFormat(const std::string& val); + + void __set_compressed(const bool val); + + void __set_numBuckets(const int32_t val); + + void __set_serdeInfo(const SerDeInfo& val); + + void __set_bucketCols(const std::vector & val); + + void __set_sortCols(const std::vector & val); + + void __set_parameters(const std::map & val); + + void __set_skewedInfo(const SkewedInfo& val); + + void __set_storedAsSubDirectories(const bool val); + + bool operator == (const StorageDescriptor & rhs) const + { + if (!(cols == rhs.cols)) + return false; + if (!(location == rhs.location)) + return false; + if (!(inputFormat == rhs.inputFormat)) + return false; + if (!(outputFormat == rhs.outputFormat)) + return false; + if (!(compressed == rhs.compressed)) + return false; + if (!(numBuckets == rhs.numBuckets)) + return false; + if (!(serdeInfo == rhs.serdeInfo)) + return false; + if (!(bucketCols == rhs.bucketCols)) + return false; + if (!(sortCols == rhs.sortCols)) + return false; + if (!(parameters == rhs.parameters)) + return false; + if (__isset.skewedInfo != rhs.__isset.skewedInfo) + return false; + else if (__isset.skewedInfo && !(skewedInfo == rhs.skewedInfo)) + return false; + if (__isset.storedAsSubDirectories != rhs.__isset.storedAsSubDirectories) + return false; + else if (__isset.storedAsSubDirectories && !(storedAsSubDirectories == rhs.storedAsSubDirectories)) + return false; + return true; + } + bool operator != (const StorageDescriptor &rhs) const { + return !(*this == rhs); + } + + bool operator < (const StorageDescriptor & ) 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(StorageDescriptor &a, StorageDescriptor &b); + +std::ostream& operator<<(std::ostream& out, const StorageDescriptor& obj); + +typedef struct _Table__isset { + _Table__isset() : tableName(false), dbName(false), owner(false), createTime(false), lastAccessTime(false), retention(false), sd(false), partitionKeys(false), parameters(false), viewOriginalText(false), viewExpandedText(false), tableType(false), privileges(false), temporary(true), rewriteEnabled(false) {} + bool tableName :1; + bool dbName :1; + bool owner :1; + bool createTime :1; + bool lastAccessTime :1; + bool retention :1; + bool sd :1; + bool partitionKeys :1; + bool parameters :1; + bool viewOriginalText :1; + bool viewExpandedText :1; + bool tableType :1; + bool privileges :1; + bool temporary :1; + bool rewriteEnabled :1; +} _Table__isset; + +class Table : public virtual ::apache::thrift::TBase { + public: + + Table(const Table&); + Table& operator=(const Table&); + Table() : tableName(), dbName(), owner(), createTime(0), lastAccessTime(0), retention(0), viewOriginalText(), viewExpandedText(), tableType(), temporary(false), rewriteEnabled(0) { + } + + virtual ~Table() noexcept; + std::string tableName; + std::string dbName; + std::string owner; + int32_t createTime; + int32_t lastAccessTime; + int32_t retention; + StorageDescriptor sd; + std::vector partitionKeys; + std::map parameters; + std::string viewOriginalText; + std::string viewExpandedText; + std::string tableType; + PrincipalPrivilegeSet privileges; + bool temporary; + bool rewriteEnabled; + + _Table__isset __isset; + + void __set_tableName(const std::string& val); + + void __set_dbName(const std::string& val); + + void __set_owner(const std::string& val); + + void __set_createTime(const int32_t val); + + void __set_lastAccessTime(const int32_t val); + + void __set_retention(const int32_t val); + + void __set_sd(const StorageDescriptor& val); + + void __set_partitionKeys(const std::vector & val); + + void __set_parameters(const std::map & val); + + void __set_viewOriginalText(const std::string& val); + + void __set_viewExpandedText(const std::string& val); + + void __set_tableType(const std::string& val); + + void __set_privileges(const PrincipalPrivilegeSet& val); + + void __set_temporary(const bool val); + + void __set_rewriteEnabled(const bool val); + + bool operator == (const Table & rhs) const + { + if (!(tableName == rhs.tableName)) + return false; + if (!(dbName == rhs.dbName)) + return false; + if (!(owner == rhs.owner)) + return false; + if (!(createTime == rhs.createTime)) + return false; + if (!(lastAccessTime == rhs.lastAccessTime)) + return false; + if (!(retention == rhs.retention)) + return false; + if (!(sd == rhs.sd)) + return false; + if (!(partitionKeys == rhs.partitionKeys)) + return false; + if (!(parameters == rhs.parameters)) + return false; + if (!(viewOriginalText == rhs.viewOriginalText)) + return false; + if (!(viewExpandedText == rhs.viewExpandedText)) + return false; + if (!(tableType == rhs.tableType)) + return false; + if (__isset.privileges != rhs.__isset.privileges) + return false; + else if (__isset.privileges && !(privileges == rhs.privileges)) + return false; + if (__isset.temporary != rhs.__isset.temporary) + return false; + else if (__isset.temporary && !(temporary == rhs.temporary)) + return false; + if (__isset.rewriteEnabled != rhs.__isset.rewriteEnabled) + return false; + else if (__isset.rewriteEnabled && !(rewriteEnabled == rhs.rewriteEnabled)) + return false; + return true; + } + bool operator != (const Table &rhs) const { + return !(*this == rhs); + } + + bool operator < (const Table & ) 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(Table &a, Table &b); + +std::ostream& operator<<(std::ostream& out, const Table& obj); + +typedef struct _Partition__isset { + _Partition__isset() : values(false), dbName(false), tableName(false), createTime(false), lastAccessTime(false), sd(false), parameters(false), privileges(false) {} + bool values :1; + bool dbName :1; + bool tableName :1; + bool createTime :1; + bool lastAccessTime :1; + bool sd :1; + bool parameters :1; + bool privileges :1; +} _Partition__isset; + +class Partition : public virtual ::apache::thrift::TBase { + public: + + Partition(const Partition&); + Partition& operator=(const Partition&); + Partition() : dbName(), tableName(), createTime(0), lastAccessTime(0) { + } + + virtual ~Partition() noexcept; + std::vector values; + std::string dbName; + std::string tableName; + int32_t createTime; + int32_t lastAccessTime; + StorageDescriptor sd; + std::map parameters; + PrincipalPrivilegeSet privileges; + + _Partition__isset __isset; + + void __set_values(const std::vector & val); + + void __set_dbName(const std::string& val); + + void __set_tableName(const std::string& val); + + void __set_createTime(const int32_t val); + + void __set_lastAccessTime(const int32_t val); + + void __set_sd(const StorageDescriptor& val); + + void __set_parameters(const std::map & val); + + void __set_privileges(const PrincipalPrivilegeSet& val); + + bool operator == (const Partition & rhs) const + { + if (!(values == rhs.values)) + return false; + if (!(dbName == rhs.dbName)) + return false; + if (!(tableName == rhs.tableName)) + return false; + if (!(createTime == rhs.createTime)) + return false; + if (!(lastAccessTime == rhs.lastAccessTime)) + return false; + if (!(sd == rhs.sd)) + return false; + if (!(parameters == rhs.parameters)) + return false; + if (__isset.privileges != rhs.__isset.privileges) + return false; + else if (__isset.privileges && !(privileges == rhs.privileges)) + return false; + return true; + } + bool operator != (const Partition &rhs) const { + return !(*this == rhs); + } + + bool operator < (const Partition & ) 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(Partition &a, Partition &b); + +std::ostream& operator<<(std::ostream& out, const Partition& obj); + +typedef struct _PartitionWithoutSD__isset { + _PartitionWithoutSD__isset() : values(false), createTime(false), lastAccessTime(false), relativePath(false), parameters(false), privileges(false) {} + bool values :1; + bool createTime :1; + bool lastAccessTime :1; + bool relativePath :1; + bool parameters :1; + bool privileges :1; +} _PartitionWithoutSD__isset; + +class PartitionWithoutSD : public virtual ::apache::thrift::TBase { + public: + + PartitionWithoutSD(const PartitionWithoutSD&); + PartitionWithoutSD& operator=(const PartitionWithoutSD&); + PartitionWithoutSD() : createTime(0), lastAccessTime(0), relativePath() { + } + + virtual ~PartitionWithoutSD() noexcept; + std::vector values; + int32_t createTime; + int32_t lastAccessTime; + std::string relativePath; + std::map parameters; + PrincipalPrivilegeSet privileges; + + _PartitionWithoutSD__isset __isset; + + void __set_values(const std::vector & val); + + void __set_createTime(const int32_t val); + + void __set_lastAccessTime(const int32_t val); + + void __set_relativePath(const std::string& val); + + void __set_parameters(const std::map & val); + + void __set_privileges(const PrincipalPrivilegeSet& val); + + bool operator == (const PartitionWithoutSD & rhs) const + { + if (!(values == rhs.values)) + return false; + if (!(createTime == rhs.createTime)) + return false; + if (!(lastAccessTime == rhs.lastAccessTime)) + return false; + if (!(relativePath == rhs.relativePath)) + return false; + if (!(parameters == rhs.parameters)) + return false; + if (__isset.privileges != rhs.__isset.privileges) + return false; + else if (__isset.privileges && !(privileges == rhs.privileges)) + return false; + return true; + } + bool operator != (const PartitionWithoutSD &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PartitionWithoutSD & ) 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(PartitionWithoutSD &a, PartitionWithoutSD &b); + +std::ostream& operator<<(std::ostream& out, const PartitionWithoutSD& obj); + +typedef struct _PartitionSpecWithSharedSD__isset { + _PartitionSpecWithSharedSD__isset() : partitions(false), sd(false) {} + bool partitions :1; + bool sd :1; +} _PartitionSpecWithSharedSD__isset; + +class PartitionSpecWithSharedSD : public virtual ::apache::thrift::TBase { + public: + + PartitionSpecWithSharedSD(const PartitionSpecWithSharedSD&); + PartitionSpecWithSharedSD& operator=(const PartitionSpecWithSharedSD&); + PartitionSpecWithSharedSD() { + } + + virtual ~PartitionSpecWithSharedSD() noexcept; + std::vector partitions; + StorageDescriptor sd; + + _PartitionSpecWithSharedSD__isset __isset; + + void __set_partitions(const std::vector & val); + + void __set_sd(const StorageDescriptor& val); + + bool operator == (const PartitionSpecWithSharedSD & rhs) const + { + if (!(partitions == rhs.partitions)) + return false; + if (!(sd == rhs.sd)) + return false; + return true; + } + bool operator != (const PartitionSpecWithSharedSD &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PartitionSpecWithSharedSD & ) 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(PartitionSpecWithSharedSD &a, PartitionSpecWithSharedSD &b); + +std::ostream& operator<<(std::ostream& out, const PartitionSpecWithSharedSD& obj); + +typedef struct _PartitionListComposingSpec__isset { + _PartitionListComposingSpec__isset() : partitions(false) {} + bool partitions :1; +} _PartitionListComposingSpec__isset; + +class PartitionListComposingSpec : public virtual ::apache::thrift::TBase { + public: + + PartitionListComposingSpec(const PartitionListComposingSpec&); + PartitionListComposingSpec& operator=(const PartitionListComposingSpec&); + PartitionListComposingSpec() { + } + + virtual ~PartitionListComposingSpec() noexcept; + std::vector partitions; + + _PartitionListComposingSpec__isset __isset; + + void __set_partitions(const std::vector & val); + + bool operator == (const PartitionListComposingSpec & rhs) const + { + if (!(partitions == rhs.partitions)) + return false; + return true; + } + bool operator != (const PartitionListComposingSpec &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PartitionListComposingSpec & ) 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(PartitionListComposingSpec &a, PartitionListComposingSpec &b); + +std::ostream& operator<<(std::ostream& out, const PartitionListComposingSpec& obj); + +typedef struct _PartitionSpec__isset { + _PartitionSpec__isset() : dbName(false), tableName(false), rootPath(false), sharedSDPartitionSpec(false), partitionList(false) {} + bool dbName :1; + bool tableName :1; + bool rootPath :1; + bool sharedSDPartitionSpec :1; + bool partitionList :1; +} _PartitionSpec__isset; + +class PartitionSpec : public virtual ::apache::thrift::TBase { + public: + + PartitionSpec(const PartitionSpec&); + PartitionSpec& operator=(const PartitionSpec&); + PartitionSpec() : dbName(), tableName(), rootPath() { + } + + virtual ~PartitionSpec() noexcept; + std::string dbName; + std::string tableName; + std::string rootPath; + PartitionSpecWithSharedSD sharedSDPartitionSpec; + PartitionListComposingSpec partitionList; + + _PartitionSpec__isset __isset; + + void __set_dbName(const std::string& val); + + void __set_tableName(const std::string& val); + + void __set_rootPath(const std::string& val); + + void __set_sharedSDPartitionSpec(const PartitionSpecWithSharedSD& val); + + void __set_partitionList(const PartitionListComposingSpec& val); + + bool operator == (const PartitionSpec & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(tableName == rhs.tableName)) + return false; + if (!(rootPath == rhs.rootPath)) + return false; + if (__isset.sharedSDPartitionSpec != rhs.__isset.sharedSDPartitionSpec) + return false; + else if (__isset.sharedSDPartitionSpec && !(sharedSDPartitionSpec == rhs.sharedSDPartitionSpec)) + return false; + if (__isset.partitionList != rhs.__isset.partitionList) + return false; + else if (__isset.partitionList && !(partitionList == rhs.partitionList)) + return false; + return true; + } + bool operator != (const PartitionSpec &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PartitionSpec & ) 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(PartitionSpec &a, PartitionSpec &b); + +std::ostream& operator<<(std::ostream& out, const PartitionSpec& obj); + +typedef struct _Index__isset { + _Index__isset() : indexName(false), indexHandlerClass(false), dbName(false), origTableName(false), createTime(false), lastAccessTime(false), indexTableName(false), sd(false), parameters(false), deferredRebuild(false) {} + bool indexName :1; + bool indexHandlerClass :1; + bool dbName :1; + bool origTableName :1; + bool createTime :1; + bool lastAccessTime :1; + bool indexTableName :1; + bool sd :1; + bool parameters :1; + bool deferredRebuild :1; +} _Index__isset; + +class Index : public virtual ::apache::thrift::TBase { + public: + + Index(const Index&); + Index& operator=(const Index&); + Index() : indexName(), indexHandlerClass(), dbName(), origTableName(), createTime(0), lastAccessTime(0), indexTableName(), deferredRebuild(0) { + } + + virtual ~Index() noexcept; + std::string indexName; + std::string indexHandlerClass; + std::string dbName; + std::string origTableName; + int32_t createTime; + int32_t lastAccessTime; + std::string indexTableName; + StorageDescriptor sd; + std::map parameters; + bool deferredRebuild; + + _Index__isset __isset; + + void __set_indexName(const std::string& val); + + void __set_indexHandlerClass(const std::string& val); + + void __set_dbName(const std::string& val); + + void __set_origTableName(const std::string& val); + + void __set_createTime(const int32_t val); + + void __set_lastAccessTime(const int32_t val); + + void __set_indexTableName(const std::string& val); + + void __set_sd(const StorageDescriptor& val); + + void __set_parameters(const std::map & val); + + void __set_deferredRebuild(const bool val); + + bool operator == (const Index & rhs) const + { + if (!(indexName == rhs.indexName)) + return false; + if (!(indexHandlerClass == rhs.indexHandlerClass)) + return false; + if (!(dbName == rhs.dbName)) + return false; + if (!(origTableName == rhs.origTableName)) + return false; + if (!(createTime == rhs.createTime)) + return false; + if (!(lastAccessTime == rhs.lastAccessTime)) + return false; + if (!(indexTableName == rhs.indexTableName)) + return false; + if (!(sd == rhs.sd)) + return false; + if (!(parameters == rhs.parameters)) + return false; + if (!(deferredRebuild == rhs.deferredRebuild)) + return false; + return true; + } + bool operator != (const Index &rhs) const { + return !(*this == rhs); + } + + bool operator < (const Index & ) 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(Index &a, Index &b); + +std::ostream& operator<<(std::ostream& out, const Index& obj); + +typedef struct _BooleanColumnStatsData__isset { + _BooleanColumnStatsData__isset() : bitVectors(false) {} + bool bitVectors :1; +} _BooleanColumnStatsData__isset; + +class BooleanColumnStatsData : public virtual ::apache::thrift::TBase { + public: + + BooleanColumnStatsData(const BooleanColumnStatsData&); + BooleanColumnStatsData& operator=(const BooleanColumnStatsData&); + BooleanColumnStatsData() : numTrues(0), numFalses(0), numNulls(0), bitVectors() { + } + + virtual ~BooleanColumnStatsData() noexcept; + int64_t numTrues; + int64_t numFalses; + int64_t numNulls; + std::string bitVectors; + + _BooleanColumnStatsData__isset __isset; + + void __set_numTrues(const int64_t val); + + void __set_numFalses(const int64_t val); + + void __set_numNulls(const int64_t val); + + void __set_bitVectors(const std::string& val); + + bool operator == (const BooleanColumnStatsData & rhs) const + { + if (!(numTrues == rhs.numTrues)) + return false; + if (!(numFalses == rhs.numFalses)) + return false; + if (!(numNulls == rhs.numNulls)) + return false; + if (__isset.bitVectors != rhs.__isset.bitVectors) + return false; + else if (__isset.bitVectors && !(bitVectors == rhs.bitVectors)) + return false; + return true; + } + bool operator != (const BooleanColumnStatsData &rhs) const { + return !(*this == rhs); + } + + bool operator < (const BooleanColumnStatsData & ) 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(BooleanColumnStatsData &a, BooleanColumnStatsData &b); + +std::ostream& operator<<(std::ostream& out, const BooleanColumnStatsData& obj); + +typedef struct _DoubleColumnStatsData__isset { + _DoubleColumnStatsData__isset() : lowValue(false), highValue(false), bitVectors(false) {} + bool lowValue :1; + bool highValue :1; + bool bitVectors :1; +} _DoubleColumnStatsData__isset; + +class DoubleColumnStatsData : public virtual ::apache::thrift::TBase { + public: + + DoubleColumnStatsData(const DoubleColumnStatsData&); + DoubleColumnStatsData& operator=(const DoubleColumnStatsData&); + DoubleColumnStatsData() : lowValue(0), highValue(0), numNulls(0), numDVs(0), bitVectors() { + } + + virtual ~DoubleColumnStatsData() noexcept; + double lowValue; + double highValue; + int64_t numNulls; + int64_t numDVs; + std::string bitVectors; + + _DoubleColumnStatsData__isset __isset; + + void __set_lowValue(const double val); + + void __set_highValue(const double val); + + void __set_numNulls(const int64_t val); + + void __set_numDVs(const int64_t val); + + void __set_bitVectors(const std::string& val); + + bool operator == (const DoubleColumnStatsData & rhs) const + { + if (__isset.lowValue != rhs.__isset.lowValue) + return false; + else if (__isset.lowValue && !(lowValue == rhs.lowValue)) + return false; + if (__isset.highValue != rhs.__isset.highValue) + return false; + else if (__isset.highValue && !(highValue == rhs.highValue)) + return false; + if (!(numNulls == rhs.numNulls)) + return false; + if (!(numDVs == rhs.numDVs)) + return false; + if (__isset.bitVectors != rhs.__isset.bitVectors) + return false; + else if (__isset.bitVectors && !(bitVectors == rhs.bitVectors)) + return false; + return true; + } + bool operator != (const DoubleColumnStatsData &rhs) const { + return !(*this == rhs); + } + + bool operator < (const DoubleColumnStatsData & ) 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(DoubleColumnStatsData &a, DoubleColumnStatsData &b); + +std::ostream& operator<<(std::ostream& out, const DoubleColumnStatsData& obj); + +typedef struct _LongColumnStatsData__isset { + _LongColumnStatsData__isset() : lowValue(false), highValue(false), bitVectors(false) {} + bool lowValue :1; + bool highValue :1; + bool bitVectors :1; +} _LongColumnStatsData__isset; + +class LongColumnStatsData : public virtual ::apache::thrift::TBase { + public: + + LongColumnStatsData(const LongColumnStatsData&); + LongColumnStatsData& operator=(const LongColumnStatsData&); + LongColumnStatsData() : lowValue(0), highValue(0), numNulls(0), numDVs(0), bitVectors() { + } + + virtual ~LongColumnStatsData() noexcept; + int64_t lowValue; + int64_t highValue; + int64_t numNulls; + int64_t numDVs; + std::string bitVectors; + + _LongColumnStatsData__isset __isset; + + void __set_lowValue(const int64_t val); + + void __set_highValue(const int64_t val); + + void __set_numNulls(const int64_t val); + + void __set_numDVs(const int64_t val); + + void __set_bitVectors(const std::string& val); + + bool operator == (const LongColumnStatsData & rhs) const + { + if (__isset.lowValue != rhs.__isset.lowValue) + return false; + else if (__isset.lowValue && !(lowValue == rhs.lowValue)) + return false; + if (__isset.highValue != rhs.__isset.highValue) + return false; + else if (__isset.highValue && !(highValue == rhs.highValue)) + return false; + if (!(numNulls == rhs.numNulls)) + return false; + if (!(numDVs == rhs.numDVs)) + return false; + if (__isset.bitVectors != rhs.__isset.bitVectors) + return false; + else if (__isset.bitVectors && !(bitVectors == rhs.bitVectors)) + return false; + return true; + } + bool operator != (const LongColumnStatsData &rhs) const { + return !(*this == rhs); + } + + bool operator < (const LongColumnStatsData & ) 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(LongColumnStatsData &a, LongColumnStatsData &b); + +std::ostream& operator<<(std::ostream& out, const LongColumnStatsData& obj); + +typedef struct _StringColumnStatsData__isset { + _StringColumnStatsData__isset() : bitVectors(false) {} + bool bitVectors :1; +} _StringColumnStatsData__isset; + +class StringColumnStatsData : public virtual ::apache::thrift::TBase { + public: + + StringColumnStatsData(const StringColumnStatsData&); + StringColumnStatsData& operator=(const StringColumnStatsData&); + StringColumnStatsData() : maxColLen(0), avgColLen(0), numNulls(0), numDVs(0), bitVectors() { + } + + virtual ~StringColumnStatsData() noexcept; + int64_t maxColLen; + double avgColLen; + int64_t numNulls; + int64_t numDVs; + std::string bitVectors; + + _StringColumnStatsData__isset __isset; + + void __set_maxColLen(const int64_t val); + + void __set_avgColLen(const double val); + + void __set_numNulls(const int64_t val); + + void __set_numDVs(const int64_t val); + + void __set_bitVectors(const std::string& val); + + bool operator == (const StringColumnStatsData & rhs) const + { + if (!(maxColLen == rhs.maxColLen)) + return false; + if (!(avgColLen == rhs.avgColLen)) + return false; + if (!(numNulls == rhs.numNulls)) + return false; + if (!(numDVs == rhs.numDVs)) + return false; + if (__isset.bitVectors != rhs.__isset.bitVectors) + return false; + else if (__isset.bitVectors && !(bitVectors == rhs.bitVectors)) + return false; + return true; + } + bool operator != (const StringColumnStatsData &rhs) const { + return !(*this == rhs); + } + + bool operator < (const StringColumnStatsData & ) 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(StringColumnStatsData &a, StringColumnStatsData &b); + +std::ostream& operator<<(std::ostream& out, const StringColumnStatsData& obj); + +typedef struct _BinaryColumnStatsData__isset { + _BinaryColumnStatsData__isset() : bitVectors(false) {} + bool bitVectors :1; +} _BinaryColumnStatsData__isset; + +class BinaryColumnStatsData : public virtual ::apache::thrift::TBase { + public: + + BinaryColumnStatsData(const BinaryColumnStatsData&); + BinaryColumnStatsData& operator=(const BinaryColumnStatsData&); + BinaryColumnStatsData() : maxColLen(0), avgColLen(0), numNulls(0), bitVectors() { + } + + virtual ~BinaryColumnStatsData() noexcept; + int64_t maxColLen; + double avgColLen; + int64_t numNulls; + std::string bitVectors; + + _BinaryColumnStatsData__isset __isset; + + void __set_maxColLen(const int64_t val); + + void __set_avgColLen(const double val); + + void __set_numNulls(const int64_t val); + + void __set_bitVectors(const std::string& val); + + bool operator == (const BinaryColumnStatsData & rhs) const + { + if (!(maxColLen == rhs.maxColLen)) + return false; + if (!(avgColLen == rhs.avgColLen)) + return false; + if (!(numNulls == rhs.numNulls)) + return false; + if (__isset.bitVectors != rhs.__isset.bitVectors) + return false; + else if (__isset.bitVectors && !(bitVectors == rhs.bitVectors)) + return false; + return true; + } + bool operator != (const BinaryColumnStatsData &rhs) const { + return !(*this == rhs); + } + + bool operator < (const BinaryColumnStatsData & ) 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(BinaryColumnStatsData &a, BinaryColumnStatsData &b); + +std::ostream& operator<<(std::ostream& out, const BinaryColumnStatsData& obj); + + +class Decimal : public virtual ::apache::thrift::TBase { + public: + + Decimal(const Decimal&); + Decimal& operator=(const Decimal&); + Decimal() : unscaled(), scale(0) { + } + + virtual ~Decimal() noexcept; + std::string unscaled; + int16_t scale; + + void __set_unscaled(const std::string& val); + + void __set_scale(const int16_t val); + + bool operator == (const Decimal & rhs) const + { + if (!(unscaled == rhs.unscaled)) + return false; + if (!(scale == rhs.scale)) + return false; + return true; + } + bool operator != (const Decimal &rhs) const { + return !(*this == rhs); + } + + bool operator < (const Decimal & ) 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(Decimal &a, Decimal &b); + +std::ostream& operator<<(std::ostream& out, const Decimal& obj); + +typedef struct _DecimalColumnStatsData__isset { + _DecimalColumnStatsData__isset() : lowValue(false), highValue(false), bitVectors(false) {} + bool lowValue :1; + bool highValue :1; + bool bitVectors :1; +} _DecimalColumnStatsData__isset; + +class DecimalColumnStatsData : public virtual ::apache::thrift::TBase { + public: + + DecimalColumnStatsData(const DecimalColumnStatsData&); + DecimalColumnStatsData& operator=(const DecimalColumnStatsData&); + DecimalColumnStatsData() : numNulls(0), numDVs(0), bitVectors() { + } + + virtual ~DecimalColumnStatsData() noexcept; + Decimal lowValue; + Decimal highValue; + int64_t numNulls; + int64_t numDVs; + std::string bitVectors; + + _DecimalColumnStatsData__isset __isset; + + void __set_lowValue(const Decimal& val); + + void __set_highValue(const Decimal& val); + + void __set_numNulls(const int64_t val); + + void __set_numDVs(const int64_t val); + + void __set_bitVectors(const std::string& val); + + bool operator == (const DecimalColumnStatsData & rhs) const + { + if (__isset.lowValue != rhs.__isset.lowValue) + return false; + else if (__isset.lowValue && !(lowValue == rhs.lowValue)) + return false; + if (__isset.highValue != rhs.__isset.highValue) + return false; + else if (__isset.highValue && !(highValue == rhs.highValue)) + return false; + if (!(numNulls == rhs.numNulls)) + return false; + if (!(numDVs == rhs.numDVs)) + return false; + if (__isset.bitVectors != rhs.__isset.bitVectors) + return false; + else if (__isset.bitVectors && !(bitVectors == rhs.bitVectors)) + return false; + return true; + } + bool operator != (const DecimalColumnStatsData &rhs) const { + return !(*this == rhs); + } + + bool operator < (const DecimalColumnStatsData & ) 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(DecimalColumnStatsData &a, DecimalColumnStatsData &b); + +std::ostream& operator<<(std::ostream& out, const DecimalColumnStatsData& obj); + + +class Date : public virtual ::apache::thrift::TBase { + public: + + Date(const Date&); + Date& operator=(const Date&); + Date() : daysSinceEpoch(0) { + } + + virtual ~Date() noexcept; + int64_t daysSinceEpoch; + + void __set_daysSinceEpoch(const int64_t val); + + bool operator == (const Date & rhs) const + { + if (!(daysSinceEpoch == rhs.daysSinceEpoch)) + return false; + return true; + } + bool operator != (const Date &rhs) const { + return !(*this == rhs); + } + + bool operator < (const Date & ) 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(Date &a, Date &b); + +std::ostream& operator<<(std::ostream& out, const Date& obj); + +typedef struct _DateColumnStatsData__isset { + _DateColumnStatsData__isset() : lowValue(false), highValue(false), bitVectors(false) {} + bool lowValue :1; + bool highValue :1; + bool bitVectors :1; +} _DateColumnStatsData__isset; + +class DateColumnStatsData : public virtual ::apache::thrift::TBase { + public: + + DateColumnStatsData(const DateColumnStatsData&); + DateColumnStatsData& operator=(const DateColumnStatsData&); + DateColumnStatsData() : numNulls(0), numDVs(0), bitVectors() { + } + + virtual ~DateColumnStatsData() noexcept; + Date lowValue; + Date highValue; + int64_t numNulls; + int64_t numDVs; + std::string bitVectors; + + _DateColumnStatsData__isset __isset; + + void __set_lowValue(const Date& val); + + void __set_highValue(const Date& val); + + void __set_numNulls(const int64_t val); + + void __set_numDVs(const int64_t val); + + void __set_bitVectors(const std::string& val); + + bool operator == (const DateColumnStatsData & rhs) const + { + if (__isset.lowValue != rhs.__isset.lowValue) + return false; + else if (__isset.lowValue && !(lowValue == rhs.lowValue)) + return false; + if (__isset.highValue != rhs.__isset.highValue) + return false; + else if (__isset.highValue && !(highValue == rhs.highValue)) + return false; + if (!(numNulls == rhs.numNulls)) + return false; + if (!(numDVs == rhs.numDVs)) + return false; + if (__isset.bitVectors != rhs.__isset.bitVectors) + return false; + else if (__isset.bitVectors && !(bitVectors == rhs.bitVectors)) + return false; + return true; + } + bool operator != (const DateColumnStatsData &rhs) const { + return !(*this == rhs); + } + + bool operator < (const DateColumnStatsData & ) 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(DateColumnStatsData &a, DateColumnStatsData &b); + +std::ostream& operator<<(std::ostream& out, const DateColumnStatsData& obj); + +typedef struct _ColumnStatisticsData__isset { + _ColumnStatisticsData__isset() : booleanStats(false), longStats(false), doubleStats(false), stringStats(false), binaryStats(false), decimalStats(false), dateStats(false) {} + bool booleanStats :1; + bool longStats :1; + bool doubleStats :1; + bool stringStats :1; + bool binaryStats :1; + bool decimalStats :1; + bool dateStats :1; +} _ColumnStatisticsData__isset; + +class ColumnStatisticsData : public virtual ::apache::thrift::TBase { + public: + + ColumnStatisticsData(const ColumnStatisticsData&); + ColumnStatisticsData& operator=(const ColumnStatisticsData&); + ColumnStatisticsData() { + } + + virtual ~ColumnStatisticsData() noexcept; + BooleanColumnStatsData booleanStats; + LongColumnStatsData longStats; + DoubleColumnStatsData doubleStats; + StringColumnStatsData stringStats; + BinaryColumnStatsData binaryStats; + DecimalColumnStatsData decimalStats; + DateColumnStatsData dateStats; + + _ColumnStatisticsData__isset __isset; + + void __set_booleanStats(const BooleanColumnStatsData& val); + + void __set_longStats(const LongColumnStatsData& val); + + void __set_doubleStats(const DoubleColumnStatsData& val); + + void __set_stringStats(const StringColumnStatsData& val); + + void __set_binaryStats(const BinaryColumnStatsData& val); + + void __set_decimalStats(const DecimalColumnStatsData& val); + + void __set_dateStats(const DateColumnStatsData& val); + + bool operator == (const ColumnStatisticsData & rhs) const + { + if (__isset.booleanStats != rhs.__isset.booleanStats) + return false; + else if (__isset.booleanStats && !(booleanStats == rhs.booleanStats)) + return false; + if (__isset.longStats != rhs.__isset.longStats) + return false; + else if (__isset.longStats && !(longStats == rhs.longStats)) + return false; + if (__isset.doubleStats != rhs.__isset.doubleStats) + return false; + else if (__isset.doubleStats && !(doubleStats == rhs.doubleStats)) + return false; + if (__isset.stringStats != rhs.__isset.stringStats) + return false; + else if (__isset.stringStats && !(stringStats == rhs.stringStats)) + return false; + if (__isset.binaryStats != rhs.__isset.binaryStats) + return false; + else if (__isset.binaryStats && !(binaryStats == rhs.binaryStats)) + return false; + if (__isset.decimalStats != rhs.__isset.decimalStats) + return false; + else if (__isset.decimalStats && !(decimalStats == rhs.decimalStats)) + return false; + if (__isset.dateStats != rhs.__isset.dateStats) + return false; + else if (__isset.dateStats && !(dateStats == rhs.dateStats)) + return false; + return true; + } + bool operator != (const ColumnStatisticsData &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ColumnStatisticsData & ) 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(ColumnStatisticsData &a, ColumnStatisticsData &b); + +std::ostream& operator<<(std::ostream& out, const ColumnStatisticsData& obj); + + +class ColumnStatisticsObj : public virtual ::apache::thrift::TBase { + public: + + ColumnStatisticsObj(const ColumnStatisticsObj&); + ColumnStatisticsObj& operator=(const ColumnStatisticsObj&); + ColumnStatisticsObj() : colName(), colType() { + } + + virtual ~ColumnStatisticsObj() noexcept; + std::string colName; + std::string colType; + ColumnStatisticsData statsData; + + void __set_colName(const std::string& val); + + void __set_colType(const std::string& val); + + void __set_statsData(const ColumnStatisticsData& val); + + bool operator == (const ColumnStatisticsObj & rhs) const + { + if (!(colName == rhs.colName)) + return false; + if (!(colType == rhs.colType)) + return false; + if (!(statsData == rhs.statsData)) + return false; + return true; + } + bool operator != (const ColumnStatisticsObj &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ColumnStatisticsObj & ) 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(ColumnStatisticsObj &a, ColumnStatisticsObj &b); + +std::ostream& operator<<(std::ostream& out, const ColumnStatisticsObj& obj); + +typedef struct _ColumnStatisticsDesc__isset { + _ColumnStatisticsDesc__isset() : partName(false), lastAnalyzed(false) {} + bool partName :1; + bool lastAnalyzed :1; +} _ColumnStatisticsDesc__isset; + +class ColumnStatisticsDesc : public virtual ::apache::thrift::TBase { + public: + + ColumnStatisticsDesc(const ColumnStatisticsDesc&); + ColumnStatisticsDesc& operator=(const ColumnStatisticsDesc&); + ColumnStatisticsDesc() : isTblLevel(0), dbName(), tableName(), partName(), lastAnalyzed(0) { + } + + virtual ~ColumnStatisticsDesc() noexcept; + bool isTblLevel; + std::string dbName; + std::string tableName; + std::string partName; + int64_t lastAnalyzed; + + _ColumnStatisticsDesc__isset __isset; + + void __set_isTblLevel(const bool val); + + void __set_dbName(const std::string& val); + + void __set_tableName(const std::string& val); + + void __set_partName(const std::string& val); + + void __set_lastAnalyzed(const int64_t val); + + bool operator == (const ColumnStatisticsDesc & rhs) const + { + if (!(isTblLevel == rhs.isTblLevel)) + return false; + if (!(dbName == rhs.dbName)) + return false; + if (!(tableName == rhs.tableName)) + return false; + if (__isset.partName != rhs.__isset.partName) + return false; + else if (__isset.partName && !(partName == rhs.partName)) + return false; + if (__isset.lastAnalyzed != rhs.__isset.lastAnalyzed) + return false; + else if (__isset.lastAnalyzed && !(lastAnalyzed == rhs.lastAnalyzed)) + return false; + return true; + } + bool operator != (const ColumnStatisticsDesc &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ColumnStatisticsDesc & ) 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(ColumnStatisticsDesc &a, ColumnStatisticsDesc &b); + +std::ostream& operator<<(std::ostream& out, const ColumnStatisticsDesc& obj); + + +class ColumnStatistics : public virtual ::apache::thrift::TBase { + public: + + ColumnStatistics(const ColumnStatistics&); + ColumnStatistics& operator=(const ColumnStatistics&); + ColumnStatistics() { + } + + virtual ~ColumnStatistics() noexcept; + ColumnStatisticsDesc statsDesc; + std::vector statsObj; + + void __set_statsDesc(const ColumnStatisticsDesc& val); + + void __set_statsObj(const std::vector & val); + + bool operator == (const ColumnStatistics & rhs) const + { + if (!(statsDesc == rhs.statsDesc)) + return false; + if (!(statsObj == rhs.statsObj)) + return false; + return true; + } + bool operator != (const ColumnStatistics &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ColumnStatistics & ) 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(ColumnStatistics &a, ColumnStatistics &b); + +std::ostream& operator<<(std::ostream& out, const ColumnStatistics& obj); + + +class AggrStats : public virtual ::apache::thrift::TBase { + public: + + AggrStats(const AggrStats&); + AggrStats& operator=(const AggrStats&); + AggrStats() : partsFound(0) { + } + + virtual ~AggrStats() noexcept; + std::vector colStats; + int64_t partsFound; + + void __set_colStats(const std::vector & val); + + void __set_partsFound(const int64_t val); + + bool operator == (const AggrStats & rhs) const + { + if (!(colStats == rhs.colStats)) + return false; + if (!(partsFound == rhs.partsFound)) + return false; + return true; + } + bool operator != (const AggrStats &rhs) const { + return !(*this == rhs); + } + + bool operator < (const AggrStats & ) 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(AggrStats &a, AggrStats &b); + +std::ostream& operator<<(std::ostream& out, const AggrStats& obj); + +typedef struct _SetPartitionsStatsRequest__isset { + _SetPartitionsStatsRequest__isset() : needMerge(false) {} + bool needMerge :1; +} _SetPartitionsStatsRequest__isset; + +class SetPartitionsStatsRequest : public virtual ::apache::thrift::TBase { + public: + + SetPartitionsStatsRequest(const SetPartitionsStatsRequest&); + SetPartitionsStatsRequest& operator=(const SetPartitionsStatsRequest&); + SetPartitionsStatsRequest() : needMerge(0) { + } + + virtual ~SetPartitionsStatsRequest() noexcept; + std::vector colStats; + bool needMerge; + + _SetPartitionsStatsRequest__isset __isset; + + void __set_colStats(const std::vector & val); + + void __set_needMerge(const bool val); + + bool operator == (const SetPartitionsStatsRequest & rhs) const + { + if (!(colStats == rhs.colStats)) + return false; + if (__isset.needMerge != rhs.__isset.needMerge) + return false; + else if (__isset.needMerge && !(needMerge == rhs.needMerge)) + return false; + return true; + } + bool operator != (const SetPartitionsStatsRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const SetPartitionsStatsRequest & ) 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(SetPartitionsStatsRequest &a, SetPartitionsStatsRequest &b); + +std::ostream& operator<<(std::ostream& out, const SetPartitionsStatsRequest& obj); + +typedef struct _Schema__isset { + _Schema__isset() : fieldSchemas(false), properties(false) {} + bool fieldSchemas :1; + bool properties :1; +} _Schema__isset; + +class Schema : public virtual ::apache::thrift::TBase { + public: + + Schema(const Schema&); + Schema& operator=(const Schema&); + Schema() { + } + + virtual ~Schema() noexcept; + std::vector fieldSchemas; + std::map properties; + + _Schema__isset __isset; + + void __set_fieldSchemas(const std::vector & val); + + void __set_properties(const std::map & val); + + bool operator == (const Schema & rhs) const + { + if (!(fieldSchemas == rhs.fieldSchemas)) + return false; + if (!(properties == rhs.properties)) + return false; + return true; + } + bool operator != (const Schema &rhs) const { + return !(*this == rhs); + } + + bool operator < (const Schema & ) 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(Schema &a, Schema &b); + +std::ostream& operator<<(std::ostream& out, const Schema& obj); + +typedef struct _EnvironmentContext__isset { + _EnvironmentContext__isset() : properties(false) {} + bool properties :1; +} _EnvironmentContext__isset; + +class EnvironmentContext : public virtual ::apache::thrift::TBase { + public: + + EnvironmentContext(const EnvironmentContext&); + EnvironmentContext& operator=(const EnvironmentContext&); + EnvironmentContext() { + } + + virtual ~EnvironmentContext() noexcept; + std::map properties; + + _EnvironmentContext__isset __isset; + + void __set_properties(const std::map & val); + + bool operator == (const EnvironmentContext & rhs) const + { + if (!(properties == rhs.properties)) + return false; + return true; + } + bool operator != (const EnvironmentContext &rhs) const { + return !(*this == rhs); + } + + bool operator < (const EnvironmentContext & ) 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(EnvironmentContext &a, EnvironmentContext &b); + +std::ostream& operator<<(std::ostream& out, const EnvironmentContext& obj); + + +class PrimaryKeysRequest : public virtual ::apache::thrift::TBase { + public: + + PrimaryKeysRequest(const PrimaryKeysRequest&); + PrimaryKeysRequest& operator=(const PrimaryKeysRequest&); + PrimaryKeysRequest() : db_name(), tbl_name() { + } + + virtual ~PrimaryKeysRequest() noexcept; + std::string db_name; + std::string tbl_name; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + bool operator == (const PrimaryKeysRequest & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + return true; + } + bool operator != (const PrimaryKeysRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PrimaryKeysRequest & ) 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(PrimaryKeysRequest &a, PrimaryKeysRequest &b); + +std::ostream& operator<<(std::ostream& out, const PrimaryKeysRequest& obj); + + +class PrimaryKeysResponse : public virtual ::apache::thrift::TBase { + public: + + PrimaryKeysResponse(const PrimaryKeysResponse&); + PrimaryKeysResponse& operator=(const PrimaryKeysResponse&); + PrimaryKeysResponse() { + } + + virtual ~PrimaryKeysResponse() noexcept; + std::vector primaryKeys; + + void __set_primaryKeys(const std::vector & val); + + bool operator == (const PrimaryKeysResponse & rhs) const + { + if (!(primaryKeys == rhs.primaryKeys)) + return false; + return true; + } + bool operator != (const PrimaryKeysResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PrimaryKeysResponse & ) 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(PrimaryKeysResponse &a, PrimaryKeysResponse &b); + +std::ostream& operator<<(std::ostream& out, const PrimaryKeysResponse& obj); + +typedef struct _ForeignKeysRequest__isset { + _ForeignKeysRequest__isset() : parent_db_name(false), parent_tbl_name(false), foreign_db_name(false), foreign_tbl_name(false) {} + bool parent_db_name :1; + bool parent_tbl_name :1; + bool foreign_db_name :1; + bool foreign_tbl_name :1; +} _ForeignKeysRequest__isset; + +class ForeignKeysRequest : public virtual ::apache::thrift::TBase { + public: + + ForeignKeysRequest(const ForeignKeysRequest&); + ForeignKeysRequest& operator=(const ForeignKeysRequest&); + ForeignKeysRequest() : parent_db_name(), parent_tbl_name(), foreign_db_name(), foreign_tbl_name() { + } + + virtual ~ForeignKeysRequest() noexcept; + std::string parent_db_name; + std::string parent_tbl_name; + std::string foreign_db_name; + std::string foreign_tbl_name; + + _ForeignKeysRequest__isset __isset; + + void __set_parent_db_name(const std::string& val); + + void __set_parent_tbl_name(const std::string& val); + + void __set_foreign_db_name(const std::string& val); + + void __set_foreign_tbl_name(const std::string& val); + + bool operator == (const ForeignKeysRequest & rhs) const + { + if (!(parent_db_name == rhs.parent_db_name)) + return false; + if (!(parent_tbl_name == rhs.parent_tbl_name)) + return false; + if (!(foreign_db_name == rhs.foreign_db_name)) + return false; + if (!(foreign_tbl_name == rhs.foreign_tbl_name)) + return false; + return true; + } + bool operator != (const ForeignKeysRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ForeignKeysRequest & ) 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(ForeignKeysRequest &a, ForeignKeysRequest &b); + +std::ostream& operator<<(std::ostream& out, const ForeignKeysRequest& obj); + + +class ForeignKeysResponse : public virtual ::apache::thrift::TBase { + public: + + ForeignKeysResponse(const ForeignKeysResponse&); + ForeignKeysResponse& operator=(const ForeignKeysResponse&); + ForeignKeysResponse() { + } + + virtual ~ForeignKeysResponse() noexcept; + std::vector foreignKeys; + + void __set_foreignKeys(const std::vector & val); + + bool operator == (const ForeignKeysResponse & rhs) const + { + if (!(foreignKeys == rhs.foreignKeys)) + return false; + return true; + } + bool operator != (const ForeignKeysResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ForeignKeysResponse & ) 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(ForeignKeysResponse &a, ForeignKeysResponse &b); + +std::ostream& operator<<(std::ostream& out, const ForeignKeysResponse& obj); + + +class UniqueConstraintsRequest : public virtual ::apache::thrift::TBase { + public: + + UniqueConstraintsRequest(const UniqueConstraintsRequest&); + UniqueConstraintsRequest& operator=(const UniqueConstraintsRequest&); + UniqueConstraintsRequest() : db_name(), tbl_name() { + } + + virtual ~UniqueConstraintsRequest() noexcept; + std::string db_name; + std::string tbl_name; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + bool operator == (const UniqueConstraintsRequest & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + return true; + } + bool operator != (const UniqueConstraintsRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const UniqueConstraintsRequest & ) 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(UniqueConstraintsRequest &a, UniqueConstraintsRequest &b); + +std::ostream& operator<<(std::ostream& out, const UniqueConstraintsRequest& obj); + + +class UniqueConstraintsResponse : public virtual ::apache::thrift::TBase { + public: + + UniqueConstraintsResponse(const UniqueConstraintsResponse&); + UniqueConstraintsResponse& operator=(const UniqueConstraintsResponse&); + UniqueConstraintsResponse() { + } + + virtual ~UniqueConstraintsResponse() noexcept; + std::vector uniqueConstraints; + + void __set_uniqueConstraints(const std::vector & val); + + bool operator == (const UniqueConstraintsResponse & rhs) const + { + if (!(uniqueConstraints == rhs.uniqueConstraints)) + return false; + return true; + } + bool operator != (const UniqueConstraintsResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const UniqueConstraintsResponse & ) 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(UniqueConstraintsResponse &a, UniqueConstraintsResponse &b); + +std::ostream& operator<<(std::ostream& out, const UniqueConstraintsResponse& obj); + + +class NotNullConstraintsRequest : public virtual ::apache::thrift::TBase { + public: + + NotNullConstraintsRequest(const NotNullConstraintsRequest&); + NotNullConstraintsRequest& operator=(const NotNullConstraintsRequest&); + NotNullConstraintsRequest() : db_name(), tbl_name() { + } + + virtual ~NotNullConstraintsRequest() noexcept; + std::string db_name; + std::string tbl_name; + + void __set_db_name(const std::string& val); + + void __set_tbl_name(const std::string& val); + + bool operator == (const NotNullConstraintsRequest & rhs) const + { + if (!(db_name == rhs.db_name)) + return false; + if (!(tbl_name == rhs.tbl_name)) + return false; + return true; + } + bool operator != (const NotNullConstraintsRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const NotNullConstraintsRequest & ) 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(NotNullConstraintsRequest &a, NotNullConstraintsRequest &b); + +std::ostream& operator<<(std::ostream& out, const NotNullConstraintsRequest& obj); + + +class NotNullConstraintsResponse : public virtual ::apache::thrift::TBase { + public: + + NotNullConstraintsResponse(const NotNullConstraintsResponse&); + NotNullConstraintsResponse& operator=(const NotNullConstraintsResponse&); + NotNullConstraintsResponse() { + } + + virtual ~NotNullConstraintsResponse() noexcept; + std::vector notNullConstraints; + + void __set_notNullConstraints(const std::vector & val); + + bool operator == (const NotNullConstraintsResponse & rhs) const + { + if (!(notNullConstraints == rhs.notNullConstraints)) + return false; + return true; + } + bool operator != (const NotNullConstraintsResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const NotNullConstraintsResponse & ) 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(NotNullConstraintsResponse &a, NotNullConstraintsResponse &b); + +std::ostream& operator<<(std::ostream& out, const NotNullConstraintsResponse& obj); + + +class DropConstraintRequest : public virtual ::apache::thrift::TBase { + public: + + DropConstraintRequest(const DropConstraintRequest&); + DropConstraintRequest& operator=(const DropConstraintRequest&); + DropConstraintRequest() : dbname(), tablename(), constraintname() { + } + + virtual ~DropConstraintRequest() noexcept; + std::string dbname; + std::string tablename; + std::string constraintname; + + void __set_dbname(const std::string& val); + + void __set_tablename(const std::string& val); + + void __set_constraintname(const std::string& val); + + bool operator == (const DropConstraintRequest & rhs) const + { + if (!(dbname == rhs.dbname)) + return false; + if (!(tablename == rhs.tablename)) + return false; + if (!(constraintname == rhs.constraintname)) + return false; + return true; + } + bool operator != (const DropConstraintRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const DropConstraintRequest & ) 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(DropConstraintRequest &a, DropConstraintRequest &b); + +std::ostream& operator<<(std::ostream& out, const DropConstraintRequest& obj); + + +class AddPrimaryKeyRequest : public virtual ::apache::thrift::TBase { + public: + + AddPrimaryKeyRequest(const AddPrimaryKeyRequest&); + AddPrimaryKeyRequest& operator=(const AddPrimaryKeyRequest&); + AddPrimaryKeyRequest() { + } + + virtual ~AddPrimaryKeyRequest() noexcept; + std::vector primaryKeyCols; + + void __set_primaryKeyCols(const std::vector & val); + + bool operator == (const AddPrimaryKeyRequest & rhs) const + { + if (!(primaryKeyCols == rhs.primaryKeyCols)) + return false; + return true; + } + bool operator != (const AddPrimaryKeyRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const AddPrimaryKeyRequest & ) 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(AddPrimaryKeyRequest &a, AddPrimaryKeyRequest &b); + +std::ostream& operator<<(std::ostream& out, const AddPrimaryKeyRequest& obj); + + +class AddForeignKeyRequest : public virtual ::apache::thrift::TBase { + public: + + AddForeignKeyRequest(const AddForeignKeyRequest&); + AddForeignKeyRequest& operator=(const AddForeignKeyRequest&); + AddForeignKeyRequest() { + } + + virtual ~AddForeignKeyRequest() noexcept; + std::vector foreignKeyCols; + + void __set_foreignKeyCols(const std::vector & val); + + bool operator == (const AddForeignKeyRequest & rhs) const + { + if (!(foreignKeyCols == rhs.foreignKeyCols)) + return false; + return true; + } + bool operator != (const AddForeignKeyRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const AddForeignKeyRequest & ) 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(AddForeignKeyRequest &a, AddForeignKeyRequest &b); + +std::ostream& operator<<(std::ostream& out, const AddForeignKeyRequest& obj); + + +class AddUniqueConstraintRequest : public virtual ::apache::thrift::TBase { + public: + + AddUniqueConstraintRequest(const AddUniqueConstraintRequest&); + AddUniqueConstraintRequest& operator=(const AddUniqueConstraintRequest&); + AddUniqueConstraintRequest() { + } + + virtual ~AddUniqueConstraintRequest() noexcept; + std::vector uniqueConstraintCols; + + void __set_uniqueConstraintCols(const std::vector & val); + + bool operator == (const AddUniqueConstraintRequest & rhs) const + { + if (!(uniqueConstraintCols == rhs.uniqueConstraintCols)) + return false; + return true; + } + bool operator != (const AddUniqueConstraintRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const AddUniqueConstraintRequest & ) 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(AddUniqueConstraintRequest &a, AddUniqueConstraintRequest &b); + +std::ostream& operator<<(std::ostream& out, const AddUniqueConstraintRequest& obj); + + +class AddNotNullConstraintRequest : public virtual ::apache::thrift::TBase { + public: + + AddNotNullConstraintRequest(const AddNotNullConstraintRequest&); + AddNotNullConstraintRequest& operator=(const AddNotNullConstraintRequest&); + AddNotNullConstraintRequest() { + } + + virtual ~AddNotNullConstraintRequest() noexcept; + std::vector notNullConstraintCols; + + void __set_notNullConstraintCols(const std::vector & val); + + bool operator == (const AddNotNullConstraintRequest & rhs) const + { + if (!(notNullConstraintCols == rhs.notNullConstraintCols)) + return false; + return true; + } + bool operator != (const AddNotNullConstraintRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const AddNotNullConstraintRequest & ) 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(AddNotNullConstraintRequest &a, AddNotNullConstraintRequest &b); + +std::ostream& operator<<(std::ostream& out, const AddNotNullConstraintRequest& obj); + + +class PartitionsByExprResult : public virtual ::apache::thrift::TBase { + public: + + PartitionsByExprResult(const PartitionsByExprResult&); + PartitionsByExprResult& operator=(const PartitionsByExprResult&); + PartitionsByExprResult() : hasUnknownPartitions(0) { + } + + virtual ~PartitionsByExprResult() noexcept; + std::vector partitions; + bool hasUnknownPartitions; + + void __set_partitions(const std::vector & val); + + void __set_hasUnknownPartitions(const bool val); + + bool operator == (const PartitionsByExprResult & rhs) const + { + if (!(partitions == rhs.partitions)) + return false; + if (!(hasUnknownPartitions == rhs.hasUnknownPartitions)) + return false; + return true; + } + bool operator != (const PartitionsByExprResult &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PartitionsByExprResult & ) 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(PartitionsByExprResult &a, PartitionsByExprResult &b); + +std::ostream& operator<<(std::ostream& out, const PartitionsByExprResult& obj); + +typedef struct _PartitionsByExprRequest__isset { + _PartitionsByExprRequest__isset() : defaultPartitionName(false), maxParts(true) {} + bool defaultPartitionName :1; + bool maxParts :1; +} _PartitionsByExprRequest__isset; + +class PartitionsByExprRequest : public virtual ::apache::thrift::TBase { + public: + + PartitionsByExprRequest(const PartitionsByExprRequest&); + PartitionsByExprRequest& operator=(const PartitionsByExprRequest&); + PartitionsByExprRequest() : dbName(), tblName(), expr(), defaultPartitionName(), maxParts(-1) { + } + + virtual ~PartitionsByExprRequest() noexcept; + std::string dbName; + std::string tblName; + std::string expr; + std::string defaultPartitionName; + int16_t maxParts; + + _PartitionsByExprRequest__isset __isset; + + void __set_dbName(const std::string& val); + + void __set_tblName(const std::string& val); + + void __set_expr(const std::string& val); + + void __set_defaultPartitionName(const std::string& val); + + void __set_maxParts(const int16_t val); + + bool operator == (const PartitionsByExprRequest & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(tblName == rhs.tblName)) + return false; + if (!(expr == rhs.expr)) + return false; + if (__isset.defaultPartitionName != rhs.__isset.defaultPartitionName) + return false; + else if (__isset.defaultPartitionName && !(defaultPartitionName == rhs.defaultPartitionName)) + return false; + if (__isset.maxParts != rhs.__isset.maxParts) + return false; + else if (__isset.maxParts && !(maxParts == rhs.maxParts)) + return false; + return true; + } + bool operator != (const PartitionsByExprRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PartitionsByExprRequest & ) 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(PartitionsByExprRequest &a, PartitionsByExprRequest &b); + +std::ostream& operator<<(std::ostream& out, const PartitionsByExprRequest& obj); + + +class TableStatsResult : public virtual ::apache::thrift::TBase { + public: + + TableStatsResult(const TableStatsResult&); + TableStatsResult& operator=(const TableStatsResult&); + TableStatsResult() { + } + + virtual ~TableStatsResult() noexcept; + std::vector tableStats; + + void __set_tableStats(const std::vector & val); + + bool operator == (const TableStatsResult & rhs) const + { + if (!(tableStats == rhs.tableStats)) + return false; + return true; + } + bool operator != (const TableStatsResult &rhs) const { + return !(*this == rhs); + } + + bool operator < (const TableStatsResult & ) 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(TableStatsResult &a, TableStatsResult &b); + +std::ostream& operator<<(std::ostream& out, const TableStatsResult& obj); + + +class PartitionsStatsResult : public virtual ::apache::thrift::TBase { + public: + + PartitionsStatsResult(const PartitionsStatsResult&); + PartitionsStatsResult& operator=(const PartitionsStatsResult&); + PartitionsStatsResult() { + } + + virtual ~PartitionsStatsResult() noexcept; + std::map > partStats; + + void __set_partStats(const std::map > & val); + + bool operator == (const PartitionsStatsResult & rhs) const + { + if (!(partStats == rhs.partStats)) + return false; + return true; + } + bool operator != (const PartitionsStatsResult &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PartitionsStatsResult & ) 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(PartitionsStatsResult &a, PartitionsStatsResult &b); + +std::ostream& operator<<(std::ostream& out, const PartitionsStatsResult& obj); + + +class TableStatsRequest : public virtual ::apache::thrift::TBase { + public: + + TableStatsRequest(const TableStatsRequest&); + TableStatsRequest& operator=(const TableStatsRequest&); + TableStatsRequest() : dbName(), tblName() { + } + + virtual ~TableStatsRequest() noexcept; + std::string dbName; + std::string tblName; + std::vector colNames; + + void __set_dbName(const std::string& val); + + void __set_tblName(const std::string& val); + + void __set_colNames(const std::vector & val); + + bool operator == (const TableStatsRequest & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(tblName == rhs.tblName)) + return false; + if (!(colNames == rhs.colNames)) + return false; + return true; + } + bool operator != (const TableStatsRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const TableStatsRequest & ) 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(TableStatsRequest &a, TableStatsRequest &b); + +std::ostream& operator<<(std::ostream& out, const TableStatsRequest& obj); + + +class PartitionsStatsRequest : public virtual ::apache::thrift::TBase { + public: + + PartitionsStatsRequest(const PartitionsStatsRequest&); + PartitionsStatsRequest& operator=(const PartitionsStatsRequest&); + PartitionsStatsRequest() : dbName(), tblName() { + } + + virtual ~PartitionsStatsRequest() noexcept; + std::string dbName; + std::string tblName; + std::vector colNames; + std::vector partNames; + + void __set_dbName(const std::string& val); + + void __set_tblName(const std::string& val); + + void __set_colNames(const std::vector & val); + + void __set_partNames(const std::vector & val); + + bool operator == (const PartitionsStatsRequest & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(tblName == rhs.tblName)) + return false; + if (!(colNames == rhs.colNames)) + return false; + if (!(partNames == rhs.partNames)) + return false; + return true; + } + bool operator != (const PartitionsStatsRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PartitionsStatsRequest & ) 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(PartitionsStatsRequest &a, PartitionsStatsRequest &b); + +std::ostream& operator<<(std::ostream& out, const PartitionsStatsRequest& obj); + +typedef struct _AddPartitionsResult__isset { + _AddPartitionsResult__isset() : partitions(false) {} + bool partitions :1; +} _AddPartitionsResult__isset; + +class AddPartitionsResult : public virtual ::apache::thrift::TBase { + public: + + AddPartitionsResult(const AddPartitionsResult&); + AddPartitionsResult& operator=(const AddPartitionsResult&); + AddPartitionsResult() { + } + + virtual ~AddPartitionsResult() noexcept; + std::vector partitions; + + _AddPartitionsResult__isset __isset; + + void __set_partitions(const std::vector & val); + + bool operator == (const AddPartitionsResult & rhs) const + { + if (__isset.partitions != rhs.__isset.partitions) + return false; + else if (__isset.partitions && !(partitions == rhs.partitions)) + return false; + return true; + } + bool operator != (const AddPartitionsResult &rhs) const { + return !(*this == rhs); + } + + bool operator < (const AddPartitionsResult & ) 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(AddPartitionsResult &a, AddPartitionsResult &b); + +std::ostream& operator<<(std::ostream& out, const AddPartitionsResult& obj); + +typedef struct _AddPartitionsRequest__isset { + _AddPartitionsRequest__isset() : needResult(true) {} + bool needResult :1; +} _AddPartitionsRequest__isset; + +class AddPartitionsRequest : public virtual ::apache::thrift::TBase { + public: + + AddPartitionsRequest(const AddPartitionsRequest&); + AddPartitionsRequest& operator=(const AddPartitionsRequest&); + AddPartitionsRequest() : dbName(), tblName(), ifNotExists(0), needResult(true) { + } + + virtual ~AddPartitionsRequest() noexcept; + std::string dbName; + std::string tblName; + std::vector parts; + bool ifNotExists; + bool needResult; + + _AddPartitionsRequest__isset __isset; + + void __set_dbName(const std::string& val); + + void __set_tblName(const std::string& val); + + void __set_parts(const std::vector & val); + + void __set_ifNotExists(const bool val); + + void __set_needResult(const bool val); + + bool operator == (const AddPartitionsRequest & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(tblName == rhs.tblName)) + return false; + if (!(parts == rhs.parts)) + return false; + if (!(ifNotExists == rhs.ifNotExists)) + return false; + if (__isset.needResult != rhs.__isset.needResult) + return false; + else if (__isset.needResult && !(needResult == rhs.needResult)) + return false; + return true; + } + bool operator != (const AddPartitionsRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const AddPartitionsRequest & ) 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(AddPartitionsRequest &a, AddPartitionsRequest &b); + +std::ostream& operator<<(std::ostream& out, const AddPartitionsRequest& obj); + +typedef struct _DropPartitionsResult__isset { + _DropPartitionsResult__isset() : partitions(false) {} + bool partitions :1; +} _DropPartitionsResult__isset; + +class DropPartitionsResult : public virtual ::apache::thrift::TBase { + public: + + DropPartitionsResult(const DropPartitionsResult&); + DropPartitionsResult& operator=(const DropPartitionsResult&); + DropPartitionsResult() { + } + + virtual ~DropPartitionsResult() noexcept; + std::vector partitions; + + _DropPartitionsResult__isset __isset; + + void __set_partitions(const std::vector & val); + + bool operator == (const DropPartitionsResult & rhs) const + { + if (__isset.partitions != rhs.__isset.partitions) + return false; + else if (__isset.partitions && !(partitions == rhs.partitions)) + return false; + return true; + } + bool operator != (const DropPartitionsResult &rhs) const { + return !(*this == rhs); + } + + bool operator < (const DropPartitionsResult & ) 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(DropPartitionsResult &a, DropPartitionsResult &b); + +std::ostream& operator<<(std::ostream& out, const DropPartitionsResult& obj); + +typedef struct _DropPartitionsExpr__isset { + _DropPartitionsExpr__isset() : partArchiveLevel(false) {} + bool partArchiveLevel :1; +} _DropPartitionsExpr__isset; + +class DropPartitionsExpr : public virtual ::apache::thrift::TBase { + public: + + DropPartitionsExpr(const DropPartitionsExpr&); + DropPartitionsExpr& operator=(const DropPartitionsExpr&); + DropPartitionsExpr() : expr(), partArchiveLevel(0) { + } + + virtual ~DropPartitionsExpr() noexcept; + std::string expr; + int32_t partArchiveLevel; + + _DropPartitionsExpr__isset __isset; + + void __set_expr(const std::string& val); + + void __set_partArchiveLevel(const int32_t val); + + bool operator == (const DropPartitionsExpr & rhs) const + { + if (!(expr == rhs.expr)) + return false; + if (__isset.partArchiveLevel != rhs.__isset.partArchiveLevel) + return false; + else if (__isset.partArchiveLevel && !(partArchiveLevel == rhs.partArchiveLevel)) + return false; + return true; + } + bool operator != (const DropPartitionsExpr &rhs) const { + return !(*this == rhs); + } + + bool operator < (const DropPartitionsExpr & ) 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(DropPartitionsExpr &a, DropPartitionsExpr &b); + +std::ostream& operator<<(std::ostream& out, const DropPartitionsExpr& obj); + +typedef struct _RequestPartsSpec__isset { + _RequestPartsSpec__isset() : names(false), exprs(false) {} + bool names :1; + bool exprs :1; +} _RequestPartsSpec__isset; + +class RequestPartsSpec : public virtual ::apache::thrift::TBase { + public: + + RequestPartsSpec(const RequestPartsSpec&); + RequestPartsSpec& operator=(const RequestPartsSpec&); + RequestPartsSpec() { + } + + virtual ~RequestPartsSpec() noexcept; + std::vector names; + std::vector exprs; + + _RequestPartsSpec__isset __isset; + + void __set_names(const std::vector & val); + + void __set_exprs(const std::vector & val); + + bool operator == (const RequestPartsSpec & rhs) const + { + if (__isset.names != rhs.__isset.names) + return false; + else if (__isset.names && !(names == rhs.names)) + return false; + if (__isset.exprs != rhs.__isset.exprs) + return false; + else if (__isset.exprs && !(exprs == rhs.exprs)) + return false; + return true; + } + bool operator != (const RequestPartsSpec &rhs) const { + return !(*this == rhs); + } + + bool operator < (const RequestPartsSpec & ) 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(RequestPartsSpec &a, RequestPartsSpec &b); + +std::ostream& operator<<(std::ostream& out, const RequestPartsSpec& obj); + +typedef struct _DropPartitionsRequest__isset { + _DropPartitionsRequest__isset() : deleteData(false), ifExists(true), ignoreProtection(false), environmentContext(false), needResult(true) {} + bool deleteData :1; + bool ifExists :1; + bool ignoreProtection :1; + bool environmentContext :1; + bool needResult :1; +} _DropPartitionsRequest__isset; + +class DropPartitionsRequest : public virtual ::apache::thrift::TBase { + public: + + DropPartitionsRequest(const DropPartitionsRequest&); + DropPartitionsRequest& operator=(const DropPartitionsRequest&); + DropPartitionsRequest() : dbName(), tblName(), deleteData(0), ifExists(true), ignoreProtection(0), needResult(true) { + } + + virtual ~DropPartitionsRequest() noexcept; + std::string dbName; + std::string tblName; + RequestPartsSpec parts; + bool deleteData; + bool ifExists; + bool ignoreProtection; + EnvironmentContext environmentContext; + bool needResult; + + _DropPartitionsRequest__isset __isset; + + void __set_dbName(const std::string& val); + + void __set_tblName(const std::string& val); + + void __set_parts(const RequestPartsSpec& val); + + void __set_deleteData(const bool val); + + void __set_ifExists(const bool val); + + void __set_ignoreProtection(const bool val); + + void __set_environmentContext(const EnvironmentContext& val); + + void __set_needResult(const bool val); + + bool operator == (const DropPartitionsRequest & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(tblName == rhs.tblName)) + return false; + if (!(parts == rhs.parts)) + return false; + if (__isset.deleteData != rhs.__isset.deleteData) + return false; + else if (__isset.deleteData && !(deleteData == rhs.deleteData)) + return false; + if (__isset.ifExists != rhs.__isset.ifExists) + return false; + else if (__isset.ifExists && !(ifExists == rhs.ifExists)) + return false; + if (__isset.ignoreProtection != rhs.__isset.ignoreProtection) + return false; + else if (__isset.ignoreProtection && !(ignoreProtection == rhs.ignoreProtection)) + return false; + if (__isset.environmentContext != rhs.__isset.environmentContext) + return false; + else if (__isset.environmentContext && !(environmentContext == rhs.environmentContext)) + return false; + if (__isset.needResult != rhs.__isset.needResult) + return false; + else if (__isset.needResult && !(needResult == rhs.needResult)) + return false; + return true; + } + bool operator != (const DropPartitionsRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const DropPartitionsRequest & ) 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(DropPartitionsRequest &a, DropPartitionsRequest &b); + +std::ostream& operator<<(std::ostream& out, const DropPartitionsRequest& obj); + +typedef struct _PartitionValuesRequest__isset { + _PartitionValuesRequest__isset() : applyDistinct(true), filter(false), partitionOrder(false), ascending(true), maxParts(true) {} + bool applyDistinct :1; + bool filter :1; + bool partitionOrder :1; + bool ascending :1; + bool maxParts :1; +} _PartitionValuesRequest__isset; + +class PartitionValuesRequest : public virtual ::apache::thrift::TBase { + public: + + PartitionValuesRequest(const PartitionValuesRequest&); + PartitionValuesRequest& operator=(const PartitionValuesRequest&); + PartitionValuesRequest() : dbName(), tblName(), applyDistinct(true), filter(), ascending(true), maxParts(-1LL) { + } + + virtual ~PartitionValuesRequest() noexcept; + std::string dbName; + std::string tblName; + std::vector partitionKeys; + bool applyDistinct; + std::string filter; + std::vector partitionOrder; + bool ascending; + int64_t maxParts; + + _PartitionValuesRequest__isset __isset; + + void __set_dbName(const std::string& val); + + void __set_tblName(const std::string& val); + + void __set_partitionKeys(const std::vector & val); + + void __set_applyDistinct(const bool val); + + void __set_filter(const std::string& val); + + void __set_partitionOrder(const std::vector & val); + + void __set_ascending(const bool val); + + void __set_maxParts(const int64_t val); + + bool operator == (const PartitionValuesRequest & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(tblName == rhs.tblName)) + return false; + if (!(partitionKeys == rhs.partitionKeys)) + return false; + if (__isset.applyDistinct != rhs.__isset.applyDistinct) + return false; + else if (__isset.applyDistinct && !(applyDistinct == rhs.applyDistinct)) + return false; + if (__isset.filter != rhs.__isset.filter) + return false; + else if (__isset.filter && !(filter == rhs.filter)) + return false; + if (__isset.partitionOrder != rhs.__isset.partitionOrder) + return false; + else if (__isset.partitionOrder && !(partitionOrder == rhs.partitionOrder)) + return false; + if (__isset.ascending != rhs.__isset.ascending) + return false; + else if (__isset.ascending && !(ascending == rhs.ascending)) + return false; + if (__isset.maxParts != rhs.__isset.maxParts) + return false; + else if (__isset.maxParts && !(maxParts == rhs.maxParts)) + return false; + return true; + } + bool operator != (const PartitionValuesRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PartitionValuesRequest & ) 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(PartitionValuesRequest &a, PartitionValuesRequest &b); + +std::ostream& operator<<(std::ostream& out, const PartitionValuesRequest& obj); + + +class PartitionValuesRow : public virtual ::apache::thrift::TBase { + public: + + PartitionValuesRow(const PartitionValuesRow&); + PartitionValuesRow& operator=(const PartitionValuesRow&); + PartitionValuesRow() { + } + + virtual ~PartitionValuesRow() noexcept; + std::vector row; + + void __set_row(const std::vector & val); + + bool operator == (const PartitionValuesRow & rhs) const + { + if (!(row == rhs.row)) + return false; + return true; + } + bool operator != (const PartitionValuesRow &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PartitionValuesRow & ) 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(PartitionValuesRow &a, PartitionValuesRow &b); + +std::ostream& operator<<(std::ostream& out, const PartitionValuesRow& obj); + + +class PartitionValuesResponse : public virtual ::apache::thrift::TBase { + public: + + PartitionValuesResponse(const PartitionValuesResponse&); + PartitionValuesResponse& operator=(const PartitionValuesResponse&); + PartitionValuesResponse() { + } + + virtual ~PartitionValuesResponse() noexcept; + std::vector partitionValues; + + void __set_partitionValues(const std::vector & val); + + bool operator == (const PartitionValuesResponse & rhs) const + { + if (!(partitionValues == rhs.partitionValues)) + return false; + return true; + } + bool operator != (const PartitionValuesResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PartitionValuesResponse & ) 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(PartitionValuesResponse &a, PartitionValuesResponse &b); + +std::ostream& operator<<(std::ostream& out, const PartitionValuesResponse& obj); + +typedef struct _ResourceUri__isset { + _ResourceUri__isset() : resourceType(false), uri(false) {} + bool resourceType :1; + bool uri :1; +} _ResourceUri__isset; + +class ResourceUri : public virtual ::apache::thrift::TBase { + public: + + ResourceUri(const ResourceUri&); + ResourceUri& operator=(const ResourceUri&); + ResourceUri() : resourceType((ResourceType::type)0), uri() { + } + + virtual ~ResourceUri() noexcept; + ResourceType::type resourceType; + std::string uri; + + _ResourceUri__isset __isset; + + void __set_resourceType(const ResourceType::type val); + + void __set_uri(const std::string& val); + + bool operator == (const ResourceUri & rhs) const + { + if (!(resourceType == rhs.resourceType)) + return false; + if (!(uri == rhs.uri)) + return false; + return true; + } + bool operator != (const ResourceUri &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ResourceUri & ) 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(ResourceUri &a, ResourceUri &b); + +std::ostream& operator<<(std::ostream& out, const ResourceUri& obj); + +typedef struct _Function__isset { + _Function__isset() : functionName(false), dbName(false), className(false), ownerName(false), ownerType(false), createTime(false), functionType(false), resourceUris(false) {} + bool functionName :1; + bool dbName :1; + bool className :1; + bool ownerName :1; + bool ownerType :1; + bool createTime :1; + bool functionType :1; + bool resourceUris :1; +} _Function__isset; + +class Function : public virtual ::apache::thrift::TBase { + public: + + Function(const Function&); + Function& operator=(const Function&); + Function() : functionName(), dbName(), className(), ownerName(), ownerType((PrincipalType::type)0), createTime(0), functionType((FunctionType::type)0) { + } + + virtual ~Function() noexcept; + std::string functionName; + std::string dbName; + std::string className; + std::string ownerName; + PrincipalType::type ownerType; + int32_t createTime; + FunctionType::type functionType; + std::vector resourceUris; + + _Function__isset __isset; + + void __set_functionName(const std::string& val); + + void __set_dbName(const std::string& val); + + void __set_className(const std::string& val); + + void __set_ownerName(const std::string& val); + + void __set_ownerType(const PrincipalType::type val); + + void __set_createTime(const int32_t val); + + void __set_functionType(const FunctionType::type val); + + void __set_resourceUris(const std::vector & val); + + bool operator == (const Function & rhs) const + { + if (!(functionName == rhs.functionName)) + return false; + if (!(dbName == rhs.dbName)) + return false; + if (!(className == rhs.className)) + return false; + if (!(ownerName == rhs.ownerName)) + return false; + if (!(ownerType == rhs.ownerType)) + return false; + if (!(createTime == rhs.createTime)) + return false; + if (!(functionType == rhs.functionType)) + return false; + if (!(resourceUris == rhs.resourceUris)) + return false; + return true; + } + bool operator != (const Function &rhs) const { + return !(*this == rhs); + } + + bool operator < (const Function & ) 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(Function &a, Function &b); + +std::ostream& operator<<(std::ostream& out, const Function& obj); + +typedef struct _TxnInfo__isset { + _TxnInfo__isset() : agentInfo(true), heartbeatCount(true), metaInfo(false), startedTime(false), lastHeartbeatTime(false) {} + bool agentInfo :1; + bool heartbeatCount :1; + bool metaInfo :1; + bool startedTime :1; + bool lastHeartbeatTime :1; +} _TxnInfo__isset; + +class TxnInfo : public virtual ::apache::thrift::TBase { + public: + + TxnInfo(const TxnInfo&); + TxnInfo& operator=(const TxnInfo&); + TxnInfo() : id(0), state((TxnState::type)0), user(), hostname(), agentInfo("Unknown"), heartbeatCount(0), metaInfo(), startedTime(0), lastHeartbeatTime(0) { + } + + virtual ~TxnInfo() noexcept; + int64_t id; + TxnState::type state; + std::string user; + std::string hostname; + std::string agentInfo; + int32_t heartbeatCount; + std::string metaInfo; + int64_t startedTime; + int64_t lastHeartbeatTime; + + _TxnInfo__isset __isset; + + void __set_id(const int64_t val); + + void __set_state(const TxnState::type val); + + void __set_user(const std::string& val); + + void __set_hostname(const std::string& val); + + void __set_agentInfo(const std::string& val); + + void __set_heartbeatCount(const int32_t val); + + void __set_metaInfo(const std::string& val); + + void __set_startedTime(const int64_t val); + + void __set_lastHeartbeatTime(const int64_t val); + + bool operator == (const TxnInfo & rhs) const + { + if (!(id == rhs.id)) + return false; + if (!(state == rhs.state)) + return false; + if (!(user == rhs.user)) + return false; + if (!(hostname == rhs.hostname)) + return false; + if (__isset.agentInfo != rhs.__isset.agentInfo) + return false; + else if (__isset.agentInfo && !(agentInfo == rhs.agentInfo)) + return false; + if (__isset.heartbeatCount != rhs.__isset.heartbeatCount) + return false; + else if (__isset.heartbeatCount && !(heartbeatCount == rhs.heartbeatCount)) + return false; + if (__isset.metaInfo != rhs.__isset.metaInfo) + return false; + else if (__isset.metaInfo && !(metaInfo == rhs.metaInfo)) + return false; + if (__isset.startedTime != rhs.__isset.startedTime) + return false; + else if (__isset.startedTime && !(startedTime == rhs.startedTime)) + return false; + if (__isset.lastHeartbeatTime != rhs.__isset.lastHeartbeatTime) + return false; + else if (__isset.lastHeartbeatTime && !(lastHeartbeatTime == rhs.lastHeartbeatTime)) + return false; + return true; + } + bool operator != (const TxnInfo &rhs) const { + return !(*this == rhs); + } + + bool operator < (const TxnInfo & ) 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(TxnInfo &a, TxnInfo &b); + +std::ostream& operator<<(std::ostream& out, const TxnInfo& obj); + + +class GetOpenTxnsInfoResponse : public virtual ::apache::thrift::TBase { + public: + + GetOpenTxnsInfoResponse(const GetOpenTxnsInfoResponse&); + GetOpenTxnsInfoResponse& operator=(const GetOpenTxnsInfoResponse&); + GetOpenTxnsInfoResponse() : txn_high_water_mark(0) { + } + + virtual ~GetOpenTxnsInfoResponse() noexcept; + int64_t txn_high_water_mark; + std::vector open_txns; + + void __set_txn_high_water_mark(const int64_t val); + + void __set_open_txns(const std::vector & val); + + bool operator == (const GetOpenTxnsInfoResponse & rhs) const + { + if (!(txn_high_water_mark == rhs.txn_high_water_mark)) + return false; + if (!(open_txns == rhs.open_txns)) + return false; + return true; + } + bool operator != (const GetOpenTxnsInfoResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GetOpenTxnsInfoResponse & ) 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(GetOpenTxnsInfoResponse &a, GetOpenTxnsInfoResponse &b); + +std::ostream& operator<<(std::ostream& out, const GetOpenTxnsInfoResponse& obj); + +typedef struct _GetOpenTxnsResponse__isset { + _GetOpenTxnsResponse__isset() : min_open_txn(false) {} + bool min_open_txn :1; +} _GetOpenTxnsResponse__isset; + +class GetOpenTxnsResponse : public virtual ::apache::thrift::TBase { + public: + + GetOpenTxnsResponse(const GetOpenTxnsResponse&); + GetOpenTxnsResponse& operator=(const GetOpenTxnsResponse&); + GetOpenTxnsResponse() : txn_high_water_mark(0), min_open_txn(0), abortedBits() { + } + + virtual ~GetOpenTxnsResponse() noexcept; + int64_t txn_high_water_mark; + std::vector open_txns; + int64_t min_open_txn; + std::string abortedBits; + + _GetOpenTxnsResponse__isset __isset; + + void __set_txn_high_water_mark(const int64_t val); + + void __set_open_txns(const std::vector & val); + + void __set_min_open_txn(const int64_t val); + + void __set_abortedBits(const std::string& val); + + bool operator == (const GetOpenTxnsResponse & rhs) const + { + if (!(txn_high_water_mark == rhs.txn_high_water_mark)) + return false; + if (!(open_txns == rhs.open_txns)) + return false; + if (__isset.min_open_txn != rhs.__isset.min_open_txn) + return false; + else if (__isset.min_open_txn && !(min_open_txn == rhs.min_open_txn)) + return false; + if (!(abortedBits == rhs.abortedBits)) + return false; + return true; + } + bool operator != (const GetOpenTxnsResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GetOpenTxnsResponse & ) 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(GetOpenTxnsResponse &a, GetOpenTxnsResponse &b); + +std::ostream& operator<<(std::ostream& out, const GetOpenTxnsResponse& obj); + +typedef struct _OpenTxnRequest__isset { + _OpenTxnRequest__isset() : agentInfo(true) {} + bool agentInfo :1; +} _OpenTxnRequest__isset; + +class OpenTxnRequest : public virtual ::apache::thrift::TBase { + public: + + OpenTxnRequest(const OpenTxnRequest&); + OpenTxnRequest& operator=(const OpenTxnRequest&); + OpenTxnRequest() : num_txns(0), user(), hostname(), agentInfo("Unknown") { + } + + virtual ~OpenTxnRequest() noexcept; + int32_t num_txns; + std::string user; + std::string hostname; + std::string agentInfo; + + _OpenTxnRequest__isset __isset; + + void __set_num_txns(const int32_t val); + + void __set_user(const std::string& val); + + void __set_hostname(const std::string& val); + + void __set_agentInfo(const std::string& val); + + bool operator == (const OpenTxnRequest & rhs) const + { + if (!(num_txns == rhs.num_txns)) + return false; + if (!(user == rhs.user)) + return false; + if (!(hostname == rhs.hostname)) + return false; + if (__isset.agentInfo != rhs.__isset.agentInfo) + return false; + else if (__isset.agentInfo && !(agentInfo == rhs.agentInfo)) + return false; + return true; + } + bool operator != (const OpenTxnRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const OpenTxnRequest & ) 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(OpenTxnRequest &a, OpenTxnRequest &b); + +std::ostream& operator<<(std::ostream& out, const OpenTxnRequest& obj); + + +class OpenTxnsResponse : public virtual ::apache::thrift::TBase { + public: + + OpenTxnsResponse(const OpenTxnsResponse&); + OpenTxnsResponse& operator=(const OpenTxnsResponse&); + OpenTxnsResponse() { + } + + virtual ~OpenTxnsResponse() noexcept; + std::vector txn_ids; + + void __set_txn_ids(const std::vector & val); + + bool operator == (const OpenTxnsResponse & rhs) const + { + if (!(txn_ids == rhs.txn_ids)) + return false; + return true; + } + bool operator != (const OpenTxnsResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const OpenTxnsResponse & ) 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(OpenTxnsResponse &a, OpenTxnsResponse &b); + +std::ostream& operator<<(std::ostream& out, const OpenTxnsResponse& obj); + + +class AbortTxnRequest : public virtual ::apache::thrift::TBase { + public: + + AbortTxnRequest(const AbortTxnRequest&); + AbortTxnRequest& operator=(const AbortTxnRequest&); + AbortTxnRequest() : txnid(0) { + } + + virtual ~AbortTxnRequest() noexcept; + int64_t txnid; + + void __set_txnid(const int64_t val); + + bool operator == (const AbortTxnRequest & rhs) const + { + if (!(txnid == rhs.txnid)) + return false; + return true; + } + bool operator != (const AbortTxnRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const AbortTxnRequest & ) 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(AbortTxnRequest &a, AbortTxnRequest &b); + +std::ostream& operator<<(std::ostream& out, const AbortTxnRequest& obj); + + +class AbortTxnsRequest : public virtual ::apache::thrift::TBase { + public: + + AbortTxnsRequest(const AbortTxnsRequest&); + AbortTxnsRequest& operator=(const AbortTxnsRequest&); + AbortTxnsRequest() { + } + + virtual ~AbortTxnsRequest() noexcept; + std::vector txn_ids; + + void __set_txn_ids(const std::vector & val); + + bool operator == (const AbortTxnsRequest & rhs) const + { + if (!(txn_ids == rhs.txn_ids)) + return false; + return true; + } + bool operator != (const AbortTxnsRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const AbortTxnsRequest & ) 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(AbortTxnsRequest &a, AbortTxnsRequest &b); + +std::ostream& operator<<(std::ostream& out, const AbortTxnsRequest& obj); + + +class CommitTxnRequest : public virtual ::apache::thrift::TBase { + public: + + CommitTxnRequest(const CommitTxnRequest&); + CommitTxnRequest& operator=(const CommitTxnRequest&); + CommitTxnRequest() : txnid(0) { + } + + virtual ~CommitTxnRequest() noexcept; + int64_t txnid; + + void __set_txnid(const int64_t val); + + bool operator == (const CommitTxnRequest & rhs) const + { + if (!(txnid == rhs.txnid)) + return false; + return true; + } + bool operator != (const CommitTxnRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const CommitTxnRequest & ) 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(CommitTxnRequest &a, CommitTxnRequest &b); + +std::ostream& operator<<(std::ostream& out, const CommitTxnRequest& obj); + +typedef struct _LockComponent__isset { + _LockComponent__isset() : tablename(false), partitionname(false), operationType(true), isAcid(true), isDynamicPartitionWrite(true) {} + bool tablename :1; + bool partitionname :1; + bool operationType :1; + bool isAcid :1; + bool isDynamicPartitionWrite :1; +} _LockComponent__isset; + +class LockComponent : public virtual ::apache::thrift::TBase { + public: + + LockComponent(const LockComponent&); + LockComponent& operator=(const LockComponent&); + LockComponent() : type((LockType::type)0), level((LockLevel::type)0), dbname(), tablename(), partitionname(), operationType((DataOperationType::type)5), isAcid(false), isDynamicPartitionWrite(false) { + operationType = (DataOperationType::type)5; + + } + + virtual ~LockComponent() noexcept; + LockType::type type; + LockLevel::type level; + std::string dbname; + std::string tablename; + std::string partitionname; + DataOperationType::type operationType; + bool isAcid; + bool isDynamicPartitionWrite; + + _LockComponent__isset __isset; + + void __set_type(const LockType::type val); + + void __set_level(const LockLevel::type val); + + void __set_dbname(const std::string& val); + + void __set_tablename(const std::string& val); + + void __set_partitionname(const std::string& val); + + void __set_operationType(const DataOperationType::type val); + + void __set_isAcid(const bool val); + + void __set_isDynamicPartitionWrite(const bool val); + + bool operator == (const LockComponent & rhs) const + { + if (!(type == rhs.type)) + return false; + if (!(level == rhs.level)) + return false; + if (!(dbname == rhs.dbname)) + return false; + if (__isset.tablename != rhs.__isset.tablename) + return false; + else if (__isset.tablename && !(tablename == rhs.tablename)) + return false; + if (__isset.partitionname != rhs.__isset.partitionname) + return false; + else if (__isset.partitionname && !(partitionname == rhs.partitionname)) + return false; + if (__isset.operationType != rhs.__isset.operationType) + return false; + else if (__isset.operationType && !(operationType == rhs.operationType)) + return false; + if (__isset.isAcid != rhs.__isset.isAcid) + return false; + else if (__isset.isAcid && !(isAcid == rhs.isAcid)) + return false; + if (__isset.isDynamicPartitionWrite != rhs.__isset.isDynamicPartitionWrite) + return false; + else if (__isset.isDynamicPartitionWrite && !(isDynamicPartitionWrite == rhs.isDynamicPartitionWrite)) + return false; + return true; + } + bool operator != (const LockComponent &rhs) const { + return !(*this == rhs); + } + + bool operator < (const LockComponent & ) 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(LockComponent &a, LockComponent &b); + +std::ostream& operator<<(std::ostream& out, const LockComponent& obj); + +typedef struct _LockRequest__isset { + _LockRequest__isset() : txnid(false), agentInfo(true) {} + bool txnid :1; + bool agentInfo :1; +} _LockRequest__isset; + +class LockRequest : public virtual ::apache::thrift::TBase { + public: + + LockRequest(const LockRequest&); + LockRequest& operator=(const LockRequest&); + LockRequest() : txnid(0), user(), hostname(), agentInfo("Unknown") { + } + + virtual ~LockRequest() noexcept; + std::vector component; + int64_t txnid; + std::string user; + std::string hostname; + std::string agentInfo; + + _LockRequest__isset __isset; + + void __set_component(const std::vector & val); + + void __set_txnid(const int64_t val); + + void __set_user(const std::string& val); + + void __set_hostname(const std::string& val); + + void __set_agentInfo(const std::string& val); + + bool operator == (const LockRequest & rhs) const + { + if (!(component == rhs.component)) + return false; + if (__isset.txnid != rhs.__isset.txnid) + return false; + else if (__isset.txnid && !(txnid == rhs.txnid)) + return false; + if (!(user == rhs.user)) + return false; + if (!(hostname == rhs.hostname)) + return false; + if (__isset.agentInfo != rhs.__isset.agentInfo) + return false; + else if (__isset.agentInfo && !(agentInfo == rhs.agentInfo)) + return false; + return true; + } + bool operator != (const LockRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const LockRequest & ) 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(LockRequest &a, LockRequest &b); + +std::ostream& operator<<(std::ostream& out, const LockRequest& obj); + + +class LockResponse : public virtual ::apache::thrift::TBase { + public: + + LockResponse(const LockResponse&); + LockResponse& operator=(const LockResponse&); + LockResponse() : lockid(0), state((LockState::type)0) { + } + + virtual ~LockResponse() noexcept; + int64_t lockid; + LockState::type state; + + void __set_lockid(const int64_t val); + + void __set_state(const LockState::type val); + + bool operator == (const LockResponse & rhs) const + { + if (!(lockid == rhs.lockid)) + return false; + if (!(state == rhs.state)) + return false; + return true; + } + bool operator != (const LockResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const LockResponse & ) 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(LockResponse &a, LockResponse &b); + +std::ostream& operator<<(std::ostream& out, const LockResponse& obj); + +typedef struct _CheckLockRequest__isset { + _CheckLockRequest__isset() : txnid(false), elapsed_ms(false) {} + bool txnid :1; + bool elapsed_ms :1; +} _CheckLockRequest__isset; + +class CheckLockRequest : public virtual ::apache::thrift::TBase { + public: + + CheckLockRequest(const CheckLockRequest&); + CheckLockRequest& operator=(const CheckLockRequest&); + CheckLockRequest() : lockid(0), txnid(0), elapsed_ms(0) { + } + + virtual ~CheckLockRequest() noexcept; + int64_t lockid; + int64_t txnid; + int64_t elapsed_ms; + + _CheckLockRequest__isset __isset; + + void __set_lockid(const int64_t val); + + void __set_txnid(const int64_t val); + + void __set_elapsed_ms(const int64_t val); + + bool operator == (const CheckLockRequest & rhs) const + { + if (!(lockid == rhs.lockid)) + return false; + if (__isset.txnid != rhs.__isset.txnid) + return false; + else if (__isset.txnid && !(txnid == rhs.txnid)) + return false; + if (__isset.elapsed_ms != rhs.__isset.elapsed_ms) + return false; + else if (__isset.elapsed_ms && !(elapsed_ms == rhs.elapsed_ms)) + return false; + return true; + } + bool operator != (const CheckLockRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const CheckLockRequest & ) 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(CheckLockRequest &a, CheckLockRequest &b); + +std::ostream& operator<<(std::ostream& out, const CheckLockRequest& obj); + + +class UnlockRequest : public virtual ::apache::thrift::TBase { + public: + + UnlockRequest(const UnlockRequest&); + UnlockRequest& operator=(const UnlockRequest&); + UnlockRequest() : lockid(0) { + } + + virtual ~UnlockRequest() noexcept; + int64_t lockid; + + void __set_lockid(const int64_t val); + + bool operator == (const UnlockRequest & rhs) const + { + if (!(lockid == rhs.lockid)) + return false; + return true; + } + bool operator != (const UnlockRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const UnlockRequest & ) 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(UnlockRequest &a, UnlockRequest &b); + +std::ostream& operator<<(std::ostream& out, const UnlockRequest& obj); + +typedef struct _ShowLocksRequest__isset { + _ShowLocksRequest__isset() : dbname(false), tablename(false), partname(false), isExtended(true) {} + bool dbname :1; + bool tablename :1; + bool partname :1; + bool isExtended :1; +} _ShowLocksRequest__isset; + +class ShowLocksRequest : public virtual ::apache::thrift::TBase { + public: + + ShowLocksRequest(const ShowLocksRequest&); + ShowLocksRequest& operator=(const ShowLocksRequest&); + ShowLocksRequest() : dbname(), tablename(), partname(), isExtended(false) { + } + + virtual ~ShowLocksRequest() noexcept; + std::string dbname; + std::string tablename; + std::string partname; + bool isExtended; + + _ShowLocksRequest__isset __isset; + + void __set_dbname(const std::string& val); + + void __set_tablename(const std::string& val); + + void __set_partname(const std::string& val); + + void __set_isExtended(const bool val); + + bool operator == (const ShowLocksRequest & rhs) const + { + if (__isset.dbname != rhs.__isset.dbname) + return false; + else if (__isset.dbname && !(dbname == rhs.dbname)) + return false; + if (__isset.tablename != rhs.__isset.tablename) + return false; + else if (__isset.tablename && !(tablename == rhs.tablename)) + return false; + if (__isset.partname != rhs.__isset.partname) + return false; + else if (__isset.partname && !(partname == rhs.partname)) + return false; + if (__isset.isExtended != rhs.__isset.isExtended) + return false; + else if (__isset.isExtended && !(isExtended == rhs.isExtended)) + return false; + return true; + } + bool operator != (const ShowLocksRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ShowLocksRequest & ) 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(ShowLocksRequest &a, ShowLocksRequest &b); + +std::ostream& operator<<(std::ostream& out, const ShowLocksRequest& obj); + +typedef struct _ShowLocksResponseElement__isset { + _ShowLocksResponseElement__isset() : tablename(false), partname(false), txnid(false), acquiredat(false), heartbeatCount(true), agentInfo(false), blockedByExtId(false), blockedByIntId(false), lockIdInternal(false) {} + bool tablename :1; + bool partname :1; + bool txnid :1; + bool acquiredat :1; + bool heartbeatCount :1; + bool agentInfo :1; + bool blockedByExtId :1; + bool blockedByIntId :1; + bool lockIdInternal :1; +} _ShowLocksResponseElement__isset; + +class ShowLocksResponseElement : public virtual ::apache::thrift::TBase { + public: + + ShowLocksResponseElement(const ShowLocksResponseElement&); + ShowLocksResponseElement& operator=(const ShowLocksResponseElement&); + ShowLocksResponseElement() : lockid(0), dbname(), tablename(), partname(), state((LockState::type)0), type((LockType::type)0), txnid(0), lastheartbeat(0), acquiredat(0), user(), hostname(), heartbeatCount(0), agentInfo(), blockedByExtId(0), blockedByIntId(0), lockIdInternal(0) { + } + + virtual ~ShowLocksResponseElement() noexcept; + int64_t lockid; + std::string dbname; + std::string tablename; + std::string partname; + LockState::type state; + LockType::type type; + int64_t txnid; + int64_t lastheartbeat; + int64_t acquiredat; + std::string user; + std::string hostname; + int32_t heartbeatCount; + std::string agentInfo; + int64_t blockedByExtId; + int64_t blockedByIntId; + int64_t lockIdInternal; + + _ShowLocksResponseElement__isset __isset; + + void __set_lockid(const int64_t val); + + void __set_dbname(const std::string& val); + + void __set_tablename(const std::string& val); + + void __set_partname(const std::string& val); + + void __set_state(const LockState::type val); + + void __set_type(const LockType::type val); + + void __set_txnid(const int64_t val); + + void __set_lastheartbeat(const int64_t val); + + void __set_acquiredat(const int64_t val); + + void __set_user(const std::string& val); + + void __set_hostname(const std::string& val); + + void __set_heartbeatCount(const int32_t val); + + void __set_agentInfo(const std::string& val); + + void __set_blockedByExtId(const int64_t val); + + void __set_blockedByIntId(const int64_t val); + + void __set_lockIdInternal(const int64_t val); + + bool operator == (const ShowLocksResponseElement & rhs) const + { + if (!(lockid == rhs.lockid)) + return false; + if (!(dbname == rhs.dbname)) + return false; + if (__isset.tablename != rhs.__isset.tablename) + return false; + else if (__isset.tablename && !(tablename == rhs.tablename)) + return false; + if (__isset.partname != rhs.__isset.partname) + return false; + else if (__isset.partname && !(partname == rhs.partname)) + return false; + if (!(state == rhs.state)) + return false; + if (!(type == rhs.type)) + return false; + if (__isset.txnid != rhs.__isset.txnid) + return false; + else if (__isset.txnid && !(txnid == rhs.txnid)) + return false; + if (!(lastheartbeat == rhs.lastheartbeat)) + return false; + if (__isset.acquiredat != rhs.__isset.acquiredat) + return false; + else if (__isset.acquiredat && !(acquiredat == rhs.acquiredat)) + return false; + if (!(user == rhs.user)) + return false; + if (!(hostname == rhs.hostname)) + return false; + if (__isset.heartbeatCount != rhs.__isset.heartbeatCount) + return false; + else if (__isset.heartbeatCount && !(heartbeatCount == rhs.heartbeatCount)) + return false; + if (__isset.agentInfo != rhs.__isset.agentInfo) + return false; + else if (__isset.agentInfo && !(agentInfo == rhs.agentInfo)) + return false; + if (__isset.blockedByExtId != rhs.__isset.blockedByExtId) + return false; + else if (__isset.blockedByExtId && !(blockedByExtId == rhs.blockedByExtId)) + return false; + if (__isset.blockedByIntId != rhs.__isset.blockedByIntId) + return false; + else if (__isset.blockedByIntId && !(blockedByIntId == rhs.blockedByIntId)) + return false; + if (__isset.lockIdInternal != rhs.__isset.lockIdInternal) + return false; + else if (__isset.lockIdInternal && !(lockIdInternal == rhs.lockIdInternal)) + return false; + return true; + } + bool operator != (const ShowLocksResponseElement &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ShowLocksResponseElement & ) 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(ShowLocksResponseElement &a, ShowLocksResponseElement &b); + +std::ostream& operator<<(std::ostream& out, const ShowLocksResponseElement& obj); + +typedef struct _ShowLocksResponse__isset { + _ShowLocksResponse__isset() : locks(false) {} + bool locks :1; +} _ShowLocksResponse__isset; + +class ShowLocksResponse : public virtual ::apache::thrift::TBase { + public: + + ShowLocksResponse(const ShowLocksResponse&); + ShowLocksResponse& operator=(const ShowLocksResponse&); + ShowLocksResponse() { + } + + virtual ~ShowLocksResponse() noexcept; + std::vector locks; + + _ShowLocksResponse__isset __isset; + + void __set_locks(const std::vector & val); + + bool operator == (const ShowLocksResponse & rhs) const + { + if (!(locks == rhs.locks)) + return false; + return true; + } + bool operator != (const ShowLocksResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ShowLocksResponse & ) 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(ShowLocksResponse &a, ShowLocksResponse &b); + +std::ostream& operator<<(std::ostream& out, const ShowLocksResponse& obj); + +typedef struct _HeartbeatRequest__isset { + _HeartbeatRequest__isset() : lockid(false), txnid(false) {} + bool lockid :1; + bool txnid :1; +} _HeartbeatRequest__isset; + +class HeartbeatRequest : public virtual ::apache::thrift::TBase { + public: + + HeartbeatRequest(const HeartbeatRequest&); + HeartbeatRequest& operator=(const HeartbeatRequest&); + HeartbeatRequest() : lockid(0), txnid(0) { + } + + virtual ~HeartbeatRequest() noexcept; + int64_t lockid; + int64_t txnid; + + _HeartbeatRequest__isset __isset; + + void __set_lockid(const int64_t val); + + void __set_txnid(const int64_t val); + + bool operator == (const HeartbeatRequest & rhs) const + { + if (__isset.lockid != rhs.__isset.lockid) + return false; + else if (__isset.lockid && !(lockid == rhs.lockid)) + return false; + if (__isset.txnid != rhs.__isset.txnid) + return false; + else if (__isset.txnid && !(txnid == rhs.txnid)) + return false; + return true; + } + bool operator != (const HeartbeatRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const HeartbeatRequest & ) 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(HeartbeatRequest &a, HeartbeatRequest &b); + +std::ostream& operator<<(std::ostream& out, const HeartbeatRequest& obj); + + +class HeartbeatTxnRangeRequest : public virtual ::apache::thrift::TBase { + public: + + HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest&); + HeartbeatTxnRangeRequest& operator=(const HeartbeatTxnRangeRequest&); + HeartbeatTxnRangeRequest() : min(0), max(0) { + } + + virtual ~HeartbeatTxnRangeRequest() noexcept; + int64_t min; + int64_t max; + + void __set_min(const int64_t val); + + void __set_max(const int64_t val); + + bool operator == (const HeartbeatTxnRangeRequest & rhs) const + { + if (!(min == rhs.min)) + return false; + if (!(max == rhs.max)) + return false; + return true; + } + bool operator != (const HeartbeatTxnRangeRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const HeartbeatTxnRangeRequest & ) 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(HeartbeatTxnRangeRequest &a, HeartbeatTxnRangeRequest &b); + +std::ostream& operator<<(std::ostream& out, const HeartbeatTxnRangeRequest& obj); + + +class HeartbeatTxnRangeResponse : public virtual ::apache::thrift::TBase { + public: + + HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse&); + HeartbeatTxnRangeResponse& operator=(const HeartbeatTxnRangeResponse&); + HeartbeatTxnRangeResponse() { + } + + virtual ~HeartbeatTxnRangeResponse() noexcept; + std::set aborted; + std::set nosuch; + + void __set_aborted(const std::set & val); + + void __set_nosuch(const std::set & val); + + bool operator == (const HeartbeatTxnRangeResponse & rhs) const + { + if (!(aborted == rhs.aborted)) + return false; + if (!(nosuch == rhs.nosuch)) + return false; + return true; + } + bool operator != (const HeartbeatTxnRangeResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const HeartbeatTxnRangeResponse & ) 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(HeartbeatTxnRangeResponse &a, HeartbeatTxnRangeResponse &b); + +std::ostream& operator<<(std::ostream& out, const HeartbeatTxnRangeResponse& obj); + +typedef struct _CompactionRequest__isset { + _CompactionRequest__isset() : partitionname(false), runas(false), properties(false) {} + bool partitionname :1; + bool runas :1; + bool properties :1; +} _CompactionRequest__isset; + +class CompactionRequest : public virtual ::apache::thrift::TBase { + public: + + CompactionRequest(const CompactionRequest&); + CompactionRequest& operator=(const CompactionRequest&); + CompactionRequest() : dbname(), tablename(), partitionname(), type((CompactionType::type)0), runas() { + } + + virtual ~CompactionRequest() noexcept; + std::string dbname; + std::string tablename; + std::string partitionname; + CompactionType::type type; + std::string runas; + std::map properties; + + _CompactionRequest__isset __isset; + + void __set_dbname(const std::string& val); + + void __set_tablename(const std::string& val); + + void __set_partitionname(const std::string& val); + + void __set_type(const CompactionType::type val); + + void __set_runas(const std::string& val); + + void __set_properties(const std::map & val); + + bool operator == (const CompactionRequest & rhs) const + { + if (!(dbname == rhs.dbname)) + return false; + if (!(tablename == rhs.tablename)) + return false; + if (__isset.partitionname != rhs.__isset.partitionname) + return false; + else if (__isset.partitionname && !(partitionname == rhs.partitionname)) + return false; + if (!(type == rhs.type)) + return false; + if (__isset.runas != rhs.__isset.runas) + return false; + else if (__isset.runas && !(runas == rhs.runas)) + return false; + if (__isset.properties != rhs.__isset.properties) + return false; + else if (__isset.properties && !(properties == rhs.properties)) + return false; + return true; + } + bool operator != (const CompactionRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const CompactionRequest & ) 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(CompactionRequest &a, CompactionRequest &b); + +std::ostream& operator<<(std::ostream& out, const CompactionRequest& obj); + + +class CompactionResponse : public virtual ::apache::thrift::TBase { + public: + + CompactionResponse(const CompactionResponse&); + CompactionResponse& operator=(const CompactionResponse&); + CompactionResponse() : id(0), state(), accepted(0) { + } + + virtual ~CompactionResponse() noexcept; + int64_t id; + std::string state; + bool accepted; + + void __set_id(const int64_t val); + + void __set_state(const std::string& val); + + void __set_accepted(const bool val); + + bool operator == (const CompactionResponse & rhs) const + { + if (!(id == rhs.id)) + return false; + if (!(state == rhs.state)) + return false; + if (!(accepted == rhs.accepted)) + return false; + return true; + } + bool operator != (const CompactionResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const CompactionResponse & ) 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(CompactionResponse &a, CompactionResponse &b); + +std::ostream& operator<<(std::ostream& out, const CompactionResponse& obj); + + +class ShowCompactRequest : public virtual ::apache::thrift::TBase { + public: + + ShowCompactRequest(const ShowCompactRequest&); + ShowCompactRequest& operator=(const ShowCompactRequest&); + ShowCompactRequest() { + } + + virtual ~ShowCompactRequest() noexcept; + + bool operator == (const ShowCompactRequest & /* rhs */) const + { + return true; + } + bool operator != (const ShowCompactRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ShowCompactRequest & ) 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(ShowCompactRequest &a, ShowCompactRequest &b); + +std::ostream& operator<<(std::ostream& out, const ShowCompactRequest& obj); + +typedef struct _ShowCompactResponseElement__isset { + _ShowCompactResponseElement__isset() : partitionname(false), workerid(false), start(false), runAs(false), hightestTxnId(false), metaInfo(false), endTime(false), hadoopJobId(true), id(false) {} + bool partitionname :1; + bool workerid :1; + bool start :1; + bool runAs :1; + bool hightestTxnId :1; + bool metaInfo :1; + bool endTime :1; + bool hadoopJobId :1; + bool id :1; +} _ShowCompactResponseElement__isset; + +class ShowCompactResponseElement : public virtual ::apache::thrift::TBase { + public: + + ShowCompactResponseElement(const ShowCompactResponseElement&); + ShowCompactResponseElement& operator=(const ShowCompactResponseElement&); + ShowCompactResponseElement() : dbname(), tablename(), partitionname(), type((CompactionType::type)0), state(), workerid(), start(0), runAs(), hightestTxnId(0), metaInfo(), endTime(0), hadoopJobId("None"), id(0) { + } + + virtual ~ShowCompactResponseElement() noexcept; + std::string dbname; + std::string tablename; + std::string partitionname; + CompactionType::type type; + std::string state; + std::string workerid; + int64_t start; + std::string runAs; + int64_t hightestTxnId; + std::string metaInfo; + int64_t endTime; + std::string hadoopJobId; + int64_t id; + + _ShowCompactResponseElement__isset __isset; + + void __set_dbname(const std::string& val); + + void __set_tablename(const std::string& val); + + void __set_partitionname(const std::string& val); + + void __set_type(const CompactionType::type val); + + void __set_state(const std::string& val); + + void __set_workerid(const std::string& val); + + void __set_start(const int64_t val); + + void __set_runAs(const std::string& val); + + void __set_hightestTxnId(const int64_t val); + + void __set_metaInfo(const std::string& val); + + void __set_endTime(const int64_t val); + + void __set_hadoopJobId(const std::string& val); + + void __set_id(const int64_t val); + + bool operator == (const ShowCompactResponseElement & rhs) const + { + if (!(dbname == rhs.dbname)) + return false; + if (!(tablename == rhs.tablename)) + return false; + if (__isset.partitionname != rhs.__isset.partitionname) + return false; + else if (__isset.partitionname && !(partitionname == rhs.partitionname)) + return false; + if (!(type == rhs.type)) + return false; + if (!(state == rhs.state)) + return false; + if (__isset.workerid != rhs.__isset.workerid) + return false; + else if (__isset.workerid && !(workerid == rhs.workerid)) + return false; + if (__isset.start != rhs.__isset.start) + return false; + else if (__isset.start && !(start == rhs.start)) + return false; + if (__isset.runAs != rhs.__isset.runAs) + return false; + else if (__isset.runAs && !(runAs == rhs.runAs)) + return false; + if (__isset.hightestTxnId != rhs.__isset.hightestTxnId) + return false; + else if (__isset.hightestTxnId && !(hightestTxnId == rhs.hightestTxnId)) + return false; + if (__isset.metaInfo != rhs.__isset.metaInfo) + return false; + else if (__isset.metaInfo && !(metaInfo == rhs.metaInfo)) + return false; + if (__isset.endTime != rhs.__isset.endTime) + return false; + else if (__isset.endTime && !(endTime == rhs.endTime)) + return false; + if (__isset.hadoopJobId != rhs.__isset.hadoopJobId) + return false; + else if (__isset.hadoopJobId && !(hadoopJobId == rhs.hadoopJobId)) + return false; + if (__isset.id != rhs.__isset.id) + return false; + else if (__isset.id && !(id == rhs.id)) + return false; + return true; + } + bool operator != (const ShowCompactResponseElement &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ShowCompactResponseElement & ) 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(ShowCompactResponseElement &a, ShowCompactResponseElement &b); + +std::ostream& operator<<(std::ostream& out, const ShowCompactResponseElement& obj); + + +class ShowCompactResponse : public virtual ::apache::thrift::TBase { + public: + + ShowCompactResponse(const ShowCompactResponse&); + ShowCompactResponse& operator=(const ShowCompactResponse&); + ShowCompactResponse() { + } + + virtual ~ShowCompactResponse() noexcept; + std::vector compacts; + + void __set_compacts(const std::vector & val); + + bool operator == (const ShowCompactResponse & rhs) const + { + if (!(compacts == rhs.compacts)) + return false; + return true; + } + bool operator != (const ShowCompactResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ShowCompactResponse & ) 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(ShowCompactResponse &a, ShowCompactResponse &b); + +std::ostream& operator<<(std::ostream& out, const ShowCompactResponse& obj); + +typedef struct _AddDynamicPartitions__isset { + _AddDynamicPartitions__isset() : operationType(true) {} + bool operationType :1; +} _AddDynamicPartitions__isset; + +class AddDynamicPartitions : public virtual ::apache::thrift::TBase { + public: + + AddDynamicPartitions(const AddDynamicPartitions&); + AddDynamicPartitions& operator=(const AddDynamicPartitions&); + AddDynamicPartitions() : txnid(0), dbname(), tablename(), operationType((DataOperationType::type)5) { + operationType = (DataOperationType::type)5; + + } + + virtual ~AddDynamicPartitions() noexcept; + int64_t txnid; + std::string dbname; + std::string tablename; + std::vector partitionnames; + DataOperationType::type operationType; + + _AddDynamicPartitions__isset __isset; + + void __set_txnid(const int64_t val); + + void __set_dbname(const std::string& val); + + void __set_tablename(const std::string& val); + + void __set_partitionnames(const std::vector & val); + + void __set_operationType(const DataOperationType::type val); + + bool operator == (const AddDynamicPartitions & rhs) const + { + if (!(txnid == rhs.txnid)) + return false; + if (!(dbname == rhs.dbname)) + return false; + if (!(tablename == rhs.tablename)) + return false; + if (!(partitionnames == rhs.partitionnames)) + return false; + if (__isset.operationType != rhs.__isset.operationType) + return false; + else if (__isset.operationType && !(operationType == rhs.operationType)) + return false; + return true; + } + bool operator != (const AddDynamicPartitions &rhs) const { + return !(*this == rhs); + } + + bool operator < (const AddDynamicPartitions & ) 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(AddDynamicPartitions &a, AddDynamicPartitions &b); + +std::ostream& operator<<(std::ostream& out, const AddDynamicPartitions& obj); + +typedef struct _NotificationEventRequest__isset { + _NotificationEventRequest__isset() : maxEvents(false) {} + bool maxEvents :1; +} _NotificationEventRequest__isset; + +class NotificationEventRequest : public virtual ::apache::thrift::TBase { + public: + + NotificationEventRequest(const NotificationEventRequest&); + NotificationEventRequest& operator=(const NotificationEventRequest&); + NotificationEventRequest() : lastEvent(0), maxEvents(0) { + } + + virtual ~NotificationEventRequest() noexcept; + int64_t lastEvent; + int32_t maxEvents; + + _NotificationEventRequest__isset __isset; + + void __set_lastEvent(const int64_t val); + + void __set_maxEvents(const int32_t val); + + bool operator == (const NotificationEventRequest & rhs) const + { + if (!(lastEvent == rhs.lastEvent)) + return false; + if (__isset.maxEvents != rhs.__isset.maxEvents) + return false; + else if (__isset.maxEvents && !(maxEvents == rhs.maxEvents)) + return false; + return true; + } + bool operator != (const NotificationEventRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const NotificationEventRequest & ) 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(NotificationEventRequest &a, NotificationEventRequest &b); + +std::ostream& operator<<(std::ostream& out, const NotificationEventRequest& obj); + +typedef struct _NotificationEvent__isset { + _NotificationEvent__isset() : dbName(false), tableName(false), messageFormat(false) {} + bool dbName :1; + bool tableName :1; + bool messageFormat :1; +} _NotificationEvent__isset; + +class NotificationEvent : public virtual ::apache::thrift::TBase { + public: + + NotificationEvent(const NotificationEvent&); + NotificationEvent& operator=(const NotificationEvent&); + NotificationEvent() : eventId(0), eventTime(0), eventType(), dbName(), tableName(), message(), messageFormat() { + } + + virtual ~NotificationEvent() noexcept; + int64_t eventId; + int32_t eventTime; + std::string eventType; + std::string dbName; + std::string tableName; + std::string message; + std::string messageFormat; + + _NotificationEvent__isset __isset; + + void __set_eventId(const int64_t val); + + void __set_eventTime(const int32_t val); + + void __set_eventType(const std::string& val); + + void __set_dbName(const std::string& val); + + void __set_tableName(const std::string& val); + + void __set_message(const std::string& val); + + void __set_messageFormat(const std::string& val); + + bool operator == (const NotificationEvent & rhs) const + { + if (!(eventId == rhs.eventId)) + return false; + if (!(eventTime == rhs.eventTime)) + return false; + if (!(eventType == rhs.eventType)) + return false; + if (__isset.dbName != rhs.__isset.dbName) + return false; + else if (__isset.dbName && !(dbName == rhs.dbName)) + return false; + if (__isset.tableName != rhs.__isset.tableName) + return false; + else if (__isset.tableName && !(tableName == rhs.tableName)) + return false; + if (!(message == rhs.message)) + return false; + if (__isset.messageFormat != rhs.__isset.messageFormat) + return false; + else if (__isset.messageFormat && !(messageFormat == rhs.messageFormat)) + return false; + return true; + } + bool operator != (const NotificationEvent &rhs) const { + return !(*this == rhs); + } + + bool operator < (const NotificationEvent & ) 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(NotificationEvent &a, NotificationEvent &b); + +std::ostream& operator<<(std::ostream& out, const NotificationEvent& obj); + + +class NotificationEventResponse : public virtual ::apache::thrift::TBase { + public: + + NotificationEventResponse(const NotificationEventResponse&); + NotificationEventResponse& operator=(const NotificationEventResponse&); + NotificationEventResponse() { + } + + virtual ~NotificationEventResponse() noexcept; + std::vector events; + + void __set_events(const std::vector & val); + + bool operator == (const NotificationEventResponse & rhs) const + { + if (!(events == rhs.events)) + return false; + return true; + } + bool operator != (const NotificationEventResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const NotificationEventResponse & ) 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(NotificationEventResponse &a, NotificationEventResponse &b); + +std::ostream& operator<<(std::ostream& out, const NotificationEventResponse& obj); + + +class CurrentNotificationEventId : public virtual ::apache::thrift::TBase { + public: + + CurrentNotificationEventId(const CurrentNotificationEventId&); + CurrentNotificationEventId& operator=(const CurrentNotificationEventId&); + CurrentNotificationEventId() : eventId(0) { + } + + virtual ~CurrentNotificationEventId() noexcept; + int64_t eventId; + + void __set_eventId(const int64_t val); + + bool operator == (const CurrentNotificationEventId & rhs) const + { + if (!(eventId == rhs.eventId)) + return false; + return true; + } + bool operator != (const CurrentNotificationEventId &rhs) const { + return !(*this == rhs); + } + + bool operator < (const CurrentNotificationEventId & ) 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(CurrentNotificationEventId &a, CurrentNotificationEventId &b); + +std::ostream& operator<<(std::ostream& out, const CurrentNotificationEventId& obj); + + +class NotificationEventsCountRequest : public virtual ::apache::thrift::TBase { + public: + + NotificationEventsCountRequest(const NotificationEventsCountRequest&); + NotificationEventsCountRequest& operator=(const NotificationEventsCountRequest&); + NotificationEventsCountRequest() : fromEventId(0), dbName() { + } + + virtual ~NotificationEventsCountRequest() noexcept; + int64_t fromEventId; + std::string dbName; + + void __set_fromEventId(const int64_t val); + + void __set_dbName(const std::string& val); + + bool operator == (const NotificationEventsCountRequest & rhs) const + { + if (!(fromEventId == rhs.fromEventId)) + return false; + if (!(dbName == rhs.dbName)) + return false; + return true; + } + bool operator != (const NotificationEventsCountRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const NotificationEventsCountRequest & ) 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(NotificationEventsCountRequest &a, NotificationEventsCountRequest &b); + +std::ostream& operator<<(std::ostream& out, const NotificationEventsCountRequest& obj); + + +class NotificationEventsCountResponse : public virtual ::apache::thrift::TBase { + public: + + NotificationEventsCountResponse(const NotificationEventsCountResponse&); + NotificationEventsCountResponse& operator=(const NotificationEventsCountResponse&); + NotificationEventsCountResponse() : eventsCount(0) { + } + + virtual ~NotificationEventsCountResponse() noexcept; + int64_t eventsCount; + + void __set_eventsCount(const int64_t val); + + bool operator == (const NotificationEventsCountResponse & rhs) const + { + if (!(eventsCount == rhs.eventsCount)) + return false; + return true; + } + bool operator != (const NotificationEventsCountResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const NotificationEventsCountResponse & ) 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(NotificationEventsCountResponse &a, NotificationEventsCountResponse &b); + +std::ostream& operator<<(std::ostream& out, const NotificationEventsCountResponse& obj); + +typedef struct _InsertEventRequestData__isset { + _InsertEventRequestData__isset() : replace(false), filesAddedChecksum(false) {} + bool replace :1; + bool filesAddedChecksum :1; +} _InsertEventRequestData__isset; + +class InsertEventRequestData : public virtual ::apache::thrift::TBase { + public: + + InsertEventRequestData(const InsertEventRequestData&); + InsertEventRequestData& operator=(const InsertEventRequestData&); + InsertEventRequestData() : replace(0) { + } + + virtual ~InsertEventRequestData() noexcept; + bool replace; + std::vector filesAdded; + std::vector filesAddedChecksum; + + _InsertEventRequestData__isset __isset; + + void __set_replace(const bool val); + + void __set_filesAdded(const std::vector & val); + + void __set_filesAddedChecksum(const std::vector & val); + + bool operator == (const InsertEventRequestData & rhs) const + { + if (__isset.replace != rhs.__isset.replace) + return false; + else if (__isset.replace && !(replace == rhs.replace)) + return false; + if (!(filesAdded == rhs.filesAdded)) + return false; + if (__isset.filesAddedChecksum != rhs.__isset.filesAddedChecksum) + return false; + else if (__isset.filesAddedChecksum && !(filesAddedChecksum == rhs.filesAddedChecksum)) + return false; + return true; + } + bool operator != (const InsertEventRequestData &rhs) const { + return !(*this == rhs); + } + + bool operator < (const InsertEventRequestData & ) 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(InsertEventRequestData &a, InsertEventRequestData &b); + +std::ostream& operator<<(std::ostream& out, const InsertEventRequestData& obj); + +typedef struct _FireEventRequestData__isset { + _FireEventRequestData__isset() : insertData(false) {} + bool insertData :1; +} _FireEventRequestData__isset; + +class FireEventRequestData : public virtual ::apache::thrift::TBase { + public: + + FireEventRequestData(const FireEventRequestData&); + FireEventRequestData& operator=(const FireEventRequestData&); + FireEventRequestData() { + } + + virtual ~FireEventRequestData() noexcept; + InsertEventRequestData insertData; + + _FireEventRequestData__isset __isset; + + void __set_insertData(const InsertEventRequestData& val); + + bool operator == (const FireEventRequestData & rhs) const + { + if (__isset.insertData != rhs.__isset.insertData) + return false; + else if (__isset.insertData && !(insertData == rhs.insertData)) + return false; + return true; + } + bool operator != (const FireEventRequestData &rhs) const { + return !(*this == rhs); + } + + bool operator < (const FireEventRequestData & ) 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(FireEventRequestData &a, FireEventRequestData &b); + +std::ostream& operator<<(std::ostream& out, const FireEventRequestData& obj); + +typedef struct _FireEventRequest__isset { + _FireEventRequest__isset() : dbName(false), tableName(false), partitionVals(false) {} + bool dbName :1; + bool tableName :1; + bool partitionVals :1; +} _FireEventRequest__isset; + +class FireEventRequest : public virtual ::apache::thrift::TBase { + public: + + FireEventRequest(const FireEventRequest&); + FireEventRequest& operator=(const FireEventRequest&); + FireEventRequest() : successful(0), dbName(), tableName() { + } + + virtual ~FireEventRequest() noexcept; + bool successful; + FireEventRequestData data; + std::string dbName; + std::string tableName; + std::vector partitionVals; + + _FireEventRequest__isset __isset; + + void __set_successful(const bool val); + + void __set_data(const FireEventRequestData& val); + + void __set_dbName(const std::string& val); + + void __set_tableName(const std::string& val); + + void __set_partitionVals(const std::vector & val); + + bool operator == (const FireEventRequest & rhs) const + { + if (!(successful == rhs.successful)) + return false; + if (!(data == rhs.data)) + return false; + if (__isset.dbName != rhs.__isset.dbName) + return false; + else if (__isset.dbName && !(dbName == rhs.dbName)) + return false; + if (__isset.tableName != rhs.__isset.tableName) + return false; + else if (__isset.tableName && !(tableName == rhs.tableName)) + return false; + if (__isset.partitionVals != rhs.__isset.partitionVals) + return false; + else if (__isset.partitionVals && !(partitionVals == rhs.partitionVals)) + return false; + return true; + } + bool operator != (const FireEventRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const FireEventRequest & ) 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(FireEventRequest &a, FireEventRequest &b); + +std::ostream& operator<<(std::ostream& out, const FireEventRequest& obj); + + +class FireEventResponse : public virtual ::apache::thrift::TBase { + public: + + FireEventResponse(const FireEventResponse&); + FireEventResponse& operator=(const FireEventResponse&); + FireEventResponse() { + } + + virtual ~FireEventResponse() noexcept; + + bool operator == (const FireEventResponse & /* rhs */) const + { + return true; + } + bool operator != (const FireEventResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const FireEventResponse & ) 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(FireEventResponse &a, FireEventResponse &b); + +std::ostream& operator<<(std::ostream& out, const FireEventResponse& obj); + +typedef struct _MetadataPpdResult__isset { + _MetadataPpdResult__isset() : metadata(false), includeBitset(false) {} + bool metadata :1; + bool includeBitset :1; +} _MetadataPpdResult__isset; + +class MetadataPpdResult : public virtual ::apache::thrift::TBase { + public: + + MetadataPpdResult(const MetadataPpdResult&); + MetadataPpdResult& operator=(const MetadataPpdResult&); + MetadataPpdResult() : metadata(), includeBitset() { + } + + virtual ~MetadataPpdResult() noexcept; + std::string metadata; + std::string includeBitset; + + _MetadataPpdResult__isset __isset; + + void __set_metadata(const std::string& val); + + void __set_includeBitset(const std::string& val); + + bool operator == (const MetadataPpdResult & rhs) const + { + if (__isset.metadata != rhs.__isset.metadata) + return false; + else if (__isset.metadata && !(metadata == rhs.metadata)) + return false; + if (__isset.includeBitset != rhs.__isset.includeBitset) + return false; + else if (__isset.includeBitset && !(includeBitset == rhs.includeBitset)) + return false; + return true; + } + bool operator != (const MetadataPpdResult &rhs) const { + return !(*this == rhs); + } + + bool operator < (const MetadataPpdResult & ) 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(MetadataPpdResult &a, MetadataPpdResult &b); + +std::ostream& operator<<(std::ostream& out, const MetadataPpdResult& obj); + + +class GetFileMetadataByExprResult : public virtual ::apache::thrift::TBase { + public: + + GetFileMetadataByExprResult(const GetFileMetadataByExprResult&); + GetFileMetadataByExprResult& operator=(const GetFileMetadataByExprResult&); + GetFileMetadataByExprResult() : isSupported(0) { + } + + virtual ~GetFileMetadataByExprResult() noexcept; + std::map metadata; + bool isSupported; + + void __set_metadata(const std::map & val); + + void __set_isSupported(const bool val); + + bool operator == (const GetFileMetadataByExprResult & rhs) const + { + if (!(metadata == rhs.metadata)) + return false; + if (!(isSupported == rhs.isSupported)) + return false; + return true; + } + bool operator != (const GetFileMetadataByExprResult &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GetFileMetadataByExprResult & ) 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(GetFileMetadataByExprResult &a, GetFileMetadataByExprResult &b); + +std::ostream& operator<<(std::ostream& out, const GetFileMetadataByExprResult& obj); + +typedef struct _GetFileMetadataByExprRequest__isset { + _GetFileMetadataByExprRequest__isset() : doGetFooters(false), type(false) {} + bool doGetFooters :1; + bool type :1; +} _GetFileMetadataByExprRequest__isset; + +class GetFileMetadataByExprRequest : public virtual ::apache::thrift::TBase { + public: + + GetFileMetadataByExprRequest(const GetFileMetadataByExprRequest&); + GetFileMetadataByExprRequest& operator=(const GetFileMetadataByExprRequest&); + GetFileMetadataByExprRequest() : expr(), doGetFooters(0), type((FileMetadataExprType::type)0) { + } + + virtual ~GetFileMetadataByExprRequest() noexcept; + std::vector fileIds; + std::string expr; + bool doGetFooters; + FileMetadataExprType::type type; + + _GetFileMetadataByExprRequest__isset __isset; + + void __set_fileIds(const std::vector & val); + + void __set_expr(const std::string& val); + + void __set_doGetFooters(const bool val); + + void __set_type(const FileMetadataExprType::type val); + + bool operator == (const GetFileMetadataByExprRequest & rhs) const + { + if (!(fileIds == rhs.fileIds)) + return false; + if (!(expr == rhs.expr)) + return false; + if (__isset.doGetFooters != rhs.__isset.doGetFooters) + return false; + else if (__isset.doGetFooters && !(doGetFooters == rhs.doGetFooters)) + return false; + if (__isset.type != rhs.__isset.type) + return false; + else if (__isset.type && !(type == rhs.type)) + return false; + return true; + } + bool operator != (const GetFileMetadataByExprRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GetFileMetadataByExprRequest & ) 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(GetFileMetadataByExprRequest &a, GetFileMetadataByExprRequest &b); + +std::ostream& operator<<(std::ostream& out, const GetFileMetadataByExprRequest& obj); + + +class GetFileMetadataResult : public virtual ::apache::thrift::TBase { + public: + + GetFileMetadataResult(const GetFileMetadataResult&); + GetFileMetadataResult& operator=(const GetFileMetadataResult&); + GetFileMetadataResult() : isSupported(0) { + } + + virtual ~GetFileMetadataResult() noexcept; + std::map metadata; + bool isSupported; + + void __set_metadata(const std::map & val); + + void __set_isSupported(const bool val); + + bool operator == (const GetFileMetadataResult & rhs) const + { + if (!(metadata == rhs.metadata)) + return false; + if (!(isSupported == rhs.isSupported)) + return false; + return true; + } + bool operator != (const GetFileMetadataResult &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GetFileMetadataResult & ) 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(GetFileMetadataResult &a, GetFileMetadataResult &b); + +std::ostream& operator<<(std::ostream& out, const GetFileMetadataResult& obj); + + +class GetFileMetadataRequest : public virtual ::apache::thrift::TBase { + public: + + GetFileMetadataRequest(const GetFileMetadataRequest&); + GetFileMetadataRequest& operator=(const GetFileMetadataRequest&); + GetFileMetadataRequest() { + } + + virtual ~GetFileMetadataRequest() noexcept; + std::vector fileIds; + + void __set_fileIds(const std::vector & val); + + bool operator == (const GetFileMetadataRequest & rhs) const + { + if (!(fileIds == rhs.fileIds)) + return false; + return true; + } + bool operator != (const GetFileMetadataRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GetFileMetadataRequest & ) 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(GetFileMetadataRequest &a, GetFileMetadataRequest &b); + +std::ostream& operator<<(std::ostream& out, const GetFileMetadataRequest& obj); + + +class PutFileMetadataResult : public virtual ::apache::thrift::TBase { + public: + + PutFileMetadataResult(const PutFileMetadataResult&); + PutFileMetadataResult& operator=(const PutFileMetadataResult&); + PutFileMetadataResult() { + } + + virtual ~PutFileMetadataResult() noexcept; + + bool operator == (const PutFileMetadataResult & /* rhs */) const + { + return true; + } + bool operator != (const PutFileMetadataResult &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PutFileMetadataResult & ) 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(PutFileMetadataResult &a, PutFileMetadataResult &b); + +std::ostream& operator<<(std::ostream& out, const PutFileMetadataResult& obj); + +typedef struct _PutFileMetadataRequest__isset { + _PutFileMetadataRequest__isset() : type(false) {} + bool type :1; +} _PutFileMetadataRequest__isset; + +class PutFileMetadataRequest : public virtual ::apache::thrift::TBase { + public: + + PutFileMetadataRequest(const PutFileMetadataRequest&); + PutFileMetadataRequest& operator=(const PutFileMetadataRequest&); + PutFileMetadataRequest() : type((FileMetadataExprType::type)0) { + } + + virtual ~PutFileMetadataRequest() noexcept; + std::vector fileIds; + std::vector metadata; + FileMetadataExprType::type type; + + _PutFileMetadataRequest__isset __isset; + + void __set_fileIds(const std::vector & val); + + void __set_metadata(const std::vector & val); + + void __set_type(const FileMetadataExprType::type val); + + bool operator == (const PutFileMetadataRequest & rhs) const + { + if (!(fileIds == rhs.fileIds)) + return false; + if (!(metadata == rhs.metadata)) + return false; + if (__isset.type != rhs.__isset.type) + return false; + else if (__isset.type && !(type == rhs.type)) + return false; + return true; + } + bool operator != (const PutFileMetadataRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const PutFileMetadataRequest & ) 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(PutFileMetadataRequest &a, PutFileMetadataRequest &b); + +std::ostream& operator<<(std::ostream& out, const PutFileMetadataRequest& obj); + + +class ClearFileMetadataResult : public virtual ::apache::thrift::TBase { + public: + + ClearFileMetadataResult(const ClearFileMetadataResult&); + ClearFileMetadataResult& operator=(const ClearFileMetadataResult&); + ClearFileMetadataResult() { + } + + virtual ~ClearFileMetadataResult() noexcept; + + bool operator == (const ClearFileMetadataResult & /* rhs */) const + { + return true; + } + bool operator != (const ClearFileMetadataResult &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ClearFileMetadataResult & ) 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(ClearFileMetadataResult &a, ClearFileMetadataResult &b); + +std::ostream& operator<<(std::ostream& out, const ClearFileMetadataResult& obj); + + +class ClearFileMetadataRequest : public virtual ::apache::thrift::TBase { + public: + + ClearFileMetadataRequest(const ClearFileMetadataRequest&); + ClearFileMetadataRequest& operator=(const ClearFileMetadataRequest&); + ClearFileMetadataRequest() { + } + + virtual ~ClearFileMetadataRequest() noexcept; + std::vector fileIds; + + void __set_fileIds(const std::vector & val); + + bool operator == (const ClearFileMetadataRequest & rhs) const + { + if (!(fileIds == rhs.fileIds)) + return false; + return true; + } + bool operator != (const ClearFileMetadataRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ClearFileMetadataRequest & ) 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(ClearFileMetadataRequest &a, ClearFileMetadataRequest &b); + +std::ostream& operator<<(std::ostream& out, const ClearFileMetadataRequest& obj); + + +class CacheFileMetadataResult : public virtual ::apache::thrift::TBase { + public: + + CacheFileMetadataResult(const CacheFileMetadataResult&); + CacheFileMetadataResult& operator=(const CacheFileMetadataResult&); + CacheFileMetadataResult() : isSupported(0) { + } + + virtual ~CacheFileMetadataResult() noexcept; + bool isSupported; + + void __set_isSupported(const bool val); + + bool operator == (const CacheFileMetadataResult & rhs) const + { + if (!(isSupported == rhs.isSupported)) + return false; + return true; + } + bool operator != (const CacheFileMetadataResult &rhs) const { + return !(*this == rhs); + } + + bool operator < (const CacheFileMetadataResult & ) 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(CacheFileMetadataResult &a, CacheFileMetadataResult &b); + +std::ostream& operator<<(std::ostream& out, const CacheFileMetadataResult& obj); + +typedef struct _CacheFileMetadataRequest__isset { + _CacheFileMetadataRequest__isset() : partName(false), isAllParts(false) {} + bool partName :1; + bool isAllParts :1; +} _CacheFileMetadataRequest__isset; + +class CacheFileMetadataRequest : public virtual ::apache::thrift::TBase { + public: + + CacheFileMetadataRequest(const CacheFileMetadataRequest&); + CacheFileMetadataRequest& operator=(const CacheFileMetadataRequest&); + CacheFileMetadataRequest() : dbName(), tblName(), partName(), isAllParts(0) { + } + + virtual ~CacheFileMetadataRequest() noexcept; + std::string dbName; + std::string tblName; + std::string partName; + bool isAllParts; + + _CacheFileMetadataRequest__isset __isset; + + void __set_dbName(const std::string& val); + + void __set_tblName(const std::string& val); + + void __set_partName(const std::string& val); + + void __set_isAllParts(const bool val); + + bool operator == (const CacheFileMetadataRequest & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(tblName == rhs.tblName)) + return false; + if (__isset.partName != rhs.__isset.partName) + return false; + else if (__isset.partName && !(partName == rhs.partName)) + return false; + if (__isset.isAllParts != rhs.__isset.isAllParts) + return false; + else if (__isset.isAllParts && !(isAllParts == rhs.isAllParts)) + return false; + return true; + } + bool operator != (const CacheFileMetadataRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const CacheFileMetadataRequest & ) 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(CacheFileMetadataRequest &a, CacheFileMetadataRequest &b); + +std::ostream& operator<<(std::ostream& out, const CacheFileMetadataRequest& obj); + +typedef struct _GetAllFunctionsResponse__isset { + _GetAllFunctionsResponse__isset() : functions(false) {} + bool functions :1; +} _GetAllFunctionsResponse__isset; + +class GetAllFunctionsResponse : public virtual ::apache::thrift::TBase { + public: + + GetAllFunctionsResponse(const GetAllFunctionsResponse&); + GetAllFunctionsResponse& operator=(const GetAllFunctionsResponse&); + GetAllFunctionsResponse() { + } + + virtual ~GetAllFunctionsResponse() noexcept; + std::vector functions; + + _GetAllFunctionsResponse__isset __isset; + + void __set_functions(const std::vector & val); + + bool operator == (const GetAllFunctionsResponse & rhs) const + { + if (__isset.functions != rhs.__isset.functions) + return false; + else if (__isset.functions && !(functions == rhs.functions)) + return false; + return true; + } + bool operator != (const GetAllFunctionsResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GetAllFunctionsResponse & ) 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(GetAllFunctionsResponse &a, GetAllFunctionsResponse &b); + +std::ostream& operator<<(std::ostream& out, const GetAllFunctionsResponse& obj); + + +class ClientCapabilities : public virtual ::apache::thrift::TBase { + public: + + ClientCapabilities(const ClientCapabilities&); + ClientCapabilities& operator=(const ClientCapabilities&); + ClientCapabilities() { + } + + virtual ~ClientCapabilities() noexcept; + std::vector values; + + void __set_values(const std::vector & val); + + bool operator == (const ClientCapabilities & rhs) const + { + if (!(values == rhs.values)) + return false; + return true; + } + bool operator != (const ClientCapabilities &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ClientCapabilities & ) 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(ClientCapabilities &a, ClientCapabilities &b); + +std::ostream& operator<<(std::ostream& out, const ClientCapabilities& obj); + +typedef struct _GetTableRequest__isset { + _GetTableRequest__isset() : capabilities(false) {} + bool capabilities :1; +} _GetTableRequest__isset; + +class GetTableRequest : public virtual ::apache::thrift::TBase { + public: + + GetTableRequest(const GetTableRequest&); + GetTableRequest& operator=(const GetTableRequest&); + GetTableRequest() : dbName(), tblName() { + } + + virtual ~GetTableRequest() noexcept; + std::string dbName; + std::string tblName; + ClientCapabilities capabilities; + + _GetTableRequest__isset __isset; + + void __set_dbName(const std::string& val); + + void __set_tblName(const std::string& val); + + void __set_capabilities(const ClientCapabilities& val); + + bool operator == (const GetTableRequest & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(tblName == rhs.tblName)) + return false; + if (__isset.capabilities != rhs.__isset.capabilities) + return false; + else if (__isset.capabilities && !(capabilities == rhs.capabilities)) + return false; + return true; + } + bool operator != (const GetTableRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GetTableRequest & ) 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(GetTableRequest &a, GetTableRequest &b); + +std::ostream& operator<<(std::ostream& out, const GetTableRequest& obj); + + +class GetTableResult : public virtual ::apache::thrift::TBase { + public: + + GetTableResult(const GetTableResult&); + GetTableResult& operator=(const GetTableResult&); + GetTableResult() { + } + + virtual ~GetTableResult() noexcept; + Table table; + + void __set_table(const Table& val); + + bool operator == (const GetTableResult & rhs) const + { + if (!(table == rhs.table)) + return false; + return true; + } + bool operator != (const GetTableResult &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GetTableResult & ) 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(GetTableResult &a, GetTableResult &b); + +std::ostream& operator<<(std::ostream& out, const GetTableResult& obj); + +typedef struct _GetTablesRequest__isset { + _GetTablesRequest__isset() : tblNames(false), capabilities(false) {} + bool tblNames :1; + bool capabilities :1; +} _GetTablesRequest__isset; + +class GetTablesRequest : public virtual ::apache::thrift::TBase { + public: + + GetTablesRequest(const GetTablesRequest&); + GetTablesRequest& operator=(const GetTablesRequest&); + GetTablesRequest() : dbName() { + } + + virtual ~GetTablesRequest() noexcept; + std::string dbName; + std::vector tblNames; + ClientCapabilities capabilities; + + _GetTablesRequest__isset __isset; + + void __set_dbName(const std::string& val); + + void __set_tblNames(const std::vector & val); + + void __set_capabilities(const ClientCapabilities& val); + + bool operator == (const GetTablesRequest & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (__isset.tblNames != rhs.__isset.tblNames) + return false; + else if (__isset.tblNames && !(tblNames == rhs.tblNames)) + return false; + if (__isset.capabilities != rhs.__isset.capabilities) + return false; + else if (__isset.capabilities && !(capabilities == rhs.capabilities)) + return false; + return true; + } + bool operator != (const GetTablesRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GetTablesRequest & ) 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(GetTablesRequest &a, GetTablesRequest &b); + +std::ostream& operator<<(std::ostream& out, const GetTablesRequest& obj); + + +class GetTablesResult : public virtual ::apache::thrift::TBase { + public: + + GetTablesResult(const GetTablesResult&); + GetTablesResult& operator=(const GetTablesResult&); + GetTablesResult() { + } + + virtual ~GetTablesResult() noexcept; + std::vector
tables; + + void __set_tables(const std::vector
& val); + + bool operator == (const GetTablesResult & rhs) const + { + if (!(tables == rhs.tables)) + return false; + return true; + } + bool operator != (const GetTablesResult &rhs) const { + return !(*this == rhs); + } + + bool operator < (const GetTablesResult & ) 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(GetTablesResult &a, GetTablesResult &b); + +std::ostream& operator<<(std::ostream& out, const GetTablesResult& obj); + + +class CmRecycleRequest : public virtual ::apache::thrift::TBase { + public: + + CmRecycleRequest(const CmRecycleRequest&); + CmRecycleRequest& operator=(const CmRecycleRequest&); + CmRecycleRequest() : dataPath(), purge(0) { + } + + virtual ~CmRecycleRequest() noexcept; + std::string dataPath; + bool purge; + + void __set_dataPath(const std::string& val); + + void __set_purge(const bool val); + + bool operator == (const CmRecycleRequest & rhs) const + { + if (!(dataPath == rhs.dataPath)) + return false; + if (!(purge == rhs.purge)) + return false; + return true; + } + bool operator != (const CmRecycleRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const CmRecycleRequest & ) 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(CmRecycleRequest &a, CmRecycleRequest &b); + +std::ostream& operator<<(std::ostream& out, const CmRecycleRequest& obj); + + +class CmRecycleResponse : public virtual ::apache::thrift::TBase { + public: + + CmRecycleResponse(const CmRecycleResponse&); + CmRecycleResponse& operator=(const CmRecycleResponse&); + CmRecycleResponse() { + } + + virtual ~CmRecycleResponse() noexcept; + + bool operator == (const CmRecycleResponse & /* rhs */) const + { + return true; + } + bool operator != (const CmRecycleResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const CmRecycleResponse & ) 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(CmRecycleResponse &a, CmRecycleResponse &b); + +std::ostream& operator<<(std::ostream& out, const CmRecycleResponse& obj); + +typedef struct _TableMeta__isset { + _TableMeta__isset() : comments(false) {} + bool comments :1; +} _TableMeta__isset; + +class TableMeta : public virtual ::apache::thrift::TBase { + public: + + TableMeta(const TableMeta&); + TableMeta& operator=(const TableMeta&); + TableMeta() : dbName(), tableName(), tableType(), comments() { + } + + virtual ~TableMeta() noexcept; + std::string dbName; + std::string tableName; + std::string tableType; + std::string comments; + + _TableMeta__isset __isset; + + void __set_dbName(const std::string& val); + + void __set_tableName(const std::string& val); + + void __set_tableType(const std::string& val); + + void __set_comments(const std::string& val); + + bool operator == (const TableMeta & rhs) const + { + if (!(dbName == rhs.dbName)) + return false; + if (!(tableName == rhs.tableName)) + return false; + if (!(tableType == rhs.tableType)) + return false; + if (__isset.comments != rhs.__isset.comments) + return false; + else if (__isset.comments && !(comments == rhs.comments)) + return false; + return true; + } + bool operator != (const TableMeta &rhs) const { + return !(*this == rhs); + } + + bool operator < (const TableMeta & ) 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(TableMeta &a, TableMeta &b); + +std::ostream& operator<<(std::ostream& out, const TableMeta& obj); + +typedef struct _WMResourcePlan__isset { + _WMResourcePlan__isset() : status(false), queryParallelism(false), defaultPoolPath(false) {} + bool status :1; + bool queryParallelism :1; + bool defaultPoolPath :1; +} _WMResourcePlan__isset; + +class WMResourcePlan : public virtual ::apache::thrift::TBase { + public: + + WMResourcePlan(const WMResourcePlan&); + WMResourcePlan& operator=(const WMResourcePlan&); + WMResourcePlan() : name(), status((WMResourcePlanStatus::type)0), queryParallelism(0), defaultPoolPath() { + } + + virtual ~WMResourcePlan() noexcept; + std::string name; + WMResourcePlanStatus::type status; + int32_t queryParallelism; + std::string defaultPoolPath; + + _WMResourcePlan__isset __isset; + + void __set_name(const std::string& val); + + void __set_status(const WMResourcePlanStatus::type val); + + void __set_queryParallelism(const int32_t val); + + void __set_defaultPoolPath(const std::string& val); + + bool operator == (const WMResourcePlan & rhs) const + { + if (!(name == rhs.name)) + return false; + if (__isset.status != rhs.__isset.status) + return false; + else if (__isset.status && !(status == rhs.status)) + return false; + if (__isset.queryParallelism != rhs.__isset.queryParallelism) + return false; + else if (__isset.queryParallelism && !(queryParallelism == rhs.queryParallelism)) + return false; + if (__isset.defaultPoolPath != rhs.__isset.defaultPoolPath) + return false; + else if (__isset.defaultPoolPath && !(defaultPoolPath == rhs.defaultPoolPath)) + return false; + return true; + } + bool operator != (const WMResourcePlan &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMResourcePlan & ) 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(WMResourcePlan &a, WMResourcePlan &b); + +std::ostream& operator<<(std::ostream& out, const WMResourcePlan& obj); + +typedef struct _WMPool__isset { + _WMPool__isset() : allocFraction(false), queryParallelism(false), schedulingPolicy(false) {} + bool allocFraction :1; + bool queryParallelism :1; + bool schedulingPolicy :1; +} _WMPool__isset; + +class WMPool : public virtual ::apache::thrift::TBase { + public: + + WMPool(const WMPool&); + WMPool& operator=(const WMPool&); + WMPool() : resourcePlanName(), poolPath(), allocFraction(0), queryParallelism(0), schedulingPolicy() { + } + + virtual ~WMPool() noexcept; + std::string resourcePlanName; + std::string poolPath; + double allocFraction; + int32_t queryParallelism; + std::string schedulingPolicy; + + _WMPool__isset __isset; + + void __set_resourcePlanName(const std::string& val); + + void __set_poolPath(const std::string& val); + + void __set_allocFraction(const double val); + + void __set_queryParallelism(const int32_t val); + + void __set_schedulingPolicy(const std::string& val); + + bool operator == (const WMPool & rhs) const + { + if (!(resourcePlanName == rhs.resourcePlanName)) + return false; + if (!(poolPath == rhs.poolPath)) + return false; + if (__isset.allocFraction != rhs.__isset.allocFraction) + return false; + else if (__isset.allocFraction && !(allocFraction == rhs.allocFraction)) + return false; + if (__isset.queryParallelism != rhs.__isset.queryParallelism) + return false; + else if (__isset.queryParallelism && !(queryParallelism == rhs.queryParallelism)) + return false; + if (__isset.schedulingPolicy != rhs.__isset.schedulingPolicy) + return false; + else if (__isset.schedulingPolicy && !(schedulingPolicy == rhs.schedulingPolicy)) + return false; + return true; + } + bool operator != (const WMPool &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMPool & ) 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(WMPool &a, WMPool &b); + +std::ostream& operator<<(std::ostream& out, const WMPool& obj); + +typedef struct _WMTrigger__isset { + _WMTrigger__isset() : triggerExpression(false), actionExpression(false) {} + bool triggerExpression :1; + bool actionExpression :1; +} _WMTrigger__isset; + +class WMTrigger : public virtual ::apache::thrift::TBase { + public: + + WMTrigger(const WMTrigger&); + WMTrigger& operator=(const WMTrigger&); + WMTrigger() : resourcePlanName(), triggerName(), triggerExpression(), actionExpression() { + } + + virtual ~WMTrigger() noexcept; + std::string resourcePlanName; + std::string triggerName; + std::string triggerExpression; + std::string actionExpression; + + _WMTrigger__isset __isset; + + void __set_resourcePlanName(const std::string& val); + + void __set_triggerName(const std::string& val); + + void __set_triggerExpression(const std::string& val); + + void __set_actionExpression(const std::string& val); + + bool operator == (const WMTrigger & rhs) const + { + if (!(resourcePlanName == rhs.resourcePlanName)) + return false; + if (!(triggerName == rhs.triggerName)) + return false; + if (__isset.triggerExpression != rhs.__isset.triggerExpression) + return false; + else if (__isset.triggerExpression && !(triggerExpression == rhs.triggerExpression)) + return false; + if (__isset.actionExpression != rhs.__isset.actionExpression) + return false; + else if (__isset.actionExpression && !(actionExpression == rhs.actionExpression)) + return false; + return true; + } + bool operator != (const WMTrigger &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMTrigger & ) 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(WMTrigger &a, WMTrigger &b); + +std::ostream& operator<<(std::ostream& out, const WMTrigger& obj); + +typedef struct _WMMapping__isset { + _WMMapping__isset() : poolName(false), ordering(false) {} + bool poolName :1; + bool ordering :1; +} _WMMapping__isset; + +class WMMapping : public virtual ::apache::thrift::TBase { + public: + + WMMapping(const WMMapping&); + WMMapping& operator=(const WMMapping&); + WMMapping() : resourcePlanName(), entityType(), entityName(), poolName(), ordering(0) { + } + + virtual ~WMMapping() noexcept; + std::string resourcePlanName; + std::string entityType; + std::string entityName; + std::string poolName; + int32_t ordering; + + _WMMapping__isset __isset; + + void __set_resourcePlanName(const std::string& val); + + void __set_entityType(const std::string& val); + + void __set_entityName(const std::string& val); + + void __set_poolName(const std::string& val); + + void __set_ordering(const int32_t val); + + bool operator == (const WMMapping & rhs) const + { + if (!(resourcePlanName == rhs.resourcePlanName)) + return false; + if (!(entityType == rhs.entityType)) + return false; + if (!(entityName == rhs.entityName)) + return false; + if (__isset.poolName != rhs.__isset.poolName) + return false; + else if (__isset.poolName && !(poolName == rhs.poolName)) + return false; + if (__isset.ordering != rhs.__isset.ordering) + return false; + else if (__isset.ordering && !(ordering == rhs.ordering)) + return false; + return true; + } + bool operator != (const WMMapping &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMMapping & ) 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(WMMapping &a, WMMapping &b); + +std::ostream& operator<<(std::ostream& out, const WMMapping& obj); + + +class WMPoolTrigger : public virtual ::apache::thrift::TBase { + public: + + WMPoolTrigger(const WMPoolTrigger&); + WMPoolTrigger& operator=(const WMPoolTrigger&); + WMPoolTrigger() : pool(), trigger() { + } + + virtual ~WMPoolTrigger() noexcept; + std::string pool; + std::string trigger; + + void __set_pool(const std::string& val); + + void __set_trigger(const std::string& val); + + bool operator == (const WMPoolTrigger & rhs) const + { + if (!(pool == rhs.pool)) + return false; + if (!(trigger == rhs.trigger)) + return false; + return true; + } + bool operator != (const WMPoolTrigger &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMPoolTrigger & ) 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(WMPoolTrigger &a, WMPoolTrigger &b); + +std::ostream& operator<<(std::ostream& out, const WMPoolTrigger& obj); + +typedef struct _WMFullResourcePlan__isset { + _WMFullResourcePlan__isset() : mappings(false), triggers(false), poolTriggers(false) {} + bool mappings :1; + bool triggers :1; + bool poolTriggers :1; +} _WMFullResourcePlan__isset; + +class WMFullResourcePlan : public virtual ::apache::thrift::TBase { + public: + + WMFullResourcePlan(const WMFullResourcePlan&); + WMFullResourcePlan& operator=(const WMFullResourcePlan&); + WMFullResourcePlan() { + } + + virtual ~WMFullResourcePlan() noexcept; + WMResourcePlan plan; + std::vector pools; + std::vector mappings; + std::vector triggers; + std::vector poolTriggers; + + _WMFullResourcePlan__isset __isset; + + void __set_plan(const WMResourcePlan& val); + + void __set_pools(const std::vector & val); + + void __set_mappings(const std::vector & val); + + void __set_triggers(const std::vector & val); + + void __set_poolTriggers(const std::vector & val); + + bool operator == (const WMFullResourcePlan & rhs) const + { + if (!(plan == rhs.plan)) + return false; + if (!(pools == rhs.pools)) + return false; + if (__isset.mappings != rhs.__isset.mappings) + return false; + else if (__isset.mappings && !(mappings == rhs.mappings)) + return false; + if (__isset.triggers != rhs.__isset.triggers) + return false; + else if (__isset.triggers && !(triggers == rhs.triggers)) + return false; + if (__isset.poolTriggers != rhs.__isset.poolTriggers) + return false; + else if (__isset.poolTriggers && !(poolTriggers == rhs.poolTriggers)) + return false; + return true; + } + bool operator != (const WMFullResourcePlan &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMFullResourcePlan & ) 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(WMFullResourcePlan &a, WMFullResourcePlan &b); + +std::ostream& operator<<(std::ostream& out, const WMFullResourcePlan& obj); + +typedef struct _WMCreateResourcePlanRequest__isset { + _WMCreateResourcePlanRequest__isset() : resourcePlan(false) {} + bool resourcePlan :1; +} _WMCreateResourcePlanRequest__isset; + +class WMCreateResourcePlanRequest : public virtual ::apache::thrift::TBase { + public: + + WMCreateResourcePlanRequest(const WMCreateResourcePlanRequest&); + WMCreateResourcePlanRequest& operator=(const WMCreateResourcePlanRequest&); + WMCreateResourcePlanRequest() { + } + + virtual ~WMCreateResourcePlanRequest() noexcept; + WMResourcePlan resourcePlan; + + _WMCreateResourcePlanRequest__isset __isset; + + void __set_resourcePlan(const WMResourcePlan& val); + + bool operator == (const WMCreateResourcePlanRequest & rhs) const + { + if (__isset.resourcePlan != rhs.__isset.resourcePlan) + return false; + else if (__isset.resourcePlan && !(resourcePlan == rhs.resourcePlan)) + return false; + return true; + } + bool operator != (const WMCreateResourcePlanRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMCreateResourcePlanRequest & ) 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(WMCreateResourcePlanRequest &a, WMCreateResourcePlanRequest &b); + +std::ostream& operator<<(std::ostream& out, const WMCreateResourcePlanRequest& obj); + + +class WMCreateResourcePlanResponse : public virtual ::apache::thrift::TBase { + public: + + WMCreateResourcePlanResponse(const WMCreateResourcePlanResponse&); + WMCreateResourcePlanResponse& operator=(const WMCreateResourcePlanResponse&); + WMCreateResourcePlanResponse() { + } + + virtual ~WMCreateResourcePlanResponse() noexcept; + + bool operator == (const WMCreateResourcePlanResponse & /* rhs */) const + { + return true; + } + bool operator != (const WMCreateResourcePlanResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMCreateResourcePlanResponse & ) 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(WMCreateResourcePlanResponse &a, WMCreateResourcePlanResponse &b); + +std::ostream& operator<<(std::ostream& out, const WMCreateResourcePlanResponse& obj); + + +class WMGetActiveResourcePlanRequest : public virtual ::apache::thrift::TBase { + public: + + WMGetActiveResourcePlanRequest(const WMGetActiveResourcePlanRequest&); + WMGetActiveResourcePlanRequest& operator=(const WMGetActiveResourcePlanRequest&); + WMGetActiveResourcePlanRequest() { + } + + virtual ~WMGetActiveResourcePlanRequest() noexcept; + + bool operator == (const WMGetActiveResourcePlanRequest & /* rhs */) const + { + return true; + } + bool operator != (const WMGetActiveResourcePlanRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMGetActiveResourcePlanRequest & ) 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(WMGetActiveResourcePlanRequest &a, WMGetActiveResourcePlanRequest &b); + +std::ostream& operator<<(std::ostream& out, const WMGetActiveResourcePlanRequest& obj); + +typedef struct _WMGetActiveResourcePlanResponse__isset { + _WMGetActiveResourcePlanResponse__isset() : resourcePlan(false) {} + bool resourcePlan :1; +} _WMGetActiveResourcePlanResponse__isset; + +class WMGetActiveResourcePlanResponse : public virtual ::apache::thrift::TBase { + public: + + WMGetActiveResourcePlanResponse(const WMGetActiveResourcePlanResponse&); + WMGetActiveResourcePlanResponse& operator=(const WMGetActiveResourcePlanResponse&); + WMGetActiveResourcePlanResponse() { + } + + virtual ~WMGetActiveResourcePlanResponse() noexcept; + WMFullResourcePlan resourcePlan; + + _WMGetActiveResourcePlanResponse__isset __isset; + + void __set_resourcePlan(const WMFullResourcePlan& val); + + bool operator == (const WMGetActiveResourcePlanResponse & rhs) const + { + if (__isset.resourcePlan != rhs.__isset.resourcePlan) + return false; + else if (__isset.resourcePlan && !(resourcePlan == rhs.resourcePlan)) + return false; + return true; + } + bool operator != (const WMGetActiveResourcePlanResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMGetActiveResourcePlanResponse & ) 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(WMGetActiveResourcePlanResponse &a, WMGetActiveResourcePlanResponse &b); + +std::ostream& operator<<(std::ostream& out, const WMGetActiveResourcePlanResponse& obj); + +typedef struct _WMGetResourcePlanRequest__isset { + _WMGetResourcePlanRequest__isset() : resourcePlanName(false) {} + bool resourcePlanName :1; +} _WMGetResourcePlanRequest__isset; + +class WMGetResourcePlanRequest : public virtual ::apache::thrift::TBase { + public: + + WMGetResourcePlanRequest(const WMGetResourcePlanRequest&); + WMGetResourcePlanRequest& operator=(const WMGetResourcePlanRequest&); + WMGetResourcePlanRequest() : resourcePlanName() { + } + + virtual ~WMGetResourcePlanRequest() noexcept; + std::string resourcePlanName; + + _WMGetResourcePlanRequest__isset __isset; + + void __set_resourcePlanName(const std::string& val); + + bool operator == (const WMGetResourcePlanRequest & rhs) const + { + if (__isset.resourcePlanName != rhs.__isset.resourcePlanName) + return false; + else if (__isset.resourcePlanName && !(resourcePlanName == rhs.resourcePlanName)) + return false; + return true; + } + bool operator != (const WMGetResourcePlanRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMGetResourcePlanRequest & ) 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(WMGetResourcePlanRequest &a, WMGetResourcePlanRequest &b); + +std::ostream& operator<<(std::ostream& out, const WMGetResourcePlanRequest& obj); + +typedef struct _WMGetResourcePlanResponse__isset { + _WMGetResourcePlanResponse__isset() : resourcePlan(false) {} + bool resourcePlan :1; +} _WMGetResourcePlanResponse__isset; + +class WMGetResourcePlanResponse : public virtual ::apache::thrift::TBase { + public: + + WMGetResourcePlanResponse(const WMGetResourcePlanResponse&); + WMGetResourcePlanResponse& operator=(const WMGetResourcePlanResponse&); + WMGetResourcePlanResponse() { + } + + virtual ~WMGetResourcePlanResponse() noexcept; + WMResourcePlan resourcePlan; + + _WMGetResourcePlanResponse__isset __isset; + + void __set_resourcePlan(const WMResourcePlan& val); + + bool operator == (const WMGetResourcePlanResponse & rhs) const + { + if (__isset.resourcePlan != rhs.__isset.resourcePlan) + return false; + else if (__isset.resourcePlan && !(resourcePlan == rhs.resourcePlan)) + return false; + return true; + } + bool operator != (const WMGetResourcePlanResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMGetResourcePlanResponse & ) 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(WMGetResourcePlanResponse &a, WMGetResourcePlanResponse &b); + +std::ostream& operator<<(std::ostream& out, const WMGetResourcePlanResponse& obj); + + +class WMGetAllResourcePlanRequest : public virtual ::apache::thrift::TBase { + public: + + WMGetAllResourcePlanRequest(const WMGetAllResourcePlanRequest&); + WMGetAllResourcePlanRequest& operator=(const WMGetAllResourcePlanRequest&); + WMGetAllResourcePlanRequest() { + } + + virtual ~WMGetAllResourcePlanRequest() noexcept; + + bool operator == (const WMGetAllResourcePlanRequest & /* rhs */) const + { + return true; + } + bool operator != (const WMGetAllResourcePlanRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMGetAllResourcePlanRequest & ) 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(WMGetAllResourcePlanRequest &a, WMGetAllResourcePlanRequest &b); + +std::ostream& operator<<(std::ostream& out, const WMGetAllResourcePlanRequest& obj); + +typedef struct _WMGetAllResourcePlanResponse__isset { + _WMGetAllResourcePlanResponse__isset() : resourcePlans(false) {} + bool resourcePlans :1; +} _WMGetAllResourcePlanResponse__isset; + +class WMGetAllResourcePlanResponse : public virtual ::apache::thrift::TBase { + public: + + WMGetAllResourcePlanResponse(const WMGetAllResourcePlanResponse&); + WMGetAllResourcePlanResponse& operator=(const WMGetAllResourcePlanResponse&); + WMGetAllResourcePlanResponse() { + } + + virtual ~WMGetAllResourcePlanResponse() noexcept; + std::vector resourcePlans; + + _WMGetAllResourcePlanResponse__isset __isset; + + void __set_resourcePlans(const std::vector & val); + + bool operator == (const WMGetAllResourcePlanResponse & rhs) const + { + if (__isset.resourcePlans != rhs.__isset.resourcePlans) + return false; + else if (__isset.resourcePlans && !(resourcePlans == rhs.resourcePlans)) + return false; + return true; + } + bool operator != (const WMGetAllResourcePlanResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMGetAllResourcePlanResponse & ) 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(WMGetAllResourcePlanResponse &a, WMGetAllResourcePlanResponse &b); + +std::ostream& operator<<(std::ostream& out, const WMGetAllResourcePlanResponse& obj); + +typedef struct _WMAlterResourcePlanRequest__isset { + _WMAlterResourcePlanRequest__isset() : resourcePlanName(false), resourcePlan(false), isEnableAndActivate(false) {} + bool resourcePlanName :1; + bool resourcePlan :1; + bool isEnableAndActivate :1; +} _WMAlterResourcePlanRequest__isset; + +class WMAlterResourcePlanRequest : public virtual ::apache::thrift::TBase { + public: + + WMAlterResourcePlanRequest(const WMAlterResourcePlanRequest&); + WMAlterResourcePlanRequest& operator=(const WMAlterResourcePlanRequest&); + WMAlterResourcePlanRequest() : resourcePlanName(), isEnableAndActivate(0) { + } + + virtual ~WMAlterResourcePlanRequest() noexcept; + std::string resourcePlanName; + WMResourcePlan resourcePlan; + bool isEnableAndActivate; + + _WMAlterResourcePlanRequest__isset __isset; + + void __set_resourcePlanName(const std::string& val); + + void __set_resourcePlan(const WMResourcePlan& val); + + void __set_isEnableAndActivate(const bool val); + + bool operator == (const WMAlterResourcePlanRequest & rhs) const + { + if (__isset.resourcePlanName != rhs.__isset.resourcePlanName) + return false; + else if (__isset.resourcePlanName && !(resourcePlanName == rhs.resourcePlanName)) + return false; + if (__isset.resourcePlan != rhs.__isset.resourcePlan) + return false; + else if (__isset.resourcePlan && !(resourcePlan == rhs.resourcePlan)) + return false; + if (__isset.isEnableAndActivate != rhs.__isset.isEnableAndActivate) + return false; + else if (__isset.isEnableAndActivate && !(isEnableAndActivate == rhs.isEnableAndActivate)) + return false; + return true; + } + bool operator != (const WMAlterResourcePlanRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMAlterResourcePlanRequest & ) 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(WMAlterResourcePlanRequest &a, WMAlterResourcePlanRequest &b); + +std::ostream& operator<<(std::ostream& out, const WMAlterResourcePlanRequest& obj); + +typedef struct _WMAlterResourcePlanResponse__isset { + _WMAlterResourcePlanResponse__isset() : fullResourcePlan(false) {} + bool fullResourcePlan :1; +} _WMAlterResourcePlanResponse__isset; + +class WMAlterResourcePlanResponse : public virtual ::apache::thrift::TBase { + public: + + WMAlterResourcePlanResponse(const WMAlterResourcePlanResponse&); + WMAlterResourcePlanResponse& operator=(const WMAlterResourcePlanResponse&); + WMAlterResourcePlanResponse() { + } + + virtual ~WMAlterResourcePlanResponse() noexcept; + WMFullResourcePlan fullResourcePlan; + + _WMAlterResourcePlanResponse__isset __isset; + + void __set_fullResourcePlan(const WMFullResourcePlan& val); + + bool operator == (const WMAlterResourcePlanResponse & rhs) const + { + if (__isset.fullResourcePlan != rhs.__isset.fullResourcePlan) + return false; + else if (__isset.fullResourcePlan && !(fullResourcePlan == rhs.fullResourcePlan)) + return false; + return true; + } + bool operator != (const WMAlterResourcePlanResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMAlterResourcePlanResponse & ) 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(WMAlterResourcePlanResponse &a, WMAlterResourcePlanResponse &b); + +std::ostream& operator<<(std::ostream& out, const WMAlterResourcePlanResponse& obj); + +typedef struct _WMValidateResourcePlanRequest__isset { + _WMValidateResourcePlanRequest__isset() : resourcePlanName(false) {} + bool resourcePlanName :1; +} _WMValidateResourcePlanRequest__isset; + +class WMValidateResourcePlanRequest : public virtual ::apache::thrift::TBase { + public: + + WMValidateResourcePlanRequest(const WMValidateResourcePlanRequest&); + WMValidateResourcePlanRequest& operator=(const WMValidateResourcePlanRequest&); + WMValidateResourcePlanRequest() : resourcePlanName() { + } + + virtual ~WMValidateResourcePlanRequest() noexcept; + std::string resourcePlanName; + + _WMValidateResourcePlanRequest__isset __isset; + + void __set_resourcePlanName(const std::string& val); + + bool operator == (const WMValidateResourcePlanRequest & rhs) const + { + if (__isset.resourcePlanName != rhs.__isset.resourcePlanName) + return false; + else if (__isset.resourcePlanName && !(resourcePlanName == rhs.resourcePlanName)) + return false; + return true; + } + bool operator != (const WMValidateResourcePlanRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMValidateResourcePlanRequest & ) 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(WMValidateResourcePlanRequest &a, WMValidateResourcePlanRequest &b); + +std::ostream& operator<<(std::ostream& out, const WMValidateResourcePlanRequest& obj); + +typedef struct _WMValidateResourcePlanResponse__isset { + _WMValidateResourcePlanResponse__isset() : isValid(false) {} + bool isValid :1; +} _WMValidateResourcePlanResponse__isset; + +class WMValidateResourcePlanResponse : public virtual ::apache::thrift::TBase { + public: + + WMValidateResourcePlanResponse(const WMValidateResourcePlanResponse&); + WMValidateResourcePlanResponse& operator=(const WMValidateResourcePlanResponse&); + WMValidateResourcePlanResponse() : isValid(0) { + } + + virtual ~WMValidateResourcePlanResponse() noexcept; + bool isValid; + + _WMValidateResourcePlanResponse__isset __isset; + + void __set_isValid(const bool val); + + bool operator == (const WMValidateResourcePlanResponse & rhs) const + { + if (__isset.isValid != rhs.__isset.isValid) + return false; + else if (__isset.isValid && !(isValid == rhs.isValid)) + return false; + return true; + } + bool operator != (const WMValidateResourcePlanResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMValidateResourcePlanResponse & ) 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(WMValidateResourcePlanResponse &a, WMValidateResourcePlanResponse &b); + +std::ostream& operator<<(std::ostream& out, const WMValidateResourcePlanResponse& obj); + +typedef struct _WMDropResourcePlanRequest__isset { + _WMDropResourcePlanRequest__isset() : resourcePlanName(false) {} + bool resourcePlanName :1; +} _WMDropResourcePlanRequest__isset; + +class WMDropResourcePlanRequest : public virtual ::apache::thrift::TBase { + public: + + WMDropResourcePlanRequest(const WMDropResourcePlanRequest&); + WMDropResourcePlanRequest& operator=(const WMDropResourcePlanRequest&); + WMDropResourcePlanRequest() : resourcePlanName() { + } + + virtual ~WMDropResourcePlanRequest() noexcept; + std::string resourcePlanName; + + _WMDropResourcePlanRequest__isset __isset; + + void __set_resourcePlanName(const std::string& val); + + bool operator == (const WMDropResourcePlanRequest & rhs) const + { + if (__isset.resourcePlanName != rhs.__isset.resourcePlanName) + return false; + else if (__isset.resourcePlanName && !(resourcePlanName == rhs.resourcePlanName)) + return false; + return true; + } + bool operator != (const WMDropResourcePlanRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMDropResourcePlanRequest & ) 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(WMDropResourcePlanRequest &a, WMDropResourcePlanRequest &b); + +std::ostream& operator<<(std::ostream& out, const WMDropResourcePlanRequest& obj); + + +class WMDropResourcePlanResponse : public virtual ::apache::thrift::TBase { + public: + + WMDropResourcePlanResponse(const WMDropResourcePlanResponse&); + WMDropResourcePlanResponse& operator=(const WMDropResourcePlanResponse&); + WMDropResourcePlanResponse() { + } + + virtual ~WMDropResourcePlanResponse() noexcept; + + bool operator == (const WMDropResourcePlanResponse & /* rhs */) const + { + return true; + } + bool operator != (const WMDropResourcePlanResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMDropResourcePlanResponse & ) 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(WMDropResourcePlanResponse &a, WMDropResourcePlanResponse &b); + +std::ostream& operator<<(std::ostream& out, const WMDropResourcePlanResponse& obj); + +typedef struct _WMCreateTriggerRequest__isset { + _WMCreateTriggerRequest__isset() : trigger(false) {} + bool trigger :1; +} _WMCreateTriggerRequest__isset; + +class WMCreateTriggerRequest : public virtual ::apache::thrift::TBase { + public: + + WMCreateTriggerRequest(const WMCreateTriggerRequest&); + WMCreateTriggerRequest& operator=(const WMCreateTriggerRequest&); + WMCreateTriggerRequest() { + } + + virtual ~WMCreateTriggerRequest() noexcept; + WMTrigger trigger; + + _WMCreateTriggerRequest__isset __isset; + + void __set_trigger(const WMTrigger& val); + + bool operator == (const WMCreateTriggerRequest & rhs) const + { + if (__isset.trigger != rhs.__isset.trigger) + return false; + else if (__isset.trigger && !(trigger == rhs.trigger)) + return false; + return true; + } + bool operator != (const WMCreateTriggerRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMCreateTriggerRequest & ) 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(WMCreateTriggerRequest &a, WMCreateTriggerRequest &b); + +std::ostream& operator<<(std::ostream& out, const WMCreateTriggerRequest& obj); + + +class WMCreateTriggerResponse : public virtual ::apache::thrift::TBase { + public: + + WMCreateTriggerResponse(const WMCreateTriggerResponse&); + WMCreateTriggerResponse& operator=(const WMCreateTriggerResponse&); + WMCreateTriggerResponse() { + } + + virtual ~WMCreateTriggerResponse() noexcept; + + bool operator == (const WMCreateTriggerResponse & /* rhs */) const + { + return true; + } + bool operator != (const WMCreateTriggerResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMCreateTriggerResponse & ) 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(WMCreateTriggerResponse &a, WMCreateTriggerResponse &b); + +std::ostream& operator<<(std::ostream& out, const WMCreateTriggerResponse& obj); + +typedef struct _WMAlterTriggerRequest__isset { + _WMAlterTriggerRequest__isset() : trigger(false) {} + bool trigger :1; +} _WMAlterTriggerRequest__isset; + +class WMAlterTriggerRequest : public virtual ::apache::thrift::TBase { + public: + + WMAlterTriggerRequest(const WMAlterTriggerRequest&); + WMAlterTriggerRequest& operator=(const WMAlterTriggerRequest&); + WMAlterTriggerRequest() { + } + + virtual ~WMAlterTriggerRequest() noexcept; + WMTrigger trigger; + + _WMAlterTriggerRequest__isset __isset; + + void __set_trigger(const WMTrigger& val); + + bool operator == (const WMAlterTriggerRequest & rhs) const + { + if (__isset.trigger != rhs.__isset.trigger) + return false; + else if (__isset.trigger && !(trigger == rhs.trigger)) + return false; + return true; + } + bool operator != (const WMAlterTriggerRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMAlterTriggerRequest & ) 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(WMAlterTriggerRequest &a, WMAlterTriggerRequest &b); + +std::ostream& operator<<(std::ostream& out, const WMAlterTriggerRequest& obj); + + +class WMAlterTriggerResponse : public virtual ::apache::thrift::TBase { + public: + + WMAlterTriggerResponse(const WMAlterTriggerResponse&); + WMAlterTriggerResponse& operator=(const WMAlterTriggerResponse&); + WMAlterTriggerResponse() { + } + + virtual ~WMAlterTriggerResponse() noexcept; + + bool operator == (const WMAlterTriggerResponse & /* rhs */) const + { + return true; + } + bool operator != (const WMAlterTriggerResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMAlterTriggerResponse & ) 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(WMAlterTriggerResponse &a, WMAlterTriggerResponse &b); + +std::ostream& operator<<(std::ostream& out, const WMAlterTriggerResponse& obj); + +typedef struct _WMDropTriggerRequest__isset { + _WMDropTriggerRequest__isset() : resourcePlanName(false), triggerName(false) {} + bool resourcePlanName :1; + bool triggerName :1; +} _WMDropTriggerRequest__isset; + +class WMDropTriggerRequest : public virtual ::apache::thrift::TBase { + public: + + WMDropTriggerRequest(const WMDropTriggerRequest&); + WMDropTriggerRequest& operator=(const WMDropTriggerRequest&); + WMDropTriggerRequest() : resourcePlanName(), triggerName() { + } + + virtual ~WMDropTriggerRequest() noexcept; + std::string resourcePlanName; + std::string triggerName; + + _WMDropTriggerRequest__isset __isset; + + void __set_resourcePlanName(const std::string& val); + + void __set_triggerName(const std::string& val); + + bool operator == (const WMDropTriggerRequest & rhs) const + { + if (__isset.resourcePlanName != rhs.__isset.resourcePlanName) + return false; + else if (__isset.resourcePlanName && !(resourcePlanName == rhs.resourcePlanName)) + return false; + if (__isset.triggerName != rhs.__isset.triggerName) + return false; + else if (__isset.triggerName && !(triggerName == rhs.triggerName)) + return false; + return true; + } + bool operator != (const WMDropTriggerRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMDropTriggerRequest & ) 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(WMDropTriggerRequest &a, WMDropTriggerRequest &b); + +std::ostream& operator<<(std::ostream& out, const WMDropTriggerRequest& obj); + + +class WMDropTriggerResponse : public virtual ::apache::thrift::TBase { + public: + + WMDropTriggerResponse(const WMDropTriggerResponse&); + WMDropTriggerResponse& operator=(const WMDropTriggerResponse&); + WMDropTriggerResponse() { + } + + virtual ~WMDropTriggerResponse() noexcept; + + bool operator == (const WMDropTriggerResponse & /* rhs */) const + { + return true; + } + bool operator != (const WMDropTriggerResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMDropTriggerResponse & ) 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(WMDropTriggerResponse &a, WMDropTriggerResponse &b); + +std::ostream& operator<<(std::ostream& out, const WMDropTriggerResponse& obj); + +typedef struct _WMGetTriggersForResourePlanRequest__isset { + _WMGetTriggersForResourePlanRequest__isset() : resourcePlanName(false) {} + bool resourcePlanName :1; +} _WMGetTriggersForResourePlanRequest__isset; + +class WMGetTriggersForResourePlanRequest : public virtual ::apache::thrift::TBase { + public: + + WMGetTriggersForResourePlanRequest(const WMGetTriggersForResourePlanRequest&); + WMGetTriggersForResourePlanRequest& operator=(const WMGetTriggersForResourePlanRequest&); + WMGetTriggersForResourePlanRequest() : resourcePlanName() { + } + + virtual ~WMGetTriggersForResourePlanRequest() noexcept; + std::string resourcePlanName; + + _WMGetTriggersForResourePlanRequest__isset __isset; + + void __set_resourcePlanName(const std::string& val); + + bool operator == (const WMGetTriggersForResourePlanRequest & rhs) const + { + if (__isset.resourcePlanName != rhs.__isset.resourcePlanName) + return false; + else if (__isset.resourcePlanName && !(resourcePlanName == rhs.resourcePlanName)) + return false; + return true; + } + bool operator != (const WMGetTriggersForResourePlanRequest &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMGetTriggersForResourePlanRequest & ) 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(WMGetTriggersForResourePlanRequest &a, WMGetTriggersForResourePlanRequest &b); + +std::ostream& operator<<(std::ostream& out, const WMGetTriggersForResourePlanRequest& obj); + +typedef struct _WMGetTriggersForResourePlanResponse__isset { + _WMGetTriggersForResourePlanResponse__isset() : triggers(false) {} + bool triggers :1; +} _WMGetTriggersForResourePlanResponse__isset; + +class WMGetTriggersForResourePlanResponse : public virtual ::apache::thrift::TBase { + public: + + WMGetTriggersForResourePlanResponse(const WMGetTriggersForResourePlanResponse&); + WMGetTriggersForResourePlanResponse& operator=(const WMGetTriggersForResourePlanResponse&); + WMGetTriggersForResourePlanResponse() { + } + + virtual ~WMGetTriggersForResourePlanResponse() noexcept; + std::vector triggers; + + _WMGetTriggersForResourePlanResponse__isset __isset; + + void __set_triggers(const std::vector & val); + + bool operator == (const WMGetTriggersForResourePlanResponse & rhs) const + { + if (__isset.triggers != rhs.__isset.triggers) + return false; + else if (__isset.triggers && !(triggers == rhs.triggers)) + return false; + return true; + } + bool operator != (const WMGetTriggersForResourePlanResponse &rhs) const { + return !(*this == rhs); + } + + bool operator < (const WMGetTriggersForResourePlanResponse & ) 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(WMGetTriggersForResourePlanResponse &a, WMGetTriggersForResourePlanResponse &b); + +std::ostream& operator<<(std::ostream& out, const WMGetTriggersForResourePlanResponse& obj); + +typedef struct _MetaException__isset { + _MetaException__isset() : message(false) {} + bool message :1; +} _MetaException__isset; + +class MetaException : public ::apache::thrift::TException { + public: + + MetaException(const MetaException&); + MetaException& operator=(const MetaException&); + MetaException() : message() { + } + + virtual ~MetaException() noexcept; + std::string message; + + _MetaException__isset __isset; + + void __set_message(const std::string& val); + + bool operator == (const MetaException & rhs) const + { + if (!(message == rhs.message)) + return false; + return true; + } + bool operator != (const MetaException &rhs) const { + return !(*this == rhs); + } + + bool operator < (const MetaException & ) 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; + mutable std::string thriftTExceptionMessageHolder_; + const char* what() const noexcept; +}; + +void swap(MetaException &a, MetaException &b); + +std::ostream& operator<<(std::ostream& out, const MetaException& obj); + +typedef struct _UnknownTableException__isset { + _UnknownTableException__isset() : message(false) {} + bool message :1; +} _UnknownTableException__isset; + +class UnknownTableException : public ::apache::thrift::TException { + public: + + UnknownTableException(const UnknownTableException&); + UnknownTableException& operator=(const UnknownTableException&); + UnknownTableException() : message() { + } + + virtual ~UnknownTableException() noexcept; + std::string message; + + _UnknownTableException__isset __isset; + + void __set_message(const std::string& val); + + bool operator == (const UnknownTableException & rhs) const + { + if (!(message == rhs.message)) + return false; + return true; + } + bool operator != (const UnknownTableException &rhs) const { + return !(*this == rhs); + } + + bool operator < (const UnknownTableException & ) 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; + mutable std::string thriftTExceptionMessageHolder_; + const char* what() const noexcept; +}; + +void swap(UnknownTableException &a, UnknownTableException &b); + +std::ostream& operator<<(std::ostream& out, const UnknownTableException& obj); + +typedef struct _UnknownDBException__isset { + _UnknownDBException__isset() : message(false) {} + bool message :1; +} _UnknownDBException__isset; + +class UnknownDBException : public ::apache::thrift::TException { + public: + + UnknownDBException(const UnknownDBException&); + UnknownDBException& operator=(const UnknownDBException&); + UnknownDBException() : message() { + } + + virtual ~UnknownDBException() noexcept; + std::string message; + + _UnknownDBException__isset __isset; + + void __set_message(const std::string& val); + + bool operator == (const UnknownDBException & rhs) const + { + if (!(message == rhs.message)) + return false; + return true; + } + bool operator != (const UnknownDBException &rhs) const { + return !(*this == rhs); + } + + bool operator < (const UnknownDBException & ) 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; + mutable std::string thriftTExceptionMessageHolder_; + const char* what() const noexcept; +}; + +void swap(UnknownDBException &a, UnknownDBException &b); + +std::ostream& operator<<(std::ostream& out, const UnknownDBException& obj); + +typedef struct _AlreadyExistsException__isset { + _AlreadyExistsException__isset() : message(false) {} + bool message :1; +} _AlreadyExistsException__isset; + +class AlreadyExistsException : public ::apache::thrift::TException { + public: + + AlreadyExistsException(const AlreadyExistsException&); + AlreadyExistsException& operator=(const AlreadyExistsException&); + AlreadyExistsException() : message() { + } + + virtual ~AlreadyExistsException() noexcept; + std::string message; + + _AlreadyExistsException__isset __isset; + + void __set_message(const std::string& val); + + bool operator == (const AlreadyExistsException & rhs) const + { + if (!(message == rhs.message)) + return false; + return true; + } + bool operator != (const AlreadyExistsException &rhs) const { + return !(*this == rhs); + } + + bool operator < (const AlreadyExistsException & ) 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; + mutable std::string thriftTExceptionMessageHolder_; + const char* what() const noexcept; +}; + +void swap(AlreadyExistsException &a, AlreadyExistsException &b); + +std::ostream& operator<<(std::ostream& out, const AlreadyExistsException& obj); + +typedef struct _InvalidPartitionException__isset { + _InvalidPartitionException__isset() : message(false) {} + bool message :1; +} _InvalidPartitionException__isset; + +class InvalidPartitionException : public ::apache::thrift::TException { + public: + + InvalidPartitionException(const InvalidPartitionException&); + InvalidPartitionException& operator=(const InvalidPartitionException&); + InvalidPartitionException() : message() { + } + + virtual ~InvalidPartitionException() noexcept; + std::string message; + + _InvalidPartitionException__isset __isset; + + void __set_message(const std::string& val); + + bool operator == (const InvalidPartitionException & rhs) const + { + if (!(message == rhs.message)) + return false; + return true; + } + bool operator != (const InvalidPartitionException &rhs) const { + return !(*this == rhs); + } + + bool operator < (const InvalidPartitionException & ) 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; + mutable std::string thriftTExceptionMessageHolder_; + const char* what() const noexcept; +}; + +void swap(InvalidPartitionException &a, InvalidPartitionException &b); + +std::ostream& operator<<(std::ostream& out, const InvalidPartitionException& obj); + +typedef struct _UnknownPartitionException__isset { + _UnknownPartitionException__isset() : message(false) {} + bool message :1; +} _UnknownPartitionException__isset; + +class UnknownPartitionException : public ::apache::thrift::TException { + public: + + UnknownPartitionException(const UnknownPartitionException&); + UnknownPartitionException& operator=(const UnknownPartitionException&); + UnknownPartitionException() : message() { + } + + virtual ~UnknownPartitionException() noexcept; + std::string message; + + _UnknownPartitionException__isset __isset; + + void __set_message(const std::string& val); + + bool operator == (const UnknownPartitionException & rhs) const + { + if (!(message == rhs.message)) + return false; + return true; + } + bool operator != (const UnknownPartitionException &rhs) const { + return !(*this == rhs); + } + + bool operator < (const UnknownPartitionException & ) 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; + mutable std::string thriftTExceptionMessageHolder_; + const char* what() const noexcept; +}; + +void swap(UnknownPartitionException &a, UnknownPartitionException &b); + +std::ostream& operator<<(std::ostream& out, const UnknownPartitionException& obj); + +typedef struct _InvalidObjectException__isset { + _InvalidObjectException__isset() : message(false) {} + bool message :1; +} _InvalidObjectException__isset; + +class InvalidObjectException : public ::apache::thrift::TException { + public: + + InvalidObjectException(const InvalidObjectException&); + InvalidObjectException& operator=(const InvalidObjectException&); + InvalidObjectException() : message() { + } + + virtual ~InvalidObjectException() noexcept; + std::string message; + + _InvalidObjectException__isset __isset; + + void __set_message(const std::string& val); + + bool operator == (const InvalidObjectException & rhs) const + { + if (!(message == rhs.message)) + return false; + return true; + } + bool operator != (const InvalidObjectException &rhs) const { + return !(*this == rhs); + } + + bool operator < (const InvalidObjectException & ) 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; + mutable std::string thriftTExceptionMessageHolder_; + const char* what() const noexcept; +}; + +void swap(InvalidObjectException &a, InvalidObjectException &b); + +std::ostream& operator<<(std::ostream& out, const InvalidObjectException& obj); + +typedef struct _NoSuchObjectException__isset { + _NoSuchObjectException__isset() : message(false) {} + bool message :1; +} _NoSuchObjectException__isset; + +class NoSuchObjectException : public ::apache::thrift::TException { + public: + + NoSuchObjectException(const NoSuchObjectException&); + NoSuchObjectException& operator=(const NoSuchObjectException&); + NoSuchObjectException() : message() { + } + + virtual ~NoSuchObjectException() noexcept; + std::string message; + + _NoSuchObjectException__isset __isset; + + void __set_message(const std::string& val); + + bool operator == (const NoSuchObjectException & rhs) const + { + if (!(message == rhs.message)) + return false; + return true; + } + bool operator != (const NoSuchObjectException &rhs) const { + return !(*this == rhs); + } + + bool operator < (const NoSuchObjectException & ) 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; + mutable std::string thriftTExceptionMessageHolder_; + const char* what() const noexcept; +}; + +void swap(NoSuchObjectException &a, NoSuchObjectException &b); + +std::ostream& operator<<(std::ostream& out, const NoSuchObjectException& obj); + +typedef struct _IndexAlreadyExistsException__isset { + _IndexAlreadyExistsException__isset() : message(false) {} + bool message :1; +} _IndexAlreadyExistsException__isset; + +class IndexAlreadyExistsException : public ::apache::thrift::TException { + public: + + IndexAlreadyExistsException(const IndexAlreadyExistsException&); + IndexAlreadyExistsException& operator=(const IndexAlreadyExistsException&); + IndexAlreadyExistsException() : message() { + } + + virtual ~IndexAlreadyExistsException() noexcept; + std::string message; + + _IndexAlreadyExistsException__isset __isset; + + void __set_message(const std::string& val); + + bool operator == (const IndexAlreadyExistsException & rhs) const + { + if (!(message == rhs.message)) + return false; + return true; + } + bool operator != (const IndexAlreadyExistsException &rhs) const { + return !(*this == rhs); + } + + bool operator < (const IndexAlreadyExistsException & ) 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; + mutable std::string thriftTExceptionMessageHolder_; + const char* what() const noexcept; +}; + +void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b); + +std::ostream& operator<<(std::ostream& out, const IndexAlreadyExistsException& obj); + +typedef struct _InvalidOperationException__isset { + _InvalidOperationException__isset() : message(false) {} + bool message :1; +} _InvalidOperationException__isset; + +class InvalidOperationException : public ::apache::thrift::TException { + public: + + InvalidOperationException(const InvalidOperationException&); + InvalidOperationException& operator=(const InvalidOperationException&); + InvalidOperationException() : message() { + } + + virtual ~InvalidOperationException() noexcept; + std::string message; + + _InvalidOperationException__isset __isset; + + void __set_message(const std::string& val); + + bool operator == (const InvalidOperationException & rhs) const + { + if (!(message == rhs.message)) + return false; + return true; + } + bool operator != (const InvalidOperationException &rhs) const { + return !(*this == rhs); + } + + bool operator < (const InvalidOperationException & ) 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; + mutable std::string thriftTExceptionMessageHolder_; + const char* what() const noexcept; +}; + +void swap(InvalidOperationException &a, InvalidOperationException &b); + +std::ostream& operator<<(std::ostream& out, const InvalidOperationException& obj); + +typedef struct _ConfigValSecurityException__isset { + _ConfigValSecurityException__isset() : message(false) {} + bool message :1; +} _ConfigValSecurityException__isset; + +class ConfigValSecurityException : public ::apache::thrift::TException { + public: + + ConfigValSecurityException(const ConfigValSecurityException&); + ConfigValSecurityException& operator=(const ConfigValSecurityException&); + ConfigValSecurityException() : message() { + } + + virtual ~ConfigValSecurityException() noexcept; + std::string message; + + _ConfigValSecurityException__isset __isset; + + void __set_message(const std::string& val); + + bool operator == (const ConfigValSecurityException & rhs) const + { + if (!(message == rhs.message)) + return false; + return true; + } + bool operator != (const ConfigValSecurityException &rhs) const { + return !(*this == rhs); + } + + bool operator < (const ConfigValSecurityException & ) 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; + mutable std::string thriftTExceptionMessageHolder_; + const char* what() const noexcept; +}; + +void swap(ConfigValSecurityException &a, ConfigValSecurityException &b); + +std::ostream& operator<<(std::ostream& out, const ConfigValSecurityException& obj); + +typedef struct _InvalidInputException__isset { + _InvalidInputException__isset() : message(false) {} + bool message :1; +} _InvalidInputException__isset; + +class InvalidInputException : public ::apache::thrift::TException { + public: + + InvalidInputException(const InvalidInputException&); + InvalidInputException& operator=(const InvalidInputException&); + InvalidInputException() : message() { + } + + virtual ~InvalidInputException() noexcept; + std::string message; + + _InvalidInputException__isset __isset; + + void __set_message(const std::string& val); + + bool operator == (const InvalidInputException & rhs) const + { + if (!(message == rhs.message)) + return false; + return true; + } + bool operator != (const InvalidInputException &rhs) const { + return !(*this == rhs); + } + + bool operator < (const InvalidInputException & ) 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; + mutable std::string thriftTExceptionMessageHolder_; + const char* what() const noexcept; +}; + +void swap(InvalidInputException &a, InvalidInputException &b); + +std::ostream& operator<<(std::ostream& out, const InvalidInputException& obj); + +typedef struct _NoSuchTxnException__isset { + _NoSuchTxnException__isset() : message(false) {} + bool message :1; +} _NoSuchTxnException__isset; + +class NoSuchTxnException : public ::apache::thrift::TException { + public: + + NoSuchTxnException(const NoSuchTxnException&); + NoSuchTxnException& operator=(const NoSuchTxnException&); + NoSuchTxnException() : message() { + } + + virtual ~NoSuchTxnException() noexcept; + std::string message; + + _NoSuchTxnException__isset __isset; + + void __set_message(const std::string& val); + + bool operator == (const NoSuchTxnException & rhs) const + { + if (!(message == rhs.message)) + return false; + return true; + } + bool operator != (const NoSuchTxnException &rhs) const { + return !(*this == rhs); + } + + bool operator < (const NoSuchTxnException & ) 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; + mutable std::string thriftTExceptionMessageHolder_; + const char* what() const noexcept; +}; + +void swap(NoSuchTxnException &a, NoSuchTxnException &b); + +std::ostream& operator<<(std::ostream& out, const NoSuchTxnException& obj); + +typedef struct _TxnAbortedException__isset { + _TxnAbortedException__isset() : message(false) {} + bool message :1; +} _TxnAbortedException__isset; + +class TxnAbortedException : public ::apache::thrift::TException { + public: + + TxnAbortedException(const TxnAbortedException&); + TxnAbortedException& operator=(const TxnAbortedException&); + TxnAbortedException() : message() { + } + + virtual ~TxnAbortedException() noexcept; + std::string message; + + _TxnAbortedException__isset __isset; + + void __set_message(const std::string& val); + + bool operator == (const TxnAbortedException & rhs) const + { + if (!(message == rhs.message)) + return false; + return true; + } + bool operator != (const TxnAbortedException &rhs) const { + return !(*this == rhs); + } + + bool operator < (const TxnAbortedException & ) 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; + mutable std::string thriftTExceptionMessageHolder_; + const char* what() const noexcept; +}; + +void swap(TxnAbortedException &a, TxnAbortedException &b); + +std::ostream& operator<<(std::ostream& out, const TxnAbortedException& obj); + +typedef struct _TxnOpenException__isset { + _TxnOpenException__isset() : message(false) {} + bool message :1; +} _TxnOpenException__isset; + +class TxnOpenException : public ::apache::thrift::TException { + public: + + TxnOpenException(const TxnOpenException&); + TxnOpenException& operator=(const TxnOpenException&); + TxnOpenException() : message() { + } + + virtual ~TxnOpenException() noexcept; + std::string message; + + _TxnOpenException__isset __isset; + + void __set_message(const std::string& val); + + bool operator == (const TxnOpenException & rhs) const + { + if (!(message == rhs.message)) + return false; + return true; + } + bool operator != (const TxnOpenException &rhs) const { + return !(*this == rhs); + } + + bool operator < (const TxnOpenException & ) 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; + mutable std::string thriftTExceptionMessageHolder_; + const char* what() const noexcept; +}; + +void swap(TxnOpenException &a, TxnOpenException &b); + +std::ostream& operator<<(std::ostream& out, const TxnOpenException& obj); + +typedef struct _NoSuchLockException__isset { + _NoSuchLockException__isset() : message(false) {} + bool message :1; +} _NoSuchLockException__isset; + +class NoSuchLockException : public ::apache::thrift::TException { + public: + + NoSuchLockException(const NoSuchLockException&); + NoSuchLockException& operator=(const NoSuchLockException&); + NoSuchLockException() : message() { + } + + virtual ~NoSuchLockException() noexcept; + std::string message; + + _NoSuchLockException__isset __isset; + + void __set_message(const std::string& val); + + bool operator == (const NoSuchLockException & rhs) const + { + if (!(message == rhs.message)) + return false; + return true; + } + bool operator != (const NoSuchLockException &rhs) const { + return !(*this == rhs); + } + + bool operator < (const NoSuchLockException & ) 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; + mutable std::string thriftTExceptionMessageHolder_; + const char* what() const noexcept; +}; + +void swap(NoSuchLockException &a, NoSuchLockException &b); + +std::ostream& operator<<(std::ostream& out, const NoSuchLockException& obj); + +}}} // namespace + +#endif diff --git a/contrib/thrift-cmake/CMakeLists.txt b/contrib/thrift-cmake/CMakeLists.txt index 1a4e5bba898..14fb18cbc89 100644 --- a/contrib/thrift-cmake/CMakeLists.txt +++ b/contrib/thrift-cmake/CMakeLists.txt @@ -40,25 +40,23 @@ set(thriftcpp_threads_SOURCES "${LIBRARY_DIR}/src/thrift/concurrency/Mutex.cpp" ) add_library(${THRIFT_LIBRARY} ${thriftcpp_SOURCES} ${thriftcpp_threads_SOURCES}) -# set_target_properties(${THRIFT_LIBRARY} PROPERTIES CXX_STANDARD 14) # REMOVE after https://github.com/apache/thrift/pull/1641 -# target_include_directories(${THRIFT_LIBRARY} SYSTEM PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src") target_include_directories(${THRIFT_LIBRARY} SYSTEM PUBLIC "${THRIFT_INCLUDE_DIR}") target_link_libraries (${THRIFT_LIBRARY} PRIVATE boost::headers_only) -set(COMPILER_DIR "${ClickHouse_SOURCE_DIR}/contrib/thrift/compiler/cpp") -configure_file(${COMPILER_DIR}/src/thrift/version.h.in ${COMPILER_DIR}/src/thrift/version.h) -set(thrift_compiler_SOURCES - ${COMPILER_DIR}/src/thrift/main.cc - ${COMPILER_DIR}/src/thrift/audit/t_audit.cpp - ${COMPILER_DIR}/src/thrift/generate/t_cpp_generator.cc - ${COMPILER_DIR}/src/thrift/common.cc - ${COMPILER_DIR}/src/thrift/generate/t_generator.cc - ${COMPILER_DIR}/src/thrift/parse/t_typedef.cc - ${COMPILER_DIR}/src/thrift/parse/parse.cc - ${COMPILER_DIR}/src/thrift/version.h - ${COMPILER_DIR}/src/thrift/thrifty.cc - ${COMPILER_DIR}/src/thrift/thriftl.cc - ${COMPILER_DIR}/src/thrift/thrifty.hh -) -add_executable(${THRIFT_COMPILER} ${thrift_compiler_SOURCES}) -include_directories(${THRIFT_COMPILER} SYSTEM ${COMPILER_DIR}/src) +# set(COMPILER_DIR "${ClickHouse_SOURCE_DIR}/contrib/thrift/compiler/cpp") +# configure_file(${COMPILER_DIR}/src/thrift/version.h.in ${COMPILER_DIR}/src/thrift/version.h) +# set(thrift_compiler_SOURCES +# ${COMPILER_DIR}/src/thrift/main.cc +# ${COMPILER_DIR}/src/thrift/audit/t_audit.cpp +# ${COMPILER_DIR}/src/thrift/generate/t_cpp_generator.cc +# ${COMPILER_DIR}/src/thrift/common.cc +# ${COMPILER_DIR}/src/thrift/generate/t_generator.cc +# ${COMPILER_DIR}/src/thrift/parse/t_typedef.cc +# ${COMPILER_DIR}/src/thrift/parse/parse.cc +# ${COMPILER_DIR}/src/thrift/version.h +# ${COMPILER_DIR}/src/thrift/thrifty.cc +# ${COMPILER_DIR}/src/thrift/thriftl.cc +# ${COMPILER_DIR}/src/thrift/thrifty.hh +# ) +# add_executable(${THRIFT_COMPILER} ${thrift_compiler_SOURCES}) +# include_directories(${THRIFT_COMPILER} SYSTEM ${COMPILER_DIR}/src) From 5a45bc97649373e61caf59821d8d0bde3d30c7fe Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 9 Nov 2021 10:55:15 +0800 Subject: [PATCH 0028/1260] fix build error --- contrib/hive-metastore/CMakeLists.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/contrib/hive-metastore/CMakeLists.txt b/contrib/hive-metastore/CMakeLists.txt index 2b1129f0618..c4c59490b05 100644 --- a/contrib/hive-metastore/CMakeLists.txt +++ b/contrib/hive-metastore/CMakeLists.txt @@ -1,7 +1,7 @@ set (SRCS - ${CMAKE_CURRENT_BINARY_DIR}/hive_metastore_constants.cpp - ${CMAKE_CURRENT_BINARY_DIR}/hive_metastore_types.cpp - ${CMAKE_CURRENT_BINARY_DIR}/ThriftHiveMetastore.cpp + ${CMAKE_CURRENT_LIST_DIR}/hive_metastore_constants.cpp + ${CMAKE_CURRENT_LIST_DIR}/hive_metastore_types.cpp + ${CMAKE_CURRENT_LIST_DIR}/ThriftHiveMetastore.cpp ) # set(THRIFT_COMPILER_PATH ${ClickHouse_BINARY_DIR}/contrib/thrift-cmake/thrift-compiler) From 875075f34045f8d21a7353a09ca4072cd8383447 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 9 Nov 2021 11:54:00 +0800 Subject: [PATCH 0029/1260] fix build error --- contrib/arrow-cmake/CMakeLists.txt | 2 +- contrib/thrift-cmake/CMakeLists.txt | 5 ++++- contrib/thrift-cmake/build/cmake/config.h.in | 1 + 3 files changed, 6 insertions(+), 2 deletions(-) create mode 120000 contrib/thrift-cmake/build/cmake/config.h.in diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index e3ecfbd31dd..2fc77f65803 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -369,7 +369,7 @@ set(PARQUET_SRCS #list(TRANSFORM PARQUET_SRCS PREPEND "${LIBRARY_DIR}/") # cmake 3.12 add_library(${PARQUET_LIBRARY} ${PARQUET_SRCS}) target_include_directories(${PARQUET_LIBRARY} SYSTEM PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src" "${CMAKE_CURRENT_SOURCE_DIR}/cpp/src" PRIVATE ${OPENSSL_INCLUDE_DIR}) -include("${ClickHouse_SOURCE_DIR}/contrib/thrift/build/cmake/ConfigureChecks.cmake") # makes config.h +# include("${ClickHouse_SOURCE_DIR}/contrib/thrift/build/cmake/ConfigureChecks.cmake") # makes config.h target_link_libraries(${PARQUET_LIBRARY} PUBLIC ${ARROW_LIBRARY} PRIVATE ${THRIFT_LIBRARY} boost::headers_only boost::regex ${OPENSSL_LIBRARIES}) if (SANITIZE STREQUAL "undefined") diff --git a/contrib/thrift-cmake/CMakeLists.txt b/contrib/thrift-cmake/CMakeLists.txt index 14fb18cbc89..73eb0389ae0 100644 --- a/contrib/thrift-cmake/CMakeLists.txt +++ b/contrib/thrift-cmake/CMakeLists.txt @@ -39,8 +39,11 @@ set(thriftcpp_threads_SOURCES "${LIBRARY_DIR}/src/thrift/concurrency/Monitor.cpp" "${LIBRARY_DIR}/src/thrift/concurrency/Mutex.cpp" ) + +include("${ClickHouse_SOURCE_DIR}/contrib/thrift/build/cmake/ConfigureChecks.cmake") # makes config.h + add_library(${THRIFT_LIBRARY} ${thriftcpp_SOURCES} ${thriftcpp_threads_SOURCES}) -target_include_directories(${THRIFT_LIBRARY} SYSTEM PUBLIC "${THRIFT_INCLUDE_DIR}") +target_include_directories(${THRIFT_LIBRARY} SYSTEM PUBLIC "${THRIFT_INCLUDE_DIR}" ${CMAKE_CURRENT_BINARY_DIR}) target_link_libraries (${THRIFT_LIBRARY} PRIVATE boost::headers_only) # set(COMPILER_DIR "${ClickHouse_SOURCE_DIR}/contrib/thrift/compiler/cpp") diff --git a/contrib/thrift-cmake/build/cmake/config.h.in b/contrib/thrift-cmake/build/cmake/config.h.in new file mode 120000 index 00000000000..eb28c214748 --- /dev/null +++ b/contrib/thrift-cmake/build/cmake/config.h.in @@ -0,0 +1 @@ +../../../thrift/build/cmake/config.h.in \ No newline at end of file From b9b6af104fe59e28e215aca0dcabc233427d6eaf Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 9 Nov 2021 12:18:37 +0800 Subject: [PATCH 0030/1260] fix build error --- contrib/thrift-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/thrift-cmake/CMakeLists.txt b/contrib/thrift-cmake/CMakeLists.txt index 73eb0389ae0..26e6b4d085c 100644 --- a/contrib/thrift-cmake/CMakeLists.txt +++ b/contrib/thrift-cmake/CMakeLists.txt @@ -44,7 +44,7 @@ include("${ClickHouse_SOURCE_DIR}/contrib/thrift/build/cmake/ConfigureChecks.cma add_library(${THRIFT_LIBRARY} ${thriftcpp_SOURCES} ${thriftcpp_threads_SOURCES}) target_include_directories(${THRIFT_LIBRARY} SYSTEM PUBLIC "${THRIFT_INCLUDE_DIR}" ${CMAKE_CURRENT_BINARY_DIR}) -target_link_libraries (${THRIFT_LIBRARY} PRIVATE boost::headers_only) +target_link_libraries (${THRIFT_LIBRARY} PUBLIC boost::headers_only) # set(COMPILER_DIR "${ClickHouse_SOURCE_DIR}/contrib/thrift/compiler/cpp") # configure_file(${COMPILER_DIR}/src/thrift/version.h.in ${COMPILER_DIR}/src/thrift/version.h) From 5dbbe4b9c5bb8a4d183d456886c9e072648fec2f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 9 Nov 2021 13:14:53 +0800 Subject: [PATCH 0031/1260] fix build error --- docker/test/fasttest/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 925ac6f797c..671357bd8fc 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -186,6 +186,7 @@ function clone_submodules contrib/dragonbox contrib/fast_float contrib/NuRaft + contrib/thrift ) git submodule sync From b042e2b8f83d0a93efaaf688fda7374c4c93228e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 9 Nov 2021 14:38:46 +0800 Subject: [PATCH 0032/1260] fix pvs check --- src/IO/HadoopSnappyReadBuffer.h | 2 +- src/Storages/Hive/StorageHive.cpp | 4 +--- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/IO/HadoopSnappyReadBuffer.h b/src/IO/HadoopSnappyReadBuffer.h index c2427ab6bb8..b1b67399e56 100644 --- a/src/IO/HadoopSnappyReadBuffer.h +++ b/src/IO/HadoopSnappyReadBuffer.h @@ -44,7 +44,7 @@ public: total_uncompressed_length = 0; } - Status result; + Status result = Status::OK; private: inline bool checkBufferLength(int max) const; diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 8f23b036628..3ee58b405b5 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -265,8 +265,6 @@ StorageHive::StorageHive( switch (hdfs_file_format) { case FileFormat::TEXT: - format_name = "CSVWithNames"; - break; case FileFormat::LZO_TEXT: format_name = "CSVWithNames"; break; @@ -622,7 +620,7 @@ void registerStorageHive(StorageFactory & factory) args.columns, args.constraints, args.comment, - partition_by ? partition_by->ptr() : nullptr, + partition_by->ptr(), std::move(hive_settings), args.getContext()); }, From 7859abc58a531d43fff8a2bb0fea3098ff41ffac Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 9 Nov 2021 17:10:40 +0800 Subject: [PATCH 0033/1260] fix build error --- src/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index eb8d8ebd3ca..3e240ec7be8 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -468,7 +468,7 @@ if (USE_BROTLI) endif() if (USE_SNAPPY) - target_link_libraries (clickhouse_common_io PRIVATE ${SNAPPY_LIBRARY}) + target_link_libraries (clickhouse_common_io PUBLIC ${SNAPPY_LIBRARY}) target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${SNAPPY_INCLUDE_DIR}) endif() From 54ac5486ca1bbe554d999064f67bed7792a92b93 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 9 Nov 2021 17:14:53 +0800 Subject: [PATCH 0034/1260] fix build error --- src/Common/ErrorCodes.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 0f5f07fbfea..8fcd5a56df0 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -595,7 +595,8 @@ M(625, IO_SETUP_ERROR) \ M(626, CANNOT_SKIP_UNKNOWN_FIELD) \ M(627, SNAPPY_UNCOMPRESS_FAILED) \ - M(628, NO_HIVEMETASTORE) \ + M(628, SNAPPY_COMPRESS_FAILED) \ + M(629, NO_HIVEMETASTORE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ From c47bb97fcecfbd8d3ff3ea97c7933e86b2a25c1f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 9 Nov 2021 20:20:45 +0800 Subject: [PATCH 0035/1260] compile storage hive only USE_PARQUET AND USE_ORC enalbed --- src/CMakeLists.txt | 12 +++++++----- src/Storages/Hive/HiveCommon.cpp | 4 +--- src/Storages/Hive/HiveCommon.h | 4 +++- src/Storages/Hive/HiveFile.cpp | 10 ++++------ src/Storages/Hive/HiveFile.h | 6 ++++-- src/Storages/Hive/HiveSettings.cpp | 6 +++--- src/Storages/Hive/HiveSettings.h | 4 +++- src/Storages/Hive/StorageHive.cpp | 7 +++---- src/Storages/Hive/StorageHive.h | 3 ++- 9 files changed, 30 insertions(+), 26 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 3e240ec7be8..2fdffecca08 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -107,8 +107,11 @@ endif() if (USE_HDFS) add_headers_and_sources(dbms Storages/HDFS) - add_headers_and_sources(dbms Storages/Hive) add_headers_and_sources(dbms Disks/HDFS) + + if (USE_ORC AND USE_PARQUET) + add_headers_and_sources(dbms Storages/Hive) + endif() endif() if(USE_FILELOG) @@ -445,10 +448,9 @@ if (USE_HDFS) dbms_target_link_libraries(PRIVATE ${HDFS3_LIBRARY}) dbms_target_include_directories (SYSTEM BEFORE PUBLIC ${HDFS3_INCLUDE_DIR}) - # foreach (module ${all_modules}) - # target_include_directories (${module} SYSTEM BEFORE PUBLIC ${ClickHouse_BINARY_DIR}/contrib/hive-metastore) - # endforeach () - dbms_target_link_libraries(PRIVATE hivemetastore) + if (USE_ORC AND USE_PARQUET) + dbms_target_link_libraries(PRIVATE hivemetastore) + endif() endif() if (USE_AWS_S3) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index b94e98e6356..17b3b3eb04f 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -1,8 +1,6 @@ #include - -#if USE_HDFS - +#if USE_HDFS && USE_ORC && USE_PARQUET namespace DB { namespace ErrorCodes diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index 0a8fba5c07c..eeed9ae5903 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -1,8 +1,10 @@ #pragma once #include +#include "config_formats.h" + +#if USE_HDFS && USE_ORC && USE_PARQUET -#if USE_HDFS #include #include #include diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index b9ea7c85758..572a1db100f 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -1,11 +1,10 @@ -#include -#include -#include +#include -#if USE_HDFS -#include +#if USE_HDFS && USE_ORC && USE_PARQUET #include +#include +#include #include #include #include @@ -21,7 +20,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index ada46ff0038..9b4737487df 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -1,8 +1,9 @@ #pragma once #include -#include "Core/Block.h" +#include "config_formats.h" + +#if USE_HDFS && USE_ORC && USE_PARQUET -#if USE_HDFS #include #include @@ -11,6 +12,7 @@ #include #include +#include #include #include #include diff --git a/src/Storages/Hive/HiveSettings.cpp b/src/Storages/Hive/HiveSettings.cpp index efdfc1c954b..80a8b085057 100644 --- a/src/Storages/Hive/HiveSettings.cpp +++ b/src/Storages/Hive/HiveSettings.cpp @@ -1,11 +1,11 @@ -#include +#include + +#if USE_HDFS && USE_ORC && USE_PARQUET -#if USE_HDFS #include #include #include #include -#include namespace DB { diff --git a/src/Storages/Hive/HiveSettings.h b/src/Storages/Hive/HiveSettings.h index 60505ee6f11..2fa704fe116 100644 --- a/src/Storages/Hive/HiveSettings.h +++ b/src/Storages/Hive/HiveSettings.h @@ -1,8 +1,10 @@ #pragma once #include +#include "config_formats.h" + +#if USE_HDFS && USE_ORC && USE_PARQUET -#if USE_HDFS #include #include #include diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 3ee58b405b5..072c4c9e05a 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -1,7 +1,6 @@ -#include -#include "Interpreters/TreeRewriter.h" +#include -#if USE_HDFS +#if USE_HDFS && USE_ORC && USE_PARQUET #include #include @@ -19,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -35,7 +35,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index b94153fd4c3..2007325f408 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -1,8 +1,9 @@ #pragma once #include +#include "config_formats.h" -#if USE_HDFS +#if USE_HDFS && USE_ORC && USE_PARQUET #include #include From 68fc3e761d377ee81a51ca76cabd6e597db9309a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 10 Nov 2021 10:53:03 +0800 Subject: [PATCH 0036/1260] fix build error --- contrib/hive-metastore/CMakeLists.txt | 9 --------- src/CMakeLists.txt | 1 + src/Interpreters/Context.cpp | 8 ++++---- src/Interpreters/Context.h | 4 ++-- 4 files changed, 7 insertions(+), 15 deletions(-) diff --git a/contrib/hive-metastore/CMakeLists.txt b/contrib/hive-metastore/CMakeLists.txt index c4c59490b05..0792093cfb1 100644 --- a/contrib/hive-metastore/CMakeLists.txt +++ b/contrib/hive-metastore/CMakeLists.txt @@ -4,15 +4,6 @@ set (SRCS ${CMAKE_CURRENT_LIST_DIR}/ThriftHiveMetastore.cpp ) -# set(THRIFT_COMPILER_PATH ${ClickHouse_BINARY_DIR}/contrib/thrift-cmake/thrift-compiler) -# add_custom_command( -# OUTPUT ${SRCS} -# COMMAND ${THRIFT_COMPILER_PATH} -# ARGS -r --gen cpp -out ${CMAKE_CURRENT_BINARY_DIR} ${CMAKE_CURRENT_LIST_DIR}/hive_metastore.thrift -# COMMENT "Running cpp thrift compiler for hive_metastore.thrift" -# DEPENDS ${THRIFT_COMPILER} -# VERBATIM) - add_library(hivemetastore ${SRCS}) target_link_libraries(hivemetastore PUBLIC ${THRIFT_LIBRARY}) target_include_directories(hivemetastore SYSTEM PUBLIC ${CMAKE_CURRENT_LIST_DIR}) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 2fdffecca08..dc2cc9a2a66 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -450,6 +450,7 @@ if (USE_HDFS) if (USE_ORC AND USE_PARQUET) dbms_target_link_libraries(PRIVATE hivemetastore) + dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/hive-metastore) endif() endif() diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a580c6dc923..6baf9d00d0b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -88,7 +88,7 @@ #include #include -#if USE_HDFS +#if USE_HDFS && USE_ORC && USE_PARQUET #include #include #include @@ -163,7 +163,7 @@ struct ContextSharedPart mutable std::mutex storage_policies_mutex; /// Separate mutex for re-initialization of zookeeper session. This operation could take a long time and must not interfere with another operations. mutable std::mutex zookeeper_mutex; -#if USE_HDFS +#if USE_HDFS && USE_ORC && USE_PARQUET /// Separate mutex for re-initialization of hive metastore client. This operation could take a long time and must not interfere with another operations. mutable std::mutex hive_metastore_mutex; #endif @@ -179,7 +179,7 @@ struct ContextSharedPart mutable std::map auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients. ConfigurationPtr auxiliary_zookeepers_config; /// Stores auxiliary zookeepers configs -#if USE_HDFS +#if USE_HDFS && USE_ORC && USE_PARQUET mutable std::map hive_metastore_clients; /// Map for hive metastore clients #endif @@ -1824,7 +1824,7 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const return shared->zookeeper; } -#if USE_HDFS +#if USE_HDFS && USE_ORC && USE_PARQUET HMSClientPtr Context::getHMSClient(const String & name) const { using namespace apache::thrift; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 47f35b8acf0..c2d97bc8127 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -174,7 +174,7 @@ private: std::unique_ptr shared; }; -#if USE_HDFS +#if USE_HDFS && USE_ORC && USE_PARQUET class HMSClient; using HMSClientPtr = std::shared_ptr; #endif @@ -693,7 +693,7 @@ public: // Reload Zookeeper void reloadZooKeeperIfChanged(const ConfigurationPtr & config) const; -#if USE_HDFS +#if USE_HDFS && USE_ORC && USE_PARQUET HMSClientPtr getHMSClient(const String & name) const; #endif From 79cf9fb7cecd51a36128b7a44e4155647df50eca Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 10 Nov 2021 11:41:20 +0800 Subject: [PATCH 0037/1260] fix code style --- src/Storages/HDFS/HDFSCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/HDFS/HDFSCommon.cpp b/src/Storages/HDFS/HDFSCommon.cpp index f7fff38f3d2..197a9d2590e 100644 --- a/src/Storages/HDFS/HDFSCommon.cpp +++ b/src/Storages/HDFS/HDFSCommon.cpp @@ -213,7 +213,7 @@ String getNameNodeCluster(const String &hdfs_url) return hdfs_url.substr(pos1, pos2 - pos1); } - + void checkHDFSURL(const String & url) { if (!re2::RE2::FullMatch(url, HDFS_URL_REGEXP)) From dc77eafcf475abf9d6ab7974276e20581f4c1fca Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 10 Nov 2021 15:28:08 +0800 Subject: [PATCH 0038/1260] fix build error --- src/Interpreters/Context.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index c2d97bc8127..ae699f5d6e0 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -18,6 +18,7 @@ #include #include "config_core.h" +#include "config_formats.h" #include #include From 5170369fde1797186c93ea4779b866629ca04e75 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 10 Nov 2021 18:24:01 +0800 Subject: [PATCH 0039/1260] fix build error --- src/Storages/registerStorages.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 1dd8d8bb772..323991e0c5a 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -3,6 +3,7 @@ #include #include "config_core.h" +#include "config_formats.h" namespace DB { @@ -33,9 +34,13 @@ void registerStorageCOS(StorageFactory & factory); #if USE_HDFS void registerStorageHDFS(StorageFactory & factory); + +#if USE_ORC && USE_PARQUET void registerStorageHive(StorageFactory & factory); #endif +#endif + void registerStorageODBC(StorageFactory & factory); void registerStorageJDBC(StorageFactory & factory); @@ -105,9 +110,13 @@ void registerStorages() #if USE_HDFS registerStorageHDFS(factory); + + #if USE_ORC && USE_PARQUET registerStorageHive(factory); #endif + #endif + registerStorageODBC(factory); registerStorageJDBC(factory); From deef4d4dbe93887c38f227cb638e07a03a55e78e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 11 Nov 2021 11:49:54 +0800 Subject: [PATCH 0040/1260] add options read_bool_as_uint8 when parse csv --- .../Serializations/SerializationNumber.cpp | 4 +-- src/Formats/FormatSettings.h | 1 + src/IO/ReadHelpers.h | 28 +++++++++++++++---- src/Storages/Hive/StorageHive.cpp | 3 ++ 4 files changed, 29 insertions(+), 7 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index 7e8e99f76d0..482433052be 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -92,10 +92,10 @@ void SerializationNumber::deserializeTextJSON(IColumn & column, ReadBuffer & } template -void SerializationNumber::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +void SerializationNumber::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { FieldType x; - readCSV(x, istr); + readCSV(x, istr, settings.csv.read_bool_as_uint8); assert_cast &>(column).getData().push_back(x); } diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index d7e26ed239c..97c89b4b13f 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -79,6 +79,7 @@ struct FormatSettings bool crlf_end_of_line = false; bool input_format_enum_as_number = false; bool input_format_arrays_as_nested_csv = false; + bool read_bool_as_uint8 = false; Names input_field_names; String null_representation = "\\N"; } csv; diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 64e62ee009f..de9cd857dd7 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1037,8 +1037,9 @@ inline void readDoubleQuoted(LocalDateTime & x, ReadBuffer & buf) /// CSV, for numbers, dates: quotes are optional, no special escaping rules. +/// read_bool_as_uint8 enable read "t" and "f" as UInt8 value in Hive TEXT File. template -inline void readCSVSimple(T & x, ReadBuffer & buf) +inline void readCSVSimple(T & x, ReadBuffer & buf, bool read_bool_as_uint8 = false) { if (buf.eof()) throwReadAfterEOF(); @@ -1048,13 +1049,29 @@ inline void readCSVSimple(T & x, ReadBuffer & buf) if (maybe_quote == '\'' || maybe_quote == '\"') ++buf.position(); - readText(x, buf); + if constexpr (std::is_same_v) + { + if (read_bool_as_uint8 && (*buf.position() == 't' || *buf.position() == 'f')) + { + bool v = false; + readBoolTextWord(v, buf); + x = v ? 1 : 0; + } + else + { + readText(x, buf); + } + } + else + { + readText(x, buf); + } if (maybe_quote == '\'' || maybe_quote == '\"') assertChar(maybe_quote, buf); } -// Enable read "t" and "f" as UInt8 value in Hive TEXT File. +/* inline void readCSVSimple(UInt8 & x, ReadBuffer & buf) { if (buf.eof()) @@ -1069,7 +1086,7 @@ inline void readCSVSimple(UInt8 & x, ReadBuffer & buf) { bool tmp = false; readBoolTextWord(tmp, buf); - x = tmp; + x = tmp ? 1 : 0; } else { @@ -1079,10 +1096,11 @@ inline void readCSVSimple(UInt8 & x, ReadBuffer & buf) if (maybe_quote == '\'' || maybe_quote == '\"') assertChar(maybe_quote, buf); } +*/ template inline std::enable_if_t, void> -readCSV(T & x, ReadBuffer & buf) { readCSVSimple(x, buf); } +readCSV(T & x, ReadBuffer & buf, bool read_bool_as_uint8 = false) { readCSVSimple(x, buf, read_bool_as_uint8); } inline void readCSV(String & x, ReadBuffer & buf, const FormatSettings::CSV & settings) { readCSVString(x, buf, settings); } inline void readCSV(LocalDate & x, ReadBuffer & buf) { readCSVSimple(x, buf); } diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 072c4c9e05a..da7d6222eb3 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -115,8 +115,11 @@ public: { to_read_block.erase(name_type.name); } + + /// initialize format settings of CSV format_settings.csv.delimiter = '\x01'; format_settings.csv.input_field_names = text_input_field_names; + format_settings.csv.read_bool_as_uint8 = true; } String getName() const override { return "Hive"; } From a4cd9f516bbd857dd972ffea4f43f22f7a0752f3 Mon Sep 17 00:00:00 2001 From: lgbo Date: Thu, 11 Nov 2021 14:12:15 +0800 Subject: [PATCH 0041/1260] fixed on pr small code refinement base on the pr --- src/IO/RemoteReadBufferCache.cpp | 135 +++++++++++++++---------------- src/IO/RemoteReadBufferCache.h | 25 +++--- 2 files changed, 81 insertions(+), 79 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index f540f9da985..ac706d3db4a 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include #include @@ -10,6 +9,7 @@ #include #include "Common/Exception.h" #include +#include #include namespace DB @@ -24,10 +24,10 @@ namespace ErrorCodes std::shared_ptr RemoteCacheController::recover(const std::string & local_path_, std::function const & finish_callback) { - Poco::File dir_handle(local_path_); - Poco::File data_file(local_path_ + "/data.bin"); - Poco::File meta_file(local_path_ + "/meta.txt"); - if (!dir_handle.exists() || !data_file.exists() || !meta_file.exists()) + std::filesystem::path dir_handle(local_path_); + std::filesystem::path data_file(local_path_ + "/data.bin"); + std::filesystem::path meta_file(local_path_ + "meta.txt"); + if (!std::filesystem::exists(dir_handle) || !std::filesystem::exists(data_file) || !std::filesystem::exists(meta_file)) { LOG_ERROR(&Poco::Logger::get("RemoteCacheController"), "not exists directory" + local_path_); return nullptr; @@ -46,7 +46,7 @@ RemoteCacheController::recover(const std::string & local_path_, std::function(schema, cluster, remote_path, mod_ts, local_path_, nullptr, finish_callback); cntrl->download_finished = true; @@ -79,7 +79,7 @@ RemoteCacheController::RemoteCacheController( current_offset = 0; remote_readbuffer = readbuffer_; // setup local files - out_file = new Poco::FileOutputStream(local_path_ + "/data.bin", std::ios::out | std::ios::binary); + out_file = new std::ofstream(local_path_ + "/data.bin", std::ios::out | std::ios::binary); out_file->flush(); Poco::JSON::Object jobj; @@ -98,7 +98,7 @@ RemoteCacheController::RemoteCacheController( } } -int RemoteCacheController::waitMoreData(size_t start_offset_, size_t end_offset_) +RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offset_, size_t end_offset_) { std::unique_lock lock{mutex}; if (download_finished) @@ -114,7 +114,7 @@ int RemoteCacheController::waitMoreData(size_t start_offset_, size_t end_offset_ if (start_offset_ >= current_offset) { lock.unlock(); - return -1; + return RemoteReadBufferCacheError::END_OF_FILE; } } else // block until more data is ready @@ -122,14 +122,14 @@ int RemoteCacheController::waitMoreData(size_t start_offset_, size_t end_offset_ if (current_offset >= end_offset_) { lock.unlock(); - return 0; + return RemoteReadBufferCacheError::OK; } else more_data_signal.wait(lock, [this, end_offset_] { return this->download_finished || this->current_offset >= end_offset_; }); } LOG_TRACE(&Poco::Logger::get("RemoteCacheController"), "get more data to read"); lock.unlock(); - return 0; + return RemoteReadBufferCacheError::OK; } void RemoteCacheController::backgroupDownload(std::function const & finish_callback) @@ -207,20 +207,21 @@ void RemoteCacheController::close() { // delete the directory LOG_TRACE(&Poco::Logger::get("RemoteCacheController"), "release local resource " + remote_path + ", " + local_path); - Poco::File file(local_path); - file.remove(true); + + std::filesystem::remove(local_path); } -FILE * RemoteCacheController::allocFile(std::string * local_path_) +std::tuple RemoteCacheController::allocFile() { + std::string result_local_path; if (download_finished) - *local_path_ = local_path + "/data.bin"; + result_local_path = local_path + "/data.bin"; FILE * fs = fopen((local_path + "/data.bin").c_str(), "r"); if (fs == nullptr) - return fs; + return {fs, result_local_path}; std::lock_guard lock{mutex}; opened_file_streams.insert(fs); - return fs; + return {fs, result_local_path}; } void RemoteCacheController::deallocFile(FILE * fs) @@ -241,7 +242,7 @@ void RemoteCacheController::deallocFile(FILE * fs) LocalCachedFileReader::LocalCachedFileReader(RemoteCacheController * cntrl_, size_t size_) : offset(0), file_size(size_), fs(nullptr), controller(cntrl_) { - fs = controller->allocFile(&local_path); + std::tie(fs, local_path) = controller->allocFile(); if (fs == nullptr) throw Exception("alloc file failed.", ErrorCodes::BAD_GET); } @@ -253,7 +254,7 @@ LocalCachedFileReader::~LocalCachedFileReader() size_t LocalCachedFileReader::read(char * buf, size_t size) { auto wret = controller->waitMoreData(offset, offset + size); - if (wret < 0) + if (wret != RemoteReadBufferCacheError::OK) return 0; std::lock_guard lock(mutex); auto ret_size = fread(buf, 1, size, fs); @@ -282,8 +283,8 @@ size_t LocalCachedFileReader::size() LOG_TRACE(&Poco::Logger::get("LocalCachedFileReader"), "empty local_path"); return 0; } - Poco::File file_handle(local_path); - auto ret = file_handle.getSize(); + + auto ret = std::filesystem::file_size(local_path); file_size = ret; return ret; } @@ -320,7 +321,7 @@ std::unique_ptr RemoteReadBuffer::create( auto rrb = std::make_unique(buff_size); auto * raw_rbp = readbuffer.release(); std::shared_ptr srb(raw_rbp); - RemoteReadBufferCache::CreateReaderError error; + RemoteReadBufferCacheError error; int retry = 0; do { @@ -330,10 +331,10 @@ std::unique_ptr RemoteReadBuffer::create( std::tie(rrb->file_reader, error) = RemoteReadBufferCache::instance().createReader(schema_, cluster_, remote_path_, mod_ts_, file_size_, srb); retry++; - } while (error == RemoteReadBufferCache::CreateReaderError::FILE_INVALID && retry < 10); + } while (error == RemoteReadBufferCacheError::FILE_INVALID && retry < 10); if (rrb->file_reader == nullptr) { - LOG_ERROR(&Poco::Logger::get("RemoteReadBuffer"), "allocate local file failed for " + remote_path_ + "@" + std::to_string(error)); + LOG_ERROR(&Poco::Logger::get("RemoteReadBuffer"), "allocate local file failed for " + remote_path_ + "{}", error); rrb->original_readbuffer = srb; } return rrb; @@ -400,7 +401,7 @@ off_t RemoteReadBuffer::seek(off_t offset, int whence) off_t RemoteReadBuffer::getPosition() { - return file_reader->getOffset() - (working_buffer.end() - pos); + return file_reader->getOffset() - available(); } RemoteReadBufferCache::RemoteReadBufferCache() = default; @@ -421,40 +422,39 @@ void RemoteReadBufferCache::initOnce(const std::string & dir, size_t limit_size_ limit_size = limit_size_; // scan local disk dir and recover the cache metas - Poco::File root_dir(local_path_prefix); - if (!root_dir.exists()) + std::filesystem::path root_dir(local_path_prefix); + if (std::filesystem::exists(root_dir)) + { + LOG_INFO(log, "{} not exists. this cache will be disable", local_path_prefix); return; + } auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; // four level dir. ///`` - std::vector schema_dirs; - root_dir.list(schema_dirs); - for (auto & schema_file : schema_dirs) + for (auto const &schema_dir : std::filesystem::directory_iterator{root_dir}) { - std::vector cluster_dirs; - schema_file.list(cluster_dirs); - for (auto & cluster_file : cluster_dirs) + for (auto const &cluster_dir : std::filesystem::directory_iterator{schema_dir.path()}) { - std::vector file_dir1; - cluster_file.list(file_dir1); - for (auto & file1 : file_dir1) + for (auto const &first_hash_dir : std::filesystem::directory_iterator{cluster_dir.path()}) { - std::vector file_dir2; - file1.list(file_dir2); - for (auto & file2 : file_dir2) + for (auto const &second_hash_dir : std::filesystem::directory_iterator{first_hash_dir.path()}) { - if (caches.find(file2.path()) != caches.end()) + for (auto const &file : std::filesystem::directory_iterator{second_hash_dir.path()}) { - LOG_ERROR(log, "ridiculous!! "); - continue; + std::string path = file.path().string(); + if (caches.count(path)) + { + LOG_ERROR(log, "duplicated file:{}", path); + continue; + } + auto cache_cntrl = RemoteCacheController::recover(path, callback); + if (cache_cntrl == nullptr) + continue; + CacheCell cell; + cell.cache_controller = cache_cntrl; + cell.key_iterator = keys.insert(keys.end(), path); + caches[path] = cell; } - auto cache_cntrl = RemoteCacheController::recover(file2.path(), callback); - if (cache_cntrl == nullptr) - continue; - CacheCell cell; - cell.cache_controller = cache_cntrl; - cell.key_iterator = keys.insert(keys.end(), file2.path()); - caches[file2.path()] = cell; } } } @@ -479,7 +479,7 @@ RemoteReadBufferCache::calculateLocalPath(const std::string & schema_, const std return local_path; } -std::tuple, RemoteReadBufferCache::CreateReaderError> RemoteReadBufferCache::createReader( +std::tuple, RemoteReadBufferCacheError> RemoteReadBufferCache::createReader( const std::string & schema, const std::string & cluster, const std::string & remote_path, @@ -490,43 +490,43 @@ std::tuple, RemoteReadBufferCache::Create if (!hasInitialized()) { LOG_ERROR(log, "RemoteReadBufferCache not init"); - return {nullptr, CreateReaderError::NOT_INIT}; + return {nullptr, RemoteReadBufferCacheError::NOT_INIT}; } auto local_path = calculateLocalPath(schema, cluster, remote_path); std::lock_guard lock(mutex); - auto citer = caches.find(local_path); - if (citer != caches.end()) + auto cache_iter = caches.find(local_path); + if (cache_iter != caches.end()) { // if the file has been update on remote side, we need to redownload it - if (citer->second.cache_controller->getLastModTS() != mod_ts) + if (cache_iter->second.cache_controller->getLastModTS() != mod_ts) { LOG_TRACE(log, - "remote file has been updated. " + remote_path + ":" + std::to_string(citer->second.cache_controller->getLastModTS()) + "->" + "remote file has been updated. " + remote_path + ":" + std::to_string(cache_iter->second.cache_controller->getLastModTS()) + "->" + std::to_string(mod_ts)); - citer->second.cache_controller->markInvalid(); + cache_iter->second.cache_controller->markInvalid(); } else { // move the key to the list end - keys.splice(keys.end(), keys, citer->second.key_iterator); - return {std::make_shared(citer->second.cache_controller.get(), file_size), CreateReaderError::OK}; + keys.splice(keys.end(), keys, cache_iter->second.key_iterator); + return {std::make_shared(cache_iter->second.cache_controller.get(), file_size), RemoteReadBufferCacheError::OK}; } } auto clear_ret = clearLocalCache(); - citer = caches.find(local_path); - if (citer != caches.end()) + cache_iter = caches.find(local_path); + if (cache_iter != caches.end()) { - if (citer->second.cache_controller->isValid()) + if (cache_iter->second.cache_controller->isValid()) { // move the key to the list end, this case should not happen? - keys.splice(keys.end(), keys, citer->second.key_iterator); - return {std::make_shared(citer->second.cache_controller.get(), file_size), CreateReaderError::OK}; + keys.splice(keys.end(), keys, cache_iter->second.key_iterator); + return {std::make_shared(cache_iter->second.cache_controller.get(), file_size), RemoteReadBufferCacheError::OK}; } else { // maybe someone is holding this file - return {nullptr, CreateReaderError::FILE_INVALID}; + return {nullptr, RemoteReadBufferCacheError::FILE_INVALID}; } } @@ -534,11 +534,10 @@ std::tuple, RemoteReadBufferCache::Create if (!clear_ret) { LOG_ERROR(log, "local cache is full, return nullptr"); - return {nullptr, CreateReaderError::DISK_FULL}; + return {nullptr, RemoteReadBufferCacheError::DISK_FULL}; } - Poco::File file(local_path); - file.createDirectories(); + std::filesystem::create_directory(local_path); auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; auto cache_cntrl = std::make_shared(schema, cluster, remote_path, mod_ts, local_path, readbuffer, callback); @@ -547,7 +546,7 @@ std::tuple, RemoteReadBufferCache::Create cc.key_iterator = keys.insert(keys.end(), local_path); caches[local_path] = cc; - return {std::make_shared(cache_cntrl.get(), file_size), CreateReaderError::OK}; + return {std::make_shared(cache_cntrl.get(), file_size), RemoteReadBufferCacheError::OK}; } bool RemoteReadBufferCache::clearLocalCache() diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index f79f093150b..95d93dced7b 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -14,6 +14,16 @@ namespace DB { +enum class RemoteReadBufferCacheError :int8_t +{ + OK, + NOT_INIT = 10, + DISK_FULL = 11, + FILE_INVALID = 12, + + END_OF_FILE = 20, +}; + /** * */ @@ -38,7 +48,7 @@ public: * called by LocalCachedFileReader, must be used in pair * local_path will be empty if the file has not been downloaded */ - FILE * allocFile(std::string * local_path); + std::tuple allocFile(); void deallocFile(FILE * fs_); /** @@ -60,7 +70,7 @@ public: * - 0: has more data to read * - -1: has reach eof */ - int waitMoreData(size_t start_offset_, size_t end_offset_); + RemoteReadBufferCacheError waitMoreData(size_t start_offset_, size_t end_offset_); inline size_t size() const { return current_offset; } @@ -102,7 +112,7 @@ private: std::string cluster; std::shared_ptr remote_readbuffer; - Poco::FileOutputStream * out_file; + std::ofstream * out_file = nullptr; }; /** @@ -172,14 +182,7 @@ public: void initOnce(const std::string & dir, size_t limit_size); inline bool hasInitialized() const { return inited; } - enum CreateReaderError - { - OK = 0, - NOT_INIT = -1, - DISK_FULL = -2, - FILE_INVALID = -3 - }; - std::tuple, CreateReaderError> createReader( + std::tuple, RemoteReadBufferCacheError> createReader( const std::string & schema, const std::string & cluster, const std::string & remote_path, From c99ae74d50d761f347c09fede2da69eba7ac05c2 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 11 Nov 2021 15:33:58 +0800 Subject: [PATCH 0042/1260] fixed: condition for path exists is wrong --- src/IO/RemoteReadBufferCache.cpp | 41 ++++++++++++++------------------ src/IO/RemoteReadBufferCache.h | 7 ++---- 2 files changed, 20 insertions(+), 28 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index ac706d3db4a..eda0400b1a0 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -26,10 +26,10 @@ RemoteCacheController::recover(const std::string & local_path_, std::functionget("last_mod_ts").convert(); if (downloaded == "false") { - LOG_ERROR(&Poco::Logger::get("RemoteCacheController"), "not a downloaded file " + local_path_); + LOG_ERROR(&Poco::Logger::get("RemoteCacheController"), "not a downloaded file: " + local_path_); return nullptr; } auto size = std::filesystem::file_size(data_file); @@ -127,7 +127,6 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs else more_data_signal.wait(lock, [this, end_offset_] { return this->download_finished || this->current_offset >= end_offset_; }); } - LOG_TRACE(&Poco::Logger::get("RemoteCacheController"), "get more data to read"); lock.unlock(); return RemoteReadBufferCacheError::OK; } @@ -206,9 +205,8 @@ RemoteCacheController::~RemoteCacheController() void RemoteCacheController::close() { // delete the directory - LOG_TRACE(&Poco::Logger::get("RemoteCacheController"), "release local resource " + remote_path + ", " + local_path); - - std::filesystem::remove(local_path); + LOG_TRACE(&Poco::Logger::get("RemoteCacheController"), "release local resource: " + remote_path + ", " + local_path); + std::filesystem::remove_all(local_path); } std::tuple RemoteCacheController::allocFile() @@ -283,7 +281,7 @@ size_t LocalCachedFileReader::size() LOG_TRACE(&Poco::Logger::get("LocalCachedFileReader"), "empty local_path"); return 0; } - + auto ret = std::filesystem::file_size(local_path); file_size = ret; return ret; @@ -423,7 +421,7 @@ void RemoteReadBufferCache::initOnce(const std::string & dir, size_t limit_size_ // scan local disk dir and recover the cache metas std::filesystem::path root_dir(local_path_prefix); - if (std::filesystem::exists(root_dir)) + if (!std::filesystem::exists(root_dir)) { LOG_INFO(log, "{} not exists. this cache will be disable", local_path_prefix); return; @@ -439,22 +437,19 @@ void RemoteReadBufferCache::initOnce(const std::string & dir, size_t limit_size_ { for (auto const &second_hash_dir : std::filesystem::directory_iterator{first_hash_dir.path()}) { - for (auto const &file : std::filesystem::directory_iterator{second_hash_dir.path()}) + std::string path = second_hash_dir.path().string(); + if (caches.count(path)) { - std::string path = file.path().string(); - if (caches.count(path)) - { - LOG_ERROR(log, "duplicated file:{}", path); - continue; - } - auto cache_cntrl = RemoteCacheController::recover(path, callback); - if (cache_cntrl == nullptr) - continue; - CacheCell cell; - cell.cache_controller = cache_cntrl; - cell.key_iterator = keys.insert(keys.end(), path); - caches[path] = cell; + LOG_ERROR(log, "duplicated file:{}", path); + continue; } + auto cache_cntrl = RemoteCacheController::recover(path, callback); + if (cache_cntrl == nullptr) + continue; + CacheCell cell; + cell.cache_controller = cache_cntrl; + cell.key_iterator = keys.insert(keys.end(), path); + caches[path] = cell; } } } diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 95d93dced7b..eb8ed3199ec 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -23,7 +23,7 @@ enum class RemoteReadBufferCacheError :int8_t END_OF_FILE = 20, }; - + /** * */ @@ -66,9 +66,6 @@ public: * called in LocalCachedFileReader read(), the reading process would be blocked until * enough data be downloaded. * If the file has finished download, the process would be block - * returns - * - 0: has more data to read - * - -1: has reach eof */ RemoteReadBufferCacheError waitMoreData(size_t start_offset_, size_t end_offset_); @@ -195,7 +192,7 @@ private: std::atomic inited = false; std::mutex mutex; - size_t limit_size; + size_t limit_size = 0; std::atomic total_size; Poco::Logger * log = &Poco::Logger::get("RemoteReadBufferCache"); From 9ad441c56d19ab028a5eaae2c117fad47115d710 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 11 Nov 2021 19:50:08 +0800 Subject: [PATCH 0043/1260] rewrite codes base on comment --- programs/server/Server.cpp | 2 +- src/IO/RemoteReadBufferCache.cpp | 40 ++++++++++++++++++-------------- src/IO/RemoteReadBufferCache.h | 15 +++++++----- 3 files changed, 33 insertions(+), 24 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ca54268d82b..b81170038bc 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -507,7 +507,7 @@ if (ThreadFuzzer::instance().isEffective()) global_context->addWarningMessage("ThreadFuzzer is enabled. Application will run slowly and unstable."); if (config().has("local_cache_dir") && config().has("local_cache_quota")) - RemoteReadBufferCache::instance().initOnce(config().getString("local_cache_dir"), config().getUInt64("local_cache_quota")); + RemoteReadBufferCache::instance().initOnce(config().getString("local_cache_dir"), config().getUInt64("local_cache_quota"), config().getUInt64("local_cache_bytes_read_before_flush",DBMS_DEFAULT_BUFFER_SIZE)); #if defined(SANITIZER) global_context->addWarningMessage("Server was built with sanitizer. It will work slowly."); diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index eda0400b1a0..571f1c41976 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -40,7 +40,7 @@ RemoteCacheController::recover(const std::string & local_path_, std::functionget("schema").convert(); auto cluster = meta_jobj->get("cluster").convert(); auto downloaded = meta_jobj->get("downloaded").convert(); - auto mod_ts = meta_jobj->get("last_mod_ts").convert(); + auto mod_ts = meta_jobj->get("last_modification_timestamp").convert(); if (downloaded == "false") { LOG_ERROR(&Poco::Logger::get("RemoteCacheController"), "not a downloaded file: " + local_path_); @@ -48,7 +48,7 @@ RemoteCacheController::recover(const std::string & local_path_, std::function(schema, cluster, remote_path, mod_ts, local_path_, nullptr, finish_callback); + auto cntrl = std::make_shared(schema, cluster, remote_path, mod_ts, local_path_, 0, nullptr, finish_callback); cntrl->download_finished = true; cntrl->current_offset = size; meta_fs.close(); @@ -61,8 +61,9 @@ RemoteCacheController::RemoteCacheController( const std::string & schema_, const std::string & cluster_, const std::string & path_, - UInt64 mod_ts_, + UInt64 ts_, const std::string & local_path_, + size_t cache_bytes_before_flush_, std::shared_ptr readbuffer_, std::function const & finish_callback) { @@ -71,7 +72,8 @@ RemoteCacheController::RemoteCacheController( cluster = cluster_; local_path = local_path_; remote_path = path_; - last_mod_ts = mod_ts_; + last_modification_timestamp = ts_; + local_cache_bytes_read_before_flush = cache_bytes_before_flush_; valid = true; if (readbuffer_ != nullptr) { @@ -87,7 +89,7 @@ RemoteCacheController::RemoteCacheController( jobj.set("cluster", cluster_); jobj.set("remote_path", path_); jobj.set("downloaded", "false"); - jobj.set("last_mod_ts", mod_ts_); + jobj.set("last_modification_timestamp", ts_); std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM jobj.stringify(buf); Poco::FileOutputStream meta_file(local_path_ + "/meta.txt", std::ios::out); @@ -107,7 +109,6 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs { download_thread->wait(); LOG_TRACE(&Poco::Logger::get("RemoteCacheController"), "try to release down thread"); - delete download_thread; download_thread = nullptr; } // finish reading @@ -133,18 +134,19 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs void RemoteCacheController::backgroupDownload(std::function const & finish_callback) { - download_thread = new ThreadPool(1); + download_thread.reset(new ThreadPool(1)); auto task = [this, finish_callback]() { - size_t n = 0; + size_t unflush_bytes = 0; size_t total_bytes = 0; while (!remote_readbuffer->eof()) { - size_t bytes = remote_readbuffer->buffer().end() - remote_readbuffer->position(); + size_t bytes = remote_readbuffer->available(); out_file->write(remote_readbuffer->position(), bytes); remote_readbuffer->position() += bytes; total_bytes += bytes; - if (n++ % 10 == 0) + unflush_bytes += bytes; + if (unflush_bytes >= local_cache_bytes_read_before_flush) { std::unique_lock lock(mutex); current_offset += total_bytes; @@ -152,12 +154,13 @@ void RemoteCacheController::backgroupDownload(std::functionclose(); delete out_file; out_file = nullptr; @@ -171,18 +174,21 @@ void RemoteCacheController::backgroupDownload(std::functionscheduleOrThrow(task); } -void RemoteCacheController::flush() +void RemoteCacheController::flush(bool need_flush_meta_) { if (out_file != nullptr) { out_file->flush(); } + + if (!need_flush_meta_) + return; Poco::JSON::Object jobj; jobj.set("schema", schema); jobj.set("cluster", cluster); jobj.set("remote_path", remote_path); jobj.set("downloaded", download_finished ? "true" : "false"); - jobj.set("last_mod_ts", last_mod_ts); + jobj.set("last_modification_timestamp", last_modification_timestamp); std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM jobj.stringify(buf); @@ -198,7 +204,6 @@ RemoteCacheController::~RemoteCacheController() if (download_thread != nullptr) { download_thread->wait(); - delete download_thread; } } @@ -412,12 +417,13 @@ RemoteReadBufferCache & RemoteReadBufferCache::instance() return instance; } -void RemoteReadBufferCache::initOnce(const std::string & dir, size_t limit_size_) +void RemoteReadBufferCache::initOnce(const std::string & dir, size_t limit_size_, size_t bytes_read_before_flush_) { LOG_TRACE(log, "init local cache. path: {}, limit {}", dir, limit_size_); std::lock_guard lock(mutex); local_path_prefix = dir; limit_size = limit_size_; + local_cache_bytes_read_before_flush = bytes_read_before_flush_; // scan local disk dir and recover the cache metas std::filesystem::path root_dir(local_path_prefix); @@ -532,10 +538,10 @@ std::tuple, RemoteReadBufferCacheError> R return {nullptr, RemoteReadBufferCacheError::DISK_FULL}; } - std::filesystem::create_directory(local_path); + std::filesystem::create_directories(local_path); auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; - auto cache_cntrl = std::make_shared(schema, cluster, remote_path, mod_ts, local_path, readbuffer, callback); + auto cache_cntrl = std::make_shared(schema, cluster, remote_path, mod_ts, local_path, local_cache_bytes_read_before_flush, readbuffer, callback); CacheCell cc; cc.cache_controller = cache_cntrl; cc.key_iterator = keys.insert(keys.end(), local_path); diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index eb8ed3199ec..109953e7263 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -34,8 +34,9 @@ public: const std::string & schema_, const std::string & cluster_, const std::string & path_, - UInt64 mod_ts, + UInt64 ts, const std::string & local_path_, + size_t cache_bytes_before_flush_, std::shared_ptr readbuffer_, std::function const & finish_callback); ~RemoteCacheController(); // the local files will be deleted in descontructor @@ -74,7 +75,7 @@ public: inline const std::string & getLocalPath() { return local_path; } inline const std::string & getRemotePath() { return remote_path; } - inline UInt64 getLastModTS() const { return last_mod_ts; } + inline UInt64 getLastModTS() const { return last_modification_timestamp; } inline void markInvalid() { std::lock_guard lock(mutex); @@ -88,13 +89,13 @@ public: private: // flush file and meta info into disk - void flush(); + void flush(bool need_flush_meta_ = false); void backgroupDownload(std::function const & finish_callback); std::mutex mutex; std::condition_variable more_data_signal; - ThreadPool * download_thread; + std::shared_ptr download_thread; std::set opened_file_streams; @@ -102,12 +103,13 @@ private: bool download_finished; bool valid; size_t current_offset; - UInt64 last_mod_ts; + UInt64 last_modification_timestamp; std::string local_path; std::string remote_path; std::string schema; std::string cluster; + size_t local_cache_bytes_read_before_flush; std::shared_ptr remote_readbuffer; std::ofstream * out_file = nullptr; }; @@ -176,7 +178,7 @@ public: // global instance static RemoteReadBufferCache & instance(); - void initOnce(const std::string & dir, size_t limit_size); + void initOnce(const std::string & dir, size_t limit_size, size_t bytes_read_before_flush_); inline bool hasInitialized() const { return inited; } std::tuple, RemoteReadBufferCacheError> createReader( @@ -193,6 +195,7 @@ private: std::atomic inited = false; std::mutex mutex; size_t limit_size = 0; + size_t local_cache_bytes_read_before_flush = 0; std::atomic total_size; Poco::Logger * log = &Poco::Logger::get("RemoteReadBufferCache"); From eb9074488c98105b721ce671bba2b746407a9672 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 12 Nov 2021 10:11:17 +0800 Subject: [PATCH 0044/1260] fix code style --- src/IO/HadoopSnappyReadBuffer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/HadoopSnappyReadBuffer.cpp b/src/IO/HadoopSnappyReadBuffer.cpp index 1adb911d595..324df67e900 100644 --- a/src/IO/HadoopSnappyReadBuffer.cpp +++ b/src/IO/HadoopSnappyReadBuffer.cpp @@ -48,7 +48,7 @@ inline uint32_t HadoopSnappyDecoder::readLength(const char * in) uint32_t b2 = *(reinterpret_cast(in + 1)); uint32_t b3 = *(reinterpret_cast(in + 2)); uint32_t b4 = *(reinterpret_cast(in + 3)); - uint32_t res = ((b1 << 24) + (b2 << 16) + (b3 << 8) + (b4 << 0)); + uint32_t res = ((b1 << 24) + (b2 << 16) + (b3 << 8) + b4); return res; } From 74b31ce6a96d53f71f1a2ac50ef2fff7d9dc2e4b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 12 Nov 2021 10:23:27 +0800 Subject: [PATCH 0045/1260] fix code style --- src/IO/RemoteReadBufferCache.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 571f1c41976..2fdbe4cd495 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -10,6 +10,7 @@ #include "Common/Exception.h" #include #include +#include #include namespace DB @@ -134,7 +135,7 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs void RemoteCacheController::backgroupDownload(std::function const & finish_callback) { - download_thread.reset(new ThreadPool(1)); + download_thread = std::make_shared(1); auto task = [this, finish_callback]() { size_t unflush_bytes = 0; From a66a23c2e7b3eeab886ec9aa24e0ada0b92baea1 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 12 Nov 2021 13:12:24 +0800 Subject: [PATCH 0046/1260] code modification based on the pr comment --- src/IO/RemoteReadBufferCache.cpp | 227 +++++++++++++++--------------- src/IO/RemoteReadBufferCache.h | 73 ++++++---- src/Storages/Hive/StorageHive.cpp | 6 +- 3 files changed, 158 insertions(+), 148 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 2fdbe4cd495..08f834b411b 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -1,15 +1,14 @@ #include +#include +#include #include +#include #include #include #include -#include -#include -#include #include #include "Common/Exception.h" #include -#include #include #include @@ -23,35 +22,38 @@ namespace ErrorCodes } std::shared_ptr -RemoteCacheController::recover(const std::string & local_path_, std::function const & finish_callback) +RemoteCacheController::recover( + const std::filesystem::path & local_path_, + std::function const & finish_callback) { - std::filesystem::path dir_handle(local_path_); - std::filesystem::path data_file(local_path_ + "/data.bin"); - std::filesystem::path meta_file(local_path_ + "/meta.txt"); + auto & dir_handle = local_path_; + std::filesystem::path data_file = local_path_ / "data.bin"; + std::filesystem::path meta_file = local_path_ / "meta.txt"; if (!std::filesystem::exists(dir_handle) || !std::filesystem::exists(data_file) || !std::filesystem::exists(meta_file)) { - LOG_ERROR(&Poco::Logger::get("RemoteCacheController"), "not exists directory:" + local_path_); + LOG_ERROR(&Poco::Logger::get("RemoteCacheController"), "not exists directory:" + local_path_.string()); return nullptr; } - std::ifstream meta_fs(local_path_ + "/meta.txt"); + std::ifstream meta_fs(meta_file); Poco::JSON::Parser meta_parser; auto meta_jobj = meta_parser.parse(meta_fs).extract(); auto remote_path = meta_jobj->get("remote_path").convert(); auto schema = meta_jobj->get("schema").convert(); auto cluster = meta_jobj->get("cluster").convert(); auto downloaded = meta_jobj->get("downloaded").convert(); - auto mod_ts = meta_jobj->get("last_modification_timestamp").convert(); + auto modification_ts = meta_jobj->get("last_modification_timestamp").convert(); if (downloaded == "false") { - LOG_ERROR(&Poco::Logger::get("RemoteCacheController"), "not a downloaded file: " + local_path_); + LOG_ERROR(&Poco::Logger::get("RemoteCacheController"), "not a downloaded file: " + local_path_.string()); return nullptr; } - auto size = std::filesystem::file_size(data_file); + auto file_size = std::filesystem::file_size(data_file); - auto cntrl = std::make_shared(schema, cluster, remote_path, mod_ts, local_path_, 0, nullptr, finish_callback); + RemoteFileMeta remote_file_meta (schema, cluster, remote_path, modification_ts, file_size); + auto cntrl = std::make_shared(remote_file_meta, local_path_, 0, nullptr, finish_callback); cntrl->download_finished = true; - cntrl->current_offset = size; + cntrl->current_offset = file_size; meta_fs.close(); finish_callback(cntrl.get()); @@ -59,21 +61,18 @@ RemoteCacheController::recover(const std::string & local_path_, std::function readbuffer_, std::function const & finish_callback) { download_thread = nullptr; - schema = schema_; - cluster = cluster_; + schema = remote_file_meta.schema; + cluster = remote_file_meta.cluster; local_path = local_path_; - remote_path = path_; - last_modification_timestamp = ts_; + remote_path = remote_file_meta.path; + last_modification_timestamp = remote_file_meta.last_modification_timestamp; local_cache_bytes_read_before_flush = cache_bytes_before_flush_; valid = true; if (readbuffer_ != nullptr) @@ -82,18 +81,18 @@ RemoteCacheController::RemoteCacheController( current_offset = 0; remote_readbuffer = readbuffer_; // setup local files - out_file = new std::ofstream(local_path_ + "/data.bin", std::ios::out | std::ios::binary); + out_file = std::make_unique(local_path_ / "data.bin", std::ios::out | std::ios::binary); out_file->flush(); Poco::JSON::Object jobj; - jobj.set("schema", schema_); - jobj.set("cluster", cluster_); - jobj.set("remote_path", path_); + jobj.set("schema", schema); + jobj.set("cluster", cluster); + jobj.set("remote_path", remote_path); jobj.set("downloaded", "false"); - jobj.set("last_modification_timestamp", ts_); - std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + jobj.set("last_modification_timestamp", last_modification_timestamp); + std::stringstream buf;// STYLE_CHECK_ALLOW_STD_STRING_STREAM jobj.stringify(buf); - Poco::FileOutputStream meta_file(local_path_ + "/meta.txt", std::ios::out); + std::ofstream meta_file(local_path_ / "meta.txt", std::ios::out); meta_file.write(buf.str().c_str(), buf.str().size()); meta_file.close(); @@ -127,7 +126,7 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs return RemoteReadBufferCacheError::OK; } else - more_data_signal.wait(lock, [this, end_offset_] { return this->download_finished || this->current_offset >= end_offset_; }); + more_data_signal.wait(lock, [this, end_offset_] { return this->download_finished || this->current_offset >= end_offset_ ;}); } lock.unlock(); return RemoteReadBufferCacheError::OK; @@ -143,6 +142,7 @@ void RemoteCacheController::backgroupDownload(std::functioneof()) { size_t bytes = remote_readbuffer->available(); + out_file->write(remote_readbuffer->position(), bytes); remote_readbuffer->position() += bytes; total_bytes += bytes; @@ -163,21 +163,22 @@ void RemoteCacheController::backgroupDownload(std::functionclose(); - delete out_file; out_file = nullptr; remote_readbuffer = nullptr; lock.unlock(); more_data_signal.notify_all(); finish_callback(this); LOG_TRACE( - &Poco::Logger::get("RemoteCacheController"), "finish download.{} into {}. size:{} ", remote_path, local_path, current_offset); + &Poco::Logger::get("RemoteCacheController"), + "finish download.{} into {}. size:{} ", + remote_path, local_path.string(), current_offset); }; download_thread->scheduleOrThrow(task); } void RemoteCacheController::flush(bool need_flush_meta_) { - if (out_file != nullptr) + if (out_file) { out_file->flush(); } @@ -190,18 +191,16 @@ void RemoteCacheController::flush(bool need_flush_meta_) jobj.set("remote_path", remote_path); jobj.set("downloaded", download_finished ? "true" : "false"); jobj.set("last_modification_timestamp", last_modification_timestamp); - std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + std::stringstream buf;// STYLE_CHECK_ALLOW_STD_STRING_STREAM jobj.stringify(buf); - std::ofstream meta_file(local_path + "/meta.txt", std::ios::out); + std::ofstream meta_file(local_path / "meta.txt", std::ios::out); meta_file << buf.str(); meta_file.close(); } RemoteCacheController::~RemoteCacheController() { - delete out_file; - if (download_thread != nullptr) { download_thread->wait(); @@ -211,16 +210,16 @@ RemoteCacheController::~RemoteCacheController() void RemoteCacheController::close() { // delete the directory - LOG_TRACE(&Poco::Logger::get("RemoteCacheController"), "release local resource: " + remote_path + ", " + local_path); + LOG_TRACE(&Poco::Logger::get("RemoteCacheController"), "release local resource: " + remote_path + ", " + local_path.string()); std::filesystem::remove_all(local_path); } -std::tuple RemoteCacheController::allocFile() +std::tuple RemoteCacheController::allocFile() { - std::string result_local_path; + std::filesystem::path result_local_path; if (download_finished) - result_local_path = local_path + "/data.bin"; - FILE * fs = fopen((local_path + "/data.bin").c_str(), "r"); + result_local_path = local_path / "data.bin"; + FILE * fs = fopen((local_path / "data.bin").string().c_str(), "r"); if (fs == nullptr) return {fs, result_local_path}; std::lock_guard lock{mutex}; @@ -301,11 +300,7 @@ RemoteReadBuffer::RemoteReadBuffer(size_t buff_size) : BufferWithOwnMemory RemoteReadBuffer::create( - const std::string & schema_, - const std::string & cluster_, - const std::string & remote_path_, - UInt64 mod_ts_, - size_t file_size_, + const RemoteFileMeta &remote_file_meta_, std::unique_ptr readbuffer) { size_t buff_size = DBMS_DEFAULT_BUFFER_SIZE; @@ -322,6 +317,7 @@ std::unique_ptr RemoteReadBuffer::create( if (buff_size == 0) buff_size = DBMS_DEFAULT_BUFFER_SIZE; + auto & remote_path = remote_file_meta_.path; auto rrb = std::make_unique(buff_size); auto * raw_rbp = readbuffer.release(); std::shared_ptr srb(raw_rbp); @@ -333,12 +329,12 @@ std::unique_ptr RemoteReadBuffer::create( usleep(20 * retry); std::tie(rrb->file_reader, error) - = RemoteReadBufferCache::instance().createReader(schema_, cluster_, remote_path_, mod_ts_, file_size_, srb); + = RemoteReadBufferCache::instance().createReader(remote_file_meta_, srb); retry++; } while (error == RemoteReadBufferCacheError::FILE_INVALID && retry < 10); if (rrb->file_reader == nullptr) { - LOG_ERROR(&Poco::Logger::get("RemoteReadBuffer"), "allocate local file failed for " + remote_path_ + "{}", error); + LOG_ERROR(&Poco::Logger::get("RemoteReadBuffer"), "allocate local file failed for " + remote_path + "{}", error); rrb->original_readbuffer = srb; } return rrb; @@ -418,9 +414,43 @@ RemoteReadBufferCache & RemoteReadBufferCache::instance() return instance; } -void RemoteReadBufferCache::initOnce(const std::string & dir, size_t limit_size_, size_t bytes_read_before_flush_) +void RemoteReadBufferCache::recover_cached_files_meta( + const std::filesystem::path ¤t_path, + size_t current_depth, + size_t max_depth, + std::function const & finish_callback) { - LOG_TRACE(log, "init local cache. path: {}, limit {}", dir, limit_size_); + if (current_depth >= max_depth) + { + for (auto const & dir : std::filesystem::directory_iterator{current_path}) + { + std::string path = dir.path(); + if (caches.count(path)) + { + LOG_ERROR(log, "duplicated file:{}", path); + continue; + } + auto cache_cntrl = RemoteCacheController::recover(path, finish_callback); + if (!cache_cntrl) + continue; + auto &cell = caches[path]; + cell.cache_controller = cache_cntrl; + cell.key_iterator = keys.insert(keys.end(), path); + + } + return; + } + + for (auto const &dir : std::filesystem::directory_iterator{current_path}) + { + recover_cached_files_meta(dir.path(), current_depth + 1, max_depth, finish_callback); + } + +} + +void RemoteReadBufferCache::initOnce(const std::filesystem::path & dir, size_t limit_size_, size_t bytes_read_before_flush_) +{ + LOG_TRACE(log, "init local cache. path: {}, limit {}", dir.string(), limit_size_); std::lock_guard lock(mutex); local_path_prefix = dir; limit_size = limit_size_; @@ -435,58 +465,22 @@ void RemoteReadBufferCache::initOnce(const std::string & dir, size_t limit_size_ } auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; - // four level dir. ///`` - for (auto const &schema_dir : std::filesystem::directory_iterator{root_dir}) - { - for (auto const &cluster_dir : std::filesystem::directory_iterator{schema_dir.path()}) - { - for (auto const &first_hash_dir : std::filesystem::directory_iterator{cluster_dir.path()}) - { - for (auto const &second_hash_dir : std::filesystem::directory_iterator{first_hash_dir.path()}) - { - std::string path = second_hash_dir.path().string(); - if (caches.count(path)) - { - LOG_ERROR(log, "duplicated file:{}", path); - continue; - } - auto cache_cntrl = RemoteCacheController::recover(path, callback); - if (cache_cntrl == nullptr) - continue; - CacheCell cell; - cell.cache_controller = cache_cntrl; - cell.key_iterator = keys.insert(keys.end(), path); - caches[path] = cell; - } - } - } - } + // four level dir. /// + recover_cached_files_meta(root_dir, 1, 4, callback); inited = true; } -std::string -RemoteReadBufferCache::calculateLocalPath(const std::string & schema_, const std::string & cluster_, const std::string & remote_path_) +std::filesystem::path RemoteReadBufferCache::calculateLocalPath(const RemoteFileMeta & meta) { - std::string local_path = local_path_prefix + "/" + schema_ + "/" + cluster_; + auto path_prefix = std::filesystem::path(local_path_prefix) / meta.schema / meta.cluster; - Poco::MD5Engine md5; - Poco::DigestOutputStream outstr(md5); - outstr << remote_path_; - outstr.flush(); //to pass everything to the digest engine - const Poco::DigestEngine::Digest & digest = md5.digest(); - std::string md5string = Poco::DigestEngine::digestToHex(digest); - - local_path += "/" + md5string.substr(0, 3) + "/" + md5string; - - return local_path; + UInt128 hashcode = sipHash128(meta.path.c_str(), meta.path.size()); + std::string hashcode_str = getHexUIntLowercase(hashcode); + return path_prefix / hashcode_str.substr(0,3) / hashcode_str; } std::tuple, RemoteReadBufferCacheError> RemoteReadBufferCache::createReader( - const std::string & schema, - const std::string & cluster, - const std::string & remote_path, - UInt64 mod_ts, - size_t file_size, + const RemoteFileMeta &remote_file_meta, std::shared_ptr & readbuffer) { if (!hasInitialized()) @@ -494,17 +488,20 @@ std::tuple, RemoteReadBufferCacheError> R LOG_ERROR(log, "RemoteReadBufferCache not init"); return {nullptr, RemoteReadBufferCacheError::NOT_INIT}; } - auto local_path = calculateLocalPath(schema, cluster, remote_path); + auto remote_path = remote_file_meta.path; + auto & file_size = remote_file_meta.file_size; + auto & last_modification_timestamp = remote_file_meta.last_modification_timestamp; + auto local_path = calculateLocalPath(remote_file_meta); std::lock_guard lock(mutex); auto cache_iter = caches.find(local_path); if (cache_iter != caches.end()) { // if the file has been update on remote side, we need to redownload it - if (cache_iter->second.cache_controller->getLastModTS() != mod_ts) + if (cache_iter->second.cache_controller->getLastModificationTimestamp() != last_modification_timestamp) { LOG_TRACE(log, - "remote file has been updated. " + remote_path + ":" + std::to_string(cache_iter->second.cache_controller->getLastModTS()) + "->" - + std::to_string(mod_ts)); + "remote file has been updated. " + remote_path + ":" + std::to_string(cache_iter->second.cache_controller->getLastModificationTimestamp()) + "->" + + std::to_string(last_modification_timestamp)); cache_iter->second.cache_controller->markInvalid(); } else @@ -542,7 +539,7 @@ std::tuple, RemoteReadBufferCacheError> R std::filesystem::create_directories(local_path); auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; - auto cache_cntrl = std::make_shared(schema, cluster, remote_path, mod_ts, local_path, local_cache_bytes_read_before_flush, readbuffer, callback); + auto cache_cntrl = std::make_shared(remote_file_meta, local_path, local_cache_bytes_read_before_flush, readbuffer, callback); CacheCell cc; cc.cache_controller = cache_cntrl; cc.key_iterator = keys.insert(keys.end(), local_path); @@ -555,15 +552,15 @@ bool RemoteReadBufferCache::clearLocalCache() { for (auto it = keys.begin(); it != keys.end();) { - auto cit = caches.find(*it); - auto cntrl = cit->second.cache_controller; + auto cache_it = caches.find(*it); + auto cntrl = cache_it->second.cache_controller; if (!cntrl->isValid() && cntrl->closable()) { LOG_TRACE(log, "clear invalid cache: " + *it); - total_size = total_size > cit->second.cache_controller->size() ? total_size - cit->second.cache_controller->size() : 0; + total_size = total_size > cache_it->second.cache_controller->size() ? total_size - cache_it->second.cache_controller->size() : 0; cntrl->close(); it = keys.erase(it); - caches.erase(cit); + caches.erase(cache_it); } else it++; @@ -573,25 +570,25 @@ bool RemoteReadBufferCache::clearLocalCache() { if (total_size < limit_size) break; - auto cit = caches.find(*it); - if (cit == caches.end()) + auto cache_it = caches.find(*it); + if (cache_it == caches.end()) { throw Exception("file not found in cache?" + *it, ErrorCodes::LOGICAL_ERROR); } - if (cit->second.cache_controller->closable()) + if (cache_it->second.cache_controller->closable()) { - total_size = total_size > cit->second.cache_controller->size() ? total_size - cit->second.cache_controller->size() : 0; - cit->second.cache_controller->close(); - caches.erase(cit); + total_size = total_size > cache_it->second.cache_controller->size() ? total_size - cache_it->second.cache_controller->size() : 0; + cache_it->second.cache_controller->close(); + caches.erase(cache_it); it = keys.erase(it); - LOG_TRACE(log, "clear local file {} for {}. key size:{}. next{}", cit->second.cache_controller->getLocalPath(), - cit->second.cache_controller->getRemotePath(), keys.size(), *it); + LOG_TRACE(log, "clear local file {} for {}. key size:{}. next{}", cache_it->second.cache_controller->getLocalPath().string(), + cache_it->second.cache_controller->getRemotePath(), keys.size(), *it); } else break; } LOG_TRACE(log, "keys size:{}, total_size:{}, limit size:{}", keys.size(), total_size, limit_size); - return total_size < limit_size * 1.5; + return total_size < limit_size; } } diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 109953e7263..3d4a3880289 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include @@ -24,6 +24,28 @@ enum class RemoteReadBufferCacheError :int8_t END_OF_FILE = 20, }; +struct RemoteFileMeta +{ + RemoteFileMeta( + const std::string & schema_, + const std::string & cluster_, + const std::string & path_, + UInt64 last_modification_timestamp_, + size_t file_size_): + schema(schema_), + cluster(cluster_), + path(path_), + last_modification_timestamp(last_modification_timestamp_), + file_size(file_size_) + {} + + std::string schema; // Hive, S2 etc. + std::string cluster; + std::string path; + UInt64 last_modification_timestamp; + size_t file_size; +}; + /** * */ @@ -31,11 +53,8 @@ class RemoteCacheController { public: RemoteCacheController( - const std::string & schema_, - const std::string & cluster_, - const std::string & path_, - UInt64 ts, - const std::string & local_path_, + const RemoteFileMeta &meta, + const std::filesystem::path & local_path_, size_t cache_bytes_before_flush_, std::shared_ptr readbuffer_, std::function const & finish_callback); @@ -43,13 +62,14 @@ public: // recover from local disk static std::shared_ptr - recover(const std::string & local_path, std::function const & finish_callback); + recover(const std::filesystem::path & local_path, std::function const & finish_callback); /** - * called by LocalCachedFileReader, must be used in pair - * local_path will be empty if the file has not been downloaded + * Called by LocalCachedFileReader, must be used in pair + * The second value of the return tuple is the local_path to store file. + * It will be empty if the file has not been downloaded */ - std::tuple allocFile(); + std::tuple allocFile(); void deallocFile(FILE * fs_); /** @@ -72,10 +92,10 @@ public: inline size_t size() const { return current_offset; } - inline const std::string & getLocalPath() { return local_path; } + inline const std::filesystem::path & getLocalPath() { return local_path; } inline const std::string & getRemotePath() { return remote_path; } - inline UInt64 getLastModTS() const { return last_modification_timestamp; } + inline UInt64 getLastModificationTimestamp() const { return last_modification_timestamp; } inline void markInvalid() { std::lock_guard lock(mutex); @@ -104,14 +124,14 @@ private: bool valid; size_t current_offset; UInt64 last_modification_timestamp; - std::string local_path; + std::filesystem::path local_path; std::string remote_path; std::string schema; std::string cluster; size_t local_cache_bytes_read_before_flush; std::shared_ptr remote_readbuffer; - std::ofstream * out_file = nullptr; + std::unique_ptr out_file; }; /** @@ -135,7 +155,7 @@ private: size_t offset; size_t file_size; FILE * fs; - std::string local_path; + std::filesystem::path local_path; RemoteCacheController * controller; }; @@ -149,11 +169,7 @@ public: explicit RemoteReadBuffer(size_t buff_size); ~RemoteReadBuffer() override; static std::unique_ptr create( - const std::string & schema_, - const std::string & cluster_, - const std::string & remote_path_, - UInt64 mod_ts_, - size_t file_size_, + const RemoteFileMeta &remote_file_meta_, std::unique_ptr readbuffer); bool nextImpl() override; @@ -167,7 +183,6 @@ private: std::shared_ptr original_readbuffer; }; - class RemoteReadBufferCache { protected: @@ -178,15 +193,11 @@ public: // global instance static RemoteReadBufferCache & instance(); - void initOnce(const std::string & dir, size_t limit_size, size_t bytes_read_before_flush_); + void initOnce(const std::filesystem::path & dir, size_t limit_size, size_t bytes_read_before_flush_); inline bool hasInitialized() const { return inited; } std::tuple, RemoteReadBufferCacheError> createReader( - const std::string & schema, - const std::string & cluster, - const std::string & remote_path, - UInt64 mod_ts, - size_t file_size, + const RemoteFileMeta &remote_file_meta, std::shared_ptr & readbuffer); private: @@ -207,7 +218,13 @@ private: std::list keys; std::map caches; - std::string calculateLocalPath(const std::string & schema_, const std::string & cluster_, const std::string & remote_path_); + std::filesystem::path calculateLocalPath(const RemoteFileMeta &meta); + + void recover_cached_files_meta( + const std::filesystem::path & current_path_, + size_t current_depth, + size_t max_depth, + std::function const & finish_callback); bool clearLocalCache(); }; diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index da7d6222eb3..71a829b764d 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -155,11 +155,7 @@ public: } std::unique_ptr remote_read_buf = RemoteReadBuffer::create( - "Hive", - getNameNodeCluster(hdfs_namenode_url), - uri_with_path, - curr_file->getLastModTs(), - curr_file->getSize(), + {"Hive", getNameNodeCluster(hdfs_namenode_url), uri_with_path, curr_file->getLastModTs(), curr_file->getSize()}, std::move(raw_read_buf)); // std::unique_ptr remote_read_buf = std::move(raw_read_buf); if (curr_file->getFormat() == StorageHive::FileFormat::TEXT) From d0de8ce961d50cb1dd00a22597cb9c297758c33f Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 12 Nov 2021 13:56:06 +0800 Subject: [PATCH 0047/1260] fixed : code style --- src/IO/RemoteReadBufferCache.cpp | 3 +-- src/IO/RemoteReadBufferCache.h | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 08f834b411b..869a174bed4 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -126,7 +126,7 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs return RemoteReadBufferCacheError::OK; } else - more_data_signal.wait(lock, [this, end_offset_] { return this->download_finished || this->current_offset >= end_offset_ ;}); + more_data_signal.wait(lock, [this, end_offset_] { return this->download_finished || this->current_offset >= end_offset_; }); } lock.unlock(); return RemoteReadBufferCacheError::OK; @@ -436,7 +436,6 @@ void RemoteReadBufferCache::recover_cached_files_meta( auto &cell = caches[path]; cell.cache_controller = cache_cntrl; cell.key_iterator = keys.insert(keys.end(), path); - } return; } diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 3d4a3880289..6188662577f 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -221,7 +221,7 @@ private: std::filesystem::path calculateLocalPath(const RemoteFileMeta &meta); void recover_cached_files_meta( - const std::filesystem::path & current_path_, + const std::filesystem::path & current_path_, size_t current_depth, size_t max_depth, std::function const & finish_callback); From 11c8729b80bccac94130f75bb50243056e663274 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 12 Nov 2021 14:30:31 +0800 Subject: [PATCH 0048/1260] fixed : code style --- src/IO/RemoteReadBufferCache.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 6188662577f..7b02ff1ca60 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -66,7 +66,7 @@ public: /** * Called by LocalCachedFileReader, must be used in pair - * The second value of the return tuple is the local_path to store file. + * The second value of the return tuple is the local_path to store file. * It will be empty if the file has not been downloaded */ std::tuple allocFile(); From dcb65af181691bc3967531e28eabd93db001b12f Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 12 Nov 2021 16:43:35 +0800 Subject: [PATCH 0049/1260] fixed : remove compile warnings --- src/IO/RemoteReadBufferCache.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 869a174bed4..8910ac4a97a 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -26,7 +26,7 @@ RemoteCacheController::recover( const std::filesystem::path & local_path_, std::function const & finish_callback) { - auto & dir_handle = local_path_; + const auto & dir_handle = local_path_; std::filesystem::path data_file = local_path_ / "data.bin"; std::filesystem::path meta_file = local_path_ / "meta.txt"; if (!std::filesystem::exists(dir_handle) || !std::filesystem::exists(data_file) || !std::filesystem::exists(meta_file)) @@ -317,7 +317,7 @@ std::unique_ptr RemoteReadBuffer::create( if (buff_size == 0) buff_size = DBMS_DEFAULT_BUFFER_SIZE; - auto & remote_path = remote_file_meta_.path; + const auto & remote_path = remote_file_meta_.path; auto rrb = std::make_unique(buff_size); auto * raw_rbp = readbuffer.release(); std::shared_ptr srb(raw_rbp); @@ -488,8 +488,8 @@ std::tuple, RemoteReadBufferCacheError> R return {nullptr, RemoteReadBufferCacheError::NOT_INIT}; } auto remote_path = remote_file_meta.path; - auto & file_size = remote_file_meta.file_size; - auto & last_modification_timestamp = remote_file_meta.last_modification_timestamp; + const auto & file_size = remote_file_meta.file_size; + const auto & last_modification_timestamp = remote_file_meta.last_modification_timestamp; auto local_path = calculateLocalPath(remote_file_meta); std::lock_guard lock(mutex); auto cache_iter = caches.find(local_path); From 50156059aab3031895d89af77d24f6356802eac7 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 15 Nov 2021 10:09:21 +0800 Subject: [PATCH 0050/1260] fixed: change local path format --- src/IO/RemoteReadBufferCache.cpp | 26 ++++++++++---------------- src/IO/RemoteReadBufferCache.h | 2 +- 2 files changed, 11 insertions(+), 17 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 8910ac4a97a..614676e9022 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -414,7 +414,7 @@ RemoteReadBufferCache & RemoteReadBufferCache::instance() return instance; } -void RemoteReadBufferCache::recover_cached_files_meta( +void RemoteReadBufferCache::recoverCachedFilesMeta( const std::filesystem::path ¤t_path, size_t current_depth, size_t max_depth, @@ -425,16 +425,11 @@ void RemoteReadBufferCache::recover_cached_files_meta( for (auto const & dir : std::filesystem::directory_iterator{current_path}) { std::string path = dir.path(); - if (caches.count(path)) - { - LOG_ERROR(log, "duplicated file:{}", path); - continue; - } - auto cache_cntrl = RemoteCacheController::recover(path, finish_callback); - if (!cache_cntrl) + auto cache_controller = RemoteCacheController::recover(path, finish_callback); + if (!cache_controller) continue; auto &cell = caches[path]; - cell.cache_controller = cache_cntrl; + cell.cache_controller = cache_controller; cell.key_iterator = keys.insert(keys.end(), path); } return; @@ -442,7 +437,7 @@ void RemoteReadBufferCache::recover_cached_files_meta( for (auto const &dir : std::filesystem::directory_iterator{current_path}) { - recover_cached_files_meta(dir.path(), current_depth + 1, max_depth, finish_callback); + recoverCachedFilesMeta(dir.path(), current_depth + 1, max_depth, finish_callback); } } @@ -464,18 +459,17 @@ void RemoteReadBufferCache::initOnce(const std::filesystem::path & dir, size_t l } auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; - // four level dir. /// - recover_cached_files_meta(root_dir, 1, 4, callback); + // tw0 level dir. // + recoverCachedFilesMeta(root_dir, 1, 2, callback); inited = true; } std::filesystem::path RemoteReadBufferCache::calculateLocalPath(const RemoteFileMeta & meta) { - auto path_prefix = std::filesystem::path(local_path_prefix) / meta.schema / meta.cluster; - - UInt128 hashcode = sipHash128(meta.path.c_str(), meta.path.size()); + std::string full_path = meta.schema + ":" + meta.cluster + ":" + meta.path; + UInt128 hashcode = sipHash128(full_path.c_str(), full_path.size()); std::string hashcode_str = getHexUIntLowercase(hashcode); - return path_prefix / hashcode_str.substr(0,3) / hashcode_str; + return std::filesystem::path(local_path_prefix) / hashcode_str.substr(0,3) / hashcode_str; } std::tuple, RemoteReadBufferCacheError> RemoteReadBufferCache::createReader( diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 7b02ff1ca60..8a9c535ab1d 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -220,7 +220,7 @@ private: std::filesystem::path calculateLocalPath(const RemoteFileMeta &meta); - void recover_cached_files_meta( + void recoverCachedFilesMeta( const std::filesystem::path & current_path_, size_t current_depth, size_t max_depth, From d1eb628f0c333406eba3c7fb334c4f355009311d Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 15 Nov 2021 11:16:58 +0800 Subject: [PATCH 0051/1260] fixed: change local path format --- src/IO/RemoteReadBufferCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 614676e9022..a3dcfca876e 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -459,7 +459,7 @@ void RemoteReadBufferCache::initOnce(const std::filesystem::path & dir, size_t l } auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; - // tw0 level dir. // + // two level dir. // recoverCachedFilesMeta(root_dir, 1, 2, callback); inited = true; } From e2d928c3b6d059b67a7a13beffb32a0d1f9fdb04 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 15 Nov 2021 14:25:57 +0800 Subject: [PATCH 0052/1260] fixed: change local path format --- src/IO/RemoteReadBufferCache.cpp | 33 ++++++++++---------------------- src/IO/RemoteReadBufferCache.h | 4 +--- 2 files changed, 11 insertions(+), 26 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index a3dcfca876e..b4627d5ca17 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -415,31 +415,19 @@ RemoteReadBufferCache & RemoteReadBufferCache::instance() } void RemoteReadBufferCache::recoverCachedFilesMeta( - const std::filesystem::path ¤t_path, - size_t current_depth, - size_t max_depth, + const std::filesystem::path &path_, std::function const & finish_callback) { - if (current_depth >= max_depth) + for (auto const & dir : std::filesystem::directory_iterator{path_}) { - for (auto const & dir : std::filesystem::directory_iterator{current_path}) - { - std::string path = dir.path(); - auto cache_controller = RemoteCacheController::recover(path, finish_callback); - if (!cache_controller) - continue; - auto &cell = caches[path]; - cell.cache_controller = cache_controller; - cell.key_iterator = keys.insert(keys.end(), path); - } - return; + std::string path = dir.path(); + auto cache_controller = RemoteCacheController::recover(path, finish_callback); + if (!cache_controller) + continue; + auto &cell = caches[path]; + cell.cache_controller = cache_controller; + cell.key_iterator = keys.insert(keys.end(), path); } - - for (auto const &dir : std::filesystem::directory_iterator{current_path}) - { - recoverCachedFilesMeta(dir.path(), current_depth + 1, max_depth, finish_callback); - } - } void RemoteReadBufferCache::initOnce(const std::filesystem::path & dir, size_t limit_size_, size_t bytes_read_before_flush_) @@ -459,8 +447,7 @@ void RemoteReadBufferCache::initOnce(const std::filesystem::path & dir, size_t l } auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; - // two level dir. // - recoverCachedFilesMeta(root_dir, 1, 2, callback); + recoverCachedFilesMeta(root_dir, callback); inited = true; } diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 8a9c535ab1d..82eaf69f940 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -221,9 +221,7 @@ private: std::filesystem::path calculateLocalPath(const RemoteFileMeta &meta); void recoverCachedFilesMeta( - const std::filesystem::path & current_path_, - size_t current_depth, - size_t max_depth, + const std::filesystem::path & path_, std::function const & finish_callback); bool clearLocalCache(); }; From a41a917b519f46a9adc1be17c9b377bd79fba209 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 15 Nov 2021 16:47:12 +0800 Subject: [PATCH 0053/1260] fixed : make recovering local cache meta informations run in backgroup --- programs/server/Server.cpp | 10 ++-- src/IO/RemoteReadBufferCache.cpp | 80 ++++++++++++++++++-------------- src/IO/RemoteReadBufferCache.h | 7 ++- 3 files changed, 58 insertions(+), 39 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b81170038bc..54657e420f1 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -506,9 +506,6 @@ int Server::main(const std::vector & /*args*/) if (ThreadFuzzer::instance().isEffective()) global_context->addWarningMessage("ThreadFuzzer is enabled. Application will run slowly and unstable."); -if (config().has("local_cache_dir") && config().has("local_cache_quota")) - RemoteReadBufferCache::instance().initOnce(config().getString("local_cache_dir"), config().getUInt64("local_cache_quota"), config().getUInt64("local_cache_bytes_read_before_flush",DBMS_DEFAULT_BUFFER_SIZE)); - #if defined(SANITIZER) global_context->addWarningMessage("Server was built with sanitizer. It will work slowly."); #endif @@ -519,6 +516,13 @@ if (config().has("local_cache_dir") && config().has("local_cache_quota")) // ignore `max_thread_pool_size` in configs we fetch from ZK, but oh well. GlobalThreadPool::initialize(config().getUInt("max_thread_pool_size", 10000)); + if (config().has("local_cache_dir") && config().has("local_cache_quota")) + RemoteReadBufferCache::instance().initOnce( + config().getString("local_cache_dir"), + config().getUInt64("local_cache_quota"), + config().getUInt64("local_cache_bytes_read_before_flush",DBMS_DEFAULT_BUFFER_SIZE), + config().getUInt64("local_cache_max_threads", 1000)); + ConnectionCollector::init(global_context, config().getUInt("max_threads_for_connection_collector", 10)); bool has_zookeeper = config().has("zookeeper"); diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index b4627d5ca17..545f2d40cb8 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -67,7 +67,6 @@ RemoteCacheController::RemoteCacheController( std::shared_ptr readbuffer_, std::function const & finish_callback) { - download_thread = nullptr; schema = remote_file_meta.schema; cluster = remote_file_meta.cluster; local_path = local_path_; @@ -105,12 +104,6 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs std::unique_lock lock{mutex}; if (download_finished) { - if (download_thread != nullptr) - { - download_thread->wait(); - LOG_TRACE(&Poco::Logger::get("RemoteCacheController"), "try to release down thread"); - download_thread = nullptr; - } // finish reading if (start_offset_ >= current_offset) { @@ -134,7 +127,6 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs void RemoteCacheController::backgroupDownload(std::function const & finish_callback) { - download_thread = std::make_shared(1); auto task = [this, finish_callback]() { size_t unflush_bytes = 0; @@ -173,7 +165,7 @@ void RemoteCacheController::backgroupDownload(std::functionscheduleOrThrow(task); + RemoteReadBufferCache::instance().GetThreadPool()->scheduleOrThrow(task); } void RemoteCacheController::flush(bool need_flush_meta_) @@ -199,14 +191,7 @@ void RemoteCacheController::flush(bool need_flush_meta_) meta_file.close(); } -RemoteCacheController::~RemoteCacheController() -{ - if (download_thread != nullptr) - { - download_thread->wait(); - } -} - +RemoteCacheController::~RemoteCacheController() = default; void RemoteCacheController::close() { // delete the directory @@ -406,7 +391,10 @@ off_t RemoteReadBuffer::getPosition() RemoteReadBufferCache::RemoteReadBufferCache() = default; -RemoteReadBufferCache::~RemoteReadBufferCache() = default; +RemoteReadBufferCache::~RemoteReadBufferCache() +{ + threadPool->wait(); +} RemoteReadBufferCache & RemoteReadBufferCache::instance() { @@ -415,28 +403,43 @@ RemoteReadBufferCache & RemoteReadBufferCache::instance() } void RemoteReadBufferCache::recoverCachedFilesMeta( - const std::filesystem::path &path_, + const std::filesystem::path ¤t_path, + size_t current_depth, + size_t max_depth, std::function const & finish_callback) { - for (auto const & dir : std::filesystem::directory_iterator{path_}) + if (current_depth >= max_depth) { - std::string path = dir.path(); - auto cache_controller = RemoteCacheController::recover(path, finish_callback); - if (!cache_controller) - continue; - auto &cell = caches[path]; - cell.cache_controller = cache_controller; - cell.key_iterator = keys.insert(keys.end(), path); + for (auto const & dir : std::filesystem::directory_iterator{current_path}) + { + std::string path = dir.path(); + auto cache_controller = RemoteCacheController::recover(path, finish_callback); + if (!cache_controller) + continue; + auto &cell = caches[path]; + cell.cache_controller = cache_controller; + cell.key_iterator = keys.insert(keys.end(), path); + } + return; } + + for (auto const &dir : std::filesystem::directory_iterator{current_path}) + { + recoverCachedFilesMeta(dir.path(), current_depth + 1, max_depth, finish_callback); + } + } -void RemoteReadBufferCache::initOnce(const std::filesystem::path & dir, size_t limit_size_, size_t bytes_read_before_flush_) +void RemoteReadBufferCache::initOnce(const std::filesystem::path & dir, + size_t limit_size_, + size_t bytes_read_before_flush_, + size_t max_threads) { LOG_TRACE(log, "init local cache. path: {}, limit {}", dir.string(), limit_size_); - std::lock_guard lock(mutex); local_path_prefix = dir; limit_size = limit_size_; local_cache_bytes_read_before_flush = bytes_read_before_flush_; + threadPool = std::make_shared(max_threads, 1000, 1000, false); // scan local disk dir and recover the cache metas std::filesystem::path root_dir(local_path_prefix); @@ -445,10 +448,17 @@ void RemoteReadBufferCache::initOnce(const std::filesystem::path & dir, size_t l LOG_INFO(log, "{} not exists. this cache will be disable", local_path_prefix); return; } - auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; - - recoverCachedFilesMeta(root_dir, callback); - inited = true; + + auto recover_task = [this, root_dir]() + { + auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; + std::lock_guard lock(this->mutex); + // two level dir. // + recoverCachedFilesMeta(root_dir, 1, 2, callback); + this->inited = true; + LOG_TRACE(this->log, "recovered from disk "); + }; + GetThreadPool()->scheduleOrThrow(recover_task); } std::filesystem::path RemoteReadBufferCache::calculateLocalPath(const RemoteFileMeta & meta) @@ -463,11 +473,13 @@ std::tuple, RemoteReadBufferCacheError> R const RemoteFileMeta &remote_file_meta, std::shared_ptr & readbuffer) { + // If something is wrong on startup, rollback to read from the orignal ReadBuffer if (!hasInitialized()) { - LOG_ERROR(log, "RemoteReadBufferCache not init"); + LOG_ERROR(log, "RemoteReadBufferCache has not initialized"); return {nullptr, RemoteReadBufferCacheError::NOT_INIT}; } + auto remote_path = remote_file_meta.path; const auto & file_size = remote_file_meta.file_size; const auto & last_modification_timestamp = remote_file_meta.last_modification_timestamp; diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 82eaf69f940..f8830c9b5fa 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -115,7 +115,6 @@ private: std::mutex mutex; std::condition_variable more_data_signal; - std::shared_ptr download_thread; std::set opened_file_streams; @@ -192,8 +191,9 @@ public: ~RemoteReadBufferCache(); // global instance static RemoteReadBufferCache & instance(); + std::shared_ptr GetThreadPool(){ return threadPool; } - void initOnce(const std::filesystem::path & dir, size_t limit_size, size_t bytes_read_before_flush_); + void initOnce(const std::filesystem::path & dir, size_t limit_size, size_t bytes_read_before_flush_, size_t max_threads); inline bool hasInitialized() const { return inited; } std::tuple, RemoteReadBufferCacheError> createReader( @@ -203,6 +203,7 @@ public: private: std::string local_path_prefix; + std::shared_ptr threadPool; std::atomic inited = false; std::mutex mutex; size_t limit_size = 0; @@ -222,6 +223,8 @@ private: void recoverCachedFilesMeta( const std::filesystem::path & path_, + size_t current_depth, + size_t max_depth, std::function const & finish_callback); bool clearLocalCache(); }; From 78eefe2cac96c3ee43b033467259988e6fcb0b02 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 15 Nov 2021 17:51:09 +0800 Subject: [PATCH 0054/1260] fixed code style --- src/IO/RemoteReadBufferCache.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 545f2d40cb8..d3aca13cf91 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -448,7 +448,6 @@ void RemoteReadBufferCache::initOnce(const std::filesystem::path & dir, LOG_INFO(log, "{} not exists. this cache will be disable", local_path_prefix); return; } - auto recover_task = [this, root_dir]() { auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; From 41d3ec2481a4fbbab1d8a141a4646bf16bc44fb1 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 15 Nov 2021 19:08:58 +0800 Subject: [PATCH 0055/1260] fixed typos --- src/IO/RemoteReadBufferCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index d3aca13cf91..3bf83fa81d7 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -472,7 +472,7 @@ std::tuple, RemoteReadBufferCacheError> R const RemoteFileMeta &remote_file_meta, std::shared_ptr & readbuffer) { - // If something is wrong on startup, rollback to read from the orignal ReadBuffer + // If something is wrong on startup, rollback to read from the original ReadBuffer if (!hasInitialized()) { LOG_ERROR(log, "RemoteReadBufferCache has not initialized"); From 74aeb835c065ebc0f19030ed948e210ac9cd571e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 15 Nov 2021 23:34:48 +0800 Subject: [PATCH 0056/1260] commit again --- contrib/CMakeLists.txt | 2 +- contrib/hive-metastore-cmake/CMakeLists.txt | 9 +++++++++ contrib/hive-metastore/CMakeLists.txt | 9 --------- 3 files changed, 10 insertions(+), 10 deletions(-) create mode 100644 contrib/hive-metastore-cmake/CMakeLists.txt delete mode 100644 contrib/hive-metastore/CMakeLists.txt diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 3d1296f01ed..81960d54ec3 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -349,7 +349,7 @@ if (USE_S2_GEOMETRY) add_subdirectory(s2geometry-cmake) endif() -add_subdirectory (hive-metastore) +add_subdirectory (hive-metastore-cmake) # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear diff --git a/contrib/hive-metastore-cmake/CMakeLists.txt b/contrib/hive-metastore-cmake/CMakeLists.txt new file mode 100644 index 00000000000..669ade7d2ad --- /dev/null +++ b/contrib/hive-metastore-cmake/CMakeLists.txt @@ -0,0 +1,9 @@ +set (SRCS + ${ClickHouse_SOURCE_DIR}/contrib/hive-metastore/hive_metastore_constants.cpp + ${ClickHouse_SOURCE_DIR}/contrib/hive-metastore/hive_metastore_types.cpp + ${ClickHouse_SOURCE_DIR}/contrib/hive-metastore/ThriftHiveMetastore.cpp +) + +add_library(hivemetastore ${SRCS}) +target_link_libraries(hivemetastore PUBLIC ${THRIFT_LIBRARY}) +target_include_directories(hivemetastore SYSTEM PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/hive-metastore) diff --git a/contrib/hive-metastore/CMakeLists.txt b/contrib/hive-metastore/CMakeLists.txt deleted file mode 100644 index 0792093cfb1..00000000000 --- a/contrib/hive-metastore/CMakeLists.txt +++ /dev/null @@ -1,9 +0,0 @@ -set (SRCS - ${CMAKE_CURRENT_LIST_DIR}/hive_metastore_constants.cpp - ${CMAKE_CURRENT_LIST_DIR}/hive_metastore_types.cpp - ${CMAKE_CURRENT_LIST_DIR}/ThriftHiveMetastore.cpp -) - -add_library(hivemetastore ${SRCS}) -target_link_libraries(hivemetastore PUBLIC ${THRIFT_LIBRARY}) -target_include_directories(hivemetastore SYSTEM PUBLIC ${CMAKE_CURRENT_LIST_DIR}) From 970d90ab1557863b555f1f3706b324e2446a19db Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 16 Nov 2021 11:09:27 +0800 Subject: [PATCH 0057/1260] fixed code style --- src/IO/RemoteReadBufferCache.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index f8830c9b5fa..5228c53ddd6 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -222,7 +222,7 @@ private: std::filesystem::path calculateLocalPath(const RemoteFileMeta &meta); void recoverCachedFilesMeta( - const std::filesystem::path & path_, + const std::filesystem::path & current_path, size_t current_depth, size_t max_depth, std::function const & finish_callback); From 92b8ec5758e14e943057d2654c64852e7340d69b Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 16 Nov 2021 11:11:58 +0800 Subject: [PATCH 0058/1260] fixed code style --- src/IO/RemoteReadBufferCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 3bf83fa81d7..b93de46398c 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -403,7 +403,7 @@ RemoteReadBufferCache & RemoteReadBufferCache::instance() } void RemoteReadBufferCache::recoverCachedFilesMeta( - const std::filesystem::path ¤t_path, + const std::filesystem::path & current_path, size_t current_depth, size_t max_depth, std::function const & finish_callback) From c2915161e2b91b0cb044e8f1187ad6ca956f153f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 16 Nov 2021 11:32:08 +0800 Subject: [PATCH 0059/1260] add submodule hive-metasotre --- .gitmodules | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.gitmodules b/.gitmodules index 8ad81b5094f..cb906d45719 100644 --- a/.gitmodules +++ b/.gitmodules @@ -256,3 +256,6 @@ [submodule "contrib/sysroot"] path = contrib/sysroot url = https://github.com/ClickHouse-Extras/sysroot.git +[submodule "contrib/hive-metastore"] + path = contrib/hive-metastore + url = https://github.com/ClickHouse-Extras/hive-metastore.git From 11d3c99b98abf8f67ba1b3965c093b1327037756 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 16 Nov 2021 11:36:26 +0800 Subject: [PATCH 0060/1260] remove local hive-metastore --- .../hive-metastore/ThriftHiveMetastore.cpp | 79541 ---------------- contrib/hive-metastore/ThriftHiveMetastore.h | 25246 ----- contrib/hive-metastore/hive_metastore.thrift | 1807 - .../hive_metastore_constants.cpp | 65 - .../hive-metastore/hive_metastore_constants.h | 48 - .../hive-metastore/hive_metastore_types.cpp | 26465 ----- contrib/hive-metastore/hive_metastore_types.h | 10002 -- 7 files changed, 143174 deletions(-) delete mode 100644 contrib/hive-metastore/ThriftHiveMetastore.cpp delete mode 100644 contrib/hive-metastore/ThriftHiveMetastore.h delete mode 100644 contrib/hive-metastore/hive_metastore.thrift delete mode 100644 contrib/hive-metastore/hive_metastore_constants.cpp delete mode 100644 contrib/hive-metastore/hive_metastore_constants.h delete mode 100644 contrib/hive-metastore/hive_metastore_types.cpp delete mode 100644 contrib/hive-metastore/hive_metastore_types.h diff --git a/contrib/hive-metastore/ThriftHiveMetastore.cpp b/contrib/hive-metastore/ThriftHiveMetastore.cpp deleted file mode 100644 index f6209bac7b0..00000000000 --- a/contrib/hive-metastore/ThriftHiveMetastore.cpp +++ /dev/null @@ -1,79541 +0,0 @@ -/** - * Autogenerated by Thrift Compiler () - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -#include "ThriftHiveMetastore.h" - -namespace Apache { namespace Hadoop { namespace Hive { - - -ThriftHiveMetastore_getMetaConf_args::~ThriftHiveMetastore_getMetaConf_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_getMetaConf_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->key); - this->__isset.key = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_getMetaConf_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_getMetaConf_args"); - - xfer += oprot->writeFieldBegin("key", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->key); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_getMetaConf_pargs::~ThriftHiveMetastore_getMetaConf_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_getMetaConf_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_getMetaConf_pargs"); - - xfer += oprot->writeFieldBegin("key", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->key))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_getMetaConf_result::~ThriftHiveMetastore_getMetaConf_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_getMetaConf_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_getMetaConf_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_getMetaConf_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0); - xfer += oprot->writeString(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_getMetaConf_presult::~ThriftHiveMetastore_getMetaConf_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_getMetaConf_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_setMetaConf_args::~ThriftHiveMetastore_setMetaConf_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_setMetaConf_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->key); - this->__isset.key = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->value); - this->__isset.value = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_setMetaConf_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_setMetaConf_args"); - - xfer += oprot->writeFieldBegin("key", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->key); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("value", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->value); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_setMetaConf_pargs::~ThriftHiveMetastore_setMetaConf_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_setMetaConf_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_setMetaConf_pargs"); - - xfer += oprot->writeFieldBegin("key", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->key))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("value", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->value))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_setMetaConf_result::~ThriftHiveMetastore_setMetaConf_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_setMetaConf_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_setMetaConf_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_setMetaConf_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_setMetaConf_presult::~ThriftHiveMetastore_setMetaConf_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_setMetaConf_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_create_database_args::~ThriftHiveMetastore_create_database_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_database_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->database.read(iprot); - this->__isset.database = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_database_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_database_args"); - - xfer += oprot->writeFieldBegin("database", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->database.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_database_pargs::~ThriftHiveMetastore_create_database_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_database_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_database_pargs"); - - xfer += oprot->writeFieldBegin("database", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->database)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_database_result::~ThriftHiveMetastore_create_database_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_database_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_database_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_database_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_database_presult::~ThriftHiveMetastore_create_database_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_database_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_database_args::~ThriftHiveMetastore_get_database_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_database_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->name); - this->__isset.name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_database_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_database_args"); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_database_pargs::~ThriftHiveMetastore_get_database_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_database_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_database_pargs"); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_database_result::~ThriftHiveMetastore_get_database_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_database_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_database_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_database_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_database_presult::~ThriftHiveMetastore_get_database_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_database_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_drop_database_args::~ThriftHiveMetastore_drop_database_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_database_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->name); - this->__isset.name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->deleteData); - this->__isset.deleteData = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->cascade); - this->__isset.cascade = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_database_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_database_args"); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 2); - xfer += oprot->writeBool(this->deleteData); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("cascade", ::apache::thrift::protocol::T_BOOL, 3); - xfer += oprot->writeBool(this->cascade); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_database_pargs::~ThriftHiveMetastore_drop_database_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_database_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_database_pargs"); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 2); - xfer += oprot->writeBool((*(this->deleteData))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("cascade", ::apache::thrift::protocol::T_BOOL, 3); - xfer += oprot->writeBool((*(this->cascade))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_database_result::~ThriftHiveMetastore_drop_database_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_database_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_database_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_database_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_database_presult::~ThriftHiveMetastore_drop_database_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_database_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_databases_args::~ThriftHiveMetastore_get_databases_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_databases_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->pattern); - this->__isset.pattern = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_databases_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_databases_args"); - - xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->pattern); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_databases_pargs::~ThriftHiveMetastore_get_databases_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_databases_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_databases_pargs"); - - xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->pattern))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_databases_result::~ThriftHiveMetastore_get_databases_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_databases_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size982; - ::apache::thrift::protocol::TType _etype985; - xfer += iprot->readListBegin(_etype985, _size982); - this->success.resize(_size982); - uint32_t _i986; - for (_i986 = 0; _i986 < _size982; ++_i986) - { - xfer += iprot->readString(this->success[_i986]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_databases_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter987; - for (_iter987 = this->success.begin(); _iter987 != this->success.end(); ++_iter987) - { - xfer += oprot->writeString((*_iter987)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_databases_presult::~ThriftHiveMetastore_get_databases_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_databases_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size988; - ::apache::thrift::protocol::TType _etype991; - xfer += iprot->readListBegin(_etype991, _size988); - (*(this->success)).resize(_size988); - uint32_t _i992; - for (_i992 = 0; _i992 < _size988; ++_i992) - { - xfer += iprot->readString((*(this->success))[_i992]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_all_databases_args::~ThriftHiveMetastore_get_all_databases_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_databases_args::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 ThriftHiveMetastore_get_all_databases_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_databases_args"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_all_databases_pargs::~ThriftHiveMetastore_get_all_databases_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_databases_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_databases_pargs"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_all_databases_result::~ThriftHiveMetastore_get_all_databases_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_databases_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size993; - ::apache::thrift::protocol::TType _etype996; - xfer += iprot->readListBegin(_etype996, _size993); - this->success.resize(_size993); - uint32_t _i997; - for (_i997 = 0; _i997 < _size993; ++_i997) - { - xfer += iprot->readString(this->success[_i997]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_databases_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter998; - for (_iter998 = this->success.begin(); _iter998 != this->success.end(); ++_iter998) - { - xfer += oprot->writeString((*_iter998)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_all_databases_presult::~ThriftHiveMetastore_get_all_databases_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_databases_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size999; - ::apache::thrift::protocol::TType _etype1002; - xfer += iprot->readListBegin(_etype1002, _size999); - (*(this->success)).resize(_size999); - uint32_t _i1003; - for (_i1003 = 0; _i1003 < _size999; ++_i1003) - { - xfer += iprot->readString((*(this->success))[_i1003]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_alter_database_args::~ThriftHiveMetastore_alter_database_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_database_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbname); - this->__isset.dbname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->db.read(iprot); - this->__isset.db = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_database_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_database_args"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbname); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("db", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->db.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_database_pargs::~ThriftHiveMetastore_alter_database_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_database_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_database_pargs"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->dbname))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("db", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += (*(this->db)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_database_result::~ThriftHiveMetastore_alter_database_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_database_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_database_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_database_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_database_presult::~ThriftHiveMetastore_alter_database_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_database_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_type_args::~ThriftHiveMetastore_get_type_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_type_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->name); - this->__isset.name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_type_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_args"); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_type_pargs::~ThriftHiveMetastore_get_type_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_type_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_pargs"); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_type_result::~ThriftHiveMetastore_get_type_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_type_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_type_presult::~ThriftHiveMetastore_get_type_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_type_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_create_type_args::~ThriftHiveMetastore_create_type_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_type_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->type.read(iprot); - this->__isset.type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_type_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_type_args"); - - xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->type.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_type_pargs::~ThriftHiveMetastore_create_type_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_type_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_type_pargs"); - - xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->type)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_type_result::~ThriftHiveMetastore_create_type_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_type_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_type_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_type_presult::~ThriftHiveMetastore_create_type_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_type_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_drop_type_args::~ThriftHiveMetastore_drop_type_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_type_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->type); - this->__isset.type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_type_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_type_args"); - - xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->type); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_type_pargs::~ThriftHiveMetastore_drop_type_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_type_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_type_pargs"); - - xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->type))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_type_result::~ThriftHiveMetastore_drop_type_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_type_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_type_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_type_presult::~ThriftHiveMetastore_drop_type_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_type_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_type_all_args::~ThriftHiveMetastore_get_type_all_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_type_all_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->name); - this->__isset.name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_type_all_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_all_args"); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_type_all_pargs::~ThriftHiveMetastore_get_type_all_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_type_all_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_all_pargs"); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_type_all_result::~ThriftHiveMetastore_get_type_all_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_type_all_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->success.clear(); - uint32_t _size1004; - ::apache::thrift::protocol::TType _ktype1005; - ::apache::thrift::protocol::TType _vtype1006; - xfer += iprot->readMapBegin(_ktype1005, _vtype1006, _size1004); - uint32_t _i1008; - for (_i1008 = 0; _i1008 < _size1004; ++_i1008) - { - std::string _key1009; - xfer += iprot->readString(_key1009); - Type& _val1010 = this->success[_key1009]; - xfer += _val1010.read(iprot); - } - xfer += iprot->readMapEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_all_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::map ::const_iterator _iter1011; - for (_iter1011 = this->success.begin(); _iter1011 != this->success.end(); ++_iter1011) - { - xfer += oprot->writeString(_iter1011->first); - xfer += _iter1011->second.write(oprot); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_type_all_presult::~ThriftHiveMetastore_get_type_all_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_type_all_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - (*(this->success)).clear(); - uint32_t _size1012; - ::apache::thrift::protocol::TType _ktype1013; - ::apache::thrift::protocol::TType _vtype1014; - xfer += iprot->readMapBegin(_ktype1013, _vtype1014, _size1012); - uint32_t _i1016; - for (_i1016 = 0; _i1016 < _size1012; ++_i1016) - { - std::string _key1017; - xfer += iprot->readString(_key1017); - Type& _val1018 = (*(this->success))[_key1017]; - xfer += _val1018.read(iprot); - } - xfer += iprot->readMapEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_fields_args::~ThriftHiveMetastore_get_fields_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_fields_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->table_name); - this->__isset.table_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_fields_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->table_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_fields_pargs::~ThriftHiveMetastore_get_fields_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_fields_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->table_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_fields_result::~ThriftHiveMetastore_get_fields_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_fields_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1019; - ::apache::thrift::protocol::TType _etype1022; - xfer += iprot->readListBegin(_etype1022, _size1019); - this->success.resize(_size1019); - uint32_t _i1023; - for (_i1023 = 0; _i1023 < _size1019; ++_i1023) - { - xfer += this->success[_i1023].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1024; - for (_iter1024 = this->success.begin(); _iter1024 != this->success.end(); ++_iter1024) - { - xfer += (*_iter1024).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_fields_presult::~ThriftHiveMetastore_get_fields_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_fields_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1025; - ::apache::thrift::protocol::TType _etype1028; - xfer += iprot->readListBegin(_etype1028, _size1025); - (*(this->success)).resize(_size1025); - uint32_t _i1029; - for (_i1029 = 0; _i1029 < _size1025; ++_i1029) - { - xfer += (*(this->success))[_i1029].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_fields_with_environment_context_args::~ThriftHiveMetastore_get_fields_with_environment_context_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_fields_with_environment_context_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->table_name); - this->__isset.table_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->environment_context.read(iprot); - this->__isset.environment_context = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_fields_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_with_environment_context_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->table_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->environment_context.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_fields_with_environment_context_pargs::~ThriftHiveMetastore_get_fields_with_environment_context_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_fields_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_with_environment_context_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->table_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += (*(this->environment_context)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_fields_with_environment_context_result::~ThriftHiveMetastore_get_fields_with_environment_context_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1030; - ::apache::thrift::protocol::TType _etype1033; - xfer += iprot->readListBegin(_etype1033, _size1030); - this->success.resize(_size1030); - uint32_t _i1034; - for (_i1034 = 0; _i1034 < _size1030; ++_i1034) - { - xfer += this->success[_i1034].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_with_environment_context_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1035; - for (_iter1035 = this->success.begin(); _iter1035 != this->success.end(); ++_iter1035) - { - xfer += (*_iter1035).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_fields_with_environment_context_presult::~ThriftHiveMetastore_get_fields_with_environment_context_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1036; - ::apache::thrift::protocol::TType _etype1039; - xfer += iprot->readListBegin(_etype1039, _size1036); - (*(this->success)).resize(_size1036); - uint32_t _i1040; - for (_i1040 = 0; _i1040 < _size1036; ++_i1040) - { - xfer += (*(this->success))[_i1040].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_schema_args::~ThriftHiveMetastore_get_schema_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_schema_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->table_name); - this->__isset.table_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_schema_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_schema_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->table_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_schema_pargs::~ThriftHiveMetastore_get_schema_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_schema_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_schema_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->table_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_schema_result::~ThriftHiveMetastore_get_schema_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_schema_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1041; - ::apache::thrift::protocol::TType _etype1044; - xfer += iprot->readListBegin(_etype1044, _size1041); - this->success.resize(_size1041); - uint32_t _i1045; - for (_i1045 = 0; _i1045 < _size1041; ++_i1045) - { - xfer += this->success[_i1045].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_schema_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1046; - for (_iter1046 = this->success.begin(); _iter1046 != this->success.end(); ++_iter1046) - { - xfer += (*_iter1046).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_schema_presult::~ThriftHiveMetastore_get_schema_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_schema_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1047; - ::apache::thrift::protocol::TType _etype1050; - xfer += iprot->readListBegin(_etype1050, _size1047); - (*(this->success)).resize(_size1047); - uint32_t _i1051; - for (_i1051 = 0; _i1051 < _size1047; ++_i1051) - { - xfer += (*(this->success))[_i1051].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_schema_with_environment_context_args::~ThriftHiveMetastore_get_schema_with_environment_context_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_schema_with_environment_context_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->table_name); - this->__isset.table_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->environment_context.read(iprot); - this->__isset.environment_context = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_schema_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_schema_with_environment_context_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->table_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->environment_context.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_schema_with_environment_context_pargs::~ThriftHiveMetastore_get_schema_with_environment_context_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_schema_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_schema_with_environment_context_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->table_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += (*(this->environment_context)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_schema_with_environment_context_result::~ThriftHiveMetastore_get_schema_with_environment_context_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1052; - ::apache::thrift::protocol::TType _etype1055; - xfer += iprot->readListBegin(_etype1055, _size1052); - this->success.resize(_size1052); - uint32_t _i1056; - for (_i1056 = 0; _i1056 < _size1052; ++_i1056) - { - xfer += this->success[_i1056].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_schema_with_environment_context_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1057; - for (_iter1057 = this->success.begin(); _iter1057 != this->success.end(); ++_iter1057) - { - xfer += (*_iter1057).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_schema_with_environment_context_presult::~ThriftHiveMetastore_get_schema_with_environment_context_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1058; - ::apache::thrift::protocol::TType _etype1061; - xfer += iprot->readListBegin(_etype1061, _size1058); - (*(this->success)).resize(_size1058); - uint32_t _i1062; - for (_i1062 = 0; _i1062 < _size1058; ++_i1062) - { - xfer += (*(this->success))[_i1062].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_create_table_args::~ThriftHiveMetastore_create_table_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_table_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->tbl.read(iprot); - this->__isset.tbl = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_table_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_args"); - - xfer += oprot->writeFieldBegin("tbl", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->tbl.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_table_pargs::~ThriftHiveMetastore_create_table_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_table_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_pargs"); - - xfer += oprot->writeFieldBegin("tbl", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->tbl)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_table_result::~ThriftHiveMetastore_create_table_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_table_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_table_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o4) { - xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->o4.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_table_presult::~ThriftHiveMetastore_create_table_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_table_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_create_table_with_environment_context_args::~ThriftHiveMetastore_create_table_with_environment_context_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_table_with_environment_context_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->tbl.read(iprot); - this->__isset.tbl = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->environment_context.read(iprot); - this->__isset.environment_context = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_table_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_with_environment_context_args"); - - xfer += oprot->writeFieldBegin("tbl", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->tbl.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->environment_context.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_table_with_environment_context_pargs::~ThriftHiveMetastore_create_table_with_environment_context_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_table_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_with_environment_context_pargs"); - - xfer += oprot->writeFieldBegin("tbl", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->tbl)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += (*(this->environment_context)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_table_with_environment_context_result::~ThriftHiveMetastore_create_table_with_environment_context_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_table_with_environment_context_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_table_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_with_environment_context_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o4) { - xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->o4.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_table_with_environment_context_presult::~ThriftHiveMetastore_create_table_with_environment_context_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_table_with_environment_context_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_create_table_with_constraints_args::~ThriftHiveMetastore_create_table_with_constraints_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_table_with_constraints_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->tbl.read(iprot); - this->__isset.tbl = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->primaryKeys.clear(); - uint32_t _size1063; - ::apache::thrift::protocol::TType _etype1066; - xfer += iprot->readListBegin(_etype1066, _size1063); - this->primaryKeys.resize(_size1063); - uint32_t _i1067; - for (_i1067 = 0; _i1067 < _size1063; ++_i1067) - { - xfer += this->primaryKeys[_i1067].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.primaryKeys = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->foreignKeys.clear(); - uint32_t _size1068; - ::apache::thrift::protocol::TType _etype1071; - xfer += iprot->readListBegin(_etype1071, _size1068); - this->foreignKeys.resize(_size1068); - uint32_t _i1072; - for (_i1072 = 0; _i1072 < _size1068; ++_i1072) - { - xfer += this->foreignKeys[_i1072].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.foreignKeys = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->uniqueConstraints.clear(); - uint32_t _size1073; - ::apache::thrift::protocol::TType _etype1076; - xfer += iprot->readListBegin(_etype1076, _size1073); - this->uniqueConstraints.resize(_size1073); - uint32_t _i1077; - for (_i1077 = 0; _i1077 < _size1073; ++_i1077) - { - xfer += this->uniqueConstraints[_i1077].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.uniqueConstraints = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->notNullConstraints.clear(); - uint32_t _size1078; - ::apache::thrift::protocol::TType _etype1081; - xfer += iprot->readListBegin(_etype1081, _size1078); - this->notNullConstraints.resize(_size1078); - uint32_t _i1082; - for (_i1082 = 0; _i1082 < _size1078; ++_i1082) - { - xfer += this->notNullConstraints[_i1082].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.notNullConstraints = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_with_constraints_args"); - - xfer += oprot->writeFieldBegin("tbl", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->tbl.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->primaryKeys.size())); - std::vector ::const_iterator _iter1083; - for (_iter1083 = this->primaryKeys.begin(); _iter1083 != this->primaryKeys.end(); ++_iter1083) - { - xfer += (*_iter1083).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->foreignKeys.size())); - std::vector ::const_iterator _iter1084; - for (_iter1084 = this->foreignKeys.begin(); _iter1084 != this->foreignKeys.end(); ++_iter1084) - { - xfer += (*_iter1084).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->uniqueConstraints.size())); - std::vector ::const_iterator _iter1085; - for (_iter1085 = this->uniqueConstraints.begin(); _iter1085 != this->uniqueConstraints.end(); ++_iter1085) - { - xfer += (*_iter1085).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->notNullConstraints.size())); - std::vector ::const_iterator _iter1086; - for (_iter1086 = this->notNullConstraints.begin(); _iter1086 != this->notNullConstraints.end(); ++_iter1086) - { - xfer += (*_iter1086).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_table_with_constraints_pargs::~ThriftHiveMetastore_create_table_with_constraints_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_with_constraints_pargs"); - - xfer += oprot->writeFieldBegin("tbl", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->tbl)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->primaryKeys)).size())); - std::vector ::const_iterator _iter1087; - for (_iter1087 = (*(this->primaryKeys)).begin(); _iter1087 != (*(this->primaryKeys)).end(); ++_iter1087) - { - xfer += (*_iter1087).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->foreignKeys)).size())); - std::vector ::const_iterator _iter1088; - for (_iter1088 = (*(this->foreignKeys)).begin(); _iter1088 != (*(this->foreignKeys)).end(); ++_iter1088) - { - xfer += (*_iter1088).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->uniqueConstraints)).size())); - std::vector ::const_iterator _iter1089; - for (_iter1089 = (*(this->uniqueConstraints)).begin(); _iter1089 != (*(this->uniqueConstraints)).end(); ++_iter1089) - { - xfer += (*_iter1089).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->notNullConstraints)).size())); - std::vector ::const_iterator _iter1090; - for (_iter1090 = (*(this->notNullConstraints)).begin(); _iter1090 != (*(this->notNullConstraints)).end(); ++_iter1090) - { - xfer += (*_iter1090).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_table_with_constraints_result::~ThriftHiveMetastore_create_table_with_constraints_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_table_with_constraints_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_table_with_constraints_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_table_with_constraints_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o4) { - xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->o4.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_table_with_constraints_presult::~ThriftHiveMetastore_create_table_with_constraints_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_table_with_constraints_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_drop_constraint_args::~ThriftHiveMetastore_drop_constraint_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_constraint_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->req.read(iprot); - this->__isset.req = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_constraint_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_constraint_args"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->req.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_constraint_pargs::~ThriftHiveMetastore_drop_constraint_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_constraint_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_constraint_pargs"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->req)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_constraint_result::~ThriftHiveMetastore_drop_constraint_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_constraint_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_constraint_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_constraint_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_constraint_presult::~ThriftHiveMetastore_drop_constraint_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_constraint_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_add_primary_key_args::~ThriftHiveMetastore_add_primary_key_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_primary_key_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->req.read(iprot); - this->__isset.req = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_primary_key_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_primary_key_args"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->req.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_primary_key_pargs::~ThriftHiveMetastore_add_primary_key_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_primary_key_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_primary_key_pargs"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->req)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_primary_key_result::~ThriftHiveMetastore_add_primary_key_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_primary_key_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_primary_key_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_primary_key_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_primary_key_presult::~ThriftHiveMetastore_add_primary_key_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_primary_key_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_add_foreign_key_args::~ThriftHiveMetastore_add_foreign_key_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_foreign_key_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->req.read(iprot); - this->__isset.req = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_foreign_key_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_foreign_key_args"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->req.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_foreign_key_pargs::~ThriftHiveMetastore_add_foreign_key_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_foreign_key_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_foreign_key_pargs"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->req)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_foreign_key_result::~ThriftHiveMetastore_add_foreign_key_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_foreign_key_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_foreign_key_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_foreign_key_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_foreign_key_presult::~ThriftHiveMetastore_add_foreign_key_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_foreign_key_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_add_unique_constraint_args::~ThriftHiveMetastore_add_unique_constraint_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_unique_constraint_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->req.read(iprot); - this->__isset.req = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_unique_constraint_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_unique_constraint_args"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->req.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_unique_constraint_pargs::~ThriftHiveMetastore_add_unique_constraint_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_unique_constraint_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_unique_constraint_pargs"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->req)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_unique_constraint_result::~ThriftHiveMetastore_add_unique_constraint_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_unique_constraint_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_unique_constraint_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_unique_constraint_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_unique_constraint_presult::~ThriftHiveMetastore_add_unique_constraint_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_unique_constraint_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_add_not_null_constraint_args::~ThriftHiveMetastore_add_not_null_constraint_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_not_null_constraint_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->req.read(iprot); - this->__isset.req = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_not_null_constraint_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_not_null_constraint_args"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->req.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_not_null_constraint_pargs::~ThriftHiveMetastore_add_not_null_constraint_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_not_null_constraint_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_not_null_constraint_pargs"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->req)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_not_null_constraint_result::~ThriftHiveMetastore_add_not_null_constraint_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_not_null_constraint_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_not_null_constraint_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_not_null_constraint_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_not_null_constraint_presult::~ThriftHiveMetastore_add_not_null_constraint_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_not_null_constraint_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_drop_table_args::~ThriftHiveMetastore_drop_table_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_table_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbname); - this->__isset.dbname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->name); - this->__isset.name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->deleteData); - this->__isset.deleteData = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_table_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_args"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbname); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 3); - xfer += oprot->writeBool(this->deleteData); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_table_pargs::~ThriftHiveMetastore_drop_table_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_table_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_pargs"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->dbname))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 3); - xfer += oprot->writeBool((*(this->deleteData))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_table_result::~ThriftHiveMetastore_drop_table_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_table_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_table_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_table_presult::~ThriftHiveMetastore_drop_table_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_table_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_drop_table_with_environment_context_args::~ThriftHiveMetastore_drop_table_with_environment_context_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbname); - this->__isset.dbname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->name); - this->__isset.name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->deleteData); - this->__isset.deleteData = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->environment_context.read(iprot); - this->__isset.environment_context = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_table_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_args"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbname); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 3); - xfer += oprot->writeBool(this->deleteData); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->environment_context.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_table_with_environment_context_pargs::~ThriftHiveMetastore_drop_table_with_environment_context_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_table_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_pargs"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->dbname))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 3); - xfer += oprot->writeBool((*(this->deleteData))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += (*(this->environment_context)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_table_with_environment_context_result::~ThriftHiveMetastore_drop_table_with_environment_context_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_table_with_environment_context_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_table_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_table_with_environment_context_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_table_with_environment_context_presult::~ThriftHiveMetastore_drop_table_with_environment_context_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_table_with_environment_context_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_truncate_table_args::~ThriftHiveMetastore_truncate_table_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_truncate_table_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbName); - this->__isset.dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tableName); - this->__isset.tableName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->partNames.clear(); - uint32_t _size1091; - ::apache::thrift::protocol::TType _etype1094; - xfer += iprot->readListBegin(_etype1094, _size1091); - this->partNames.resize(_size1091); - uint32_t _i1095; - for (_i1095 = 0; _i1095 < _size1091; ++_i1095) - { - xfer += iprot->readString(this->partNames[_i1095]); - } - xfer += iprot->readListEnd(); - } - this->__isset.partNames = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_truncate_table_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_truncate_table_args"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tableName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partNames.size())); - std::vector ::const_iterator _iter1096; - for (_iter1096 = this->partNames.begin(); _iter1096 != this->partNames.end(); ++_iter1096) - { - xfer += oprot->writeString((*_iter1096)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_truncate_table_pargs::~ThriftHiveMetastore_truncate_table_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_truncate_table_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_truncate_table_pargs"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->dbName))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tableName))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->partNames)).size())); - std::vector ::const_iterator _iter1097; - for (_iter1097 = (*(this->partNames)).begin(); _iter1097 != (*(this->partNames)).end(); ++_iter1097) - { - xfer += oprot->writeString((*_iter1097)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_truncate_table_result::~ThriftHiveMetastore_truncate_table_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_truncate_table_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_truncate_table_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_truncate_table_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_truncate_table_presult::~ThriftHiveMetastore_truncate_table_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_truncate_table_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_tables_args::~ThriftHiveMetastore_get_tables_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_tables_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->pattern); - this->__isset.pattern = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_tables_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->pattern); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_tables_pargs::~ThriftHiveMetastore_get_tables_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_tables_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->pattern))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_tables_result::~ThriftHiveMetastore_get_tables_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_tables_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1098; - ::apache::thrift::protocol::TType _etype1101; - xfer += iprot->readListBegin(_etype1101, _size1098); - this->success.resize(_size1098); - uint32_t _i1102; - for (_i1102 = 0; _i1102 < _size1098; ++_i1102) - { - xfer += iprot->readString(this->success[_i1102]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1103; - for (_iter1103 = this->success.begin(); _iter1103 != this->success.end(); ++_iter1103) - { - xfer += oprot->writeString((*_iter1103)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_tables_presult::~ThriftHiveMetastore_get_tables_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_tables_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1104; - ::apache::thrift::protocol::TType _etype1107; - xfer += iprot->readListBegin(_etype1107, _size1104); - (*(this->success)).resize(_size1104); - uint32_t _i1108; - for (_i1108 = 0; _i1108 < _size1104; ++_i1108) - { - xfer += iprot->readString((*(this->success))[_i1108]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_tables_by_type_args::~ThriftHiveMetastore_get_tables_by_type_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_tables_by_type_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->pattern); - this->__isset.pattern = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tableType); - this->__isset.tableType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_tables_by_type_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->pattern); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->tableType); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_tables_by_type_pargs::~ThriftHiveMetastore_get_tables_by_type_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_tables_by_type_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->pattern))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->tableType))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_tables_by_type_result::~ThriftHiveMetastore_get_tables_by_type_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_tables_by_type_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1109; - ::apache::thrift::protocol::TType _etype1112; - xfer += iprot->readListBegin(_etype1112, _size1109); - this->success.resize(_size1109); - uint32_t _i1113; - for (_i1113 = 0; _i1113 < _size1109; ++_i1113) - { - xfer += iprot->readString(this->success[_i1113]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_tables_by_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1114; - for (_iter1114 = this->success.begin(); _iter1114 != this->success.end(); ++_iter1114) - { - xfer += oprot->writeString((*_iter1114)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_tables_by_type_presult::~ThriftHiveMetastore_get_tables_by_type_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_tables_by_type_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1115; - ::apache::thrift::protocol::TType _etype1118; - xfer += iprot->readListBegin(_etype1118, _size1115); - (*(this->success)).resize(_size1115); - uint32_t _i1119; - for (_i1119 = 0; _i1119 < _size1115; ++_i1119) - { - xfer += iprot->readString((*(this->success))[_i1119]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_table_meta_args::~ThriftHiveMetastore_get_table_meta_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_meta_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_patterns); - this->__isset.db_patterns = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_patterns); - this->__isset.tbl_patterns = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->tbl_types.clear(); - uint32_t _size1120; - ::apache::thrift::protocol::TType _etype1123; - xfer += iprot->readListBegin(_etype1123, _size1120); - this->tbl_types.resize(_size1120); - uint32_t _i1124; - for (_i1124 = 0; _i1124 < _size1120; ++_i1124) - { - xfer += iprot->readString(this->tbl_types[_i1124]); - } - xfer += iprot->readListEnd(); - } - this->__isset.tbl_types = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_meta_args"); - - xfer += oprot->writeFieldBegin("db_patterns", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_patterns); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_patterns", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_patterns); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->tbl_types.size())); - std::vector ::const_iterator _iter1125; - for (_iter1125 = this->tbl_types.begin(); _iter1125 != this->tbl_types.end(); ++_iter1125) - { - xfer += oprot->writeString((*_iter1125)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_meta_pargs::~ThriftHiveMetastore_get_table_meta_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_meta_pargs"); - - xfer += oprot->writeFieldBegin("db_patterns", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_patterns))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_patterns", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_patterns))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->tbl_types)).size())); - std::vector ::const_iterator _iter1126; - for (_iter1126 = (*(this->tbl_types)).begin(); _iter1126 != (*(this->tbl_types)).end(); ++_iter1126) - { - xfer += oprot->writeString((*_iter1126)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_meta_result::~ThriftHiveMetastore_get_table_meta_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_meta_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1127; - ::apache::thrift::protocol::TType _etype1130; - xfer += iprot->readListBegin(_etype1130, _size1127); - this->success.resize(_size1127); - uint32_t _i1131; - for (_i1131 = 0; _i1131 < _size1127; ++_i1131) - { - xfer += this->success[_i1131].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_table_meta_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_meta_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1132; - for (_iter1132 = this->success.begin(); _iter1132 != this->success.end(); ++_iter1132) - { - xfer += (*_iter1132).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_meta_presult::~ThriftHiveMetastore_get_table_meta_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_meta_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1133; - ::apache::thrift::protocol::TType _etype1136; - xfer += iprot->readListBegin(_etype1136, _size1133); - (*(this->success)).resize(_size1133); - uint32_t _i1137; - for (_i1137 = 0; _i1137 < _size1133; ++_i1137) - { - xfer += (*(this->success))[_i1137].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_all_tables_args::~ThriftHiveMetastore_get_all_tables_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_tables_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_all_tables_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_tables_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_all_tables_pargs::~ThriftHiveMetastore_get_all_tables_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_tables_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_tables_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_all_tables_result::~ThriftHiveMetastore_get_all_tables_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_tables_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1138; - ::apache::thrift::protocol::TType _etype1141; - xfer += iprot->readListBegin(_etype1141, _size1138); - this->success.resize(_size1138); - uint32_t _i1142; - for (_i1142 = 0; _i1142 < _size1138; ++_i1142) - { - xfer += iprot->readString(this->success[_i1142]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_tables_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1143; - for (_iter1143 = this->success.begin(); _iter1143 != this->success.end(); ++_iter1143) - { - xfer += oprot->writeString((*_iter1143)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_all_tables_presult::~ThriftHiveMetastore_get_all_tables_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_tables_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1144; - ::apache::thrift::protocol::TType _etype1147; - xfer += iprot->readListBegin(_etype1147, _size1144); - (*(this->success)).resize(_size1144); - uint32_t _i1148; - for (_i1148 = 0; _i1148 < _size1144; ++_i1148) - { - xfer += iprot->readString((*(this->success))[_i1148]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_table_args::~ThriftHiveMetastore_get_table_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbname); - this->__isset.dbname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_table_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_args"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbname); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_pargs::~ThriftHiveMetastore_get_table_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_pargs"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->dbname))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_result::~ThriftHiveMetastore_get_table_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_table_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_presult::~ThriftHiveMetastore_get_table_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_table_objects_by_name_args::~ThriftHiveMetastore_get_table_objects_by_name_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbname); - this->__isset.dbname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->tbl_names.clear(); - uint32_t _size1149; - ::apache::thrift::protocol::TType _etype1152; - xfer += iprot->readListBegin(_etype1152, _size1149); - this->tbl_names.resize(_size1149); - uint32_t _i1153; - for (_i1153 = 0; _i1153 < _size1149; ++_i1153) - { - xfer += iprot->readString(this->tbl_names[_i1153]); - } - xfer += iprot->readListEnd(); - } - this->__isset.tbl_names = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_objects_by_name_args"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbname); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->tbl_names.size())); - std::vector ::const_iterator _iter1154; - for (_iter1154 = this->tbl_names.begin(); _iter1154 != this->tbl_names.end(); ++_iter1154) - { - xfer += oprot->writeString((*_iter1154)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_objects_by_name_pargs::~ThriftHiveMetastore_get_table_objects_by_name_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_objects_by_name_pargs"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->dbname))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->tbl_names)).size())); - std::vector ::const_iterator _iter1155; - for (_iter1155 = (*(this->tbl_names)).begin(); _iter1155 != (*(this->tbl_names)).end(); ++_iter1155) - { - xfer += oprot->writeString((*_iter1155)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_objects_by_name_result::~ThriftHiveMetastore_get_table_objects_by_name_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1156; - ::apache::thrift::protocol::TType _etype1159; - xfer += iprot->readListBegin(_etype1159, _size1156); - this->success.resize(_size1156); - uint32_t _i1160; - for (_i1160 = 0; _i1160 < _size1156; ++_i1160) - { - xfer += this->success[_i1160].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_objects_by_name_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector
::const_iterator _iter1161; - for (_iter1161 = this->success.begin(); _iter1161 != this->success.end(); ++_iter1161) - { - xfer += (*_iter1161).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_objects_by_name_presult::~ThriftHiveMetastore_get_table_objects_by_name_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_objects_by_name_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1162; - ::apache::thrift::protocol::TType _etype1165; - xfer += iprot->readListBegin(_etype1165, _size1162); - (*(this->success)).resize(_size1162); - uint32_t _i1166; - for (_i1166 = 0; _i1166 < _size1162; ++_i1166) - { - xfer += (*(this->success))[_i1166].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_table_req_args::~ThriftHiveMetastore_get_table_req_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_req_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->req.read(iprot); - this->__isset.req = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_table_req_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_req_args"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->req.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_req_pargs::~ThriftHiveMetastore_get_table_req_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_req_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_req_pargs"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->req)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_req_result::~ThriftHiveMetastore_get_table_req_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_req_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_table_req_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_req_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_req_presult::~ThriftHiveMetastore_get_table_req_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_req_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_table_objects_by_name_req_args::~ThriftHiveMetastore_get_table_objects_by_name_req_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_objects_by_name_req_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->req.read(iprot); - this->__isset.req = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_table_objects_by_name_req_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_objects_by_name_req_args"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->req.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_objects_by_name_req_pargs::~ThriftHiveMetastore_get_table_objects_by_name_req_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_objects_by_name_req_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_objects_by_name_req_pargs"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->req)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_objects_by_name_req_result::~ThriftHiveMetastore_get_table_objects_by_name_req_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_objects_by_name_req_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_table_objects_by_name_req_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_objects_by_name_req_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_objects_by_name_req_presult::~ThriftHiveMetastore_get_table_objects_by_name_req_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_objects_by_name_req_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_table_names_by_filter_args::~ThriftHiveMetastore_get_table_names_by_filter_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_names_by_filter_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbname); - this->__isset.dbname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->filter); - this->__isset.filter = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I16) { - xfer += iprot->readI16(this->max_tables); - this->__isset.max_tables = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_table_names_by_filter_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_names_by_filter_args"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbname); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->filter); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_tables", ::apache::thrift::protocol::T_I16, 3); - xfer += oprot->writeI16(this->max_tables); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_names_by_filter_pargs::~ThriftHiveMetastore_get_table_names_by_filter_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_names_by_filter_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_names_by_filter_pargs"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->dbname))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->filter))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_tables", ::apache::thrift::protocol::T_I16, 3); - xfer += oprot->writeI16((*(this->max_tables))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_names_by_filter_result::~ThriftHiveMetastore_get_table_names_by_filter_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1167; - ::apache::thrift::protocol::TType _etype1170; - xfer += iprot->readListBegin(_etype1170, _size1167); - this->success.resize(_size1167); - uint32_t _i1171; - for (_i1171 = 0; _i1171 < _size1167; ++_i1171) - { - xfer += iprot->readString(this->success[_i1171]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_names_by_filter_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1172; - for (_iter1172 = this->success.begin(); _iter1172 != this->success.end(); ++_iter1172) - { - xfer += oprot->writeString((*_iter1172)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_names_by_filter_presult::~ThriftHiveMetastore_get_table_names_by_filter_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_names_by_filter_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1173; - ::apache::thrift::protocol::TType _etype1176; - xfer += iprot->readListBegin(_etype1176, _size1173); - (*(this->success)).resize(_size1173); - uint32_t _i1177; - for (_i1177 = 0; _i1177 < _size1173; ++_i1177) - { - xfer += iprot->readString((*(this->success))[_i1177]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_alter_table_args::~ThriftHiveMetastore_alter_table_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_table_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbname); - this->__isset.dbname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->new_tbl.read(iprot); - this->__isset.new_tbl = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_table_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_args"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbname); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_tbl", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->new_tbl.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_table_pargs::~ThriftHiveMetastore_alter_table_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_table_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_pargs"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->dbname))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_tbl", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += (*(this->new_tbl)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_table_result::~ThriftHiveMetastore_alter_table_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_table_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_table_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_table_presult::~ThriftHiveMetastore_alter_table_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_table_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_alter_table_with_environment_context_args::~ThriftHiveMetastore_alter_table_with_environment_context_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_table_with_environment_context_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbname); - this->__isset.dbname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->new_tbl.read(iprot); - this->__isset.new_tbl = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->environment_context.read(iprot); - this->__isset.environment_context = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_table_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_with_environment_context_args"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbname); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_tbl", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->new_tbl.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->environment_context.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_table_with_environment_context_pargs::~ThriftHiveMetastore_alter_table_with_environment_context_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_table_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_with_environment_context_pargs"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->dbname))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_tbl", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += (*(this->new_tbl)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += (*(this->environment_context)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_table_with_environment_context_result::~ThriftHiveMetastore_alter_table_with_environment_context_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_table_with_environment_context_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_table_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_with_environment_context_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_table_with_environment_context_presult::~ThriftHiveMetastore_alter_table_with_environment_context_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_table_with_environment_context_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_alter_table_with_cascade_args::~ThriftHiveMetastore_alter_table_with_cascade_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_table_with_cascade_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbname); - this->__isset.dbname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->new_tbl.read(iprot); - this->__isset.new_tbl = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->cascade); - this->__isset.cascade = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_table_with_cascade_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_with_cascade_args"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbname); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_tbl", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->new_tbl.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("cascade", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool(this->cascade); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_table_with_cascade_pargs::~ThriftHiveMetastore_alter_table_with_cascade_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_table_with_cascade_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_with_cascade_pargs"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->dbname))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_tbl", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += (*(this->new_tbl)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("cascade", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool((*(this->cascade))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_table_with_cascade_result::~ThriftHiveMetastore_alter_table_with_cascade_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_table_with_cascade_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_table_with_cascade_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_table_with_cascade_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_table_with_cascade_presult::~ThriftHiveMetastore_alter_table_with_cascade_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_table_with_cascade_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_add_partition_args::~ThriftHiveMetastore_add_partition_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partition_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->new_part.read(iprot); - this->__isset.new_part = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_partition_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partition_args"); - - xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->new_part.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_partition_pargs::~ThriftHiveMetastore_add_partition_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partition_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partition_pargs"); - - xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->new_part)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_partition_result::~ThriftHiveMetastore_add_partition_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partition_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_partition_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partition_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_partition_presult::~ThriftHiveMetastore_add_partition_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partition_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_add_partition_with_environment_context_args::~ThriftHiveMetastore_add_partition_with_environment_context_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partition_with_environment_context_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->new_part.read(iprot); - this->__isset.new_part = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->environment_context.read(iprot); - this->__isset.environment_context = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_partition_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partition_with_environment_context_args"); - - xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->new_part.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->environment_context.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_partition_with_environment_context_pargs::~ThriftHiveMetastore_add_partition_with_environment_context_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partition_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partition_with_environment_context_pargs"); - - xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->new_part)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += (*(this->environment_context)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_partition_with_environment_context_result::~ThriftHiveMetastore_add_partition_with_environment_context_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partition_with_environment_context_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_partition_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partition_with_environment_context_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_partition_with_environment_context_presult::~ThriftHiveMetastore_add_partition_with_environment_context_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partition_with_environment_context_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_add_partitions_args::~ThriftHiveMetastore_add_partitions_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partitions_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->new_parts.clear(); - uint32_t _size1178; - ::apache::thrift::protocol::TType _etype1181; - xfer += iprot->readListBegin(_etype1181, _size1178); - this->new_parts.resize(_size1178); - uint32_t _i1182; - for (_i1182 = 0; _i1182 < _size1178; ++_i1182) - { - xfer += this->new_parts[_i1182].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.new_parts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_partitions_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_args"); - - xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->new_parts.size())); - std::vector ::const_iterator _iter1183; - for (_iter1183 = this->new_parts.begin(); _iter1183 != this->new_parts.end(); ++_iter1183) - { - xfer += (*_iter1183).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_partitions_pargs::~ThriftHiveMetastore_add_partitions_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partitions_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_pargs"); - - xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->new_parts)).size())); - std::vector ::const_iterator _iter1184; - for (_iter1184 = (*(this->new_parts)).begin(); _iter1184 != (*(this->new_parts)).end(); ++_iter1184) - { - xfer += (*_iter1184).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_partitions_result::~ThriftHiveMetastore_add_partitions_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partitions_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_partitions_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_I32, 0); - xfer += oprot->writeI32(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_partitions_presult::~ThriftHiveMetastore_add_partitions_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partitions_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_add_partitions_pspec_args::~ThriftHiveMetastore_add_partitions_pspec_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partitions_pspec_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->new_parts.clear(); - uint32_t _size1185; - ::apache::thrift::protocol::TType _etype1188; - xfer += iprot->readListBegin(_etype1188, _size1185); - this->new_parts.resize(_size1185); - uint32_t _i1189; - for (_i1189 = 0; _i1189 < _size1185; ++_i1189) - { - xfer += this->new_parts[_i1189].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.new_parts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_partitions_pspec_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_pspec_args"); - - xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->new_parts.size())); - std::vector ::const_iterator _iter1190; - for (_iter1190 = this->new_parts.begin(); _iter1190 != this->new_parts.end(); ++_iter1190) - { - xfer += (*_iter1190).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_partitions_pspec_pargs::~ThriftHiveMetastore_add_partitions_pspec_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partitions_pspec_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_pspec_pargs"); - - xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->new_parts)).size())); - std::vector ::const_iterator _iter1191; - for (_iter1191 = (*(this->new_parts)).begin(); _iter1191 != (*(this->new_parts)).end(); ++_iter1191) - { - xfer += (*_iter1191).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_partitions_pspec_result::~ThriftHiveMetastore_add_partitions_pspec_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partitions_pspec_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_partitions_pspec_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_pspec_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_I32, 0); - xfer += oprot->writeI32(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_partitions_pspec_presult::~ThriftHiveMetastore_add_partitions_pspec_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partitions_pspec_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_append_partition_args::~ThriftHiveMetastore_append_partition_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_append_partition_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->part_vals.clear(); - uint32_t _size1192; - ::apache::thrift::protocol::TType _etype1195; - xfer += iprot->readListBegin(_etype1195, _size1192); - this->part_vals.resize(_size1192); - uint32_t _i1196; - for (_i1196 = 0; _i1196 < _size1192; ++_i1196) - { - xfer += iprot->readString(this->part_vals[_i1196]); - } - xfer += iprot->readListEnd(); - } - this->__isset.part_vals = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_append_partition_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter1197; - for (_iter1197 = this->part_vals.begin(); _iter1197 != this->part_vals.end(); ++_iter1197) - { - xfer += oprot->writeString((*_iter1197)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_append_partition_pargs::~ThriftHiveMetastore_append_partition_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_append_partition_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter1198; - for (_iter1198 = (*(this->part_vals)).begin(); _iter1198 != (*(this->part_vals)).end(); ++_iter1198) - { - xfer += oprot->writeString((*_iter1198)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_append_partition_result::~ThriftHiveMetastore_append_partition_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_append_partition_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_append_partition_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_append_partition_presult::~ThriftHiveMetastore_append_partition_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_append_partition_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_add_partitions_req_args::~ThriftHiveMetastore_add_partitions_req_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partitions_req_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_partitions_req_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_req_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_partitions_req_pargs::~ThriftHiveMetastore_add_partitions_req_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partitions_req_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_req_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_partitions_req_result::~ThriftHiveMetastore_add_partitions_req_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partitions_req_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_partitions_req_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_partitions_req_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_partitions_req_presult::~ThriftHiveMetastore_add_partitions_req_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_partitions_req_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_append_partition_with_environment_context_args::~ThriftHiveMetastore_append_partition_with_environment_context_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->part_vals.clear(); - uint32_t _size1199; - ::apache::thrift::protocol::TType _etype1202; - xfer += iprot->readListBegin(_etype1202, _size1199); - this->part_vals.resize(_size1199); - uint32_t _i1203; - for (_i1203 = 0; _i1203 < _size1199; ++_i1203) - { - xfer += iprot->readString(this->part_vals[_i1203]); - } - xfer += iprot->readListEnd(); - } - this->__isset.part_vals = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->environment_context.read(iprot); - this->__isset.environment_context = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_with_environment_context_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter1204; - for (_iter1204 = this->part_vals.begin(); _iter1204 != this->part_vals.end(); ++_iter1204) - { - xfer += oprot->writeString((*_iter1204)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->environment_context.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_append_partition_with_environment_context_pargs::~ThriftHiveMetastore_append_partition_with_environment_context_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_append_partition_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_with_environment_context_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter1205; - for (_iter1205 = (*(this->part_vals)).begin(); _iter1205 != (*(this->part_vals)).end(); ++_iter1205) - { - xfer += oprot->writeString((*_iter1205)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += (*(this->environment_context)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_append_partition_with_environment_context_result::~ThriftHiveMetastore_append_partition_with_environment_context_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_append_partition_with_environment_context_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_append_partition_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_with_environment_context_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_append_partition_with_environment_context_presult::~ThriftHiveMetastore_append_partition_with_environment_context_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_append_partition_with_environment_context_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_append_partition_by_name_args::~ThriftHiveMetastore_append_partition_by_name_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_append_partition_by_name_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->part_name); - this->__isset.part_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_append_partition_by_name_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_by_name_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->part_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_append_partition_by_name_pargs::~ThriftHiveMetastore_append_partition_by_name_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_append_partition_by_name_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_by_name_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->part_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_append_partition_by_name_result::~ThriftHiveMetastore_append_partition_by_name_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_append_partition_by_name_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_append_partition_by_name_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_by_name_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_append_partition_by_name_presult::~ThriftHiveMetastore_append_partition_by_name_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_append_partition_by_name_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_append_partition_by_name_with_environment_context_args::~ThriftHiveMetastore_append_partition_by_name_with_environment_context_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_append_partition_by_name_with_environment_context_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->part_name); - this->__isset.part_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->environment_context.read(iprot); - this->__isset.environment_context = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_append_partition_by_name_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_by_name_with_environment_context_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->part_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->environment_context.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs::~ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->part_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += (*(this->environment_context)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_append_partition_by_name_with_environment_context_result::~ThriftHiveMetastore_append_partition_by_name_with_environment_context_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_append_partition_by_name_with_environment_context_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_append_partition_by_name_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_append_partition_by_name_with_environment_context_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult::~ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_drop_partition_args::~ThriftHiveMetastore_drop_partition_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partition_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->part_vals.clear(); - uint32_t _size1206; - ::apache::thrift::protocol::TType _etype1209; - xfer += iprot->readListBegin(_etype1209, _size1206); - this->part_vals.resize(_size1206); - uint32_t _i1210; - for (_i1210 = 0; _i1210 < _size1206; ++_i1210) - { - xfer += iprot->readString(this->part_vals[_i1210]); - } - xfer += iprot->readListEnd(); - } - this->__isset.part_vals = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->deleteData); - this->__isset.deleteData = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_partition_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter1211; - for (_iter1211 = this->part_vals.begin(); _iter1211 != this->part_vals.end(); ++_iter1211) - { - xfer += oprot->writeString((*_iter1211)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool(this->deleteData); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_partition_pargs::~ThriftHiveMetastore_drop_partition_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partition_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter1212; - for (_iter1212 = (*(this->part_vals)).begin(); _iter1212 != (*(this->part_vals)).end(); ++_iter1212) - { - xfer += oprot->writeString((*_iter1212)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool((*(this->deleteData))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_partition_result::~ThriftHiveMetastore_drop_partition_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partition_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_partition_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_partition_presult::~ThriftHiveMetastore_drop_partition_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partition_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_drop_partition_with_environment_context_args::~ThriftHiveMetastore_drop_partition_with_environment_context_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->part_vals.clear(); - uint32_t _size1213; - ::apache::thrift::protocol::TType _etype1216; - xfer += iprot->readListBegin(_etype1216, _size1213); - this->part_vals.resize(_size1213); - uint32_t _i1217; - for (_i1217 = 0; _i1217 < _size1213; ++_i1217) - { - xfer += iprot->readString(this->part_vals[_i1217]); - } - xfer += iprot->readListEnd(); - } - this->__isset.part_vals = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->deleteData); - this->__isset.deleteData = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->environment_context.read(iprot); - this->__isset.environment_context = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_with_environment_context_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter1218; - for (_iter1218 = this->part_vals.begin(); _iter1218 != this->part_vals.end(); ++_iter1218) - { - xfer += oprot->writeString((*_iter1218)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool(this->deleteData); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 5); - xfer += this->environment_context.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_partition_with_environment_context_pargs::~ThriftHiveMetastore_drop_partition_with_environment_context_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_with_environment_context_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter1219; - for (_iter1219 = (*(this->part_vals)).begin(); _iter1219 != (*(this->part_vals)).end(); ++_iter1219) - { - xfer += oprot->writeString((*_iter1219)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool((*(this->deleteData))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 5); - xfer += (*(this->environment_context)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_partition_with_environment_context_result::~ThriftHiveMetastore_drop_partition_with_environment_context_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_with_environment_context_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_partition_with_environment_context_presult::~ThriftHiveMetastore_drop_partition_with_environment_context_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_drop_partition_by_name_args::~ThriftHiveMetastore_drop_partition_by_name_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partition_by_name_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->part_name); - this->__isset.part_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->deleteData); - this->__isset.deleteData = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_partition_by_name_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_by_name_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->part_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool(this->deleteData); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_partition_by_name_pargs::~ThriftHiveMetastore_drop_partition_by_name_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partition_by_name_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_by_name_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->part_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool((*(this->deleteData))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_partition_by_name_result::~ThriftHiveMetastore_drop_partition_by_name_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partition_by_name_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_partition_by_name_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_by_name_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_partition_by_name_presult::~ThriftHiveMetastore_drop_partition_by_name_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partition_by_name_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args::~ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->part_name); - this->__isset.part_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->deleteData); - this->__isset.deleteData = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->environment_context.read(iprot); - this->__isset.environment_context = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->part_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool(this->deleteData); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 5); - xfer += this->environment_context.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs::~ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->part_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool((*(this->deleteData))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 5); - xfer += (*(this->environment_context)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result::~ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult::~ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_drop_partitions_req_args::~ThriftHiveMetastore_drop_partitions_req_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partitions_req_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->req.read(iprot); - this->__isset.req = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_partitions_req_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partitions_req_args"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->req.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_partitions_req_pargs::~ThriftHiveMetastore_drop_partitions_req_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partitions_req_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partitions_req_pargs"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->req)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_partitions_req_result::~ThriftHiveMetastore_drop_partitions_req_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partitions_req_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_partitions_req_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_partitions_req_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_partitions_req_presult::~ThriftHiveMetastore_drop_partitions_req_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_partitions_req_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_partition_args::~ThriftHiveMetastore_get_partition_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->part_vals.clear(); - uint32_t _size1220; - ::apache::thrift::protocol::TType _etype1223; - xfer += iprot->readListBegin(_etype1223, _size1220); - this->part_vals.resize(_size1220); - uint32_t _i1224; - for (_i1224 = 0; _i1224 < _size1220; ++_i1224) - { - xfer += iprot->readString(this->part_vals[_i1224]); - } - xfer += iprot->readListEnd(); - } - this->__isset.part_vals = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partition_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter1225; - for (_iter1225 = this->part_vals.begin(); _iter1225 != this->part_vals.end(); ++_iter1225) - { - xfer += oprot->writeString((*_iter1225)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_pargs::~ThriftHiveMetastore_get_partition_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter1226; - for (_iter1226 = (*(this->part_vals)).begin(); _iter1226 != (*(this->part_vals)).end(); ++_iter1226) - { - xfer += oprot->writeString((*_iter1226)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_result::~ThriftHiveMetastore_get_partition_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partition_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_presult::~ThriftHiveMetastore_get_partition_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_exchange_partition_args::~ThriftHiveMetastore_exchange_partition_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_exchange_partition_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->partitionSpecs.clear(); - uint32_t _size1227; - ::apache::thrift::protocol::TType _ktype1228; - ::apache::thrift::protocol::TType _vtype1229; - xfer += iprot->readMapBegin(_ktype1228, _vtype1229, _size1227); - uint32_t _i1231; - for (_i1231 = 0; _i1231 < _size1227; ++_i1231) - { - std::string _key1232; - xfer += iprot->readString(_key1232); - std::string& _val1233 = this->partitionSpecs[_key1232]; - xfer += iprot->readString(_val1233); - } - xfer += iprot->readMapEnd(); - } - this->__isset.partitionSpecs = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->source_db); - this->__isset.source_db = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->source_table_name); - this->__isset.source_table_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dest_db); - this->__isset.dest_db = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dest_table_name); - this->__isset.dest_table_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_exchange_partition_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_exchange_partition_args"); - - xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->partitionSpecs.size())); - std::map ::const_iterator _iter1234; - for (_iter1234 = this->partitionSpecs.begin(); _iter1234 != this->partitionSpecs.end(); ++_iter1234) - { - xfer += oprot->writeString(_iter1234->first); - xfer += oprot->writeString(_iter1234->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("source_db", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->source_db); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("source_table_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->source_table_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dest_db", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->dest_db); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dest_table_name", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString(this->dest_table_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_exchange_partition_pargs::~ThriftHiveMetastore_exchange_partition_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_exchange_partition_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_exchange_partition_pargs"); - - xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast((*(this->partitionSpecs)).size())); - std::map ::const_iterator _iter1235; - for (_iter1235 = (*(this->partitionSpecs)).begin(); _iter1235 != (*(this->partitionSpecs)).end(); ++_iter1235) - { - xfer += oprot->writeString(_iter1235->first); - xfer += oprot->writeString(_iter1235->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("source_db", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->source_db))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("source_table_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->source_table_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dest_db", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString((*(this->dest_db))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dest_table_name", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString((*(this->dest_table_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_exchange_partition_result::~ThriftHiveMetastore_exchange_partition_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_exchange_partition_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_exchange_partition_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_exchange_partition_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o4) { - xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->o4.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_exchange_partition_presult::~ThriftHiveMetastore_exchange_partition_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_exchange_partition_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_exchange_partitions_args::~ThriftHiveMetastore_exchange_partitions_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_exchange_partitions_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->partitionSpecs.clear(); - uint32_t _size1236; - ::apache::thrift::protocol::TType _ktype1237; - ::apache::thrift::protocol::TType _vtype1238; - xfer += iprot->readMapBegin(_ktype1237, _vtype1238, _size1236); - uint32_t _i1240; - for (_i1240 = 0; _i1240 < _size1236; ++_i1240) - { - std::string _key1241; - xfer += iprot->readString(_key1241); - std::string& _val1242 = this->partitionSpecs[_key1241]; - xfer += iprot->readString(_val1242); - } - xfer += iprot->readMapEnd(); - } - this->__isset.partitionSpecs = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->source_db); - this->__isset.source_db = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->source_table_name); - this->__isset.source_table_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dest_db); - this->__isset.dest_db = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dest_table_name); - this->__isset.dest_table_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_exchange_partitions_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_exchange_partitions_args"); - - xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->partitionSpecs.size())); - std::map ::const_iterator _iter1243; - for (_iter1243 = this->partitionSpecs.begin(); _iter1243 != this->partitionSpecs.end(); ++_iter1243) - { - xfer += oprot->writeString(_iter1243->first); - xfer += oprot->writeString(_iter1243->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("source_db", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->source_db); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("source_table_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->source_table_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dest_db", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->dest_db); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dest_table_name", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString(this->dest_table_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_exchange_partitions_pargs::~ThriftHiveMetastore_exchange_partitions_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_exchange_partitions_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_exchange_partitions_pargs"); - - xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast((*(this->partitionSpecs)).size())); - std::map ::const_iterator _iter1244; - for (_iter1244 = (*(this->partitionSpecs)).begin(); _iter1244 != (*(this->partitionSpecs)).end(); ++_iter1244) - { - xfer += oprot->writeString(_iter1244->first); - xfer += oprot->writeString(_iter1244->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("source_db", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->source_db))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("source_table_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->source_table_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dest_db", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString((*(this->dest_db))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dest_table_name", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString((*(this->dest_table_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_exchange_partitions_result::~ThriftHiveMetastore_exchange_partitions_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_exchange_partitions_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1245; - ::apache::thrift::protocol::TType _etype1248; - xfer += iprot->readListBegin(_etype1248, _size1245); - this->success.resize(_size1245); - uint32_t _i1249; - for (_i1249 = 0; _i1249 < _size1245; ++_i1249) - { - xfer += this->success[_i1249].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_exchange_partitions_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_exchange_partitions_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1250; - for (_iter1250 = this->success.begin(); _iter1250 != this->success.end(); ++_iter1250) - { - xfer += (*_iter1250).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o4) { - xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->o4.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_exchange_partitions_presult::~ThriftHiveMetastore_exchange_partitions_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_exchange_partitions_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1251; - ::apache::thrift::protocol::TType _etype1254; - xfer += iprot->readListBegin(_etype1254, _size1251); - (*(this->success)).resize(_size1251); - uint32_t _i1255; - for (_i1255 = 0; _i1255 < _size1251; ++_i1255) - { - xfer += (*(this->success))[_i1255].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_partition_with_auth_args::~ThriftHiveMetastore_get_partition_with_auth_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_with_auth_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->part_vals.clear(); - uint32_t _size1256; - ::apache::thrift::protocol::TType _etype1259; - xfer += iprot->readListBegin(_etype1259, _size1256); - this->part_vals.resize(_size1256); - uint32_t _i1260; - for (_i1260 = 0; _i1260 < _size1256; ++_i1260) - { - xfer += iprot->readString(this->part_vals[_i1260]); - } - xfer += iprot->readListEnd(); - } - this->__isset.part_vals = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->user_name); - this->__isset.user_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->group_names.clear(); - uint32_t _size1261; - ::apache::thrift::protocol::TType _etype1264; - xfer += iprot->readListBegin(_etype1264, _size1261); - this->group_names.resize(_size1261); - uint32_t _i1265; - for (_i1265 = 0; _i1265 < _size1261; ++_i1265) - { - xfer += iprot->readString(this->group_names[_i1265]); - } - xfer += iprot->readListEnd(); - } - this->__isset.group_names = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_with_auth_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter1266; - for (_iter1266 = this->part_vals.begin(); _iter1266 != this->part_vals.end(); ++_iter1266) - { - xfer += oprot->writeString((*_iter1266)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->user_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); - std::vector ::const_iterator _iter1267; - for (_iter1267 = this->group_names.begin(); _iter1267 != this->group_names.end(); ++_iter1267) - { - xfer += oprot->writeString((*_iter1267)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_with_auth_pargs::~ThriftHiveMetastore_get_partition_with_auth_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_with_auth_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter1268; - for (_iter1268 = (*(this->part_vals)).begin(); _iter1268 != (*(this->part_vals)).end(); ++_iter1268) - { - xfer += oprot->writeString((*_iter1268)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString((*(this->user_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); - std::vector ::const_iterator _iter1269; - for (_iter1269 = (*(this->group_names)).begin(); _iter1269 != (*(this->group_names)).end(); ++_iter1269) - { - xfer += oprot->writeString((*_iter1269)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_with_auth_result::~ThriftHiveMetastore_get_partition_with_auth_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_with_auth_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partition_with_auth_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_with_auth_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_with_auth_presult::~ThriftHiveMetastore_get_partition_with_auth_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_with_auth_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_partition_by_name_args::~ThriftHiveMetastore_get_partition_by_name_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_by_name_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->part_name); - this->__isset.part_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partition_by_name_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_by_name_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->part_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_by_name_pargs::~ThriftHiveMetastore_get_partition_by_name_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_by_name_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_by_name_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->part_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_by_name_result::~ThriftHiveMetastore_get_partition_by_name_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_by_name_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partition_by_name_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_by_name_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_by_name_presult::~ThriftHiveMetastore_get_partition_by_name_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_by_name_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_partitions_args::~ThriftHiveMetastore_get_partitions_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I16) { - xfer += iprot->readI16(this->max_parts); - this->__isset.max_parts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 3); - xfer += oprot->writeI16(this->max_parts); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_pargs::~ThriftHiveMetastore_get_partitions_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 3); - xfer += oprot->writeI16((*(this->max_parts))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_result::~ThriftHiveMetastore_get_partitions_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1270; - ::apache::thrift::protocol::TType _etype1273; - xfer += iprot->readListBegin(_etype1273, _size1270); - this->success.resize(_size1270); - uint32_t _i1274; - for (_i1274 = 0; _i1274 < _size1270; ++_i1274) - { - xfer += this->success[_i1274].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1275; - for (_iter1275 = this->success.begin(); _iter1275 != this->success.end(); ++_iter1275) - { - xfer += (*_iter1275).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_presult::~ThriftHiveMetastore_get_partitions_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1276; - ::apache::thrift::protocol::TType _etype1279; - xfer += iprot->readListBegin(_etype1279, _size1276); - (*(this->success)).resize(_size1276); - uint32_t _i1280; - for (_i1280 = 0; _i1280 < _size1276; ++_i1280) - { - xfer += (*(this->success))[_i1280].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_partitions_with_auth_args::~ThriftHiveMetastore_get_partitions_with_auth_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I16) { - xfer += iprot->readI16(this->max_parts); - this->__isset.max_parts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->user_name); - this->__isset.user_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->group_names.clear(); - uint32_t _size1281; - ::apache::thrift::protocol::TType _etype1284; - xfer += iprot->readListBegin(_etype1284, _size1281); - this->group_names.resize(_size1281); - uint32_t _i1285; - for (_i1285 = 0; _i1285 < _size1281; ++_i1285) - { - xfer += iprot->readString(this->group_names[_i1285]); - } - xfer += iprot->readListEnd(); - } - this->__isset.group_names = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_with_auth_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 3); - xfer += oprot->writeI16(this->max_parts); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->user_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); - std::vector ::const_iterator _iter1286; - for (_iter1286 = this->group_names.begin(); _iter1286 != this->group_names.end(); ++_iter1286) - { - xfer += oprot->writeString((*_iter1286)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_with_auth_pargs::~ThriftHiveMetastore_get_partitions_with_auth_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_with_auth_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_with_auth_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 3); - xfer += oprot->writeI16((*(this->max_parts))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString((*(this->user_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); - std::vector ::const_iterator _iter1287; - for (_iter1287 = (*(this->group_names)).begin(); _iter1287 != (*(this->group_names)).end(); ++_iter1287) - { - xfer += oprot->writeString((*_iter1287)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_with_auth_result::~ThriftHiveMetastore_get_partitions_with_auth_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1288; - ::apache::thrift::protocol::TType _etype1291; - xfer += iprot->readListBegin(_etype1291, _size1288); - this->success.resize(_size1288); - uint32_t _i1292; - for (_i1292 = 0; _i1292 < _size1288; ++_i1292) - { - xfer += this->success[_i1292].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_with_auth_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1293; - for (_iter1293 = this->success.begin(); _iter1293 != this->success.end(); ++_iter1293) - { - xfer += (*_iter1293).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_with_auth_presult::~ThriftHiveMetastore_get_partitions_with_auth_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1294; - ::apache::thrift::protocol::TType _etype1297; - xfer += iprot->readListBegin(_etype1297, _size1294); - (*(this->success)).resize(_size1294); - uint32_t _i1298; - for (_i1298 = 0; _i1298 < _size1294; ++_i1298) - { - xfer += (*(this->success))[_i1298].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_partitions_pspec_args::~ThriftHiveMetastore_get_partitions_pspec_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_pspec_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->max_parts); - this->__isset.max_parts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_pspec_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_pspec_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32(this->max_parts); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_pspec_pargs::~ThriftHiveMetastore_get_partitions_pspec_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_pspec_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_pspec_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32((*(this->max_parts))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_pspec_result::~ThriftHiveMetastore_get_partitions_pspec_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_pspec_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1299; - ::apache::thrift::protocol::TType _etype1302; - xfer += iprot->readListBegin(_etype1302, _size1299); - this->success.resize(_size1299); - uint32_t _i1303; - for (_i1303 = 0; _i1303 < _size1299; ++_i1303) - { - xfer += this->success[_i1303].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_pspec_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_pspec_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1304; - for (_iter1304 = this->success.begin(); _iter1304 != this->success.end(); ++_iter1304) - { - xfer += (*_iter1304).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_pspec_presult::~ThriftHiveMetastore_get_partitions_pspec_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1305; - ::apache::thrift::protocol::TType _etype1308; - xfer += iprot->readListBegin(_etype1308, _size1305); - (*(this->success)).resize(_size1305); - uint32_t _i1309; - for (_i1309 = 0; _i1309 < _size1305; ++_i1309) - { - xfer += (*(this->success))[_i1309].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_partition_names_args::~ThriftHiveMetastore_get_partition_names_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_names_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I16) { - xfer += iprot->readI16(this->max_parts); - this->__isset.max_parts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partition_names_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_names_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 3); - xfer += oprot->writeI16(this->max_parts); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_names_pargs::~ThriftHiveMetastore_get_partition_names_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_names_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_names_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 3); - xfer += oprot->writeI16((*(this->max_parts))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_names_result::~ThriftHiveMetastore_get_partition_names_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_names_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1310; - ::apache::thrift::protocol::TType _etype1313; - xfer += iprot->readListBegin(_etype1313, _size1310); - this->success.resize(_size1310); - uint32_t _i1314; - for (_i1314 = 0; _i1314 < _size1310; ++_i1314) - { - xfer += iprot->readString(this->success[_i1314]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partition_names_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_names_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1315; - for (_iter1315 = this->success.begin(); _iter1315 != this->success.end(); ++_iter1315) - { - xfer += oprot->writeString((*_iter1315)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_names_presult::~ThriftHiveMetastore_get_partition_names_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_names_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1316; - ::apache::thrift::protocol::TType _etype1319; - xfer += iprot->readListBegin(_etype1319, _size1316); - (*(this->success)).resize(_size1316); - uint32_t _i1320; - for (_i1320 = 0; _i1320 < _size1316; ++_i1320) - { - xfer += iprot->readString((*(this->success))[_i1320]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_partition_values_args::~ThriftHiveMetastore_get_partition_values_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_values_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partition_values_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_values_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_values_pargs::~ThriftHiveMetastore_get_partition_values_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_values_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_values_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_values_result::~ThriftHiveMetastore_get_partition_values_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_values_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partition_values_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_values_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_values_presult::~ThriftHiveMetastore_get_partition_values_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_values_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_partitions_ps_args::~ThriftHiveMetastore_get_partitions_ps_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_ps_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->part_vals.clear(); - uint32_t _size1321; - ::apache::thrift::protocol::TType _etype1324; - xfer += iprot->readListBegin(_etype1324, _size1321); - this->part_vals.resize(_size1321); - uint32_t _i1325; - for (_i1325 = 0; _i1325 < _size1321; ++_i1325) - { - xfer += iprot->readString(this->part_vals[_i1325]); - } - xfer += iprot->readListEnd(); - } - this->__isset.part_vals = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I16) { - xfer += iprot->readI16(this->max_parts); - this->__isset.max_parts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_ps_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_ps_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter1326; - for (_iter1326 = this->part_vals.begin(); _iter1326 != this->part_vals.end(); ++_iter1326) - { - xfer += oprot->writeString((*_iter1326)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4); - xfer += oprot->writeI16(this->max_parts); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_ps_pargs::~ThriftHiveMetastore_get_partitions_ps_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_ps_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_ps_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter1327; - for (_iter1327 = (*(this->part_vals)).begin(); _iter1327 != (*(this->part_vals)).end(); ++_iter1327) - { - xfer += oprot->writeString((*_iter1327)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4); - xfer += oprot->writeI16((*(this->max_parts))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_ps_result::~ThriftHiveMetastore_get_partitions_ps_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_ps_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1328; - ::apache::thrift::protocol::TType _etype1331; - xfer += iprot->readListBegin(_etype1331, _size1328); - this->success.resize(_size1328); - uint32_t _i1332; - for (_i1332 = 0; _i1332 < _size1328; ++_i1332) - { - xfer += this->success[_i1332].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_ps_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_ps_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1333; - for (_iter1333 = this->success.begin(); _iter1333 != this->success.end(); ++_iter1333) - { - xfer += (*_iter1333).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_ps_presult::~ThriftHiveMetastore_get_partitions_ps_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_ps_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1334; - ::apache::thrift::protocol::TType _etype1337; - xfer += iprot->readListBegin(_etype1337, _size1334); - (*(this->success)).resize(_size1334); - uint32_t _i1338; - for (_i1338 = 0; _i1338 < _size1334; ++_i1338) - { - xfer += (*(this->success))[_i1338].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_partitions_ps_with_auth_args::~ThriftHiveMetastore_get_partitions_ps_with_auth_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->part_vals.clear(); - uint32_t _size1339; - ::apache::thrift::protocol::TType _etype1342; - xfer += iprot->readListBegin(_etype1342, _size1339); - this->part_vals.resize(_size1339); - uint32_t _i1343; - for (_i1343 = 0; _i1343 < _size1339; ++_i1343) - { - xfer += iprot->readString(this->part_vals[_i1343]); - } - xfer += iprot->readListEnd(); - } - this->__isset.part_vals = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I16) { - xfer += iprot->readI16(this->max_parts); - this->__isset.max_parts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->user_name); - this->__isset.user_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->group_names.clear(); - uint32_t _size1344; - ::apache::thrift::protocol::TType _etype1347; - xfer += iprot->readListBegin(_etype1347, _size1344); - this->group_names.resize(_size1344); - uint32_t _i1348; - for (_i1348 = 0; _i1348 < _size1344; ++_i1348) - { - xfer += iprot->readString(this->group_names[_i1348]); - } - xfer += iprot->readListEnd(); - } - this->__isset.group_names = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_ps_with_auth_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter1349; - for (_iter1349 = this->part_vals.begin(); _iter1349 != this->part_vals.end(); ++_iter1349) - { - xfer += oprot->writeString((*_iter1349)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4); - xfer += oprot->writeI16(this->max_parts); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString(this->user_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); - std::vector ::const_iterator _iter1350; - for (_iter1350 = this->group_names.begin(); _iter1350 != this->group_names.end(); ++_iter1350) - { - xfer += oprot->writeString((*_iter1350)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::~ThriftHiveMetastore_get_partitions_ps_with_auth_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_ps_with_auth_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter1351; - for (_iter1351 = (*(this->part_vals)).begin(); _iter1351 != (*(this->part_vals)).end(); ++_iter1351) - { - xfer += oprot->writeString((*_iter1351)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4); - xfer += oprot->writeI16((*(this->max_parts))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString((*(this->user_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); - std::vector ::const_iterator _iter1352; - for (_iter1352 = (*(this->group_names)).begin(); _iter1352 != (*(this->group_names)).end(); ++_iter1352) - { - xfer += oprot->writeString((*_iter1352)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_ps_with_auth_result::~ThriftHiveMetastore_get_partitions_ps_with_auth_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1353; - ::apache::thrift::protocol::TType _etype1356; - xfer += iprot->readListBegin(_etype1356, _size1353); - this->success.resize(_size1353); - uint32_t _i1357; - for (_i1357 = 0; _i1357 < _size1353; ++_i1357) - { - xfer += this->success[_i1357].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_ps_with_auth_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1358; - for (_iter1358 = this->success.begin(); _iter1358 != this->success.end(); ++_iter1358) - { - xfer += (*_iter1358).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_ps_with_auth_presult::~ThriftHiveMetastore_get_partitions_ps_with_auth_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1359; - ::apache::thrift::protocol::TType _etype1362; - xfer += iprot->readListBegin(_etype1362, _size1359); - (*(this->success)).resize(_size1359); - uint32_t _i1363; - for (_i1363 = 0; _i1363 < _size1359; ++_i1363) - { - xfer += (*(this->success))[_i1363].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_partition_names_ps_args::~ThriftHiveMetastore_get_partition_names_ps_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_names_ps_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->part_vals.clear(); - uint32_t _size1364; - ::apache::thrift::protocol::TType _etype1367; - xfer += iprot->readListBegin(_etype1367, _size1364); - this->part_vals.resize(_size1364); - uint32_t _i1368; - for (_i1368 = 0; _i1368 < _size1364; ++_i1368) - { - xfer += iprot->readString(this->part_vals[_i1368]); - } - xfer += iprot->readListEnd(); - } - this->__isset.part_vals = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I16) { - xfer += iprot->readI16(this->max_parts); - this->__isset.max_parts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partition_names_ps_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_names_ps_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter1369; - for (_iter1369 = this->part_vals.begin(); _iter1369 != this->part_vals.end(); ++_iter1369) - { - xfer += oprot->writeString((*_iter1369)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4); - xfer += oprot->writeI16(this->max_parts); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_names_ps_pargs::~ThriftHiveMetastore_get_partition_names_ps_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_names_ps_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_names_ps_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter1370; - for (_iter1370 = (*(this->part_vals)).begin(); _iter1370 != (*(this->part_vals)).end(); ++_iter1370) - { - xfer += oprot->writeString((*_iter1370)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4); - xfer += oprot->writeI16((*(this->max_parts))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_names_ps_result::~ThriftHiveMetastore_get_partition_names_ps_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_names_ps_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1371; - ::apache::thrift::protocol::TType _etype1374; - xfer += iprot->readListBegin(_etype1374, _size1371); - this->success.resize(_size1371); - uint32_t _i1375; - for (_i1375 = 0; _i1375 < _size1371; ++_i1375) - { - xfer += iprot->readString(this->success[_i1375]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partition_names_ps_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_names_ps_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1376; - for (_iter1376 = this->success.begin(); _iter1376 != this->success.end(); ++_iter1376) - { - xfer += oprot->writeString((*_iter1376)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_names_ps_presult::~ThriftHiveMetastore_get_partition_names_ps_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_names_ps_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1377; - ::apache::thrift::protocol::TType _etype1380; - xfer += iprot->readListBegin(_etype1380, _size1377); - (*(this->success)).resize(_size1377); - uint32_t _i1381; - for (_i1381 = 0; _i1381 < _size1377; ++_i1381) - { - xfer += iprot->readString((*(this->success))[_i1381]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_partitions_by_filter_args::~ThriftHiveMetastore_get_partitions_by_filter_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_by_filter_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->filter); - this->__isset.filter = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I16) { - xfer += iprot->readI16(this->max_parts); - this->__isset.max_parts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_by_filter_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_filter_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->filter); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4); - xfer += oprot->writeI16(this->max_parts); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_by_filter_pargs::~ThriftHiveMetastore_get_partitions_by_filter_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_by_filter_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_filter_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->filter))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I16, 4); - xfer += oprot->writeI16((*(this->max_parts))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_by_filter_result::~ThriftHiveMetastore_get_partitions_by_filter_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_by_filter_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1382; - ::apache::thrift::protocol::TType _etype1385; - xfer += iprot->readListBegin(_etype1385, _size1382); - this->success.resize(_size1382); - uint32_t _i1386; - for (_i1386 = 0; _i1386 < _size1382; ++_i1386) - { - xfer += this->success[_i1386].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_by_filter_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_filter_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1387; - for (_iter1387 = this->success.begin(); _iter1387 != this->success.end(); ++_iter1387) - { - xfer += (*_iter1387).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_by_filter_presult::~ThriftHiveMetastore_get_partitions_by_filter_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_by_filter_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1388; - ::apache::thrift::protocol::TType _etype1391; - xfer += iprot->readListBegin(_etype1391, _size1388); - (*(this->success)).resize(_size1388); - uint32_t _i1392; - for (_i1392 = 0; _i1392 < _size1388; ++_i1392) - { - xfer += (*(this->success))[_i1392].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_part_specs_by_filter_args::~ThriftHiveMetastore_get_part_specs_by_filter_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_part_specs_by_filter_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->filter); - this->__isset.filter = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->max_parts); - this->__isset.max_parts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_part_specs_by_filter_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_part_specs_by_filter_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->filter); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32(this->max_parts); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_part_specs_by_filter_pargs::~ThriftHiveMetastore_get_part_specs_by_filter_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_part_specs_by_filter_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_part_specs_by_filter_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->filter))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_parts", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32((*(this->max_parts))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_part_specs_by_filter_result::~ThriftHiveMetastore_get_part_specs_by_filter_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_part_specs_by_filter_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1393; - ::apache::thrift::protocol::TType _etype1396; - xfer += iprot->readListBegin(_etype1396, _size1393); - this->success.resize(_size1393); - uint32_t _i1397; - for (_i1397 = 0; _i1397 < _size1393; ++_i1397) - { - xfer += this->success[_i1397].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_part_specs_by_filter_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_part_specs_by_filter_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1398; - for (_iter1398 = this->success.begin(); _iter1398 != this->success.end(); ++_iter1398) - { - xfer += (*_iter1398).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_part_specs_by_filter_presult::~ThriftHiveMetastore_get_part_specs_by_filter_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_part_specs_by_filter_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1399; - ::apache::thrift::protocol::TType _etype1402; - xfer += iprot->readListBegin(_etype1402, _size1399); - (*(this->success)).resize(_size1399); - uint32_t _i1403; - for (_i1403 = 0; _i1403 < _size1399; ++_i1403) - { - xfer += (*(this->success))[_i1403].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_partitions_by_expr_args::~ThriftHiveMetastore_get_partitions_by_expr_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_by_expr_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->req.read(iprot); - this->__isset.req = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_by_expr_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_expr_args"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->req.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_by_expr_pargs::~ThriftHiveMetastore_get_partitions_by_expr_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_by_expr_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_expr_pargs"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->req)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_by_expr_result::~ThriftHiveMetastore_get_partitions_by_expr_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_by_expr_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_by_expr_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_expr_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_by_expr_presult::~ThriftHiveMetastore_get_partitions_by_expr_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_by_expr_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_num_partitions_by_filter_args::~ThriftHiveMetastore_get_num_partitions_by_filter_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_num_partitions_by_filter_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->filter); - this->__isset.filter = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_num_partitions_by_filter_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_num_partitions_by_filter_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->filter); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_num_partitions_by_filter_pargs::~ThriftHiveMetastore_get_num_partitions_by_filter_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_num_partitions_by_filter_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_num_partitions_by_filter_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->filter))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_num_partitions_by_filter_result::~ThriftHiveMetastore_get_num_partitions_by_filter_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_num_partitions_by_filter_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_num_partitions_by_filter_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_num_partitions_by_filter_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_I32, 0); - xfer += oprot->writeI32(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_num_partitions_by_filter_presult::~ThriftHiveMetastore_get_num_partitions_by_filter_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_num_partitions_by_filter_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_partitions_by_names_args::~ThriftHiveMetastore_get_partitions_by_names_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_by_names_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->names.clear(); - uint32_t _size1404; - ::apache::thrift::protocol::TType _etype1407; - xfer += iprot->readListBegin(_etype1407, _size1404); - this->names.resize(_size1404); - uint32_t _i1408; - for (_i1408 = 0; _i1408 < _size1404; ++_i1408) - { - xfer += iprot->readString(this->names[_i1408]); - } - xfer += iprot->readListEnd(); - } - this->__isset.names = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_by_names_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_names_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->names.size())); - std::vector ::const_iterator _iter1409; - for (_iter1409 = this->names.begin(); _iter1409 != this->names.end(); ++_iter1409) - { - xfer += oprot->writeString((*_iter1409)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_by_names_pargs::~ThriftHiveMetastore_get_partitions_by_names_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_by_names_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_names_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->names)).size())); - std::vector ::const_iterator _iter1410; - for (_iter1410 = (*(this->names)).begin(); _iter1410 != (*(this->names)).end(); ++_iter1410) - { - xfer += oprot->writeString((*_iter1410)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_by_names_result::~ThriftHiveMetastore_get_partitions_by_names_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_by_names_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1411; - ::apache::thrift::protocol::TType _etype1414; - xfer += iprot->readListBegin(_etype1414, _size1411); - this->success.resize(_size1411); - uint32_t _i1415; - for (_i1415 = 0; _i1415 < _size1411; ++_i1415) - { - xfer += this->success[_i1415].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_by_names_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_by_names_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1416; - for (_iter1416 = this->success.begin(); _iter1416 != this->success.end(); ++_iter1416) - { - xfer += (*_iter1416).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_by_names_presult::~ThriftHiveMetastore_get_partitions_by_names_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_by_names_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1417; - ::apache::thrift::protocol::TType _etype1420; - xfer += iprot->readListBegin(_etype1420, _size1417); - (*(this->success)).resize(_size1417); - uint32_t _i1421; - for (_i1421 = 0; _i1421 < _size1417; ++_i1421) - { - xfer += (*(this->success))[_i1421].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_alter_partition_args::~ThriftHiveMetastore_alter_partition_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_partition_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->new_part.read(iprot); - this->__isset.new_part = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_partition_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partition_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->new_part.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_partition_pargs::~ThriftHiveMetastore_alter_partition_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_partition_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partition_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += (*(this->new_part)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_partition_result::~ThriftHiveMetastore_alter_partition_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_partition_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_partition_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partition_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_partition_presult::~ThriftHiveMetastore_alter_partition_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_partition_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_alter_partitions_args::~ThriftHiveMetastore_alter_partitions_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_partitions_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->new_parts.clear(); - uint32_t _size1422; - ::apache::thrift::protocol::TType _etype1425; - xfer += iprot->readListBegin(_etype1425, _size1422); - this->new_parts.resize(_size1422); - uint32_t _i1426; - for (_i1426 = 0; _i1426 < _size1422; ++_i1426) - { - xfer += this->new_parts[_i1426].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.new_parts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_partitions_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partitions_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->new_parts.size())); - std::vector ::const_iterator _iter1427; - for (_iter1427 = this->new_parts.begin(); _iter1427 != this->new_parts.end(); ++_iter1427) - { - xfer += (*_iter1427).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_partitions_pargs::~ThriftHiveMetastore_alter_partitions_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_partitions_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partitions_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->new_parts)).size())); - std::vector ::const_iterator _iter1428; - for (_iter1428 = (*(this->new_parts)).begin(); _iter1428 != (*(this->new_parts)).end(); ++_iter1428) - { - xfer += (*_iter1428).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_partitions_result::~ThriftHiveMetastore_alter_partitions_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_partitions_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_partitions_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partitions_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_partitions_presult::~ThriftHiveMetastore_alter_partitions_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_partitions_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_alter_partitions_with_environment_context_args::~ThriftHiveMetastore_alter_partitions_with_environment_context_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->new_parts.clear(); - uint32_t _size1429; - ::apache::thrift::protocol::TType _etype1432; - xfer += iprot->readListBegin(_etype1432, _size1429); - this->new_parts.resize(_size1429); - uint32_t _i1433; - for (_i1433 = 0; _i1433 < _size1429; ++_i1433) - { - xfer += this->new_parts[_i1433].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.new_parts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->environment_context.read(iprot); - this->__isset.environment_context = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partitions_with_environment_context_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->new_parts.size())); - std::vector ::const_iterator _iter1434; - for (_iter1434 = this->new_parts.begin(); _iter1434 != this->new_parts.end(); ++_iter1434) - { - xfer += (*_iter1434).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->environment_context.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_partitions_with_environment_context_pargs::~ThriftHiveMetastore_alter_partitions_with_environment_context_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partitions_with_environment_context_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast((*(this->new_parts)).size())); - std::vector ::const_iterator _iter1435; - for (_iter1435 = (*(this->new_parts)).begin(); _iter1435 != (*(this->new_parts)).end(); ++_iter1435) - { - xfer += (*_iter1435).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += (*(this->environment_context)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_partitions_with_environment_context_result::~ThriftHiveMetastore_alter_partitions_with_environment_context_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partitions_with_environment_context_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_partitions_with_environment_context_presult::~ThriftHiveMetastore_alter_partitions_with_environment_context_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_alter_partition_with_environment_context_args::~ThriftHiveMetastore_alter_partition_with_environment_context_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_partition_with_environment_context_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->new_part.read(iprot); - this->__isset.new_part = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->environment_context.read(iprot); - this->__isset.environment_context = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_partition_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partition_with_environment_context_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->new_part.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->environment_context.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_partition_with_environment_context_pargs::~ThriftHiveMetastore_alter_partition_with_environment_context_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_partition_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partition_with_environment_context_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += (*(this->new_part)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += (*(this->environment_context)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_partition_with_environment_context_result::~ThriftHiveMetastore_alter_partition_with_environment_context_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_partition_with_environment_context_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_partition_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_partition_with_environment_context_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_partition_with_environment_context_presult::~ThriftHiveMetastore_alter_partition_with_environment_context_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_partition_with_environment_context_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_rename_partition_args::~ThriftHiveMetastore_rename_partition_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_rename_partition_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->part_vals.clear(); - uint32_t _size1436; - ::apache::thrift::protocol::TType _etype1439; - xfer += iprot->readListBegin(_etype1439, _size1436); - this->part_vals.resize(_size1436); - uint32_t _i1440; - for (_i1440 = 0; _i1440 < _size1436; ++_i1440) - { - xfer += iprot->readString(this->part_vals[_i1440]); - } - xfer += iprot->readListEnd(); - } - this->__isset.part_vals = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->new_part.read(iprot); - this->__isset.new_part = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_rename_partition_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_rename_partition_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter1441; - for (_iter1441 = this->part_vals.begin(); _iter1441 != this->part_vals.end(); ++_iter1441) - { - xfer += oprot->writeString((*_iter1441)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->new_part.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_rename_partition_pargs::~ThriftHiveMetastore_rename_partition_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_rename_partition_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_rename_partition_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter1442; - for (_iter1442 = (*(this->part_vals)).begin(); _iter1442 != (*(this->part_vals)).end(); ++_iter1442) - { - xfer += oprot->writeString((*_iter1442)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_part", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += (*(this->new_part)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_rename_partition_result::~ThriftHiveMetastore_rename_partition_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_rename_partition_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_rename_partition_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_rename_partition_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_rename_partition_presult::~ThriftHiveMetastore_rename_partition_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_rename_partition_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_partition_name_has_valid_characters_args::~ThriftHiveMetastore_partition_name_has_valid_characters_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->part_vals.clear(); - uint32_t _size1443; - ::apache::thrift::protocol::TType _etype1446; - xfer += iprot->readListBegin(_etype1446, _size1443); - this->part_vals.resize(_size1443); - uint32_t _i1447; - for (_i1447 = 0; _i1447 < _size1443; ++_i1447) - { - xfer += iprot->readString(this->part_vals[_i1447]); - } - xfer += iprot->readListEnd(); - } - this->__isset.part_vals = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->throw_exception); - this->__isset.throw_exception = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_has_valid_characters_args"); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::vector ::const_iterator _iter1448; - for (_iter1448 = this->part_vals.begin(); _iter1448 != this->part_vals.end(); ++_iter1448) - { - xfer += oprot->writeString((*_iter1448)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("throw_exception", ::apache::thrift::protocol::T_BOOL, 2); - xfer += oprot->writeBool(this->throw_exception); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_partition_name_has_valid_characters_pargs::~ThriftHiveMetastore_partition_name_has_valid_characters_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_has_valid_characters_pargs"); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::vector ::const_iterator _iter1449; - for (_iter1449 = (*(this->part_vals)).begin(); _iter1449 != (*(this->part_vals)).end(); ++_iter1449) - { - xfer += oprot->writeString((*_iter1449)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("throw_exception", ::apache::thrift::protocol::T_BOOL, 2); - xfer += oprot->writeBool((*(this->throw_exception))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_partition_name_has_valid_characters_result::~ThriftHiveMetastore_partition_name_has_valid_characters_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_has_valid_characters_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_partition_name_has_valid_characters_presult::~ThriftHiveMetastore_partition_name_has_valid_characters_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_config_value_args::~ThriftHiveMetastore_get_config_value_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_config_value_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->name); - this->__isset.name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->defaultValue); - this->__isset.defaultValue = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_config_value_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_config_value_args"); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("defaultValue", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->defaultValue); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_config_value_pargs::~ThriftHiveMetastore_get_config_value_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_config_value_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_config_value_pargs"); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("defaultValue", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->defaultValue))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_config_value_result::~ThriftHiveMetastore_get_config_value_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_config_value_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_config_value_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_config_value_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0); - xfer += oprot->writeString(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_config_value_presult::~ThriftHiveMetastore_get_config_value_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_config_value_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_partition_name_to_vals_args::~ThriftHiveMetastore_partition_name_to_vals_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_partition_name_to_vals_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->part_name); - this->__isset.part_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_partition_name_to_vals_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_to_vals_args"); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->part_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_partition_name_to_vals_pargs::~ThriftHiveMetastore_partition_name_to_vals_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_partition_name_to_vals_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_to_vals_pargs"); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->part_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_partition_name_to_vals_result::~ThriftHiveMetastore_partition_name_to_vals_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_partition_name_to_vals_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1450; - ::apache::thrift::protocol::TType _etype1453; - xfer += iprot->readListBegin(_etype1453, _size1450); - this->success.resize(_size1450); - uint32_t _i1454; - for (_i1454 = 0; _i1454 < _size1450; ++_i1454) - { - xfer += iprot->readString(this->success[_i1454]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_partition_name_to_vals_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_to_vals_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1455; - for (_iter1455 = this->success.begin(); _iter1455 != this->success.end(); ++_iter1455) - { - xfer += oprot->writeString((*_iter1455)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_partition_name_to_vals_presult::~ThriftHiveMetastore_partition_name_to_vals_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_partition_name_to_vals_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1456; - ::apache::thrift::protocol::TType _etype1459; - xfer += iprot->readListBegin(_etype1459, _size1456); - (*(this->success)).resize(_size1456); - uint32_t _i1460; - for (_i1460 = 0; _i1460 < _size1456; ++_i1460) - { - xfer += iprot->readString((*(this->success))[_i1460]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_partition_name_to_spec_args::~ThriftHiveMetastore_partition_name_to_spec_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_partition_name_to_spec_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->part_name); - this->__isset.part_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_partition_name_to_spec_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_to_spec_args"); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->part_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_partition_name_to_spec_pargs::~ThriftHiveMetastore_partition_name_to_spec_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_partition_name_to_spec_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_to_spec_pargs"); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->part_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_partition_name_to_spec_result::~ThriftHiveMetastore_partition_name_to_spec_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_partition_name_to_spec_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->success.clear(); - uint32_t _size1461; - ::apache::thrift::protocol::TType _ktype1462; - ::apache::thrift::protocol::TType _vtype1463; - xfer += iprot->readMapBegin(_ktype1462, _vtype1463, _size1461); - uint32_t _i1465; - for (_i1465 = 0; _i1465 < _size1461; ++_i1465) - { - std::string _key1466; - xfer += iprot->readString(_key1466); - std::string& _val1467 = this->success[_key1466]; - xfer += iprot->readString(_val1467); - } - xfer += iprot->readMapEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_partition_name_to_spec_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_partition_name_to_spec_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::map ::const_iterator _iter1468; - for (_iter1468 = this->success.begin(); _iter1468 != this->success.end(); ++_iter1468) - { - xfer += oprot->writeString(_iter1468->first); - xfer += oprot->writeString(_iter1468->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_partition_name_to_spec_presult::~ThriftHiveMetastore_partition_name_to_spec_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_partition_name_to_spec_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - (*(this->success)).clear(); - uint32_t _size1469; - ::apache::thrift::protocol::TType _ktype1470; - ::apache::thrift::protocol::TType _vtype1471; - xfer += iprot->readMapBegin(_ktype1470, _vtype1471, _size1469); - uint32_t _i1473; - for (_i1473 = 0; _i1473 < _size1469; ++_i1473) - { - std::string _key1474; - xfer += iprot->readString(_key1474); - std::string& _val1475 = (*(this->success))[_key1474]; - xfer += iprot->readString(_val1475); - } - xfer += iprot->readMapEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_markPartitionForEvent_args::~ThriftHiveMetastore_markPartitionForEvent_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_markPartitionForEvent_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->part_vals.clear(); - uint32_t _size1476; - ::apache::thrift::protocol::TType _ktype1477; - ::apache::thrift::protocol::TType _vtype1478; - xfer += iprot->readMapBegin(_ktype1477, _vtype1478, _size1476); - uint32_t _i1480; - for (_i1480 = 0; _i1480 < _size1476; ++_i1480) - { - std::string _key1481; - xfer += iprot->readString(_key1481); - std::string& _val1482 = this->part_vals[_key1481]; - xfer += iprot->readString(_val1482); - } - xfer += iprot->readMapEnd(); - } - this->__isset.part_vals = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1483; - xfer += iprot->readI32(ecast1483); - this->eventType = (PartitionEventType::type)ecast1483; - this->__isset.eventType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_markPartitionForEvent_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_markPartitionForEvent_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::map ::const_iterator _iter1484; - for (_iter1484 = this->part_vals.begin(); _iter1484 != this->part_vals.end(); ++_iter1484) - { - xfer += oprot->writeString(_iter1484->first); - xfer += oprot->writeString(_iter1484->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("eventType", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32((int32_t)this->eventType); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_markPartitionForEvent_pargs::~ThriftHiveMetastore_markPartitionForEvent_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_markPartitionForEvent_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_markPartitionForEvent_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::map ::const_iterator _iter1485; - for (_iter1485 = (*(this->part_vals)).begin(); _iter1485 != (*(this->part_vals)).end(); ++_iter1485) - { - xfer += oprot->writeString(_iter1485->first); - xfer += oprot->writeString(_iter1485->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("eventType", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32((int32_t)(*(this->eventType))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_markPartitionForEvent_result::~ThriftHiveMetastore_markPartitionForEvent_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_markPartitionForEvent_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o5.read(iprot); - this->__isset.o5 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o6.read(iprot); - this->__isset.o6 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_markPartitionForEvent_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_markPartitionForEvent_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o4) { - xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->o4.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o5) { - xfer += oprot->writeFieldBegin("o5", ::apache::thrift::protocol::T_STRUCT, 5); - xfer += this->o5.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o6) { - xfer += oprot->writeFieldBegin("o6", ::apache::thrift::protocol::T_STRUCT, 6); - xfer += this->o6.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_markPartitionForEvent_presult::~ThriftHiveMetastore_markPartitionForEvent_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_markPartitionForEvent_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o5.read(iprot); - this->__isset.o5 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o6.read(iprot); - this->__isset.o6 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_isPartitionMarkedForEvent_args::~ThriftHiveMetastore_isPartitionMarkedForEvent_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->part_vals.clear(); - uint32_t _size1486; - ::apache::thrift::protocol::TType _ktype1487; - ::apache::thrift::protocol::TType _vtype1488; - xfer += iprot->readMapBegin(_ktype1487, _vtype1488, _size1486); - uint32_t _i1490; - for (_i1490 = 0; _i1490 < _size1486; ++_i1490) - { - std::string _key1491; - xfer += iprot->readString(_key1491); - std::string& _val1492 = this->part_vals[_key1491]; - xfer += iprot->readString(_val1492); - } - xfer += iprot->readMapEnd(); - } - this->__isset.part_vals = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1493; - xfer += iprot->readI32(ecast1493); - this->eventType = (PartitionEventType::type)ecast1493; - this->__isset.eventType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_isPartitionMarkedForEvent_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->part_vals.size())); - std::map ::const_iterator _iter1494; - for (_iter1494 = this->part_vals.begin(); _iter1494 != this->part_vals.end(); ++_iter1494) - { - xfer += oprot->writeString(_iter1494->first); - xfer += oprot->writeString(_iter1494->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("eventType", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32((int32_t)this->eventType); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_isPartitionMarkedForEvent_pargs::~ThriftHiveMetastore_isPartitionMarkedForEvent_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_isPartitionMarkedForEvent_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast((*(this->part_vals)).size())); - std::map ::const_iterator _iter1495; - for (_iter1495 = (*(this->part_vals)).begin(); _iter1495 != (*(this->part_vals)).end(); ++_iter1495) - { - xfer += oprot->writeString(_iter1495->first); - xfer += oprot->writeString(_iter1495->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("eventType", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32((int32_t)(*(this->eventType))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_isPartitionMarkedForEvent_result::~ThriftHiveMetastore_isPartitionMarkedForEvent_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o5.read(iprot); - this->__isset.o5 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o6.read(iprot); - this->__isset.o6 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_isPartitionMarkedForEvent_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o4) { - xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->o4.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o5) { - xfer += oprot->writeFieldBegin("o5", ::apache::thrift::protocol::T_STRUCT, 5); - xfer += this->o5.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o6) { - xfer += oprot->writeFieldBegin("o6", ::apache::thrift::protocol::T_STRUCT, 6); - xfer += this->o6.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_isPartitionMarkedForEvent_presult::~ThriftHiveMetastore_isPartitionMarkedForEvent_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o5.read(iprot); - this->__isset.o5 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o6.read(iprot); - this->__isset.o6 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_add_index_args::~ThriftHiveMetastore_add_index_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_index_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->new_index.read(iprot); - this->__isset.new_index = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->index_table.read(iprot); - this->__isset.index_table = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_index_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_index_args"); - - xfer += oprot->writeFieldBegin("new_index", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->new_index.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("index_table", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->index_table.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_index_pargs::~ThriftHiveMetastore_add_index_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_index_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_index_pargs"); - - xfer += oprot->writeFieldBegin("new_index", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->new_index)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("index_table", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += (*(this->index_table)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_index_result::~ThriftHiveMetastore_add_index_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_index_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_index_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_index_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_index_presult::~ThriftHiveMetastore_add_index_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_index_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_alter_index_args::~ThriftHiveMetastore_alter_index_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_index_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbname); - this->__isset.dbname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->base_tbl_name); - this->__isset.base_tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->idx_name); - this->__isset.idx_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->new_idx.read(iprot); - this->__isset.new_idx = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_index_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_index_args"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbname); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("base_tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->base_tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("idx_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->idx_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_idx", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->new_idx.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_index_pargs::~ThriftHiveMetastore_alter_index_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_index_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_index_pargs"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->dbname))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("base_tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->base_tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("idx_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->idx_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("new_idx", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += (*(this->new_idx)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_index_result::~ThriftHiveMetastore_alter_index_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_index_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_index_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_index_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_index_presult::~ThriftHiveMetastore_alter_index_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_index_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_drop_index_by_name_args::~ThriftHiveMetastore_drop_index_by_name_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_index_by_name_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->index_name); - this->__isset.index_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->deleteData); - this->__isset.deleteData = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_index_by_name_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_index_by_name_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("index_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->index_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool(this->deleteData); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_index_by_name_pargs::~ThriftHiveMetastore_drop_index_by_name_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_index_by_name_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_index_by_name_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("index_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->index_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool((*(this->deleteData))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_index_by_name_result::~ThriftHiveMetastore_drop_index_by_name_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_index_by_name_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_index_by_name_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_index_by_name_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_index_by_name_presult::~ThriftHiveMetastore_drop_index_by_name_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_index_by_name_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_index_by_name_args::~ThriftHiveMetastore_get_index_by_name_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_index_by_name_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->index_name); - this->__isset.index_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_index_by_name_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_index_by_name_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("index_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->index_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_index_by_name_pargs::~ThriftHiveMetastore_get_index_by_name_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_index_by_name_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_index_by_name_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("index_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->index_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_index_by_name_result::~ThriftHiveMetastore_get_index_by_name_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_index_by_name_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_index_by_name_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_index_by_name_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_index_by_name_presult::~ThriftHiveMetastore_get_index_by_name_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_index_by_name_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_indexes_args::~ThriftHiveMetastore_get_indexes_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_indexes_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I16) { - xfer += iprot->readI16(this->max_indexes); - this->__isset.max_indexes = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_indexes_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_indexes_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_indexes", ::apache::thrift::protocol::T_I16, 3); - xfer += oprot->writeI16(this->max_indexes); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_indexes_pargs::~ThriftHiveMetastore_get_indexes_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_indexes_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_indexes_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_indexes", ::apache::thrift::protocol::T_I16, 3); - xfer += oprot->writeI16((*(this->max_indexes))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_indexes_result::~ThriftHiveMetastore_get_indexes_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_indexes_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1496; - ::apache::thrift::protocol::TType _etype1499; - xfer += iprot->readListBegin(_etype1499, _size1496); - this->success.resize(_size1496); - uint32_t _i1500; - for (_i1500 = 0; _i1500 < _size1496; ++_i1500) - { - xfer += this->success[_i1500].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_indexes_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_indexes_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1501; - for (_iter1501 = this->success.begin(); _iter1501 != this->success.end(); ++_iter1501) - { - xfer += (*_iter1501).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_indexes_presult::~ThriftHiveMetastore_get_indexes_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_indexes_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1502; - ::apache::thrift::protocol::TType _etype1505; - xfer += iprot->readListBegin(_etype1505, _size1502); - (*(this->success)).resize(_size1502); - uint32_t _i1506; - for (_i1506 = 0; _i1506 < _size1502; ++_i1506) - { - xfer += (*(this->success))[_i1506].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_index_names_args::~ThriftHiveMetastore_get_index_names_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_index_names_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I16) { - xfer += iprot->readI16(this->max_indexes); - this->__isset.max_indexes = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_index_names_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_index_names_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_indexes", ::apache::thrift::protocol::T_I16, 3); - xfer += oprot->writeI16(this->max_indexes); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_index_names_pargs::~ThriftHiveMetastore_get_index_names_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_index_names_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_index_names_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_indexes", ::apache::thrift::protocol::T_I16, 3); - xfer += oprot->writeI16((*(this->max_indexes))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_index_names_result::~ThriftHiveMetastore_get_index_names_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_index_names_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1507; - ::apache::thrift::protocol::TType _etype1510; - xfer += iprot->readListBegin(_etype1510, _size1507); - this->success.resize(_size1507); - uint32_t _i1511; - for (_i1511 = 0; _i1511 < _size1507; ++_i1511) - { - xfer += iprot->readString(this->success[_i1511]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_index_names_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_index_names_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1512; - for (_iter1512 = this->success.begin(); _iter1512 != this->success.end(); ++_iter1512) - { - xfer += oprot->writeString((*_iter1512)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_index_names_presult::~ThriftHiveMetastore_get_index_names_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_index_names_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1513; - ::apache::thrift::protocol::TType _etype1516; - xfer += iprot->readListBegin(_etype1516, _size1513); - (*(this->success)).resize(_size1513); - uint32_t _i1517; - for (_i1517 = 0; _i1517 < _size1513; ++_i1517) - { - xfer += iprot->readString((*(this->success))[_i1517]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_primary_keys_args::~ThriftHiveMetastore_get_primary_keys_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_primary_keys_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_primary_keys_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_primary_keys_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_primary_keys_pargs::~ThriftHiveMetastore_get_primary_keys_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_primary_keys_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_primary_keys_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_primary_keys_result::~ThriftHiveMetastore_get_primary_keys_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_primary_keys_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_primary_keys_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_primary_keys_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_primary_keys_presult::~ThriftHiveMetastore_get_primary_keys_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_primary_keys_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_foreign_keys_args::~ThriftHiveMetastore_get_foreign_keys_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_foreign_keys_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_foreign_keys_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_foreign_keys_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_foreign_keys_pargs::~ThriftHiveMetastore_get_foreign_keys_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_foreign_keys_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_foreign_keys_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_foreign_keys_result::~ThriftHiveMetastore_get_foreign_keys_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_foreign_keys_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_foreign_keys_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_foreign_keys_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_foreign_keys_presult::~ThriftHiveMetastore_get_foreign_keys_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_foreign_keys_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_unique_constraints_args::~ThriftHiveMetastore_get_unique_constraints_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_unique_constraints_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_unique_constraints_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_unique_constraints_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_unique_constraints_pargs::~ThriftHiveMetastore_get_unique_constraints_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_unique_constraints_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_unique_constraints_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_unique_constraints_result::~ThriftHiveMetastore_get_unique_constraints_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_unique_constraints_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_unique_constraints_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_unique_constraints_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_unique_constraints_presult::~ThriftHiveMetastore_get_unique_constraints_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_unique_constraints_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_not_null_constraints_args::~ThriftHiveMetastore_get_not_null_constraints_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_not_null_constraints_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_not_null_constraints_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_not_null_constraints_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_not_null_constraints_pargs::~ThriftHiveMetastore_get_not_null_constraints_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_not_null_constraints_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_not_null_constraints_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_not_null_constraints_result::~ThriftHiveMetastore_get_not_null_constraints_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_not_null_constraints_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_not_null_constraints_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_not_null_constraints_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_not_null_constraints_presult::~ThriftHiveMetastore_get_not_null_constraints_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_not_null_constraints_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_update_table_column_statistics_args::~ThriftHiveMetastore_update_table_column_statistics_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_update_table_column_statistics_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->stats_obj.read(iprot); - this->__isset.stats_obj = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_update_table_column_statistics_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_table_column_statistics_args"); - - xfer += oprot->writeFieldBegin("stats_obj", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->stats_obj.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_update_table_column_statistics_pargs::~ThriftHiveMetastore_update_table_column_statistics_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_update_table_column_statistics_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_table_column_statistics_pargs"); - - xfer += oprot->writeFieldBegin("stats_obj", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->stats_obj)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_update_table_column_statistics_result::~ThriftHiveMetastore_update_table_column_statistics_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_update_table_column_statistics_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_update_table_column_statistics_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_table_column_statistics_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o4) { - xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->o4.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_update_table_column_statistics_presult::~ThriftHiveMetastore_update_table_column_statistics_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_update_table_column_statistics_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_update_partition_column_statistics_args::~ThriftHiveMetastore_update_partition_column_statistics_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_update_partition_column_statistics_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->stats_obj.read(iprot); - this->__isset.stats_obj = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_update_partition_column_statistics_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_partition_column_statistics_args"); - - xfer += oprot->writeFieldBegin("stats_obj", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->stats_obj.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_update_partition_column_statistics_pargs::~ThriftHiveMetastore_update_partition_column_statistics_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_update_partition_column_statistics_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_partition_column_statistics_pargs"); - - xfer += oprot->writeFieldBegin("stats_obj", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->stats_obj)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_update_partition_column_statistics_result::~ThriftHiveMetastore_update_partition_column_statistics_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_update_partition_column_statistics_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_update_partition_column_statistics_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_partition_column_statistics_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o4) { - xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->o4.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_update_partition_column_statistics_presult::~ThriftHiveMetastore_update_partition_column_statistics_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_update_partition_column_statistics_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_table_column_statistics_args::~ThriftHiveMetastore_get_table_column_statistics_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_column_statistics_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->col_name); - this->__isset.col_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_table_column_statistics_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_column_statistics_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("col_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->col_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_column_statistics_pargs::~ThriftHiveMetastore_get_table_column_statistics_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_column_statistics_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_column_statistics_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("col_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->col_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_column_statistics_result::~ThriftHiveMetastore_get_table_column_statistics_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_column_statistics_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_table_column_statistics_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_column_statistics_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o4) { - xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->o4.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_column_statistics_presult::~ThriftHiveMetastore_get_table_column_statistics_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_column_statistics_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_partition_column_statistics_args::~ThriftHiveMetastore_get_partition_column_statistics_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_column_statistics_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->part_name); - this->__isset.part_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->col_name); - this->__isset.col_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partition_column_statistics_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_column_statistics_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->part_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("col_name", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->col_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_column_statistics_pargs::~ThriftHiveMetastore_get_partition_column_statistics_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_column_statistics_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_column_statistics_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->part_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("col_name", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString((*(this->col_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_column_statistics_result::~ThriftHiveMetastore_get_partition_column_statistics_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_column_statistics_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partition_column_statistics_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partition_column_statistics_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o4) { - xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->o4.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partition_column_statistics_presult::~ThriftHiveMetastore_get_partition_column_statistics_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partition_column_statistics_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_table_statistics_req_args::~ThriftHiveMetastore_get_table_statistics_req_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_statistics_req_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_table_statistics_req_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_statistics_req_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_statistics_req_pargs::~ThriftHiveMetastore_get_table_statistics_req_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_statistics_req_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_statistics_req_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_statistics_req_result::~ThriftHiveMetastore_get_table_statistics_req_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_statistics_req_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_table_statistics_req_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_table_statistics_req_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_table_statistics_req_presult::~ThriftHiveMetastore_get_table_statistics_req_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_table_statistics_req_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_partitions_statistics_req_args::~ThriftHiveMetastore_get_partitions_statistics_req_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_statistics_req_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_statistics_req_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_statistics_req_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_statistics_req_pargs::~ThriftHiveMetastore_get_partitions_statistics_req_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_statistics_req_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_statistics_req_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_statistics_req_result::~ThriftHiveMetastore_get_partitions_statistics_req_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_statistics_req_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_partitions_statistics_req_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_partitions_statistics_req_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_partitions_statistics_req_presult::~ThriftHiveMetastore_get_partitions_statistics_req_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_partitions_statistics_req_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_aggr_stats_for_args::~ThriftHiveMetastore_get_aggr_stats_for_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_aggr_stats_for_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_aggr_stats_for_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_aggr_stats_for_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_aggr_stats_for_pargs::~ThriftHiveMetastore_get_aggr_stats_for_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_aggr_stats_for_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_aggr_stats_for_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_aggr_stats_for_result::~ThriftHiveMetastore_get_aggr_stats_for_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_aggr_stats_for_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_aggr_stats_for_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_aggr_stats_for_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_aggr_stats_for_presult::~ThriftHiveMetastore_get_aggr_stats_for_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_aggr_stats_for_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_set_aggr_stats_for_args::~ThriftHiveMetastore_set_aggr_stats_for_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_set_aggr_stats_for_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_set_aggr_stats_for_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_aggr_stats_for_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_set_aggr_stats_for_pargs::~ThriftHiveMetastore_set_aggr_stats_for_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_set_aggr_stats_for_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_aggr_stats_for_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_set_aggr_stats_for_result::~ThriftHiveMetastore_set_aggr_stats_for_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_set_aggr_stats_for_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_set_aggr_stats_for_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_aggr_stats_for_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o4) { - xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->o4.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_set_aggr_stats_for_presult::~ThriftHiveMetastore_set_aggr_stats_for_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_set_aggr_stats_for_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_delete_partition_column_statistics_args::~ThriftHiveMetastore_delete_partition_column_statistics_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_delete_partition_column_statistics_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->part_name); - this->__isset.part_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->col_name); - this->__isset.col_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_delete_partition_column_statistics_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_delete_partition_column_statistics_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->part_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("col_name", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->col_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_delete_partition_column_statistics_pargs::~ThriftHiveMetastore_delete_partition_column_statistics_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_delete_partition_column_statistics_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_delete_partition_column_statistics_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("part_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->part_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("col_name", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString((*(this->col_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_delete_partition_column_statistics_result::~ThriftHiveMetastore_delete_partition_column_statistics_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_delete_partition_column_statistics_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_delete_partition_column_statistics_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_delete_partition_column_statistics_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o4) { - xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->o4.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_delete_partition_column_statistics_presult::~ThriftHiveMetastore_delete_partition_column_statistics_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_delete_partition_column_statistics_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_delete_table_column_statistics_args::~ThriftHiveMetastore_delete_table_column_statistics_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_delete_table_column_statistics_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->db_name); - this->__isset.db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - this->__isset.tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->col_name); - this->__isset.col_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_delete_table_column_statistics_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_delete_table_column_statistics_args"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("col_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->col_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_delete_table_column_statistics_pargs::~ThriftHiveMetastore_delete_table_column_statistics_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_delete_table_column_statistics_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_delete_table_column_statistics_pargs"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->db_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->tbl_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("col_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString((*(this->col_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_delete_table_column_statistics_result::~ThriftHiveMetastore_delete_table_column_statistics_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_delete_table_column_statistics_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_delete_table_column_statistics_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_delete_table_column_statistics_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o4) { - xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->o4.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_delete_table_column_statistics_presult::~ThriftHiveMetastore_delete_table_column_statistics_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_delete_table_column_statistics_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_create_function_args::~ThriftHiveMetastore_create_function_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_function_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->func.read(iprot); - this->__isset.func = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_function_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_function_args"); - - xfer += oprot->writeFieldBegin("func", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->func.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_function_pargs::~ThriftHiveMetastore_create_function_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_function_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_function_pargs"); - - xfer += oprot->writeFieldBegin("func", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->func)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_function_result::~ThriftHiveMetastore_create_function_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_function_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_function_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_function_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o4) { - xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->o4.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_function_presult::~ThriftHiveMetastore_create_function_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_function_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_drop_function_args::~ThriftHiveMetastore_drop_function_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_function_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbName); - this->__isset.dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->funcName); - this->__isset.funcName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_function_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_function_args"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("funcName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->funcName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_function_pargs::~ThriftHiveMetastore_drop_function_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_function_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_function_pargs"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->dbName))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("funcName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->funcName))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_function_result::~ThriftHiveMetastore_drop_function_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_function_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_function_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_function_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_function_presult::~ThriftHiveMetastore_drop_function_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_function_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_alter_function_args::~ThriftHiveMetastore_alter_function_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_function_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbName); - this->__isset.dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->funcName); - this->__isset.funcName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->newFunc.read(iprot); - this->__isset.newFunc = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_function_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_function_args"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("funcName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->funcName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("newFunc", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->newFunc.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_function_pargs::~ThriftHiveMetastore_alter_function_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_function_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_function_pargs"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->dbName))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("funcName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->funcName))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("newFunc", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += (*(this->newFunc)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_function_result::~ThriftHiveMetastore_alter_function_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_function_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_function_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_function_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_function_presult::~ThriftHiveMetastore_alter_function_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_function_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_functions_args::~ThriftHiveMetastore_get_functions_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_functions_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbName); - this->__isset.dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->pattern); - this->__isset.pattern = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_functions_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_functions_args"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->pattern); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_functions_pargs::~ThriftHiveMetastore_get_functions_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_functions_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_functions_pargs"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->dbName))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->pattern))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_functions_result::~ThriftHiveMetastore_get_functions_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_functions_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1518; - ::apache::thrift::protocol::TType _etype1521; - xfer += iprot->readListBegin(_etype1521, _size1518); - this->success.resize(_size1518); - uint32_t _i1522; - for (_i1522 = 0; _i1522 < _size1518; ++_i1522) - { - xfer += iprot->readString(this->success[_i1522]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_functions_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_functions_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1523; - for (_iter1523 = this->success.begin(); _iter1523 != this->success.end(); ++_iter1523) - { - xfer += oprot->writeString((*_iter1523)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_functions_presult::~ThriftHiveMetastore_get_functions_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_functions_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1524; - ::apache::thrift::protocol::TType _etype1527; - xfer += iprot->readListBegin(_etype1527, _size1524); - (*(this->success)).resize(_size1524); - uint32_t _i1528; - for (_i1528 = 0; _i1528 < _size1524; ++_i1528) - { - xfer += iprot->readString((*(this->success))[_i1528]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_function_args::~ThriftHiveMetastore_get_function_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_function_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbName); - this->__isset.dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->funcName); - this->__isset.funcName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_function_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_function_args"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("funcName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->funcName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_function_pargs::~ThriftHiveMetastore_get_function_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_function_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_function_pargs"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->dbName))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("funcName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->funcName))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_function_result::~ThriftHiveMetastore_get_function_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_function_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_function_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_function_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_function_presult::~ThriftHiveMetastore_get_function_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_function_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_all_functions_args::~ThriftHiveMetastore_get_all_functions_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_functions_args::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 ThriftHiveMetastore_get_all_functions_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_functions_args"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_all_functions_pargs::~ThriftHiveMetastore_get_all_functions_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_functions_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_functions_pargs"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_all_functions_result::~ThriftHiveMetastore_get_all_functions_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_functions_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_all_functions_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_functions_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_all_functions_presult::~ThriftHiveMetastore_get_all_functions_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_functions_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_create_role_args::~ThriftHiveMetastore_create_role_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_role_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->role.read(iprot); - this->__isset.role = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_role_args"); - - xfer += oprot->writeFieldBegin("role", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->role.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_role_pargs::~ThriftHiveMetastore_create_role_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_role_pargs"); - - xfer += oprot->writeFieldBegin("role", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->role)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_role_result::~ThriftHiveMetastore_create_role_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_role_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_role_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_role_presult::~ThriftHiveMetastore_create_role_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_role_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_drop_role_args::~ThriftHiveMetastore_drop_role_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_role_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->role_name); - this->__isset.role_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_role_args"); - - xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->role_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_role_pargs::~ThriftHiveMetastore_drop_role_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_role_pargs"); - - xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->role_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_role_result::~ThriftHiveMetastore_drop_role_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_role_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_role_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_role_presult::~ThriftHiveMetastore_drop_role_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_role_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_role_names_args::~ThriftHiveMetastore_get_role_names_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_role_names_args::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 ThriftHiveMetastore_get_role_names_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_names_args"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_role_names_pargs::~ThriftHiveMetastore_get_role_names_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_role_names_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_names_pargs"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_role_names_result::~ThriftHiveMetastore_get_role_names_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_role_names_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1529; - ::apache::thrift::protocol::TType _etype1532; - xfer += iprot->readListBegin(_etype1532, _size1529); - this->success.resize(_size1529); - uint32_t _i1533; - for (_i1533 = 0; _i1533 < _size1529; ++_i1533) - { - xfer += iprot->readString(this->success[_i1533]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_role_names_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_names_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1534; - for (_iter1534 = this->success.begin(); _iter1534 != this->success.end(); ++_iter1534) - { - xfer += oprot->writeString((*_iter1534)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_role_names_presult::~ThriftHiveMetastore_get_role_names_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_role_names_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1535; - ::apache::thrift::protocol::TType _etype1538; - xfer += iprot->readListBegin(_etype1538, _size1535); - (*(this->success)).resize(_size1535); - uint32_t _i1539; - for (_i1539 = 0; _i1539 < _size1535; ++_i1539) - { - xfer += iprot->readString((*(this->success))[_i1539]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_grant_role_args::~ThriftHiveMetastore_grant_role_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_grant_role_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->role_name); - this->__isset.role_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->principal_name); - this->__isset.principal_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1540; - xfer += iprot->readI32(ecast1540); - this->principal_type = (PrincipalType::type)ecast1540; - this->__isset.principal_type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->grantor); - this->__isset.grantor = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1541; - xfer += iprot->readI32(ecast1541); - this->grantorType = (PrincipalType::type)ecast1541; - this->__isset.grantorType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->grant_option); - this->__isset.grant_option = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_grant_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_role_args"); - - xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->role_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->principal_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32((int32_t)this->principal_type); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("grantor", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->grantor); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("grantorType", ::apache::thrift::protocol::T_I32, 5); - xfer += oprot->writeI32((int32_t)this->grantorType); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("grant_option", ::apache::thrift::protocol::T_BOOL, 6); - xfer += oprot->writeBool(this->grant_option); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_grant_role_pargs::~ThriftHiveMetastore_grant_role_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_grant_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_role_pargs"); - - xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->role_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->principal_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32((int32_t)(*(this->principal_type))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("grantor", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString((*(this->grantor))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("grantorType", ::apache::thrift::protocol::T_I32, 5); - xfer += oprot->writeI32((int32_t)(*(this->grantorType))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("grant_option", ::apache::thrift::protocol::T_BOOL, 6); - xfer += oprot->writeBool((*(this->grant_option))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_grant_role_result::~ThriftHiveMetastore_grant_role_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_grant_role_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_grant_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_role_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_grant_role_presult::~ThriftHiveMetastore_grant_role_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_grant_role_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_revoke_role_args::~ThriftHiveMetastore_revoke_role_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_revoke_role_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->role_name); - this->__isset.role_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->principal_name); - this->__isset.principal_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1542; - xfer += iprot->readI32(ecast1542); - this->principal_type = (PrincipalType::type)ecast1542; - this->__isset.principal_type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_revoke_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_role_args"); - - xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->role_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->principal_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32((int32_t)this->principal_type); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_revoke_role_pargs::~ThriftHiveMetastore_revoke_role_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_revoke_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_role_pargs"); - - xfer += oprot->writeFieldBegin("role_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->role_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->principal_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32((int32_t)(*(this->principal_type))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_revoke_role_result::~ThriftHiveMetastore_revoke_role_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_revoke_role_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_revoke_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_role_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_revoke_role_presult::~ThriftHiveMetastore_revoke_role_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_revoke_role_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_list_roles_args::~ThriftHiveMetastore_list_roles_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_list_roles_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->principal_name); - this->__isset.principal_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1543; - xfer += iprot->readI32(ecast1543); - this->principal_type = (PrincipalType::type)ecast1543; - this->__isset.principal_type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_list_roles_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_roles_args"); - - xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->principal_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32((int32_t)this->principal_type); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_list_roles_pargs::~ThriftHiveMetastore_list_roles_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_list_roles_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_roles_pargs"); - - xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->principal_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32((int32_t)(*(this->principal_type))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_list_roles_result::~ThriftHiveMetastore_list_roles_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_list_roles_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1544; - ::apache::thrift::protocol::TType _etype1547; - xfer += iprot->readListBegin(_etype1547, _size1544); - this->success.resize(_size1544); - uint32_t _i1548; - for (_i1548 = 0; _i1548 < _size1544; ++_i1548) - { - xfer += this->success[_i1548].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_list_roles_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_roles_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1549; - for (_iter1549 = this->success.begin(); _iter1549 != this->success.end(); ++_iter1549) - { - xfer += (*_iter1549).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_list_roles_presult::~ThriftHiveMetastore_list_roles_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_list_roles_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1550; - ::apache::thrift::protocol::TType _etype1553; - xfer += iprot->readListBegin(_etype1553, _size1550); - (*(this->success)).resize(_size1550); - uint32_t _i1554; - for (_i1554 = 0; _i1554 < _size1550; ++_i1554) - { - xfer += (*(this->success))[_i1554].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_grant_revoke_role_args::~ThriftHiveMetastore_grant_revoke_role_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_grant_revoke_role_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_grant_revoke_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_revoke_role_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_grant_revoke_role_pargs::~ThriftHiveMetastore_grant_revoke_role_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_grant_revoke_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_revoke_role_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_grant_revoke_role_result::~ThriftHiveMetastore_grant_revoke_role_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_grant_revoke_role_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_grant_revoke_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_revoke_role_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_grant_revoke_role_presult::~ThriftHiveMetastore_grant_revoke_role_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_grant_revoke_role_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_principals_in_role_args::~ThriftHiveMetastore_get_principals_in_role_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_principals_in_role_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_principals_in_role_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_principals_in_role_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_principals_in_role_pargs::~ThriftHiveMetastore_get_principals_in_role_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_principals_in_role_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_principals_in_role_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_principals_in_role_result::~ThriftHiveMetastore_get_principals_in_role_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_principals_in_role_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_principals_in_role_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_principals_in_role_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_principals_in_role_presult::~ThriftHiveMetastore_get_principals_in_role_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_principals_in_role_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_role_grants_for_principal_args::~ThriftHiveMetastore_get_role_grants_for_principal_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_role_grants_for_principal_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_role_grants_for_principal_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_grants_for_principal_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_role_grants_for_principal_pargs::~ThriftHiveMetastore_get_role_grants_for_principal_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_role_grants_for_principal_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_grants_for_principal_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_role_grants_for_principal_result::~ThriftHiveMetastore_get_role_grants_for_principal_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_role_grants_for_principal_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_role_grants_for_principal_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_role_grants_for_principal_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_role_grants_for_principal_presult::~ThriftHiveMetastore_get_role_grants_for_principal_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_role_grants_for_principal_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_privilege_set_args::~ThriftHiveMetastore_get_privilege_set_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_privilege_set_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->hiveObject.read(iprot); - this->__isset.hiveObject = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->user_name); - this->__isset.user_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->group_names.clear(); - uint32_t _size1555; - ::apache::thrift::protocol::TType _etype1558; - xfer += iprot->readListBegin(_etype1558, _size1555); - this->group_names.resize(_size1555); - uint32_t _i1559; - for (_i1559 = 0; _i1559 < _size1555; ++_i1559) - { - xfer += iprot->readString(this->group_names[_i1559]); - } - xfer += iprot->readListEnd(); - } - this->__isset.group_names = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_privilege_set_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_privilege_set_args"); - - xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->hiveObject.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->user_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); - std::vector ::const_iterator _iter1560; - for (_iter1560 = this->group_names.begin(); _iter1560 != this->group_names.end(); ++_iter1560) - { - xfer += oprot->writeString((*_iter1560)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_privilege_set_pargs::~ThriftHiveMetastore_get_privilege_set_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_privilege_set_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_privilege_set_pargs"); - - xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->hiveObject)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->user_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); - std::vector ::const_iterator _iter1561; - for (_iter1561 = (*(this->group_names)).begin(); _iter1561 != (*(this->group_names)).end(); ++_iter1561) - { - xfer += oprot->writeString((*_iter1561)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_privilege_set_result::~ThriftHiveMetastore_get_privilege_set_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_privilege_set_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_privilege_set_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_privilege_set_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_privilege_set_presult::~ThriftHiveMetastore_get_privilege_set_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_privilege_set_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_list_privileges_args::~ThriftHiveMetastore_list_privileges_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_list_privileges_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->principal_name); - this->__isset.principal_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast1562; - xfer += iprot->readI32(ecast1562); - this->principal_type = (PrincipalType::type)ecast1562; - this->__isset.principal_type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->hiveObject.read(iprot); - this->__isset.hiveObject = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_list_privileges_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_privileges_args"); - - xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->principal_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32((int32_t)this->principal_type); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->hiveObject.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_list_privileges_pargs::~ThriftHiveMetastore_list_privileges_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_list_privileges_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_privileges_pargs"); - - xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->principal_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32((int32_t)(*(this->principal_type))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += (*(this->hiveObject)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_list_privileges_result::~ThriftHiveMetastore_list_privileges_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_list_privileges_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1563; - ::apache::thrift::protocol::TType _etype1566; - xfer += iprot->readListBegin(_etype1566, _size1563); - this->success.resize(_size1563); - uint32_t _i1567; - for (_i1567 = 0; _i1567 < _size1563; ++_i1567) - { - xfer += this->success[_i1567].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_list_privileges_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_list_privileges_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->success.size())); - std::vector ::const_iterator _iter1568; - for (_iter1568 = this->success.begin(); _iter1568 != this->success.end(); ++_iter1568) - { - xfer += (*_iter1568).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_list_privileges_presult::~ThriftHiveMetastore_list_privileges_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_list_privileges_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1569; - ::apache::thrift::protocol::TType _etype1572; - xfer += iprot->readListBegin(_etype1572, _size1569); - (*(this->success)).resize(_size1569); - uint32_t _i1573; - for (_i1573 = 0; _i1573 < _size1569; ++_i1573) - { - xfer += (*(this->success))[_i1573].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_grant_privileges_args::~ThriftHiveMetastore_grant_privileges_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_grant_privileges_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->privileges.read(iprot); - this->__isset.privileges = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_grant_privileges_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_privileges_args"); - - xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->privileges.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_grant_privileges_pargs::~ThriftHiveMetastore_grant_privileges_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_grant_privileges_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_privileges_pargs"); - - xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->privileges)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_grant_privileges_result::~ThriftHiveMetastore_grant_privileges_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_grant_privileges_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_grant_privileges_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_privileges_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_grant_privileges_presult::~ThriftHiveMetastore_grant_privileges_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_grant_privileges_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_revoke_privileges_args::~ThriftHiveMetastore_revoke_privileges_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_revoke_privileges_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->privileges.read(iprot); - this->__isset.privileges = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_revoke_privileges_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_privileges_args"); - - xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->privileges.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_revoke_privileges_pargs::~ThriftHiveMetastore_revoke_privileges_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_revoke_privileges_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_privileges_pargs"); - - xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->privileges)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_revoke_privileges_result::~ThriftHiveMetastore_revoke_privileges_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_revoke_privileges_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_revoke_privileges_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_revoke_privileges_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_revoke_privileges_presult::~ThriftHiveMetastore_revoke_privileges_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_revoke_privileges_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_grant_revoke_privileges_args::~ThriftHiveMetastore_grant_revoke_privileges_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_grant_revoke_privileges_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_grant_revoke_privileges_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_revoke_privileges_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_grant_revoke_privileges_pargs::~ThriftHiveMetastore_grant_revoke_privileges_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_grant_revoke_privileges_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_revoke_privileges_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_grant_revoke_privileges_result::~ThriftHiveMetastore_grant_revoke_privileges_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_grant_revoke_privileges_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_grant_revoke_privileges_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_grant_revoke_privileges_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_grant_revoke_privileges_presult::~ThriftHiveMetastore_grant_revoke_privileges_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_grant_revoke_privileges_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_set_ugi_args::~ThriftHiveMetastore_set_ugi_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_set_ugi_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->user_name); - this->__isset.user_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->group_names.clear(); - uint32_t _size1574; - ::apache::thrift::protocol::TType _etype1577; - xfer += iprot->readListBegin(_etype1577, _size1574); - this->group_names.resize(_size1574); - uint32_t _i1578; - for (_i1578 = 0; _i1578 < _size1574; ++_i1578) - { - xfer += iprot->readString(this->group_names[_i1578]); - } - xfer += iprot->readListEnd(); - } - this->__isset.group_names = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_set_ugi_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_ugi_args"); - - xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->user_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->group_names.size())); - std::vector ::const_iterator _iter1579; - for (_iter1579 = this->group_names.begin(); _iter1579 != this->group_names.end(); ++_iter1579) - { - xfer += oprot->writeString((*_iter1579)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_set_ugi_pargs::~ThriftHiveMetastore_set_ugi_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_set_ugi_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_ugi_pargs"); - - xfer += oprot->writeFieldBegin("user_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->user_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*(this->group_names)).size())); - std::vector ::const_iterator _iter1580; - for (_iter1580 = (*(this->group_names)).begin(); _iter1580 != (*(this->group_names)).end(); ++_iter1580) - { - xfer += oprot->writeString((*_iter1580)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_set_ugi_result::~ThriftHiveMetastore_set_ugi_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_set_ugi_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1581; - ::apache::thrift::protocol::TType _etype1584; - xfer += iprot->readListBegin(_etype1584, _size1581); - this->success.resize(_size1581); - uint32_t _i1585; - for (_i1585 = 0; _i1585 < _size1581; ++_i1585) - { - xfer += iprot->readString(this->success[_i1585]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_set_ugi_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_ugi_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1586; - for (_iter1586 = this->success.begin(); _iter1586 != this->success.end(); ++_iter1586) - { - xfer += oprot->writeString((*_iter1586)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_set_ugi_presult::~ThriftHiveMetastore_set_ugi_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_set_ugi_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1587; - ::apache::thrift::protocol::TType _etype1590; - xfer += iprot->readListBegin(_etype1590, _size1587); - (*(this->success)).resize(_size1587); - uint32_t _i1591; - for (_i1591 = 0; _i1591 < _size1587; ++_i1591) - { - xfer += iprot->readString((*(this->success))[_i1591]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_delegation_token_args::~ThriftHiveMetastore_get_delegation_token_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_delegation_token_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->token_owner); - this->__isset.token_owner = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->renewer_kerberos_principal_name); - this->__isset.renewer_kerberos_principal_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_delegation_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_delegation_token_args"); - - xfer += oprot->writeFieldBegin("token_owner", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->token_owner); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("renewer_kerberos_principal_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->renewer_kerberos_principal_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_delegation_token_pargs::~ThriftHiveMetastore_get_delegation_token_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_delegation_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_delegation_token_pargs"); - - xfer += oprot->writeFieldBegin("token_owner", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->token_owner))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("renewer_kerberos_principal_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->renewer_kerberos_principal_name))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_delegation_token_result::~ThriftHiveMetastore_get_delegation_token_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_delegation_token_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_delegation_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_delegation_token_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0); - xfer += oprot->writeString(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_delegation_token_presult::~ThriftHiveMetastore_get_delegation_token_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_delegation_token_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_renew_delegation_token_args::~ThriftHiveMetastore_renew_delegation_token_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_renew_delegation_token_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->token_str_form); - this->__isset.token_str_form = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_renew_delegation_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_renew_delegation_token_args"); - - xfer += oprot->writeFieldBegin("token_str_form", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->token_str_form); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_renew_delegation_token_pargs::~ThriftHiveMetastore_renew_delegation_token_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_renew_delegation_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_renew_delegation_token_pargs"); - - xfer += oprot->writeFieldBegin("token_str_form", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->token_str_form))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_renew_delegation_token_result::~ThriftHiveMetastore_renew_delegation_token_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_renew_delegation_token_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_renew_delegation_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_renew_delegation_token_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_I64, 0); - xfer += oprot->writeI64(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_renew_delegation_token_presult::~ThriftHiveMetastore_renew_delegation_token_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_renew_delegation_token_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_cancel_delegation_token_args::~ThriftHiveMetastore_cancel_delegation_token_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_cancel_delegation_token_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->token_str_form); - this->__isset.token_str_form = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_cancel_delegation_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_cancel_delegation_token_args"); - - xfer += oprot->writeFieldBegin("token_str_form", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->token_str_form); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_cancel_delegation_token_pargs::~ThriftHiveMetastore_cancel_delegation_token_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_cancel_delegation_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_cancel_delegation_token_pargs"); - - xfer += oprot->writeFieldBegin("token_str_form", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->token_str_form))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_cancel_delegation_token_result::~ThriftHiveMetastore_cancel_delegation_token_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_cancel_delegation_token_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_cancel_delegation_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_cancel_delegation_token_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_cancel_delegation_token_presult::~ThriftHiveMetastore_cancel_delegation_token_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_cancel_delegation_token_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_add_token_args::~ThriftHiveMetastore_add_token_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_token_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->token_identifier); - this->__isset.token_identifier = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->delegation_token); - this->__isset.delegation_token = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_token_args"); - - xfer += oprot->writeFieldBegin("token_identifier", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->token_identifier); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("delegation_token", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->delegation_token); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_token_pargs::~ThriftHiveMetastore_add_token_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_token_pargs"); - - xfer += oprot->writeFieldBegin("token_identifier", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->token_identifier))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("delegation_token", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->delegation_token))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_token_result::~ThriftHiveMetastore_add_token_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_token_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_token_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_token_presult::~ThriftHiveMetastore_add_token_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_token_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_remove_token_args::~ThriftHiveMetastore_remove_token_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_remove_token_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->token_identifier); - this->__isset.token_identifier = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_remove_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_remove_token_args"); - - xfer += oprot->writeFieldBegin("token_identifier", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->token_identifier); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_remove_token_pargs::~ThriftHiveMetastore_remove_token_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_remove_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_remove_token_pargs"); - - xfer += oprot->writeFieldBegin("token_identifier", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->token_identifier))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_remove_token_result::~ThriftHiveMetastore_remove_token_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_remove_token_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_remove_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_remove_token_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_remove_token_presult::~ThriftHiveMetastore_remove_token_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_remove_token_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_token_args::~ThriftHiveMetastore_get_token_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_token_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->token_identifier); - this->__isset.token_identifier = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_token_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_token_args"); - - xfer += oprot->writeFieldBegin("token_identifier", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->token_identifier); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_token_pargs::~ThriftHiveMetastore_get_token_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_token_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_token_pargs"); - - xfer += oprot->writeFieldBegin("token_identifier", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->token_identifier))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_token_result::~ThriftHiveMetastore_get_token_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_token_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_token_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_token_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0); - xfer += oprot->writeString(this->success); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_token_presult::~ThriftHiveMetastore_get_token_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_token_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_all_token_identifiers_args::~ThriftHiveMetastore_get_all_token_identifiers_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_token_identifiers_args::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 ThriftHiveMetastore_get_all_token_identifiers_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_token_identifiers_args"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_all_token_identifiers_pargs::~ThriftHiveMetastore_get_all_token_identifiers_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_token_identifiers_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_token_identifiers_pargs"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_all_token_identifiers_result::~ThriftHiveMetastore_get_all_token_identifiers_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_token_identifiers_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1592; - ::apache::thrift::protocol::TType _etype1595; - xfer += iprot->readListBegin(_etype1595, _size1592); - this->success.resize(_size1592); - uint32_t _i1596; - for (_i1596 = 0; _i1596 < _size1592; ++_i1596) - { - xfer += iprot->readString(this->success[_i1596]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_all_token_identifiers_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_token_identifiers_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1597; - for (_iter1597 = this->success.begin(); _iter1597 != this->success.end(); ++_iter1597) - { - xfer += oprot->writeString((*_iter1597)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_all_token_identifiers_presult::~ThriftHiveMetastore_get_all_token_identifiers_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_token_identifiers_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1598; - ::apache::thrift::protocol::TType _etype1601; - xfer += iprot->readListBegin(_etype1601, _size1598); - (*(this->success)).resize(_size1598); - uint32_t _i1602; - for (_i1602 = 0; _i1602 < _size1598; ++_i1602) - { - xfer += iprot->readString((*(this->success))[_i1602]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_add_master_key_args::~ThriftHiveMetastore_add_master_key_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_master_key_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->key); - this->__isset.key = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_master_key_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_master_key_args"); - - xfer += oprot->writeFieldBegin("key", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->key); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_master_key_pargs::~ThriftHiveMetastore_add_master_key_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_master_key_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_master_key_pargs"); - - xfer += oprot->writeFieldBegin("key", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString((*(this->key))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_master_key_result::~ThriftHiveMetastore_add_master_key_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_master_key_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_master_key_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_master_key_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_I32, 0); - xfer += oprot->writeI32(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_master_key_presult::~ThriftHiveMetastore_add_master_key_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_master_key_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_update_master_key_args::~ThriftHiveMetastore_update_master_key_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_update_master_key_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->seq_number); - this->__isset.seq_number = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->key); - this->__isset.key = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_update_master_key_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_master_key_args"); - - xfer += oprot->writeFieldBegin("seq_number", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32(this->seq_number); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("key", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->key); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_update_master_key_pargs::~ThriftHiveMetastore_update_master_key_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_update_master_key_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_master_key_pargs"); - - xfer += oprot->writeFieldBegin("seq_number", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32((*(this->seq_number))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("key", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString((*(this->key))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_update_master_key_result::~ThriftHiveMetastore_update_master_key_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_update_master_key_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_update_master_key_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_master_key_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_update_master_key_presult::~ThriftHiveMetastore_update_master_key_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_update_master_key_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_remove_master_key_args::~ThriftHiveMetastore_remove_master_key_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_remove_master_key_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->key_seq); - this->__isset.key_seq = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_remove_master_key_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_remove_master_key_args"); - - xfer += oprot->writeFieldBegin("key_seq", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32(this->key_seq); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_remove_master_key_pargs::~ThriftHiveMetastore_remove_master_key_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_remove_master_key_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_remove_master_key_pargs"); - - xfer += oprot->writeFieldBegin("key_seq", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32((*(this->key_seq))); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_remove_master_key_result::~ThriftHiveMetastore_remove_master_key_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_remove_master_key_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_remove_master_key_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_remove_master_key_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_remove_master_key_presult::~ThriftHiveMetastore_remove_master_key_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_remove_master_key_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_master_keys_args::~ThriftHiveMetastore_get_master_keys_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_master_keys_args::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 ThriftHiveMetastore_get_master_keys_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_master_keys_args"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_master_keys_pargs::~ThriftHiveMetastore_get_master_keys_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_master_keys_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_master_keys_pargs"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_master_keys_result::~ThriftHiveMetastore_get_master_keys_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_master_keys_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->success.clear(); - uint32_t _size1603; - ::apache::thrift::protocol::TType _etype1606; - xfer += iprot->readListBegin(_etype1606, _size1603); - this->success.resize(_size1603); - uint32_t _i1607; - for (_i1607 = 0; _i1607 < _size1603; ++_i1607) - { - xfer += iprot->readString(this->success[_i1607]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_master_keys_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_master_keys_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->success.size())); - std::vector ::const_iterator _iter1608; - for (_iter1608 = this->success.begin(); _iter1608 != this->success.end(); ++_iter1608) - { - xfer += oprot->writeString((*_iter1608)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_master_keys_presult::~ThriftHiveMetastore_get_master_keys_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_master_keys_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - (*(this->success)).clear(); - uint32_t _size1609; - ::apache::thrift::protocol::TType _etype1612; - xfer += iprot->readListBegin(_etype1612, _size1609); - (*(this->success)).resize(_size1609); - uint32_t _i1613; - for (_i1613 = 0; _i1613 < _size1609; ++_i1613) - { - xfer += iprot->readString((*(this->success))[_i1613]); - } - xfer += iprot->readListEnd(); - } - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_open_txns_args::~ThriftHiveMetastore_get_open_txns_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_open_txns_args::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 ThriftHiveMetastore_get_open_txns_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_open_txns_args"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_open_txns_pargs::~ThriftHiveMetastore_get_open_txns_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_open_txns_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_open_txns_pargs"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_open_txns_result::~ThriftHiveMetastore_get_open_txns_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_open_txns_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_open_txns_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_open_txns_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_open_txns_presult::~ThriftHiveMetastore_get_open_txns_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_open_txns_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_open_txns_info_args::~ThriftHiveMetastore_get_open_txns_info_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_open_txns_info_args::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 ThriftHiveMetastore_get_open_txns_info_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_open_txns_info_args"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_open_txns_info_pargs::~ThriftHiveMetastore_get_open_txns_info_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_open_txns_info_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_open_txns_info_pargs"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_open_txns_info_result::~ThriftHiveMetastore_get_open_txns_info_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_open_txns_info_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_open_txns_info_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_open_txns_info_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_open_txns_info_presult::~ThriftHiveMetastore_get_open_txns_info_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_open_txns_info_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_open_txns_args::~ThriftHiveMetastore_open_txns_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_open_txns_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->rqst.read(iprot); - this->__isset.rqst = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_open_txns_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_open_txns_args"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->rqst.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_open_txns_pargs::~ThriftHiveMetastore_open_txns_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_open_txns_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_open_txns_pargs"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->rqst)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_open_txns_result::~ThriftHiveMetastore_open_txns_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_open_txns_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_open_txns_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_open_txns_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_open_txns_presult::~ThriftHiveMetastore_open_txns_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_open_txns_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_abort_txn_args::~ThriftHiveMetastore_abort_txn_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_abort_txn_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->rqst.read(iprot); - this->__isset.rqst = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_abort_txn_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_abort_txn_args"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->rqst.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_abort_txn_pargs::~ThriftHiveMetastore_abort_txn_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_abort_txn_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_abort_txn_pargs"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->rqst)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_abort_txn_result::~ThriftHiveMetastore_abort_txn_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_abort_txn_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_abort_txn_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_abort_txn_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_abort_txn_presult::~ThriftHiveMetastore_abort_txn_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_abort_txn_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_abort_txns_args::~ThriftHiveMetastore_abort_txns_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_abort_txns_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->rqst.read(iprot); - this->__isset.rqst = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_abort_txns_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_abort_txns_args"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->rqst.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_abort_txns_pargs::~ThriftHiveMetastore_abort_txns_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_abort_txns_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_abort_txns_pargs"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->rqst)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_abort_txns_result::~ThriftHiveMetastore_abort_txns_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_abort_txns_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_abort_txns_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_abort_txns_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_abort_txns_presult::~ThriftHiveMetastore_abort_txns_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_abort_txns_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_commit_txn_args::~ThriftHiveMetastore_commit_txn_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_commit_txn_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->rqst.read(iprot); - this->__isset.rqst = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_commit_txn_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_commit_txn_args"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->rqst.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_commit_txn_pargs::~ThriftHiveMetastore_commit_txn_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_commit_txn_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_commit_txn_pargs"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->rqst)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_commit_txn_result::~ThriftHiveMetastore_commit_txn_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_commit_txn_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_commit_txn_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_commit_txn_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_commit_txn_presult::~ThriftHiveMetastore_commit_txn_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_commit_txn_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_lock_args::~ThriftHiveMetastore_lock_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_lock_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->rqst.read(iprot); - this->__isset.rqst = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_lock_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_lock_args"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->rqst.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_lock_pargs::~ThriftHiveMetastore_lock_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_lock_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_lock_pargs"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->rqst)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_lock_result::~ThriftHiveMetastore_lock_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_lock_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_lock_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_lock_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_lock_presult::~ThriftHiveMetastore_lock_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_lock_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_check_lock_args::~ThriftHiveMetastore_check_lock_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_check_lock_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->rqst.read(iprot); - this->__isset.rqst = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_check_lock_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_check_lock_args"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->rqst.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_check_lock_pargs::~ThriftHiveMetastore_check_lock_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_check_lock_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_check_lock_pargs"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->rqst)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_check_lock_result::~ThriftHiveMetastore_check_lock_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_check_lock_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_check_lock_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_check_lock_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_check_lock_presult::~ThriftHiveMetastore_check_lock_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_check_lock_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_unlock_args::~ThriftHiveMetastore_unlock_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_unlock_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->rqst.read(iprot); - this->__isset.rqst = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_unlock_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_unlock_args"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->rqst.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_unlock_pargs::~ThriftHiveMetastore_unlock_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_unlock_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_unlock_pargs"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->rqst)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_unlock_result::~ThriftHiveMetastore_unlock_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_unlock_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_unlock_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_unlock_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_unlock_presult::~ThriftHiveMetastore_unlock_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_unlock_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_show_locks_args::~ThriftHiveMetastore_show_locks_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_show_locks_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->rqst.read(iprot); - this->__isset.rqst = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_show_locks_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_show_locks_args"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->rqst.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_show_locks_pargs::~ThriftHiveMetastore_show_locks_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_show_locks_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_show_locks_pargs"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->rqst)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_show_locks_result::~ThriftHiveMetastore_show_locks_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_show_locks_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_show_locks_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_show_locks_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_show_locks_presult::~ThriftHiveMetastore_show_locks_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_show_locks_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_heartbeat_args::~ThriftHiveMetastore_heartbeat_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_heartbeat_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->ids.read(iprot); - this->__isset.ids = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_heartbeat_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_heartbeat_args"); - - xfer += oprot->writeFieldBegin("ids", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->ids.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_heartbeat_pargs::~ThriftHiveMetastore_heartbeat_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_heartbeat_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_heartbeat_pargs"); - - xfer += oprot->writeFieldBegin("ids", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->ids)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_heartbeat_result::~ThriftHiveMetastore_heartbeat_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_heartbeat_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_heartbeat_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_heartbeat_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_heartbeat_presult::~ThriftHiveMetastore_heartbeat_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_heartbeat_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_heartbeat_txn_range_args::~ThriftHiveMetastore_heartbeat_txn_range_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_heartbeat_txn_range_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->txns.read(iprot); - this->__isset.txns = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_heartbeat_txn_range_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_heartbeat_txn_range_args"); - - xfer += oprot->writeFieldBegin("txns", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->txns.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_heartbeat_txn_range_pargs::~ThriftHiveMetastore_heartbeat_txn_range_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_heartbeat_txn_range_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_heartbeat_txn_range_pargs"); - - xfer += oprot->writeFieldBegin("txns", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->txns)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_heartbeat_txn_range_result::~ThriftHiveMetastore_heartbeat_txn_range_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_heartbeat_txn_range_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_heartbeat_txn_range_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_heartbeat_txn_range_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_heartbeat_txn_range_presult::~ThriftHiveMetastore_heartbeat_txn_range_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_heartbeat_txn_range_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_compact_args::~ThriftHiveMetastore_compact_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_compact_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->rqst.read(iprot); - this->__isset.rqst = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_compact_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_compact_args"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->rqst.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_compact_pargs::~ThriftHiveMetastore_compact_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_compact_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_compact_pargs"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->rqst)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_compact_result::~ThriftHiveMetastore_compact_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_compact_result::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 ThriftHiveMetastore_compact_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_compact_result"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_compact_presult::~ThriftHiveMetastore_compact_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_compact_presult::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; -} - - -ThriftHiveMetastore_compact2_args::~ThriftHiveMetastore_compact2_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_compact2_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->rqst.read(iprot); - this->__isset.rqst = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_compact2_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_compact2_args"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->rqst.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_compact2_pargs::~ThriftHiveMetastore_compact2_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_compact2_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_compact2_pargs"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->rqst)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_compact2_result::~ThriftHiveMetastore_compact2_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_compact2_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_compact2_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_compact2_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_compact2_presult::~ThriftHiveMetastore_compact2_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_compact2_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_show_compact_args::~ThriftHiveMetastore_show_compact_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_show_compact_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->rqst.read(iprot); - this->__isset.rqst = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_show_compact_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_show_compact_args"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->rqst.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_show_compact_pargs::~ThriftHiveMetastore_show_compact_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_show_compact_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_show_compact_pargs"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->rqst)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_show_compact_result::~ThriftHiveMetastore_show_compact_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_show_compact_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_show_compact_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_show_compact_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_show_compact_presult::~ThriftHiveMetastore_show_compact_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_show_compact_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_add_dynamic_partitions_args::~ThriftHiveMetastore_add_dynamic_partitions_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_dynamic_partitions_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->rqst.read(iprot); - this->__isset.rqst = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_dynamic_partitions_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_dynamic_partitions_args"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->rqst.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_dynamic_partitions_pargs::~ThriftHiveMetastore_add_dynamic_partitions_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_dynamic_partitions_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_dynamic_partitions_pargs"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->rqst)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_dynamic_partitions_result::~ThriftHiveMetastore_add_dynamic_partitions_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_dynamic_partitions_result::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_add_dynamic_partitions_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_dynamic_partitions_result"); - - if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_add_dynamic_partitions_presult::~ThriftHiveMetastore_add_dynamic_partitions_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_add_dynamic_partitions_presult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_next_notification_args::~ThriftHiveMetastore_get_next_notification_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_next_notification_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->rqst.read(iprot); - this->__isset.rqst = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_next_notification_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_next_notification_args"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->rqst.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_next_notification_pargs::~ThriftHiveMetastore_get_next_notification_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_next_notification_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_next_notification_pargs"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->rqst)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_next_notification_result::~ThriftHiveMetastore_get_next_notification_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_next_notification_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_next_notification_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_next_notification_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_next_notification_presult::~ThriftHiveMetastore_get_next_notification_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_next_notification_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_current_notificationEventId_args::~ThriftHiveMetastore_get_current_notificationEventId_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_current_notificationEventId_args::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 ThriftHiveMetastore_get_current_notificationEventId_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_current_notificationEventId_args"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_current_notificationEventId_pargs::~ThriftHiveMetastore_get_current_notificationEventId_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_current_notificationEventId_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_current_notificationEventId_pargs"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_current_notificationEventId_result::~ThriftHiveMetastore_get_current_notificationEventId_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_current_notificationEventId_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_current_notificationEventId_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_current_notificationEventId_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_current_notificationEventId_presult::~ThriftHiveMetastore_get_current_notificationEventId_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_current_notificationEventId_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_notification_events_count_args::~ThriftHiveMetastore_get_notification_events_count_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_notification_events_count_args::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; - } - switch (fid) - { - case -1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->rqst.read(iprot); - this->__isset.rqst = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_notification_events_count_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_notification_events_count_args"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, -1); - xfer += this->rqst.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_notification_events_count_pargs::~ThriftHiveMetastore_get_notification_events_count_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_notification_events_count_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_notification_events_count_pargs"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, -1); - xfer += (*(this->rqst)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_notification_events_count_result::~ThriftHiveMetastore_get_notification_events_count_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_notification_events_count_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_notification_events_count_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_notification_events_count_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_notification_events_count_presult::~ThriftHiveMetastore_get_notification_events_count_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_notification_events_count_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_fire_listener_event_args::~ThriftHiveMetastore_fire_listener_event_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_fire_listener_event_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->rqst.read(iprot); - this->__isset.rqst = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_fire_listener_event_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_fire_listener_event_args"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->rqst.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_fire_listener_event_pargs::~ThriftHiveMetastore_fire_listener_event_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_fire_listener_event_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_fire_listener_event_pargs"); - - xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->rqst)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_fire_listener_event_result::~ThriftHiveMetastore_fire_listener_event_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_fire_listener_event_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_fire_listener_event_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_fire_listener_event_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_fire_listener_event_presult::~ThriftHiveMetastore_fire_listener_event_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_fire_listener_event_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_flushCache_args::~ThriftHiveMetastore_flushCache_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_flushCache_args::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 ThriftHiveMetastore_flushCache_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_flushCache_args"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_flushCache_pargs::~ThriftHiveMetastore_flushCache_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_flushCache_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_flushCache_pargs"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_flushCache_result::~ThriftHiveMetastore_flushCache_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_flushCache_result::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 ThriftHiveMetastore_flushCache_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_flushCache_result"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_flushCache_presult::~ThriftHiveMetastore_flushCache_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_flushCache_presult::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; -} - - -ThriftHiveMetastore_cm_recycle_args::~ThriftHiveMetastore_cm_recycle_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_cm_recycle_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_cm_recycle_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_cm_recycle_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_cm_recycle_pargs::~ThriftHiveMetastore_cm_recycle_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_cm_recycle_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_cm_recycle_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_cm_recycle_result::~ThriftHiveMetastore_cm_recycle_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_cm_recycle_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_cm_recycle_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_cm_recycle_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_cm_recycle_presult::~ThriftHiveMetastore_cm_recycle_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_cm_recycle_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_file_metadata_by_expr_args::~ThriftHiveMetastore_get_file_metadata_by_expr_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->req.read(iprot); - this->__isset.req = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_by_expr_args"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->req.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_file_metadata_by_expr_pargs::~ThriftHiveMetastore_get_file_metadata_by_expr_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_by_expr_pargs"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->req)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_file_metadata_by_expr_result::~ThriftHiveMetastore_get_file_metadata_by_expr_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_by_expr_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_file_metadata_by_expr_presult::~ThriftHiveMetastore_get_file_metadata_by_expr_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_file_metadata_args::~ThriftHiveMetastore_get_file_metadata_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_file_metadata_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->req.read(iprot); - this->__isset.req = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_file_metadata_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_args"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->req.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_file_metadata_pargs::~ThriftHiveMetastore_get_file_metadata_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_file_metadata_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_pargs"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->req)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_file_metadata_result::~ThriftHiveMetastore_get_file_metadata_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_file_metadata_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_file_metadata_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_file_metadata_presult::~ThriftHiveMetastore_get_file_metadata_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_file_metadata_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_put_file_metadata_args::~ThriftHiveMetastore_put_file_metadata_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_put_file_metadata_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->req.read(iprot); - this->__isset.req = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_put_file_metadata_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_put_file_metadata_args"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->req.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_put_file_metadata_pargs::~ThriftHiveMetastore_put_file_metadata_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_put_file_metadata_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_put_file_metadata_pargs"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->req)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_put_file_metadata_result::~ThriftHiveMetastore_put_file_metadata_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_put_file_metadata_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_put_file_metadata_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_put_file_metadata_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_put_file_metadata_presult::~ThriftHiveMetastore_put_file_metadata_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_put_file_metadata_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_clear_file_metadata_args::~ThriftHiveMetastore_clear_file_metadata_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_clear_file_metadata_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->req.read(iprot); - this->__isset.req = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_clear_file_metadata_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_clear_file_metadata_args"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->req.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_clear_file_metadata_pargs::~ThriftHiveMetastore_clear_file_metadata_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_clear_file_metadata_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_clear_file_metadata_pargs"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->req)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_clear_file_metadata_result::~ThriftHiveMetastore_clear_file_metadata_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_clear_file_metadata_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_clear_file_metadata_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_clear_file_metadata_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_clear_file_metadata_presult::~ThriftHiveMetastore_clear_file_metadata_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_clear_file_metadata_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_cache_file_metadata_args::~ThriftHiveMetastore_cache_file_metadata_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_cache_file_metadata_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->req.read(iprot); - this->__isset.req = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_cache_file_metadata_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_cache_file_metadata_args"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->req.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_cache_file_metadata_pargs::~ThriftHiveMetastore_cache_file_metadata_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_cache_file_metadata_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_cache_file_metadata_pargs"); - - xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->req)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_cache_file_metadata_result::~ThriftHiveMetastore_cache_file_metadata_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_cache_file_metadata_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_cache_file_metadata_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_cache_file_metadata_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_cache_file_metadata_presult::~ThriftHiveMetastore_cache_file_metadata_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_cache_file_metadata_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_metastore_db_uuid_args::~ThriftHiveMetastore_get_metastore_db_uuid_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_metastore_db_uuid_args::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 ThriftHiveMetastore_get_metastore_db_uuid_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_metastore_db_uuid_args"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_metastore_db_uuid_pargs::~ThriftHiveMetastore_get_metastore_db_uuid_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_metastore_db_uuid_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_metastore_db_uuid_pargs"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_metastore_db_uuid_result::~ThriftHiveMetastore_get_metastore_db_uuid_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_metastore_db_uuid_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_metastore_db_uuid_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_metastore_db_uuid_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0); - xfer += oprot->writeString(this->success); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_metastore_db_uuid_presult::~ThriftHiveMetastore_get_metastore_db_uuid_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_metastore_db_uuid_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString((*(this->success))); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_create_resource_plan_args::~ThriftHiveMetastore_create_resource_plan_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_resource_plan_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_resource_plan_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_resource_plan_pargs::~ThriftHiveMetastore_create_resource_plan_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_resource_plan_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_resource_plan_result::~ThriftHiveMetastore_create_resource_plan_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_resource_plan_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_resource_plan_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_resource_plan_presult::~ThriftHiveMetastore_create_resource_plan_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_resource_plan_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_resource_plan_args::~ThriftHiveMetastore_get_resource_plan_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_resource_plan_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_resource_plan_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_resource_plan_pargs::~ThriftHiveMetastore_get_resource_plan_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_resource_plan_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_resource_plan_result::~ThriftHiveMetastore_get_resource_plan_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_resource_plan_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_resource_plan_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_resource_plan_presult::~ThriftHiveMetastore_get_resource_plan_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_resource_plan_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_active_resource_plan_args::~ThriftHiveMetastore_get_active_resource_plan_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_active_resource_plan_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_active_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_active_resource_plan_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_active_resource_plan_pargs::~ThriftHiveMetastore_get_active_resource_plan_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_active_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_active_resource_plan_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_active_resource_plan_result::~ThriftHiveMetastore_get_active_resource_plan_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_active_resource_plan_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_active_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_active_resource_plan_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_active_resource_plan_presult::~ThriftHiveMetastore_get_active_resource_plan_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_active_resource_plan_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_all_resource_plans_args::~ThriftHiveMetastore_get_all_resource_plans_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_resource_plans_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_all_resource_plans_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_resource_plans_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_all_resource_plans_pargs::~ThriftHiveMetastore_get_all_resource_plans_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_resource_plans_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_resource_plans_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_all_resource_plans_result::~ThriftHiveMetastore_get_all_resource_plans_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_resource_plans_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_all_resource_plans_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_resource_plans_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_all_resource_plans_presult::~ThriftHiveMetastore_get_all_resource_plans_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_all_resource_plans_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_alter_resource_plan_args::~ThriftHiveMetastore_alter_resource_plan_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_resource_plan_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_resource_plan_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_resource_plan_pargs::~ThriftHiveMetastore_alter_resource_plan_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_resource_plan_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_resource_plan_result::~ThriftHiveMetastore_alter_resource_plan_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_resource_plan_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_resource_plan_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_resource_plan_presult::~ThriftHiveMetastore_alter_resource_plan_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_resource_plan_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_validate_resource_plan_args::~ThriftHiveMetastore_validate_resource_plan_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_validate_resource_plan_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_validate_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_validate_resource_plan_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_validate_resource_plan_pargs::~ThriftHiveMetastore_validate_resource_plan_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_validate_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_validate_resource_plan_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_validate_resource_plan_result::~ThriftHiveMetastore_validate_resource_plan_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_validate_resource_plan_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_validate_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_validate_resource_plan_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_validate_resource_plan_presult::~ThriftHiveMetastore_validate_resource_plan_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_validate_resource_plan_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_drop_resource_plan_args::~ThriftHiveMetastore_drop_resource_plan_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_resource_plan_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_resource_plan_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_resource_plan_pargs::~ThriftHiveMetastore_drop_resource_plan_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_resource_plan_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_resource_plan_result::~ThriftHiveMetastore_drop_resource_plan_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_resource_plan_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_resource_plan_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_resource_plan_presult::~ThriftHiveMetastore_drop_resource_plan_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_resource_plan_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_create_wm_trigger_args::~ThriftHiveMetastore_create_wm_trigger_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_wm_trigger_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_wm_trigger_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_wm_trigger_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_wm_trigger_pargs::~ThriftHiveMetastore_create_wm_trigger_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_wm_trigger_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_wm_trigger_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_wm_trigger_result::~ThriftHiveMetastore_create_wm_trigger_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_wm_trigger_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_create_wm_trigger_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_wm_trigger_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o4) { - xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->o4.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_create_wm_trigger_presult::~ThriftHiveMetastore_create_wm_trigger_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_create_wm_trigger_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o4.read(iprot); - this->__isset.o4 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_alter_wm_trigger_args::~ThriftHiveMetastore_alter_wm_trigger_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_wm_trigger_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_wm_trigger_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_wm_trigger_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_wm_trigger_pargs::~ThriftHiveMetastore_alter_wm_trigger_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_wm_trigger_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_wm_trigger_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_wm_trigger_result::~ThriftHiveMetastore_alter_wm_trigger_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_wm_trigger_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_alter_wm_trigger_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_wm_trigger_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_alter_wm_trigger_presult::~ThriftHiveMetastore_alter_wm_trigger_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_alter_wm_trigger_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_drop_wm_trigger_args::~ThriftHiveMetastore_drop_wm_trigger_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_wm_trigger_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_wm_trigger_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_wm_trigger_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_wm_trigger_pargs::~ThriftHiveMetastore_drop_wm_trigger_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_wm_trigger_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_wm_trigger_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_wm_trigger_result::~ThriftHiveMetastore_drop_wm_trigger_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_wm_trigger_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_drop_wm_trigger_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_wm_trigger_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o3) { - xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->o3.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_drop_wm_trigger_presult::~ThriftHiveMetastore_drop_wm_trigger_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_drop_wm_trigger_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o3.read(iprot); - this->__isset.o3 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - - -ThriftHiveMetastore_get_triggers_for_resourceplan_args::~ThriftHiveMetastore_get_triggers_for_resourceplan_args() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_triggers_for_resourceplan_args::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->request.read(iprot); - this->__isset.request = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_triggers_for_resourceplan_args::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_triggers_for_resourceplan_args"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->request.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_triggers_for_resourceplan_pargs::~ThriftHiveMetastore_get_triggers_for_resourceplan_pargs() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_triggers_for_resourceplan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_triggers_for_resourceplan_pargs"); - - xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += (*(this->request)).write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_triggers_for_resourceplan_result::~ThriftHiveMetastore_get_triggers_for_resourceplan_result() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_triggers_for_resourceplan_result::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->success.read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ThriftHiveMetastore_get_triggers_for_resourceplan_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - - uint32_t xfer = 0; - - xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_triggers_for_resourceplan_result"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); - xfer += this->success.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o1) { - xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->o1.write(oprot); - xfer += oprot->writeFieldEnd(); - } else if (this->__isset.o2) { - xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->o2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - - -ThriftHiveMetastore_get_triggers_for_resourceplan_presult::~ThriftHiveMetastore_get_triggers_for_resourceplan_presult() noexcept { -} - - -uint32_t ThriftHiveMetastore_get_triggers_for_resourceplan_presult::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; - } - switch (fid) - { - case 0: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += (*(this->success)).read(iprot); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o1.read(iprot); - this->__isset.o1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->o2.read(iprot); - this->__isset.o2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -void ThriftHiveMetastoreClient::getMetaConf(std::string& _return, const std::string& key) -{ - send_getMetaConf(key); - recv_getMetaConf(_return); -} - -void ThriftHiveMetastoreClient::send_getMetaConf(const std::string& key) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("getMetaConf", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_getMetaConf_pargs args; - args.key = &key; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_getMetaConf(std::string& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("getMetaConf") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_getMetaConf_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "getMetaConf failed: unknown result"); -} - -void ThriftHiveMetastoreClient::setMetaConf(const std::string& key, const std::string& value) -{ - send_setMetaConf(key, value); - recv_setMetaConf(); -} - -void ThriftHiveMetastoreClient::send_setMetaConf(const std::string& key, const std::string& value) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("setMetaConf", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_setMetaConf_pargs args; - args.key = &key; - args.value = &value; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_setMetaConf() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("setMetaConf") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_setMetaConf_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - return; -} - -void ThriftHiveMetastoreClient::create_database(const Database& database) -{ - send_create_database(database); - recv_create_database(); -} - -void ThriftHiveMetastoreClient::send_create_database(const Database& database) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("create_database", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_database_pargs args; - args.database = &database; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_create_database() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_database") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_create_database_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - return; -} - -void ThriftHiveMetastoreClient::get_database(Database& _return, const std::string& name) -{ - send_get_database(name); - recv_get_database(_return); -} - -void ThriftHiveMetastoreClient::send_get_database(const std::string& name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_database", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_database_pargs args; - args.name = &name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_database(Database& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_database") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_database_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_database failed: unknown result"); -} - -void ThriftHiveMetastoreClient::drop_database(const std::string& name, const bool deleteData, const bool cascade) -{ - send_drop_database(name, deleteData, cascade); - recv_drop_database(); -} - -void ThriftHiveMetastoreClient::send_drop_database(const std::string& name, const bool deleteData, const bool cascade) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_database", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_database_pargs args; - args.name = &name; - args.deleteData = &deleteData; - args.cascade = &cascade; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_drop_database() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_database") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_drop_database_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - return; -} - -void ThriftHiveMetastoreClient::get_databases(std::vector & _return, const std::string& pattern) -{ - send_get_databases(pattern); - recv_get_databases(_return); -} - -void ThriftHiveMetastoreClient::send_get_databases(const std::string& pattern) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_databases", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_databases_pargs args; - args.pattern = &pattern; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_databases(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_databases") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_databases_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_databases failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_all_databases(std::vector & _return) -{ - send_get_all_databases(); - recv_get_all_databases(_return); -} - -void ThriftHiveMetastoreClient::send_get_all_databases() -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_all_databases", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_all_databases_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_all_databases(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_all_databases") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_all_databases_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_databases failed: unknown result"); -} - -void ThriftHiveMetastoreClient::alter_database(const std::string& dbname, const Database& db) -{ - send_alter_database(dbname, db); - recv_alter_database(); -} - -void ThriftHiveMetastoreClient::send_alter_database(const std::string& dbname, const Database& db) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_database", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_database_pargs args; - args.dbname = &dbname; - args.db = &db; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_alter_database() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_database") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_alter_database_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -void ThriftHiveMetastoreClient::get_type(Type& _return, const std::string& name) -{ - send_get_type(name); - recv_get_type(_return); -} - -void ThriftHiveMetastoreClient::send_get_type(const std::string& name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_type", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_type_pargs args; - args.name = &name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_type(Type& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_type") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_type_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_type failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::create_type(const Type& type) -{ - send_create_type(type); - return recv_create_type(); -} - -void ThriftHiveMetastoreClient::send_create_type(const Type& type) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("create_type", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_type_pargs args; - args.type = &type; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_create_type() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_type") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_create_type_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_type failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::drop_type(const std::string& type) -{ - send_drop_type(type); - return recv_drop_type(); -} - -void ThriftHiveMetastoreClient::send_drop_type(const std::string& type) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_type", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_type_pargs args; - args.type = &type; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_drop_type() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_type") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_drop_type_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_type failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_type_all(std::map & _return, const std::string& name) -{ - send_get_type_all(name); - recv_get_type_all(_return); -} - -void ThriftHiveMetastoreClient::send_get_type_all(const std::string& name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_type_all", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_type_all_pargs args; - args.name = &name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_type_all(std::map & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_type_all") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_type_all_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_type_all failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_fields(std::vector & _return, const std::string& db_name, const std::string& table_name) -{ - send_get_fields(db_name, table_name); - recv_get_fields(_return); -} - -void ThriftHiveMetastoreClient::send_get_fields(const std::string& db_name, const std::string& table_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_fields", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_fields_pargs args; - args.db_name = &db_name; - args.table_name = &table_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_fields(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_fields") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_fields_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_fields failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_fields_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) -{ - send_get_fields_with_environment_context(db_name, table_name, environment_context); - recv_get_fields_with_environment_context(_return); -} - -void ThriftHiveMetastoreClient::send_get_fields_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_fields_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_fields_with_environment_context_pargs args; - args.db_name = &db_name; - args.table_name = &table_name; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_fields_with_environment_context(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_fields_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_fields_with_environment_context_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_fields_with_environment_context failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_schema(std::vector & _return, const std::string& db_name, const std::string& table_name) -{ - send_get_schema(db_name, table_name); - recv_get_schema(_return); -} - -void ThriftHiveMetastoreClient::send_get_schema(const std::string& db_name, const std::string& table_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_schema", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_schema_pargs args; - args.db_name = &db_name; - args.table_name = &table_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_schema(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_schema") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_schema_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_schema failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_schema_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) -{ - send_get_schema_with_environment_context(db_name, table_name, environment_context); - recv_get_schema_with_environment_context(_return); -} - -void ThriftHiveMetastoreClient::send_get_schema_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_schema_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_schema_with_environment_context_pargs args; - args.db_name = &db_name; - args.table_name = &table_name; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_schema_with_environment_context(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_schema_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_schema_with_environment_context_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_schema_with_environment_context failed: unknown result"); -} - -void ThriftHiveMetastoreClient::create_table(const Table& tbl) -{ - send_create_table(tbl); - recv_create_table(); -} - -void ThriftHiveMetastoreClient::send_create_table(const Table& tbl) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("create_table", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_table_pargs args; - args.tbl = &tbl; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_create_table() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_table") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_create_table_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } - return; -} - -void ThriftHiveMetastoreClient::create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) -{ - send_create_table_with_environment_context(tbl, environment_context); - recv_create_table_with_environment_context(); -} - -void ThriftHiveMetastoreClient::send_create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("create_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_table_with_environment_context_pargs args; - args.tbl = &tbl; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_create_table_with_environment_context() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_table_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_create_table_with_environment_context_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } - return; -} - -void ThriftHiveMetastoreClient::create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints) -{ - send_create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints); - recv_create_table_with_constraints(); -} - -void ThriftHiveMetastoreClient::send_create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("create_table_with_constraints", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_table_with_constraints_pargs args; - args.tbl = &tbl; - args.primaryKeys = &primaryKeys; - args.foreignKeys = &foreignKeys; - args.uniqueConstraints = &uniqueConstraints; - args.notNullConstraints = ¬NullConstraints; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_create_table_with_constraints() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_table_with_constraints") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_create_table_with_constraints_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } - return; -} - -void ThriftHiveMetastoreClient::drop_constraint(const DropConstraintRequest& req) -{ - send_drop_constraint(req); - recv_drop_constraint(); -} - -void ThriftHiveMetastoreClient::send_drop_constraint(const DropConstraintRequest& req) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_constraint", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_constraint_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_drop_constraint() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_constraint") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_drop_constraint_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o3) { - throw result.o3; - } - return; -} - -void ThriftHiveMetastoreClient::add_primary_key(const AddPrimaryKeyRequest& req) -{ - send_add_primary_key(req); - recv_add_primary_key(); -} - -void ThriftHiveMetastoreClient::send_add_primary_key(const AddPrimaryKeyRequest& req) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("add_primary_key", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_primary_key_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_add_primary_key() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_primary_key") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_add_primary_key_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -void ThriftHiveMetastoreClient::add_foreign_key(const AddForeignKeyRequest& req) -{ - send_add_foreign_key(req); - recv_add_foreign_key(); -} - -void ThriftHiveMetastoreClient::send_add_foreign_key(const AddForeignKeyRequest& req) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("add_foreign_key", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_foreign_key_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_add_foreign_key() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_foreign_key") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_add_foreign_key_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -void ThriftHiveMetastoreClient::add_unique_constraint(const AddUniqueConstraintRequest& req) -{ - send_add_unique_constraint(req); - recv_add_unique_constraint(); -} - -void ThriftHiveMetastoreClient::send_add_unique_constraint(const AddUniqueConstraintRequest& req) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("add_unique_constraint", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_unique_constraint_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_add_unique_constraint() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_unique_constraint") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_add_unique_constraint_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -void ThriftHiveMetastoreClient::add_not_null_constraint(const AddNotNullConstraintRequest& req) -{ - send_add_not_null_constraint(req); - recv_add_not_null_constraint(); -} - -void ThriftHiveMetastoreClient::send_add_not_null_constraint(const AddNotNullConstraintRequest& req) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("add_not_null_constraint", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_not_null_constraint_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_add_not_null_constraint() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_not_null_constraint") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_add_not_null_constraint_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -void ThriftHiveMetastoreClient::drop_table(const std::string& dbname, const std::string& name, const bool deleteData) -{ - send_drop_table(dbname, name, deleteData); - recv_drop_table(); -} - -void ThriftHiveMetastoreClient::send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_table", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_table_pargs args; - args.dbname = &dbname; - args.name = &name; - args.deleteData = &deleteData; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_drop_table() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_table") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_drop_table_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o3) { - throw result.o3; - } - return; -} - -void ThriftHiveMetastoreClient::drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) -{ - send_drop_table_with_environment_context(dbname, name, deleteData, environment_context); - recv_drop_table_with_environment_context(); -} - -void ThriftHiveMetastoreClient::send_drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_table_with_environment_context_pargs args; - args.dbname = &dbname; - args.name = &name; - args.deleteData = &deleteData; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_drop_table_with_environment_context() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_table_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_drop_table_with_environment_context_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o3) { - throw result.o3; - } - return; -} - -void ThriftHiveMetastoreClient::truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames) -{ - send_truncate_table(dbName, tableName, partNames); - recv_truncate_table(); -} - -void ThriftHiveMetastoreClient::send_truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("truncate_table", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_truncate_table_pargs args; - args.dbName = &dbName; - args.tableName = &tableName; - args.partNames = &partNames; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_truncate_table() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("truncate_table") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_truncate_table_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - return; -} - -void ThriftHiveMetastoreClient::get_tables(std::vector & _return, const std::string& db_name, const std::string& pattern) -{ - send_get_tables(db_name, pattern); - recv_get_tables(_return); -} - -void ThriftHiveMetastoreClient::send_get_tables(const std::string& db_name, const std::string& pattern) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_tables", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_tables_pargs args; - args.db_name = &db_name; - args.pattern = &pattern; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_tables(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_tables") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_tables_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_tables failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_tables_by_type(std::vector & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType) -{ - send_get_tables_by_type(db_name, pattern, tableType); - recv_get_tables_by_type(_return); -} - -void ThriftHiveMetastoreClient::send_get_tables_by_type(const std::string& db_name, const std::string& pattern, const std::string& tableType) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_tables_by_type", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_tables_by_type_pargs args; - args.db_name = &db_name; - args.pattern = &pattern; - args.tableType = &tableType; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_tables_by_type(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_tables_by_type") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_tables_by_type_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_tables_by_type failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_table_meta(std::vector & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types) -{ - send_get_table_meta(db_patterns, tbl_patterns, tbl_types); - recv_get_table_meta(_return); -} - -void ThriftHiveMetastoreClient::send_get_table_meta(const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_table_meta", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_table_meta_pargs args; - args.db_patterns = &db_patterns; - args.tbl_patterns = &tbl_patterns; - args.tbl_types = &tbl_types; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_table_meta(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_table_meta") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_table_meta_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_meta failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_all_tables(std::vector & _return, const std::string& db_name) -{ - send_get_all_tables(db_name); - recv_get_all_tables(_return); -} - -void ThriftHiveMetastoreClient::send_get_all_tables(const std::string& db_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_all_tables", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_all_tables_pargs args; - args.db_name = &db_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_all_tables(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_all_tables") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_all_tables_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_tables failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_table(Table& _return, const std::string& dbname, const std::string& tbl_name) -{ - send_get_table(dbname, tbl_name); - recv_get_table(_return); -} - -void ThriftHiveMetastoreClient::send_get_table(const std::string& dbname, const std::string& tbl_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_table", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_table_pargs args; - args.dbname = &dbname; - args.tbl_name = &tbl_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_table(Table& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_table") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_table_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_table_objects_by_name(std::vector
& _return, const std::string& dbname, const std::vector & tbl_names) -{ - send_get_table_objects_by_name(dbname, tbl_names); - recv_get_table_objects_by_name(_return); -} - -void ThriftHiveMetastoreClient::send_get_table_objects_by_name(const std::string& dbname, const std::vector & tbl_names) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_table_objects_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_table_objects_by_name_pargs args; - args.dbname = &dbname; - args.tbl_names = &tbl_names; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_table_objects_by_name(std::vector
& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_table_objects_by_name") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_table_objects_by_name_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_objects_by_name failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_table_req(GetTableResult& _return, const GetTableRequest& req) -{ - send_get_table_req(req); - recv_get_table_req(_return); -} - -void ThriftHiveMetastoreClient::send_get_table_req(const GetTableRequest& req) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_table_req", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_table_req_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_table_req(GetTableResult& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_table_req") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_table_req_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_req failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req) -{ - send_get_table_objects_by_name_req(req); - recv_get_table_objects_by_name_req(_return); -} - -void ThriftHiveMetastoreClient::send_get_table_objects_by_name_req(const GetTablesRequest& req) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_table_objects_by_name_req", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_table_objects_by_name_req_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_table_objects_by_name_req(GetTablesResult& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_table_objects_by_name_req") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_table_objects_by_name_req_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_objects_by_name_req failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_table_names_by_filter(std::vector & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables) -{ - send_get_table_names_by_filter(dbname, filter, max_tables); - recv_get_table_names_by_filter(_return); -} - -void ThriftHiveMetastoreClient::send_get_table_names_by_filter(const std::string& dbname, const std::string& filter, const int16_t max_tables) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_table_names_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_table_names_by_filter_pargs args; - args.dbname = &dbname; - args.filter = &filter; - args.max_tables = &max_tables; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_table_names_by_filter(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_table_names_by_filter") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_table_names_by_filter_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_names_by_filter failed: unknown result"); -} - -void ThriftHiveMetastoreClient::alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) -{ - send_alter_table(dbname, tbl_name, new_tbl); - recv_alter_table(); -} - -void ThriftHiveMetastoreClient::send_alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_table", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_table_pargs args; - args.dbname = &dbname; - args.tbl_name = &tbl_name; - args.new_tbl = &new_tbl; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_alter_table() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_table") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_alter_table_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -void ThriftHiveMetastoreClient::alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) -{ - send_alter_table_with_environment_context(dbname, tbl_name, new_tbl, environment_context); - recv_alter_table_with_environment_context(); -} - -void ThriftHiveMetastoreClient::send_alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_table_with_environment_context_pargs args; - args.dbname = &dbname; - args.tbl_name = &tbl_name; - args.new_tbl = &new_tbl; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_alter_table_with_environment_context() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_table_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_alter_table_with_environment_context_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -void ThriftHiveMetastoreClient::alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade) -{ - send_alter_table_with_cascade(dbname, tbl_name, new_tbl, cascade); - recv_alter_table_with_cascade(); -} - -void ThriftHiveMetastoreClient::send_alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_table_with_cascade", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_table_with_cascade_pargs args; - args.dbname = &dbname; - args.tbl_name = &tbl_name; - args.new_tbl = &new_tbl; - args.cascade = &cascade; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_alter_table_with_cascade() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_table_with_cascade") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_alter_table_with_cascade_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -void ThriftHiveMetastoreClient::add_partition(Partition& _return, const Partition& new_part) -{ - send_add_partition(new_part); - recv_add_partition(_return); -} - -void ThriftHiveMetastoreClient::send_add_partition(const Partition& new_part) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("add_partition", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_partition_pargs args; - args.new_part = &new_part; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_add_partition(Partition& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_partition") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_add_partition_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partition failed: unknown result"); -} - -void ThriftHiveMetastoreClient::add_partition_with_environment_context(Partition& _return, const Partition& new_part, const EnvironmentContext& environment_context) -{ - send_add_partition_with_environment_context(new_part, environment_context); - recv_add_partition_with_environment_context(_return); -} - -void ThriftHiveMetastoreClient::send_add_partition_with_environment_context(const Partition& new_part, const EnvironmentContext& environment_context) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("add_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_partition_with_environment_context_pargs args; - args.new_part = &new_part; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_add_partition_with_environment_context(Partition& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_partition_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_add_partition_with_environment_context_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partition_with_environment_context failed: unknown result"); -} - -int32_t ThriftHiveMetastoreClient::add_partitions(const std::vector & new_parts) -{ - send_add_partitions(new_parts); - return recv_add_partitions(); -} - -void ThriftHiveMetastoreClient::send_add_partitions(const std::vector & new_parts) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("add_partitions", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_partitions_pargs args; - args.new_parts = &new_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -int32_t ThriftHiveMetastoreClient::recv_add_partitions() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_partitions") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - int32_t _return; - ThriftHiveMetastore_add_partitions_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partitions failed: unknown result"); -} - -int32_t ThriftHiveMetastoreClient::add_partitions_pspec(const std::vector & new_parts) -{ - send_add_partitions_pspec(new_parts); - return recv_add_partitions_pspec(); -} - -void ThriftHiveMetastoreClient::send_add_partitions_pspec(const std::vector & new_parts) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("add_partitions_pspec", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_partitions_pspec_pargs args; - args.new_parts = &new_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -int32_t ThriftHiveMetastoreClient::recv_add_partitions_pspec() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_partitions_pspec") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - int32_t _return; - ThriftHiveMetastore_add_partitions_pspec_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partitions_pspec failed: unknown result"); -} - -void ThriftHiveMetastoreClient::append_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) -{ - send_append_partition(db_name, tbl_name, part_vals); - recv_append_partition(_return); -} - -void ThriftHiveMetastoreClient::send_append_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("append_partition", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_append_partition_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_append_partition(Partition& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("append_partition") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_append_partition_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition failed: unknown result"); -} - -void ThriftHiveMetastoreClient::add_partitions_req(AddPartitionsResult& _return, const AddPartitionsRequest& request) -{ - send_add_partitions_req(request); - recv_add_partitions_req(_return); -} - -void ThriftHiveMetastoreClient::send_add_partitions_req(const AddPartitionsRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("add_partitions_req", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_partitions_req_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_add_partitions_req(AddPartitionsResult& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_partitions_req") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_add_partitions_req_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partitions_req failed: unknown result"); -} - -void ThriftHiveMetastoreClient::append_partition_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context) -{ - send_append_partition_with_environment_context(db_name, tbl_name, part_vals, environment_context); - recv_append_partition_with_environment_context(_return); -} - -void ThriftHiveMetastoreClient::send_append_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_append_partition_with_environment_context_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_append_partition_with_environment_context(Partition& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("append_partition_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_append_partition_with_environment_context_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition_with_environment_context failed: unknown result"); -} - -void ThriftHiveMetastoreClient::append_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) -{ - send_append_partition_by_name(db_name, tbl_name, part_name); - recv_append_partition_by_name(_return); -} - -void ThriftHiveMetastoreClient::send_append_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("append_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_append_partition_by_name_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_append_partition_by_name(Partition& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("append_partition_by_name") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_append_partition_by_name_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition_by_name failed: unknown result"); -} - -void ThriftHiveMetastoreClient::append_partition_by_name_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context) -{ - send_append_partition_by_name_with_environment_context(db_name, tbl_name, part_name, environment_context); - recv_append_partition_by_name_with_environment_context(_return); -} - -void ThriftHiveMetastoreClient::send_append_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_append_partition_by_name_with_environment_context(Partition& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("append_partition_by_name_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition_by_name_with_environment_context failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) -{ - send_drop_partition(db_name, tbl_name, part_vals, deleteData); - return recv_drop_partition(); -} - -void ThriftHiveMetastoreClient::send_drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_partition", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_partition_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.deleteData = &deleteData; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_drop_partition() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_partition") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_drop_partition_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context) -{ - send_drop_partition_with_environment_context(db_name, tbl_name, part_vals, deleteData, environment_context); - return recv_drop_partition_with_environment_context(); -} - -void ThriftHiveMetastoreClient::send_drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_partition_with_environment_context_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.deleteData = &deleteData; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_drop_partition_with_environment_context() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_partition_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_drop_partition_with_environment_context_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_with_environment_context failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData) -{ - send_drop_partition_by_name(db_name, tbl_name, part_name, deleteData); - return recv_drop_partition_by_name(); -} - -void ThriftHiveMetastoreClient::send_drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_partition_by_name_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.deleteData = &deleteData; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_drop_partition_by_name() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_partition_by_name") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_drop_partition_by_name_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_by_name failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context) -{ - send_drop_partition_by_name_with_environment_context(db_name, tbl_name, part_name, deleteData, environment_context); - return recv_drop_partition_by_name_with_environment_context(); -} - -void ThriftHiveMetastoreClient::send_drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.deleteData = &deleteData; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_drop_partition_by_name_with_environment_context() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_partition_by_name_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_by_name_with_environment_context failed: unknown result"); -} - -void ThriftHiveMetastoreClient::drop_partitions_req(DropPartitionsResult& _return, const DropPartitionsRequest& req) -{ - send_drop_partitions_req(req); - recv_drop_partitions_req(_return); -} - -void ThriftHiveMetastoreClient::send_drop_partitions_req(const DropPartitionsRequest& req) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_partitions_req", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_partitions_req_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_drop_partitions_req(DropPartitionsResult& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_partitions_req") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_drop_partitions_req_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partitions_req failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) -{ - send_get_partition(db_name, tbl_name, part_vals); - recv_get_partition(_return); -} - -void ThriftHiveMetastoreClient::send_get_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partition_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_partition(Partition& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partition") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_partition_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition failed: unknown result"); -} - -void ThriftHiveMetastoreClient::exchange_partition(Partition& _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) -{ - send_exchange_partition(partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); - recv_exchange_partition(_return); -} - -void ThriftHiveMetastoreClient::send_exchange_partition(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_exchange_partition_pargs args; - args.partitionSpecs = &partitionSpecs; - args.source_db = &source_db; - args.source_table_name = &source_table_name; - args.dest_db = &dest_db; - args.dest_table_name = &dest_table_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_exchange_partition(Partition& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("exchange_partition") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_exchange_partition_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "exchange_partition failed: unknown result"); -} - -void ThriftHiveMetastoreClient::exchange_partitions(std::vector & _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) -{ - send_exchange_partitions(partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); - recv_exchange_partitions(_return); -} - -void ThriftHiveMetastoreClient::send_exchange_partitions(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("exchange_partitions", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_exchange_partitions_pargs args; - args.partitionSpecs = &partitionSpecs; - args.source_db = &source_db; - args.source_table_name = &source_table_name; - args.dest_db = &dest_db; - args.dest_table_name = &dest_table_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_exchange_partitions(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("exchange_partitions") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_exchange_partitions_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "exchange_partitions failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_partition_with_auth(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names) -{ - send_get_partition_with_auth(db_name, tbl_name, part_vals, user_name, group_names); - recv_get_partition_with_auth(_return); -} - -void ThriftHiveMetastoreClient::send_get_partition_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partition_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partition_with_auth_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.user_name = &user_name; - args.group_names = &group_names; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_partition_with_auth(Partition& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partition_with_auth") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_partition_with_auth_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_with_auth failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) -{ - send_get_partition_by_name(db_name, tbl_name, part_name); - recv_get_partition_by_name(_return); -} - -void ThriftHiveMetastoreClient::send_get_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partition_by_name_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_partition_by_name(Partition& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partition_by_name") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_partition_by_name_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_by_name failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_partitions(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) -{ - send_get_partitions(db_name, tbl_name, max_parts); - recv_get_partitions(_return); -} - -void ThriftHiveMetastoreClient::send_get_partitions(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.max_parts = &max_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_partitions(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_partitions_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_partitions_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) -{ - send_get_partitions_with_auth(db_name, tbl_name, max_parts, user_name, group_names); - recv_get_partitions_with_auth(_return); -} - -void ThriftHiveMetastoreClient::send_get_partitions_with_auth(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_with_auth_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.max_parts = &max_parts; - args.user_name = &user_name; - args.group_names = &group_names; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_partitions_with_auth(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions_with_auth") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_partitions_with_auth_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_with_auth failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_partitions_pspec(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int32_t max_parts) -{ - send_get_partitions_pspec(db_name, tbl_name, max_parts); - recv_get_partitions_pspec(_return); -} - -void ThriftHiveMetastoreClient::send_get_partitions_pspec(const std::string& db_name, const std::string& tbl_name, const int32_t max_parts) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partitions_pspec", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_pspec_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.max_parts = &max_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_partitions_pspec(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions_pspec") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_partitions_pspec_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_pspec failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_partition_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) -{ - send_get_partition_names(db_name, tbl_name, max_parts); - recv_get_partition_names(_return); -} - -void ThriftHiveMetastoreClient::send_get_partition_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partition_names_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.max_parts = &max_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_partition_names(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partition_names") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_partition_names_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_names failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_partition_values(PartitionValuesResponse& _return, const PartitionValuesRequest& request) -{ - send_get_partition_values(request); - recv_get_partition_values(_return); -} - -void ThriftHiveMetastoreClient::send_get_partition_values(const PartitionValuesRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partition_values", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partition_values_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_partition_values(PartitionValuesResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partition_values") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_partition_values_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_values failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_partitions_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) -{ - send_get_partitions_ps(db_name, tbl_name, part_vals, max_parts); - recv_get_partitions_ps(_return); -} - -void ThriftHiveMetastoreClient::send_get_partitions_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_ps_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.max_parts = &max_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_partitions_ps(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions_ps") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_partitions_ps_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_ps failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_partitions_ps_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) -{ - send_get_partitions_ps_with_auth(db_name, tbl_name, part_vals, max_parts, user_name, group_names); - recv_get_partitions_ps_with_auth(_return); -} - -void ThriftHiveMetastoreClient::send_get_partitions_ps_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_ps_with_auth_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.max_parts = &max_parts; - args.user_name = &user_name; - args.group_names = &group_names; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_partitions_ps_with_auth(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions_ps_with_auth") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_partitions_ps_with_auth_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_ps_with_auth failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_partition_names_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) -{ - send_get_partition_names_ps(db_name, tbl_name, part_vals, max_parts); - recv_get_partition_names_ps(_return); -} - -void ThriftHiveMetastoreClient::send_get_partition_names_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partition_names_ps_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.max_parts = &max_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_partition_names_ps(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partition_names_ps") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_partition_names_ps_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_names_ps failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_partitions_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts) -{ - send_get_partitions_by_filter(db_name, tbl_name, filter, max_parts); - recv_get_partitions_by_filter(_return); -} - -void ThriftHiveMetastoreClient::send_get_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_by_filter_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.filter = &filter; - args.max_parts = &max_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_partitions_by_filter(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions_by_filter") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_partitions_by_filter_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_filter failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_part_specs_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts) -{ - send_get_part_specs_by_filter(db_name, tbl_name, filter, max_parts); - recv_get_part_specs_by_filter(_return); -} - -void ThriftHiveMetastoreClient::send_get_part_specs_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_part_specs_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_part_specs_by_filter_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.filter = &filter; - args.max_parts = &max_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_part_specs_by_filter(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_part_specs_by_filter") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_part_specs_by_filter_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_part_specs_by_filter failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_partitions_by_expr(PartitionsByExprResult& _return, const PartitionsByExprRequest& req) -{ - send_get_partitions_by_expr(req); - recv_get_partitions_by_expr(_return); -} - -void ThriftHiveMetastoreClient::send_get_partitions_by_expr(const PartitionsByExprRequest& req) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partitions_by_expr", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_by_expr_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_partitions_by_expr(PartitionsByExprResult& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions_by_expr") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_partitions_by_expr_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_expr failed: unknown result"); -} - -int32_t ThriftHiveMetastoreClient::get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter) -{ - send_get_num_partitions_by_filter(db_name, tbl_name, filter); - return recv_get_num_partitions_by_filter(); -} - -void ThriftHiveMetastoreClient::send_get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_num_partitions_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_num_partitions_by_filter_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.filter = &filter; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -int32_t ThriftHiveMetastoreClient::recv_get_num_partitions_by_filter() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_num_partitions_by_filter") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - int32_t _return; - ThriftHiveMetastore_get_num_partitions_by_filter_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_num_partitions_by_filter failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_partitions_by_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & names) -{ - send_get_partitions_by_names(db_name, tbl_name, names); - recv_get_partitions_by_names(_return); -} - -void ThriftHiveMetastoreClient::send_get_partitions_by_names(const std::string& db_name, const std::string& tbl_name, const std::vector & names) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_by_names_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.names = &names; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_partitions_by_names(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions_by_names") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_partitions_by_names_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_names failed: unknown result"); -} - -void ThriftHiveMetastoreClient::alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part) -{ - send_alter_partition(db_name, tbl_name, new_part); - recv_alter_partition(); -} - -void ThriftHiveMetastoreClient::send_alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_partition_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.new_part = &new_part; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_alter_partition() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_partition") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_alter_partition_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -void ThriftHiveMetastoreClient::alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) -{ - send_alter_partitions(db_name, tbl_name, new_parts); - recv_alter_partitions(); -} - -void ThriftHiveMetastoreClient::send_alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_partitions_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.new_parts = &new_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_alter_partitions() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_partitions") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_alter_partitions_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -void ThriftHiveMetastoreClient::alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context) -{ - send_alter_partitions_with_environment_context(db_name, tbl_name, new_parts, environment_context); - recv_alter_partitions_with_environment_context(); -} - -void ThriftHiveMetastoreClient::send_alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_partitions_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_partitions_with_environment_context_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.new_parts = &new_parts; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_alter_partitions_with_environment_context() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_partitions_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_alter_partitions_with_environment_context_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -void ThriftHiveMetastoreClient::alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context) -{ - send_alter_partition_with_environment_context(db_name, tbl_name, new_part, environment_context); - recv_alter_partition_with_environment_context(); -} - -void ThriftHiveMetastoreClient::send_alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_partition_with_environment_context_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.new_part = &new_part; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_alter_partition_with_environment_context() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_partition_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_alter_partition_with_environment_context_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -void ThriftHiveMetastoreClient::rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) -{ - send_rename_partition(db_name, tbl_name, part_vals, new_part); - recv_rename_partition(); -} - -void ThriftHiveMetastoreClient::send_rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_rename_partition_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.new_part = &new_part; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_rename_partition() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("rename_partition") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_rename_partition_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -bool ThriftHiveMetastoreClient::partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) -{ - send_partition_name_has_valid_characters(part_vals, throw_exception); - return recv_partition_name_has_valid_characters(); -} - -void ThriftHiveMetastoreClient::send_partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("partition_name_has_valid_characters", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_partition_name_has_valid_characters_pargs args; - args.part_vals = &part_vals; - args.throw_exception = &throw_exception; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_partition_name_has_valid_characters() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("partition_name_has_valid_characters") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_partition_name_has_valid_characters_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_has_valid_characters failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_config_value(std::string& _return, const std::string& name, const std::string& defaultValue) -{ - send_get_config_value(name, defaultValue); - recv_get_config_value(_return); -} - -void ThriftHiveMetastoreClient::send_get_config_value(const std::string& name, const std::string& defaultValue) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_config_value_pargs args; - args.name = &name; - args.defaultValue = &defaultValue; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_config_value(std::string& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_config_value") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_config_value_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_config_value failed: unknown result"); -} - -void ThriftHiveMetastoreClient::partition_name_to_vals(std::vector & _return, const std::string& part_name) -{ - send_partition_name_to_vals(part_name); - recv_partition_name_to_vals(_return); -} - -void ThriftHiveMetastoreClient::send_partition_name_to_vals(const std::string& part_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_partition_name_to_vals_pargs args; - args.part_name = &part_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_partition_name_to_vals(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("partition_name_to_vals") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_partition_name_to_vals_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_to_vals failed: unknown result"); -} - -void ThriftHiveMetastoreClient::partition_name_to_spec(std::map & _return, const std::string& part_name) -{ - send_partition_name_to_spec(part_name); - recv_partition_name_to_spec(_return); -} - -void ThriftHiveMetastoreClient::send_partition_name_to_spec(const std::string& part_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_partition_name_to_spec_pargs args; - args.part_name = &part_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_partition_name_to_spec(std::map & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("partition_name_to_spec") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_partition_name_to_spec_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_to_spec failed: unknown result"); -} - -void ThriftHiveMetastoreClient::markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) -{ - send_markPartitionForEvent(db_name, tbl_name, part_vals, eventType); - recv_markPartitionForEvent(); -} - -void ThriftHiveMetastoreClient::send_markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_markPartitionForEvent_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.eventType = &eventType; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_markPartitionForEvent() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("markPartitionForEvent") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_markPartitionForEvent_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } - if (result.__isset.o5) { - throw result.o5; - } - if (result.__isset.o6) { - throw result.o6; - } - return; -} - -bool ThriftHiveMetastoreClient::isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) -{ - send_isPartitionMarkedForEvent(db_name, tbl_name, part_vals, eventType); - return recv_isPartitionMarkedForEvent(); -} - -void ThriftHiveMetastoreClient::send_isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_isPartitionMarkedForEvent_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.eventType = &eventType; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_isPartitionMarkedForEvent() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("isPartitionMarkedForEvent") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_isPartitionMarkedForEvent_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } - if (result.__isset.o5) { - throw result.o5; - } - if (result.__isset.o6) { - throw result.o6; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "isPartitionMarkedForEvent failed: unknown result"); -} - -void ThriftHiveMetastoreClient::add_index(Index& _return, const Index& new_index, const Table& index_table) -{ - send_add_index(new_index, index_table); - recv_add_index(_return); -} - -void ThriftHiveMetastoreClient::send_add_index(const Index& new_index, const Table& index_table) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("add_index", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_index_pargs args; - args.new_index = &new_index; - args.index_table = &index_table; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_add_index(Index& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_index") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_add_index_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_index failed: unknown result"); -} - -void ThriftHiveMetastoreClient::alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) -{ - send_alter_index(dbname, base_tbl_name, idx_name, new_idx); - recv_alter_index(); -} - -void ThriftHiveMetastoreClient::send_alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_index_pargs args; - args.dbname = &dbname; - args.base_tbl_name = &base_tbl_name; - args.idx_name = &idx_name; - args.new_idx = &new_idx; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_alter_index() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_index") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_alter_index_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -bool ThriftHiveMetastoreClient::drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData) -{ - send_drop_index_by_name(db_name, tbl_name, index_name, deleteData); - return recv_drop_index_by_name(); -} - -void ThriftHiveMetastoreClient::send_drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_index_by_name_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.index_name = &index_name; - args.deleteData = &deleteData; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_drop_index_by_name() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_index_by_name") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_drop_index_by_name_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_index_by_name failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_index_by_name(Index& _return, const std::string& db_name, const std::string& tbl_name, const std::string& index_name) -{ - send_get_index_by_name(db_name, tbl_name, index_name); - recv_get_index_by_name(_return); -} - -void ThriftHiveMetastoreClient::send_get_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_index_by_name_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.index_name = &index_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_index_by_name(Index& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_index_by_name") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_index_by_name_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_index_by_name failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_indexes(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) -{ - send_get_indexes(db_name, tbl_name, max_indexes); - recv_get_indexes(_return); -} - -void ThriftHiveMetastoreClient::send_get_indexes(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_indexes_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.max_indexes = &max_indexes; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_indexes(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_indexes") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_indexes_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_indexes failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_index_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) -{ - send_get_index_names(db_name, tbl_name, max_indexes); - recv_get_index_names(_return); -} - -void ThriftHiveMetastoreClient::send_get_index_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_index_names_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.max_indexes = &max_indexes; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_index_names(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_index_names") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_index_names_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_index_names failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request) -{ - send_get_primary_keys(request); - recv_get_primary_keys(_return); -} - -void ThriftHiveMetastoreClient::send_get_primary_keys(const PrimaryKeysRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_primary_keys", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_primary_keys_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_primary_keys(PrimaryKeysResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_primary_keys") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_primary_keys_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_primary_keys failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request) -{ - send_get_foreign_keys(request); - recv_get_foreign_keys(_return); -} - -void ThriftHiveMetastoreClient::send_get_foreign_keys(const ForeignKeysRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_foreign_keys", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_foreign_keys_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_foreign_keys(ForeignKeysResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_foreign_keys") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_foreign_keys_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_foreign_keys failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_unique_constraints(UniqueConstraintsResponse& _return, const UniqueConstraintsRequest& request) -{ - send_get_unique_constraints(request); - recv_get_unique_constraints(_return); -} - -void ThriftHiveMetastoreClient::send_get_unique_constraints(const UniqueConstraintsRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_unique_constraints", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_unique_constraints_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_unique_constraints(UniqueConstraintsResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_unique_constraints") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_unique_constraints_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_unique_constraints failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_not_null_constraints(NotNullConstraintsResponse& _return, const NotNullConstraintsRequest& request) -{ - send_get_not_null_constraints(request); - recv_get_not_null_constraints(_return); -} - -void ThriftHiveMetastoreClient::send_get_not_null_constraints(const NotNullConstraintsRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_not_null_constraints", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_not_null_constraints_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_not_null_constraints(NotNullConstraintsResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_not_null_constraints") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_not_null_constraints_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_not_null_constraints failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::update_table_column_statistics(const ColumnStatistics& stats_obj) -{ - send_update_table_column_statistics(stats_obj); - return recv_update_table_column_statistics(); -} - -void ThriftHiveMetastoreClient::send_update_table_column_statistics(const ColumnStatistics& stats_obj) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("update_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_update_table_column_statistics_pargs args; - args.stats_obj = &stats_obj; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_update_table_column_statistics() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("update_table_column_statistics") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_update_table_column_statistics_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "update_table_column_statistics failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::update_partition_column_statistics(const ColumnStatistics& stats_obj) -{ - send_update_partition_column_statistics(stats_obj); - return recv_update_partition_column_statistics(); -} - -void ThriftHiveMetastoreClient::send_update_partition_column_statistics(const ColumnStatistics& stats_obj) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_update_partition_column_statistics_pargs args; - args.stats_obj = &stats_obj; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_update_partition_column_statistics() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("update_partition_column_statistics") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_update_partition_column_statistics_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "update_partition_column_statistics failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_table_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& col_name) -{ - send_get_table_column_statistics(db_name, tbl_name, col_name); - recv_get_table_column_statistics(_return); -} - -void ThriftHiveMetastoreClient::send_get_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_table_column_statistics_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.col_name = &col_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_table_column_statistics(ColumnStatistics& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_table_column_statistics") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_table_column_statistics_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_column_statistics failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_partition_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) -{ - send_get_partition_column_statistics(db_name, tbl_name, part_name, col_name); - recv_get_partition_column_statistics(_return); -} - -void ThriftHiveMetastoreClient::send_get_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partition_column_statistics_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.col_name = &col_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_partition_column_statistics(ColumnStatistics& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partition_column_statistics") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_partition_column_statistics_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_column_statistics failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_table_statistics_req(TableStatsResult& _return, const TableStatsRequest& request) -{ - send_get_table_statistics_req(request); - recv_get_table_statistics_req(_return); -} - -void ThriftHiveMetastoreClient::send_get_table_statistics_req(const TableStatsRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_table_statistics_req", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_table_statistics_req_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_table_statistics_req(TableStatsResult& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_table_statistics_req") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_table_statistics_req_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_statistics_req failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_partitions_statistics_req(PartitionsStatsResult& _return, const PartitionsStatsRequest& request) -{ - send_get_partitions_statistics_req(request); - recv_get_partitions_statistics_req(_return); -} - -void ThriftHiveMetastoreClient::send_get_partitions_statistics_req(const PartitionsStatsRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partitions_statistics_req", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_statistics_req_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_partitions_statistics_req(PartitionsStatsResult& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions_statistics_req") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_partitions_statistics_req_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_statistics_req failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_aggr_stats_for(AggrStats& _return, const PartitionsStatsRequest& request) -{ - send_get_aggr_stats_for(request); - recv_get_aggr_stats_for(_return); -} - -void ThriftHiveMetastoreClient::send_get_aggr_stats_for(const PartitionsStatsRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_aggr_stats_for", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_aggr_stats_for_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_aggr_stats_for(AggrStats& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_aggr_stats_for") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_aggr_stats_for_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_aggr_stats_for failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::set_aggr_stats_for(const SetPartitionsStatsRequest& request) -{ - send_set_aggr_stats_for(request); - return recv_set_aggr_stats_for(); -} - -void ThriftHiveMetastoreClient::send_set_aggr_stats_for(const SetPartitionsStatsRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("set_aggr_stats_for", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_set_aggr_stats_for_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_set_aggr_stats_for() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("set_aggr_stats_for") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_set_aggr_stats_for_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "set_aggr_stats_for failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) -{ - send_delete_partition_column_statistics(db_name, tbl_name, part_name, col_name); - return recv_delete_partition_column_statistics(); -} - -void ThriftHiveMetastoreClient::send_delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_delete_partition_column_statistics_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.col_name = &col_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_delete_partition_column_statistics() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("delete_partition_column_statistics") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_delete_partition_column_statistics_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "delete_partition_column_statistics failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) -{ - send_delete_table_column_statistics(db_name, tbl_name, col_name); - return recv_delete_table_column_statistics(); -} - -void ThriftHiveMetastoreClient::send_delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_delete_table_column_statistics_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.col_name = &col_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_delete_table_column_statistics() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("delete_table_column_statistics") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_delete_table_column_statistics_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "delete_table_column_statistics failed: unknown result"); -} - -void ThriftHiveMetastoreClient::create_function(const Function& func) -{ - send_create_function(func); - recv_create_function(); -} - -void ThriftHiveMetastoreClient::send_create_function(const Function& func) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("create_function", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_function_pargs args; - args.func = &func; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_create_function() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_function") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_create_function_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } - return; -} - -void ThriftHiveMetastoreClient::drop_function(const std::string& dbName, const std::string& funcName) -{ - send_drop_function(dbName, funcName); - recv_drop_function(); -} - -void ThriftHiveMetastoreClient::send_drop_function(const std::string& dbName, const std::string& funcName) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_function", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_function_pargs args; - args.dbName = &dbName; - args.funcName = &funcName; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_drop_function() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_function") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_drop_function_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o3) { - throw result.o3; - } - return; -} - -void ThriftHiveMetastoreClient::alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc) -{ - send_alter_function(dbName, funcName, newFunc); - recv_alter_function(); -} - -void ThriftHiveMetastoreClient::send_alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_function", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_function_pargs args; - args.dbName = &dbName; - args.funcName = &funcName; - args.newFunc = &newFunc; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_alter_function() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_function") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_alter_function_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -void ThriftHiveMetastoreClient::get_functions(std::vector & _return, const std::string& dbName, const std::string& pattern) -{ - send_get_functions(dbName, pattern); - recv_get_functions(_return); -} - -void ThriftHiveMetastoreClient::send_get_functions(const std::string& dbName, const std::string& pattern) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_functions", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_functions_pargs args; - args.dbName = &dbName; - args.pattern = &pattern; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_functions(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_functions") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_functions_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_functions failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_function(Function& _return, const std::string& dbName, const std::string& funcName) -{ - send_get_function(dbName, funcName); - recv_get_function(_return); -} - -void ThriftHiveMetastoreClient::send_get_function(const std::string& dbName, const std::string& funcName) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_function", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_function_pargs args; - args.dbName = &dbName; - args.funcName = &funcName; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_function(Function& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_function") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_function_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_function failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_all_functions(GetAllFunctionsResponse& _return) -{ - send_get_all_functions(); - recv_get_all_functions(_return); -} - -void ThriftHiveMetastoreClient::send_get_all_functions() -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_all_functions", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_all_functions_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_all_functions(GetAllFunctionsResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_all_functions") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_all_functions_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_functions failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::create_role(const Role& role) -{ - send_create_role(role); - return recv_create_role(); -} - -void ThriftHiveMetastoreClient::send_create_role(const Role& role) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("create_role", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_role_pargs args; - args.role = &role; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_create_role() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_role") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_create_role_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_role failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::drop_role(const std::string& role_name) -{ - send_drop_role(role_name); - return recv_drop_role(); -} - -void ThriftHiveMetastoreClient::send_drop_role(const std::string& role_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_role", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_role_pargs args; - args.role_name = &role_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_drop_role() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_role") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_drop_role_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_role failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_role_names(std::vector & _return) -{ - send_get_role_names(); - recv_get_role_names(_return); -} - -void ThriftHiveMetastoreClient::send_get_role_names() -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_role_names", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_role_names_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_role_names(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_role_names") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_role_names_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_role_names failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option) -{ - send_grant_role(role_name, principal_name, principal_type, grantor, grantorType, grant_option); - return recv_grant_role(); -} - -void ThriftHiveMetastoreClient::send_grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("grant_role", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_grant_role_pargs args; - args.role_name = &role_name; - args.principal_name = &principal_name; - args.principal_type = &principal_type; - args.grantor = &grantor; - args.grantorType = &grantorType; - args.grant_option = &grant_option; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_grant_role() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("grant_role") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_grant_role_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_role failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type) -{ - send_revoke_role(role_name, principal_name, principal_type); - return recv_revoke_role(); -} - -void ThriftHiveMetastoreClient::send_revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("revoke_role", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_revoke_role_pargs args; - args.role_name = &role_name; - args.principal_name = &principal_name; - args.principal_type = &principal_type; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_revoke_role() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("revoke_role") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_revoke_role_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "revoke_role failed: unknown result"); -} - -void ThriftHiveMetastoreClient::list_roles(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type) -{ - send_list_roles(principal_name, principal_type); - recv_list_roles(_return); -} - -void ThriftHiveMetastoreClient::send_list_roles(const std::string& principal_name, const PrincipalType::type principal_type) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("list_roles", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_list_roles_pargs args; - args.principal_name = &principal_name; - args.principal_type = &principal_type; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_list_roles(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("list_roles") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_list_roles_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "list_roles failed: unknown result"); -} - -void ThriftHiveMetastoreClient::grant_revoke_role(GrantRevokeRoleResponse& _return, const GrantRevokeRoleRequest& request) -{ - send_grant_revoke_role(request); - recv_grant_revoke_role(_return); -} - -void ThriftHiveMetastoreClient::send_grant_revoke_role(const GrantRevokeRoleRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("grant_revoke_role", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_grant_revoke_role_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_grant_revoke_role(GrantRevokeRoleResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("grant_revoke_role") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_grant_revoke_role_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_revoke_role failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_principals_in_role(GetPrincipalsInRoleResponse& _return, const GetPrincipalsInRoleRequest& request) -{ - send_get_principals_in_role(request); - recv_get_principals_in_role(_return); -} - -void ThriftHiveMetastoreClient::send_get_principals_in_role(const GetPrincipalsInRoleRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_principals_in_role", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_principals_in_role_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_principals_in_role(GetPrincipalsInRoleResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_principals_in_role") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_principals_in_role_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_principals_in_role failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return, const GetRoleGrantsForPrincipalRequest& request) -{ - send_get_role_grants_for_principal(request); - recv_get_role_grants_for_principal(_return); -} - -void ThriftHiveMetastoreClient::send_get_role_grants_for_principal(const GetRoleGrantsForPrincipalRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_role_grants_for_principal", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_role_grants_for_principal_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_role_grants_for_principal") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_role_grants_for_principal_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_role_grants_for_principal failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_privilege_set(PrincipalPrivilegeSet& _return, const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names) -{ - send_get_privilege_set(hiveObject, user_name, group_names); - recv_get_privilege_set(_return); -} - -void ThriftHiveMetastoreClient::send_get_privilege_set(const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_privilege_set", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_privilege_set_pargs args; - args.hiveObject = &hiveObject; - args.user_name = &user_name; - args.group_names = &group_names; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_privilege_set(PrincipalPrivilegeSet& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_privilege_set") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_privilege_set_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_privilege_set failed: unknown result"); -} - -void ThriftHiveMetastoreClient::list_privileges(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) -{ - send_list_privileges(principal_name, principal_type, hiveObject); - recv_list_privileges(_return); -} - -void ThriftHiveMetastoreClient::send_list_privileges(const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("list_privileges", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_list_privileges_pargs args; - args.principal_name = &principal_name; - args.principal_type = &principal_type; - args.hiveObject = &hiveObject; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_list_privileges(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("list_privileges") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_list_privileges_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "list_privileges failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::grant_privileges(const PrivilegeBag& privileges) -{ - send_grant_privileges(privileges); - return recv_grant_privileges(); -} - -void ThriftHiveMetastoreClient::send_grant_privileges(const PrivilegeBag& privileges) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("grant_privileges", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_grant_privileges_pargs args; - args.privileges = &privileges; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_grant_privileges() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("grant_privileges") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_grant_privileges_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_privileges failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::revoke_privileges(const PrivilegeBag& privileges) -{ - send_revoke_privileges(privileges); - return recv_revoke_privileges(); -} - -void ThriftHiveMetastoreClient::send_revoke_privileges(const PrivilegeBag& privileges) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("revoke_privileges", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_revoke_privileges_pargs args; - args.privileges = &privileges; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_revoke_privileges() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("revoke_privileges") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_revoke_privileges_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "revoke_privileges failed: unknown result"); -} - -void ThriftHiveMetastoreClient::grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const GrantRevokePrivilegeRequest& request) -{ - send_grant_revoke_privileges(request); - recv_grant_revoke_privileges(_return); -} - -void ThriftHiveMetastoreClient::send_grant_revoke_privileges(const GrantRevokePrivilegeRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("grant_revoke_privileges", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_grant_revoke_privileges_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_grant_revoke_privileges(GrantRevokePrivilegeResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("grant_revoke_privileges") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_grant_revoke_privileges_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_revoke_privileges failed: unknown result"); -} - -void ThriftHiveMetastoreClient::set_ugi(std::vector & _return, const std::string& user_name, const std::vector & group_names) -{ - send_set_ugi(user_name, group_names); - recv_set_ugi(_return); -} - -void ThriftHiveMetastoreClient::send_set_ugi(const std::string& user_name, const std::vector & group_names) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("set_ugi", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_set_ugi_pargs args; - args.user_name = &user_name; - args.group_names = &group_names; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_set_ugi(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("set_ugi") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_set_ugi_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "set_ugi failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name) -{ - send_get_delegation_token(token_owner, renewer_kerberos_principal_name); - recv_get_delegation_token(_return); -} - -void ThriftHiveMetastoreClient::send_get_delegation_token(const std::string& token_owner, const std::string& renewer_kerberos_principal_name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_delegation_token_pargs args; - args.token_owner = &token_owner; - args.renewer_kerberos_principal_name = &renewer_kerberos_principal_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_delegation_token(std::string& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_delegation_token") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_delegation_token_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_delegation_token failed: unknown result"); -} - -int64_t ThriftHiveMetastoreClient::renew_delegation_token(const std::string& token_str_form) -{ - send_renew_delegation_token(token_str_form); - return recv_renew_delegation_token(); -} - -void ThriftHiveMetastoreClient::send_renew_delegation_token(const std::string& token_str_form) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("renew_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_renew_delegation_token_pargs args; - args.token_str_form = &token_str_form; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -int64_t ThriftHiveMetastoreClient::recv_renew_delegation_token() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("renew_delegation_token") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - int64_t _return; - ThriftHiveMetastore_renew_delegation_token_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "renew_delegation_token failed: unknown result"); -} - -void ThriftHiveMetastoreClient::cancel_delegation_token(const std::string& token_str_form) -{ - send_cancel_delegation_token(token_str_form); - recv_cancel_delegation_token(); -} - -void ThriftHiveMetastoreClient::send_cancel_delegation_token(const std::string& token_str_form) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("cancel_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_cancel_delegation_token_pargs args; - args.token_str_form = &token_str_form; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_cancel_delegation_token() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("cancel_delegation_token") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_cancel_delegation_token_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - return; -} - -bool ThriftHiveMetastoreClient::add_token(const std::string& token_identifier, const std::string& delegation_token) -{ - send_add_token(token_identifier, delegation_token); - return recv_add_token(); -} - -void ThriftHiveMetastoreClient::send_add_token(const std::string& token_identifier, const std::string& delegation_token) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("add_token", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_token_pargs args; - args.token_identifier = &token_identifier; - args.delegation_token = &delegation_token; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_add_token() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_token") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_add_token_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_token failed: unknown result"); -} - -bool ThriftHiveMetastoreClient::remove_token(const std::string& token_identifier) -{ - send_remove_token(token_identifier); - return recv_remove_token(); -} - -void ThriftHiveMetastoreClient::send_remove_token(const std::string& token_identifier) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("remove_token", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_remove_token_pargs args; - args.token_identifier = &token_identifier; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_remove_token() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("remove_token") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_remove_token_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "remove_token failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_token(std::string& _return, const std::string& token_identifier) -{ - send_get_token(token_identifier); - recv_get_token(_return); -} - -void ThriftHiveMetastoreClient::send_get_token(const std::string& token_identifier) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_token", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_token_pargs args; - args.token_identifier = &token_identifier; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_token(std::string& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_token") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_token_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_token failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_all_token_identifiers(std::vector & _return) -{ - send_get_all_token_identifiers(); - recv_get_all_token_identifiers(_return); -} - -void ThriftHiveMetastoreClient::send_get_all_token_identifiers() -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_all_token_identifiers", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_all_token_identifiers_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_all_token_identifiers(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_all_token_identifiers") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_all_token_identifiers_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_token_identifiers failed: unknown result"); -} - -int32_t ThriftHiveMetastoreClient::add_master_key(const std::string& key) -{ - send_add_master_key(key); - return recv_add_master_key(); -} - -void ThriftHiveMetastoreClient::send_add_master_key(const std::string& key) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("add_master_key", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_master_key_pargs args; - args.key = &key; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -int32_t ThriftHiveMetastoreClient::recv_add_master_key() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_master_key") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - int32_t _return; - ThriftHiveMetastore_add_master_key_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_master_key failed: unknown result"); -} - -void ThriftHiveMetastoreClient::update_master_key(const int32_t seq_number, const std::string& key) -{ - send_update_master_key(seq_number, key); - recv_update_master_key(); -} - -void ThriftHiveMetastoreClient::send_update_master_key(const int32_t seq_number, const std::string& key) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("update_master_key", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_update_master_key_pargs args; - args.seq_number = &seq_number; - args.key = &key; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_update_master_key() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("update_master_key") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_update_master_key_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -bool ThriftHiveMetastoreClient::remove_master_key(const int32_t key_seq) -{ - send_remove_master_key(key_seq); - return recv_remove_master_key(); -} - -void ThriftHiveMetastoreClient::send_remove_master_key(const int32_t key_seq) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("remove_master_key", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_remove_master_key_pargs args; - args.key_seq = &key_seq; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -bool ThriftHiveMetastoreClient::recv_remove_master_key() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("remove_master_key") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - bool _return; - ThriftHiveMetastore_remove_master_key_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - return _return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "remove_master_key failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_master_keys(std::vector & _return) -{ - send_get_master_keys(); - recv_get_master_keys(_return); -} - -void ThriftHiveMetastoreClient::send_get_master_keys() -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_master_keys", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_master_keys_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_master_keys(std::vector & _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_master_keys") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_master_keys_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_master_keys failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_open_txns(GetOpenTxnsResponse& _return) -{ - send_get_open_txns(); - recv_get_open_txns(_return); -} - -void ThriftHiveMetastoreClient::send_get_open_txns() -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_open_txns", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_open_txns_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_open_txns(GetOpenTxnsResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_open_txns") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_open_txns_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_open_txns failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_open_txns_info(GetOpenTxnsInfoResponse& _return) -{ - send_get_open_txns_info(); - recv_get_open_txns_info(_return); -} - -void ThriftHiveMetastoreClient::send_get_open_txns_info() -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_open_txns_info", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_open_txns_info_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_open_txns_info(GetOpenTxnsInfoResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_open_txns_info") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_open_txns_info_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_open_txns_info failed: unknown result"); -} - -void ThriftHiveMetastoreClient::open_txns(OpenTxnsResponse& _return, const OpenTxnRequest& rqst) -{ - send_open_txns(rqst); - recv_open_txns(_return); -} - -void ThriftHiveMetastoreClient::send_open_txns(const OpenTxnRequest& rqst) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("open_txns", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_open_txns_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_open_txns(OpenTxnsResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("open_txns") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_open_txns_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "open_txns failed: unknown result"); -} - -void ThriftHiveMetastoreClient::abort_txn(const AbortTxnRequest& rqst) -{ - send_abort_txn(rqst); - recv_abort_txn(); -} - -void ThriftHiveMetastoreClient::send_abort_txn(const AbortTxnRequest& rqst) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("abort_txn", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_abort_txn_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_abort_txn() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("abort_txn") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_abort_txn_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - return; -} - -void ThriftHiveMetastoreClient::abort_txns(const AbortTxnsRequest& rqst) -{ - send_abort_txns(rqst); - recv_abort_txns(); -} - -void ThriftHiveMetastoreClient::send_abort_txns(const AbortTxnsRequest& rqst) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("abort_txns", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_abort_txns_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_abort_txns() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("abort_txns") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_abort_txns_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - return; -} - -void ThriftHiveMetastoreClient::commit_txn(const CommitTxnRequest& rqst) -{ - send_commit_txn(rqst); - recv_commit_txn(); -} - -void ThriftHiveMetastoreClient::send_commit_txn(const CommitTxnRequest& rqst) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("commit_txn", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_commit_txn_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_commit_txn() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("commit_txn") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_commit_txn_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -void ThriftHiveMetastoreClient::lock(LockResponse& _return, const LockRequest& rqst) -{ - send_lock(rqst); - recv_lock(_return); -} - -void ThriftHiveMetastoreClient::send_lock(const LockRequest& rqst) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("lock", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_lock_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_lock(LockResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("lock") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_lock_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "lock failed: unknown result"); -} - -void ThriftHiveMetastoreClient::check_lock(LockResponse& _return, const CheckLockRequest& rqst) -{ - send_check_lock(rqst); - recv_check_lock(_return); -} - -void ThriftHiveMetastoreClient::send_check_lock(const CheckLockRequest& rqst) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("check_lock", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_check_lock_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_check_lock(LockResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("check_lock") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_check_lock_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "check_lock failed: unknown result"); -} - -void ThriftHiveMetastoreClient::unlock(const UnlockRequest& rqst) -{ - send_unlock(rqst); - recv_unlock(); -} - -void ThriftHiveMetastoreClient::send_unlock(const UnlockRequest& rqst) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("unlock", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_unlock_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_unlock() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("unlock") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_unlock_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -void ThriftHiveMetastoreClient::show_locks(ShowLocksResponse& _return, const ShowLocksRequest& rqst) -{ - send_show_locks(rqst); - recv_show_locks(_return); -} - -void ThriftHiveMetastoreClient::send_show_locks(const ShowLocksRequest& rqst) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("show_locks", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_show_locks_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_show_locks(ShowLocksResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("show_locks") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_show_locks_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "show_locks failed: unknown result"); -} - -void ThriftHiveMetastoreClient::heartbeat(const HeartbeatRequest& ids) -{ - send_heartbeat(ids); - recv_heartbeat(); -} - -void ThriftHiveMetastoreClient::send_heartbeat(const HeartbeatRequest& ids) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("heartbeat", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_heartbeat_pargs args; - args.ids = &ids; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_heartbeat() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("heartbeat") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_heartbeat_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - return; -} - -void ThriftHiveMetastoreClient::heartbeat_txn_range(HeartbeatTxnRangeResponse& _return, const HeartbeatTxnRangeRequest& txns) -{ - send_heartbeat_txn_range(txns); - recv_heartbeat_txn_range(_return); -} - -void ThriftHiveMetastoreClient::send_heartbeat_txn_range(const HeartbeatTxnRangeRequest& txns) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("heartbeat_txn_range", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_heartbeat_txn_range_pargs args; - args.txns = &txns; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_heartbeat_txn_range(HeartbeatTxnRangeResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("heartbeat_txn_range") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_heartbeat_txn_range_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "heartbeat_txn_range failed: unknown result"); -} - -void ThriftHiveMetastoreClient::compact(const CompactionRequest& rqst) -{ - send_compact(rqst); - recv_compact(); -} - -void ThriftHiveMetastoreClient::send_compact(const CompactionRequest& rqst) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("compact", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_compact_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_compact() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("compact") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_compact_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - return; -} - -void ThriftHiveMetastoreClient::compact2(CompactionResponse& _return, const CompactionRequest& rqst) -{ - send_compact2(rqst); - recv_compact2(_return); -} - -void ThriftHiveMetastoreClient::send_compact2(const CompactionRequest& rqst) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("compact2", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_compact2_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_compact2(CompactionResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("compact2") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_compact2_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "compact2 failed: unknown result"); -} - -void ThriftHiveMetastoreClient::show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst) -{ - send_show_compact(rqst); - recv_show_compact(_return); -} - -void ThriftHiveMetastoreClient::send_show_compact(const ShowCompactRequest& rqst) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("show_compact", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_show_compact_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_show_compact(ShowCompactResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("show_compact") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_show_compact_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "show_compact failed: unknown result"); -} - -void ThriftHiveMetastoreClient::add_dynamic_partitions(const AddDynamicPartitions& rqst) -{ - send_add_dynamic_partitions(rqst); - recv_add_dynamic_partitions(); -} - -void ThriftHiveMetastoreClient::send_add_dynamic_partitions(const AddDynamicPartitions& rqst) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("add_dynamic_partitions", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_dynamic_partitions_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_add_dynamic_partitions() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_dynamic_partitions") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_add_dynamic_partitions_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - return; -} - -void ThriftHiveMetastoreClient::get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst) -{ - send_get_next_notification(rqst); - recv_get_next_notification(_return); -} - -void ThriftHiveMetastoreClient::send_get_next_notification(const NotificationEventRequest& rqst) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_next_notification", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_next_notification_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_next_notification(NotificationEventResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_next_notification") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_next_notification_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_next_notification failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_current_notificationEventId(CurrentNotificationEventId& _return) -{ - send_get_current_notificationEventId(); - recv_get_current_notificationEventId(_return); -} - -void ThriftHiveMetastoreClient::send_get_current_notificationEventId() -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_current_notificationEventId", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_current_notificationEventId_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_current_notificationEventId(CurrentNotificationEventId& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_current_notificationEventId") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_current_notificationEventId_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_current_notificationEventId failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_notification_events_count(NotificationEventsCountResponse& _return, const NotificationEventsCountRequest& rqst) -{ - send_get_notification_events_count(rqst); - recv_get_notification_events_count(_return); -} - -void ThriftHiveMetastoreClient::send_get_notification_events_count(const NotificationEventsCountRequest& rqst) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_notification_events_count", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_notification_events_count_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_notification_events_count(NotificationEventsCountResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_notification_events_count") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_notification_events_count_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_notification_events_count failed: unknown result"); -} - -void ThriftHiveMetastoreClient::fire_listener_event(FireEventResponse& _return, const FireEventRequest& rqst) -{ - send_fire_listener_event(rqst); - recv_fire_listener_event(_return); -} - -void ThriftHiveMetastoreClient::send_fire_listener_event(const FireEventRequest& rqst) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("fire_listener_event", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_fire_listener_event_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_fire_listener_event(FireEventResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("fire_listener_event") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_fire_listener_event_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "fire_listener_event failed: unknown result"); -} - -void ThriftHiveMetastoreClient::flushCache() -{ - send_flushCache(); - recv_flushCache(); -} - -void ThriftHiveMetastoreClient::send_flushCache() -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("flushCache", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_flushCache_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_flushCache() -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("flushCache") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_flushCache_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - return; -} - -void ThriftHiveMetastoreClient::cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request) -{ - send_cm_recycle(request); - recv_cm_recycle(_return); -} - -void ThriftHiveMetastoreClient::send_cm_recycle(const CmRecycleRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("cm_recycle", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_cm_recycle_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_cm_recycle(CmRecycleResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("cm_recycle") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_cm_recycle_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "cm_recycle failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req) -{ - send_get_file_metadata_by_expr(req); - recv_get_file_metadata_by_expr(_return); -} - -void ThriftHiveMetastoreClient::send_get_file_metadata_by_expr(const GetFileMetadataByExprRequest& req) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_file_metadata_by_expr", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_file_metadata_by_expr_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_file_metadata_by_expr(GetFileMetadataByExprResult& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_file_metadata_by_expr") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_file_metadata_by_expr_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_file_metadata_by_expr failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req) -{ - send_get_file_metadata(req); - recv_get_file_metadata(_return); -} - -void ThriftHiveMetastoreClient::send_get_file_metadata(const GetFileMetadataRequest& req) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_file_metadata", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_file_metadata_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_file_metadata(GetFileMetadataResult& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_file_metadata") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_file_metadata_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_file_metadata failed: unknown result"); -} - -void ThriftHiveMetastoreClient::put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req) -{ - send_put_file_metadata(req); - recv_put_file_metadata(_return); -} - -void ThriftHiveMetastoreClient::send_put_file_metadata(const PutFileMetadataRequest& req) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("put_file_metadata", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_put_file_metadata_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_put_file_metadata(PutFileMetadataResult& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("put_file_metadata") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_put_file_metadata_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "put_file_metadata failed: unknown result"); -} - -void ThriftHiveMetastoreClient::clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req) -{ - send_clear_file_metadata(req); - recv_clear_file_metadata(_return); -} - -void ThriftHiveMetastoreClient::send_clear_file_metadata(const ClearFileMetadataRequest& req) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("clear_file_metadata", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_clear_file_metadata_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_clear_file_metadata(ClearFileMetadataResult& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("clear_file_metadata") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_clear_file_metadata_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "clear_file_metadata failed: unknown result"); -} - -void ThriftHiveMetastoreClient::cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req) -{ - send_cache_file_metadata(req); - recv_cache_file_metadata(_return); -} - -void ThriftHiveMetastoreClient::send_cache_file_metadata(const CacheFileMetadataRequest& req) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("cache_file_metadata", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_cache_file_metadata_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_cache_file_metadata(CacheFileMetadataResult& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("cache_file_metadata") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_cache_file_metadata_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "cache_file_metadata failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_metastore_db_uuid(std::string& _return) -{ - send_get_metastore_db_uuid(); - recv_get_metastore_db_uuid(_return); -} - -void ThriftHiveMetastoreClient::send_get_metastore_db_uuid() -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_metastore_db_uuid", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_metastore_db_uuid_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_metastore_db_uuid(std::string& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_metastore_db_uuid") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_metastore_db_uuid_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_metastore_db_uuid failed: unknown result"); -} - -void ThriftHiveMetastoreClient::create_resource_plan(WMCreateResourcePlanResponse& _return, const WMCreateResourcePlanRequest& request) -{ - send_create_resource_plan(request); - recv_create_resource_plan(_return); -} - -void ThriftHiveMetastoreClient::send_create_resource_plan(const WMCreateResourcePlanRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("create_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_resource_plan_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_create_resource_plan(WMCreateResourcePlanResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_resource_plan") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_create_resource_plan_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_resource_plan failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_resource_plan(WMGetResourcePlanResponse& _return, const WMGetResourcePlanRequest& request) -{ - send_get_resource_plan(request); - recv_get_resource_plan(_return); -} - -void ThriftHiveMetastoreClient::send_get_resource_plan(const WMGetResourcePlanRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_resource_plan_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_resource_plan(WMGetResourcePlanResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_resource_plan") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_resource_plan_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_resource_plan failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_active_resource_plan(WMGetActiveResourcePlanResponse& _return, const WMGetActiveResourcePlanRequest& request) -{ - send_get_active_resource_plan(request); - recv_get_active_resource_plan(_return); -} - -void ThriftHiveMetastoreClient::send_get_active_resource_plan(const WMGetActiveResourcePlanRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_active_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_active_resource_plan_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_active_resource_plan(WMGetActiveResourcePlanResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_active_resource_plan") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_active_resource_plan_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_active_resource_plan failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_all_resource_plans(WMGetAllResourcePlanResponse& _return, const WMGetAllResourcePlanRequest& request) -{ - send_get_all_resource_plans(request); - recv_get_all_resource_plans(_return); -} - -void ThriftHiveMetastoreClient::send_get_all_resource_plans(const WMGetAllResourcePlanRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_all_resource_plans", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_all_resource_plans_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_all_resource_plans(WMGetAllResourcePlanResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_all_resource_plans") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_all_resource_plans_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_resource_plans failed: unknown result"); -} - -void ThriftHiveMetastoreClient::alter_resource_plan(WMAlterResourcePlanResponse& _return, const WMAlterResourcePlanRequest& request) -{ - send_alter_resource_plan(request); - recv_alter_resource_plan(_return); -} - -void ThriftHiveMetastoreClient::send_alter_resource_plan(const WMAlterResourcePlanRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_resource_plan_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_alter_resource_plan(WMAlterResourcePlanResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_resource_plan") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_alter_resource_plan_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "alter_resource_plan failed: unknown result"); -} - -void ThriftHiveMetastoreClient::validate_resource_plan(WMValidateResourcePlanResponse& _return, const WMValidateResourcePlanRequest& request) -{ - send_validate_resource_plan(request); - recv_validate_resource_plan(_return); -} - -void ThriftHiveMetastoreClient::send_validate_resource_plan(const WMValidateResourcePlanRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("validate_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_validate_resource_plan_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_validate_resource_plan(WMValidateResourcePlanResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("validate_resource_plan") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_validate_resource_plan_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "validate_resource_plan failed: unknown result"); -} - -void ThriftHiveMetastoreClient::drop_resource_plan(WMDropResourcePlanResponse& _return, const WMDropResourcePlanRequest& request) -{ - send_drop_resource_plan(request); - recv_drop_resource_plan(_return); -} - -void ThriftHiveMetastoreClient::send_drop_resource_plan(const WMDropResourcePlanRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_resource_plan_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_drop_resource_plan(WMDropResourcePlanResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_resource_plan") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_drop_resource_plan_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_resource_plan failed: unknown result"); -} - -void ThriftHiveMetastoreClient::create_wm_trigger(WMCreateTriggerResponse& _return, const WMCreateTriggerRequest& request) -{ - send_create_wm_trigger(request); - recv_create_wm_trigger(_return); -} - -void ThriftHiveMetastoreClient::send_create_wm_trigger(const WMCreateTriggerRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("create_wm_trigger", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_wm_trigger_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_create_wm_trigger(WMCreateTriggerResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_wm_trigger") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_create_wm_trigger_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - if (result.__isset.o4) { - throw result.o4; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_wm_trigger failed: unknown result"); -} - -void ThriftHiveMetastoreClient::alter_wm_trigger(WMAlterTriggerResponse& _return, const WMAlterTriggerRequest& request) -{ - send_alter_wm_trigger(request); - recv_alter_wm_trigger(_return); -} - -void ThriftHiveMetastoreClient::send_alter_wm_trigger(const WMAlterTriggerRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_wm_trigger", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_wm_trigger_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_alter_wm_trigger(WMAlterTriggerResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_wm_trigger") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_alter_wm_trigger_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "alter_wm_trigger failed: unknown result"); -} - -void ThriftHiveMetastoreClient::drop_wm_trigger(WMDropTriggerResponse& _return, const WMDropTriggerRequest& request) -{ - send_drop_wm_trigger(request); - recv_drop_wm_trigger(_return); -} - -void ThriftHiveMetastoreClient::send_drop_wm_trigger(const WMDropTriggerRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_wm_trigger", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_wm_trigger_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_drop_wm_trigger(WMDropTriggerResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_wm_trigger") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_drop_wm_trigger_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_wm_trigger failed: unknown result"); -} - -void ThriftHiveMetastoreClient::get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return, const WMGetTriggersForResourePlanRequest& request) -{ - send_get_triggers_for_resourceplan(request); - recv_get_triggers_for_resourceplan(_return); -} - -void ThriftHiveMetastoreClient::send_get_triggers_for_resourceplan(const WMGetTriggersForResourePlanRequest& request) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_triggers_for_resourceplan", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_triggers_for_resourceplan_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} - -void ThriftHiveMetastoreClient::recv_get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_triggers_for_resourceplan") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - ThriftHiveMetastore_get_triggers_for_resourceplan_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_triggers_for_resourceplan failed: unknown result"); -} - -bool ThriftHiveMetastoreProcessor::dispatchCall(::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, const std::string& fname, int32_t seqid, void* callContext) { - ProcessMap::iterator pfn; - pfn = processMap_.find(fname); - if (pfn == processMap_.end()) { - iprot->skip(::apache::thrift::protocol::T_STRUCT); - iprot->readMessageEnd(); - iprot->getTransport()->readEnd(); - ::apache::thrift::TApplicationException x(::apache::thrift::TApplicationException::UNKNOWN_METHOD, "Invalid method name: '"+fname+"'"); - oprot->writeMessageBegin(fname, ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return true; - } - (this->*(pfn->second))(seqid, iprot, oprot, callContext); - return true; -} - -void ThriftHiveMetastoreProcessor::process_getMetaConf(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.getMetaConf", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.getMetaConf"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.getMetaConf"); - } - - ThriftHiveMetastore_getMetaConf_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.getMetaConf", bytes); - } - - ThriftHiveMetastore_getMetaConf_result result; - try { - iface_->getMetaConf(result.success, args.key); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.getMetaConf"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("getMetaConf", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.getMetaConf"); - } - - oprot->writeMessageBegin("getMetaConf", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.getMetaConf", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_setMetaConf(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.setMetaConf", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.setMetaConf"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.setMetaConf"); - } - - ThriftHiveMetastore_setMetaConf_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.setMetaConf", bytes); - } - - ThriftHiveMetastore_setMetaConf_result result; - try { - iface_->setMetaConf(args.key, args.value); - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.setMetaConf"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("setMetaConf", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.setMetaConf"); - } - - oprot->writeMessageBegin("setMetaConf", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.setMetaConf", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_create_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_database", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_database"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_database"); - } - - ThriftHiveMetastore_create_database_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_database", bytes); - } - - ThriftHiveMetastore_create_database_result result; - try { - iface_->create_database(args.database); - } catch (AlreadyExistsException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_database"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_database"); - } - - oprot->writeMessageBegin("create_database", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_database", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_database", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_database"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_database"); - } - - ThriftHiveMetastore_get_database_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_database", bytes); - } - - ThriftHiveMetastore_get_database_result result; - try { - iface_->get_database(result.success, args.name); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_database"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_database"); - } - - oprot->writeMessageBegin("get_database", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_database", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_drop_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_database", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_database"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_database"); - } - - ThriftHiveMetastore_drop_database_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_database", bytes); - } - - ThriftHiveMetastore_drop_database_result result; - try { - iface_->drop_database(args.name, args.deleteData, args.cascade); - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (InvalidOperationException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_database"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_database"); - } - - oprot->writeMessageBegin("drop_database", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_database", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_databases(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_databases", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_databases"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_databases"); - } - - ThriftHiveMetastore_get_databases_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_databases", bytes); - } - - ThriftHiveMetastore_get_databases_result result; - try { - iface_->get_databases(result.success, args.pattern); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_databases"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_databases", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_databases"); - } - - oprot->writeMessageBegin("get_databases", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_databases", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_all_databases(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_all_databases", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_all_databases"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_databases"); - } - - ThriftHiveMetastore_get_all_databases_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_all_databases", bytes); - } - - ThriftHiveMetastore_get_all_databases_result result; - try { - iface_->get_all_databases(result.success); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_all_databases"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_all_databases", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_databases"); - } - - oprot->writeMessageBegin("get_all_databases", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_all_databases", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_alter_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_database", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_database"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_database"); - } - - ThriftHiveMetastore_alter_database_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_database", bytes); - } - - ThriftHiveMetastore_alter_database_result result; - try { - iface_->alter_database(args.dbname, args.db); - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_database"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_database"); - } - - oprot->writeMessageBegin("alter_database", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_database", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_type", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_type"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_type"); - } - - ThriftHiveMetastore_get_type_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_type", bytes); - } - - ThriftHiveMetastore_get_type_result result; - try { - iface_->get_type(result.success, args.name); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_type"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_type", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_type"); - } - - oprot->writeMessageBegin("get_type", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_type", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_create_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_type", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_type"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_type"); - } - - ThriftHiveMetastore_create_type_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_type", bytes); - } - - ThriftHiveMetastore_create_type_result result; - try { - result.success = iface_->create_type(args.type); - result.__isset.success = true; - } catch (AlreadyExistsException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_type"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_type", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_type"); - } - - oprot->writeMessageBegin("create_type", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_type", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_drop_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_type", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_type"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_type"); - } - - ThriftHiveMetastore_drop_type_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_type", bytes); - } - - ThriftHiveMetastore_drop_type_result result; - try { - result.success = iface_->drop_type(args.type); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_type"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_type", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_type"); - } - - oprot->writeMessageBegin("drop_type", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_type", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_type_all(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_type_all", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_type_all"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_type_all"); - } - - ThriftHiveMetastore_get_type_all_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_type_all", bytes); - } - - ThriftHiveMetastore_get_type_all_result result; - try { - iface_->get_type_all(result.success, args.name); - result.__isset.success = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_type_all"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_type_all", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_type_all"); - } - - oprot->writeMessageBegin("get_type_all", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_type_all", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_fields(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_fields", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_fields"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_fields"); - } - - ThriftHiveMetastore_get_fields_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_fields", bytes); - } - - ThriftHiveMetastore_get_fields_result result; - try { - iface_->get_fields(result.success, args.db_name, args.table_name); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (UnknownTableException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (UnknownDBException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_fields"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_fields", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_fields"); - } - - oprot->writeMessageBegin("get_fields", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_fields", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_fields_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_fields_with_environment_context", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_fields_with_environment_context"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_fields_with_environment_context"); - } - - ThriftHiveMetastore_get_fields_with_environment_context_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_fields_with_environment_context", bytes); - } - - ThriftHiveMetastore_get_fields_with_environment_context_result result; - try { - iface_->get_fields_with_environment_context(result.success, args.db_name, args.table_name, args.environment_context); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (UnknownTableException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (UnknownDBException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_fields_with_environment_context"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_fields_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_fields_with_environment_context"); - } - - oprot->writeMessageBegin("get_fields_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_fields_with_environment_context", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_schema(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_schema", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_schema"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_schema"); - } - - ThriftHiveMetastore_get_schema_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_schema", bytes); - } - - ThriftHiveMetastore_get_schema_result result; - try { - iface_->get_schema(result.success, args.db_name, args.table_name); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (UnknownTableException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (UnknownDBException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_schema"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_schema", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_schema"); - } - - oprot->writeMessageBegin("get_schema", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_schema", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_schema_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_schema_with_environment_context", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_schema_with_environment_context"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_schema_with_environment_context"); - } - - ThriftHiveMetastore_get_schema_with_environment_context_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_schema_with_environment_context", bytes); - } - - ThriftHiveMetastore_get_schema_with_environment_context_result result; - try { - iface_->get_schema_with_environment_context(result.success, args.db_name, args.table_name, args.environment_context); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (UnknownTableException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (UnknownDBException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_schema_with_environment_context"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_schema_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_schema_with_environment_context"); - } - - oprot->writeMessageBegin("get_schema_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_schema_with_environment_context", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_create_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_table", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_table"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_table"); - } - - ThriftHiveMetastore_create_table_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_table", bytes); - } - - ThriftHiveMetastore_create_table_result result; - try { - iface_->create_table(args.tbl); - } catch (AlreadyExistsException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (NoSuchObjectException &o4) { - result.o4 = o4; - result.__isset.o4 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_table"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_table"); - } - - oprot->writeMessageBegin("create_table", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_table", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_create_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_table_with_environment_context", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_table_with_environment_context"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_table_with_environment_context"); - } - - ThriftHiveMetastore_create_table_with_environment_context_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_table_with_environment_context", bytes); - } - - ThriftHiveMetastore_create_table_with_environment_context_result result; - try { - iface_->create_table_with_environment_context(args.tbl, args.environment_context); - } catch (AlreadyExistsException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (NoSuchObjectException &o4) { - result.o4 = o4; - result.__isset.o4 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_table_with_environment_context"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_table_with_environment_context"); - } - - oprot->writeMessageBegin("create_table_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_table_with_environment_context", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_create_table_with_constraints(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_table_with_constraints", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_table_with_constraints"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_table_with_constraints"); - } - - ThriftHiveMetastore_create_table_with_constraints_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_table_with_constraints", bytes); - } - - ThriftHiveMetastore_create_table_with_constraints_result result; - try { - iface_->create_table_with_constraints(args.tbl, args.primaryKeys, args.foreignKeys, args.uniqueConstraints, args.notNullConstraints); - } catch (AlreadyExistsException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (NoSuchObjectException &o4) { - result.o4 = o4; - result.__isset.o4 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_table_with_constraints"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_table_with_constraints", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_table_with_constraints"); - } - - oprot->writeMessageBegin("create_table_with_constraints", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_table_with_constraints", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_drop_constraint(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_constraint", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_constraint"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_constraint"); - } - - ThriftHiveMetastore_drop_constraint_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_constraint", bytes); - } - - ThriftHiveMetastore_drop_constraint_result result; - try { - iface_->drop_constraint(args.req); - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_constraint"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_constraint", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_constraint"); - } - - oprot->writeMessageBegin("drop_constraint", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_constraint", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_add_primary_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_primary_key", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_primary_key"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_primary_key"); - } - - ThriftHiveMetastore_add_primary_key_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_primary_key", bytes); - } - - ThriftHiveMetastore_add_primary_key_result result; - try { - iface_->add_primary_key(args.req); - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_primary_key"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_primary_key", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_primary_key"); - } - - oprot->writeMessageBegin("add_primary_key", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_primary_key", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_add_foreign_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_foreign_key", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_foreign_key"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_foreign_key"); - } - - ThriftHiveMetastore_add_foreign_key_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_foreign_key", bytes); - } - - ThriftHiveMetastore_add_foreign_key_result result; - try { - iface_->add_foreign_key(args.req); - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_foreign_key"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_foreign_key", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_foreign_key"); - } - - oprot->writeMessageBegin("add_foreign_key", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_foreign_key", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_add_unique_constraint(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_unique_constraint", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_unique_constraint"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_unique_constraint"); - } - - ThriftHiveMetastore_add_unique_constraint_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_unique_constraint", bytes); - } - - ThriftHiveMetastore_add_unique_constraint_result result; - try { - iface_->add_unique_constraint(args.req); - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_unique_constraint"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_unique_constraint", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_unique_constraint"); - } - - oprot->writeMessageBegin("add_unique_constraint", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_unique_constraint", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_add_not_null_constraint(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_not_null_constraint", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_not_null_constraint"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_not_null_constraint"); - } - - ThriftHiveMetastore_add_not_null_constraint_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_not_null_constraint", bytes); - } - - ThriftHiveMetastore_add_not_null_constraint_result result; - try { - iface_->add_not_null_constraint(args.req); - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_not_null_constraint"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_not_null_constraint", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_not_null_constraint"); - } - - oprot->writeMessageBegin("add_not_null_constraint", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_not_null_constraint", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_drop_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_table", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_table"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_table"); - } - - ThriftHiveMetastore_drop_table_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_table", bytes); - } - - ThriftHiveMetastore_drop_table_result result; - try { - iface_->drop_table(args.dbname, args.name, args.deleteData); - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_table"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_table"); - } - - oprot->writeMessageBegin("drop_table", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_table", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_drop_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_table_with_environment_context", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); - } - - ThriftHiveMetastore_drop_table_with_environment_context_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_table_with_environment_context", bytes); - } - - ThriftHiveMetastore_drop_table_with_environment_context_result result; - try { - iface_->drop_table_with_environment_context(args.dbname, args.name, args.deleteData, args.environment_context); - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); - } - - oprot->writeMessageBegin("drop_table_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_table_with_environment_context", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_truncate_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.truncate_table", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.truncate_table"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.truncate_table"); - } - - ThriftHiveMetastore_truncate_table_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.truncate_table", bytes); - } - - ThriftHiveMetastore_truncate_table_result result; - try { - iface_->truncate_table(args.dbName, args.tableName, args.partNames); - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.truncate_table"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("truncate_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.truncate_table"); - } - - oprot->writeMessageBegin("truncate_table", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.truncate_table", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_tables", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_tables"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_tables"); - } - - ThriftHiveMetastore_get_tables_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_tables", bytes); - } - - ThriftHiveMetastore_get_tables_result result; - try { - iface_->get_tables(result.success, args.db_name, args.pattern); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_tables"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_tables", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_tables"); - } - - oprot->writeMessageBegin("get_tables", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_tables", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_tables_by_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_tables_by_type", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_tables_by_type"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_tables_by_type"); - } - - ThriftHiveMetastore_get_tables_by_type_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_tables_by_type", bytes); - } - - ThriftHiveMetastore_get_tables_by_type_result result; - try { - iface_->get_tables_by_type(result.success, args.db_name, args.pattern, args.tableType); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_tables_by_type"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_tables_by_type", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_tables_by_type"); - } - - oprot->writeMessageBegin("get_tables_by_type", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_tables_by_type", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_table_meta(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_meta", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_meta"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_meta"); - } - - ThriftHiveMetastore_get_table_meta_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_meta", bytes); - } - - ThriftHiveMetastore_get_table_meta_result result; - try { - iface_->get_table_meta(result.success, args.db_patterns, args.tbl_patterns, args.tbl_types); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_meta"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_table_meta", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_meta"); - } - - oprot->writeMessageBegin("get_table_meta", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_meta", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_all_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_all_tables", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_all_tables"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_tables"); - } - - ThriftHiveMetastore_get_all_tables_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_all_tables", bytes); - } - - ThriftHiveMetastore_get_all_tables_result result; - try { - iface_->get_all_tables(result.success, args.db_name); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_all_tables"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_all_tables", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_tables"); - } - - oprot->writeMessageBegin("get_all_tables", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_all_tables", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table"); - } - - ThriftHiveMetastore_get_table_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table", bytes); - } - - ThriftHiveMetastore_get_table_result result; - try { - iface_->get_table(result.success, args.dbname, args.tbl_name); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table"); - } - - oprot->writeMessageBegin("get_table", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_table_objects_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_objects_by_name", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_objects_by_name"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_objects_by_name"); - } - - ThriftHiveMetastore_get_table_objects_by_name_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_objects_by_name", bytes); - } - - ThriftHiveMetastore_get_table_objects_by_name_result result; - try { - iface_->get_table_objects_by_name(result.success, args.dbname, args.tbl_names); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_objects_by_name"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_table_objects_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_objects_by_name"); - } - - oprot->writeMessageBegin("get_table_objects_by_name", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_objects_by_name", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_table_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_req", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_req"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_req"); - } - - ThriftHiveMetastore_get_table_req_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_req", bytes); - } - - ThriftHiveMetastore_get_table_req_result result; - try { - iface_->get_table_req(result.success, args.req); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_req"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_table_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_req"); - } - - oprot->writeMessageBegin("get_table_req", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_req", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_table_objects_by_name_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_objects_by_name_req", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_objects_by_name_req"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_objects_by_name_req"); - } - - ThriftHiveMetastore_get_table_objects_by_name_req_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_objects_by_name_req", bytes); - } - - ThriftHiveMetastore_get_table_objects_by_name_req_result result; - try { - iface_->get_table_objects_by_name_req(result.success, args.req); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (InvalidOperationException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (UnknownDBException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_objects_by_name_req"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_table_objects_by_name_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_objects_by_name_req"); - } - - oprot->writeMessageBegin("get_table_objects_by_name_req", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_objects_by_name_req", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_table_names_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_names_by_filter", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_names_by_filter"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_names_by_filter"); - } - - ThriftHiveMetastore_get_table_names_by_filter_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_names_by_filter", bytes); - } - - ThriftHiveMetastore_get_table_names_by_filter_result result; - try { - iface_->get_table_names_by_filter(result.success, args.dbname, args.filter, args.max_tables); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (InvalidOperationException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (UnknownDBException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_names_by_filter"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_table_names_by_filter", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_names_by_filter"); - } - - oprot->writeMessageBegin("get_table_names_by_filter", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_names_by_filter", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_alter_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_table", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_table"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_table"); - } - - ThriftHiveMetastore_alter_table_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_table", bytes); - } - - ThriftHiveMetastore_alter_table_result result; - try { - iface_->alter_table(args.dbname, args.tbl_name, args.new_tbl); - } catch (InvalidOperationException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_table"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_table"); - } - - oprot->writeMessageBegin("alter_table", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_table", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_alter_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_table_with_environment_context", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); - } - - ThriftHiveMetastore_alter_table_with_environment_context_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_table_with_environment_context", bytes); - } - - ThriftHiveMetastore_alter_table_with_environment_context_result result; - try { - iface_->alter_table_with_environment_context(args.dbname, args.tbl_name, args.new_tbl, args.environment_context); - } catch (InvalidOperationException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); - } - - oprot->writeMessageBegin("alter_table_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_table_with_environment_context", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_alter_table_with_cascade(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_table_with_cascade", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_table_with_cascade"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_table_with_cascade"); - } - - ThriftHiveMetastore_alter_table_with_cascade_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_table_with_cascade", bytes); - } - - ThriftHiveMetastore_alter_table_with_cascade_result result; - try { - iface_->alter_table_with_cascade(args.dbname, args.tbl_name, args.new_tbl, args.cascade); - } catch (InvalidOperationException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_table_with_cascade"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_table_with_cascade", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_table_with_cascade"); - } - - oprot->writeMessageBegin("alter_table_with_cascade", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_table_with_cascade", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_add_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partition", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partition"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partition"); - } - - ThriftHiveMetastore_add_partition_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partition", bytes); - } - - ThriftHiveMetastore_add_partition_result result; - try { - iface_->add_partition(result.success, args.new_part); - result.__isset.success = true; - } catch (InvalidObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partition"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partition"); - } - - oprot->writeMessageBegin("add_partition", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partition", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_add_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partition_with_environment_context", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); - } - - ThriftHiveMetastore_add_partition_with_environment_context_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partition_with_environment_context", bytes); - } - - ThriftHiveMetastore_add_partition_with_environment_context_result result; - try { - iface_->add_partition_with_environment_context(result.success, args.new_part, args.environment_context); - result.__isset.success = true; - } catch (InvalidObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); - } - - oprot->writeMessageBegin("add_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partition_with_environment_context", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_add_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partitions", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partitions"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partitions"); - } - - ThriftHiveMetastore_add_partitions_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partitions", bytes); - } - - ThriftHiveMetastore_add_partitions_result result; - try { - result.success = iface_->add_partitions(args.new_parts); - result.__isset.success = true; - } catch (InvalidObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partitions"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partitions"); - } - - oprot->writeMessageBegin("add_partitions", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partitions", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_add_partitions_pspec(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partitions_pspec", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partitions_pspec"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partitions_pspec"); - } - - ThriftHiveMetastore_add_partitions_pspec_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partitions_pspec", bytes); - } - - ThriftHiveMetastore_add_partitions_pspec_result result; - try { - result.success = iface_->add_partitions_pspec(args.new_parts); - result.__isset.success = true; - } catch (InvalidObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partitions_pspec"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_partitions_pspec", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partitions_pspec"); - } - - oprot->writeMessageBegin("add_partitions_pspec", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partitions_pspec", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_append_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition"); - } - - ThriftHiveMetastore_append_partition_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition", bytes); - } - - ThriftHiveMetastore_append_partition_result result; - try { - iface_->append_partition(result.success, args.db_name, args.tbl_name, args.part_vals); - result.__isset.success = true; - } catch (InvalidObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("append_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition"); - } - - oprot->writeMessageBegin("append_partition", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_add_partitions_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partitions_req", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partitions_req"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partitions_req"); - } - - ThriftHiveMetastore_add_partitions_req_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partitions_req", bytes); - } - - ThriftHiveMetastore_add_partitions_req_result result; - try { - iface_->add_partitions_req(result.success, args.request); - result.__isset.success = true; - } catch (InvalidObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partitions_req"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_partitions_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partitions_req"); - } - - oprot->writeMessageBegin("add_partitions_req", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partitions_req", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_append_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition_with_environment_context", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); - } - - ThriftHiveMetastore_append_partition_with_environment_context_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition_with_environment_context", bytes); - } - - ThriftHiveMetastore_append_partition_with_environment_context_result result; - try { - iface_->append_partition_with_environment_context(result.success, args.db_name, args.tbl_name, args.part_vals, args.environment_context); - result.__isset.success = true; - } catch (InvalidObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); - } - - oprot->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition_with_environment_context", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_append_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition_by_name", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition_by_name"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_by_name"); - } - - ThriftHiveMetastore_append_partition_by_name_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition_by_name", bytes); - } - - ThriftHiveMetastore_append_partition_by_name_result result; - try { - iface_->append_partition_by_name(result.success, args.db_name, args.tbl_name, args.part_name); - result.__isset.success = true; - } catch (InvalidObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition_by_name"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("append_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_by_name"); - } - - oprot->writeMessageBegin("append_partition_by_name", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition_by_name", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_append_partition_by_name_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition_by_name_with_environment_context", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); - } - - ThriftHiveMetastore_append_partition_by_name_with_environment_context_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context", bytes); - } - - ThriftHiveMetastore_append_partition_by_name_with_environment_context_result result; - try { - iface_->append_partition_by_name_with_environment_context(result.success, args.db_name, args.tbl_name, args.part_name, args.environment_context); - result.__isset.success = true; - } catch (InvalidObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); - } - - oprot->writeMessageBegin("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_drop_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition"); - } - - ThriftHiveMetastore_drop_partition_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition", bytes); - } - - ThriftHiveMetastore_drop_partition_result result; - try { - result.success = iface_->drop_partition(args.db_name, args.tbl_name, args.part_vals, args.deleteData); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition"); - } - - oprot->writeMessageBegin("drop_partition", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_drop_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition_with_environment_context", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); - } - - ThriftHiveMetastore_drop_partition_with_environment_context_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context", bytes); - } - - ThriftHiveMetastore_drop_partition_with_environment_context_result result; - try { - result.success = iface_->drop_partition_with_environment_context(args.db_name, args.tbl_name, args.part_vals, args.deleteData, args.environment_context); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); - } - - oprot->writeMessageBegin("drop_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_drop_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition_by_name", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition_by_name"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_by_name"); - } - - ThriftHiveMetastore_drop_partition_by_name_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition_by_name", bytes); - } - - ThriftHiveMetastore_drop_partition_by_name_result result; - try { - result.success = iface_->drop_partition_by_name(args.db_name, args.tbl_name, args.part_name, args.deleteData); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition_by_name"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name"); - } - - oprot->writeMessageBegin("drop_partition_by_name", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_drop_partition_by_name_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition_by_name_with_environment_context", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); - } - - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context", bytes); - } - - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result result; - try { - result.success = iface_->drop_partition_by_name_with_environment_context(args.db_name, args.tbl_name, args.part_name, args.deleteData, args.environment_context); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); - } - - oprot->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_drop_partitions_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partitions_req", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partitions_req"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partitions_req"); - } - - ThriftHiveMetastore_drop_partitions_req_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partitions_req", bytes); - } - - ThriftHiveMetastore_drop_partitions_req_result result; - try { - iface_->drop_partitions_req(result.success, args.req); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partitions_req"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_partitions_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partitions_req"); - } - - oprot->writeMessageBegin("drop_partitions_req", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partitions_req", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition"); - } - - ThriftHiveMetastore_get_partition_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition", bytes); - } - - ThriftHiveMetastore_get_partition_result result; - try { - iface_->get_partition(result.success, args.db_name, args.tbl_name, args.part_vals); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition"); - } - - oprot->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_exchange_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.exchange_partition", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.exchange_partition"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.exchange_partition"); - } - - ThriftHiveMetastore_exchange_partition_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.exchange_partition", bytes); - } - - ThriftHiveMetastore_exchange_partition_result result; - try { - iface_->exchange_partition(result.success, args.partitionSpecs, args.source_db, args.source_table_name, args.dest_db, args.dest_table_name); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (InvalidObjectException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (InvalidInputException &o4) { - result.o4 = o4; - result.__isset.o4 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.exchange_partition"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.exchange_partition"); - } - - oprot->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.exchange_partition", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_exchange_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.exchange_partitions", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.exchange_partitions"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.exchange_partitions"); - } - - ThriftHiveMetastore_exchange_partitions_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.exchange_partitions", bytes); - } - - ThriftHiveMetastore_exchange_partitions_result result; - try { - iface_->exchange_partitions(result.success, args.partitionSpecs, args.source_db, args.source_table_name, args.dest_db, args.dest_table_name); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (InvalidObjectException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (InvalidInputException &o4) { - result.o4 = o4; - result.__isset.o4 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.exchange_partitions"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("exchange_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.exchange_partitions"); - } - - oprot->writeMessageBegin("exchange_partitions", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.exchange_partitions", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_partition_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_with_auth", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_with_auth"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_with_auth"); - } - - ThriftHiveMetastore_get_partition_with_auth_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_with_auth", bytes); - } - - ThriftHiveMetastore_get_partition_with_auth_result result; - try { - iface_->get_partition_with_auth(result.success, args.db_name, args.tbl_name, args.part_vals, args.user_name, args.group_names); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_with_auth"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_with_auth"); - } - - oprot->writeMessageBegin("get_partition_with_auth", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_with_auth", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_by_name", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_by_name"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_by_name"); - } - - ThriftHiveMetastore_get_partition_by_name_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_by_name", bytes); - } - - ThriftHiveMetastore_get_partition_by_name_result result; - try { - iface_->get_partition_by_name(result.success, args.db_name, args.tbl_name, args.part_name); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_by_name"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_by_name"); - } - - oprot->writeMessageBegin("get_partition_by_name", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_by_name", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions"); - } - - ThriftHiveMetastore_get_partitions_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions", bytes); - } - - ThriftHiveMetastore_get_partitions_result result; - try { - iface_->get_partitions(result.success, args.db_name, args.tbl_name, args.max_parts); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions"); - } - - oprot->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_partitions_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_with_auth", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_with_auth"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_with_auth"); - } - - ThriftHiveMetastore_get_partitions_with_auth_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_with_auth", bytes); - } - - ThriftHiveMetastore_get_partitions_with_auth_result result; - try { - iface_->get_partitions_with_auth(result.success, args.db_name, args.tbl_name, args.max_parts, args.user_name, args.group_names); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_with_auth"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_with_auth"); - } - - oprot->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_with_auth", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_partitions_pspec(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_pspec", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_pspec"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_pspec"); - } - - ThriftHiveMetastore_get_partitions_pspec_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_pspec", bytes); - } - - ThriftHiveMetastore_get_partitions_pspec_result result; - try { - iface_->get_partitions_pspec(result.success, args.db_name, args.tbl_name, args.max_parts); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_pspec"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_pspec", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_pspec"); - } - - oprot->writeMessageBegin("get_partitions_pspec", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_pspec", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_partition_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_names", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_names"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_names"); - } - - ThriftHiveMetastore_get_partition_names_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_names", bytes); - } - - ThriftHiveMetastore_get_partition_names_result result; - try { - iface_->get_partition_names(result.success, args.db_name, args.tbl_name, args.max_parts); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_names"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_names"); - } - - oprot->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_names", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_partition_values(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_values", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_values"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_values"); - } - - ThriftHiveMetastore_get_partition_values_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_values", bytes); - } - - ThriftHiveMetastore_get_partition_values_result result; - try { - iface_->get_partition_values(result.success, args.request); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_values"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition_values", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_values"); - } - - oprot->writeMessageBegin("get_partition_values", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_values", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_partitions_ps(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_ps", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_ps"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_ps"); - } - - ThriftHiveMetastore_get_partitions_ps_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_ps", bytes); - } - - ThriftHiveMetastore_get_partitions_ps_result result; - try { - iface_->get_partitions_ps(result.success, args.db_name, args.tbl_name, args.part_vals, args.max_parts); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_ps"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_ps"); - } - - oprot->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_ps", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_partitions_ps_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_ps_with_auth", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); - } - - ThriftHiveMetastore_get_partitions_ps_with_auth_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth", bytes); - } - - ThriftHiveMetastore_get_partitions_ps_with_auth_result result; - try { - iface_->get_partitions_ps_with_auth(result.success, args.db_name, args.tbl_name, args.part_vals, args.max_parts, args.user_name, args.group_names); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); - } - - oprot->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_partition_names_ps(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_names_ps", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_names_ps"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_names_ps"); - } - - ThriftHiveMetastore_get_partition_names_ps_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_names_ps", bytes); - } - - ThriftHiveMetastore_get_partition_names_ps_result result; - try { - iface_->get_partition_names_ps(result.success, args.db_name, args.tbl_name, args.part_vals, args.max_parts); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_names_ps"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_names_ps"); - } - - oprot->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_names_ps", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_partitions_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_by_filter", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_by_filter"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_by_filter"); - } - - ThriftHiveMetastore_get_partitions_by_filter_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_by_filter", bytes); - } - - ThriftHiveMetastore_get_partitions_by_filter_result result; - try { - iface_->get_partitions_by_filter(result.success, args.db_name, args.tbl_name, args.filter, args.max_parts); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_by_filter"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_by_filter"); - } - - oprot->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_by_filter", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_part_specs_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_part_specs_by_filter", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_part_specs_by_filter"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_part_specs_by_filter"); - } - - ThriftHiveMetastore_get_part_specs_by_filter_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_part_specs_by_filter", bytes); - } - - ThriftHiveMetastore_get_part_specs_by_filter_result result; - try { - iface_->get_part_specs_by_filter(result.success, args.db_name, args.tbl_name, args.filter, args.max_parts); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_part_specs_by_filter"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_part_specs_by_filter", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_part_specs_by_filter"); - } - - oprot->writeMessageBegin("get_part_specs_by_filter", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_part_specs_by_filter", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_partitions_by_expr(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_by_expr", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_by_expr"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_by_expr"); - } - - ThriftHiveMetastore_get_partitions_by_expr_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_by_expr", bytes); - } - - ThriftHiveMetastore_get_partitions_by_expr_result result; - try { - iface_->get_partitions_by_expr(result.success, args.req); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_by_expr"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_by_expr", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_by_expr"); - } - - oprot->writeMessageBegin("get_partitions_by_expr", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_by_expr", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_num_partitions_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_num_partitions_by_filter", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_num_partitions_by_filter"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_num_partitions_by_filter"); - } - - ThriftHiveMetastore_get_num_partitions_by_filter_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_num_partitions_by_filter", bytes); - } - - ThriftHiveMetastore_get_num_partitions_by_filter_result result; - try { - result.success = iface_->get_num_partitions_by_filter(args.db_name, args.tbl_name, args.filter); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_num_partitions_by_filter"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_num_partitions_by_filter", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_num_partitions_by_filter"); - } - - oprot->writeMessageBegin("get_num_partitions_by_filter", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_num_partitions_by_filter", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_partitions_by_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_by_names", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_by_names"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_by_names"); - } - - ThriftHiveMetastore_get_partitions_by_names_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_by_names", bytes); - } - - ThriftHiveMetastore_get_partitions_by_names_result result; - try { - iface_->get_partitions_by_names(result.success, args.db_name, args.tbl_name, args.names); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_by_names"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_by_names"); - } - - oprot->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_by_names", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_alter_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_partition", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partition"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partition"); - } - - ThriftHiveMetastore_alter_partition_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_partition", bytes); - } - - ThriftHiveMetastore_alter_partition_result result; - try { - iface_->alter_partition(args.db_name, args.tbl_name, args.new_part); - } catch (InvalidOperationException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partition"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partition"); - } - - oprot->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_partition", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_alter_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_partitions", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partitions"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partitions"); - } - - ThriftHiveMetastore_alter_partitions_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_partitions", bytes); - } - - ThriftHiveMetastore_alter_partitions_result result; - try { - iface_->alter_partitions(args.db_name, args.tbl_name, args.new_parts); - } catch (InvalidOperationException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partitions"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partitions"); - } - - oprot->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_partitions", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_alter_partitions_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_partitions_with_environment_context", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partitions_with_environment_context"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partitions_with_environment_context"); - } - - ThriftHiveMetastore_alter_partitions_with_environment_context_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_partitions_with_environment_context", bytes); - } - - ThriftHiveMetastore_alter_partitions_with_environment_context_result result; - try { - iface_->alter_partitions_with_environment_context(args.db_name, args.tbl_name, args.new_parts, args.environment_context); - } catch (InvalidOperationException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partitions_with_environment_context"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_partitions_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partitions_with_environment_context"); - } - - oprot->writeMessageBegin("alter_partitions_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_partitions_with_environment_context", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_alter_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_partition_with_environment_context", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); - } - - ThriftHiveMetastore_alter_partition_with_environment_context_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context", bytes); - } - - ThriftHiveMetastore_alter_partition_with_environment_context_result result; - try { - iface_->alter_partition_with_environment_context(args.db_name, args.tbl_name, args.new_part, args.environment_context); - } catch (InvalidOperationException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); - } - - oprot->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_rename_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.rename_partition", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.rename_partition"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.rename_partition"); - } - - ThriftHiveMetastore_rename_partition_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.rename_partition", bytes); - } - - ThriftHiveMetastore_rename_partition_result result; - try { - iface_->rename_partition(args.db_name, args.tbl_name, args.part_vals, args.new_part); - } catch (InvalidOperationException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.rename_partition"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.rename_partition"); - } - - oprot->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.rename_partition", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_partition_name_has_valid_characters(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.partition_name_has_valid_characters", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); - } - - ThriftHiveMetastore_partition_name_has_valid_characters_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters", bytes); - } - - ThriftHiveMetastore_partition_name_has_valid_characters_result result; - try { - result.success = iface_->partition_name_has_valid_characters(args.part_vals, args.throw_exception); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("partition_name_has_valid_characters", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); - } - - oprot->writeMessageBegin("partition_name_has_valid_characters", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_config_value(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_config_value", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_config_value"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_config_value"); - } - - ThriftHiveMetastore_get_config_value_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_config_value", bytes); - } - - ThriftHiveMetastore_get_config_value_result result; - try { - iface_->get_config_value(result.success, args.name, args.defaultValue); - result.__isset.success = true; - } catch (ConfigValSecurityException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_config_value"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_config_value"); - } - - oprot->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_config_value", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_partition_name_to_vals(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.partition_name_to_vals", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_to_vals"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_to_vals"); - } - - ThriftHiveMetastore_partition_name_to_vals_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.partition_name_to_vals", bytes); - } - - ThriftHiveMetastore_partition_name_to_vals_result result; - try { - iface_->partition_name_to_vals(result.success, args.part_name); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.partition_name_to_vals"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_to_vals"); - } - - oprot->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.partition_name_to_vals", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_partition_name_to_spec(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.partition_name_to_spec", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_to_spec"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_to_spec"); - } - - ThriftHiveMetastore_partition_name_to_spec_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.partition_name_to_spec", bytes); - } - - ThriftHiveMetastore_partition_name_to_spec_result result; - try { - iface_->partition_name_to_spec(result.success, args.part_name); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.partition_name_to_spec"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_to_spec"); - } - - oprot->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.partition_name_to_spec", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_markPartitionForEvent(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.markPartitionForEvent", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.markPartitionForEvent"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.markPartitionForEvent"); - } - - ThriftHiveMetastore_markPartitionForEvent_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.markPartitionForEvent", bytes); - } - - ThriftHiveMetastore_markPartitionForEvent_result result; - try { - iface_->markPartitionForEvent(args.db_name, args.tbl_name, args.part_vals, args.eventType); - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (UnknownDBException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (UnknownTableException &o4) { - result.o4 = o4; - result.__isset.o4 = true; - } catch (UnknownPartitionException &o5) { - result.o5 = o5; - result.__isset.o5 = true; - } catch (InvalidPartitionException &o6) { - result.o6 = o6; - result.__isset.o6 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.markPartitionForEvent"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.markPartitionForEvent"); - } - - oprot->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.markPartitionForEvent", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_isPartitionMarkedForEvent(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.isPartitionMarkedForEvent", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); - } - - ThriftHiveMetastore_isPartitionMarkedForEvent_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent", bytes); - } - - ThriftHiveMetastore_isPartitionMarkedForEvent_result result; - try { - result.success = iface_->isPartitionMarkedForEvent(args.db_name, args.tbl_name, args.part_vals, args.eventType); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (UnknownDBException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (UnknownTableException &o4) { - result.o4 = o4; - result.__isset.o4 = true; - } catch (UnknownPartitionException &o5) { - result.o5 = o5; - result.__isset.o5 = true; - } catch (InvalidPartitionException &o6) { - result.o6 = o6; - result.__isset.o6 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); - } - - oprot->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_add_index(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_index", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_index"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_index"); - } - - ThriftHiveMetastore_add_index_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_index", bytes); - } - - ThriftHiveMetastore_add_index_result result; - try { - iface_->add_index(result.success, args.new_index, args.index_table); - result.__isset.success = true; - } catch (InvalidObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_index"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_index", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_index"); - } - - oprot->writeMessageBegin("add_index", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_index", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_alter_index(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_index", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_index"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_index"); - } - - ThriftHiveMetastore_alter_index_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_index", bytes); - } - - ThriftHiveMetastore_alter_index_result result; - try { - iface_->alter_index(args.dbname, args.base_tbl_name, args.idx_name, args.new_idx); - } catch (InvalidOperationException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_index"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_index"); - } - - oprot->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_index", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_drop_index_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_index_by_name", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_index_by_name"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_index_by_name"); - } - - ThriftHiveMetastore_drop_index_by_name_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_index_by_name", bytes); - } - - ThriftHiveMetastore_drop_index_by_name_result result; - try { - result.success = iface_->drop_index_by_name(args.db_name, args.tbl_name, args.index_name, args.deleteData); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_index_by_name"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_index_by_name"); - } - - oprot->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_index_by_name", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_index_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_index_by_name", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_index_by_name"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_index_by_name"); - } - - ThriftHiveMetastore_get_index_by_name_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_index_by_name", bytes); - } - - ThriftHiveMetastore_get_index_by_name_result result; - try { - iface_->get_index_by_name(result.success, args.db_name, args.tbl_name, args.index_name); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_index_by_name"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_index_by_name"); - } - - oprot->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_index_by_name", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_indexes(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_indexes", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_indexes"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_indexes"); - } - - ThriftHiveMetastore_get_indexes_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_indexes", bytes); - } - - ThriftHiveMetastore_get_indexes_result result; - try { - iface_->get_indexes(result.success, args.db_name, args.tbl_name, args.max_indexes); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_indexes"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_indexes"); - } - - oprot->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_indexes", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_index_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_index_names", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_index_names"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_index_names"); - } - - ThriftHiveMetastore_get_index_names_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_index_names", bytes); - } - - ThriftHiveMetastore_get_index_names_result result; - try { - iface_->get_index_names(result.success, args.db_name, args.tbl_name, args.max_indexes); - result.__isset.success = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_index_names"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_index_names"); - } - - oprot->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_index_names", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_primary_keys(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_primary_keys", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_primary_keys"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_primary_keys"); - } - - ThriftHiveMetastore_get_primary_keys_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_primary_keys", bytes); - } - - ThriftHiveMetastore_get_primary_keys_result result; - try { - iface_->get_primary_keys(result.success, args.request); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_primary_keys"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_primary_keys", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_primary_keys"); - } - - oprot->writeMessageBegin("get_primary_keys", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_primary_keys", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_foreign_keys(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_foreign_keys", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_foreign_keys"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_foreign_keys"); - } - - ThriftHiveMetastore_get_foreign_keys_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_foreign_keys", bytes); - } - - ThriftHiveMetastore_get_foreign_keys_result result; - try { - iface_->get_foreign_keys(result.success, args.request); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_foreign_keys"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_foreign_keys", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_foreign_keys"); - } - - oprot->writeMessageBegin("get_foreign_keys", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_foreign_keys", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_unique_constraints(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_unique_constraints", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_unique_constraints"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_unique_constraints"); - } - - ThriftHiveMetastore_get_unique_constraints_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_unique_constraints", bytes); - } - - ThriftHiveMetastore_get_unique_constraints_result result; - try { - iface_->get_unique_constraints(result.success, args.request); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_unique_constraints"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_unique_constraints", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_unique_constraints"); - } - - oprot->writeMessageBegin("get_unique_constraints", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_unique_constraints", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_not_null_constraints(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_not_null_constraints", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_not_null_constraints"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_not_null_constraints"); - } - - ThriftHiveMetastore_get_not_null_constraints_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_not_null_constraints", bytes); - } - - ThriftHiveMetastore_get_not_null_constraints_result result; - try { - iface_->get_not_null_constraints(result.success, args.request); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_not_null_constraints"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_not_null_constraints", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_not_null_constraints"); - } - - oprot->writeMessageBegin("get_not_null_constraints", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_not_null_constraints", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_update_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.update_table_column_statistics", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.update_table_column_statistics"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.update_table_column_statistics"); - } - - ThriftHiveMetastore_update_table_column_statistics_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.update_table_column_statistics", bytes); - } - - ThriftHiveMetastore_update_table_column_statistics_result result; - try { - result.success = iface_->update_table_column_statistics(args.stats_obj); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (InvalidInputException &o4) { - result.o4 = o4; - result.__isset.o4 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.update_table_column_statistics"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("update_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.update_table_column_statistics"); - } - - oprot->writeMessageBegin("update_table_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.update_table_column_statistics", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_update_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.update_partition_column_statistics", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.update_partition_column_statistics"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.update_partition_column_statistics"); - } - - ThriftHiveMetastore_update_partition_column_statistics_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.update_partition_column_statistics", bytes); - } - - ThriftHiveMetastore_update_partition_column_statistics_result result; - try { - result.success = iface_->update_partition_column_statistics(args.stats_obj); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (InvalidInputException &o4) { - result.o4 = o4; - result.__isset.o4 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.update_partition_column_statistics"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.update_partition_column_statistics"); - } - - oprot->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.update_partition_column_statistics", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_column_statistics", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_column_statistics"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_column_statistics"); - } - - ThriftHiveMetastore_get_table_column_statistics_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_column_statistics", bytes); - } - - ThriftHiveMetastore_get_table_column_statistics_result result; - try { - iface_->get_table_column_statistics(result.success, args.db_name, args.tbl_name, args.col_name); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (InvalidInputException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (InvalidObjectException &o4) { - result.o4 = o4; - result.__isset.o4 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_column_statistics"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_column_statistics"); - } - - oprot->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_column_statistics", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_column_statistics", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_column_statistics"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_column_statistics"); - } - - ThriftHiveMetastore_get_partition_column_statistics_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_column_statistics", bytes); - } - - ThriftHiveMetastore_get_partition_column_statistics_result result; - try { - iface_->get_partition_column_statistics(result.success, args.db_name, args.tbl_name, args.part_name, args.col_name); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (InvalidInputException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (InvalidObjectException &o4) { - result.o4 = o4; - result.__isset.o4 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_column_statistics"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_column_statistics"); - } - - oprot->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_column_statistics", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_table_statistics_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_statistics_req", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_statistics_req"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_statistics_req"); - } - - ThriftHiveMetastore_get_table_statistics_req_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_statistics_req", bytes); - } - - ThriftHiveMetastore_get_table_statistics_req_result result; - try { - iface_->get_table_statistics_req(result.success, args.request); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_statistics_req"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_table_statistics_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_statistics_req"); - } - - oprot->writeMessageBegin("get_table_statistics_req", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_statistics_req", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_partitions_statistics_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_statistics_req", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_statistics_req"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_statistics_req"); - } - - ThriftHiveMetastore_get_partitions_statistics_req_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_statistics_req", bytes); - } - - ThriftHiveMetastore_get_partitions_statistics_req_result result; - try { - iface_->get_partitions_statistics_req(result.success, args.request); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_statistics_req"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_statistics_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_statistics_req"); - } - - oprot->writeMessageBegin("get_partitions_statistics_req", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_statistics_req", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_aggr_stats_for(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_aggr_stats_for", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_aggr_stats_for"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_aggr_stats_for"); - } - - ThriftHiveMetastore_get_aggr_stats_for_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_aggr_stats_for", bytes); - } - - ThriftHiveMetastore_get_aggr_stats_for_result result; - try { - iface_->get_aggr_stats_for(result.success, args.request); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_aggr_stats_for"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_aggr_stats_for", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_aggr_stats_for"); - } - - oprot->writeMessageBegin("get_aggr_stats_for", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_aggr_stats_for", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_set_aggr_stats_for(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.set_aggr_stats_for", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.set_aggr_stats_for"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.set_aggr_stats_for"); - } - - ThriftHiveMetastore_set_aggr_stats_for_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.set_aggr_stats_for", bytes); - } - - ThriftHiveMetastore_set_aggr_stats_for_result result; - try { - result.success = iface_->set_aggr_stats_for(args.request); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (InvalidInputException &o4) { - result.o4 = o4; - result.__isset.o4 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.set_aggr_stats_for"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("set_aggr_stats_for", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.set_aggr_stats_for"); - } - - oprot->writeMessageBegin("set_aggr_stats_for", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.set_aggr_stats_for", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_delete_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.delete_partition_column_statistics", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); - } - - ThriftHiveMetastore_delete_partition_column_statistics_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.delete_partition_column_statistics", bytes); - } - - ThriftHiveMetastore_delete_partition_column_statistics_result result; - try { - result.success = iface_->delete_partition_column_statistics(args.db_name, args.tbl_name, args.part_name, args.col_name); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (InvalidObjectException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (InvalidInputException &o4) { - result.o4 = o4; - result.__isset.o4 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); - } - - oprot->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.delete_partition_column_statistics", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_delete_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.delete_table_column_statistics", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.delete_table_column_statistics"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.delete_table_column_statistics"); - } - - ThriftHiveMetastore_delete_table_column_statistics_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.delete_table_column_statistics", bytes); - } - - ThriftHiveMetastore_delete_table_column_statistics_result result; - try { - result.success = iface_->delete_table_column_statistics(args.db_name, args.tbl_name, args.col_name); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (InvalidObjectException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (InvalidInputException &o4) { - result.o4 = o4; - result.__isset.o4 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.delete_table_column_statistics"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.delete_table_column_statistics"); - } - - oprot->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.delete_table_column_statistics", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_create_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_function", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_function"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_function"); - } - - ThriftHiveMetastore_create_function_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_function", bytes); - } - - ThriftHiveMetastore_create_function_result result; - try { - iface_->create_function(args.func); - } catch (AlreadyExistsException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (NoSuchObjectException &o4) { - result.o4 = o4; - result.__isset.o4 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_function"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_function", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_function"); - } - - oprot->writeMessageBegin("create_function", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_function", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_drop_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_function", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_function"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_function"); - } - - ThriftHiveMetastore_drop_function_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_function", bytes); - } - - ThriftHiveMetastore_drop_function_result result; - try { - iface_->drop_function(args.dbName, args.funcName); - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_function"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_function", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_function"); - } - - oprot->writeMessageBegin("drop_function", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_function", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_alter_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_function", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_function"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_function"); - } - - ThriftHiveMetastore_alter_function_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_function", bytes); - } - - ThriftHiveMetastore_alter_function_result result; - try { - iface_->alter_function(args.dbName, args.funcName, args.newFunc); - } catch (InvalidOperationException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_function"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_function", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_function"); - } - - oprot->writeMessageBegin("alter_function", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_function", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_functions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_functions", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_functions"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_functions"); - } - - ThriftHiveMetastore_get_functions_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_functions", bytes); - } - - ThriftHiveMetastore_get_functions_result result; - try { - iface_->get_functions(result.success, args.dbName, args.pattern); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_functions"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_functions", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_functions"); - } - - oprot->writeMessageBegin("get_functions", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_functions", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_function", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_function"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_function"); - } - - ThriftHiveMetastore_get_function_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_function", bytes); - } - - ThriftHiveMetastore_get_function_result result; - try { - iface_->get_function(result.success, args.dbName, args.funcName); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_function"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_function", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_function"); - } - - oprot->writeMessageBegin("get_function", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_function", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_all_functions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_all_functions", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_all_functions"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_functions"); - } - - ThriftHiveMetastore_get_all_functions_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_all_functions", bytes); - } - - ThriftHiveMetastore_get_all_functions_result result; - try { - iface_->get_all_functions(result.success); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_all_functions"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_all_functions", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_functions"); - } - - oprot->writeMessageBegin("get_all_functions", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_all_functions", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_create_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_role", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_role"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_role"); - } - - ThriftHiveMetastore_create_role_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_role", bytes); - } - - ThriftHiveMetastore_create_role_result result; - try { - result.success = iface_->create_role(args.role); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_role"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_role"); - } - - oprot->writeMessageBegin("create_role", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_role", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_drop_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_role", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_role"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_role"); - } - - ThriftHiveMetastore_drop_role_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_role", bytes); - } - - ThriftHiveMetastore_drop_role_result result; - try { - result.success = iface_->drop_role(args.role_name); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_role"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_role"); - } - - oprot->writeMessageBegin("drop_role", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_role", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_role_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_role_names", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_role_names"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_role_names"); - } - - ThriftHiveMetastore_get_role_names_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_role_names", bytes); - } - - ThriftHiveMetastore_get_role_names_result result; - try { - iface_->get_role_names(result.success); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_role_names"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_role_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_role_names"); - } - - oprot->writeMessageBegin("get_role_names", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_role_names", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_grant_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.grant_role", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.grant_role"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.grant_role"); - } - - ThriftHiveMetastore_grant_role_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.grant_role", bytes); - } - - ThriftHiveMetastore_grant_role_result result; - try { - result.success = iface_->grant_role(args.role_name, args.principal_name, args.principal_type, args.grantor, args.grantorType, args.grant_option); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.grant_role"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("grant_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.grant_role"); - } - - oprot->writeMessageBegin("grant_role", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.grant_role", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_revoke_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.revoke_role", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.revoke_role"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.revoke_role"); - } - - ThriftHiveMetastore_revoke_role_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.revoke_role", bytes); - } - - ThriftHiveMetastore_revoke_role_result result; - try { - result.success = iface_->revoke_role(args.role_name, args.principal_name, args.principal_type); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.revoke_role"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("revoke_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.revoke_role"); - } - - oprot->writeMessageBegin("revoke_role", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.revoke_role", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_list_roles(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.list_roles", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.list_roles"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.list_roles"); - } - - ThriftHiveMetastore_list_roles_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.list_roles", bytes); - } - - ThriftHiveMetastore_list_roles_result result; - try { - iface_->list_roles(result.success, args.principal_name, args.principal_type); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.list_roles"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("list_roles", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.list_roles"); - } - - oprot->writeMessageBegin("list_roles", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.list_roles", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_grant_revoke_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.grant_revoke_role", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.grant_revoke_role"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.grant_revoke_role"); - } - - ThriftHiveMetastore_grant_revoke_role_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.grant_revoke_role", bytes); - } - - ThriftHiveMetastore_grant_revoke_role_result result; - try { - iface_->grant_revoke_role(result.success, args.request); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.grant_revoke_role"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("grant_revoke_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.grant_revoke_role"); - } - - oprot->writeMessageBegin("grant_revoke_role", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.grant_revoke_role", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_principals_in_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_principals_in_role", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_principals_in_role"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_principals_in_role"); - } - - ThriftHiveMetastore_get_principals_in_role_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_principals_in_role", bytes); - } - - ThriftHiveMetastore_get_principals_in_role_result result; - try { - iface_->get_principals_in_role(result.success, args.request); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_principals_in_role"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_principals_in_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_principals_in_role"); - } - - oprot->writeMessageBegin("get_principals_in_role", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_principals_in_role", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_role_grants_for_principal(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_role_grants_for_principal", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_role_grants_for_principal"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_role_grants_for_principal"); - } - - ThriftHiveMetastore_get_role_grants_for_principal_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_role_grants_for_principal", bytes); - } - - ThriftHiveMetastore_get_role_grants_for_principal_result result; - try { - iface_->get_role_grants_for_principal(result.success, args.request); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_role_grants_for_principal"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_role_grants_for_principal", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_role_grants_for_principal"); - } - - oprot->writeMessageBegin("get_role_grants_for_principal", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_role_grants_for_principal", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_privilege_set(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_privilege_set", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_privilege_set"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_privilege_set"); - } - - ThriftHiveMetastore_get_privilege_set_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_privilege_set", bytes); - } - - ThriftHiveMetastore_get_privilege_set_result result; - try { - iface_->get_privilege_set(result.success, args.hiveObject, args.user_name, args.group_names); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_privilege_set"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_privilege_set", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_privilege_set"); - } - - oprot->writeMessageBegin("get_privilege_set", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_privilege_set", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_list_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.list_privileges", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.list_privileges"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.list_privileges"); - } - - ThriftHiveMetastore_list_privileges_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.list_privileges", bytes); - } - - ThriftHiveMetastore_list_privileges_result result; - try { - iface_->list_privileges(result.success, args.principal_name, args.principal_type, args.hiveObject); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.list_privileges"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("list_privileges", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.list_privileges"); - } - - oprot->writeMessageBegin("list_privileges", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.list_privileges", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_grant_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.grant_privileges", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.grant_privileges"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.grant_privileges"); - } - - ThriftHiveMetastore_grant_privileges_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.grant_privileges", bytes); - } - - ThriftHiveMetastore_grant_privileges_result result; - try { - result.success = iface_->grant_privileges(args.privileges); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.grant_privileges"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("grant_privileges", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.grant_privileges"); - } - - oprot->writeMessageBegin("grant_privileges", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.grant_privileges", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_revoke_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.revoke_privileges", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.revoke_privileges"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.revoke_privileges"); - } - - ThriftHiveMetastore_revoke_privileges_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.revoke_privileges", bytes); - } - - ThriftHiveMetastore_revoke_privileges_result result; - try { - result.success = iface_->revoke_privileges(args.privileges); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.revoke_privileges"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("revoke_privileges", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.revoke_privileges"); - } - - oprot->writeMessageBegin("revoke_privileges", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.revoke_privileges", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_grant_revoke_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.grant_revoke_privileges", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.grant_revoke_privileges"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.grant_revoke_privileges"); - } - - ThriftHiveMetastore_grant_revoke_privileges_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.grant_revoke_privileges", bytes); - } - - ThriftHiveMetastore_grant_revoke_privileges_result result; - try { - iface_->grant_revoke_privileges(result.success, args.request); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.grant_revoke_privileges"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("grant_revoke_privileges", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.grant_revoke_privileges"); - } - - oprot->writeMessageBegin("grant_revoke_privileges", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.grant_revoke_privileges", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_set_ugi(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.set_ugi", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.set_ugi"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.set_ugi"); - } - - ThriftHiveMetastore_set_ugi_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.set_ugi", bytes); - } - - ThriftHiveMetastore_set_ugi_result result; - try { - iface_->set_ugi(result.success, args.user_name, args.group_names); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.set_ugi"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("set_ugi", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.set_ugi"); - } - - oprot->writeMessageBegin("set_ugi", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.set_ugi", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_delegation_token", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_delegation_token"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_delegation_token"); - } - - ThriftHiveMetastore_get_delegation_token_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_delegation_token", bytes); - } - - ThriftHiveMetastore_get_delegation_token_result result; - try { - iface_->get_delegation_token(result.success, args.token_owner, args.renewer_kerberos_principal_name); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_delegation_token"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_delegation_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_delegation_token"); - } - - oprot->writeMessageBegin("get_delegation_token", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_delegation_token", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_renew_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.renew_delegation_token", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.renew_delegation_token"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.renew_delegation_token"); - } - - ThriftHiveMetastore_renew_delegation_token_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.renew_delegation_token", bytes); - } - - ThriftHiveMetastore_renew_delegation_token_result result; - try { - result.success = iface_->renew_delegation_token(args.token_str_form); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.renew_delegation_token"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("renew_delegation_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.renew_delegation_token"); - } - - oprot->writeMessageBegin("renew_delegation_token", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.renew_delegation_token", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_cancel_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.cancel_delegation_token", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.cancel_delegation_token"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.cancel_delegation_token"); - } - - ThriftHiveMetastore_cancel_delegation_token_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.cancel_delegation_token", bytes); - } - - ThriftHiveMetastore_cancel_delegation_token_result result; - try { - iface_->cancel_delegation_token(args.token_str_form); - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.cancel_delegation_token"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("cancel_delegation_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.cancel_delegation_token"); - } - - oprot->writeMessageBegin("cancel_delegation_token", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.cancel_delegation_token", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_add_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_token", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_token"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_token"); - } - - ThriftHiveMetastore_add_token_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_token", bytes); - } - - ThriftHiveMetastore_add_token_result result; - try { - result.success = iface_->add_token(args.token_identifier, args.delegation_token); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_token"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_token"); - } - - oprot->writeMessageBegin("add_token", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_token", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_remove_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.remove_token", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.remove_token"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.remove_token"); - } - - ThriftHiveMetastore_remove_token_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.remove_token", bytes); - } - - ThriftHiveMetastore_remove_token_result result; - try { - result.success = iface_->remove_token(args.token_identifier); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.remove_token"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("remove_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.remove_token"); - } - - oprot->writeMessageBegin("remove_token", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.remove_token", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_token", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_token"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_token"); - } - - ThriftHiveMetastore_get_token_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_token", bytes); - } - - ThriftHiveMetastore_get_token_result result; - try { - iface_->get_token(result.success, args.token_identifier); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_token"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_token"); - } - - oprot->writeMessageBegin("get_token", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_token", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_all_token_identifiers(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_all_token_identifiers", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_all_token_identifiers"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_token_identifiers"); - } - - ThriftHiveMetastore_get_all_token_identifiers_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_all_token_identifiers", bytes); - } - - ThriftHiveMetastore_get_all_token_identifiers_result result; - try { - iface_->get_all_token_identifiers(result.success); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_all_token_identifiers"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_all_token_identifiers", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_token_identifiers"); - } - - oprot->writeMessageBegin("get_all_token_identifiers", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_all_token_identifiers", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_add_master_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_master_key", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_master_key"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_master_key"); - } - - ThriftHiveMetastore_add_master_key_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_master_key", bytes); - } - - ThriftHiveMetastore_add_master_key_result result; - try { - result.success = iface_->add_master_key(args.key); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_master_key"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_master_key", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_master_key"); - } - - oprot->writeMessageBegin("add_master_key", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_master_key", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_update_master_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.update_master_key", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.update_master_key"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.update_master_key"); - } - - ThriftHiveMetastore_update_master_key_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.update_master_key", bytes); - } - - ThriftHiveMetastore_update_master_key_result result; - try { - iface_->update_master_key(args.seq_number, args.key); - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.update_master_key"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("update_master_key", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.update_master_key"); - } - - oprot->writeMessageBegin("update_master_key", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.update_master_key", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_remove_master_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.remove_master_key", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.remove_master_key"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.remove_master_key"); - } - - ThriftHiveMetastore_remove_master_key_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.remove_master_key", bytes); - } - - ThriftHiveMetastore_remove_master_key_result result; - try { - result.success = iface_->remove_master_key(args.key_seq); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.remove_master_key"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("remove_master_key", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.remove_master_key"); - } - - oprot->writeMessageBegin("remove_master_key", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.remove_master_key", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_master_keys(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_master_keys", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_master_keys"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_master_keys"); - } - - ThriftHiveMetastore_get_master_keys_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_master_keys", bytes); - } - - ThriftHiveMetastore_get_master_keys_result result; - try { - iface_->get_master_keys(result.success); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_master_keys"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_master_keys", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_master_keys"); - } - - oprot->writeMessageBegin("get_master_keys", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_master_keys", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_open_txns(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_open_txns", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_open_txns"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_open_txns"); - } - - ThriftHiveMetastore_get_open_txns_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_open_txns", bytes); - } - - ThriftHiveMetastore_get_open_txns_result result; - try { - iface_->get_open_txns(result.success); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_open_txns"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_open_txns", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_open_txns"); - } - - oprot->writeMessageBegin("get_open_txns", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_open_txns", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_open_txns_info(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_open_txns_info", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_open_txns_info"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_open_txns_info"); - } - - ThriftHiveMetastore_get_open_txns_info_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_open_txns_info", bytes); - } - - ThriftHiveMetastore_get_open_txns_info_result result; - try { - iface_->get_open_txns_info(result.success); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_open_txns_info"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_open_txns_info", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_open_txns_info"); - } - - oprot->writeMessageBegin("get_open_txns_info", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_open_txns_info", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_open_txns(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.open_txns", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.open_txns"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.open_txns"); - } - - ThriftHiveMetastore_open_txns_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.open_txns", bytes); - } - - ThriftHiveMetastore_open_txns_result result; - try { - iface_->open_txns(result.success, args.rqst); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.open_txns"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("open_txns", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.open_txns"); - } - - oprot->writeMessageBegin("open_txns", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.open_txns", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_abort_txn(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.abort_txn", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.abort_txn"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.abort_txn"); - } - - ThriftHiveMetastore_abort_txn_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.abort_txn", bytes); - } - - ThriftHiveMetastore_abort_txn_result result; - try { - iface_->abort_txn(args.rqst); - } catch (NoSuchTxnException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.abort_txn"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("abort_txn", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.abort_txn"); - } - - oprot->writeMessageBegin("abort_txn", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.abort_txn", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_abort_txns(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.abort_txns", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.abort_txns"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.abort_txns"); - } - - ThriftHiveMetastore_abort_txns_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.abort_txns", bytes); - } - - ThriftHiveMetastore_abort_txns_result result; - try { - iface_->abort_txns(args.rqst); - } catch (NoSuchTxnException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.abort_txns"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("abort_txns", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.abort_txns"); - } - - oprot->writeMessageBegin("abort_txns", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.abort_txns", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_commit_txn(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.commit_txn", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.commit_txn"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.commit_txn"); - } - - ThriftHiveMetastore_commit_txn_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.commit_txn", bytes); - } - - ThriftHiveMetastore_commit_txn_result result; - try { - iface_->commit_txn(args.rqst); - } catch (NoSuchTxnException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (TxnAbortedException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.commit_txn"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("commit_txn", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.commit_txn"); - } - - oprot->writeMessageBegin("commit_txn", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.commit_txn", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_lock(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.lock", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.lock"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.lock"); - } - - ThriftHiveMetastore_lock_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.lock", bytes); - } - - ThriftHiveMetastore_lock_result result; - try { - iface_->lock(result.success, args.rqst); - result.__isset.success = true; - } catch (NoSuchTxnException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (TxnAbortedException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.lock"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("lock", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.lock"); - } - - oprot->writeMessageBegin("lock", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.lock", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_check_lock(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.check_lock", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.check_lock"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.check_lock"); - } - - ThriftHiveMetastore_check_lock_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.check_lock", bytes); - } - - ThriftHiveMetastore_check_lock_result result; - try { - iface_->check_lock(result.success, args.rqst); - result.__isset.success = true; - } catch (NoSuchTxnException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (TxnAbortedException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (NoSuchLockException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.check_lock"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("check_lock", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.check_lock"); - } - - oprot->writeMessageBegin("check_lock", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.check_lock", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_unlock(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.unlock", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.unlock"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.unlock"); - } - - ThriftHiveMetastore_unlock_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.unlock", bytes); - } - - ThriftHiveMetastore_unlock_result result; - try { - iface_->unlock(args.rqst); - } catch (NoSuchLockException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (TxnOpenException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.unlock"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("unlock", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.unlock"); - } - - oprot->writeMessageBegin("unlock", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.unlock", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_show_locks(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.show_locks", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.show_locks"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.show_locks"); - } - - ThriftHiveMetastore_show_locks_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.show_locks", bytes); - } - - ThriftHiveMetastore_show_locks_result result; - try { - iface_->show_locks(result.success, args.rqst); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.show_locks"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("show_locks", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.show_locks"); - } - - oprot->writeMessageBegin("show_locks", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.show_locks", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_heartbeat(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.heartbeat", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.heartbeat"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.heartbeat"); - } - - ThriftHiveMetastore_heartbeat_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.heartbeat", bytes); - } - - ThriftHiveMetastore_heartbeat_result result; - try { - iface_->heartbeat(args.ids); - } catch (NoSuchLockException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchTxnException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (TxnAbortedException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.heartbeat"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("heartbeat", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.heartbeat"); - } - - oprot->writeMessageBegin("heartbeat", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.heartbeat", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_heartbeat_txn_range(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.heartbeat_txn_range", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.heartbeat_txn_range"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.heartbeat_txn_range"); - } - - ThriftHiveMetastore_heartbeat_txn_range_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.heartbeat_txn_range", bytes); - } - - ThriftHiveMetastore_heartbeat_txn_range_result result; - try { - iface_->heartbeat_txn_range(result.success, args.txns); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.heartbeat_txn_range"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("heartbeat_txn_range", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.heartbeat_txn_range"); - } - - oprot->writeMessageBegin("heartbeat_txn_range", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.heartbeat_txn_range", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_compact(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.compact", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.compact"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.compact"); - } - - ThriftHiveMetastore_compact_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.compact", bytes); - } - - ThriftHiveMetastore_compact_result result; - try { - iface_->compact(args.rqst); - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.compact"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("compact", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.compact"); - } - - oprot->writeMessageBegin("compact", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.compact", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_compact2(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.compact2", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.compact2"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.compact2"); - } - - ThriftHiveMetastore_compact2_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.compact2", bytes); - } - - ThriftHiveMetastore_compact2_result result; - try { - iface_->compact2(result.success, args.rqst); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.compact2"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("compact2", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.compact2"); - } - - oprot->writeMessageBegin("compact2", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.compact2", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_show_compact(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.show_compact", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.show_compact"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.show_compact"); - } - - ThriftHiveMetastore_show_compact_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.show_compact", bytes); - } - - ThriftHiveMetastore_show_compact_result result; - try { - iface_->show_compact(result.success, args.rqst); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.show_compact"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("show_compact", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.show_compact"); - } - - oprot->writeMessageBegin("show_compact", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.show_compact", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_add_dynamic_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_dynamic_partitions", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_dynamic_partitions"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_dynamic_partitions"); - } - - ThriftHiveMetastore_add_dynamic_partitions_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_dynamic_partitions", bytes); - } - - ThriftHiveMetastore_add_dynamic_partitions_result result; - try { - iface_->add_dynamic_partitions(args.rqst); - } catch (NoSuchTxnException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (TxnAbortedException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_dynamic_partitions"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_dynamic_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_dynamic_partitions"); - } - - oprot->writeMessageBegin("add_dynamic_partitions", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_dynamic_partitions", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_next_notification(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_next_notification", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_next_notification"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_next_notification"); - } - - ThriftHiveMetastore_get_next_notification_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_next_notification", bytes); - } - - ThriftHiveMetastore_get_next_notification_result result; - try { - iface_->get_next_notification(result.success, args.rqst); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_next_notification"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_next_notification", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_next_notification"); - } - - oprot->writeMessageBegin("get_next_notification", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_next_notification", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_current_notificationEventId(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_current_notificationEventId", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_current_notificationEventId"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_current_notificationEventId"); - } - - ThriftHiveMetastore_get_current_notificationEventId_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_current_notificationEventId", bytes); - } - - ThriftHiveMetastore_get_current_notificationEventId_result result; - try { - iface_->get_current_notificationEventId(result.success); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_current_notificationEventId"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_current_notificationEventId", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_current_notificationEventId"); - } - - oprot->writeMessageBegin("get_current_notificationEventId", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_current_notificationEventId", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_notification_events_count(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_notification_events_count", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_notification_events_count"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_notification_events_count"); - } - - ThriftHiveMetastore_get_notification_events_count_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_notification_events_count", bytes); - } - - ThriftHiveMetastore_get_notification_events_count_result result; - try { - iface_->get_notification_events_count(result.success, args.rqst); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_notification_events_count"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_notification_events_count", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_notification_events_count"); - } - - oprot->writeMessageBegin("get_notification_events_count", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_notification_events_count", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_fire_listener_event(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.fire_listener_event", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.fire_listener_event"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.fire_listener_event"); - } - - ThriftHiveMetastore_fire_listener_event_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.fire_listener_event", bytes); - } - - ThriftHiveMetastore_fire_listener_event_result result; - try { - iface_->fire_listener_event(result.success, args.rqst); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.fire_listener_event"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("fire_listener_event", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.fire_listener_event"); - } - - oprot->writeMessageBegin("fire_listener_event", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.fire_listener_event", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_flushCache(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.flushCache", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.flushCache"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.flushCache"); - } - - ThriftHiveMetastore_flushCache_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.flushCache", bytes); - } - - ThriftHiveMetastore_flushCache_result result; - try { - iface_->flushCache(); - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.flushCache"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("flushCache", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.flushCache"); - } - - oprot->writeMessageBegin("flushCache", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.flushCache", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_cm_recycle(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.cm_recycle", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.cm_recycle"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.cm_recycle"); - } - - ThriftHiveMetastore_cm_recycle_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.cm_recycle", bytes); - } - - ThriftHiveMetastore_cm_recycle_result result; - try { - iface_->cm_recycle(result.success, args.request); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.cm_recycle"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("cm_recycle", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.cm_recycle"); - } - - oprot->writeMessageBegin("cm_recycle", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.cm_recycle", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_file_metadata_by_expr(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_file_metadata_by_expr", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_file_metadata_by_expr"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_file_metadata_by_expr"); - } - - ThriftHiveMetastore_get_file_metadata_by_expr_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_file_metadata_by_expr", bytes); - } - - ThriftHiveMetastore_get_file_metadata_by_expr_result result; - try { - iface_->get_file_metadata_by_expr(result.success, args.req); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_file_metadata_by_expr"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_file_metadata_by_expr", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_file_metadata_by_expr"); - } - - oprot->writeMessageBegin("get_file_metadata_by_expr", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_file_metadata_by_expr", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_file_metadata", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_file_metadata"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_file_metadata"); - } - - ThriftHiveMetastore_get_file_metadata_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_file_metadata", bytes); - } - - ThriftHiveMetastore_get_file_metadata_result result; - try { - iface_->get_file_metadata(result.success, args.req); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_file_metadata"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_file_metadata", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_file_metadata"); - } - - oprot->writeMessageBegin("get_file_metadata", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_file_metadata", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_put_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.put_file_metadata", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.put_file_metadata"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.put_file_metadata"); - } - - ThriftHiveMetastore_put_file_metadata_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.put_file_metadata", bytes); - } - - ThriftHiveMetastore_put_file_metadata_result result; - try { - iface_->put_file_metadata(result.success, args.req); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.put_file_metadata"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("put_file_metadata", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.put_file_metadata"); - } - - oprot->writeMessageBegin("put_file_metadata", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.put_file_metadata", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_clear_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.clear_file_metadata", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.clear_file_metadata"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.clear_file_metadata"); - } - - ThriftHiveMetastore_clear_file_metadata_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.clear_file_metadata", bytes); - } - - ThriftHiveMetastore_clear_file_metadata_result result; - try { - iface_->clear_file_metadata(result.success, args.req); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.clear_file_metadata"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("clear_file_metadata", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.clear_file_metadata"); - } - - oprot->writeMessageBegin("clear_file_metadata", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.clear_file_metadata", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_cache_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.cache_file_metadata", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.cache_file_metadata"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.cache_file_metadata"); - } - - ThriftHiveMetastore_cache_file_metadata_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.cache_file_metadata", bytes); - } - - ThriftHiveMetastore_cache_file_metadata_result result; - try { - iface_->cache_file_metadata(result.success, args.req); - result.__isset.success = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.cache_file_metadata"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("cache_file_metadata", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.cache_file_metadata"); - } - - oprot->writeMessageBegin("cache_file_metadata", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.cache_file_metadata", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_metastore_db_uuid(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_metastore_db_uuid", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_metastore_db_uuid"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_metastore_db_uuid"); - } - - ThriftHiveMetastore_get_metastore_db_uuid_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_metastore_db_uuid", bytes); - } - - ThriftHiveMetastore_get_metastore_db_uuid_result result; - try { - iface_->get_metastore_db_uuid(result.success); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_metastore_db_uuid"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_metastore_db_uuid", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_metastore_db_uuid"); - } - - oprot->writeMessageBegin("get_metastore_db_uuid", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_metastore_db_uuid", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_create_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_resource_plan", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_resource_plan"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_resource_plan"); - } - - ThriftHiveMetastore_create_resource_plan_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_resource_plan", bytes); - } - - ThriftHiveMetastore_create_resource_plan_result result; - try { - iface_->create_resource_plan(result.success, args.request); - result.__isset.success = true; - } catch (AlreadyExistsException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_resource_plan"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_resource_plan", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_resource_plan"); - } - - oprot->writeMessageBegin("create_resource_plan", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_resource_plan", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_resource_plan", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_resource_plan"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_resource_plan"); - } - - ThriftHiveMetastore_get_resource_plan_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_resource_plan", bytes); - } - - ThriftHiveMetastore_get_resource_plan_result result; - try { - iface_->get_resource_plan(result.success, args.request); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_resource_plan"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_resource_plan", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_resource_plan"); - } - - oprot->writeMessageBegin("get_resource_plan", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_resource_plan", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_active_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_active_resource_plan", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_active_resource_plan"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_active_resource_plan"); - } - - ThriftHiveMetastore_get_active_resource_plan_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_active_resource_plan", bytes); - } - - ThriftHiveMetastore_get_active_resource_plan_result result; - try { - iface_->get_active_resource_plan(result.success, args.request); - result.__isset.success = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_active_resource_plan"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_active_resource_plan", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_active_resource_plan"); - } - - oprot->writeMessageBegin("get_active_resource_plan", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_active_resource_plan", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_all_resource_plans(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_all_resource_plans", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_all_resource_plans"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_resource_plans"); - } - - ThriftHiveMetastore_get_all_resource_plans_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_all_resource_plans", bytes); - } - - ThriftHiveMetastore_get_all_resource_plans_result result; - try { - iface_->get_all_resource_plans(result.success, args.request); - result.__isset.success = true; - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_all_resource_plans"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_all_resource_plans", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_resource_plans"); - } - - oprot->writeMessageBegin("get_all_resource_plans", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_all_resource_plans", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_alter_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_resource_plan", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_resource_plan"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_resource_plan"); - } - - ThriftHiveMetastore_alter_resource_plan_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_resource_plan", bytes); - } - - ThriftHiveMetastore_alter_resource_plan_result result; - try { - iface_->alter_resource_plan(result.success, args.request); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (InvalidOperationException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_resource_plan"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_resource_plan", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_resource_plan"); - } - - oprot->writeMessageBegin("alter_resource_plan", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_resource_plan", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_validate_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.validate_resource_plan", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.validate_resource_plan"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.validate_resource_plan"); - } - - ThriftHiveMetastore_validate_resource_plan_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.validate_resource_plan", bytes); - } - - ThriftHiveMetastore_validate_resource_plan_result result; - try { - iface_->validate_resource_plan(result.success, args.request); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.validate_resource_plan"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("validate_resource_plan", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.validate_resource_plan"); - } - - oprot->writeMessageBegin("validate_resource_plan", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.validate_resource_plan", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_drop_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_resource_plan", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_resource_plan"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_resource_plan"); - } - - ThriftHiveMetastore_drop_resource_plan_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_resource_plan", bytes); - } - - ThriftHiveMetastore_drop_resource_plan_result result; - try { - iface_->drop_resource_plan(result.success, args.request); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (InvalidOperationException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_resource_plan"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_resource_plan", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_resource_plan"); - } - - oprot->writeMessageBegin("drop_resource_plan", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_resource_plan", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_create_wm_trigger(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_wm_trigger", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_wm_trigger"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_wm_trigger"); - } - - ThriftHiveMetastore_create_wm_trigger_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_wm_trigger", bytes); - } - - ThriftHiveMetastore_create_wm_trigger_result result; - try { - iface_->create_wm_trigger(result.success, args.request); - result.__isset.success = true; - } catch (AlreadyExistsException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (InvalidObjectException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (MetaException &o4) { - result.o4 = o4; - result.__isset.o4 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_wm_trigger"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_wm_trigger", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_wm_trigger"); - } - - oprot->writeMessageBegin("create_wm_trigger", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_wm_trigger", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_alter_wm_trigger(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_wm_trigger", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_wm_trigger"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_wm_trigger"); - } - - ThriftHiveMetastore_alter_wm_trigger_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_wm_trigger", bytes); - } - - ThriftHiveMetastore_alter_wm_trigger_result result; - try { - iface_->alter_wm_trigger(result.success, args.request); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_wm_trigger"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_wm_trigger", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_wm_trigger"); - } - - oprot->writeMessageBegin("alter_wm_trigger", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_wm_trigger", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_drop_wm_trigger(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_wm_trigger", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_wm_trigger"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_wm_trigger"); - } - - ThriftHiveMetastore_drop_wm_trigger_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_wm_trigger", bytes); - } - - ThriftHiveMetastore_drop_wm_trigger_result result; - try { - iface_->drop_wm_trigger(result.success, args.request); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (InvalidOperationException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (MetaException &o3) { - result.o3 = o3; - result.__isset.o3 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_wm_trigger"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_wm_trigger", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_wm_trigger"); - } - - oprot->writeMessageBegin("drop_wm_trigger", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_wm_trigger", bytes); - } -} - -void ThriftHiveMetastoreProcessor::process_get_triggers_for_resourceplan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) -{ - void* ctx = NULL; - if (this->eventHandler_.get() != NULL) { - ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_triggers_for_resourceplan", callContext); - } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_triggers_for_resourceplan"); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_triggers_for_resourceplan"); - } - - ThriftHiveMetastore_get_triggers_for_resourceplan_args args; - args.read(iprot); - iprot->readMessageEnd(); - uint32_t bytes = iprot->getTransport()->readEnd(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_triggers_for_resourceplan", bytes); - } - - ThriftHiveMetastore_get_triggers_for_resourceplan_result result; - try { - iface_->get_triggers_for_resourceplan(result.success, args.request); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = true; - } catch (const std::exception& e) { - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_triggers_for_resourceplan"); - } - - ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_triggers_for_resourceplan", ::apache::thrift::protocol::T_EXCEPTION, seqid); - x.write(oprot); - oprot->writeMessageEnd(); - oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - return; - } - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_triggers_for_resourceplan"); - } - - oprot->writeMessageBegin("get_triggers_for_resourceplan", ::apache::thrift::protocol::T_REPLY, seqid); - result.write(oprot); - oprot->writeMessageEnd(); - bytes = oprot->getTransport()->writeEnd(); - oprot->getTransport()->flush(); - - if (this->eventHandler_.get() != NULL) { - this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_triggers_for_resourceplan", bytes); - } -} - -::std::shared_ptr< ::apache::thrift::TProcessor > ThriftHiveMetastoreProcessorFactory::getProcessor(const ::apache::thrift::TConnectionInfo& connInfo) { - ::apache::thrift::ReleaseHandler< ThriftHiveMetastoreIfFactory > cleanup(handlerFactory_); - ::std::shared_ptr< ThriftHiveMetastoreIf > handler(handlerFactory_->getHandler(connInfo), cleanup); - ::std::shared_ptr< ::apache::thrift::TProcessor > processor(new ThriftHiveMetastoreProcessor(handler)); - return processor; -} - -void ThriftHiveMetastoreConcurrentClient::getMetaConf(std::string& _return, const std::string& key) -{ - int32_t seqid = send_getMetaConf(key); - recv_getMetaConf(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_getMetaConf(const std::string& key) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("getMetaConf", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_getMetaConf_pargs args; - args.key = &key; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_getMetaConf(std::string& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("getMetaConf") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_getMetaConf_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "getMetaConf failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::setMetaConf(const std::string& key, const std::string& value) -{ - int32_t seqid = send_setMetaConf(key, value); - recv_setMetaConf(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_setMetaConf(const std::string& key, const std::string& value) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("setMetaConf", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_setMetaConf_pargs args; - args.key = &key; - args.value = &value; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_setMetaConf(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("setMetaConf") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_setMetaConf_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::create_database(const Database& database) -{ - int32_t seqid = send_create_database(database); - recv_create_database(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_create_database(const Database& database) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("create_database", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_database_pargs args; - args.database = &database; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_create_database(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_database") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_create_database_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_database(Database& _return, const std::string& name) -{ - int32_t seqid = send_get_database(name); - recv_get_database(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_database(const std::string& name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_database", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_database_pargs args; - args.name = &name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_database(Database& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_database") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_database_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_database failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::drop_database(const std::string& name, const bool deleteData, const bool cascade) -{ - int32_t seqid = send_drop_database(name, deleteData, cascade); - recv_drop_database(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_drop_database(const std::string& name, const bool deleteData, const bool cascade) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("drop_database", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_database_pargs args; - args.name = &name; - args.deleteData = &deleteData; - args.cascade = &cascade; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_drop_database(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_database") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_drop_database_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_databases(std::vector & _return, const std::string& pattern) -{ - int32_t seqid = send_get_databases(pattern); - recv_get_databases(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_databases(const std::string& pattern) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_databases", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_databases_pargs args; - args.pattern = &pattern; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_databases(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_databases") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_databases_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_databases failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_all_databases(std::vector & _return) -{ - int32_t seqid = send_get_all_databases(); - recv_get_all_databases(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_all_databases() -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_all_databases", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_all_databases_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_all_databases(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_all_databases") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_all_databases_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_databases failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::alter_database(const std::string& dbname, const Database& db) -{ - int32_t seqid = send_alter_database(dbname, db); - recv_alter_database(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_alter_database(const std::string& dbname, const Database& db) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("alter_database", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_database_pargs args; - args.dbname = &dbname; - args.db = &db; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_alter_database(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_database") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_alter_database_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_type(Type& _return, const std::string& name) -{ - int32_t seqid = send_get_type(name); - recv_get_type(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_type(const std::string& name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_type", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_type_pargs args; - args.name = &name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_type(Type& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_type") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_type_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_type failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::create_type(const Type& type) -{ - int32_t seqid = send_create_type(type); - return recv_create_type(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_create_type(const Type& type) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("create_type", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_type_pargs args; - args.type = &type; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_create_type(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_type") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_create_type_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_type failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::drop_type(const std::string& type) -{ - int32_t seqid = send_drop_type(type); - return recv_drop_type(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_drop_type(const std::string& type) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("drop_type", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_type_pargs args; - args.type = &type; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_drop_type(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_type") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_drop_type_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_type failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_type_all(std::map & _return, const std::string& name) -{ - int32_t seqid = send_get_type_all(name); - recv_get_type_all(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_type_all(const std::string& name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_type_all", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_type_all_pargs args; - args.name = &name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_type_all(std::map & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_type_all") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_type_all_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_type_all failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_fields(std::vector & _return, const std::string& db_name, const std::string& table_name) -{ - int32_t seqid = send_get_fields(db_name, table_name); - recv_get_fields(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_fields(const std::string& db_name, const std::string& table_name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_fields", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_fields_pargs args; - args.db_name = &db_name; - args.table_name = &table_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_fields(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_fields") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_fields_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_fields failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_fields_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) -{ - int32_t seqid = send_get_fields_with_environment_context(db_name, table_name, environment_context); - recv_get_fields_with_environment_context(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_fields_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_fields_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_fields_with_environment_context_pargs args; - args.db_name = &db_name; - args.table_name = &table_name; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_fields_with_environment_context(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_fields_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_fields_with_environment_context_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_fields_with_environment_context failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_schema(std::vector & _return, const std::string& db_name, const std::string& table_name) -{ - int32_t seqid = send_get_schema(db_name, table_name); - recv_get_schema(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_schema(const std::string& db_name, const std::string& table_name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_schema", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_schema_pargs args; - args.db_name = &db_name; - args.table_name = &table_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_schema(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_schema") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_schema_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_schema failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_schema_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) -{ - int32_t seqid = send_get_schema_with_environment_context(db_name, table_name, environment_context); - recv_get_schema_with_environment_context(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_schema_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_schema_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_schema_with_environment_context_pargs args; - args.db_name = &db_name; - args.table_name = &table_name; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_schema_with_environment_context(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_schema_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_schema_with_environment_context_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_schema_with_environment_context failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::create_table(const Table& tbl) -{ - int32_t seqid = send_create_table(tbl); - recv_create_table(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_create_table(const Table& tbl) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("create_table", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_table_pargs args; - args.tbl = &tbl; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_create_table(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_table") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_create_table_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - if (result.__isset.o4) { - sentry.commit(); - throw result.o4; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) -{ - int32_t seqid = send_create_table_with_environment_context(tbl, environment_context); - recv_create_table_with_environment_context(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("create_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_table_with_environment_context_pargs args; - args.tbl = &tbl; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_create_table_with_environment_context(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_table_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_create_table_with_environment_context_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - if (result.__isset.o4) { - sentry.commit(); - throw result.o4; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints) -{ - int32_t seqid = send_create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints); - recv_create_table_with_constraints(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("create_table_with_constraints", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_table_with_constraints_pargs args; - args.tbl = &tbl; - args.primaryKeys = &primaryKeys; - args.foreignKeys = &foreignKeys; - args.uniqueConstraints = &uniqueConstraints; - args.notNullConstraints = ¬NullConstraints; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_create_table_with_constraints(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_table_with_constraints") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_create_table_with_constraints_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - if (result.__isset.o4) { - sentry.commit(); - throw result.o4; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::drop_constraint(const DropConstraintRequest& req) -{ - int32_t seqid = send_drop_constraint(req); - recv_drop_constraint(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_drop_constraint(const DropConstraintRequest& req) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("drop_constraint", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_constraint_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_drop_constraint(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_constraint") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_drop_constraint_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::add_primary_key(const AddPrimaryKeyRequest& req) -{ - int32_t seqid = send_add_primary_key(req); - recv_add_primary_key(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_add_primary_key(const AddPrimaryKeyRequest& req) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("add_primary_key", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_primary_key_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_add_primary_key(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_primary_key") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_add_primary_key_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::add_foreign_key(const AddForeignKeyRequest& req) -{ - int32_t seqid = send_add_foreign_key(req); - recv_add_foreign_key(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_add_foreign_key(const AddForeignKeyRequest& req) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("add_foreign_key", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_foreign_key_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_add_foreign_key(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_foreign_key") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_add_foreign_key_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::add_unique_constraint(const AddUniqueConstraintRequest& req) -{ - int32_t seqid = send_add_unique_constraint(req); - recv_add_unique_constraint(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_add_unique_constraint(const AddUniqueConstraintRequest& req) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("add_unique_constraint", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_unique_constraint_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_add_unique_constraint(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_unique_constraint") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_add_unique_constraint_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::add_not_null_constraint(const AddNotNullConstraintRequest& req) -{ - int32_t seqid = send_add_not_null_constraint(req); - recv_add_not_null_constraint(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_add_not_null_constraint(const AddNotNullConstraintRequest& req) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("add_not_null_constraint", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_not_null_constraint_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_add_not_null_constraint(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_not_null_constraint") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_add_not_null_constraint_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::drop_table(const std::string& dbname, const std::string& name, const bool deleteData) -{ - int32_t seqid = send_drop_table(dbname, name, deleteData); - recv_drop_table(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("drop_table", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_table_pargs args; - args.dbname = &dbname; - args.name = &name; - args.deleteData = &deleteData; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_drop_table(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_table") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_drop_table_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) -{ - int32_t seqid = send_drop_table_with_environment_context(dbname, name, deleteData, environment_context); - recv_drop_table_with_environment_context(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("drop_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_table_with_environment_context_pargs args; - args.dbname = &dbname; - args.name = &name; - args.deleteData = &deleteData; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_drop_table_with_environment_context(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_table_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_drop_table_with_environment_context_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames) -{ - int32_t seqid = send_truncate_table(dbName, tableName, partNames); - recv_truncate_table(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("truncate_table", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_truncate_table_pargs args; - args.dbName = &dbName; - args.tableName = &tableName; - args.partNames = &partNames; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_truncate_table(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("truncate_table") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_truncate_table_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_tables(std::vector & _return, const std::string& db_name, const std::string& pattern) -{ - int32_t seqid = send_get_tables(db_name, pattern); - recv_get_tables(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_tables(const std::string& db_name, const std::string& pattern) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_tables", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_tables_pargs args; - args.db_name = &db_name; - args.pattern = &pattern; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_tables(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_tables") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_tables_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_tables failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_tables_by_type(std::vector & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType) -{ - int32_t seqid = send_get_tables_by_type(db_name, pattern, tableType); - recv_get_tables_by_type(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_tables_by_type(const std::string& db_name, const std::string& pattern, const std::string& tableType) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_tables_by_type", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_tables_by_type_pargs args; - args.db_name = &db_name; - args.pattern = &pattern; - args.tableType = &tableType; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_tables_by_type(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_tables_by_type") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_tables_by_type_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_tables_by_type failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_table_meta(std::vector & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types) -{ - int32_t seqid = send_get_table_meta(db_patterns, tbl_patterns, tbl_types); - recv_get_table_meta(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_table_meta(const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_table_meta", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_table_meta_pargs args; - args.db_patterns = &db_patterns; - args.tbl_patterns = &tbl_patterns; - args.tbl_types = &tbl_types; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_table_meta(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_table_meta") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_table_meta_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_meta failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_all_tables(std::vector & _return, const std::string& db_name) -{ - int32_t seqid = send_get_all_tables(db_name); - recv_get_all_tables(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_all_tables(const std::string& db_name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_all_tables", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_all_tables_pargs args; - args.db_name = &db_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_all_tables(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_all_tables") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_all_tables_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_tables failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_table(Table& _return, const std::string& dbname, const std::string& tbl_name) -{ - int32_t seqid = send_get_table(dbname, tbl_name); - recv_get_table(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_table(const std::string& dbname, const std::string& tbl_name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_table", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_table_pargs args; - args.dbname = &dbname; - args.tbl_name = &tbl_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_table(Table& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_table") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_table_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_table_objects_by_name(std::vector
& _return, const std::string& dbname, const std::vector & tbl_names) -{ - int32_t seqid = send_get_table_objects_by_name(dbname, tbl_names); - recv_get_table_objects_by_name(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_table_objects_by_name(const std::string& dbname, const std::vector & tbl_names) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_table_objects_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_table_objects_by_name_pargs args; - args.dbname = &dbname; - args.tbl_names = &tbl_names; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_table_objects_by_name(std::vector
& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_table_objects_by_name") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_table_objects_by_name_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_objects_by_name failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_table_req(GetTableResult& _return, const GetTableRequest& req) -{ - int32_t seqid = send_get_table_req(req); - recv_get_table_req(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_table_req(const GetTableRequest& req) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_table_req", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_table_req_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_table_req(GetTableResult& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_table_req") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_table_req_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_req failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req) -{ - int32_t seqid = send_get_table_objects_by_name_req(req); - recv_get_table_objects_by_name_req(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_table_objects_by_name_req(const GetTablesRequest& req) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_table_objects_by_name_req", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_table_objects_by_name_req_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_table_objects_by_name_req(GetTablesResult& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_table_objects_by_name_req") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_table_objects_by_name_req_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_objects_by_name_req failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_table_names_by_filter(std::vector & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables) -{ - int32_t seqid = send_get_table_names_by_filter(dbname, filter, max_tables); - recv_get_table_names_by_filter(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_table_names_by_filter(const std::string& dbname, const std::string& filter, const int16_t max_tables) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_table_names_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_table_names_by_filter_pargs args; - args.dbname = &dbname; - args.filter = &filter; - args.max_tables = &max_tables; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_table_names_by_filter(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_table_names_by_filter") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_table_names_by_filter_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_names_by_filter failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) -{ - int32_t seqid = send_alter_table(dbname, tbl_name, new_tbl); - recv_alter_table(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("alter_table", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_table_pargs args; - args.dbname = &dbname; - args.tbl_name = &tbl_name; - args.new_tbl = &new_tbl; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_alter_table(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_table") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_alter_table_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) -{ - int32_t seqid = send_alter_table_with_environment_context(dbname, tbl_name, new_tbl, environment_context); - recv_alter_table_with_environment_context(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("alter_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_table_with_environment_context_pargs args; - args.dbname = &dbname; - args.tbl_name = &tbl_name; - args.new_tbl = &new_tbl; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_alter_table_with_environment_context(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_table_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_alter_table_with_environment_context_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade) -{ - int32_t seqid = send_alter_table_with_cascade(dbname, tbl_name, new_tbl, cascade); - recv_alter_table_with_cascade(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("alter_table_with_cascade", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_table_with_cascade_pargs args; - args.dbname = &dbname; - args.tbl_name = &tbl_name; - args.new_tbl = &new_tbl; - args.cascade = &cascade; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_alter_table_with_cascade(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_table_with_cascade") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_alter_table_with_cascade_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::add_partition(Partition& _return, const Partition& new_part) -{ - int32_t seqid = send_add_partition(new_part); - recv_add_partition(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_add_partition(const Partition& new_part) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("add_partition", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_partition_pargs args; - args.new_part = &new_part; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_add_partition(Partition& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_partition") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_add_partition_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partition failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::add_partition_with_environment_context(Partition& _return, const Partition& new_part, const EnvironmentContext& environment_context) -{ - int32_t seqid = send_add_partition_with_environment_context(new_part, environment_context); - recv_add_partition_with_environment_context(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_add_partition_with_environment_context(const Partition& new_part, const EnvironmentContext& environment_context) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("add_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_partition_with_environment_context_pargs args; - args.new_part = &new_part; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_add_partition_with_environment_context(Partition& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_partition_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_add_partition_with_environment_context_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partition_with_environment_context failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -int32_t ThriftHiveMetastoreConcurrentClient::add_partitions(const std::vector & new_parts) -{ - int32_t seqid = send_add_partitions(new_parts); - return recv_add_partitions(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_add_partitions(const std::vector & new_parts) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("add_partitions", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_partitions_pargs args; - args.new_parts = &new_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -int32_t ThriftHiveMetastoreConcurrentClient::recv_add_partitions(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_partitions") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - int32_t _return; - ThriftHiveMetastore_add_partitions_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partitions failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -int32_t ThriftHiveMetastoreConcurrentClient::add_partitions_pspec(const std::vector & new_parts) -{ - int32_t seqid = send_add_partitions_pspec(new_parts); - return recv_add_partitions_pspec(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_add_partitions_pspec(const std::vector & new_parts) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("add_partitions_pspec", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_partitions_pspec_pargs args; - args.new_parts = &new_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -int32_t ThriftHiveMetastoreConcurrentClient::recv_add_partitions_pspec(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_partitions_pspec") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - int32_t _return; - ThriftHiveMetastore_add_partitions_pspec_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partitions_pspec failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::append_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) -{ - int32_t seqid = send_append_partition(db_name, tbl_name, part_vals); - recv_append_partition(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_append_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("append_partition", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_append_partition_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_append_partition(Partition& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("append_partition") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_append_partition_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::add_partitions_req(AddPartitionsResult& _return, const AddPartitionsRequest& request) -{ - int32_t seqid = send_add_partitions_req(request); - recv_add_partitions_req(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_add_partitions_req(const AddPartitionsRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("add_partitions_req", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_partitions_req_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_add_partitions_req(AddPartitionsResult& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_partitions_req") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_add_partitions_req_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_partitions_req failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::append_partition_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context) -{ - int32_t seqid = send_append_partition_with_environment_context(db_name, tbl_name, part_vals, environment_context); - recv_append_partition_with_environment_context(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_append_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_append_partition_with_environment_context_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_append_partition_with_environment_context(Partition& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("append_partition_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_append_partition_with_environment_context_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition_with_environment_context failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::append_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) -{ - int32_t seqid = send_append_partition_by_name(db_name, tbl_name, part_name); - recv_append_partition_by_name(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_append_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("append_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_append_partition_by_name_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_append_partition_by_name(Partition& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("append_partition_by_name") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_append_partition_by_name_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition_by_name failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::append_partition_by_name_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context) -{ - int32_t seqid = send_append_partition_by_name_with_environment_context(db_name, tbl_name, part_name, environment_context); - recv_append_partition_by_name_with_environment_context(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_append_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_append_partition_by_name_with_environment_context(Partition& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("append_partition_by_name_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "append_partition_by_name_with_environment_context failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) -{ - int32_t seqid = send_drop_partition(db_name, tbl_name, part_vals, deleteData); - return recv_drop_partition(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("drop_partition", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_partition_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.deleteData = &deleteData; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_drop_partition(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_partition") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_drop_partition_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context) -{ - int32_t seqid = send_drop_partition_with_environment_context(db_name, tbl_name, part_vals, deleteData, environment_context); - return recv_drop_partition_with_environment_context(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("drop_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_partition_with_environment_context_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.deleteData = &deleteData; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_drop_partition_with_environment_context(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_partition_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_drop_partition_with_environment_context_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_with_environment_context failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData) -{ - int32_t seqid = send_drop_partition_by_name(db_name, tbl_name, part_name, deleteData); - return recv_drop_partition_by_name(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("drop_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_partition_by_name_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.deleteData = &deleteData; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_drop_partition_by_name(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_partition_by_name") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_drop_partition_by_name_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_by_name failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context) -{ - int32_t seqid = send_drop_partition_by_name_with_environment_context(db_name, tbl_name, part_name, deleteData, environment_context); - return recv_drop_partition_by_name_with_environment_context(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.deleteData = &deleteData; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_drop_partition_by_name_with_environment_context(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_partition_by_name_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partition_by_name_with_environment_context failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::drop_partitions_req(DropPartitionsResult& _return, const DropPartitionsRequest& req) -{ - int32_t seqid = send_drop_partitions_req(req); - recv_drop_partitions_req(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_drop_partitions_req(const DropPartitionsRequest& req) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("drop_partitions_req", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_partitions_req_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_drop_partitions_req(DropPartitionsResult& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_partitions_req") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_drop_partitions_req_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_partitions_req failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) -{ - int32_t seqid = send_get_partition(db_name, tbl_name, part_vals); - recv_get_partition(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partition_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_partition(Partition& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partition") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_partition_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::exchange_partition(Partition& _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) -{ - int32_t seqid = send_exchange_partition(partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); - recv_exchange_partition(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_exchange_partition(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_exchange_partition_pargs args; - args.partitionSpecs = &partitionSpecs; - args.source_db = &source_db; - args.source_table_name = &source_table_name; - args.dest_db = &dest_db; - args.dest_table_name = &dest_table_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_exchange_partition(Partition& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("exchange_partition") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_exchange_partition_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - if (result.__isset.o4) { - sentry.commit(); - throw result.o4; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "exchange_partition failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::exchange_partitions(std::vector & _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) -{ - int32_t seqid = send_exchange_partitions(partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); - recv_exchange_partitions(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_exchange_partitions(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("exchange_partitions", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_exchange_partitions_pargs args; - args.partitionSpecs = &partitionSpecs; - args.source_db = &source_db; - args.source_table_name = &source_table_name; - args.dest_db = &dest_db; - args.dest_table_name = &dest_table_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_exchange_partitions(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("exchange_partitions") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_exchange_partitions_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - if (result.__isset.o4) { - sentry.commit(); - throw result.o4; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "exchange_partitions failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_partition_with_auth(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names) -{ - int32_t seqid = send_get_partition_with_auth(db_name, tbl_name, part_vals, user_name, group_names); - recv_get_partition_with_auth(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_partition_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_partition_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partition_with_auth_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.user_name = &user_name; - args.group_names = &group_names; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_partition_with_auth(Partition& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partition_with_auth") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_partition_with_auth_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_with_auth failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) -{ - int32_t seqid = send_get_partition_by_name(db_name, tbl_name, part_name); - recv_get_partition_by_name(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partition_by_name_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_partition_by_name(Partition& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partition_by_name") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_partition_by_name_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_by_name failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_partitions(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) -{ - int32_t seqid = send_get_partitions(db_name, tbl_name, max_parts); - recv_get_partitions(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.max_parts = &max_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_partitions(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_partitions_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_partitions_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) -{ - int32_t seqid = send_get_partitions_with_auth(db_name, tbl_name, max_parts, user_name, group_names); - recv_get_partitions_with_auth(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions_with_auth(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_with_auth_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.max_parts = &max_parts; - args.user_name = &user_name; - args.group_names = &group_names; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_partitions_with_auth(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions_with_auth") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_partitions_with_auth_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_with_auth failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_partitions_pspec(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int32_t max_parts) -{ - int32_t seqid = send_get_partitions_pspec(db_name, tbl_name, max_parts); - recv_get_partitions_pspec(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions_pspec(const std::string& db_name, const std::string& tbl_name, const int32_t max_parts) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_partitions_pspec", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_pspec_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.max_parts = &max_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_partitions_pspec(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions_pspec") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_partitions_pspec_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_pspec failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_partition_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) -{ - int32_t seqid = send_get_partition_names(db_name, tbl_name, max_parts); - recv_get_partition_names(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_partition_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partition_names_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.max_parts = &max_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_partition_names(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partition_names") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_partition_names_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_names failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_partition_values(PartitionValuesResponse& _return, const PartitionValuesRequest& request) -{ - int32_t seqid = send_get_partition_values(request); - recv_get_partition_values(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_partition_values(const PartitionValuesRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_partition_values", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partition_values_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_partition_values(PartitionValuesResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partition_values") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_partition_values_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_values failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_partitions_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) -{ - int32_t seqid = send_get_partitions_ps(db_name, tbl_name, part_vals, max_parts); - recv_get_partitions_ps(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_ps_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.max_parts = &max_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_partitions_ps(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions_ps") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_partitions_ps_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_ps failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_partitions_ps_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) -{ - int32_t seqid = send_get_partitions_ps_with_auth(db_name, tbl_name, part_vals, max_parts, user_name, group_names); - recv_get_partitions_ps_with_auth(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions_ps_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_ps_with_auth_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.max_parts = &max_parts; - args.user_name = &user_name; - args.group_names = &group_names; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_partitions_ps_with_auth(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions_ps_with_auth") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_partitions_ps_with_auth_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_ps_with_auth failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_partition_names_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) -{ - int32_t seqid = send_get_partition_names_ps(db_name, tbl_name, part_vals, max_parts); - recv_get_partition_names_ps(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_partition_names_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partition_names_ps_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.max_parts = &max_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_partition_names_ps(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partition_names_ps") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_partition_names_ps_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_names_ps failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_partitions_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts) -{ - int32_t seqid = send_get_partitions_by_filter(db_name, tbl_name, filter, max_parts); - recv_get_partitions_by_filter(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_by_filter_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.filter = &filter; - args.max_parts = &max_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_partitions_by_filter(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions_by_filter") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_partitions_by_filter_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_filter failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_part_specs_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts) -{ - int32_t seqid = send_get_part_specs_by_filter(db_name, tbl_name, filter, max_parts); - recv_get_part_specs_by_filter(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_part_specs_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_part_specs_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_part_specs_by_filter_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.filter = &filter; - args.max_parts = &max_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_part_specs_by_filter(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_part_specs_by_filter") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_part_specs_by_filter_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_part_specs_by_filter failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_partitions_by_expr(PartitionsByExprResult& _return, const PartitionsByExprRequest& req) -{ - int32_t seqid = send_get_partitions_by_expr(req); - recv_get_partitions_by_expr(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions_by_expr(const PartitionsByExprRequest& req) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_partitions_by_expr", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_by_expr_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_partitions_by_expr(PartitionsByExprResult& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions_by_expr") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_partitions_by_expr_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_expr failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -int32_t ThriftHiveMetastoreConcurrentClient::get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter) -{ - int32_t seqid = send_get_num_partitions_by_filter(db_name, tbl_name, filter); - return recv_get_num_partitions_by_filter(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_num_partitions_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_num_partitions_by_filter_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.filter = &filter; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -int32_t ThriftHiveMetastoreConcurrentClient::recv_get_num_partitions_by_filter(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_num_partitions_by_filter") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - int32_t _return; - ThriftHiveMetastore_get_num_partitions_by_filter_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_num_partitions_by_filter failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_partitions_by_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & names) -{ - int32_t seqid = send_get_partitions_by_names(db_name, tbl_name, names); - recv_get_partitions_by_names(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions_by_names(const std::string& db_name, const std::string& tbl_name, const std::vector & names) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_by_names_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.names = &names; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_partitions_by_names(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions_by_names") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_partitions_by_names_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_names failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part) -{ - int32_t seqid = send_alter_partition(db_name, tbl_name, new_part); - recv_alter_partition(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_partition_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.new_part = &new_part; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_alter_partition(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_partition") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_alter_partition_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) -{ - int32_t seqid = send_alter_partitions(db_name, tbl_name, new_parts); - recv_alter_partitions(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_partitions_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.new_parts = &new_parts; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_alter_partitions(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_partitions") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_alter_partitions_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context) -{ - int32_t seqid = send_alter_partitions_with_environment_context(db_name, tbl_name, new_parts, environment_context); - recv_alter_partitions_with_environment_context(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("alter_partitions_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_partitions_with_environment_context_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.new_parts = &new_parts; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_alter_partitions_with_environment_context(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_partitions_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_alter_partitions_with_environment_context_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context) -{ - int32_t seqid = send_alter_partition_with_environment_context(db_name, tbl_name, new_part, environment_context); - recv_alter_partition_with_environment_context(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_partition_with_environment_context_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.new_part = &new_part; - args.environment_context = &environment_context; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_alter_partition_with_environment_context(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_partition_with_environment_context") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_alter_partition_with_environment_context_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) -{ - int32_t seqid = send_rename_partition(db_name, tbl_name, part_vals, new_part); - recv_rename_partition(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_rename_partition_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.new_part = &new_part; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_rename_partition(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("rename_partition") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_rename_partition_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) -{ - int32_t seqid = send_partition_name_has_valid_characters(part_vals, throw_exception); - return recv_partition_name_has_valid_characters(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("partition_name_has_valid_characters", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_partition_name_has_valid_characters_pargs args; - args.part_vals = &part_vals; - args.throw_exception = &throw_exception; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_partition_name_has_valid_characters(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("partition_name_has_valid_characters") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_partition_name_has_valid_characters_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_has_valid_characters failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_config_value(std::string& _return, const std::string& name, const std::string& defaultValue) -{ - int32_t seqid = send_get_config_value(name, defaultValue); - recv_get_config_value(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_config_value(const std::string& name, const std::string& defaultValue) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_config_value_pargs args; - args.name = &name; - args.defaultValue = &defaultValue; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_config_value(std::string& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_config_value") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_config_value_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_config_value failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::partition_name_to_vals(std::vector & _return, const std::string& part_name) -{ - int32_t seqid = send_partition_name_to_vals(part_name); - recv_partition_name_to_vals(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_partition_name_to_vals(const std::string& part_name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_partition_name_to_vals_pargs args; - args.part_name = &part_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_partition_name_to_vals(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("partition_name_to_vals") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_partition_name_to_vals_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_to_vals failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::partition_name_to_spec(std::map & _return, const std::string& part_name) -{ - int32_t seqid = send_partition_name_to_spec(part_name); - recv_partition_name_to_spec(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_partition_name_to_spec(const std::string& part_name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_partition_name_to_spec_pargs args; - args.part_name = &part_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_partition_name_to_spec(std::map & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("partition_name_to_spec") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_partition_name_to_spec_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_to_spec failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) -{ - int32_t seqid = send_markPartitionForEvent(db_name, tbl_name, part_vals, eventType); - recv_markPartitionForEvent(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_markPartitionForEvent_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.eventType = &eventType; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_markPartitionForEvent(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("markPartitionForEvent") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_markPartitionForEvent_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - if (result.__isset.o4) { - sentry.commit(); - throw result.o4; - } - if (result.__isset.o5) { - sentry.commit(); - throw result.o5; - } - if (result.__isset.o6) { - sentry.commit(); - throw result.o6; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) -{ - int32_t seqid = send_isPartitionMarkedForEvent(db_name, tbl_name, part_vals, eventType); - return recv_isPartitionMarkedForEvent(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_isPartitionMarkedForEvent_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.eventType = &eventType; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_isPartitionMarkedForEvent(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("isPartitionMarkedForEvent") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_isPartitionMarkedForEvent_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - if (result.__isset.o4) { - sentry.commit(); - throw result.o4; - } - if (result.__isset.o5) { - sentry.commit(); - throw result.o5; - } - if (result.__isset.o6) { - sentry.commit(); - throw result.o6; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "isPartitionMarkedForEvent failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::add_index(Index& _return, const Index& new_index, const Table& index_table) -{ - int32_t seqid = send_add_index(new_index, index_table); - recv_add_index(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_add_index(const Index& new_index, const Table& index_table) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("add_index", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_index_pargs args; - args.new_index = &new_index; - args.index_table = &index_table; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_add_index(Index& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_index") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_add_index_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_index failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) -{ - int32_t seqid = send_alter_index(dbname, base_tbl_name, idx_name, new_idx); - recv_alter_index(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_index_pargs args; - args.dbname = &dbname; - args.base_tbl_name = &base_tbl_name; - args.idx_name = &idx_name; - args.new_idx = &new_idx; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_alter_index(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_index") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_alter_index_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData) -{ - int32_t seqid = send_drop_index_by_name(db_name, tbl_name, index_name, deleteData); - return recv_drop_index_by_name(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_index_by_name_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.index_name = &index_name; - args.deleteData = &deleteData; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_drop_index_by_name(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_index_by_name") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_drop_index_by_name_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_index_by_name failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_index_by_name(Index& _return, const std::string& db_name, const std::string& tbl_name, const std::string& index_name) -{ - int32_t seqid = send_get_index_by_name(db_name, tbl_name, index_name); - recv_get_index_by_name(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_index_by_name_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.index_name = &index_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_index_by_name(Index& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_index_by_name") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_index_by_name_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_index_by_name failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_indexes(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) -{ - int32_t seqid = send_get_indexes(db_name, tbl_name, max_indexes); - recv_get_indexes(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_indexes(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_indexes_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.max_indexes = &max_indexes; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_indexes(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_indexes") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_indexes_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_indexes failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_index_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) -{ - int32_t seqid = send_get_index_names(db_name, tbl_name, max_indexes); - recv_get_index_names(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_index_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_index_names_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.max_indexes = &max_indexes; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_index_names(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_index_names") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_index_names_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_index_names failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request) -{ - int32_t seqid = send_get_primary_keys(request); - recv_get_primary_keys(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_primary_keys(const PrimaryKeysRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_primary_keys", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_primary_keys_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_primary_keys(PrimaryKeysResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_primary_keys") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_primary_keys_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_primary_keys failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request) -{ - int32_t seqid = send_get_foreign_keys(request); - recv_get_foreign_keys(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_foreign_keys(const ForeignKeysRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_foreign_keys", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_foreign_keys_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_foreign_keys(ForeignKeysResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_foreign_keys") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_foreign_keys_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_foreign_keys failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_unique_constraints(UniqueConstraintsResponse& _return, const UniqueConstraintsRequest& request) -{ - int32_t seqid = send_get_unique_constraints(request); - recv_get_unique_constraints(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_unique_constraints(const UniqueConstraintsRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_unique_constraints", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_unique_constraints_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_unique_constraints(UniqueConstraintsResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_unique_constraints") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_unique_constraints_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_unique_constraints failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_not_null_constraints(NotNullConstraintsResponse& _return, const NotNullConstraintsRequest& request) -{ - int32_t seqid = send_get_not_null_constraints(request); - recv_get_not_null_constraints(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_not_null_constraints(const NotNullConstraintsRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_not_null_constraints", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_not_null_constraints_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_not_null_constraints(NotNullConstraintsResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_not_null_constraints") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_not_null_constraints_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_not_null_constraints failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::update_table_column_statistics(const ColumnStatistics& stats_obj) -{ - int32_t seqid = send_update_table_column_statistics(stats_obj); - return recv_update_table_column_statistics(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_update_table_column_statistics(const ColumnStatistics& stats_obj) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("update_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_update_table_column_statistics_pargs args; - args.stats_obj = &stats_obj; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_update_table_column_statistics(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("update_table_column_statistics") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_update_table_column_statistics_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - if (result.__isset.o4) { - sentry.commit(); - throw result.o4; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "update_table_column_statistics failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::update_partition_column_statistics(const ColumnStatistics& stats_obj) -{ - int32_t seqid = send_update_partition_column_statistics(stats_obj); - return recv_update_partition_column_statistics(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_update_partition_column_statistics(const ColumnStatistics& stats_obj) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_update_partition_column_statistics_pargs args; - args.stats_obj = &stats_obj; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_update_partition_column_statistics(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("update_partition_column_statistics") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_update_partition_column_statistics_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - if (result.__isset.o4) { - sentry.commit(); - throw result.o4; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "update_partition_column_statistics failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_table_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& col_name) -{ - int32_t seqid = send_get_table_column_statistics(db_name, tbl_name, col_name); - recv_get_table_column_statistics(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_table_column_statistics_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.col_name = &col_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_table_column_statistics(ColumnStatistics& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_table_column_statistics") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_table_column_statistics_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - if (result.__isset.o4) { - sentry.commit(); - throw result.o4; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_column_statistics failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_partition_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) -{ - int32_t seqid = send_get_partition_column_statistics(db_name, tbl_name, part_name, col_name); - recv_get_partition_column_statistics(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partition_column_statistics_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.col_name = &col_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_partition_column_statistics(ColumnStatistics& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partition_column_statistics") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_partition_column_statistics_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - if (result.__isset.o4) { - sentry.commit(); - throw result.o4; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_column_statistics failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_table_statistics_req(TableStatsResult& _return, const TableStatsRequest& request) -{ - int32_t seqid = send_get_table_statistics_req(request); - recv_get_table_statistics_req(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_table_statistics_req(const TableStatsRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_table_statistics_req", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_table_statistics_req_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_table_statistics_req(TableStatsResult& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_table_statistics_req") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_table_statistics_req_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_statistics_req failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_partitions_statistics_req(PartitionsStatsResult& _return, const PartitionsStatsRequest& request) -{ - int32_t seqid = send_get_partitions_statistics_req(request); - recv_get_partitions_statistics_req(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_partitions_statistics_req(const PartitionsStatsRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_partitions_statistics_req", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_partitions_statistics_req_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_partitions_statistics_req(PartitionsStatsResult& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_partitions_statistics_req") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_partitions_statistics_req_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_statistics_req failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_aggr_stats_for(AggrStats& _return, const PartitionsStatsRequest& request) -{ - int32_t seqid = send_get_aggr_stats_for(request); - recv_get_aggr_stats_for(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_aggr_stats_for(const PartitionsStatsRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_aggr_stats_for", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_aggr_stats_for_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_aggr_stats_for(AggrStats& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_aggr_stats_for") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_aggr_stats_for_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_aggr_stats_for failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::set_aggr_stats_for(const SetPartitionsStatsRequest& request) -{ - int32_t seqid = send_set_aggr_stats_for(request); - return recv_set_aggr_stats_for(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_set_aggr_stats_for(const SetPartitionsStatsRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("set_aggr_stats_for", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_set_aggr_stats_for_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_set_aggr_stats_for(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("set_aggr_stats_for") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_set_aggr_stats_for_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - if (result.__isset.o4) { - sentry.commit(); - throw result.o4; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "set_aggr_stats_for failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) -{ - int32_t seqid = send_delete_partition_column_statistics(db_name, tbl_name, part_name, col_name); - return recv_delete_partition_column_statistics(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_delete_partition_column_statistics_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.col_name = &col_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_delete_partition_column_statistics(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("delete_partition_column_statistics") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_delete_partition_column_statistics_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - if (result.__isset.o4) { - sentry.commit(); - throw result.o4; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "delete_partition_column_statistics failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) -{ - int32_t seqid = send_delete_table_column_statistics(db_name, tbl_name, col_name); - return recv_delete_table_column_statistics(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_delete_table_column_statistics_pargs args; - args.db_name = &db_name; - args.tbl_name = &tbl_name; - args.col_name = &col_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_delete_table_column_statistics(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("delete_table_column_statistics") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_delete_table_column_statistics_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - if (result.__isset.o4) { - sentry.commit(); - throw result.o4; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "delete_table_column_statistics failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::create_function(const Function& func) -{ - int32_t seqid = send_create_function(func); - recv_create_function(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_create_function(const Function& func) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("create_function", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_function_pargs args; - args.func = &func; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_create_function(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_function") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_create_function_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - if (result.__isset.o4) { - sentry.commit(); - throw result.o4; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::drop_function(const std::string& dbName, const std::string& funcName) -{ - int32_t seqid = send_drop_function(dbName, funcName); - recv_drop_function(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_drop_function(const std::string& dbName, const std::string& funcName) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("drop_function", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_function_pargs args; - args.dbName = &dbName; - args.funcName = &funcName; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_drop_function(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_function") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_drop_function_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc) -{ - int32_t seqid = send_alter_function(dbName, funcName, newFunc); - recv_alter_function(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("alter_function", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_function_pargs args; - args.dbName = &dbName; - args.funcName = &funcName; - args.newFunc = &newFunc; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_alter_function(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_function") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_alter_function_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_functions(std::vector & _return, const std::string& dbName, const std::string& pattern) -{ - int32_t seqid = send_get_functions(dbName, pattern); - recv_get_functions(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_functions(const std::string& dbName, const std::string& pattern) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_functions", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_functions_pargs args; - args.dbName = &dbName; - args.pattern = &pattern; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_functions(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_functions") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_functions_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_functions failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_function(Function& _return, const std::string& dbName, const std::string& funcName) -{ - int32_t seqid = send_get_function(dbName, funcName); - recv_get_function(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_function(const std::string& dbName, const std::string& funcName) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_function", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_function_pargs args; - args.dbName = &dbName; - args.funcName = &funcName; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_function(Function& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_function") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_function_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_function failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_all_functions(GetAllFunctionsResponse& _return) -{ - int32_t seqid = send_get_all_functions(); - recv_get_all_functions(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_all_functions() -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_all_functions", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_all_functions_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_all_functions(GetAllFunctionsResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_all_functions") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_all_functions_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_functions failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::create_role(const Role& role) -{ - int32_t seqid = send_create_role(role); - return recv_create_role(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_create_role(const Role& role) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("create_role", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_role_pargs args; - args.role = &role; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_create_role(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_role") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_create_role_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_role failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::drop_role(const std::string& role_name) -{ - int32_t seqid = send_drop_role(role_name); - return recv_drop_role(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_drop_role(const std::string& role_name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("drop_role", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_role_pargs args; - args.role_name = &role_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_drop_role(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_role") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_drop_role_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_role failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_role_names(std::vector & _return) -{ - int32_t seqid = send_get_role_names(); - recv_get_role_names(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_role_names() -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_role_names", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_role_names_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_role_names(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_role_names") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_role_names_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_role_names failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option) -{ - int32_t seqid = send_grant_role(role_name, principal_name, principal_type, grantor, grantorType, grant_option); - return recv_grant_role(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("grant_role", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_grant_role_pargs args; - args.role_name = &role_name; - args.principal_name = &principal_name; - args.principal_type = &principal_type; - args.grantor = &grantor; - args.grantorType = &grantorType; - args.grant_option = &grant_option; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_grant_role(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("grant_role") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_grant_role_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_role failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type) -{ - int32_t seqid = send_revoke_role(role_name, principal_name, principal_type); - return recv_revoke_role(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("revoke_role", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_revoke_role_pargs args; - args.role_name = &role_name; - args.principal_name = &principal_name; - args.principal_type = &principal_type; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_revoke_role(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("revoke_role") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_revoke_role_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "revoke_role failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::list_roles(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type) -{ - int32_t seqid = send_list_roles(principal_name, principal_type); - recv_list_roles(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_list_roles(const std::string& principal_name, const PrincipalType::type principal_type) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("list_roles", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_list_roles_pargs args; - args.principal_name = &principal_name; - args.principal_type = &principal_type; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_list_roles(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("list_roles") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_list_roles_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "list_roles failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::grant_revoke_role(GrantRevokeRoleResponse& _return, const GrantRevokeRoleRequest& request) -{ - int32_t seqid = send_grant_revoke_role(request); - recv_grant_revoke_role(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_grant_revoke_role(const GrantRevokeRoleRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("grant_revoke_role", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_grant_revoke_role_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_grant_revoke_role(GrantRevokeRoleResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("grant_revoke_role") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_grant_revoke_role_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_revoke_role failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_principals_in_role(GetPrincipalsInRoleResponse& _return, const GetPrincipalsInRoleRequest& request) -{ - int32_t seqid = send_get_principals_in_role(request); - recv_get_principals_in_role(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_principals_in_role(const GetPrincipalsInRoleRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_principals_in_role", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_principals_in_role_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_principals_in_role(GetPrincipalsInRoleResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_principals_in_role") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_principals_in_role_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_principals_in_role failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return, const GetRoleGrantsForPrincipalRequest& request) -{ - int32_t seqid = send_get_role_grants_for_principal(request); - recv_get_role_grants_for_principal(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_role_grants_for_principal(const GetRoleGrantsForPrincipalRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_role_grants_for_principal", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_role_grants_for_principal_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_role_grants_for_principal") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_role_grants_for_principal_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_role_grants_for_principal failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_privilege_set(PrincipalPrivilegeSet& _return, const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names) -{ - int32_t seqid = send_get_privilege_set(hiveObject, user_name, group_names); - recv_get_privilege_set(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_privilege_set(const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_privilege_set", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_privilege_set_pargs args; - args.hiveObject = &hiveObject; - args.user_name = &user_name; - args.group_names = &group_names; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_privilege_set(PrincipalPrivilegeSet& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_privilege_set") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_privilege_set_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_privilege_set failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::list_privileges(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) -{ - int32_t seqid = send_list_privileges(principal_name, principal_type, hiveObject); - recv_list_privileges(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_list_privileges(const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("list_privileges", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_list_privileges_pargs args; - args.principal_name = &principal_name; - args.principal_type = &principal_type; - args.hiveObject = &hiveObject; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_list_privileges(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("list_privileges") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_list_privileges_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "list_privileges failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::grant_privileges(const PrivilegeBag& privileges) -{ - int32_t seqid = send_grant_privileges(privileges); - return recv_grant_privileges(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_grant_privileges(const PrivilegeBag& privileges) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("grant_privileges", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_grant_privileges_pargs args; - args.privileges = &privileges; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_grant_privileges(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("grant_privileges") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_grant_privileges_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_privileges failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::revoke_privileges(const PrivilegeBag& privileges) -{ - int32_t seqid = send_revoke_privileges(privileges); - return recv_revoke_privileges(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_revoke_privileges(const PrivilegeBag& privileges) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("revoke_privileges", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_revoke_privileges_pargs args; - args.privileges = &privileges; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_revoke_privileges(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("revoke_privileges") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_revoke_privileges_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "revoke_privileges failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const GrantRevokePrivilegeRequest& request) -{ - int32_t seqid = send_grant_revoke_privileges(request); - recv_grant_revoke_privileges(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_grant_revoke_privileges(const GrantRevokePrivilegeRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("grant_revoke_privileges", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_grant_revoke_privileges_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("grant_revoke_privileges") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_grant_revoke_privileges_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_revoke_privileges failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::set_ugi(std::vector & _return, const std::string& user_name, const std::vector & group_names) -{ - int32_t seqid = send_set_ugi(user_name, group_names); - recv_set_ugi(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_set_ugi(const std::string& user_name, const std::vector & group_names) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("set_ugi", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_set_ugi_pargs args; - args.user_name = &user_name; - args.group_names = &group_names; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_set_ugi(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("set_ugi") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_set_ugi_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "set_ugi failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name) -{ - int32_t seqid = send_get_delegation_token(token_owner, renewer_kerberos_principal_name); - recv_get_delegation_token(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_delegation_token(const std::string& token_owner, const std::string& renewer_kerberos_principal_name) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_delegation_token_pargs args; - args.token_owner = &token_owner; - args.renewer_kerberos_principal_name = &renewer_kerberos_principal_name; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_delegation_token(std::string& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_delegation_token") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_delegation_token_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_delegation_token failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -int64_t ThriftHiveMetastoreConcurrentClient::renew_delegation_token(const std::string& token_str_form) -{ - int32_t seqid = send_renew_delegation_token(token_str_form); - return recv_renew_delegation_token(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_renew_delegation_token(const std::string& token_str_form) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("renew_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_renew_delegation_token_pargs args; - args.token_str_form = &token_str_form; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -int64_t ThriftHiveMetastoreConcurrentClient::recv_renew_delegation_token(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("renew_delegation_token") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - int64_t _return; - ThriftHiveMetastore_renew_delegation_token_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "renew_delegation_token failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::cancel_delegation_token(const std::string& token_str_form) -{ - int32_t seqid = send_cancel_delegation_token(token_str_form); - recv_cancel_delegation_token(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_cancel_delegation_token(const std::string& token_str_form) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("cancel_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_cancel_delegation_token_pargs args; - args.token_str_form = &token_str_form; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_cancel_delegation_token(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("cancel_delegation_token") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_cancel_delegation_token_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::add_token(const std::string& token_identifier, const std::string& delegation_token) -{ - int32_t seqid = send_add_token(token_identifier, delegation_token); - return recv_add_token(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_add_token(const std::string& token_identifier, const std::string& delegation_token) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("add_token", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_token_pargs args; - args.token_identifier = &token_identifier; - args.delegation_token = &delegation_token; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_add_token(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_token") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_add_token_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_token failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::remove_token(const std::string& token_identifier) -{ - int32_t seqid = send_remove_token(token_identifier); - return recv_remove_token(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_remove_token(const std::string& token_identifier) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("remove_token", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_remove_token_pargs args; - args.token_identifier = &token_identifier; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_remove_token(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("remove_token") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_remove_token_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "remove_token failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_token(std::string& _return, const std::string& token_identifier) -{ - int32_t seqid = send_get_token(token_identifier); - recv_get_token(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_token(const std::string& token_identifier) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_token", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_token_pargs args; - args.token_identifier = &token_identifier; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_token(std::string& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_token") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_token_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_token failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_all_token_identifiers(std::vector & _return) -{ - int32_t seqid = send_get_all_token_identifiers(); - recv_get_all_token_identifiers(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_all_token_identifiers() -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_all_token_identifiers", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_all_token_identifiers_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_all_token_identifiers(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_all_token_identifiers") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_all_token_identifiers_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_token_identifiers failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -int32_t ThriftHiveMetastoreConcurrentClient::add_master_key(const std::string& key) -{ - int32_t seqid = send_add_master_key(key); - return recv_add_master_key(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_add_master_key(const std::string& key) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("add_master_key", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_master_key_pargs args; - args.key = &key; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -int32_t ThriftHiveMetastoreConcurrentClient::recv_add_master_key(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_master_key") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - int32_t _return; - ThriftHiveMetastore_add_master_key_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_master_key failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::update_master_key(const int32_t seq_number, const std::string& key) -{ - int32_t seqid = send_update_master_key(seq_number, key); - recv_update_master_key(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_update_master_key(const int32_t seq_number, const std::string& key) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("update_master_key", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_update_master_key_pargs args; - args.seq_number = &seq_number; - args.key = &key; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_update_master_key(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("update_master_key") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_update_master_key_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -bool ThriftHiveMetastoreConcurrentClient::remove_master_key(const int32_t key_seq) -{ - int32_t seqid = send_remove_master_key(key_seq); - return recv_remove_master_key(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_remove_master_key(const int32_t key_seq) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("remove_master_key", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_remove_master_key_pargs args; - args.key_seq = &key_seq; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -bool ThriftHiveMetastoreConcurrentClient::recv_remove_master_key(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("remove_master_key") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - bool _return; - ThriftHiveMetastore_remove_master_key_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - sentry.commit(); - return _return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "remove_master_key failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_master_keys(std::vector & _return) -{ - int32_t seqid = send_get_master_keys(); - recv_get_master_keys(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_master_keys() -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_master_keys", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_master_keys_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_master_keys(std::vector & _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_master_keys") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_master_keys_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_master_keys failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_open_txns(GetOpenTxnsResponse& _return) -{ - int32_t seqid = send_get_open_txns(); - recv_get_open_txns(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_open_txns() -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_open_txns", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_open_txns_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_open_txns(GetOpenTxnsResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_open_txns") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_open_txns_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_open_txns failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_open_txns_info(GetOpenTxnsInfoResponse& _return) -{ - int32_t seqid = send_get_open_txns_info(); - recv_get_open_txns_info(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_open_txns_info() -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_open_txns_info", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_open_txns_info_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_open_txns_info(GetOpenTxnsInfoResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_open_txns_info") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_open_txns_info_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_open_txns_info failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::open_txns(OpenTxnsResponse& _return, const OpenTxnRequest& rqst) -{ - int32_t seqid = send_open_txns(rqst); - recv_open_txns(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_open_txns(const OpenTxnRequest& rqst) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("open_txns", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_open_txns_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_open_txns(OpenTxnsResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("open_txns") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_open_txns_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "open_txns failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::abort_txn(const AbortTxnRequest& rqst) -{ - int32_t seqid = send_abort_txn(rqst); - recv_abort_txn(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_abort_txn(const AbortTxnRequest& rqst) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("abort_txn", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_abort_txn_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_abort_txn(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("abort_txn") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_abort_txn_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::abort_txns(const AbortTxnsRequest& rqst) -{ - int32_t seqid = send_abort_txns(rqst); - recv_abort_txns(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_abort_txns(const AbortTxnsRequest& rqst) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("abort_txns", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_abort_txns_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_abort_txns(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("abort_txns") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_abort_txns_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::commit_txn(const CommitTxnRequest& rqst) -{ - int32_t seqid = send_commit_txn(rqst); - recv_commit_txn(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_commit_txn(const CommitTxnRequest& rqst) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("commit_txn", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_commit_txn_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_commit_txn(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("commit_txn") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_commit_txn_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::lock(LockResponse& _return, const LockRequest& rqst) -{ - int32_t seqid = send_lock(rqst); - recv_lock(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_lock(const LockRequest& rqst) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("lock", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_lock_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_lock(LockResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("lock") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_lock_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "lock failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::check_lock(LockResponse& _return, const CheckLockRequest& rqst) -{ - int32_t seqid = send_check_lock(rqst); - recv_check_lock(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_check_lock(const CheckLockRequest& rqst) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("check_lock", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_check_lock_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_check_lock(LockResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("check_lock") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_check_lock_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "check_lock failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::unlock(const UnlockRequest& rqst) -{ - int32_t seqid = send_unlock(rqst); - recv_unlock(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_unlock(const UnlockRequest& rqst) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("unlock", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_unlock_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_unlock(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("unlock") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_unlock_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::show_locks(ShowLocksResponse& _return, const ShowLocksRequest& rqst) -{ - int32_t seqid = send_show_locks(rqst); - recv_show_locks(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_show_locks(const ShowLocksRequest& rqst) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("show_locks", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_show_locks_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_show_locks(ShowLocksResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("show_locks") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_show_locks_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "show_locks failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::heartbeat(const HeartbeatRequest& ids) -{ - int32_t seqid = send_heartbeat(ids); - recv_heartbeat(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_heartbeat(const HeartbeatRequest& ids) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("heartbeat", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_heartbeat_pargs args; - args.ids = &ids; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_heartbeat(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("heartbeat") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_heartbeat_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::heartbeat_txn_range(HeartbeatTxnRangeResponse& _return, const HeartbeatTxnRangeRequest& txns) -{ - int32_t seqid = send_heartbeat_txn_range(txns); - recv_heartbeat_txn_range(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_heartbeat_txn_range(const HeartbeatTxnRangeRequest& txns) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("heartbeat_txn_range", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_heartbeat_txn_range_pargs args; - args.txns = &txns; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_heartbeat_txn_range(HeartbeatTxnRangeResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("heartbeat_txn_range") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_heartbeat_txn_range_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "heartbeat_txn_range failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::compact(const CompactionRequest& rqst) -{ - int32_t seqid = send_compact(rqst); - recv_compact(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_compact(const CompactionRequest& rqst) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("compact", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_compact_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_compact(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("compact") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_compact_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::compact2(CompactionResponse& _return, const CompactionRequest& rqst) -{ - int32_t seqid = send_compact2(rqst); - recv_compact2(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_compact2(const CompactionRequest& rqst) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("compact2", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_compact2_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_compact2(CompactionResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("compact2") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_compact2_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "compact2 failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst) -{ - int32_t seqid = send_show_compact(rqst); - recv_show_compact(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_show_compact(const ShowCompactRequest& rqst) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("show_compact", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_show_compact_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_show_compact(ShowCompactResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("show_compact") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_show_compact_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "show_compact failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::add_dynamic_partitions(const AddDynamicPartitions& rqst) -{ - int32_t seqid = send_add_dynamic_partitions(rqst); - recv_add_dynamic_partitions(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_add_dynamic_partitions(const AddDynamicPartitions& rqst) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("add_dynamic_partitions", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_add_dynamic_partitions_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_add_dynamic_partitions(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("add_dynamic_partitions") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_add_dynamic_partitions_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst) -{ - int32_t seqid = send_get_next_notification(rqst); - recv_get_next_notification(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_next_notification(const NotificationEventRequest& rqst) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_next_notification", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_next_notification_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_next_notification(NotificationEventResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_next_notification") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_next_notification_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_next_notification failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_current_notificationEventId(CurrentNotificationEventId& _return) -{ - int32_t seqid = send_get_current_notificationEventId(); - recv_get_current_notificationEventId(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_current_notificationEventId() -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_current_notificationEventId", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_current_notificationEventId_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_current_notificationEventId(CurrentNotificationEventId& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_current_notificationEventId") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_current_notificationEventId_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_current_notificationEventId failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_notification_events_count(NotificationEventsCountResponse& _return, const NotificationEventsCountRequest& rqst) -{ - int32_t seqid = send_get_notification_events_count(rqst); - recv_get_notification_events_count(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_notification_events_count(const NotificationEventsCountRequest& rqst) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_notification_events_count", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_notification_events_count_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_notification_events_count(NotificationEventsCountResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_notification_events_count") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_notification_events_count_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_notification_events_count failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::fire_listener_event(FireEventResponse& _return, const FireEventRequest& rqst) -{ - int32_t seqid = send_fire_listener_event(rqst); - recv_fire_listener_event(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_fire_listener_event(const FireEventRequest& rqst) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("fire_listener_event", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_fire_listener_event_pargs args; - args.rqst = &rqst; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_fire_listener_event(FireEventResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("fire_listener_event") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_fire_listener_event_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "fire_listener_event failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::flushCache() -{ - int32_t seqid = send_flushCache(); - recv_flushCache(seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_flushCache() -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("flushCache", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_flushCache_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_flushCache(const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("flushCache") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_flushCache_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - sentry.commit(); - return; - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request) -{ - int32_t seqid = send_cm_recycle(request); - recv_cm_recycle(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_cm_recycle(const CmRecycleRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("cm_recycle", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_cm_recycle_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_cm_recycle(CmRecycleResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("cm_recycle") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_cm_recycle_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "cm_recycle failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req) -{ - int32_t seqid = send_get_file_metadata_by_expr(req); - recv_get_file_metadata_by_expr(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_file_metadata_by_expr(const GetFileMetadataByExprRequest& req) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_file_metadata_by_expr", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_file_metadata_by_expr_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_file_metadata_by_expr") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_file_metadata_by_expr_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_file_metadata_by_expr failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req) -{ - int32_t seqid = send_get_file_metadata(req); - recv_get_file_metadata(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_file_metadata(const GetFileMetadataRequest& req) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_file_metadata", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_file_metadata_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_file_metadata(GetFileMetadataResult& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_file_metadata") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_file_metadata_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_file_metadata failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req) -{ - int32_t seqid = send_put_file_metadata(req); - recv_put_file_metadata(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_put_file_metadata(const PutFileMetadataRequest& req) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("put_file_metadata", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_put_file_metadata_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_put_file_metadata(PutFileMetadataResult& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("put_file_metadata") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_put_file_metadata_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "put_file_metadata failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req) -{ - int32_t seqid = send_clear_file_metadata(req); - recv_clear_file_metadata(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_clear_file_metadata(const ClearFileMetadataRequest& req) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("clear_file_metadata", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_clear_file_metadata_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_clear_file_metadata(ClearFileMetadataResult& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("clear_file_metadata") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_clear_file_metadata_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "clear_file_metadata failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req) -{ - int32_t seqid = send_cache_file_metadata(req); - recv_cache_file_metadata(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_cache_file_metadata(const CacheFileMetadataRequest& req) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("cache_file_metadata", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_cache_file_metadata_pargs args; - args.req = &req; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_cache_file_metadata(CacheFileMetadataResult& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("cache_file_metadata") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_cache_file_metadata_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "cache_file_metadata failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_metastore_db_uuid(std::string& _return) -{ - int32_t seqid = send_get_metastore_db_uuid(); - recv_get_metastore_db_uuid(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_metastore_db_uuid() -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_metastore_db_uuid", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_metastore_db_uuid_pargs args; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_metastore_db_uuid(std::string& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_metastore_db_uuid") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_metastore_db_uuid_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_metastore_db_uuid failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::create_resource_plan(WMCreateResourcePlanResponse& _return, const WMCreateResourcePlanRequest& request) -{ - int32_t seqid = send_create_resource_plan(request); - recv_create_resource_plan(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_create_resource_plan(const WMCreateResourcePlanRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("create_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_resource_plan_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_create_resource_plan(WMCreateResourcePlanResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_resource_plan") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_create_resource_plan_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_resource_plan failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_resource_plan(WMGetResourcePlanResponse& _return, const WMGetResourcePlanRequest& request) -{ - int32_t seqid = send_get_resource_plan(request); - recv_get_resource_plan(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_resource_plan(const WMGetResourcePlanRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_resource_plan_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_resource_plan(WMGetResourcePlanResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_resource_plan") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_resource_plan_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_resource_plan failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_active_resource_plan(WMGetActiveResourcePlanResponse& _return, const WMGetActiveResourcePlanRequest& request) -{ - int32_t seqid = send_get_active_resource_plan(request); - recv_get_active_resource_plan(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_active_resource_plan(const WMGetActiveResourcePlanRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_active_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_active_resource_plan_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_active_resource_plan(WMGetActiveResourcePlanResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_active_resource_plan") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_active_resource_plan_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_active_resource_plan failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_all_resource_plans(WMGetAllResourcePlanResponse& _return, const WMGetAllResourcePlanRequest& request) -{ - int32_t seqid = send_get_all_resource_plans(request); - recv_get_all_resource_plans(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_all_resource_plans(const WMGetAllResourcePlanRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_all_resource_plans", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_all_resource_plans_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_all_resource_plans(WMGetAllResourcePlanResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_all_resource_plans") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_all_resource_plans_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_resource_plans failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::alter_resource_plan(WMAlterResourcePlanResponse& _return, const WMAlterResourcePlanRequest& request) -{ - int32_t seqid = send_alter_resource_plan(request); - recv_alter_resource_plan(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_alter_resource_plan(const WMAlterResourcePlanRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("alter_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_resource_plan_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_alter_resource_plan(WMAlterResourcePlanResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_resource_plan") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_alter_resource_plan_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "alter_resource_plan failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::validate_resource_plan(WMValidateResourcePlanResponse& _return, const WMValidateResourcePlanRequest& request) -{ - int32_t seqid = send_validate_resource_plan(request); - recv_validate_resource_plan(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_validate_resource_plan(const WMValidateResourcePlanRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("validate_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_validate_resource_plan_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_validate_resource_plan(WMValidateResourcePlanResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("validate_resource_plan") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_validate_resource_plan_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "validate_resource_plan failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::drop_resource_plan(WMDropResourcePlanResponse& _return, const WMDropResourcePlanRequest& request) -{ - int32_t seqid = send_drop_resource_plan(request); - recv_drop_resource_plan(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_drop_resource_plan(const WMDropResourcePlanRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("drop_resource_plan", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_resource_plan_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_drop_resource_plan(WMDropResourcePlanResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_resource_plan") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_drop_resource_plan_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_resource_plan failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::create_wm_trigger(WMCreateTriggerResponse& _return, const WMCreateTriggerRequest& request) -{ - int32_t seqid = send_create_wm_trigger(request); - recv_create_wm_trigger(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_create_wm_trigger(const WMCreateTriggerRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("create_wm_trigger", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_create_wm_trigger_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_create_wm_trigger(WMCreateTriggerResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("create_wm_trigger") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_create_wm_trigger_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - if (result.__isset.o4) { - sentry.commit(); - throw result.o4; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_wm_trigger failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::alter_wm_trigger(WMAlterTriggerResponse& _return, const WMAlterTriggerRequest& request) -{ - int32_t seqid = send_alter_wm_trigger(request); - recv_alter_wm_trigger(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_alter_wm_trigger(const WMAlterTriggerRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("alter_wm_trigger", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_alter_wm_trigger_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_alter_wm_trigger(WMAlterTriggerResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("alter_wm_trigger") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_alter_wm_trigger_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "alter_wm_trigger failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::drop_wm_trigger(WMDropTriggerResponse& _return, const WMDropTriggerRequest& request) -{ - int32_t seqid = send_drop_wm_trigger(request); - recv_drop_wm_trigger(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_drop_wm_trigger(const WMDropTriggerRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("drop_wm_trigger", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_drop_wm_trigger_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_drop_wm_trigger(WMDropTriggerResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("drop_wm_trigger") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_drop_wm_trigger_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - if (result.__isset.o3) { - sentry.commit(); - throw result.o3; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_wm_trigger failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -void ThriftHiveMetastoreConcurrentClient::get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return, const WMGetTriggersForResourePlanRequest& request) -{ - int32_t seqid = send_get_triggers_for_resourceplan(request); - recv_get_triggers_for_resourceplan(_return, seqid); -} - -int32_t ThriftHiveMetastoreConcurrentClient::send_get_triggers_for_resourceplan(const WMGetTriggersForResourePlanRequest& request) -{ - int32_t cseqid = this->sync_.generateSeqId(); - ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); - oprot_->writeMessageBegin("get_triggers_for_resourceplan", ::apache::thrift::protocol::T_CALL, cseqid); - - ThriftHiveMetastore_get_triggers_for_resourceplan_pargs args; - args.request = &request; - args.write(oprot_); - - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); - - sentry.commit(); - return cseqid; -} - -void ThriftHiveMetastoreConcurrentClient::recv_get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return, const int32_t seqid) -{ - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - // the read mutex gets dropped and reacquired as part of waitForWork() - // The destructor of this sentry wakes up other clients - ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); - - while(true) { - if(!this->sync_.getPending(fname, mtype, rseqid)) { - iprot_->readMessageBegin(fname, mtype, rseqid); - } - if(seqid == rseqid) { - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - sentry.commit(); - throw x; - } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - } - if (fname.compare("get_triggers_for_resourceplan") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - // in a bad state, don't commit - using ::apache::thrift::protocol::TProtocolException; - throw TProtocolException(TProtocolException::INVALID_DATA); - } - ThriftHiveMetastore_get_triggers_for_resourceplan_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - - if (result.__isset.success) { - // _return pointer has now been filled - sentry.commit(); - return; - } - if (result.__isset.o1) { - sentry.commit(); - throw result.o1; - } - if (result.__isset.o2) { - sentry.commit(); - throw result.o2; - } - // in a bad state, don't commit - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_triggers_for_resourceplan failed: unknown result"); - } - // seqid != rseqid - this->sync_.updatePending(fname, mtype, rseqid); - - // this will temporarily unlock the readMutex, and let other clients get work done - this->sync_.waitForWork(seqid); - } // end while(true) -} - -}}} // namespace - diff --git a/contrib/hive-metastore/ThriftHiveMetastore.h b/contrib/hive-metastore/ThriftHiveMetastore.h deleted file mode 100644 index e86933e5011..00000000000 --- a/contrib/hive-metastore/ThriftHiveMetastore.h +++ /dev/null @@ -1,25246 +0,0 @@ -/** - * Autogenerated by Thrift Compiler () - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -#ifndef ThriftHiveMetastore_H -#define ThriftHiveMetastore_H - -#include -#include -#include "hive_metastore_types.h" - -namespace Apache { namespace Hadoop { namespace Hive { - -#ifdef _MSC_VER - #pragma warning( push ) - #pragma warning (disable : 4250 ) //inheriting methods via dominance -#endif - -class ThriftHiveMetastoreIf { - public: - virtual ~ThriftHiveMetastoreIf() {} - virtual void getMetaConf(std::string& _return, const std::string& key) = 0; - virtual void setMetaConf(const std::string& key, const std::string& value) = 0; - virtual void create_database(const Database& database) = 0; - virtual void get_database(Database& _return, const std::string& name) = 0; - virtual void drop_database(const std::string& name, const bool deleteData, const bool cascade) = 0; - virtual void get_databases(std::vector & _return, const std::string& pattern) = 0; - virtual void get_all_databases(std::vector & _return) = 0; - virtual void alter_database(const std::string& dbname, const Database& db) = 0; - virtual void get_type(Type& _return, const std::string& name) = 0; - virtual bool create_type(const Type& type) = 0; - virtual bool drop_type(const std::string& type) = 0; - virtual void get_type_all(std::map & _return, const std::string& name) = 0; - virtual void get_fields(std::vector & _return, const std::string& db_name, const std::string& table_name) = 0; - virtual void get_fields_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) = 0; - virtual void get_schema(std::vector & _return, const std::string& db_name, const std::string& table_name) = 0; - virtual void get_schema_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) = 0; - virtual void create_table(const Table& tbl) = 0; - virtual void create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) = 0; - virtual void create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints) = 0; - virtual void drop_constraint(const DropConstraintRequest& req) = 0; - virtual void add_primary_key(const AddPrimaryKeyRequest& req) = 0; - virtual void add_foreign_key(const AddForeignKeyRequest& req) = 0; - virtual void add_unique_constraint(const AddUniqueConstraintRequest& req) = 0; - virtual void add_not_null_constraint(const AddNotNullConstraintRequest& req) = 0; - virtual void drop_table(const std::string& dbname, const std::string& name, const bool deleteData) = 0; - virtual void drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) = 0; - virtual void truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames) = 0; - virtual void get_tables(std::vector & _return, const std::string& db_name, const std::string& pattern) = 0; - virtual void get_tables_by_type(std::vector & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType) = 0; - virtual void get_table_meta(std::vector & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types) = 0; - virtual void get_all_tables(std::vector & _return, const std::string& db_name) = 0; - virtual void get_table(Table& _return, const std::string& dbname, const std::string& tbl_name) = 0; - virtual void get_table_objects_by_name(std::vector
& _return, const std::string& dbname, const std::vector & tbl_names) = 0; - virtual void get_table_req(GetTableResult& _return, const GetTableRequest& req) = 0; - virtual void get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req) = 0; - virtual void get_table_names_by_filter(std::vector & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables) = 0; - virtual void alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) = 0; - virtual void alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) = 0; - virtual void alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade) = 0; - virtual void add_partition(Partition& _return, const Partition& new_part) = 0; - virtual void add_partition_with_environment_context(Partition& _return, const Partition& new_part, const EnvironmentContext& environment_context) = 0; - virtual int32_t add_partitions(const std::vector & new_parts) = 0; - virtual int32_t add_partitions_pspec(const std::vector & new_parts) = 0; - virtual void append_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) = 0; - virtual void add_partitions_req(AddPartitionsResult& _return, const AddPartitionsRequest& request) = 0; - virtual void append_partition_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context) = 0; - virtual void append_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) = 0; - virtual void append_partition_by_name_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context) = 0; - virtual bool drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) = 0; - virtual bool drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context) = 0; - virtual bool drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData) = 0; - virtual bool drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context) = 0; - virtual void drop_partitions_req(DropPartitionsResult& _return, const DropPartitionsRequest& req) = 0; - virtual void get_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) = 0; - virtual void exchange_partition(Partition& _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) = 0; - virtual void exchange_partitions(std::vector & _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) = 0; - virtual void get_partition_with_auth(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names) = 0; - virtual void get_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) = 0; - virtual void get_partitions(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) = 0; - virtual void get_partitions_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) = 0; - virtual void get_partitions_pspec(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int32_t max_parts) = 0; - virtual void get_partition_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) = 0; - virtual void get_partition_values(PartitionValuesResponse& _return, const PartitionValuesRequest& request) = 0; - virtual void get_partitions_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) = 0; - virtual void get_partitions_ps_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) = 0; - virtual void get_partition_names_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) = 0; - virtual void get_partitions_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts) = 0; - virtual void get_part_specs_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts) = 0; - virtual void get_partitions_by_expr(PartitionsByExprResult& _return, const PartitionsByExprRequest& req) = 0; - virtual int32_t get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter) = 0; - virtual void get_partitions_by_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & names) = 0; - virtual void alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part) = 0; - virtual void alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) = 0; - virtual void alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context) = 0; - virtual void alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context) = 0; - virtual void rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) = 0; - virtual bool partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) = 0; - virtual void get_config_value(std::string& _return, const std::string& name, const std::string& defaultValue) = 0; - virtual void partition_name_to_vals(std::vector & _return, const std::string& part_name) = 0; - virtual void partition_name_to_spec(std::map & _return, const std::string& part_name) = 0; - virtual void markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) = 0; - virtual bool isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) = 0; - virtual void add_index(Index& _return, const Index& new_index, const Table& index_table) = 0; - virtual void alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) = 0; - virtual bool drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData) = 0; - virtual void get_index_by_name(Index& _return, const std::string& db_name, const std::string& tbl_name, const std::string& index_name) = 0; - virtual void get_indexes(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) = 0; - virtual void get_index_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) = 0; - virtual void get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request) = 0; - virtual void get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request) = 0; - virtual void get_unique_constraints(UniqueConstraintsResponse& _return, const UniqueConstraintsRequest& request) = 0; - virtual void get_not_null_constraints(NotNullConstraintsResponse& _return, const NotNullConstraintsRequest& request) = 0; - virtual bool update_table_column_statistics(const ColumnStatistics& stats_obj) = 0; - virtual bool update_partition_column_statistics(const ColumnStatistics& stats_obj) = 0; - virtual void get_table_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& col_name) = 0; - virtual void get_partition_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) = 0; - virtual void get_table_statistics_req(TableStatsResult& _return, const TableStatsRequest& request) = 0; - virtual void get_partitions_statistics_req(PartitionsStatsResult& _return, const PartitionsStatsRequest& request) = 0; - virtual void get_aggr_stats_for(AggrStats& _return, const PartitionsStatsRequest& request) = 0; - virtual bool set_aggr_stats_for(const SetPartitionsStatsRequest& request) = 0; - virtual bool delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) = 0; - virtual bool delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) = 0; - virtual void create_function(const Function& func) = 0; - virtual void drop_function(const std::string& dbName, const std::string& funcName) = 0; - virtual void alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc) = 0; - virtual void get_functions(std::vector & _return, const std::string& dbName, const std::string& pattern) = 0; - virtual void get_function(Function& _return, const std::string& dbName, const std::string& funcName) = 0; - virtual void get_all_functions(GetAllFunctionsResponse& _return) = 0; - virtual bool create_role(const Role& role) = 0; - virtual bool drop_role(const std::string& role_name) = 0; - virtual void get_role_names(std::vector & _return) = 0; - virtual bool grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option) = 0; - virtual bool revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type) = 0; - virtual void list_roles(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type) = 0; - virtual void grant_revoke_role(GrantRevokeRoleResponse& _return, const GrantRevokeRoleRequest& request) = 0; - virtual void get_principals_in_role(GetPrincipalsInRoleResponse& _return, const GetPrincipalsInRoleRequest& request) = 0; - virtual void get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return, const GetRoleGrantsForPrincipalRequest& request) = 0; - virtual void get_privilege_set(PrincipalPrivilegeSet& _return, const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names) = 0; - virtual void list_privileges(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) = 0; - virtual bool grant_privileges(const PrivilegeBag& privileges) = 0; - virtual bool revoke_privileges(const PrivilegeBag& privileges) = 0; - virtual void grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const GrantRevokePrivilegeRequest& request) = 0; - virtual void set_ugi(std::vector & _return, const std::string& user_name, const std::vector & group_names) = 0; - virtual void get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name) = 0; - virtual int64_t renew_delegation_token(const std::string& token_str_form) = 0; - virtual void cancel_delegation_token(const std::string& token_str_form) = 0; - virtual bool add_token(const std::string& token_identifier, const std::string& delegation_token) = 0; - virtual bool remove_token(const std::string& token_identifier) = 0; - virtual void get_token(std::string& _return, const std::string& token_identifier) = 0; - virtual void get_all_token_identifiers(std::vector & _return) = 0; - virtual int32_t add_master_key(const std::string& key) = 0; - virtual void update_master_key(const int32_t seq_number, const std::string& key) = 0; - virtual bool remove_master_key(const int32_t key_seq) = 0; - virtual void get_master_keys(std::vector & _return) = 0; - virtual void get_open_txns(GetOpenTxnsResponse& _return) = 0; - virtual void get_open_txns_info(GetOpenTxnsInfoResponse& _return) = 0; - virtual void open_txns(OpenTxnsResponse& _return, const OpenTxnRequest& rqst) = 0; - virtual void abort_txn(const AbortTxnRequest& rqst) = 0; - virtual void abort_txns(const AbortTxnsRequest& rqst) = 0; - virtual void commit_txn(const CommitTxnRequest& rqst) = 0; - virtual void lock(LockResponse& _return, const LockRequest& rqst) = 0; - virtual void check_lock(LockResponse& _return, const CheckLockRequest& rqst) = 0; - virtual void unlock(const UnlockRequest& rqst) = 0; - virtual void show_locks(ShowLocksResponse& _return, const ShowLocksRequest& rqst) = 0; - virtual void heartbeat(const HeartbeatRequest& ids) = 0; - virtual void heartbeat_txn_range(HeartbeatTxnRangeResponse& _return, const HeartbeatTxnRangeRequest& txns) = 0; - virtual void compact(const CompactionRequest& rqst) = 0; - virtual void compact2(CompactionResponse& _return, const CompactionRequest& rqst) = 0; - virtual void show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst) = 0; - virtual void add_dynamic_partitions(const AddDynamicPartitions& rqst) = 0; - virtual void get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst) = 0; - virtual void get_current_notificationEventId(CurrentNotificationEventId& _return) = 0; - virtual void get_notification_events_count(NotificationEventsCountResponse& _return, const NotificationEventsCountRequest& rqst) = 0; - virtual void fire_listener_event(FireEventResponse& _return, const FireEventRequest& rqst) = 0; - virtual void flushCache() = 0; - virtual void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request) = 0; - virtual void get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req) = 0; - virtual void get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req) = 0; - virtual void put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req) = 0; - virtual void clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req) = 0; - virtual void cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req) = 0; - virtual void get_metastore_db_uuid(std::string& _return) = 0; - virtual void create_resource_plan(WMCreateResourcePlanResponse& _return, const WMCreateResourcePlanRequest& request) = 0; - virtual void get_resource_plan(WMGetResourcePlanResponse& _return, const WMGetResourcePlanRequest& request) = 0; - virtual void get_active_resource_plan(WMGetActiveResourcePlanResponse& _return, const WMGetActiveResourcePlanRequest& request) = 0; - virtual void get_all_resource_plans(WMGetAllResourcePlanResponse& _return, const WMGetAllResourcePlanRequest& request) = 0; - virtual void alter_resource_plan(WMAlterResourcePlanResponse& _return, const WMAlterResourcePlanRequest& request) = 0; - virtual void validate_resource_plan(WMValidateResourcePlanResponse& _return, const WMValidateResourcePlanRequest& request) = 0; - virtual void drop_resource_plan(WMDropResourcePlanResponse& _return, const WMDropResourcePlanRequest& request) = 0; - virtual void create_wm_trigger(WMCreateTriggerResponse& _return, const WMCreateTriggerRequest& request) = 0; - virtual void alter_wm_trigger(WMAlterTriggerResponse& _return, const WMAlterTriggerRequest& request) = 0; - virtual void drop_wm_trigger(WMDropTriggerResponse& _return, const WMDropTriggerRequest& request) = 0; - virtual void get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return, const WMGetTriggersForResourePlanRequest& request) = 0; -}; - -class ThriftHiveMetastoreIfFactory { - public: - typedef ThriftHiveMetastoreIf Handler; - - virtual ~ThriftHiveMetastoreIfFactory() {} - - virtual ThriftHiveMetastoreIf* getHandler(const ::apache::thrift::TConnectionInfo& connInfo) = 0; - virtual void releaseHandler(ThriftHiveMetastoreIf* /* handler */) = 0; -}; - -class ThriftHiveMetastoreIfSingletonFactory : virtual public ThriftHiveMetastoreIfFactory { - public: - ThriftHiveMetastoreIfSingletonFactory(const ::std::shared_ptr& iface) : iface_(iface) {} - virtual ~ThriftHiveMetastoreIfSingletonFactory() {} - - virtual ThriftHiveMetastoreIf* getHandler(const ::apache::thrift::TConnectionInfo&) { - return iface_.get(); - } - virtual void releaseHandler(ThriftHiveMetastoreIf* /* handler */) {} - - protected: - ::std::shared_ptr iface_; -}; - -class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf { - public: - virtual ~ThriftHiveMetastoreNull() {} - void getMetaConf(std::string& /* _return */, const std::string& /* key */) { - return; - } - void setMetaConf(const std::string& /* key */, const std::string& /* value */) { - return; - } - void create_database(const Database& /* database */) { - return; - } - void get_database(Database& /* _return */, const std::string& /* name */) { - return; - } - void drop_database(const std::string& /* name */, const bool /* deleteData */, const bool /* cascade */) { - return; - } - void get_databases(std::vector & /* _return */, const std::string& /* pattern */) { - return; - } - void get_all_databases(std::vector & /* _return */) { - return; - } - void alter_database(const std::string& /* dbname */, const Database& /* db */) { - return; - } - void get_type(Type& /* _return */, const std::string& /* name */) { - return; - } - bool create_type(const Type& /* type */) { - bool _return = false; - return _return; - } - bool drop_type(const std::string& /* type */) { - bool _return = false; - return _return; - } - void get_type_all(std::map & /* _return */, const std::string& /* name */) { - return; - } - void get_fields(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* table_name */) { - return; - } - void get_fields_with_environment_context(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* table_name */, const EnvironmentContext& /* environment_context */) { - return; - } - void get_schema(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* table_name */) { - return; - } - void get_schema_with_environment_context(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* table_name */, const EnvironmentContext& /* environment_context */) { - return; - } - void create_table(const Table& /* tbl */) { - return; - } - void create_table_with_environment_context(const Table& /* tbl */, const EnvironmentContext& /* environment_context */) { - return; - } - void create_table_with_constraints(const Table& /* tbl */, const std::vector & /* primaryKeys */, const std::vector & /* foreignKeys */, const std::vector & /* uniqueConstraints */, const std::vector & /* notNullConstraints */) { - return; - } - void drop_constraint(const DropConstraintRequest& /* req */) { - return; - } - void add_primary_key(const AddPrimaryKeyRequest& /* req */) { - return; - } - void add_foreign_key(const AddForeignKeyRequest& /* req */) { - return; - } - void add_unique_constraint(const AddUniqueConstraintRequest& /* req */) { - return; - } - void add_not_null_constraint(const AddNotNullConstraintRequest& /* req */) { - return; - } - void drop_table(const std::string& /* dbname */, const std::string& /* name */, const bool /* deleteData */) { - return; - } - void drop_table_with_environment_context(const std::string& /* dbname */, const std::string& /* name */, const bool /* deleteData */, const EnvironmentContext& /* environment_context */) { - return; - } - void truncate_table(const std::string& /* dbName */, const std::string& /* tableName */, const std::vector & /* partNames */) { - return; - } - void get_tables(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* pattern */) { - return; - } - void get_tables_by_type(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* pattern */, const std::string& /* tableType */) { - return; - } - void get_table_meta(std::vector & /* _return */, const std::string& /* db_patterns */, const std::string& /* tbl_patterns */, const std::vector & /* tbl_types */) { - return; - } - void get_all_tables(std::vector & /* _return */, const std::string& /* db_name */) { - return; - } - void get_table(Table& /* _return */, const std::string& /* dbname */, const std::string& /* tbl_name */) { - return; - } - void get_table_objects_by_name(std::vector
& /* _return */, const std::string& /* dbname */, const std::vector & /* tbl_names */) { - return; - } - void get_table_req(GetTableResult& /* _return */, const GetTableRequest& /* req */) { - return; - } - void get_table_objects_by_name_req(GetTablesResult& /* _return */, const GetTablesRequest& /* req */) { - return; - } - void get_table_names_by_filter(std::vector & /* _return */, const std::string& /* dbname */, const std::string& /* filter */, const int16_t /* max_tables */) { - return; - } - void alter_table(const std::string& /* dbname */, const std::string& /* tbl_name */, const Table& /* new_tbl */) { - return; - } - void alter_table_with_environment_context(const std::string& /* dbname */, const std::string& /* tbl_name */, const Table& /* new_tbl */, const EnvironmentContext& /* environment_context */) { - return; - } - void alter_table_with_cascade(const std::string& /* dbname */, const std::string& /* tbl_name */, const Table& /* new_tbl */, const bool /* cascade */) { - return; - } - void add_partition(Partition& /* _return */, const Partition& /* new_part */) { - return; - } - void add_partition_with_environment_context(Partition& /* _return */, const Partition& /* new_part */, const EnvironmentContext& /* environment_context */) { - return; - } - int32_t add_partitions(const std::vector & /* new_parts */) { - int32_t _return = 0; - return _return; - } - int32_t add_partitions_pspec(const std::vector & /* new_parts */) { - int32_t _return = 0; - return _return; - } - void append_partition(Partition& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */) { - return; - } - void add_partitions_req(AddPartitionsResult& /* _return */, const AddPartitionsRequest& /* request */) { - return; - } - void append_partition_with_environment_context(Partition& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */, const EnvironmentContext& /* environment_context */) { - return; - } - void append_partition_by_name(Partition& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* part_name */) { - return; - } - void append_partition_by_name_with_environment_context(Partition& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* part_name */, const EnvironmentContext& /* environment_context */) { - return; - } - bool drop_partition(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */, const bool /* deleteData */) { - bool _return = false; - return _return; - } - bool drop_partition_with_environment_context(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */, const bool /* deleteData */, const EnvironmentContext& /* environment_context */) { - bool _return = false; - return _return; - } - bool drop_partition_by_name(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* part_name */, const bool /* deleteData */) { - bool _return = false; - return _return; - } - bool drop_partition_by_name_with_environment_context(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* part_name */, const bool /* deleteData */, const EnvironmentContext& /* environment_context */) { - bool _return = false; - return _return; - } - void drop_partitions_req(DropPartitionsResult& /* _return */, const DropPartitionsRequest& /* req */) { - return; - } - void get_partition(Partition& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */) { - return; - } - void exchange_partition(Partition& /* _return */, const std::map & /* partitionSpecs */, const std::string& /* source_db */, const std::string& /* source_table_name */, const std::string& /* dest_db */, const std::string& /* dest_table_name */) { - return; - } - void exchange_partitions(std::vector & /* _return */, const std::map & /* partitionSpecs */, const std::string& /* source_db */, const std::string& /* source_table_name */, const std::string& /* dest_db */, const std::string& /* dest_table_name */) { - return; - } - void get_partition_with_auth(Partition& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */, const std::string& /* user_name */, const std::vector & /* group_names */) { - return; - } - void get_partition_by_name(Partition& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* part_name */) { - return; - } - void get_partitions(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const int16_t /* max_parts */) { - return; - } - void get_partitions_with_auth(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const int16_t /* max_parts */, const std::string& /* user_name */, const std::vector & /* group_names */) { - return; - } - void get_partitions_pspec(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const int32_t /* max_parts */) { - return; - } - void get_partition_names(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const int16_t /* max_parts */) { - return; - } - void get_partition_values(PartitionValuesResponse& /* _return */, const PartitionValuesRequest& /* request */) { - return; - } - void get_partitions_ps(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */, const int16_t /* max_parts */) { - return; - } - void get_partitions_ps_with_auth(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */, const int16_t /* max_parts */, const std::string& /* user_name */, const std::vector & /* group_names */) { - return; - } - void get_partition_names_ps(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */, const int16_t /* max_parts */) { - return; - } - void get_partitions_by_filter(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* filter */, const int16_t /* max_parts */) { - return; - } - void get_part_specs_by_filter(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* filter */, const int32_t /* max_parts */) { - return; - } - void get_partitions_by_expr(PartitionsByExprResult& /* _return */, const PartitionsByExprRequest& /* req */) { - return; - } - int32_t get_num_partitions_by_filter(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* filter */) { - int32_t _return = 0; - return _return; - } - void get_partitions_by_names(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* names */) { - return; - } - void alter_partition(const std::string& /* db_name */, const std::string& /* tbl_name */, const Partition& /* new_part */) { - return; - } - void alter_partitions(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* new_parts */) { - return; - } - void alter_partitions_with_environment_context(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* new_parts */, const EnvironmentContext& /* environment_context */) { - return; - } - void alter_partition_with_environment_context(const std::string& /* db_name */, const std::string& /* tbl_name */, const Partition& /* new_part */, const EnvironmentContext& /* environment_context */) { - return; - } - void rename_partition(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::vector & /* part_vals */, const Partition& /* new_part */) { - return; - } - bool partition_name_has_valid_characters(const std::vector & /* part_vals */, const bool /* throw_exception */) { - bool _return = false; - return _return; - } - void get_config_value(std::string& /* _return */, const std::string& /* name */, const std::string& /* defaultValue */) { - return; - } - void partition_name_to_vals(std::vector & /* _return */, const std::string& /* part_name */) { - return; - } - void partition_name_to_spec(std::map & /* _return */, const std::string& /* part_name */) { - return; - } - void markPartitionForEvent(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::map & /* part_vals */, const PartitionEventType::type /* eventType */) { - return; - } - bool isPartitionMarkedForEvent(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::map & /* part_vals */, const PartitionEventType::type /* eventType */) { - bool _return = false; - return _return; - } - void add_index(Index& /* _return */, const Index& /* new_index */, const Table& /* index_table */) { - return; - } - void alter_index(const std::string& /* dbname */, const std::string& /* base_tbl_name */, const std::string& /* idx_name */, const Index& /* new_idx */) { - return; - } - bool drop_index_by_name(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* index_name */, const bool /* deleteData */) { - bool _return = false; - return _return; - } - void get_index_by_name(Index& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* index_name */) { - return; - } - void get_indexes(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const int16_t /* max_indexes */) { - return; - } - void get_index_names(std::vector & /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const int16_t /* max_indexes */) { - return; - } - void get_primary_keys(PrimaryKeysResponse& /* _return */, const PrimaryKeysRequest& /* request */) { - return; - } - void get_foreign_keys(ForeignKeysResponse& /* _return */, const ForeignKeysRequest& /* request */) { - return; - } - void get_unique_constraints(UniqueConstraintsResponse& /* _return */, const UniqueConstraintsRequest& /* request */) { - return; - } - void get_not_null_constraints(NotNullConstraintsResponse& /* _return */, const NotNullConstraintsRequest& /* request */) { - return; - } - bool update_table_column_statistics(const ColumnStatistics& /* stats_obj */) { - bool _return = false; - return _return; - } - bool update_partition_column_statistics(const ColumnStatistics& /* stats_obj */) { - bool _return = false; - return _return; - } - void get_table_column_statistics(ColumnStatistics& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* col_name */) { - return; - } - void get_partition_column_statistics(ColumnStatistics& /* _return */, const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* part_name */, const std::string& /* col_name */) { - return; - } - void get_table_statistics_req(TableStatsResult& /* _return */, const TableStatsRequest& /* request */) { - return; - } - void get_partitions_statistics_req(PartitionsStatsResult& /* _return */, const PartitionsStatsRequest& /* request */) { - return; - } - void get_aggr_stats_for(AggrStats& /* _return */, const PartitionsStatsRequest& /* request */) { - return; - } - bool set_aggr_stats_for(const SetPartitionsStatsRequest& /* request */) { - bool _return = false; - return _return; - } - bool delete_partition_column_statistics(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* part_name */, const std::string& /* col_name */) { - bool _return = false; - return _return; - } - bool delete_table_column_statistics(const std::string& /* db_name */, const std::string& /* tbl_name */, const std::string& /* col_name */) { - bool _return = false; - return _return; - } - void create_function(const Function& /* func */) { - return; - } - void drop_function(const std::string& /* dbName */, const std::string& /* funcName */) { - return; - } - void alter_function(const std::string& /* dbName */, const std::string& /* funcName */, const Function& /* newFunc */) { - return; - } - void get_functions(std::vector & /* _return */, const std::string& /* dbName */, const std::string& /* pattern */) { - return; - } - void get_function(Function& /* _return */, const std::string& /* dbName */, const std::string& /* funcName */) { - return; - } - void get_all_functions(GetAllFunctionsResponse& /* _return */) { - return; - } - bool create_role(const Role& /* role */) { - bool _return = false; - return _return; - } - bool drop_role(const std::string& /* role_name */) { - bool _return = false; - return _return; - } - void get_role_names(std::vector & /* _return */) { - return; - } - bool grant_role(const std::string& /* role_name */, const std::string& /* principal_name */, const PrincipalType::type /* principal_type */, const std::string& /* grantor */, const PrincipalType::type /* grantorType */, const bool /* grant_option */) { - bool _return = false; - return _return; - } - bool revoke_role(const std::string& /* role_name */, const std::string& /* principal_name */, const PrincipalType::type /* principal_type */) { - bool _return = false; - return _return; - } - void list_roles(std::vector & /* _return */, const std::string& /* principal_name */, const PrincipalType::type /* principal_type */) { - return; - } - void grant_revoke_role(GrantRevokeRoleResponse& /* _return */, const GrantRevokeRoleRequest& /* request */) { - return; - } - void get_principals_in_role(GetPrincipalsInRoleResponse& /* _return */, const GetPrincipalsInRoleRequest& /* request */) { - return; - } - void get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& /* _return */, const GetRoleGrantsForPrincipalRequest& /* request */) { - return; - } - void get_privilege_set(PrincipalPrivilegeSet& /* _return */, const HiveObjectRef& /* hiveObject */, const std::string& /* user_name */, const std::vector & /* group_names */) { - return; - } - void list_privileges(std::vector & /* _return */, const std::string& /* principal_name */, const PrincipalType::type /* principal_type */, const HiveObjectRef& /* hiveObject */) { - return; - } - bool grant_privileges(const PrivilegeBag& /* privileges */) { - bool _return = false; - return _return; - } - bool revoke_privileges(const PrivilegeBag& /* privileges */) { - bool _return = false; - return _return; - } - void grant_revoke_privileges(GrantRevokePrivilegeResponse& /* _return */, const GrantRevokePrivilegeRequest& /* request */) { - return; - } - void set_ugi(std::vector & /* _return */, const std::string& /* user_name */, const std::vector & /* group_names */) { - return; - } - void get_delegation_token(std::string& /* _return */, const std::string& /* token_owner */, const std::string& /* renewer_kerberos_principal_name */) { - return; - } - int64_t renew_delegation_token(const std::string& /* token_str_form */) { - int64_t _return = 0; - return _return; - } - void cancel_delegation_token(const std::string& /* token_str_form */) { - return; - } - bool add_token(const std::string& /* token_identifier */, const std::string& /* delegation_token */) { - bool _return = false; - return _return; - } - bool remove_token(const std::string& /* token_identifier */) { - bool _return = false; - return _return; - } - void get_token(std::string& /* _return */, const std::string& /* token_identifier */) { - return; - } - void get_all_token_identifiers(std::vector & /* _return */) { - return; - } - int32_t add_master_key(const std::string& /* key */) { - int32_t _return = 0; - return _return; - } - void update_master_key(const int32_t /* seq_number */, const std::string& /* key */) { - return; - } - bool remove_master_key(const int32_t /* key_seq */) { - bool _return = false; - return _return; - } - void get_master_keys(std::vector & /* _return */) { - return; - } - void get_open_txns(GetOpenTxnsResponse& /* _return */) { - return; - } - void get_open_txns_info(GetOpenTxnsInfoResponse& /* _return */) { - return; - } - void open_txns(OpenTxnsResponse& /* _return */, const OpenTxnRequest& /* rqst */) { - return; - } - void abort_txn(const AbortTxnRequest& /* rqst */) { - return; - } - void abort_txns(const AbortTxnsRequest& /* rqst */) { - return; - } - void commit_txn(const CommitTxnRequest& /* rqst */) { - return; - } - void lock(LockResponse& /* _return */, const LockRequest& /* rqst */) { - return; - } - void check_lock(LockResponse& /* _return */, const CheckLockRequest& /* rqst */) { - return; - } - void unlock(const UnlockRequest& /* rqst */) { - return; - } - void show_locks(ShowLocksResponse& /* _return */, const ShowLocksRequest& /* rqst */) { - return; - } - void heartbeat(const HeartbeatRequest& /* ids */) { - return; - } - void heartbeat_txn_range(HeartbeatTxnRangeResponse& /* _return */, const HeartbeatTxnRangeRequest& /* txns */) { - return; - } - void compact(const CompactionRequest& /* rqst */) { - return; - } - void compact2(CompactionResponse& /* _return */, const CompactionRequest& /* rqst */) { - return; - } - void show_compact(ShowCompactResponse& /* _return */, const ShowCompactRequest& /* rqst */) { - return; - } - void add_dynamic_partitions(const AddDynamicPartitions& /* rqst */) { - return; - } - void get_next_notification(NotificationEventResponse& /* _return */, const NotificationEventRequest& /* rqst */) { - return; - } - void get_current_notificationEventId(CurrentNotificationEventId& /* _return */) { - return; - } - void get_notification_events_count(NotificationEventsCountResponse& /* _return */, const NotificationEventsCountRequest& /* rqst */) { - return; - } - void fire_listener_event(FireEventResponse& /* _return */, const FireEventRequest& /* rqst */) { - return; - } - void flushCache() { - return; - } - void cm_recycle(CmRecycleResponse& /* _return */, const CmRecycleRequest& /* request */) { - return; - } - void get_file_metadata_by_expr(GetFileMetadataByExprResult& /* _return */, const GetFileMetadataByExprRequest& /* req */) { - return; - } - void get_file_metadata(GetFileMetadataResult& /* _return */, const GetFileMetadataRequest& /* req */) { - return; - } - void put_file_metadata(PutFileMetadataResult& /* _return */, const PutFileMetadataRequest& /* req */) { - return; - } - void clear_file_metadata(ClearFileMetadataResult& /* _return */, const ClearFileMetadataRequest& /* req */) { - return; - } - void cache_file_metadata(CacheFileMetadataResult& /* _return */, const CacheFileMetadataRequest& /* req */) { - return; - } - void get_metastore_db_uuid(std::string& /* _return */) { - return; - } - void create_resource_plan(WMCreateResourcePlanResponse& /* _return */, const WMCreateResourcePlanRequest& /* request */) { - return; - } - void get_resource_plan(WMGetResourcePlanResponse& /* _return */, const WMGetResourcePlanRequest& /* request */) { - return; - } - void get_active_resource_plan(WMGetActiveResourcePlanResponse& /* _return */, const WMGetActiveResourcePlanRequest& /* request */) { - return; - } - void get_all_resource_plans(WMGetAllResourcePlanResponse& /* _return */, const WMGetAllResourcePlanRequest& /* request */) { - return; - } - void alter_resource_plan(WMAlterResourcePlanResponse& /* _return */, const WMAlterResourcePlanRequest& /* request */) { - return; - } - void validate_resource_plan(WMValidateResourcePlanResponse& /* _return */, const WMValidateResourcePlanRequest& /* request */) { - return; - } - void drop_resource_plan(WMDropResourcePlanResponse& /* _return */, const WMDropResourcePlanRequest& /* request */) { - return; - } - void create_wm_trigger(WMCreateTriggerResponse& /* _return */, const WMCreateTriggerRequest& /* request */) { - return; - } - void alter_wm_trigger(WMAlterTriggerResponse& /* _return */, const WMAlterTriggerRequest& /* request */) { - return; - } - void drop_wm_trigger(WMDropTriggerResponse& /* _return */, const WMDropTriggerRequest& /* request */) { - return; - } - void get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& /* _return */, const WMGetTriggersForResourePlanRequest& /* request */) { - return; - } -}; - -typedef struct _ThriftHiveMetastore_getMetaConf_args__isset { - _ThriftHiveMetastore_getMetaConf_args__isset() : key(false) {} - bool key :1; -} _ThriftHiveMetastore_getMetaConf_args__isset; - -class ThriftHiveMetastore_getMetaConf_args { - public: - - ThriftHiveMetastore_getMetaConf_args(const ThriftHiveMetastore_getMetaConf_args&); - ThriftHiveMetastore_getMetaConf_args& operator=(const ThriftHiveMetastore_getMetaConf_args&); - ThriftHiveMetastore_getMetaConf_args() : key() { - } - - virtual ~ThriftHiveMetastore_getMetaConf_args() noexcept; - std::string key; - - _ThriftHiveMetastore_getMetaConf_args__isset __isset; - - void __set_key(const std::string& val); - - bool operator == (const ThriftHiveMetastore_getMetaConf_args & rhs) const - { - if (!(key == rhs.key)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_getMetaConf_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_getMetaConf_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_getMetaConf_pargs { - public: - - - virtual ~ThriftHiveMetastore_getMetaConf_pargs() noexcept; - const std::string* key; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_getMetaConf_result__isset { - _ThriftHiveMetastore_getMetaConf_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_getMetaConf_result__isset; - -class ThriftHiveMetastore_getMetaConf_result { - public: - - ThriftHiveMetastore_getMetaConf_result(const ThriftHiveMetastore_getMetaConf_result&); - ThriftHiveMetastore_getMetaConf_result& operator=(const ThriftHiveMetastore_getMetaConf_result&); - ThriftHiveMetastore_getMetaConf_result() : success() { - } - - virtual ~ThriftHiveMetastore_getMetaConf_result() noexcept; - std::string success; - MetaException o1; - - _ThriftHiveMetastore_getMetaConf_result__isset __isset; - - void __set_success(const std::string& val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_getMetaConf_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_getMetaConf_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_getMetaConf_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_getMetaConf_presult__isset { - _ThriftHiveMetastore_getMetaConf_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_getMetaConf_presult__isset; - -class ThriftHiveMetastore_getMetaConf_presult { - public: - - - virtual ~ThriftHiveMetastore_getMetaConf_presult() noexcept; - std::string* success; - MetaException o1; - - _ThriftHiveMetastore_getMetaConf_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_setMetaConf_args__isset { - _ThriftHiveMetastore_setMetaConf_args__isset() : key(false), value(false) {} - bool key :1; - bool value :1; -} _ThriftHiveMetastore_setMetaConf_args__isset; - -class ThriftHiveMetastore_setMetaConf_args { - public: - - ThriftHiveMetastore_setMetaConf_args(const ThriftHiveMetastore_setMetaConf_args&); - ThriftHiveMetastore_setMetaConf_args& operator=(const ThriftHiveMetastore_setMetaConf_args&); - ThriftHiveMetastore_setMetaConf_args() : key(), value() { - } - - virtual ~ThriftHiveMetastore_setMetaConf_args() noexcept; - std::string key; - std::string value; - - _ThriftHiveMetastore_setMetaConf_args__isset __isset; - - void __set_key(const std::string& val); - - void __set_value(const std::string& val); - - bool operator == (const ThriftHiveMetastore_setMetaConf_args & rhs) const - { - if (!(key == rhs.key)) - return false; - if (!(value == rhs.value)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_setMetaConf_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_setMetaConf_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_setMetaConf_pargs { - public: - - - virtual ~ThriftHiveMetastore_setMetaConf_pargs() noexcept; - const std::string* key; - const std::string* value; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_setMetaConf_result__isset { - _ThriftHiveMetastore_setMetaConf_result__isset() : o1(false) {} - bool o1 :1; -} _ThriftHiveMetastore_setMetaConf_result__isset; - -class ThriftHiveMetastore_setMetaConf_result { - public: - - ThriftHiveMetastore_setMetaConf_result(const ThriftHiveMetastore_setMetaConf_result&); - ThriftHiveMetastore_setMetaConf_result& operator=(const ThriftHiveMetastore_setMetaConf_result&); - ThriftHiveMetastore_setMetaConf_result() { - } - - virtual ~ThriftHiveMetastore_setMetaConf_result() noexcept; - MetaException o1; - - _ThriftHiveMetastore_setMetaConf_result__isset __isset; - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_setMetaConf_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_setMetaConf_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_setMetaConf_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_setMetaConf_presult__isset { - _ThriftHiveMetastore_setMetaConf_presult__isset() : o1(false) {} - bool o1 :1; -} _ThriftHiveMetastore_setMetaConf_presult__isset; - -class ThriftHiveMetastore_setMetaConf_presult { - public: - - - virtual ~ThriftHiveMetastore_setMetaConf_presult() noexcept; - MetaException o1; - - _ThriftHiveMetastore_setMetaConf_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_create_database_args__isset { - _ThriftHiveMetastore_create_database_args__isset() : database(false) {} - bool database :1; -} _ThriftHiveMetastore_create_database_args__isset; - -class ThriftHiveMetastore_create_database_args { - public: - - ThriftHiveMetastore_create_database_args(const ThriftHiveMetastore_create_database_args&); - ThriftHiveMetastore_create_database_args& operator=(const ThriftHiveMetastore_create_database_args&); - ThriftHiveMetastore_create_database_args() { - } - - virtual ~ThriftHiveMetastore_create_database_args() noexcept; - Database database; - - _ThriftHiveMetastore_create_database_args__isset __isset; - - void __set_database(const Database& val); - - bool operator == (const ThriftHiveMetastore_create_database_args & rhs) const - { - if (!(database == rhs.database)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_database_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_database_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_create_database_pargs { - public: - - - virtual ~ThriftHiveMetastore_create_database_pargs() noexcept; - const Database* database; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_database_result__isset { - _ThriftHiveMetastore_create_database_result__isset() : o1(false), o2(false), o3(false) {} - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_create_database_result__isset; - -class ThriftHiveMetastore_create_database_result { - public: - - ThriftHiveMetastore_create_database_result(const ThriftHiveMetastore_create_database_result&); - ThriftHiveMetastore_create_database_result& operator=(const ThriftHiveMetastore_create_database_result&); - ThriftHiveMetastore_create_database_result() { - } - - virtual ~ThriftHiveMetastore_create_database_result() noexcept; - AlreadyExistsException o1; - InvalidObjectException o2; - MetaException o3; - - _ThriftHiveMetastore_create_database_result__isset __isset; - - void __set_o1(const AlreadyExistsException& val); - - void __set_o2(const InvalidObjectException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_create_database_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_database_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_database_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_database_presult__isset { - _ThriftHiveMetastore_create_database_presult__isset() : o1(false), o2(false), o3(false) {} - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_create_database_presult__isset; - -class ThriftHiveMetastore_create_database_presult { - public: - - - virtual ~ThriftHiveMetastore_create_database_presult() noexcept; - AlreadyExistsException o1; - InvalidObjectException o2; - MetaException o3; - - _ThriftHiveMetastore_create_database_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_database_args__isset { - _ThriftHiveMetastore_get_database_args__isset() : name(false) {} - bool name :1; -} _ThriftHiveMetastore_get_database_args__isset; - -class ThriftHiveMetastore_get_database_args { - public: - - ThriftHiveMetastore_get_database_args(const ThriftHiveMetastore_get_database_args&); - ThriftHiveMetastore_get_database_args& operator=(const ThriftHiveMetastore_get_database_args&); - ThriftHiveMetastore_get_database_args() : name() { - } - - virtual ~ThriftHiveMetastore_get_database_args() noexcept; - std::string name; - - _ThriftHiveMetastore_get_database_args__isset __isset; - - void __set_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_database_args & rhs) const - { - if (!(name == rhs.name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_database_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_database_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_database_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_database_pargs() noexcept; - const std::string* name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_database_result__isset { - _ThriftHiveMetastore_get_database_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_database_result__isset; - -class ThriftHiveMetastore_get_database_result { - public: - - ThriftHiveMetastore_get_database_result(const ThriftHiveMetastore_get_database_result&); - ThriftHiveMetastore_get_database_result& operator=(const ThriftHiveMetastore_get_database_result&); - ThriftHiveMetastore_get_database_result() { - } - - virtual ~ThriftHiveMetastore_get_database_result() noexcept; - Database success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_database_result__isset __isset; - - void __set_success(const Database& val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_database_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_database_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_database_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_database_presult__isset { - _ThriftHiveMetastore_get_database_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_database_presult__isset; - -class ThriftHiveMetastore_get_database_presult { - public: - - - virtual ~ThriftHiveMetastore_get_database_presult() noexcept; - Database* success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_database_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_drop_database_args__isset { - _ThriftHiveMetastore_drop_database_args__isset() : name(false), deleteData(false), cascade(false) {} - bool name :1; - bool deleteData :1; - bool cascade :1; -} _ThriftHiveMetastore_drop_database_args__isset; - -class ThriftHiveMetastore_drop_database_args { - public: - - ThriftHiveMetastore_drop_database_args(const ThriftHiveMetastore_drop_database_args&); - ThriftHiveMetastore_drop_database_args& operator=(const ThriftHiveMetastore_drop_database_args&); - ThriftHiveMetastore_drop_database_args() : name(), deleteData(0), cascade(0) { - } - - virtual ~ThriftHiveMetastore_drop_database_args() noexcept; - std::string name; - bool deleteData; - bool cascade; - - _ThriftHiveMetastore_drop_database_args__isset __isset; - - void __set_name(const std::string& val); - - void __set_deleteData(const bool val); - - void __set_cascade(const bool val); - - bool operator == (const ThriftHiveMetastore_drop_database_args & rhs) const - { - if (!(name == rhs.name)) - return false; - if (!(deleteData == rhs.deleteData)) - return false; - if (!(cascade == rhs.cascade)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_database_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_database_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_drop_database_pargs { - public: - - - virtual ~ThriftHiveMetastore_drop_database_pargs() noexcept; - const std::string* name; - const bool* deleteData; - const bool* cascade; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_database_result__isset { - _ThriftHiveMetastore_drop_database_result__isset() : o1(false), o2(false), o3(false) {} - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_drop_database_result__isset; - -class ThriftHiveMetastore_drop_database_result { - public: - - ThriftHiveMetastore_drop_database_result(const ThriftHiveMetastore_drop_database_result&); - ThriftHiveMetastore_drop_database_result& operator=(const ThriftHiveMetastore_drop_database_result&); - ThriftHiveMetastore_drop_database_result() { - } - - virtual ~ThriftHiveMetastore_drop_database_result() noexcept; - NoSuchObjectException o1; - InvalidOperationException o2; - MetaException o3; - - _ThriftHiveMetastore_drop_database_result__isset __isset; - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const InvalidOperationException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_drop_database_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_database_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_database_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_database_presult__isset { - _ThriftHiveMetastore_drop_database_presult__isset() : o1(false), o2(false), o3(false) {} - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_drop_database_presult__isset; - -class ThriftHiveMetastore_drop_database_presult { - public: - - - virtual ~ThriftHiveMetastore_drop_database_presult() noexcept; - NoSuchObjectException o1; - InvalidOperationException o2; - MetaException o3; - - _ThriftHiveMetastore_drop_database_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_databases_args__isset { - _ThriftHiveMetastore_get_databases_args__isset() : pattern(false) {} - bool pattern :1; -} _ThriftHiveMetastore_get_databases_args__isset; - -class ThriftHiveMetastore_get_databases_args { - public: - - ThriftHiveMetastore_get_databases_args(const ThriftHiveMetastore_get_databases_args&); - ThriftHiveMetastore_get_databases_args& operator=(const ThriftHiveMetastore_get_databases_args&); - ThriftHiveMetastore_get_databases_args() : pattern() { - } - - virtual ~ThriftHiveMetastore_get_databases_args() noexcept; - std::string pattern; - - _ThriftHiveMetastore_get_databases_args__isset __isset; - - void __set_pattern(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_databases_args & rhs) const - { - if (!(pattern == rhs.pattern)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_databases_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_databases_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_databases_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_databases_pargs() noexcept; - const std::string* pattern; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_databases_result__isset { - _ThriftHiveMetastore_get_databases_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_databases_result__isset; - -class ThriftHiveMetastore_get_databases_result { - public: - - ThriftHiveMetastore_get_databases_result(const ThriftHiveMetastore_get_databases_result&); - ThriftHiveMetastore_get_databases_result& operator=(const ThriftHiveMetastore_get_databases_result&); - ThriftHiveMetastore_get_databases_result() { - } - - virtual ~ThriftHiveMetastore_get_databases_result() noexcept; - std::vector success; - MetaException o1; - - _ThriftHiveMetastore_get_databases_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_databases_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_databases_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_databases_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_databases_presult__isset { - _ThriftHiveMetastore_get_databases_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_databases_presult__isset; - -class ThriftHiveMetastore_get_databases_presult { - public: - - - virtual ~ThriftHiveMetastore_get_databases_presult() noexcept; - std::vector * success; - MetaException o1; - - _ThriftHiveMetastore_get_databases_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - - -class ThriftHiveMetastore_get_all_databases_args { - public: - - ThriftHiveMetastore_get_all_databases_args(const ThriftHiveMetastore_get_all_databases_args&); - ThriftHiveMetastore_get_all_databases_args& operator=(const ThriftHiveMetastore_get_all_databases_args&); - ThriftHiveMetastore_get_all_databases_args() { - } - - virtual ~ThriftHiveMetastore_get_all_databases_args() noexcept; - - bool operator == (const ThriftHiveMetastore_get_all_databases_args & /* rhs */) const - { - return true; - } - bool operator != (const ThriftHiveMetastore_get_all_databases_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_all_databases_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_all_databases_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_all_databases_pargs() noexcept; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_all_databases_result__isset { - _ThriftHiveMetastore_get_all_databases_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_all_databases_result__isset; - -class ThriftHiveMetastore_get_all_databases_result { - public: - - ThriftHiveMetastore_get_all_databases_result(const ThriftHiveMetastore_get_all_databases_result&); - ThriftHiveMetastore_get_all_databases_result& operator=(const ThriftHiveMetastore_get_all_databases_result&); - ThriftHiveMetastore_get_all_databases_result() { - } - - virtual ~ThriftHiveMetastore_get_all_databases_result() noexcept; - std::vector success; - MetaException o1; - - _ThriftHiveMetastore_get_all_databases_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_all_databases_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_all_databases_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_all_databases_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_all_databases_presult__isset { - _ThriftHiveMetastore_get_all_databases_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_all_databases_presult__isset; - -class ThriftHiveMetastore_get_all_databases_presult { - public: - - - virtual ~ThriftHiveMetastore_get_all_databases_presult() noexcept; - std::vector * success; - MetaException o1; - - _ThriftHiveMetastore_get_all_databases_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_alter_database_args__isset { - _ThriftHiveMetastore_alter_database_args__isset() : dbname(false), db(false) {} - bool dbname :1; - bool db :1; -} _ThriftHiveMetastore_alter_database_args__isset; - -class ThriftHiveMetastore_alter_database_args { - public: - - ThriftHiveMetastore_alter_database_args(const ThriftHiveMetastore_alter_database_args&); - ThriftHiveMetastore_alter_database_args& operator=(const ThriftHiveMetastore_alter_database_args&); - ThriftHiveMetastore_alter_database_args() : dbname() { - } - - virtual ~ThriftHiveMetastore_alter_database_args() noexcept; - std::string dbname; - Database db; - - _ThriftHiveMetastore_alter_database_args__isset __isset; - - void __set_dbname(const std::string& val); - - void __set_db(const Database& val); - - bool operator == (const ThriftHiveMetastore_alter_database_args & rhs) const - { - if (!(dbname == rhs.dbname)) - return false; - if (!(db == rhs.db)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_database_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_database_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_alter_database_pargs { - public: - - - virtual ~ThriftHiveMetastore_alter_database_pargs() noexcept; - const std::string* dbname; - const Database* db; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_database_result__isset { - _ThriftHiveMetastore_alter_database_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_database_result__isset; - -class ThriftHiveMetastore_alter_database_result { - public: - - ThriftHiveMetastore_alter_database_result(const ThriftHiveMetastore_alter_database_result&); - ThriftHiveMetastore_alter_database_result& operator=(const ThriftHiveMetastore_alter_database_result&); - ThriftHiveMetastore_alter_database_result() { - } - - virtual ~ThriftHiveMetastore_alter_database_result() noexcept; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_alter_database_result__isset __isset; - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_alter_database_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_database_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_database_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_database_presult__isset { - _ThriftHiveMetastore_alter_database_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_database_presult__isset; - -class ThriftHiveMetastore_alter_database_presult { - public: - - - virtual ~ThriftHiveMetastore_alter_database_presult() noexcept; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_alter_database_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_type_args__isset { - _ThriftHiveMetastore_get_type_args__isset() : name(false) {} - bool name :1; -} _ThriftHiveMetastore_get_type_args__isset; - -class ThriftHiveMetastore_get_type_args { - public: - - ThriftHiveMetastore_get_type_args(const ThriftHiveMetastore_get_type_args&); - ThriftHiveMetastore_get_type_args& operator=(const ThriftHiveMetastore_get_type_args&); - ThriftHiveMetastore_get_type_args() : name() { - } - - virtual ~ThriftHiveMetastore_get_type_args() noexcept; - std::string name; - - _ThriftHiveMetastore_get_type_args__isset __isset; - - void __set_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_type_args & rhs) const - { - if (!(name == rhs.name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_type_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_type_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_type_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_type_pargs() noexcept; - const std::string* name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_type_result__isset { - _ThriftHiveMetastore_get_type_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_type_result__isset; - -class ThriftHiveMetastore_get_type_result { - public: - - ThriftHiveMetastore_get_type_result(const ThriftHiveMetastore_get_type_result&); - ThriftHiveMetastore_get_type_result& operator=(const ThriftHiveMetastore_get_type_result&); - ThriftHiveMetastore_get_type_result() { - } - - virtual ~ThriftHiveMetastore_get_type_result() noexcept; - Type success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_type_result__isset __isset; - - void __set_success(const Type& val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_type_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_type_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_type_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_type_presult__isset { - _ThriftHiveMetastore_get_type_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_type_presult__isset; - -class ThriftHiveMetastore_get_type_presult { - public: - - - virtual ~ThriftHiveMetastore_get_type_presult() noexcept; - Type* success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_type_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_create_type_args__isset { - _ThriftHiveMetastore_create_type_args__isset() : type(false) {} - bool type :1; -} _ThriftHiveMetastore_create_type_args__isset; - -class ThriftHiveMetastore_create_type_args { - public: - - ThriftHiveMetastore_create_type_args(const ThriftHiveMetastore_create_type_args&); - ThriftHiveMetastore_create_type_args& operator=(const ThriftHiveMetastore_create_type_args&); - ThriftHiveMetastore_create_type_args() { - } - - virtual ~ThriftHiveMetastore_create_type_args() noexcept; - Type type; - - _ThriftHiveMetastore_create_type_args__isset __isset; - - void __set_type(const Type& val); - - bool operator == (const ThriftHiveMetastore_create_type_args & rhs) const - { - if (!(type == rhs.type)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_type_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_type_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_create_type_pargs { - public: - - - virtual ~ThriftHiveMetastore_create_type_pargs() noexcept; - const Type* type; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_type_result__isset { - _ThriftHiveMetastore_create_type_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_create_type_result__isset; - -class ThriftHiveMetastore_create_type_result { - public: - - ThriftHiveMetastore_create_type_result(const ThriftHiveMetastore_create_type_result&); - ThriftHiveMetastore_create_type_result& operator=(const ThriftHiveMetastore_create_type_result&); - ThriftHiveMetastore_create_type_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_create_type_result() noexcept; - bool success; - AlreadyExistsException o1; - InvalidObjectException o2; - MetaException o3; - - _ThriftHiveMetastore_create_type_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const AlreadyExistsException& val); - - void __set_o2(const InvalidObjectException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_create_type_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_type_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_type_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_type_presult__isset { - _ThriftHiveMetastore_create_type_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_create_type_presult__isset; - -class ThriftHiveMetastore_create_type_presult { - public: - - - virtual ~ThriftHiveMetastore_create_type_presult() noexcept; - bool* success; - AlreadyExistsException o1; - InvalidObjectException o2; - MetaException o3; - - _ThriftHiveMetastore_create_type_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_drop_type_args__isset { - _ThriftHiveMetastore_drop_type_args__isset() : type(false) {} - bool type :1; -} _ThriftHiveMetastore_drop_type_args__isset; - -class ThriftHiveMetastore_drop_type_args { - public: - - ThriftHiveMetastore_drop_type_args(const ThriftHiveMetastore_drop_type_args&); - ThriftHiveMetastore_drop_type_args& operator=(const ThriftHiveMetastore_drop_type_args&); - ThriftHiveMetastore_drop_type_args() : type() { - } - - virtual ~ThriftHiveMetastore_drop_type_args() noexcept; - std::string type; - - _ThriftHiveMetastore_drop_type_args__isset __isset; - - void __set_type(const std::string& val); - - bool operator == (const ThriftHiveMetastore_drop_type_args & rhs) const - { - if (!(type == rhs.type)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_type_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_type_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_drop_type_pargs { - public: - - - virtual ~ThriftHiveMetastore_drop_type_pargs() noexcept; - const std::string* type; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_type_result__isset { - _ThriftHiveMetastore_drop_type_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_drop_type_result__isset; - -class ThriftHiveMetastore_drop_type_result { - public: - - ThriftHiveMetastore_drop_type_result(const ThriftHiveMetastore_drop_type_result&); - ThriftHiveMetastore_drop_type_result& operator=(const ThriftHiveMetastore_drop_type_result&); - ThriftHiveMetastore_drop_type_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_drop_type_result() noexcept; - bool success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_drop_type_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_drop_type_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_type_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_type_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_type_presult__isset { - _ThriftHiveMetastore_drop_type_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_drop_type_presult__isset; - -class ThriftHiveMetastore_drop_type_presult { - public: - - - virtual ~ThriftHiveMetastore_drop_type_presult() noexcept; - bool* success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_drop_type_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_type_all_args__isset { - _ThriftHiveMetastore_get_type_all_args__isset() : name(false) {} - bool name :1; -} _ThriftHiveMetastore_get_type_all_args__isset; - -class ThriftHiveMetastore_get_type_all_args { - public: - - ThriftHiveMetastore_get_type_all_args(const ThriftHiveMetastore_get_type_all_args&); - ThriftHiveMetastore_get_type_all_args& operator=(const ThriftHiveMetastore_get_type_all_args&); - ThriftHiveMetastore_get_type_all_args() : name() { - } - - virtual ~ThriftHiveMetastore_get_type_all_args() noexcept; - std::string name; - - _ThriftHiveMetastore_get_type_all_args__isset __isset; - - void __set_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_type_all_args & rhs) const - { - if (!(name == rhs.name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_type_all_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_type_all_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_type_all_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_type_all_pargs() noexcept; - const std::string* name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_type_all_result__isset { - _ThriftHiveMetastore_get_type_all_result__isset() : success(false), o2(false) {} - bool success :1; - bool o2 :1; -} _ThriftHiveMetastore_get_type_all_result__isset; - -class ThriftHiveMetastore_get_type_all_result { - public: - - ThriftHiveMetastore_get_type_all_result(const ThriftHiveMetastore_get_type_all_result&); - ThriftHiveMetastore_get_type_all_result& operator=(const ThriftHiveMetastore_get_type_all_result&); - ThriftHiveMetastore_get_type_all_result() { - } - - virtual ~ThriftHiveMetastore_get_type_all_result() noexcept; - std::map success; - MetaException o2; - - _ThriftHiveMetastore_get_type_all_result__isset __isset; - - void __set_success(const std::map & val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_type_all_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_type_all_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_type_all_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_type_all_presult__isset { - _ThriftHiveMetastore_get_type_all_presult__isset() : success(false), o2(false) {} - bool success :1; - bool o2 :1; -} _ThriftHiveMetastore_get_type_all_presult__isset; - -class ThriftHiveMetastore_get_type_all_presult { - public: - - - virtual ~ThriftHiveMetastore_get_type_all_presult() noexcept; - std::map * success; - MetaException o2; - - _ThriftHiveMetastore_get_type_all_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_fields_args__isset { - _ThriftHiveMetastore_get_fields_args__isset() : db_name(false), table_name(false) {} - bool db_name :1; - bool table_name :1; -} _ThriftHiveMetastore_get_fields_args__isset; - -class ThriftHiveMetastore_get_fields_args { - public: - - ThriftHiveMetastore_get_fields_args(const ThriftHiveMetastore_get_fields_args&); - ThriftHiveMetastore_get_fields_args& operator=(const ThriftHiveMetastore_get_fields_args&); - ThriftHiveMetastore_get_fields_args() : db_name(), table_name() { - } - - virtual ~ThriftHiveMetastore_get_fields_args() noexcept; - std::string db_name; - std::string table_name; - - _ThriftHiveMetastore_get_fields_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_table_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_fields_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(table_name == rhs.table_name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_fields_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_fields_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_fields_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_fields_pargs() noexcept; - const std::string* db_name; - const std::string* table_name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_fields_result__isset { - _ThriftHiveMetastore_get_fields_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_get_fields_result__isset; - -class ThriftHiveMetastore_get_fields_result { - public: - - ThriftHiveMetastore_get_fields_result(const ThriftHiveMetastore_get_fields_result&); - ThriftHiveMetastore_get_fields_result& operator=(const ThriftHiveMetastore_get_fields_result&); - ThriftHiveMetastore_get_fields_result() { - } - - virtual ~ThriftHiveMetastore_get_fields_result() noexcept; - std::vector success; - MetaException o1; - UnknownTableException o2; - UnknownDBException o3; - - _ThriftHiveMetastore_get_fields_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - void __set_o2(const UnknownTableException& val); - - void __set_o3(const UnknownDBException& val); - - bool operator == (const ThriftHiveMetastore_get_fields_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_fields_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_fields_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_fields_presult__isset { - _ThriftHiveMetastore_get_fields_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_get_fields_presult__isset; - -class ThriftHiveMetastore_get_fields_presult { - public: - - - virtual ~ThriftHiveMetastore_get_fields_presult() noexcept; - std::vector * success; - MetaException o1; - UnknownTableException o2; - UnknownDBException o3; - - _ThriftHiveMetastore_get_fields_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_fields_with_environment_context_args__isset { - _ThriftHiveMetastore_get_fields_with_environment_context_args__isset() : db_name(false), table_name(false), environment_context(false) {} - bool db_name :1; - bool table_name :1; - bool environment_context :1; -} _ThriftHiveMetastore_get_fields_with_environment_context_args__isset; - -class ThriftHiveMetastore_get_fields_with_environment_context_args { - public: - - ThriftHiveMetastore_get_fields_with_environment_context_args(const ThriftHiveMetastore_get_fields_with_environment_context_args&); - ThriftHiveMetastore_get_fields_with_environment_context_args& operator=(const ThriftHiveMetastore_get_fields_with_environment_context_args&); - ThriftHiveMetastore_get_fields_with_environment_context_args() : db_name(), table_name() { - } - - virtual ~ThriftHiveMetastore_get_fields_with_environment_context_args() noexcept; - std::string db_name; - std::string table_name; - EnvironmentContext environment_context; - - _ThriftHiveMetastore_get_fields_with_environment_context_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_table_name(const std::string& val); - - void __set_environment_context(const EnvironmentContext& val); - - bool operator == (const ThriftHiveMetastore_get_fields_with_environment_context_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(table_name == rhs.table_name)) - return false; - if (!(environment_context == rhs.environment_context)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_fields_with_environment_context_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_fields_with_environment_context_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_fields_with_environment_context_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_fields_with_environment_context_pargs() noexcept; - const std::string* db_name; - const std::string* table_name; - const EnvironmentContext* environment_context; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_fields_with_environment_context_result__isset { - _ThriftHiveMetastore_get_fields_with_environment_context_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_get_fields_with_environment_context_result__isset; - -class ThriftHiveMetastore_get_fields_with_environment_context_result { - public: - - ThriftHiveMetastore_get_fields_with_environment_context_result(const ThriftHiveMetastore_get_fields_with_environment_context_result&); - ThriftHiveMetastore_get_fields_with_environment_context_result& operator=(const ThriftHiveMetastore_get_fields_with_environment_context_result&); - ThriftHiveMetastore_get_fields_with_environment_context_result() { - } - - virtual ~ThriftHiveMetastore_get_fields_with_environment_context_result() noexcept; - std::vector success; - MetaException o1; - UnknownTableException o2; - UnknownDBException o3; - - _ThriftHiveMetastore_get_fields_with_environment_context_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - void __set_o2(const UnknownTableException& val); - - void __set_o3(const UnknownDBException& val); - - bool operator == (const ThriftHiveMetastore_get_fields_with_environment_context_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_fields_with_environment_context_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_fields_with_environment_context_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_fields_with_environment_context_presult__isset { - _ThriftHiveMetastore_get_fields_with_environment_context_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_get_fields_with_environment_context_presult__isset; - -class ThriftHiveMetastore_get_fields_with_environment_context_presult { - public: - - - virtual ~ThriftHiveMetastore_get_fields_with_environment_context_presult() noexcept; - std::vector * success; - MetaException o1; - UnknownTableException o2; - UnknownDBException o3; - - _ThriftHiveMetastore_get_fields_with_environment_context_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_schema_args__isset { - _ThriftHiveMetastore_get_schema_args__isset() : db_name(false), table_name(false) {} - bool db_name :1; - bool table_name :1; -} _ThriftHiveMetastore_get_schema_args__isset; - -class ThriftHiveMetastore_get_schema_args { - public: - - ThriftHiveMetastore_get_schema_args(const ThriftHiveMetastore_get_schema_args&); - ThriftHiveMetastore_get_schema_args& operator=(const ThriftHiveMetastore_get_schema_args&); - ThriftHiveMetastore_get_schema_args() : db_name(), table_name() { - } - - virtual ~ThriftHiveMetastore_get_schema_args() noexcept; - std::string db_name; - std::string table_name; - - _ThriftHiveMetastore_get_schema_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_table_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_schema_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(table_name == rhs.table_name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_schema_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_schema_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_schema_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_schema_pargs() noexcept; - const std::string* db_name; - const std::string* table_name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_schema_result__isset { - _ThriftHiveMetastore_get_schema_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_get_schema_result__isset; - -class ThriftHiveMetastore_get_schema_result { - public: - - ThriftHiveMetastore_get_schema_result(const ThriftHiveMetastore_get_schema_result&); - ThriftHiveMetastore_get_schema_result& operator=(const ThriftHiveMetastore_get_schema_result&); - ThriftHiveMetastore_get_schema_result() { - } - - virtual ~ThriftHiveMetastore_get_schema_result() noexcept; - std::vector success; - MetaException o1; - UnknownTableException o2; - UnknownDBException o3; - - _ThriftHiveMetastore_get_schema_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - void __set_o2(const UnknownTableException& val); - - void __set_o3(const UnknownDBException& val); - - bool operator == (const ThriftHiveMetastore_get_schema_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_schema_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_schema_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_schema_presult__isset { - _ThriftHiveMetastore_get_schema_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_get_schema_presult__isset; - -class ThriftHiveMetastore_get_schema_presult { - public: - - - virtual ~ThriftHiveMetastore_get_schema_presult() noexcept; - std::vector * success; - MetaException o1; - UnknownTableException o2; - UnknownDBException o3; - - _ThriftHiveMetastore_get_schema_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_schema_with_environment_context_args__isset { - _ThriftHiveMetastore_get_schema_with_environment_context_args__isset() : db_name(false), table_name(false), environment_context(false) {} - bool db_name :1; - bool table_name :1; - bool environment_context :1; -} _ThriftHiveMetastore_get_schema_with_environment_context_args__isset; - -class ThriftHiveMetastore_get_schema_with_environment_context_args { - public: - - ThriftHiveMetastore_get_schema_with_environment_context_args(const ThriftHiveMetastore_get_schema_with_environment_context_args&); - ThriftHiveMetastore_get_schema_with_environment_context_args& operator=(const ThriftHiveMetastore_get_schema_with_environment_context_args&); - ThriftHiveMetastore_get_schema_with_environment_context_args() : db_name(), table_name() { - } - - virtual ~ThriftHiveMetastore_get_schema_with_environment_context_args() noexcept; - std::string db_name; - std::string table_name; - EnvironmentContext environment_context; - - _ThriftHiveMetastore_get_schema_with_environment_context_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_table_name(const std::string& val); - - void __set_environment_context(const EnvironmentContext& val); - - bool operator == (const ThriftHiveMetastore_get_schema_with_environment_context_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(table_name == rhs.table_name)) - return false; - if (!(environment_context == rhs.environment_context)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_schema_with_environment_context_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_schema_with_environment_context_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_schema_with_environment_context_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_schema_with_environment_context_pargs() noexcept; - const std::string* db_name; - const std::string* table_name; - const EnvironmentContext* environment_context; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_schema_with_environment_context_result__isset { - _ThriftHiveMetastore_get_schema_with_environment_context_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_get_schema_with_environment_context_result__isset; - -class ThriftHiveMetastore_get_schema_with_environment_context_result { - public: - - ThriftHiveMetastore_get_schema_with_environment_context_result(const ThriftHiveMetastore_get_schema_with_environment_context_result&); - ThriftHiveMetastore_get_schema_with_environment_context_result& operator=(const ThriftHiveMetastore_get_schema_with_environment_context_result&); - ThriftHiveMetastore_get_schema_with_environment_context_result() { - } - - virtual ~ThriftHiveMetastore_get_schema_with_environment_context_result() noexcept; - std::vector success; - MetaException o1; - UnknownTableException o2; - UnknownDBException o3; - - _ThriftHiveMetastore_get_schema_with_environment_context_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - void __set_o2(const UnknownTableException& val); - - void __set_o3(const UnknownDBException& val); - - bool operator == (const ThriftHiveMetastore_get_schema_with_environment_context_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_schema_with_environment_context_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_schema_with_environment_context_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_schema_with_environment_context_presult__isset { - _ThriftHiveMetastore_get_schema_with_environment_context_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_get_schema_with_environment_context_presult__isset; - -class ThriftHiveMetastore_get_schema_with_environment_context_presult { - public: - - - virtual ~ThriftHiveMetastore_get_schema_with_environment_context_presult() noexcept; - std::vector * success; - MetaException o1; - UnknownTableException o2; - UnknownDBException o3; - - _ThriftHiveMetastore_get_schema_with_environment_context_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_create_table_args__isset { - _ThriftHiveMetastore_create_table_args__isset() : tbl(false) {} - bool tbl :1; -} _ThriftHiveMetastore_create_table_args__isset; - -class ThriftHiveMetastore_create_table_args { - public: - - ThriftHiveMetastore_create_table_args(const ThriftHiveMetastore_create_table_args&); - ThriftHiveMetastore_create_table_args& operator=(const ThriftHiveMetastore_create_table_args&); - ThriftHiveMetastore_create_table_args() { - } - - virtual ~ThriftHiveMetastore_create_table_args() noexcept; - Table tbl; - - _ThriftHiveMetastore_create_table_args__isset __isset; - - void __set_tbl(const Table& val); - - bool operator == (const ThriftHiveMetastore_create_table_args & rhs) const - { - if (!(tbl == rhs.tbl)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_table_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_table_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_create_table_pargs { - public: - - - virtual ~ThriftHiveMetastore_create_table_pargs() noexcept; - const Table* tbl; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_table_result__isset { - _ThriftHiveMetastore_create_table_result__isset() : o1(false), o2(false), o3(false), o4(false) {} - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_create_table_result__isset; - -class ThriftHiveMetastore_create_table_result { - public: - - ThriftHiveMetastore_create_table_result(const ThriftHiveMetastore_create_table_result&); - ThriftHiveMetastore_create_table_result& operator=(const ThriftHiveMetastore_create_table_result&); - ThriftHiveMetastore_create_table_result() { - } - - virtual ~ThriftHiveMetastore_create_table_result() noexcept; - AlreadyExistsException o1; - InvalidObjectException o2; - MetaException o3; - NoSuchObjectException o4; - - _ThriftHiveMetastore_create_table_result__isset __isset; - - void __set_o1(const AlreadyExistsException& val); - - void __set_o2(const InvalidObjectException& val); - - void __set_o3(const MetaException& val); - - void __set_o4(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_create_table_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - if (!(o4 == rhs.o4)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_table_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_table_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_table_presult__isset { - _ThriftHiveMetastore_create_table_presult__isset() : o1(false), o2(false), o3(false), o4(false) {} - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_create_table_presult__isset; - -class ThriftHiveMetastore_create_table_presult { - public: - - - virtual ~ThriftHiveMetastore_create_table_presult() noexcept; - AlreadyExistsException o1; - InvalidObjectException o2; - MetaException o3; - NoSuchObjectException o4; - - _ThriftHiveMetastore_create_table_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_create_table_with_environment_context_args__isset { - _ThriftHiveMetastore_create_table_with_environment_context_args__isset() : tbl(false), environment_context(false) {} - bool tbl :1; - bool environment_context :1; -} _ThriftHiveMetastore_create_table_with_environment_context_args__isset; - -class ThriftHiveMetastore_create_table_with_environment_context_args { - public: - - ThriftHiveMetastore_create_table_with_environment_context_args(const ThriftHiveMetastore_create_table_with_environment_context_args&); - ThriftHiveMetastore_create_table_with_environment_context_args& operator=(const ThriftHiveMetastore_create_table_with_environment_context_args&); - ThriftHiveMetastore_create_table_with_environment_context_args() { - } - - virtual ~ThriftHiveMetastore_create_table_with_environment_context_args() noexcept; - Table tbl; - EnvironmentContext environment_context; - - _ThriftHiveMetastore_create_table_with_environment_context_args__isset __isset; - - void __set_tbl(const Table& val); - - void __set_environment_context(const EnvironmentContext& val); - - bool operator == (const ThriftHiveMetastore_create_table_with_environment_context_args & rhs) const - { - if (!(tbl == rhs.tbl)) - return false; - if (!(environment_context == rhs.environment_context)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_table_with_environment_context_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_table_with_environment_context_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_create_table_with_environment_context_pargs { - public: - - - virtual ~ThriftHiveMetastore_create_table_with_environment_context_pargs() noexcept; - const Table* tbl; - const EnvironmentContext* environment_context; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_table_with_environment_context_result__isset { - _ThriftHiveMetastore_create_table_with_environment_context_result__isset() : o1(false), o2(false), o3(false), o4(false) {} - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_create_table_with_environment_context_result__isset; - -class ThriftHiveMetastore_create_table_with_environment_context_result { - public: - - ThriftHiveMetastore_create_table_with_environment_context_result(const ThriftHiveMetastore_create_table_with_environment_context_result&); - ThriftHiveMetastore_create_table_with_environment_context_result& operator=(const ThriftHiveMetastore_create_table_with_environment_context_result&); - ThriftHiveMetastore_create_table_with_environment_context_result() { - } - - virtual ~ThriftHiveMetastore_create_table_with_environment_context_result() noexcept; - AlreadyExistsException o1; - InvalidObjectException o2; - MetaException o3; - NoSuchObjectException o4; - - _ThriftHiveMetastore_create_table_with_environment_context_result__isset __isset; - - void __set_o1(const AlreadyExistsException& val); - - void __set_o2(const InvalidObjectException& val); - - void __set_o3(const MetaException& val); - - void __set_o4(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_create_table_with_environment_context_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - if (!(o4 == rhs.o4)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_table_with_environment_context_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_table_with_environment_context_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_table_with_environment_context_presult__isset { - _ThriftHiveMetastore_create_table_with_environment_context_presult__isset() : o1(false), o2(false), o3(false), o4(false) {} - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_create_table_with_environment_context_presult__isset; - -class ThriftHiveMetastore_create_table_with_environment_context_presult { - public: - - - virtual ~ThriftHiveMetastore_create_table_with_environment_context_presult() noexcept; - AlreadyExistsException o1; - InvalidObjectException o2; - MetaException o3; - NoSuchObjectException o4; - - _ThriftHiveMetastore_create_table_with_environment_context_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_create_table_with_constraints_args__isset { - _ThriftHiveMetastore_create_table_with_constraints_args__isset() : tbl(false), primaryKeys(false), foreignKeys(false), uniqueConstraints(false), notNullConstraints(false) {} - bool tbl :1; - bool primaryKeys :1; - bool foreignKeys :1; - bool uniqueConstraints :1; - bool notNullConstraints :1; -} _ThriftHiveMetastore_create_table_with_constraints_args__isset; - -class ThriftHiveMetastore_create_table_with_constraints_args { - public: - - ThriftHiveMetastore_create_table_with_constraints_args(const ThriftHiveMetastore_create_table_with_constraints_args&); - ThriftHiveMetastore_create_table_with_constraints_args& operator=(const ThriftHiveMetastore_create_table_with_constraints_args&); - ThriftHiveMetastore_create_table_with_constraints_args() { - } - - virtual ~ThriftHiveMetastore_create_table_with_constraints_args() noexcept; - Table tbl; - std::vector primaryKeys; - std::vector foreignKeys; - std::vector uniqueConstraints; - std::vector notNullConstraints; - - _ThriftHiveMetastore_create_table_with_constraints_args__isset __isset; - - void __set_tbl(const Table& val); - - void __set_primaryKeys(const std::vector & val); - - void __set_foreignKeys(const std::vector & val); - - void __set_uniqueConstraints(const std::vector & val); - - void __set_notNullConstraints(const std::vector & val); - - bool operator == (const ThriftHiveMetastore_create_table_with_constraints_args & rhs) const - { - if (!(tbl == rhs.tbl)) - return false; - if (!(primaryKeys == rhs.primaryKeys)) - return false; - if (!(foreignKeys == rhs.foreignKeys)) - return false; - if (!(uniqueConstraints == rhs.uniqueConstraints)) - return false; - if (!(notNullConstraints == rhs.notNullConstraints)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_table_with_constraints_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_table_with_constraints_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_create_table_with_constraints_pargs { - public: - - - virtual ~ThriftHiveMetastore_create_table_with_constraints_pargs() noexcept; - const Table* tbl; - const std::vector * primaryKeys; - const std::vector * foreignKeys; - const std::vector * uniqueConstraints; - const std::vector * notNullConstraints; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_table_with_constraints_result__isset { - _ThriftHiveMetastore_create_table_with_constraints_result__isset() : o1(false), o2(false), o3(false), o4(false) {} - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_create_table_with_constraints_result__isset; - -class ThriftHiveMetastore_create_table_with_constraints_result { - public: - - ThriftHiveMetastore_create_table_with_constraints_result(const ThriftHiveMetastore_create_table_with_constraints_result&); - ThriftHiveMetastore_create_table_with_constraints_result& operator=(const ThriftHiveMetastore_create_table_with_constraints_result&); - ThriftHiveMetastore_create_table_with_constraints_result() { - } - - virtual ~ThriftHiveMetastore_create_table_with_constraints_result() noexcept; - AlreadyExistsException o1; - InvalidObjectException o2; - MetaException o3; - NoSuchObjectException o4; - - _ThriftHiveMetastore_create_table_with_constraints_result__isset __isset; - - void __set_o1(const AlreadyExistsException& val); - - void __set_o2(const InvalidObjectException& val); - - void __set_o3(const MetaException& val); - - void __set_o4(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_create_table_with_constraints_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - if (!(o4 == rhs.o4)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_table_with_constraints_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_table_with_constraints_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_table_with_constraints_presult__isset { - _ThriftHiveMetastore_create_table_with_constraints_presult__isset() : o1(false), o2(false), o3(false), o4(false) {} - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_create_table_with_constraints_presult__isset; - -class ThriftHiveMetastore_create_table_with_constraints_presult { - public: - - - virtual ~ThriftHiveMetastore_create_table_with_constraints_presult() noexcept; - AlreadyExistsException o1; - InvalidObjectException o2; - MetaException o3; - NoSuchObjectException o4; - - _ThriftHiveMetastore_create_table_with_constraints_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_drop_constraint_args__isset { - _ThriftHiveMetastore_drop_constraint_args__isset() : req(false) {} - bool req :1; -} _ThriftHiveMetastore_drop_constraint_args__isset; - -class ThriftHiveMetastore_drop_constraint_args { - public: - - ThriftHiveMetastore_drop_constraint_args(const ThriftHiveMetastore_drop_constraint_args&); - ThriftHiveMetastore_drop_constraint_args& operator=(const ThriftHiveMetastore_drop_constraint_args&); - ThriftHiveMetastore_drop_constraint_args() { - } - - virtual ~ThriftHiveMetastore_drop_constraint_args() noexcept; - DropConstraintRequest req; - - _ThriftHiveMetastore_drop_constraint_args__isset __isset; - - void __set_req(const DropConstraintRequest& val); - - bool operator == (const ThriftHiveMetastore_drop_constraint_args & rhs) const - { - if (!(req == rhs.req)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_constraint_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_constraint_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_drop_constraint_pargs { - public: - - - virtual ~ThriftHiveMetastore_drop_constraint_pargs() noexcept; - const DropConstraintRequest* req; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_constraint_result__isset { - _ThriftHiveMetastore_drop_constraint_result__isset() : o1(false), o3(false) {} - bool o1 :1; - bool o3 :1; -} _ThriftHiveMetastore_drop_constraint_result__isset; - -class ThriftHiveMetastore_drop_constraint_result { - public: - - ThriftHiveMetastore_drop_constraint_result(const ThriftHiveMetastore_drop_constraint_result&); - ThriftHiveMetastore_drop_constraint_result& operator=(const ThriftHiveMetastore_drop_constraint_result&); - ThriftHiveMetastore_drop_constraint_result() { - } - - virtual ~ThriftHiveMetastore_drop_constraint_result() noexcept; - NoSuchObjectException o1; - MetaException o3; - - _ThriftHiveMetastore_drop_constraint_result__isset __isset; - - void __set_o1(const NoSuchObjectException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_drop_constraint_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_constraint_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_constraint_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_constraint_presult__isset { - _ThriftHiveMetastore_drop_constraint_presult__isset() : o1(false), o3(false) {} - bool o1 :1; - bool o3 :1; -} _ThriftHiveMetastore_drop_constraint_presult__isset; - -class ThriftHiveMetastore_drop_constraint_presult { - public: - - - virtual ~ThriftHiveMetastore_drop_constraint_presult() noexcept; - NoSuchObjectException o1; - MetaException o3; - - _ThriftHiveMetastore_drop_constraint_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_add_primary_key_args__isset { - _ThriftHiveMetastore_add_primary_key_args__isset() : req(false) {} - bool req :1; -} _ThriftHiveMetastore_add_primary_key_args__isset; - -class ThriftHiveMetastore_add_primary_key_args { - public: - - ThriftHiveMetastore_add_primary_key_args(const ThriftHiveMetastore_add_primary_key_args&); - ThriftHiveMetastore_add_primary_key_args& operator=(const ThriftHiveMetastore_add_primary_key_args&); - ThriftHiveMetastore_add_primary_key_args() { - } - - virtual ~ThriftHiveMetastore_add_primary_key_args() noexcept; - AddPrimaryKeyRequest req; - - _ThriftHiveMetastore_add_primary_key_args__isset __isset; - - void __set_req(const AddPrimaryKeyRequest& val); - - bool operator == (const ThriftHiveMetastore_add_primary_key_args & rhs) const - { - if (!(req == rhs.req)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_primary_key_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_primary_key_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_add_primary_key_pargs { - public: - - - virtual ~ThriftHiveMetastore_add_primary_key_pargs() noexcept; - const AddPrimaryKeyRequest* req; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_primary_key_result__isset { - _ThriftHiveMetastore_add_primary_key_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_add_primary_key_result__isset; - -class ThriftHiveMetastore_add_primary_key_result { - public: - - ThriftHiveMetastore_add_primary_key_result(const ThriftHiveMetastore_add_primary_key_result&); - ThriftHiveMetastore_add_primary_key_result& operator=(const ThriftHiveMetastore_add_primary_key_result&); - ThriftHiveMetastore_add_primary_key_result() { - } - - virtual ~ThriftHiveMetastore_add_primary_key_result() noexcept; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_add_primary_key_result__isset __isset; - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_add_primary_key_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_primary_key_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_primary_key_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_primary_key_presult__isset { - _ThriftHiveMetastore_add_primary_key_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_add_primary_key_presult__isset; - -class ThriftHiveMetastore_add_primary_key_presult { - public: - - - virtual ~ThriftHiveMetastore_add_primary_key_presult() noexcept; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_add_primary_key_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_add_foreign_key_args__isset { - _ThriftHiveMetastore_add_foreign_key_args__isset() : req(false) {} - bool req :1; -} _ThriftHiveMetastore_add_foreign_key_args__isset; - -class ThriftHiveMetastore_add_foreign_key_args { - public: - - ThriftHiveMetastore_add_foreign_key_args(const ThriftHiveMetastore_add_foreign_key_args&); - ThriftHiveMetastore_add_foreign_key_args& operator=(const ThriftHiveMetastore_add_foreign_key_args&); - ThriftHiveMetastore_add_foreign_key_args() { - } - - virtual ~ThriftHiveMetastore_add_foreign_key_args() noexcept; - AddForeignKeyRequest req; - - _ThriftHiveMetastore_add_foreign_key_args__isset __isset; - - void __set_req(const AddForeignKeyRequest& val); - - bool operator == (const ThriftHiveMetastore_add_foreign_key_args & rhs) const - { - if (!(req == rhs.req)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_foreign_key_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_foreign_key_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_add_foreign_key_pargs { - public: - - - virtual ~ThriftHiveMetastore_add_foreign_key_pargs() noexcept; - const AddForeignKeyRequest* req; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_foreign_key_result__isset { - _ThriftHiveMetastore_add_foreign_key_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_add_foreign_key_result__isset; - -class ThriftHiveMetastore_add_foreign_key_result { - public: - - ThriftHiveMetastore_add_foreign_key_result(const ThriftHiveMetastore_add_foreign_key_result&); - ThriftHiveMetastore_add_foreign_key_result& operator=(const ThriftHiveMetastore_add_foreign_key_result&); - ThriftHiveMetastore_add_foreign_key_result() { - } - - virtual ~ThriftHiveMetastore_add_foreign_key_result() noexcept; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_add_foreign_key_result__isset __isset; - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_add_foreign_key_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_foreign_key_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_foreign_key_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_foreign_key_presult__isset { - _ThriftHiveMetastore_add_foreign_key_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_add_foreign_key_presult__isset; - -class ThriftHiveMetastore_add_foreign_key_presult { - public: - - - virtual ~ThriftHiveMetastore_add_foreign_key_presult() noexcept; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_add_foreign_key_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_add_unique_constraint_args__isset { - _ThriftHiveMetastore_add_unique_constraint_args__isset() : req(false) {} - bool req :1; -} _ThriftHiveMetastore_add_unique_constraint_args__isset; - -class ThriftHiveMetastore_add_unique_constraint_args { - public: - - ThriftHiveMetastore_add_unique_constraint_args(const ThriftHiveMetastore_add_unique_constraint_args&); - ThriftHiveMetastore_add_unique_constraint_args& operator=(const ThriftHiveMetastore_add_unique_constraint_args&); - ThriftHiveMetastore_add_unique_constraint_args() { - } - - virtual ~ThriftHiveMetastore_add_unique_constraint_args() noexcept; - AddUniqueConstraintRequest req; - - _ThriftHiveMetastore_add_unique_constraint_args__isset __isset; - - void __set_req(const AddUniqueConstraintRequest& val); - - bool operator == (const ThriftHiveMetastore_add_unique_constraint_args & rhs) const - { - if (!(req == rhs.req)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_unique_constraint_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_unique_constraint_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_add_unique_constraint_pargs { - public: - - - virtual ~ThriftHiveMetastore_add_unique_constraint_pargs() noexcept; - const AddUniqueConstraintRequest* req; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_unique_constraint_result__isset { - _ThriftHiveMetastore_add_unique_constraint_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_add_unique_constraint_result__isset; - -class ThriftHiveMetastore_add_unique_constraint_result { - public: - - ThriftHiveMetastore_add_unique_constraint_result(const ThriftHiveMetastore_add_unique_constraint_result&); - ThriftHiveMetastore_add_unique_constraint_result& operator=(const ThriftHiveMetastore_add_unique_constraint_result&); - ThriftHiveMetastore_add_unique_constraint_result() { - } - - virtual ~ThriftHiveMetastore_add_unique_constraint_result() noexcept; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_add_unique_constraint_result__isset __isset; - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_add_unique_constraint_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_unique_constraint_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_unique_constraint_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_unique_constraint_presult__isset { - _ThriftHiveMetastore_add_unique_constraint_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_add_unique_constraint_presult__isset; - -class ThriftHiveMetastore_add_unique_constraint_presult { - public: - - - virtual ~ThriftHiveMetastore_add_unique_constraint_presult() noexcept; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_add_unique_constraint_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_add_not_null_constraint_args__isset { - _ThriftHiveMetastore_add_not_null_constraint_args__isset() : req(false) {} - bool req :1; -} _ThriftHiveMetastore_add_not_null_constraint_args__isset; - -class ThriftHiveMetastore_add_not_null_constraint_args { - public: - - ThriftHiveMetastore_add_not_null_constraint_args(const ThriftHiveMetastore_add_not_null_constraint_args&); - ThriftHiveMetastore_add_not_null_constraint_args& operator=(const ThriftHiveMetastore_add_not_null_constraint_args&); - ThriftHiveMetastore_add_not_null_constraint_args() { - } - - virtual ~ThriftHiveMetastore_add_not_null_constraint_args() noexcept; - AddNotNullConstraintRequest req; - - _ThriftHiveMetastore_add_not_null_constraint_args__isset __isset; - - void __set_req(const AddNotNullConstraintRequest& val); - - bool operator == (const ThriftHiveMetastore_add_not_null_constraint_args & rhs) const - { - if (!(req == rhs.req)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_not_null_constraint_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_not_null_constraint_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_add_not_null_constraint_pargs { - public: - - - virtual ~ThriftHiveMetastore_add_not_null_constraint_pargs() noexcept; - const AddNotNullConstraintRequest* req; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_not_null_constraint_result__isset { - _ThriftHiveMetastore_add_not_null_constraint_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_add_not_null_constraint_result__isset; - -class ThriftHiveMetastore_add_not_null_constraint_result { - public: - - ThriftHiveMetastore_add_not_null_constraint_result(const ThriftHiveMetastore_add_not_null_constraint_result&); - ThriftHiveMetastore_add_not_null_constraint_result& operator=(const ThriftHiveMetastore_add_not_null_constraint_result&); - ThriftHiveMetastore_add_not_null_constraint_result() { - } - - virtual ~ThriftHiveMetastore_add_not_null_constraint_result() noexcept; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_add_not_null_constraint_result__isset __isset; - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_add_not_null_constraint_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_not_null_constraint_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_not_null_constraint_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_not_null_constraint_presult__isset { - _ThriftHiveMetastore_add_not_null_constraint_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_add_not_null_constraint_presult__isset; - -class ThriftHiveMetastore_add_not_null_constraint_presult { - public: - - - virtual ~ThriftHiveMetastore_add_not_null_constraint_presult() noexcept; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_add_not_null_constraint_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_drop_table_args__isset { - _ThriftHiveMetastore_drop_table_args__isset() : dbname(false), name(false), deleteData(false) {} - bool dbname :1; - bool name :1; - bool deleteData :1; -} _ThriftHiveMetastore_drop_table_args__isset; - -class ThriftHiveMetastore_drop_table_args { - public: - - ThriftHiveMetastore_drop_table_args(const ThriftHiveMetastore_drop_table_args&); - ThriftHiveMetastore_drop_table_args& operator=(const ThriftHiveMetastore_drop_table_args&); - ThriftHiveMetastore_drop_table_args() : dbname(), name(), deleteData(0) { - } - - virtual ~ThriftHiveMetastore_drop_table_args() noexcept; - std::string dbname; - std::string name; - bool deleteData; - - _ThriftHiveMetastore_drop_table_args__isset __isset; - - void __set_dbname(const std::string& val); - - void __set_name(const std::string& val); - - void __set_deleteData(const bool val); - - bool operator == (const ThriftHiveMetastore_drop_table_args & rhs) const - { - if (!(dbname == rhs.dbname)) - return false; - if (!(name == rhs.name)) - return false; - if (!(deleteData == rhs.deleteData)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_table_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_table_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_drop_table_pargs { - public: - - - virtual ~ThriftHiveMetastore_drop_table_pargs() noexcept; - const std::string* dbname; - const std::string* name; - const bool* deleteData; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_table_result__isset { - _ThriftHiveMetastore_drop_table_result__isset() : o1(false), o3(false) {} - bool o1 :1; - bool o3 :1; -} _ThriftHiveMetastore_drop_table_result__isset; - -class ThriftHiveMetastore_drop_table_result { - public: - - ThriftHiveMetastore_drop_table_result(const ThriftHiveMetastore_drop_table_result&); - ThriftHiveMetastore_drop_table_result& operator=(const ThriftHiveMetastore_drop_table_result&); - ThriftHiveMetastore_drop_table_result() { - } - - virtual ~ThriftHiveMetastore_drop_table_result() noexcept; - NoSuchObjectException o1; - MetaException o3; - - _ThriftHiveMetastore_drop_table_result__isset __isset; - - void __set_o1(const NoSuchObjectException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_drop_table_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_table_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_table_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_table_presult__isset { - _ThriftHiveMetastore_drop_table_presult__isset() : o1(false), o3(false) {} - bool o1 :1; - bool o3 :1; -} _ThriftHiveMetastore_drop_table_presult__isset; - -class ThriftHiveMetastore_drop_table_presult { - public: - - - virtual ~ThriftHiveMetastore_drop_table_presult() noexcept; - NoSuchObjectException o1; - MetaException o3; - - _ThriftHiveMetastore_drop_table_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_drop_table_with_environment_context_args__isset { - _ThriftHiveMetastore_drop_table_with_environment_context_args__isset() : dbname(false), name(false), deleteData(false), environment_context(false) {} - bool dbname :1; - bool name :1; - bool deleteData :1; - bool environment_context :1; -} _ThriftHiveMetastore_drop_table_with_environment_context_args__isset; - -class ThriftHiveMetastore_drop_table_with_environment_context_args { - public: - - ThriftHiveMetastore_drop_table_with_environment_context_args(const ThriftHiveMetastore_drop_table_with_environment_context_args&); - ThriftHiveMetastore_drop_table_with_environment_context_args& operator=(const ThriftHiveMetastore_drop_table_with_environment_context_args&); - ThriftHiveMetastore_drop_table_with_environment_context_args() : dbname(), name(), deleteData(0) { - } - - virtual ~ThriftHiveMetastore_drop_table_with_environment_context_args() noexcept; - std::string dbname; - std::string name; - bool deleteData; - EnvironmentContext environment_context; - - _ThriftHiveMetastore_drop_table_with_environment_context_args__isset __isset; - - void __set_dbname(const std::string& val); - - void __set_name(const std::string& val); - - void __set_deleteData(const bool val); - - void __set_environment_context(const EnvironmentContext& val); - - bool operator == (const ThriftHiveMetastore_drop_table_with_environment_context_args & rhs) const - { - if (!(dbname == rhs.dbname)) - return false; - if (!(name == rhs.name)) - return false; - if (!(deleteData == rhs.deleteData)) - return false; - if (!(environment_context == rhs.environment_context)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_table_with_environment_context_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_table_with_environment_context_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_drop_table_with_environment_context_pargs { - public: - - - virtual ~ThriftHiveMetastore_drop_table_with_environment_context_pargs() noexcept; - const std::string* dbname; - const std::string* name; - const bool* deleteData; - const EnvironmentContext* environment_context; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_table_with_environment_context_result__isset { - _ThriftHiveMetastore_drop_table_with_environment_context_result__isset() : o1(false), o3(false) {} - bool o1 :1; - bool o3 :1; -} _ThriftHiveMetastore_drop_table_with_environment_context_result__isset; - -class ThriftHiveMetastore_drop_table_with_environment_context_result { - public: - - ThriftHiveMetastore_drop_table_with_environment_context_result(const ThriftHiveMetastore_drop_table_with_environment_context_result&); - ThriftHiveMetastore_drop_table_with_environment_context_result& operator=(const ThriftHiveMetastore_drop_table_with_environment_context_result&); - ThriftHiveMetastore_drop_table_with_environment_context_result() { - } - - virtual ~ThriftHiveMetastore_drop_table_with_environment_context_result() noexcept; - NoSuchObjectException o1; - MetaException o3; - - _ThriftHiveMetastore_drop_table_with_environment_context_result__isset __isset; - - void __set_o1(const NoSuchObjectException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_drop_table_with_environment_context_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_table_with_environment_context_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_table_with_environment_context_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_table_with_environment_context_presult__isset { - _ThriftHiveMetastore_drop_table_with_environment_context_presult__isset() : o1(false), o3(false) {} - bool o1 :1; - bool o3 :1; -} _ThriftHiveMetastore_drop_table_with_environment_context_presult__isset; - -class ThriftHiveMetastore_drop_table_with_environment_context_presult { - public: - - - virtual ~ThriftHiveMetastore_drop_table_with_environment_context_presult() noexcept; - NoSuchObjectException o1; - MetaException o3; - - _ThriftHiveMetastore_drop_table_with_environment_context_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_truncate_table_args__isset { - _ThriftHiveMetastore_truncate_table_args__isset() : dbName(false), tableName(false), partNames(false) {} - bool dbName :1; - bool tableName :1; - bool partNames :1; -} _ThriftHiveMetastore_truncate_table_args__isset; - -class ThriftHiveMetastore_truncate_table_args { - public: - - ThriftHiveMetastore_truncate_table_args(const ThriftHiveMetastore_truncate_table_args&); - ThriftHiveMetastore_truncate_table_args& operator=(const ThriftHiveMetastore_truncate_table_args&); - ThriftHiveMetastore_truncate_table_args() : dbName(), tableName() { - } - - virtual ~ThriftHiveMetastore_truncate_table_args() noexcept; - std::string dbName; - std::string tableName; - std::vector partNames; - - _ThriftHiveMetastore_truncate_table_args__isset __isset; - - void __set_dbName(const std::string& val); - - void __set_tableName(const std::string& val); - - void __set_partNames(const std::vector & val); - - bool operator == (const ThriftHiveMetastore_truncate_table_args & rhs) const - { - if (!(dbName == rhs.dbName)) - return false; - if (!(tableName == rhs.tableName)) - return false; - if (!(partNames == rhs.partNames)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_truncate_table_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_truncate_table_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_truncate_table_pargs { - public: - - - virtual ~ThriftHiveMetastore_truncate_table_pargs() noexcept; - const std::string* dbName; - const std::string* tableName; - const std::vector * partNames; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_truncate_table_result__isset { - _ThriftHiveMetastore_truncate_table_result__isset() : o1(false) {} - bool o1 :1; -} _ThriftHiveMetastore_truncate_table_result__isset; - -class ThriftHiveMetastore_truncate_table_result { - public: - - ThriftHiveMetastore_truncate_table_result(const ThriftHiveMetastore_truncate_table_result&); - ThriftHiveMetastore_truncate_table_result& operator=(const ThriftHiveMetastore_truncate_table_result&); - ThriftHiveMetastore_truncate_table_result() { - } - - virtual ~ThriftHiveMetastore_truncate_table_result() noexcept; - MetaException o1; - - _ThriftHiveMetastore_truncate_table_result__isset __isset; - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_truncate_table_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_truncate_table_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_truncate_table_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_truncate_table_presult__isset { - _ThriftHiveMetastore_truncate_table_presult__isset() : o1(false) {} - bool o1 :1; -} _ThriftHiveMetastore_truncate_table_presult__isset; - -class ThriftHiveMetastore_truncate_table_presult { - public: - - - virtual ~ThriftHiveMetastore_truncate_table_presult() noexcept; - MetaException o1; - - _ThriftHiveMetastore_truncate_table_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_tables_args__isset { - _ThriftHiveMetastore_get_tables_args__isset() : db_name(false), pattern(false) {} - bool db_name :1; - bool pattern :1; -} _ThriftHiveMetastore_get_tables_args__isset; - -class ThriftHiveMetastore_get_tables_args { - public: - - ThriftHiveMetastore_get_tables_args(const ThriftHiveMetastore_get_tables_args&); - ThriftHiveMetastore_get_tables_args& operator=(const ThriftHiveMetastore_get_tables_args&); - ThriftHiveMetastore_get_tables_args() : db_name(), pattern() { - } - - virtual ~ThriftHiveMetastore_get_tables_args() noexcept; - std::string db_name; - std::string pattern; - - _ThriftHiveMetastore_get_tables_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_pattern(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_tables_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(pattern == rhs.pattern)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_tables_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_tables_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_tables_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_tables_pargs() noexcept; - const std::string* db_name; - const std::string* pattern; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_tables_result__isset { - _ThriftHiveMetastore_get_tables_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_tables_result__isset; - -class ThriftHiveMetastore_get_tables_result { - public: - - ThriftHiveMetastore_get_tables_result(const ThriftHiveMetastore_get_tables_result&); - ThriftHiveMetastore_get_tables_result& operator=(const ThriftHiveMetastore_get_tables_result&); - ThriftHiveMetastore_get_tables_result() { - } - - virtual ~ThriftHiveMetastore_get_tables_result() noexcept; - std::vector success; - MetaException o1; - - _ThriftHiveMetastore_get_tables_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_tables_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_tables_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_tables_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_tables_presult__isset { - _ThriftHiveMetastore_get_tables_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_tables_presult__isset; - -class ThriftHiveMetastore_get_tables_presult { - public: - - - virtual ~ThriftHiveMetastore_get_tables_presult() noexcept; - std::vector * success; - MetaException o1; - - _ThriftHiveMetastore_get_tables_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_tables_by_type_args__isset { - _ThriftHiveMetastore_get_tables_by_type_args__isset() : db_name(false), pattern(false), tableType(false) {} - bool db_name :1; - bool pattern :1; - bool tableType :1; -} _ThriftHiveMetastore_get_tables_by_type_args__isset; - -class ThriftHiveMetastore_get_tables_by_type_args { - public: - - ThriftHiveMetastore_get_tables_by_type_args(const ThriftHiveMetastore_get_tables_by_type_args&); - ThriftHiveMetastore_get_tables_by_type_args& operator=(const ThriftHiveMetastore_get_tables_by_type_args&); - ThriftHiveMetastore_get_tables_by_type_args() : db_name(), pattern(), tableType() { - } - - virtual ~ThriftHiveMetastore_get_tables_by_type_args() noexcept; - std::string db_name; - std::string pattern; - std::string tableType; - - _ThriftHiveMetastore_get_tables_by_type_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_pattern(const std::string& val); - - void __set_tableType(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_tables_by_type_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(pattern == rhs.pattern)) - return false; - if (!(tableType == rhs.tableType)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_tables_by_type_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_tables_by_type_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_tables_by_type_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_tables_by_type_pargs() noexcept; - const std::string* db_name; - const std::string* pattern; - const std::string* tableType; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_tables_by_type_result__isset { - _ThriftHiveMetastore_get_tables_by_type_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_tables_by_type_result__isset; - -class ThriftHiveMetastore_get_tables_by_type_result { - public: - - ThriftHiveMetastore_get_tables_by_type_result(const ThriftHiveMetastore_get_tables_by_type_result&); - ThriftHiveMetastore_get_tables_by_type_result& operator=(const ThriftHiveMetastore_get_tables_by_type_result&); - ThriftHiveMetastore_get_tables_by_type_result() { - } - - virtual ~ThriftHiveMetastore_get_tables_by_type_result() noexcept; - std::vector success; - MetaException o1; - - _ThriftHiveMetastore_get_tables_by_type_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_tables_by_type_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_tables_by_type_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_tables_by_type_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_tables_by_type_presult__isset { - _ThriftHiveMetastore_get_tables_by_type_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_tables_by_type_presult__isset; - -class ThriftHiveMetastore_get_tables_by_type_presult { - public: - - - virtual ~ThriftHiveMetastore_get_tables_by_type_presult() noexcept; - std::vector * success; - MetaException o1; - - _ThriftHiveMetastore_get_tables_by_type_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_table_meta_args__isset { - _ThriftHiveMetastore_get_table_meta_args__isset() : db_patterns(false), tbl_patterns(false), tbl_types(false) {} - bool db_patterns :1; - bool tbl_patterns :1; - bool tbl_types :1; -} _ThriftHiveMetastore_get_table_meta_args__isset; - -class ThriftHiveMetastore_get_table_meta_args { - public: - - ThriftHiveMetastore_get_table_meta_args(const ThriftHiveMetastore_get_table_meta_args&); - ThriftHiveMetastore_get_table_meta_args& operator=(const ThriftHiveMetastore_get_table_meta_args&); - ThriftHiveMetastore_get_table_meta_args() : db_patterns(), tbl_patterns() { - } - - virtual ~ThriftHiveMetastore_get_table_meta_args() noexcept; - std::string db_patterns; - std::string tbl_patterns; - std::vector tbl_types; - - _ThriftHiveMetastore_get_table_meta_args__isset __isset; - - void __set_db_patterns(const std::string& val); - - void __set_tbl_patterns(const std::string& val); - - void __set_tbl_types(const std::vector & val); - - bool operator == (const ThriftHiveMetastore_get_table_meta_args & rhs) const - { - if (!(db_patterns == rhs.db_patterns)) - return false; - if (!(tbl_patterns == rhs.tbl_patterns)) - return false; - if (!(tbl_types == rhs.tbl_types)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_table_meta_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_table_meta_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_table_meta_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_table_meta_pargs() noexcept; - const std::string* db_patterns; - const std::string* tbl_patterns; - const std::vector * tbl_types; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_table_meta_result__isset { - _ThriftHiveMetastore_get_table_meta_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_table_meta_result__isset; - -class ThriftHiveMetastore_get_table_meta_result { - public: - - ThriftHiveMetastore_get_table_meta_result(const ThriftHiveMetastore_get_table_meta_result&); - ThriftHiveMetastore_get_table_meta_result& operator=(const ThriftHiveMetastore_get_table_meta_result&); - ThriftHiveMetastore_get_table_meta_result() { - } - - virtual ~ThriftHiveMetastore_get_table_meta_result() noexcept; - std::vector success; - MetaException o1; - - _ThriftHiveMetastore_get_table_meta_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_table_meta_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_table_meta_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_table_meta_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_table_meta_presult__isset { - _ThriftHiveMetastore_get_table_meta_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_table_meta_presult__isset; - -class ThriftHiveMetastore_get_table_meta_presult { - public: - - - virtual ~ThriftHiveMetastore_get_table_meta_presult() noexcept; - std::vector * success; - MetaException o1; - - _ThriftHiveMetastore_get_table_meta_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_all_tables_args__isset { - _ThriftHiveMetastore_get_all_tables_args__isset() : db_name(false) {} - bool db_name :1; -} _ThriftHiveMetastore_get_all_tables_args__isset; - -class ThriftHiveMetastore_get_all_tables_args { - public: - - ThriftHiveMetastore_get_all_tables_args(const ThriftHiveMetastore_get_all_tables_args&); - ThriftHiveMetastore_get_all_tables_args& operator=(const ThriftHiveMetastore_get_all_tables_args&); - ThriftHiveMetastore_get_all_tables_args() : db_name() { - } - - virtual ~ThriftHiveMetastore_get_all_tables_args() noexcept; - std::string db_name; - - _ThriftHiveMetastore_get_all_tables_args__isset __isset; - - void __set_db_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_all_tables_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_all_tables_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_all_tables_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_all_tables_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_all_tables_pargs() noexcept; - const std::string* db_name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_all_tables_result__isset { - _ThriftHiveMetastore_get_all_tables_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_all_tables_result__isset; - -class ThriftHiveMetastore_get_all_tables_result { - public: - - ThriftHiveMetastore_get_all_tables_result(const ThriftHiveMetastore_get_all_tables_result&); - ThriftHiveMetastore_get_all_tables_result& operator=(const ThriftHiveMetastore_get_all_tables_result&); - ThriftHiveMetastore_get_all_tables_result() { - } - - virtual ~ThriftHiveMetastore_get_all_tables_result() noexcept; - std::vector success; - MetaException o1; - - _ThriftHiveMetastore_get_all_tables_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_all_tables_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_all_tables_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_all_tables_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_all_tables_presult__isset { - _ThriftHiveMetastore_get_all_tables_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_all_tables_presult__isset; - -class ThriftHiveMetastore_get_all_tables_presult { - public: - - - virtual ~ThriftHiveMetastore_get_all_tables_presult() noexcept; - std::vector * success; - MetaException o1; - - _ThriftHiveMetastore_get_all_tables_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_table_args__isset { - _ThriftHiveMetastore_get_table_args__isset() : dbname(false), tbl_name(false) {} - bool dbname :1; - bool tbl_name :1; -} _ThriftHiveMetastore_get_table_args__isset; - -class ThriftHiveMetastore_get_table_args { - public: - - ThriftHiveMetastore_get_table_args(const ThriftHiveMetastore_get_table_args&); - ThriftHiveMetastore_get_table_args& operator=(const ThriftHiveMetastore_get_table_args&); - ThriftHiveMetastore_get_table_args() : dbname(), tbl_name() { - } - - virtual ~ThriftHiveMetastore_get_table_args() noexcept; - std::string dbname; - std::string tbl_name; - - _ThriftHiveMetastore_get_table_args__isset __isset; - - void __set_dbname(const std::string& val); - - void __set_tbl_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_table_args & rhs) const - { - if (!(dbname == rhs.dbname)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_table_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_table_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_table_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_table_pargs() noexcept; - const std::string* dbname; - const std::string* tbl_name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_table_result__isset { - _ThriftHiveMetastore_get_table_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_table_result__isset; - -class ThriftHiveMetastore_get_table_result { - public: - - ThriftHiveMetastore_get_table_result(const ThriftHiveMetastore_get_table_result&); - ThriftHiveMetastore_get_table_result& operator=(const ThriftHiveMetastore_get_table_result&); - ThriftHiveMetastore_get_table_result() { - } - - virtual ~ThriftHiveMetastore_get_table_result() noexcept; - Table success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_table_result__isset __isset; - - void __set_success(const Table& val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_table_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_table_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_table_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_table_presult__isset { - _ThriftHiveMetastore_get_table_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_table_presult__isset; - -class ThriftHiveMetastore_get_table_presult { - public: - - - virtual ~ThriftHiveMetastore_get_table_presult() noexcept; - Table* success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_table_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_table_objects_by_name_args__isset { - _ThriftHiveMetastore_get_table_objects_by_name_args__isset() : dbname(false), tbl_names(false) {} - bool dbname :1; - bool tbl_names :1; -} _ThriftHiveMetastore_get_table_objects_by_name_args__isset; - -class ThriftHiveMetastore_get_table_objects_by_name_args { - public: - - ThriftHiveMetastore_get_table_objects_by_name_args(const ThriftHiveMetastore_get_table_objects_by_name_args&); - ThriftHiveMetastore_get_table_objects_by_name_args& operator=(const ThriftHiveMetastore_get_table_objects_by_name_args&); - ThriftHiveMetastore_get_table_objects_by_name_args() : dbname() { - } - - virtual ~ThriftHiveMetastore_get_table_objects_by_name_args() noexcept; - std::string dbname; - std::vector tbl_names; - - _ThriftHiveMetastore_get_table_objects_by_name_args__isset __isset; - - void __set_dbname(const std::string& val); - - void __set_tbl_names(const std::vector & val); - - bool operator == (const ThriftHiveMetastore_get_table_objects_by_name_args & rhs) const - { - if (!(dbname == rhs.dbname)) - return false; - if (!(tbl_names == rhs.tbl_names)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_table_objects_by_name_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_table_objects_by_name_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_table_objects_by_name_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_table_objects_by_name_pargs() noexcept; - const std::string* dbname; - const std::vector * tbl_names; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_table_objects_by_name_result__isset { - _ThriftHiveMetastore_get_table_objects_by_name_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_table_objects_by_name_result__isset; - -class ThriftHiveMetastore_get_table_objects_by_name_result { - public: - - ThriftHiveMetastore_get_table_objects_by_name_result(const ThriftHiveMetastore_get_table_objects_by_name_result&); - ThriftHiveMetastore_get_table_objects_by_name_result& operator=(const ThriftHiveMetastore_get_table_objects_by_name_result&); - ThriftHiveMetastore_get_table_objects_by_name_result() { - } - - virtual ~ThriftHiveMetastore_get_table_objects_by_name_result() noexcept; - std::vector
success; - - _ThriftHiveMetastore_get_table_objects_by_name_result__isset __isset; - - void __set_success(const std::vector
& val); - - bool operator == (const ThriftHiveMetastore_get_table_objects_by_name_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_table_objects_by_name_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_table_objects_by_name_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_table_objects_by_name_presult__isset { - _ThriftHiveMetastore_get_table_objects_by_name_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_table_objects_by_name_presult__isset; - -class ThriftHiveMetastore_get_table_objects_by_name_presult { - public: - - - virtual ~ThriftHiveMetastore_get_table_objects_by_name_presult() noexcept; - std::vector
* success; - - _ThriftHiveMetastore_get_table_objects_by_name_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_table_req_args__isset { - _ThriftHiveMetastore_get_table_req_args__isset() : req(false) {} - bool req :1; -} _ThriftHiveMetastore_get_table_req_args__isset; - -class ThriftHiveMetastore_get_table_req_args { - public: - - ThriftHiveMetastore_get_table_req_args(const ThriftHiveMetastore_get_table_req_args&); - ThriftHiveMetastore_get_table_req_args& operator=(const ThriftHiveMetastore_get_table_req_args&); - ThriftHiveMetastore_get_table_req_args() { - } - - virtual ~ThriftHiveMetastore_get_table_req_args() noexcept; - GetTableRequest req; - - _ThriftHiveMetastore_get_table_req_args__isset __isset; - - void __set_req(const GetTableRequest& val); - - bool operator == (const ThriftHiveMetastore_get_table_req_args & rhs) const - { - if (!(req == rhs.req)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_table_req_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_table_req_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_table_req_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_table_req_pargs() noexcept; - const GetTableRequest* req; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_table_req_result__isset { - _ThriftHiveMetastore_get_table_req_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_table_req_result__isset; - -class ThriftHiveMetastore_get_table_req_result { - public: - - ThriftHiveMetastore_get_table_req_result(const ThriftHiveMetastore_get_table_req_result&); - ThriftHiveMetastore_get_table_req_result& operator=(const ThriftHiveMetastore_get_table_req_result&); - ThriftHiveMetastore_get_table_req_result() { - } - - virtual ~ThriftHiveMetastore_get_table_req_result() noexcept; - GetTableResult success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_table_req_result__isset __isset; - - void __set_success(const GetTableResult& val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_table_req_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_table_req_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_table_req_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_table_req_presult__isset { - _ThriftHiveMetastore_get_table_req_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_table_req_presult__isset; - -class ThriftHiveMetastore_get_table_req_presult { - public: - - - virtual ~ThriftHiveMetastore_get_table_req_presult() noexcept; - GetTableResult* success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_table_req_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_table_objects_by_name_req_args__isset { - _ThriftHiveMetastore_get_table_objects_by_name_req_args__isset() : req(false) {} - bool req :1; -} _ThriftHiveMetastore_get_table_objects_by_name_req_args__isset; - -class ThriftHiveMetastore_get_table_objects_by_name_req_args { - public: - - ThriftHiveMetastore_get_table_objects_by_name_req_args(const ThriftHiveMetastore_get_table_objects_by_name_req_args&); - ThriftHiveMetastore_get_table_objects_by_name_req_args& operator=(const ThriftHiveMetastore_get_table_objects_by_name_req_args&); - ThriftHiveMetastore_get_table_objects_by_name_req_args() { - } - - virtual ~ThriftHiveMetastore_get_table_objects_by_name_req_args() noexcept; - GetTablesRequest req; - - _ThriftHiveMetastore_get_table_objects_by_name_req_args__isset __isset; - - void __set_req(const GetTablesRequest& val); - - bool operator == (const ThriftHiveMetastore_get_table_objects_by_name_req_args & rhs) const - { - if (!(req == rhs.req)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_table_objects_by_name_req_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_table_objects_by_name_req_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_table_objects_by_name_req_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_table_objects_by_name_req_pargs() noexcept; - const GetTablesRequest* req; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_table_objects_by_name_req_result__isset { - _ThriftHiveMetastore_get_table_objects_by_name_req_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_get_table_objects_by_name_req_result__isset; - -class ThriftHiveMetastore_get_table_objects_by_name_req_result { - public: - - ThriftHiveMetastore_get_table_objects_by_name_req_result(const ThriftHiveMetastore_get_table_objects_by_name_req_result&); - ThriftHiveMetastore_get_table_objects_by_name_req_result& operator=(const ThriftHiveMetastore_get_table_objects_by_name_req_result&); - ThriftHiveMetastore_get_table_objects_by_name_req_result() { - } - - virtual ~ThriftHiveMetastore_get_table_objects_by_name_req_result() noexcept; - GetTablesResult success; - MetaException o1; - InvalidOperationException o2; - UnknownDBException o3; - - _ThriftHiveMetastore_get_table_objects_by_name_req_result__isset __isset; - - void __set_success(const GetTablesResult& val); - - void __set_o1(const MetaException& val); - - void __set_o2(const InvalidOperationException& val); - - void __set_o3(const UnknownDBException& val); - - bool operator == (const ThriftHiveMetastore_get_table_objects_by_name_req_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_table_objects_by_name_req_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_table_objects_by_name_req_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_table_objects_by_name_req_presult__isset { - _ThriftHiveMetastore_get_table_objects_by_name_req_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_get_table_objects_by_name_req_presult__isset; - -class ThriftHiveMetastore_get_table_objects_by_name_req_presult { - public: - - - virtual ~ThriftHiveMetastore_get_table_objects_by_name_req_presult() noexcept; - GetTablesResult* success; - MetaException o1; - InvalidOperationException o2; - UnknownDBException o3; - - _ThriftHiveMetastore_get_table_objects_by_name_req_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_table_names_by_filter_args__isset { - _ThriftHiveMetastore_get_table_names_by_filter_args__isset() : dbname(false), filter(false), max_tables(true) {} - bool dbname :1; - bool filter :1; - bool max_tables :1; -} _ThriftHiveMetastore_get_table_names_by_filter_args__isset; - -class ThriftHiveMetastore_get_table_names_by_filter_args { - public: - - ThriftHiveMetastore_get_table_names_by_filter_args(const ThriftHiveMetastore_get_table_names_by_filter_args&); - ThriftHiveMetastore_get_table_names_by_filter_args& operator=(const ThriftHiveMetastore_get_table_names_by_filter_args&); - ThriftHiveMetastore_get_table_names_by_filter_args() : dbname(), filter(), max_tables(-1) { - } - - virtual ~ThriftHiveMetastore_get_table_names_by_filter_args() noexcept; - std::string dbname; - std::string filter; - int16_t max_tables; - - _ThriftHiveMetastore_get_table_names_by_filter_args__isset __isset; - - void __set_dbname(const std::string& val); - - void __set_filter(const std::string& val); - - void __set_max_tables(const int16_t val); - - bool operator == (const ThriftHiveMetastore_get_table_names_by_filter_args & rhs) const - { - if (!(dbname == rhs.dbname)) - return false; - if (!(filter == rhs.filter)) - return false; - if (!(max_tables == rhs.max_tables)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_table_names_by_filter_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_table_names_by_filter_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_table_names_by_filter_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_table_names_by_filter_pargs() noexcept; - const std::string* dbname; - const std::string* filter; - const int16_t* max_tables; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_table_names_by_filter_result__isset { - _ThriftHiveMetastore_get_table_names_by_filter_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_get_table_names_by_filter_result__isset; - -class ThriftHiveMetastore_get_table_names_by_filter_result { - public: - - ThriftHiveMetastore_get_table_names_by_filter_result(const ThriftHiveMetastore_get_table_names_by_filter_result&); - ThriftHiveMetastore_get_table_names_by_filter_result& operator=(const ThriftHiveMetastore_get_table_names_by_filter_result&); - ThriftHiveMetastore_get_table_names_by_filter_result() { - } - - virtual ~ThriftHiveMetastore_get_table_names_by_filter_result() noexcept; - std::vector success; - MetaException o1; - InvalidOperationException o2; - UnknownDBException o3; - - _ThriftHiveMetastore_get_table_names_by_filter_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - void __set_o2(const InvalidOperationException& val); - - void __set_o3(const UnknownDBException& val); - - bool operator == (const ThriftHiveMetastore_get_table_names_by_filter_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_table_names_by_filter_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_table_names_by_filter_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_table_names_by_filter_presult__isset { - _ThriftHiveMetastore_get_table_names_by_filter_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_get_table_names_by_filter_presult__isset; - -class ThriftHiveMetastore_get_table_names_by_filter_presult { - public: - - - virtual ~ThriftHiveMetastore_get_table_names_by_filter_presult() noexcept; - std::vector * success; - MetaException o1; - InvalidOperationException o2; - UnknownDBException o3; - - _ThriftHiveMetastore_get_table_names_by_filter_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_alter_table_args__isset { - _ThriftHiveMetastore_alter_table_args__isset() : dbname(false), tbl_name(false), new_tbl(false) {} - bool dbname :1; - bool tbl_name :1; - bool new_tbl :1; -} _ThriftHiveMetastore_alter_table_args__isset; - -class ThriftHiveMetastore_alter_table_args { - public: - - ThriftHiveMetastore_alter_table_args(const ThriftHiveMetastore_alter_table_args&); - ThriftHiveMetastore_alter_table_args& operator=(const ThriftHiveMetastore_alter_table_args&); - ThriftHiveMetastore_alter_table_args() : dbname(), tbl_name() { - } - - virtual ~ThriftHiveMetastore_alter_table_args() noexcept; - std::string dbname; - std::string tbl_name; - Table new_tbl; - - _ThriftHiveMetastore_alter_table_args__isset __isset; - - void __set_dbname(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_new_tbl(const Table& val); - - bool operator == (const ThriftHiveMetastore_alter_table_args & rhs) const - { - if (!(dbname == rhs.dbname)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(new_tbl == rhs.new_tbl)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_table_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_table_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_alter_table_pargs { - public: - - - virtual ~ThriftHiveMetastore_alter_table_pargs() noexcept; - const std::string* dbname; - const std::string* tbl_name; - const Table* new_tbl; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_table_result__isset { - _ThriftHiveMetastore_alter_table_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_table_result__isset; - -class ThriftHiveMetastore_alter_table_result { - public: - - ThriftHiveMetastore_alter_table_result(const ThriftHiveMetastore_alter_table_result&); - ThriftHiveMetastore_alter_table_result& operator=(const ThriftHiveMetastore_alter_table_result&); - ThriftHiveMetastore_alter_table_result() { - } - - virtual ~ThriftHiveMetastore_alter_table_result() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_table_result__isset __isset; - - void __set_o1(const InvalidOperationException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_alter_table_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_table_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_table_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_table_presult__isset { - _ThriftHiveMetastore_alter_table_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_table_presult__isset; - -class ThriftHiveMetastore_alter_table_presult { - public: - - - virtual ~ThriftHiveMetastore_alter_table_presult() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_table_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_alter_table_with_environment_context_args__isset { - _ThriftHiveMetastore_alter_table_with_environment_context_args__isset() : dbname(false), tbl_name(false), new_tbl(false), environment_context(false) {} - bool dbname :1; - bool tbl_name :1; - bool new_tbl :1; - bool environment_context :1; -} _ThriftHiveMetastore_alter_table_with_environment_context_args__isset; - -class ThriftHiveMetastore_alter_table_with_environment_context_args { - public: - - ThriftHiveMetastore_alter_table_with_environment_context_args(const ThriftHiveMetastore_alter_table_with_environment_context_args&); - ThriftHiveMetastore_alter_table_with_environment_context_args& operator=(const ThriftHiveMetastore_alter_table_with_environment_context_args&); - ThriftHiveMetastore_alter_table_with_environment_context_args() : dbname(), tbl_name() { - } - - virtual ~ThriftHiveMetastore_alter_table_with_environment_context_args() noexcept; - std::string dbname; - std::string tbl_name; - Table new_tbl; - EnvironmentContext environment_context; - - _ThriftHiveMetastore_alter_table_with_environment_context_args__isset __isset; - - void __set_dbname(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_new_tbl(const Table& val); - - void __set_environment_context(const EnvironmentContext& val); - - bool operator == (const ThriftHiveMetastore_alter_table_with_environment_context_args & rhs) const - { - if (!(dbname == rhs.dbname)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(new_tbl == rhs.new_tbl)) - return false; - if (!(environment_context == rhs.environment_context)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_table_with_environment_context_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_table_with_environment_context_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_alter_table_with_environment_context_pargs { - public: - - - virtual ~ThriftHiveMetastore_alter_table_with_environment_context_pargs() noexcept; - const std::string* dbname; - const std::string* tbl_name; - const Table* new_tbl; - const EnvironmentContext* environment_context; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_table_with_environment_context_result__isset { - _ThriftHiveMetastore_alter_table_with_environment_context_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_table_with_environment_context_result__isset; - -class ThriftHiveMetastore_alter_table_with_environment_context_result { - public: - - ThriftHiveMetastore_alter_table_with_environment_context_result(const ThriftHiveMetastore_alter_table_with_environment_context_result&); - ThriftHiveMetastore_alter_table_with_environment_context_result& operator=(const ThriftHiveMetastore_alter_table_with_environment_context_result&); - ThriftHiveMetastore_alter_table_with_environment_context_result() { - } - - virtual ~ThriftHiveMetastore_alter_table_with_environment_context_result() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_table_with_environment_context_result__isset __isset; - - void __set_o1(const InvalidOperationException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_alter_table_with_environment_context_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_table_with_environment_context_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_table_with_environment_context_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_table_with_environment_context_presult__isset { - _ThriftHiveMetastore_alter_table_with_environment_context_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_table_with_environment_context_presult__isset; - -class ThriftHiveMetastore_alter_table_with_environment_context_presult { - public: - - - virtual ~ThriftHiveMetastore_alter_table_with_environment_context_presult() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_table_with_environment_context_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_alter_table_with_cascade_args__isset { - _ThriftHiveMetastore_alter_table_with_cascade_args__isset() : dbname(false), tbl_name(false), new_tbl(false), cascade(false) {} - bool dbname :1; - bool tbl_name :1; - bool new_tbl :1; - bool cascade :1; -} _ThriftHiveMetastore_alter_table_with_cascade_args__isset; - -class ThriftHiveMetastore_alter_table_with_cascade_args { - public: - - ThriftHiveMetastore_alter_table_with_cascade_args(const ThriftHiveMetastore_alter_table_with_cascade_args&); - ThriftHiveMetastore_alter_table_with_cascade_args& operator=(const ThriftHiveMetastore_alter_table_with_cascade_args&); - ThriftHiveMetastore_alter_table_with_cascade_args() : dbname(), tbl_name(), cascade(0) { - } - - virtual ~ThriftHiveMetastore_alter_table_with_cascade_args() noexcept; - std::string dbname; - std::string tbl_name; - Table new_tbl; - bool cascade; - - _ThriftHiveMetastore_alter_table_with_cascade_args__isset __isset; - - void __set_dbname(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_new_tbl(const Table& val); - - void __set_cascade(const bool val); - - bool operator == (const ThriftHiveMetastore_alter_table_with_cascade_args & rhs) const - { - if (!(dbname == rhs.dbname)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(new_tbl == rhs.new_tbl)) - return false; - if (!(cascade == rhs.cascade)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_table_with_cascade_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_table_with_cascade_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_alter_table_with_cascade_pargs { - public: - - - virtual ~ThriftHiveMetastore_alter_table_with_cascade_pargs() noexcept; - const std::string* dbname; - const std::string* tbl_name; - const Table* new_tbl; - const bool* cascade; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_table_with_cascade_result__isset { - _ThriftHiveMetastore_alter_table_with_cascade_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_table_with_cascade_result__isset; - -class ThriftHiveMetastore_alter_table_with_cascade_result { - public: - - ThriftHiveMetastore_alter_table_with_cascade_result(const ThriftHiveMetastore_alter_table_with_cascade_result&); - ThriftHiveMetastore_alter_table_with_cascade_result& operator=(const ThriftHiveMetastore_alter_table_with_cascade_result&); - ThriftHiveMetastore_alter_table_with_cascade_result() { - } - - virtual ~ThriftHiveMetastore_alter_table_with_cascade_result() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_table_with_cascade_result__isset __isset; - - void __set_o1(const InvalidOperationException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_alter_table_with_cascade_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_table_with_cascade_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_table_with_cascade_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_table_with_cascade_presult__isset { - _ThriftHiveMetastore_alter_table_with_cascade_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_table_with_cascade_presult__isset; - -class ThriftHiveMetastore_alter_table_with_cascade_presult { - public: - - - virtual ~ThriftHiveMetastore_alter_table_with_cascade_presult() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_table_with_cascade_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_add_partition_args__isset { - _ThriftHiveMetastore_add_partition_args__isset() : new_part(false) {} - bool new_part :1; -} _ThriftHiveMetastore_add_partition_args__isset; - -class ThriftHiveMetastore_add_partition_args { - public: - - ThriftHiveMetastore_add_partition_args(const ThriftHiveMetastore_add_partition_args&); - ThriftHiveMetastore_add_partition_args& operator=(const ThriftHiveMetastore_add_partition_args&); - ThriftHiveMetastore_add_partition_args() { - } - - virtual ~ThriftHiveMetastore_add_partition_args() noexcept; - Partition new_part; - - _ThriftHiveMetastore_add_partition_args__isset __isset; - - void __set_new_part(const Partition& val); - - bool operator == (const ThriftHiveMetastore_add_partition_args & rhs) const - { - if (!(new_part == rhs.new_part)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_partition_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_partition_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_add_partition_pargs { - public: - - - virtual ~ThriftHiveMetastore_add_partition_pargs() noexcept; - const Partition* new_part; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_partition_result__isset { - _ThriftHiveMetastore_add_partition_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_add_partition_result__isset; - -class ThriftHiveMetastore_add_partition_result { - public: - - ThriftHiveMetastore_add_partition_result(const ThriftHiveMetastore_add_partition_result&); - ThriftHiveMetastore_add_partition_result& operator=(const ThriftHiveMetastore_add_partition_result&); - ThriftHiveMetastore_add_partition_result() { - } - - virtual ~ThriftHiveMetastore_add_partition_result() noexcept; - Partition success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_add_partition_result__isset __isset; - - void __set_success(const Partition& val); - - void __set_o1(const InvalidObjectException& val); - - void __set_o2(const AlreadyExistsException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_add_partition_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_partition_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_partition_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_partition_presult__isset { - _ThriftHiveMetastore_add_partition_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_add_partition_presult__isset; - -class ThriftHiveMetastore_add_partition_presult { - public: - - - virtual ~ThriftHiveMetastore_add_partition_presult() noexcept; - Partition* success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_add_partition_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_add_partition_with_environment_context_args__isset { - _ThriftHiveMetastore_add_partition_with_environment_context_args__isset() : new_part(false), environment_context(false) {} - bool new_part :1; - bool environment_context :1; -} _ThriftHiveMetastore_add_partition_with_environment_context_args__isset; - -class ThriftHiveMetastore_add_partition_with_environment_context_args { - public: - - ThriftHiveMetastore_add_partition_with_environment_context_args(const ThriftHiveMetastore_add_partition_with_environment_context_args&); - ThriftHiveMetastore_add_partition_with_environment_context_args& operator=(const ThriftHiveMetastore_add_partition_with_environment_context_args&); - ThriftHiveMetastore_add_partition_with_environment_context_args() { - } - - virtual ~ThriftHiveMetastore_add_partition_with_environment_context_args() noexcept; - Partition new_part; - EnvironmentContext environment_context; - - _ThriftHiveMetastore_add_partition_with_environment_context_args__isset __isset; - - void __set_new_part(const Partition& val); - - void __set_environment_context(const EnvironmentContext& val); - - bool operator == (const ThriftHiveMetastore_add_partition_with_environment_context_args & rhs) const - { - if (!(new_part == rhs.new_part)) - return false; - if (!(environment_context == rhs.environment_context)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_partition_with_environment_context_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_partition_with_environment_context_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_add_partition_with_environment_context_pargs { - public: - - - virtual ~ThriftHiveMetastore_add_partition_with_environment_context_pargs() noexcept; - const Partition* new_part; - const EnvironmentContext* environment_context; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_partition_with_environment_context_result__isset { - _ThriftHiveMetastore_add_partition_with_environment_context_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_add_partition_with_environment_context_result__isset; - -class ThriftHiveMetastore_add_partition_with_environment_context_result { - public: - - ThriftHiveMetastore_add_partition_with_environment_context_result(const ThriftHiveMetastore_add_partition_with_environment_context_result&); - ThriftHiveMetastore_add_partition_with_environment_context_result& operator=(const ThriftHiveMetastore_add_partition_with_environment_context_result&); - ThriftHiveMetastore_add_partition_with_environment_context_result() { - } - - virtual ~ThriftHiveMetastore_add_partition_with_environment_context_result() noexcept; - Partition success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_add_partition_with_environment_context_result__isset __isset; - - void __set_success(const Partition& val); - - void __set_o1(const InvalidObjectException& val); - - void __set_o2(const AlreadyExistsException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_add_partition_with_environment_context_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_partition_with_environment_context_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_partition_with_environment_context_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_partition_with_environment_context_presult__isset { - _ThriftHiveMetastore_add_partition_with_environment_context_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_add_partition_with_environment_context_presult__isset; - -class ThriftHiveMetastore_add_partition_with_environment_context_presult { - public: - - - virtual ~ThriftHiveMetastore_add_partition_with_environment_context_presult() noexcept; - Partition* success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_add_partition_with_environment_context_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_add_partitions_args__isset { - _ThriftHiveMetastore_add_partitions_args__isset() : new_parts(false) {} - bool new_parts :1; -} _ThriftHiveMetastore_add_partitions_args__isset; - -class ThriftHiveMetastore_add_partitions_args { - public: - - ThriftHiveMetastore_add_partitions_args(const ThriftHiveMetastore_add_partitions_args&); - ThriftHiveMetastore_add_partitions_args& operator=(const ThriftHiveMetastore_add_partitions_args&); - ThriftHiveMetastore_add_partitions_args() { - } - - virtual ~ThriftHiveMetastore_add_partitions_args() noexcept; - std::vector new_parts; - - _ThriftHiveMetastore_add_partitions_args__isset __isset; - - void __set_new_parts(const std::vector & val); - - bool operator == (const ThriftHiveMetastore_add_partitions_args & rhs) const - { - if (!(new_parts == rhs.new_parts)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_partitions_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_partitions_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_add_partitions_pargs { - public: - - - virtual ~ThriftHiveMetastore_add_partitions_pargs() noexcept; - const std::vector * new_parts; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_partitions_result__isset { - _ThriftHiveMetastore_add_partitions_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_add_partitions_result__isset; - -class ThriftHiveMetastore_add_partitions_result { - public: - - ThriftHiveMetastore_add_partitions_result(const ThriftHiveMetastore_add_partitions_result&); - ThriftHiveMetastore_add_partitions_result& operator=(const ThriftHiveMetastore_add_partitions_result&); - ThriftHiveMetastore_add_partitions_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_add_partitions_result() noexcept; - int32_t success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_add_partitions_result__isset __isset; - - void __set_success(const int32_t val); - - void __set_o1(const InvalidObjectException& val); - - void __set_o2(const AlreadyExistsException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_add_partitions_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_partitions_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_partitions_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_partitions_presult__isset { - _ThriftHiveMetastore_add_partitions_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_add_partitions_presult__isset; - -class ThriftHiveMetastore_add_partitions_presult { - public: - - - virtual ~ThriftHiveMetastore_add_partitions_presult() noexcept; - int32_t* success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_add_partitions_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_add_partitions_pspec_args__isset { - _ThriftHiveMetastore_add_partitions_pspec_args__isset() : new_parts(false) {} - bool new_parts :1; -} _ThriftHiveMetastore_add_partitions_pspec_args__isset; - -class ThriftHiveMetastore_add_partitions_pspec_args { - public: - - ThriftHiveMetastore_add_partitions_pspec_args(const ThriftHiveMetastore_add_partitions_pspec_args&); - ThriftHiveMetastore_add_partitions_pspec_args& operator=(const ThriftHiveMetastore_add_partitions_pspec_args&); - ThriftHiveMetastore_add_partitions_pspec_args() { - } - - virtual ~ThriftHiveMetastore_add_partitions_pspec_args() noexcept; - std::vector new_parts; - - _ThriftHiveMetastore_add_partitions_pspec_args__isset __isset; - - void __set_new_parts(const std::vector & val); - - bool operator == (const ThriftHiveMetastore_add_partitions_pspec_args & rhs) const - { - if (!(new_parts == rhs.new_parts)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_partitions_pspec_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_partitions_pspec_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_add_partitions_pspec_pargs { - public: - - - virtual ~ThriftHiveMetastore_add_partitions_pspec_pargs() noexcept; - const std::vector * new_parts; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_partitions_pspec_result__isset { - _ThriftHiveMetastore_add_partitions_pspec_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_add_partitions_pspec_result__isset; - -class ThriftHiveMetastore_add_partitions_pspec_result { - public: - - ThriftHiveMetastore_add_partitions_pspec_result(const ThriftHiveMetastore_add_partitions_pspec_result&); - ThriftHiveMetastore_add_partitions_pspec_result& operator=(const ThriftHiveMetastore_add_partitions_pspec_result&); - ThriftHiveMetastore_add_partitions_pspec_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_add_partitions_pspec_result() noexcept; - int32_t success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_add_partitions_pspec_result__isset __isset; - - void __set_success(const int32_t val); - - void __set_o1(const InvalidObjectException& val); - - void __set_o2(const AlreadyExistsException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_add_partitions_pspec_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_partitions_pspec_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_partitions_pspec_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_partitions_pspec_presult__isset { - _ThriftHiveMetastore_add_partitions_pspec_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_add_partitions_pspec_presult__isset; - -class ThriftHiveMetastore_add_partitions_pspec_presult { - public: - - - virtual ~ThriftHiveMetastore_add_partitions_pspec_presult() noexcept; - int32_t* success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_add_partitions_pspec_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_append_partition_args__isset { - _ThriftHiveMetastore_append_partition_args__isset() : db_name(false), tbl_name(false), part_vals(false) {} - bool db_name :1; - bool tbl_name :1; - bool part_vals :1; -} _ThriftHiveMetastore_append_partition_args__isset; - -class ThriftHiveMetastore_append_partition_args { - public: - - ThriftHiveMetastore_append_partition_args(const ThriftHiveMetastore_append_partition_args&); - ThriftHiveMetastore_append_partition_args& operator=(const ThriftHiveMetastore_append_partition_args&); - ThriftHiveMetastore_append_partition_args() : db_name(), tbl_name() { - } - - virtual ~ThriftHiveMetastore_append_partition_args() noexcept; - std::string db_name; - std::string tbl_name; - std::vector part_vals; - - _ThriftHiveMetastore_append_partition_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_vals(const std::vector & val); - - bool operator == (const ThriftHiveMetastore_append_partition_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_vals == rhs.part_vals)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_append_partition_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_append_partition_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_append_partition_pargs { - public: - - - virtual ~ThriftHiveMetastore_append_partition_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::vector * part_vals; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_append_partition_result__isset { - _ThriftHiveMetastore_append_partition_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_append_partition_result__isset; - -class ThriftHiveMetastore_append_partition_result { - public: - - ThriftHiveMetastore_append_partition_result(const ThriftHiveMetastore_append_partition_result&); - ThriftHiveMetastore_append_partition_result& operator=(const ThriftHiveMetastore_append_partition_result&); - ThriftHiveMetastore_append_partition_result() { - } - - virtual ~ThriftHiveMetastore_append_partition_result() noexcept; - Partition success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_append_partition_result__isset __isset; - - void __set_success(const Partition& val); - - void __set_o1(const InvalidObjectException& val); - - void __set_o2(const AlreadyExistsException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_append_partition_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_append_partition_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_append_partition_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_append_partition_presult__isset { - _ThriftHiveMetastore_append_partition_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_append_partition_presult__isset; - -class ThriftHiveMetastore_append_partition_presult { - public: - - - virtual ~ThriftHiveMetastore_append_partition_presult() noexcept; - Partition* success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_append_partition_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_add_partitions_req_args__isset { - _ThriftHiveMetastore_add_partitions_req_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_add_partitions_req_args__isset; - -class ThriftHiveMetastore_add_partitions_req_args { - public: - - ThriftHiveMetastore_add_partitions_req_args(const ThriftHiveMetastore_add_partitions_req_args&); - ThriftHiveMetastore_add_partitions_req_args& operator=(const ThriftHiveMetastore_add_partitions_req_args&); - ThriftHiveMetastore_add_partitions_req_args() { - } - - virtual ~ThriftHiveMetastore_add_partitions_req_args() noexcept; - AddPartitionsRequest request; - - _ThriftHiveMetastore_add_partitions_req_args__isset __isset; - - void __set_request(const AddPartitionsRequest& val); - - bool operator == (const ThriftHiveMetastore_add_partitions_req_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_partitions_req_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_partitions_req_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_add_partitions_req_pargs { - public: - - - virtual ~ThriftHiveMetastore_add_partitions_req_pargs() noexcept; - const AddPartitionsRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_partitions_req_result__isset { - _ThriftHiveMetastore_add_partitions_req_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_add_partitions_req_result__isset; - -class ThriftHiveMetastore_add_partitions_req_result { - public: - - ThriftHiveMetastore_add_partitions_req_result(const ThriftHiveMetastore_add_partitions_req_result&); - ThriftHiveMetastore_add_partitions_req_result& operator=(const ThriftHiveMetastore_add_partitions_req_result&); - ThriftHiveMetastore_add_partitions_req_result() { - } - - virtual ~ThriftHiveMetastore_add_partitions_req_result() noexcept; - AddPartitionsResult success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_add_partitions_req_result__isset __isset; - - void __set_success(const AddPartitionsResult& val); - - void __set_o1(const InvalidObjectException& val); - - void __set_o2(const AlreadyExistsException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_add_partitions_req_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_partitions_req_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_partitions_req_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_partitions_req_presult__isset { - _ThriftHiveMetastore_add_partitions_req_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_add_partitions_req_presult__isset; - -class ThriftHiveMetastore_add_partitions_req_presult { - public: - - - virtual ~ThriftHiveMetastore_add_partitions_req_presult() noexcept; - AddPartitionsResult* success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_add_partitions_req_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_append_partition_with_environment_context_args__isset { - _ThriftHiveMetastore_append_partition_with_environment_context_args__isset() : db_name(false), tbl_name(false), part_vals(false), environment_context(false) {} - bool db_name :1; - bool tbl_name :1; - bool part_vals :1; - bool environment_context :1; -} _ThriftHiveMetastore_append_partition_with_environment_context_args__isset; - -class ThriftHiveMetastore_append_partition_with_environment_context_args { - public: - - ThriftHiveMetastore_append_partition_with_environment_context_args(const ThriftHiveMetastore_append_partition_with_environment_context_args&); - ThriftHiveMetastore_append_partition_with_environment_context_args& operator=(const ThriftHiveMetastore_append_partition_with_environment_context_args&); - ThriftHiveMetastore_append_partition_with_environment_context_args() : db_name(), tbl_name() { - } - - virtual ~ThriftHiveMetastore_append_partition_with_environment_context_args() noexcept; - std::string db_name; - std::string tbl_name; - std::vector part_vals; - EnvironmentContext environment_context; - - _ThriftHiveMetastore_append_partition_with_environment_context_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_vals(const std::vector & val); - - void __set_environment_context(const EnvironmentContext& val); - - bool operator == (const ThriftHiveMetastore_append_partition_with_environment_context_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_vals == rhs.part_vals)) - return false; - if (!(environment_context == rhs.environment_context)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_append_partition_with_environment_context_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_append_partition_with_environment_context_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_append_partition_with_environment_context_pargs { - public: - - - virtual ~ThriftHiveMetastore_append_partition_with_environment_context_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::vector * part_vals; - const EnvironmentContext* environment_context; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_append_partition_with_environment_context_result__isset { - _ThriftHiveMetastore_append_partition_with_environment_context_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_append_partition_with_environment_context_result__isset; - -class ThriftHiveMetastore_append_partition_with_environment_context_result { - public: - - ThriftHiveMetastore_append_partition_with_environment_context_result(const ThriftHiveMetastore_append_partition_with_environment_context_result&); - ThriftHiveMetastore_append_partition_with_environment_context_result& operator=(const ThriftHiveMetastore_append_partition_with_environment_context_result&); - ThriftHiveMetastore_append_partition_with_environment_context_result() { - } - - virtual ~ThriftHiveMetastore_append_partition_with_environment_context_result() noexcept; - Partition success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_append_partition_with_environment_context_result__isset __isset; - - void __set_success(const Partition& val); - - void __set_o1(const InvalidObjectException& val); - - void __set_o2(const AlreadyExistsException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_append_partition_with_environment_context_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_append_partition_with_environment_context_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_append_partition_with_environment_context_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_append_partition_with_environment_context_presult__isset { - _ThriftHiveMetastore_append_partition_with_environment_context_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_append_partition_with_environment_context_presult__isset; - -class ThriftHiveMetastore_append_partition_with_environment_context_presult { - public: - - - virtual ~ThriftHiveMetastore_append_partition_with_environment_context_presult() noexcept; - Partition* success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_append_partition_with_environment_context_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_append_partition_by_name_args__isset { - _ThriftHiveMetastore_append_partition_by_name_args__isset() : db_name(false), tbl_name(false), part_name(false) {} - bool db_name :1; - bool tbl_name :1; - bool part_name :1; -} _ThriftHiveMetastore_append_partition_by_name_args__isset; - -class ThriftHiveMetastore_append_partition_by_name_args { - public: - - ThriftHiveMetastore_append_partition_by_name_args(const ThriftHiveMetastore_append_partition_by_name_args&); - ThriftHiveMetastore_append_partition_by_name_args& operator=(const ThriftHiveMetastore_append_partition_by_name_args&); - ThriftHiveMetastore_append_partition_by_name_args() : db_name(), tbl_name(), part_name() { - } - - virtual ~ThriftHiveMetastore_append_partition_by_name_args() noexcept; - std::string db_name; - std::string tbl_name; - std::string part_name; - - _ThriftHiveMetastore_append_partition_by_name_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_append_partition_by_name_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_name == rhs.part_name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_append_partition_by_name_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_append_partition_by_name_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_append_partition_by_name_pargs { - public: - - - virtual ~ThriftHiveMetastore_append_partition_by_name_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::string* part_name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_append_partition_by_name_result__isset { - _ThriftHiveMetastore_append_partition_by_name_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_append_partition_by_name_result__isset; - -class ThriftHiveMetastore_append_partition_by_name_result { - public: - - ThriftHiveMetastore_append_partition_by_name_result(const ThriftHiveMetastore_append_partition_by_name_result&); - ThriftHiveMetastore_append_partition_by_name_result& operator=(const ThriftHiveMetastore_append_partition_by_name_result&); - ThriftHiveMetastore_append_partition_by_name_result() { - } - - virtual ~ThriftHiveMetastore_append_partition_by_name_result() noexcept; - Partition success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_append_partition_by_name_result__isset __isset; - - void __set_success(const Partition& val); - - void __set_o1(const InvalidObjectException& val); - - void __set_o2(const AlreadyExistsException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_append_partition_by_name_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_append_partition_by_name_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_append_partition_by_name_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_append_partition_by_name_presult__isset { - _ThriftHiveMetastore_append_partition_by_name_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_append_partition_by_name_presult__isset; - -class ThriftHiveMetastore_append_partition_by_name_presult { - public: - - - virtual ~ThriftHiveMetastore_append_partition_by_name_presult() noexcept; - Partition* success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_append_partition_by_name_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_append_partition_by_name_with_environment_context_args__isset { - _ThriftHiveMetastore_append_partition_by_name_with_environment_context_args__isset() : db_name(false), tbl_name(false), part_name(false), environment_context(false) {} - bool db_name :1; - bool tbl_name :1; - bool part_name :1; - bool environment_context :1; -} _ThriftHiveMetastore_append_partition_by_name_with_environment_context_args__isset; - -class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args { - public: - - ThriftHiveMetastore_append_partition_by_name_with_environment_context_args(const ThriftHiveMetastore_append_partition_by_name_with_environment_context_args&); - ThriftHiveMetastore_append_partition_by_name_with_environment_context_args& operator=(const ThriftHiveMetastore_append_partition_by_name_with_environment_context_args&); - ThriftHiveMetastore_append_partition_by_name_with_environment_context_args() : db_name(), tbl_name(), part_name() { - } - - virtual ~ThriftHiveMetastore_append_partition_by_name_with_environment_context_args() noexcept; - std::string db_name; - std::string tbl_name; - std::string part_name; - EnvironmentContext environment_context; - - _ThriftHiveMetastore_append_partition_by_name_with_environment_context_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_name(const std::string& val); - - void __set_environment_context(const EnvironmentContext& val); - - bool operator == (const ThriftHiveMetastore_append_partition_by_name_with_environment_context_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_name == rhs.part_name)) - return false; - if (!(environment_context == rhs.environment_context)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_append_partition_by_name_with_environment_context_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_append_partition_by_name_with_environment_context_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs { - public: - - - virtual ~ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::string* part_name; - const EnvironmentContext* environment_context; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_append_partition_by_name_with_environment_context_result__isset { - _ThriftHiveMetastore_append_partition_by_name_with_environment_context_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_append_partition_by_name_with_environment_context_result__isset; - -class ThriftHiveMetastore_append_partition_by_name_with_environment_context_result { - public: - - ThriftHiveMetastore_append_partition_by_name_with_environment_context_result(const ThriftHiveMetastore_append_partition_by_name_with_environment_context_result&); - ThriftHiveMetastore_append_partition_by_name_with_environment_context_result& operator=(const ThriftHiveMetastore_append_partition_by_name_with_environment_context_result&); - ThriftHiveMetastore_append_partition_by_name_with_environment_context_result() { - } - - virtual ~ThriftHiveMetastore_append_partition_by_name_with_environment_context_result() noexcept; - Partition success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_append_partition_by_name_with_environment_context_result__isset __isset; - - void __set_success(const Partition& val); - - void __set_o1(const InvalidObjectException& val); - - void __set_o2(const AlreadyExistsException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_append_partition_by_name_with_environment_context_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_append_partition_by_name_with_environment_context_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_append_partition_by_name_with_environment_context_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult__isset { - _ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult__isset; - -class ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult { - public: - - - virtual ~ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult() noexcept; - Partition* success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_append_partition_by_name_with_environment_context_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_drop_partition_args__isset { - _ThriftHiveMetastore_drop_partition_args__isset() : db_name(false), tbl_name(false), part_vals(false), deleteData(false) {} - bool db_name :1; - bool tbl_name :1; - bool part_vals :1; - bool deleteData :1; -} _ThriftHiveMetastore_drop_partition_args__isset; - -class ThriftHiveMetastore_drop_partition_args { - public: - - ThriftHiveMetastore_drop_partition_args(const ThriftHiveMetastore_drop_partition_args&); - ThriftHiveMetastore_drop_partition_args& operator=(const ThriftHiveMetastore_drop_partition_args&); - ThriftHiveMetastore_drop_partition_args() : db_name(), tbl_name(), deleteData(0) { - } - - virtual ~ThriftHiveMetastore_drop_partition_args() noexcept; - std::string db_name; - std::string tbl_name; - std::vector part_vals; - bool deleteData; - - _ThriftHiveMetastore_drop_partition_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_vals(const std::vector & val); - - void __set_deleteData(const bool val); - - bool operator == (const ThriftHiveMetastore_drop_partition_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_vals == rhs.part_vals)) - return false; - if (!(deleteData == rhs.deleteData)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_partition_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_partition_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_drop_partition_pargs { - public: - - - virtual ~ThriftHiveMetastore_drop_partition_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::vector * part_vals; - const bool* deleteData; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_partition_result__isset { - _ThriftHiveMetastore_drop_partition_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_drop_partition_result__isset; - -class ThriftHiveMetastore_drop_partition_result { - public: - - ThriftHiveMetastore_drop_partition_result(const ThriftHiveMetastore_drop_partition_result&); - ThriftHiveMetastore_drop_partition_result& operator=(const ThriftHiveMetastore_drop_partition_result&); - ThriftHiveMetastore_drop_partition_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_drop_partition_result() noexcept; - bool success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_drop_partition_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_drop_partition_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_partition_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_partition_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_partition_presult__isset { - _ThriftHiveMetastore_drop_partition_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_drop_partition_presult__isset; - -class ThriftHiveMetastore_drop_partition_presult { - public: - - - virtual ~ThriftHiveMetastore_drop_partition_presult() noexcept; - bool* success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_drop_partition_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_drop_partition_with_environment_context_args__isset { - _ThriftHiveMetastore_drop_partition_with_environment_context_args__isset() : db_name(false), tbl_name(false), part_vals(false), deleteData(false), environment_context(false) {} - bool db_name :1; - bool tbl_name :1; - bool part_vals :1; - bool deleteData :1; - bool environment_context :1; -} _ThriftHiveMetastore_drop_partition_with_environment_context_args__isset; - -class ThriftHiveMetastore_drop_partition_with_environment_context_args { - public: - - ThriftHiveMetastore_drop_partition_with_environment_context_args(const ThriftHiveMetastore_drop_partition_with_environment_context_args&); - ThriftHiveMetastore_drop_partition_with_environment_context_args& operator=(const ThriftHiveMetastore_drop_partition_with_environment_context_args&); - ThriftHiveMetastore_drop_partition_with_environment_context_args() : db_name(), tbl_name(), deleteData(0) { - } - - virtual ~ThriftHiveMetastore_drop_partition_with_environment_context_args() noexcept; - std::string db_name; - std::string tbl_name; - std::vector part_vals; - bool deleteData; - EnvironmentContext environment_context; - - _ThriftHiveMetastore_drop_partition_with_environment_context_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_vals(const std::vector & val); - - void __set_deleteData(const bool val); - - void __set_environment_context(const EnvironmentContext& val); - - bool operator == (const ThriftHiveMetastore_drop_partition_with_environment_context_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_vals == rhs.part_vals)) - return false; - if (!(deleteData == rhs.deleteData)) - return false; - if (!(environment_context == rhs.environment_context)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_partition_with_environment_context_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_partition_with_environment_context_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_drop_partition_with_environment_context_pargs { - public: - - - virtual ~ThriftHiveMetastore_drop_partition_with_environment_context_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::vector * part_vals; - const bool* deleteData; - const EnvironmentContext* environment_context; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_partition_with_environment_context_result__isset { - _ThriftHiveMetastore_drop_partition_with_environment_context_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_drop_partition_with_environment_context_result__isset; - -class ThriftHiveMetastore_drop_partition_with_environment_context_result { - public: - - ThriftHiveMetastore_drop_partition_with_environment_context_result(const ThriftHiveMetastore_drop_partition_with_environment_context_result&); - ThriftHiveMetastore_drop_partition_with_environment_context_result& operator=(const ThriftHiveMetastore_drop_partition_with_environment_context_result&); - ThriftHiveMetastore_drop_partition_with_environment_context_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_drop_partition_with_environment_context_result() noexcept; - bool success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_drop_partition_with_environment_context_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_drop_partition_with_environment_context_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_partition_with_environment_context_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_partition_with_environment_context_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_partition_with_environment_context_presult__isset { - _ThriftHiveMetastore_drop_partition_with_environment_context_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_drop_partition_with_environment_context_presult__isset; - -class ThriftHiveMetastore_drop_partition_with_environment_context_presult { - public: - - - virtual ~ThriftHiveMetastore_drop_partition_with_environment_context_presult() noexcept; - bool* success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_drop_partition_with_environment_context_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_drop_partition_by_name_args__isset { - _ThriftHiveMetastore_drop_partition_by_name_args__isset() : db_name(false), tbl_name(false), part_name(false), deleteData(false) {} - bool db_name :1; - bool tbl_name :1; - bool part_name :1; - bool deleteData :1; -} _ThriftHiveMetastore_drop_partition_by_name_args__isset; - -class ThriftHiveMetastore_drop_partition_by_name_args { - public: - - ThriftHiveMetastore_drop_partition_by_name_args(const ThriftHiveMetastore_drop_partition_by_name_args&); - ThriftHiveMetastore_drop_partition_by_name_args& operator=(const ThriftHiveMetastore_drop_partition_by_name_args&); - ThriftHiveMetastore_drop_partition_by_name_args() : db_name(), tbl_name(), part_name(), deleteData(0) { - } - - virtual ~ThriftHiveMetastore_drop_partition_by_name_args() noexcept; - std::string db_name; - std::string tbl_name; - std::string part_name; - bool deleteData; - - _ThriftHiveMetastore_drop_partition_by_name_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_name(const std::string& val); - - void __set_deleteData(const bool val); - - bool operator == (const ThriftHiveMetastore_drop_partition_by_name_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_name == rhs.part_name)) - return false; - if (!(deleteData == rhs.deleteData)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_partition_by_name_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_partition_by_name_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_drop_partition_by_name_pargs { - public: - - - virtual ~ThriftHiveMetastore_drop_partition_by_name_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::string* part_name; - const bool* deleteData; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_partition_by_name_result__isset { - _ThriftHiveMetastore_drop_partition_by_name_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_drop_partition_by_name_result__isset; - -class ThriftHiveMetastore_drop_partition_by_name_result { - public: - - ThriftHiveMetastore_drop_partition_by_name_result(const ThriftHiveMetastore_drop_partition_by_name_result&); - ThriftHiveMetastore_drop_partition_by_name_result& operator=(const ThriftHiveMetastore_drop_partition_by_name_result&); - ThriftHiveMetastore_drop_partition_by_name_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_drop_partition_by_name_result() noexcept; - bool success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_drop_partition_by_name_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_drop_partition_by_name_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_partition_by_name_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_partition_by_name_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_partition_by_name_presult__isset { - _ThriftHiveMetastore_drop_partition_by_name_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_drop_partition_by_name_presult__isset; - -class ThriftHiveMetastore_drop_partition_by_name_presult { - public: - - - virtual ~ThriftHiveMetastore_drop_partition_by_name_presult() noexcept; - bool* success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_drop_partition_by_name_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args__isset { - _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args__isset() : db_name(false), tbl_name(false), part_name(false), deleteData(false), environment_context(false) {} - bool db_name :1; - bool tbl_name :1; - bool part_name :1; - bool deleteData :1; - bool environment_context :1; -} _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args__isset; - -class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { - public: - - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args(const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args&); - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args& operator=(const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args&); - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args() : db_name(), tbl_name(), part_name(), deleteData(0) { - } - - virtual ~ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args() noexcept; - std::string db_name; - std::string tbl_name; - std::string part_name; - bool deleteData; - EnvironmentContext environment_context; - - _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_name(const std::string& val); - - void __set_deleteData(const bool val); - - void __set_environment_context(const EnvironmentContext& val); - - bool operator == (const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_name == rhs.part_name)) - return false; - if (!(deleteData == rhs.deleteData)) - return false; - if (!(environment_context == rhs.environment_context)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs { - public: - - - virtual ~ThriftHiveMetastore_drop_partition_by_name_with_environment_context_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::string* part_name; - const bool* deleteData; - const EnvironmentContext* environment_context; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result__isset { - _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result__isset; - -class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result { - public: - - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result(const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result&); - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result& operator=(const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result&); - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result() noexcept; - bool success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult__isset { - _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult__isset; - -class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult { - public: - - - virtual ~ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult() noexcept; - bool* success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_drop_partition_by_name_with_environment_context_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_drop_partitions_req_args__isset { - _ThriftHiveMetastore_drop_partitions_req_args__isset() : req(false) {} - bool req :1; -} _ThriftHiveMetastore_drop_partitions_req_args__isset; - -class ThriftHiveMetastore_drop_partitions_req_args { - public: - - ThriftHiveMetastore_drop_partitions_req_args(const ThriftHiveMetastore_drop_partitions_req_args&); - ThriftHiveMetastore_drop_partitions_req_args& operator=(const ThriftHiveMetastore_drop_partitions_req_args&); - ThriftHiveMetastore_drop_partitions_req_args() { - } - - virtual ~ThriftHiveMetastore_drop_partitions_req_args() noexcept; - DropPartitionsRequest req; - - _ThriftHiveMetastore_drop_partitions_req_args__isset __isset; - - void __set_req(const DropPartitionsRequest& val); - - bool operator == (const ThriftHiveMetastore_drop_partitions_req_args & rhs) const - { - if (!(req == rhs.req)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_partitions_req_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_partitions_req_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_drop_partitions_req_pargs { - public: - - - virtual ~ThriftHiveMetastore_drop_partitions_req_pargs() noexcept; - const DropPartitionsRequest* req; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_partitions_req_result__isset { - _ThriftHiveMetastore_drop_partitions_req_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_drop_partitions_req_result__isset; - -class ThriftHiveMetastore_drop_partitions_req_result { - public: - - ThriftHiveMetastore_drop_partitions_req_result(const ThriftHiveMetastore_drop_partitions_req_result&); - ThriftHiveMetastore_drop_partitions_req_result& operator=(const ThriftHiveMetastore_drop_partitions_req_result&); - ThriftHiveMetastore_drop_partitions_req_result() { - } - - virtual ~ThriftHiveMetastore_drop_partitions_req_result() noexcept; - DropPartitionsResult success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_drop_partitions_req_result__isset __isset; - - void __set_success(const DropPartitionsResult& val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_drop_partitions_req_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_partitions_req_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_partitions_req_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_partitions_req_presult__isset { - _ThriftHiveMetastore_drop_partitions_req_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_drop_partitions_req_presult__isset; - -class ThriftHiveMetastore_drop_partitions_req_presult { - public: - - - virtual ~ThriftHiveMetastore_drop_partitions_req_presult() noexcept; - DropPartitionsResult* success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_drop_partitions_req_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_partition_args__isset { - _ThriftHiveMetastore_get_partition_args__isset() : db_name(false), tbl_name(false), part_vals(false) {} - bool db_name :1; - bool tbl_name :1; - bool part_vals :1; -} _ThriftHiveMetastore_get_partition_args__isset; - -class ThriftHiveMetastore_get_partition_args { - public: - - ThriftHiveMetastore_get_partition_args(const ThriftHiveMetastore_get_partition_args&); - ThriftHiveMetastore_get_partition_args& operator=(const ThriftHiveMetastore_get_partition_args&); - ThriftHiveMetastore_get_partition_args() : db_name(), tbl_name() { - } - - virtual ~ThriftHiveMetastore_get_partition_args() noexcept; - std::string db_name; - std::string tbl_name; - std::vector part_vals; - - _ThriftHiveMetastore_get_partition_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_vals(const std::vector & val); - - bool operator == (const ThriftHiveMetastore_get_partition_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_vals == rhs.part_vals)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partition_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partition_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_partition_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_partition_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::vector * part_vals; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partition_result__isset { - _ThriftHiveMetastore_get_partition_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partition_result__isset; - -class ThriftHiveMetastore_get_partition_result { - public: - - ThriftHiveMetastore_get_partition_result(const ThriftHiveMetastore_get_partition_result&); - ThriftHiveMetastore_get_partition_result& operator=(const ThriftHiveMetastore_get_partition_result&); - ThriftHiveMetastore_get_partition_result() { - } - - virtual ~ThriftHiveMetastore_get_partition_result() noexcept; - Partition success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partition_result__isset __isset; - - void __set_success(const Partition& val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_partition_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partition_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partition_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partition_presult__isset { - _ThriftHiveMetastore_get_partition_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partition_presult__isset; - -class ThriftHiveMetastore_get_partition_presult { - public: - - - virtual ~ThriftHiveMetastore_get_partition_presult() noexcept; - Partition* success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partition_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_exchange_partition_args__isset { - _ThriftHiveMetastore_exchange_partition_args__isset() : partitionSpecs(false), source_db(false), source_table_name(false), dest_db(false), dest_table_name(false) {} - bool partitionSpecs :1; - bool source_db :1; - bool source_table_name :1; - bool dest_db :1; - bool dest_table_name :1; -} _ThriftHiveMetastore_exchange_partition_args__isset; - -class ThriftHiveMetastore_exchange_partition_args { - public: - - ThriftHiveMetastore_exchange_partition_args(const ThriftHiveMetastore_exchange_partition_args&); - ThriftHiveMetastore_exchange_partition_args& operator=(const ThriftHiveMetastore_exchange_partition_args&); - ThriftHiveMetastore_exchange_partition_args() : source_db(), source_table_name(), dest_db(), dest_table_name() { - } - - virtual ~ThriftHiveMetastore_exchange_partition_args() noexcept; - std::map partitionSpecs; - std::string source_db; - std::string source_table_name; - std::string dest_db; - std::string dest_table_name; - - _ThriftHiveMetastore_exchange_partition_args__isset __isset; - - void __set_partitionSpecs(const std::map & val); - - void __set_source_db(const std::string& val); - - void __set_source_table_name(const std::string& val); - - void __set_dest_db(const std::string& val); - - void __set_dest_table_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_exchange_partition_args & rhs) const - { - if (!(partitionSpecs == rhs.partitionSpecs)) - return false; - if (!(source_db == rhs.source_db)) - return false; - if (!(source_table_name == rhs.source_table_name)) - return false; - if (!(dest_db == rhs.dest_db)) - return false; - if (!(dest_table_name == rhs.dest_table_name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_exchange_partition_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_exchange_partition_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_exchange_partition_pargs { - public: - - - virtual ~ThriftHiveMetastore_exchange_partition_pargs() noexcept; - const std::map * partitionSpecs; - const std::string* source_db; - const std::string* source_table_name; - const std::string* dest_db; - const std::string* dest_table_name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_exchange_partition_result__isset { - _ThriftHiveMetastore_exchange_partition_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_exchange_partition_result__isset; - -class ThriftHiveMetastore_exchange_partition_result { - public: - - ThriftHiveMetastore_exchange_partition_result(const ThriftHiveMetastore_exchange_partition_result&); - ThriftHiveMetastore_exchange_partition_result& operator=(const ThriftHiveMetastore_exchange_partition_result&); - ThriftHiveMetastore_exchange_partition_result() { - } - - virtual ~ThriftHiveMetastore_exchange_partition_result() noexcept; - Partition success; - MetaException o1; - NoSuchObjectException o2; - InvalidObjectException o3; - InvalidInputException o4; - - _ThriftHiveMetastore_exchange_partition_result__isset __isset; - - void __set_success(const Partition& val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - void __set_o3(const InvalidObjectException& val); - - void __set_o4(const InvalidInputException& val); - - bool operator == (const ThriftHiveMetastore_exchange_partition_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - if (!(o4 == rhs.o4)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_exchange_partition_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_exchange_partition_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_exchange_partition_presult__isset { - _ThriftHiveMetastore_exchange_partition_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_exchange_partition_presult__isset; - -class ThriftHiveMetastore_exchange_partition_presult { - public: - - - virtual ~ThriftHiveMetastore_exchange_partition_presult() noexcept; - Partition* success; - MetaException o1; - NoSuchObjectException o2; - InvalidObjectException o3; - InvalidInputException o4; - - _ThriftHiveMetastore_exchange_partition_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_exchange_partitions_args__isset { - _ThriftHiveMetastore_exchange_partitions_args__isset() : partitionSpecs(false), source_db(false), source_table_name(false), dest_db(false), dest_table_name(false) {} - bool partitionSpecs :1; - bool source_db :1; - bool source_table_name :1; - bool dest_db :1; - bool dest_table_name :1; -} _ThriftHiveMetastore_exchange_partitions_args__isset; - -class ThriftHiveMetastore_exchange_partitions_args { - public: - - ThriftHiveMetastore_exchange_partitions_args(const ThriftHiveMetastore_exchange_partitions_args&); - ThriftHiveMetastore_exchange_partitions_args& operator=(const ThriftHiveMetastore_exchange_partitions_args&); - ThriftHiveMetastore_exchange_partitions_args() : source_db(), source_table_name(), dest_db(), dest_table_name() { - } - - virtual ~ThriftHiveMetastore_exchange_partitions_args() noexcept; - std::map partitionSpecs; - std::string source_db; - std::string source_table_name; - std::string dest_db; - std::string dest_table_name; - - _ThriftHiveMetastore_exchange_partitions_args__isset __isset; - - void __set_partitionSpecs(const std::map & val); - - void __set_source_db(const std::string& val); - - void __set_source_table_name(const std::string& val); - - void __set_dest_db(const std::string& val); - - void __set_dest_table_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_exchange_partitions_args & rhs) const - { - if (!(partitionSpecs == rhs.partitionSpecs)) - return false; - if (!(source_db == rhs.source_db)) - return false; - if (!(source_table_name == rhs.source_table_name)) - return false; - if (!(dest_db == rhs.dest_db)) - return false; - if (!(dest_table_name == rhs.dest_table_name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_exchange_partitions_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_exchange_partitions_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_exchange_partitions_pargs { - public: - - - virtual ~ThriftHiveMetastore_exchange_partitions_pargs() noexcept; - const std::map * partitionSpecs; - const std::string* source_db; - const std::string* source_table_name; - const std::string* dest_db; - const std::string* dest_table_name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_exchange_partitions_result__isset { - _ThriftHiveMetastore_exchange_partitions_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_exchange_partitions_result__isset; - -class ThriftHiveMetastore_exchange_partitions_result { - public: - - ThriftHiveMetastore_exchange_partitions_result(const ThriftHiveMetastore_exchange_partitions_result&); - ThriftHiveMetastore_exchange_partitions_result& operator=(const ThriftHiveMetastore_exchange_partitions_result&); - ThriftHiveMetastore_exchange_partitions_result() { - } - - virtual ~ThriftHiveMetastore_exchange_partitions_result() noexcept; - std::vector success; - MetaException o1; - NoSuchObjectException o2; - InvalidObjectException o3; - InvalidInputException o4; - - _ThriftHiveMetastore_exchange_partitions_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - void __set_o3(const InvalidObjectException& val); - - void __set_o4(const InvalidInputException& val); - - bool operator == (const ThriftHiveMetastore_exchange_partitions_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - if (!(o4 == rhs.o4)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_exchange_partitions_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_exchange_partitions_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_exchange_partitions_presult__isset { - _ThriftHiveMetastore_exchange_partitions_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_exchange_partitions_presult__isset; - -class ThriftHiveMetastore_exchange_partitions_presult { - public: - - - virtual ~ThriftHiveMetastore_exchange_partitions_presult() noexcept; - std::vector * success; - MetaException o1; - NoSuchObjectException o2; - InvalidObjectException o3; - InvalidInputException o4; - - _ThriftHiveMetastore_exchange_partitions_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_partition_with_auth_args__isset { - _ThriftHiveMetastore_get_partition_with_auth_args__isset() : db_name(false), tbl_name(false), part_vals(false), user_name(false), group_names(false) {} - bool db_name :1; - bool tbl_name :1; - bool part_vals :1; - bool user_name :1; - bool group_names :1; -} _ThriftHiveMetastore_get_partition_with_auth_args__isset; - -class ThriftHiveMetastore_get_partition_with_auth_args { - public: - - ThriftHiveMetastore_get_partition_with_auth_args(const ThriftHiveMetastore_get_partition_with_auth_args&); - ThriftHiveMetastore_get_partition_with_auth_args& operator=(const ThriftHiveMetastore_get_partition_with_auth_args&); - ThriftHiveMetastore_get_partition_with_auth_args() : db_name(), tbl_name(), user_name() { - } - - virtual ~ThriftHiveMetastore_get_partition_with_auth_args() noexcept; - std::string db_name; - std::string tbl_name; - std::vector part_vals; - std::string user_name; - std::vector group_names; - - _ThriftHiveMetastore_get_partition_with_auth_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_vals(const std::vector & val); - - void __set_user_name(const std::string& val); - - void __set_group_names(const std::vector & val); - - bool operator == (const ThriftHiveMetastore_get_partition_with_auth_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_vals == rhs.part_vals)) - return false; - if (!(user_name == rhs.user_name)) - return false; - if (!(group_names == rhs.group_names)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partition_with_auth_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partition_with_auth_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_partition_with_auth_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_partition_with_auth_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::vector * part_vals; - const std::string* user_name; - const std::vector * group_names; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partition_with_auth_result__isset { - _ThriftHiveMetastore_get_partition_with_auth_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partition_with_auth_result__isset; - -class ThriftHiveMetastore_get_partition_with_auth_result { - public: - - ThriftHiveMetastore_get_partition_with_auth_result(const ThriftHiveMetastore_get_partition_with_auth_result&); - ThriftHiveMetastore_get_partition_with_auth_result& operator=(const ThriftHiveMetastore_get_partition_with_auth_result&); - ThriftHiveMetastore_get_partition_with_auth_result() { - } - - virtual ~ThriftHiveMetastore_get_partition_with_auth_result() noexcept; - Partition success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partition_with_auth_result__isset __isset; - - void __set_success(const Partition& val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_partition_with_auth_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partition_with_auth_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partition_with_auth_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partition_with_auth_presult__isset { - _ThriftHiveMetastore_get_partition_with_auth_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partition_with_auth_presult__isset; - -class ThriftHiveMetastore_get_partition_with_auth_presult { - public: - - - virtual ~ThriftHiveMetastore_get_partition_with_auth_presult() noexcept; - Partition* success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partition_with_auth_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_partition_by_name_args__isset { - _ThriftHiveMetastore_get_partition_by_name_args__isset() : db_name(false), tbl_name(false), part_name(false) {} - bool db_name :1; - bool tbl_name :1; - bool part_name :1; -} _ThriftHiveMetastore_get_partition_by_name_args__isset; - -class ThriftHiveMetastore_get_partition_by_name_args { - public: - - ThriftHiveMetastore_get_partition_by_name_args(const ThriftHiveMetastore_get_partition_by_name_args&); - ThriftHiveMetastore_get_partition_by_name_args& operator=(const ThriftHiveMetastore_get_partition_by_name_args&); - ThriftHiveMetastore_get_partition_by_name_args() : db_name(), tbl_name(), part_name() { - } - - virtual ~ThriftHiveMetastore_get_partition_by_name_args() noexcept; - std::string db_name; - std::string tbl_name; - std::string part_name; - - _ThriftHiveMetastore_get_partition_by_name_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_partition_by_name_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_name == rhs.part_name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partition_by_name_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partition_by_name_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_partition_by_name_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_partition_by_name_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::string* part_name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partition_by_name_result__isset { - _ThriftHiveMetastore_get_partition_by_name_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partition_by_name_result__isset; - -class ThriftHiveMetastore_get_partition_by_name_result { - public: - - ThriftHiveMetastore_get_partition_by_name_result(const ThriftHiveMetastore_get_partition_by_name_result&); - ThriftHiveMetastore_get_partition_by_name_result& operator=(const ThriftHiveMetastore_get_partition_by_name_result&); - ThriftHiveMetastore_get_partition_by_name_result() { - } - - virtual ~ThriftHiveMetastore_get_partition_by_name_result() noexcept; - Partition success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partition_by_name_result__isset __isset; - - void __set_success(const Partition& val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_partition_by_name_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partition_by_name_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partition_by_name_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partition_by_name_presult__isset { - _ThriftHiveMetastore_get_partition_by_name_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partition_by_name_presult__isset; - -class ThriftHiveMetastore_get_partition_by_name_presult { - public: - - - virtual ~ThriftHiveMetastore_get_partition_by_name_presult() noexcept; - Partition* success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partition_by_name_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_args__isset { - _ThriftHiveMetastore_get_partitions_args__isset() : db_name(false), tbl_name(false), max_parts(true) {} - bool db_name :1; - bool tbl_name :1; - bool max_parts :1; -} _ThriftHiveMetastore_get_partitions_args__isset; - -class ThriftHiveMetastore_get_partitions_args { - public: - - ThriftHiveMetastore_get_partitions_args(const ThriftHiveMetastore_get_partitions_args&); - ThriftHiveMetastore_get_partitions_args& operator=(const ThriftHiveMetastore_get_partitions_args&); - ThriftHiveMetastore_get_partitions_args() : db_name(), tbl_name(), max_parts(-1) { - } - - virtual ~ThriftHiveMetastore_get_partitions_args() noexcept; - std::string db_name; - std::string tbl_name; - int16_t max_parts; - - _ThriftHiveMetastore_get_partitions_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_max_parts(const int16_t val); - - bool operator == (const ThriftHiveMetastore_get_partitions_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(max_parts == rhs.max_parts)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_partitions_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const int16_t* max_parts; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_result__isset { - _ThriftHiveMetastore_get_partitions_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_result__isset; - -class ThriftHiveMetastore_get_partitions_result { - public: - - ThriftHiveMetastore_get_partitions_result(const ThriftHiveMetastore_get_partitions_result&); - ThriftHiveMetastore_get_partitions_result& operator=(const ThriftHiveMetastore_get_partitions_result&); - ThriftHiveMetastore_get_partitions_result() { - } - - virtual ~ThriftHiveMetastore_get_partitions_result() noexcept; - std::vector success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_partitions_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_partitions_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_presult__isset { - _ThriftHiveMetastore_get_partitions_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_presult__isset; - -class ThriftHiveMetastore_get_partitions_presult { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_presult() noexcept; - std::vector * success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_partitions_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_with_auth_args__isset { - _ThriftHiveMetastore_get_partitions_with_auth_args__isset() : db_name(false), tbl_name(false), max_parts(true), user_name(false), group_names(false) {} - bool db_name :1; - bool tbl_name :1; - bool max_parts :1; - bool user_name :1; - bool group_names :1; -} _ThriftHiveMetastore_get_partitions_with_auth_args__isset; - -class ThriftHiveMetastore_get_partitions_with_auth_args { - public: - - ThriftHiveMetastore_get_partitions_with_auth_args(const ThriftHiveMetastore_get_partitions_with_auth_args&); - ThriftHiveMetastore_get_partitions_with_auth_args& operator=(const ThriftHiveMetastore_get_partitions_with_auth_args&); - ThriftHiveMetastore_get_partitions_with_auth_args() : db_name(), tbl_name(), max_parts(-1), user_name() { - } - - virtual ~ThriftHiveMetastore_get_partitions_with_auth_args() noexcept; - std::string db_name; - std::string tbl_name; - int16_t max_parts; - std::string user_name; - std::vector group_names; - - _ThriftHiveMetastore_get_partitions_with_auth_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_max_parts(const int16_t val); - - void __set_user_name(const std::string& val); - - void __set_group_names(const std::vector & val); - - bool operator == (const ThriftHiveMetastore_get_partitions_with_auth_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(max_parts == rhs.max_parts)) - return false; - if (!(user_name == rhs.user_name)) - return false; - if (!(group_names == rhs.group_names)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_with_auth_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_with_auth_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_partitions_with_auth_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_with_auth_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const int16_t* max_parts; - const std::string* user_name; - const std::vector * group_names; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_with_auth_result__isset { - _ThriftHiveMetastore_get_partitions_with_auth_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_with_auth_result__isset; - -class ThriftHiveMetastore_get_partitions_with_auth_result { - public: - - ThriftHiveMetastore_get_partitions_with_auth_result(const ThriftHiveMetastore_get_partitions_with_auth_result&); - ThriftHiveMetastore_get_partitions_with_auth_result& operator=(const ThriftHiveMetastore_get_partitions_with_auth_result&); - ThriftHiveMetastore_get_partitions_with_auth_result() { - } - - virtual ~ThriftHiveMetastore_get_partitions_with_auth_result() noexcept; - std::vector success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_partitions_with_auth_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_partitions_with_auth_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_with_auth_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_with_auth_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_with_auth_presult__isset { - _ThriftHiveMetastore_get_partitions_with_auth_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_with_auth_presult__isset; - -class ThriftHiveMetastore_get_partitions_with_auth_presult { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_with_auth_presult() noexcept; - std::vector * success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_partitions_with_auth_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_pspec_args__isset { - _ThriftHiveMetastore_get_partitions_pspec_args__isset() : db_name(false), tbl_name(false), max_parts(true) {} - bool db_name :1; - bool tbl_name :1; - bool max_parts :1; -} _ThriftHiveMetastore_get_partitions_pspec_args__isset; - -class ThriftHiveMetastore_get_partitions_pspec_args { - public: - - ThriftHiveMetastore_get_partitions_pspec_args(const ThriftHiveMetastore_get_partitions_pspec_args&); - ThriftHiveMetastore_get_partitions_pspec_args& operator=(const ThriftHiveMetastore_get_partitions_pspec_args&); - ThriftHiveMetastore_get_partitions_pspec_args() : db_name(), tbl_name(), max_parts(-1) { - } - - virtual ~ThriftHiveMetastore_get_partitions_pspec_args() noexcept; - std::string db_name; - std::string tbl_name; - int32_t max_parts; - - _ThriftHiveMetastore_get_partitions_pspec_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_max_parts(const int32_t val); - - bool operator == (const ThriftHiveMetastore_get_partitions_pspec_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(max_parts == rhs.max_parts)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_pspec_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_pspec_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_partitions_pspec_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_pspec_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const int32_t* max_parts; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_pspec_result__isset { - _ThriftHiveMetastore_get_partitions_pspec_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_pspec_result__isset; - -class ThriftHiveMetastore_get_partitions_pspec_result { - public: - - ThriftHiveMetastore_get_partitions_pspec_result(const ThriftHiveMetastore_get_partitions_pspec_result&); - ThriftHiveMetastore_get_partitions_pspec_result& operator=(const ThriftHiveMetastore_get_partitions_pspec_result&); - ThriftHiveMetastore_get_partitions_pspec_result() { - } - - virtual ~ThriftHiveMetastore_get_partitions_pspec_result() noexcept; - std::vector success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_partitions_pspec_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_partitions_pspec_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_pspec_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_pspec_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_pspec_presult__isset { - _ThriftHiveMetastore_get_partitions_pspec_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_pspec_presult__isset; - -class ThriftHiveMetastore_get_partitions_pspec_presult { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_pspec_presult() noexcept; - std::vector * success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_partitions_pspec_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_partition_names_args__isset { - _ThriftHiveMetastore_get_partition_names_args__isset() : db_name(false), tbl_name(false), max_parts(true) {} - bool db_name :1; - bool tbl_name :1; - bool max_parts :1; -} _ThriftHiveMetastore_get_partition_names_args__isset; - -class ThriftHiveMetastore_get_partition_names_args { - public: - - ThriftHiveMetastore_get_partition_names_args(const ThriftHiveMetastore_get_partition_names_args&); - ThriftHiveMetastore_get_partition_names_args& operator=(const ThriftHiveMetastore_get_partition_names_args&); - ThriftHiveMetastore_get_partition_names_args() : db_name(), tbl_name(), max_parts(-1) { - } - - virtual ~ThriftHiveMetastore_get_partition_names_args() noexcept; - std::string db_name; - std::string tbl_name; - int16_t max_parts; - - _ThriftHiveMetastore_get_partition_names_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_max_parts(const int16_t val); - - bool operator == (const ThriftHiveMetastore_get_partition_names_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(max_parts == rhs.max_parts)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partition_names_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partition_names_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_partition_names_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_partition_names_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const int16_t* max_parts; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partition_names_result__isset { - _ThriftHiveMetastore_get_partition_names_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partition_names_result__isset; - -class ThriftHiveMetastore_get_partition_names_result { - public: - - ThriftHiveMetastore_get_partition_names_result(const ThriftHiveMetastore_get_partition_names_result&); - ThriftHiveMetastore_get_partition_names_result& operator=(const ThriftHiveMetastore_get_partition_names_result&); - ThriftHiveMetastore_get_partition_names_result() { - } - - virtual ~ThriftHiveMetastore_get_partition_names_result() noexcept; - std::vector success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_partition_names_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_partition_names_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partition_names_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partition_names_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partition_names_presult__isset { - _ThriftHiveMetastore_get_partition_names_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partition_names_presult__isset; - -class ThriftHiveMetastore_get_partition_names_presult { - public: - - - virtual ~ThriftHiveMetastore_get_partition_names_presult() noexcept; - std::vector * success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_partition_names_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_partition_values_args__isset { - _ThriftHiveMetastore_get_partition_values_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_get_partition_values_args__isset; - -class ThriftHiveMetastore_get_partition_values_args { - public: - - ThriftHiveMetastore_get_partition_values_args(const ThriftHiveMetastore_get_partition_values_args&); - ThriftHiveMetastore_get_partition_values_args& operator=(const ThriftHiveMetastore_get_partition_values_args&); - ThriftHiveMetastore_get_partition_values_args() { - } - - virtual ~ThriftHiveMetastore_get_partition_values_args() noexcept; - PartitionValuesRequest request; - - _ThriftHiveMetastore_get_partition_values_args__isset __isset; - - void __set_request(const PartitionValuesRequest& val); - - bool operator == (const ThriftHiveMetastore_get_partition_values_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partition_values_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partition_values_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_partition_values_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_partition_values_pargs() noexcept; - const PartitionValuesRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partition_values_result__isset { - _ThriftHiveMetastore_get_partition_values_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partition_values_result__isset; - -class ThriftHiveMetastore_get_partition_values_result { - public: - - ThriftHiveMetastore_get_partition_values_result(const ThriftHiveMetastore_get_partition_values_result&); - ThriftHiveMetastore_get_partition_values_result& operator=(const ThriftHiveMetastore_get_partition_values_result&); - ThriftHiveMetastore_get_partition_values_result() { - } - - virtual ~ThriftHiveMetastore_get_partition_values_result() noexcept; - PartitionValuesResponse success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partition_values_result__isset __isset; - - void __set_success(const PartitionValuesResponse& val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_partition_values_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partition_values_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partition_values_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partition_values_presult__isset { - _ThriftHiveMetastore_get_partition_values_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partition_values_presult__isset; - -class ThriftHiveMetastore_get_partition_values_presult { - public: - - - virtual ~ThriftHiveMetastore_get_partition_values_presult() noexcept; - PartitionValuesResponse* success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partition_values_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_ps_args__isset { - _ThriftHiveMetastore_get_partitions_ps_args__isset() : db_name(false), tbl_name(false), part_vals(false), max_parts(true) {} - bool db_name :1; - bool tbl_name :1; - bool part_vals :1; - bool max_parts :1; -} _ThriftHiveMetastore_get_partitions_ps_args__isset; - -class ThriftHiveMetastore_get_partitions_ps_args { - public: - - ThriftHiveMetastore_get_partitions_ps_args(const ThriftHiveMetastore_get_partitions_ps_args&); - ThriftHiveMetastore_get_partitions_ps_args& operator=(const ThriftHiveMetastore_get_partitions_ps_args&); - ThriftHiveMetastore_get_partitions_ps_args() : db_name(), tbl_name(), max_parts(-1) { - } - - virtual ~ThriftHiveMetastore_get_partitions_ps_args() noexcept; - std::string db_name; - std::string tbl_name; - std::vector part_vals; - int16_t max_parts; - - _ThriftHiveMetastore_get_partitions_ps_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_vals(const std::vector & val); - - void __set_max_parts(const int16_t val); - - bool operator == (const ThriftHiveMetastore_get_partitions_ps_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_vals == rhs.part_vals)) - return false; - if (!(max_parts == rhs.max_parts)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_ps_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_ps_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_partitions_ps_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_ps_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::vector * part_vals; - const int16_t* max_parts; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_ps_result__isset { - _ThriftHiveMetastore_get_partitions_ps_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_ps_result__isset; - -class ThriftHiveMetastore_get_partitions_ps_result { - public: - - ThriftHiveMetastore_get_partitions_ps_result(const ThriftHiveMetastore_get_partitions_ps_result&); - ThriftHiveMetastore_get_partitions_ps_result& operator=(const ThriftHiveMetastore_get_partitions_ps_result&); - ThriftHiveMetastore_get_partitions_ps_result() { - } - - virtual ~ThriftHiveMetastore_get_partitions_ps_result() noexcept; - std::vector success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partitions_ps_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_partitions_ps_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_ps_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_ps_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_ps_presult__isset { - _ThriftHiveMetastore_get_partitions_ps_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_ps_presult__isset; - -class ThriftHiveMetastore_get_partitions_ps_presult { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_ps_presult() noexcept; - std::vector * success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partitions_ps_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_ps_with_auth_args__isset { - _ThriftHiveMetastore_get_partitions_ps_with_auth_args__isset() : db_name(false), tbl_name(false), part_vals(false), max_parts(true), user_name(false), group_names(false) {} - bool db_name :1; - bool tbl_name :1; - bool part_vals :1; - bool max_parts :1; - bool user_name :1; - bool group_names :1; -} _ThriftHiveMetastore_get_partitions_ps_with_auth_args__isset; - -class ThriftHiveMetastore_get_partitions_ps_with_auth_args { - public: - - ThriftHiveMetastore_get_partitions_ps_with_auth_args(const ThriftHiveMetastore_get_partitions_ps_with_auth_args&); - ThriftHiveMetastore_get_partitions_ps_with_auth_args& operator=(const ThriftHiveMetastore_get_partitions_ps_with_auth_args&); - ThriftHiveMetastore_get_partitions_ps_with_auth_args() : db_name(), tbl_name(), max_parts(-1), user_name() { - } - - virtual ~ThriftHiveMetastore_get_partitions_ps_with_auth_args() noexcept; - std::string db_name; - std::string tbl_name; - std::vector part_vals; - int16_t max_parts; - std::string user_name; - std::vector group_names; - - _ThriftHiveMetastore_get_partitions_ps_with_auth_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_vals(const std::vector & val); - - void __set_max_parts(const int16_t val); - - void __set_user_name(const std::string& val); - - void __set_group_names(const std::vector & val); - - bool operator == (const ThriftHiveMetastore_get_partitions_ps_with_auth_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_vals == rhs.part_vals)) - return false; - if (!(max_parts == rhs.max_parts)) - return false; - if (!(user_name == rhs.user_name)) - return false; - if (!(group_names == rhs.group_names)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_ps_with_auth_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_ps_with_auth_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_partitions_ps_with_auth_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_ps_with_auth_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::vector * part_vals; - const int16_t* max_parts; - const std::string* user_name; - const std::vector * group_names; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_ps_with_auth_result__isset { - _ThriftHiveMetastore_get_partitions_ps_with_auth_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_ps_with_auth_result__isset; - -class ThriftHiveMetastore_get_partitions_ps_with_auth_result { - public: - - ThriftHiveMetastore_get_partitions_ps_with_auth_result(const ThriftHiveMetastore_get_partitions_ps_with_auth_result&); - ThriftHiveMetastore_get_partitions_ps_with_auth_result& operator=(const ThriftHiveMetastore_get_partitions_ps_with_auth_result&); - ThriftHiveMetastore_get_partitions_ps_with_auth_result() { - } - - virtual ~ThriftHiveMetastore_get_partitions_ps_with_auth_result() noexcept; - std::vector success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_partitions_ps_with_auth_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_partitions_ps_with_auth_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_ps_with_auth_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_ps_with_auth_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_ps_with_auth_presult__isset { - _ThriftHiveMetastore_get_partitions_ps_with_auth_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_ps_with_auth_presult__isset; - -class ThriftHiveMetastore_get_partitions_ps_with_auth_presult { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_ps_with_auth_presult() noexcept; - std::vector * success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_partitions_ps_with_auth_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_partition_names_ps_args__isset { - _ThriftHiveMetastore_get_partition_names_ps_args__isset() : db_name(false), tbl_name(false), part_vals(false), max_parts(true) {} - bool db_name :1; - bool tbl_name :1; - bool part_vals :1; - bool max_parts :1; -} _ThriftHiveMetastore_get_partition_names_ps_args__isset; - -class ThriftHiveMetastore_get_partition_names_ps_args { - public: - - ThriftHiveMetastore_get_partition_names_ps_args(const ThriftHiveMetastore_get_partition_names_ps_args&); - ThriftHiveMetastore_get_partition_names_ps_args& operator=(const ThriftHiveMetastore_get_partition_names_ps_args&); - ThriftHiveMetastore_get_partition_names_ps_args() : db_name(), tbl_name(), max_parts(-1) { - } - - virtual ~ThriftHiveMetastore_get_partition_names_ps_args() noexcept; - std::string db_name; - std::string tbl_name; - std::vector part_vals; - int16_t max_parts; - - _ThriftHiveMetastore_get_partition_names_ps_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_vals(const std::vector & val); - - void __set_max_parts(const int16_t val); - - bool operator == (const ThriftHiveMetastore_get_partition_names_ps_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_vals == rhs.part_vals)) - return false; - if (!(max_parts == rhs.max_parts)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partition_names_ps_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partition_names_ps_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_partition_names_ps_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_partition_names_ps_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::vector * part_vals; - const int16_t* max_parts; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partition_names_ps_result__isset { - _ThriftHiveMetastore_get_partition_names_ps_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partition_names_ps_result__isset; - -class ThriftHiveMetastore_get_partition_names_ps_result { - public: - - ThriftHiveMetastore_get_partition_names_ps_result(const ThriftHiveMetastore_get_partition_names_ps_result&); - ThriftHiveMetastore_get_partition_names_ps_result& operator=(const ThriftHiveMetastore_get_partition_names_ps_result&); - ThriftHiveMetastore_get_partition_names_ps_result() { - } - - virtual ~ThriftHiveMetastore_get_partition_names_ps_result() noexcept; - std::vector success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partition_names_ps_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_partition_names_ps_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partition_names_ps_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partition_names_ps_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partition_names_ps_presult__isset { - _ThriftHiveMetastore_get_partition_names_ps_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partition_names_ps_presult__isset; - -class ThriftHiveMetastore_get_partition_names_ps_presult { - public: - - - virtual ~ThriftHiveMetastore_get_partition_names_ps_presult() noexcept; - std::vector * success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partition_names_ps_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_by_filter_args__isset { - _ThriftHiveMetastore_get_partitions_by_filter_args__isset() : db_name(false), tbl_name(false), filter(false), max_parts(true) {} - bool db_name :1; - bool tbl_name :1; - bool filter :1; - bool max_parts :1; -} _ThriftHiveMetastore_get_partitions_by_filter_args__isset; - -class ThriftHiveMetastore_get_partitions_by_filter_args { - public: - - ThriftHiveMetastore_get_partitions_by_filter_args(const ThriftHiveMetastore_get_partitions_by_filter_args&); - ThriftHiveMetastore_get_partitions_by_filter_args& operator=(const ThriftHiveMetastore_get_partitions_by_filter_args&); - ThriftHiveMetastore_get_partitions_by_filter_args() : db_name(), tbl_name(), filter(), max_parts(-1) { - } - - virtual ~ThriftHiveMetastore_get_partitions_by_filter_args() noexcept; - std::string db_name; - std::string tbl_name; - std::string filter; - int16_t max_parts; - - _ThriftHiveMetastore_get_partitions_by_filter_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_filter(const std::string& val); - - void __set_max_parts(const int16_t val); - - bool operator == (const ThriftHiveMetastore_get_partitions_by_filter_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(filter == rhs.filter)) - return false; - if (!(max_parts == rhs.max_parts)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_by_filter_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_by_filter_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_partitions_by_filter_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_by_filter_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::string* filter; - const int16_t* max_parts; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_by_filter_result__isset { - _ThriftHiveMetastore_get_partitions_by_filter_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_by_filter_result__isset; - -class ThriftHiveMetastore_get_partitions_by_filter_result { - public: - - ThriftHiveMetastore_get_partitions_by_filter_result(const ThriftHiveMetastore_get_partitions_by_filter_result&); - ThriftHiveMetastore_get_partitions_by_filter_result& operator=(const ThriftHiveMetastore_get_partitions_by_filter_result&); - ThriftHiveMetastore_get_partitions_by_filter_result() { - } - - virtual ~ThriftHiveMetastore_get_partitions_by_filter_result() noexcept; - std::vector success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partitions_by_filter_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_partitions_by_filter_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_by_filter_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_by_filter_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_by_filter_presult__isset { - _ThriftHiveMetastore_get_partitions_by_filter_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_by_filter_presult__isset; - -class ThriftHiveMetastore_get_partitions_by_filter_presult { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_by_filter_presult() noexcept; - std::vector * success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partitions_by_filter_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_part_specs_by_filter_args__isset { - _ThriftHiveMetastore_get_part_specs_by_filter_args__isset() : db_name(false), tbl_name(false), filter(false), max_parts(true) {} - bool db_name :1; - bool tbl_name :1; - bool filter :1; - bool max_parts :1; -} _ThriftHiveMetastore_get_part_specs_by_filter_args__isset; - -class ThriftHiveMetastore_get_part_specs_by_filter_args { - public: - - ThriftHiveMetastore_get_part_specs_by_filter_args(const ThriftHiveMetastore_get_part_specs_by_filter_args&); - ThriftHiveMetastore_get_part_specs_by_filter_args& operator=(const ThriftHiveMetastore_get_part_specs_by_filter_args&); - ThriftHiveMetastore_get_part_specs_by_filter_args() : db_name(), tbl_name(), filter(), max_parts(-1) { - } - - virtual ~ThriftHiveMetastore_get_part_specs_by_filter_args() noexcept; - std::string db_name; - std::string tbl_name; - std::string filter; - int32_t max_parts; - - _ThriftHiveMetastore_get_part_specs_by_filter_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_filter(const std::string& val); - - void __set_max_parts(const int32_t val); - - bool operator == (const ThriftHiveMetastore_get_part_specs_by_filter_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(filter == rhs.filter)) - return false; - if (!(max_parts == rhs.max_parts)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_part_specs_by_filter_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_part_specs_by_filter_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_part_specs_by_filter_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_part_specs_by_filter_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::string* filter; - const int32_t* max_parts; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_part_specs_by_filter_result__isset { - _ThriftHiveMetastore_get_part_specs_by_filter_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_part_specs_by_filter_result__isset; - -class ThriftHiveMetastore_get_part_specs_by_filter_result { - public: - - ThriftHiveMetastore_get_part_specs_by_filter_result(const ThriftHiveMetastore_get_part_specs_by_filter_result&); - ThriftHiveMetastore_get_part_specs_by_filter_result& operator=(const ThriftHiveMetastore_get_part_specs_by_filter_result&); - ThriftHiveMetastore_get_part_specs_by_filter_result() { - } - - virtual ~ThriftHiveMetastore_get_part_specs_by_filter_result() noexcept; - std::vector success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_part_specs_by_filter_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_part_specs_by_filter_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_part_specs_by_filter_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_part_specs_by_filter_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_part_specs_by_filter_presult__isset { - _ThriftHiveMetastore_get_part_specs_by_filter_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_part_specs_by_filter_presult__isset; - -class ThriftHiveMetastore_get_part_specs_by_filter_presult { - public: - - - virtual ~ThriftHiveMetastore_get_part_specs_by_filter_presult() noexcept; - std::vector * success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_part_specs_by_filter_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_by_expr_args__isset { - _ThriftHiveMetastore_get_partitions_by_expr_args__isset() : req(false) {} - bool req :1; -} _ThriftHiveMetastore_get_partitions_by_expr_args__isset; - -class ThriftHiveMetastore_get_partitions_by_expr_args { - public: - - ThriftHiveMetastore_get_partitions_by_expr_args(const ThriftHiveMetastore_get_partitions_by_expr_args&); - ThriftHiveMetastore_get_partitions_by_expr_args& operator=(const ThriftHiveMetastore_get_partitions_by_expr_args&); - ThriftHiveMetastore_get_partitions_by_expr_args() { - } - - virtual ~ThriftHiveMetastore_get_partitions_by_expr_args() noexcept; - PartitionsByExprRequest req; - - _ThriftHiveMetastore_get_partitions_by_expr_args__isset __isset; - - void __set_req(const PartitionsByExprRequest& val); - - bool operator == (const ThriftHiveMetastore_get_partitions_by_expr_args & rhs) const - { - if (!(req == rhs.req)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_by_expr_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_by_expr_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_partitions_by_expr_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_by_expr_pargs() noexcept; - const PartitionsByExprRequest* req; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_by_expr_result__isset { - _ThriftHiveMetastore_get_partitions_by_expr_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_by_expr_result__isset; - -class ThriftHiveMetastore_get_partitions_by_expr_result { - public: - - ThriftHiveMetastore_get_partitions_by_expr_result(const ThriftHiveMetastore_get_partitions_by_expr_result&); - ThriftHiveMetastore_get_partitions_by_expr_result& operator=(const ThriftHiveMetastore_get_partitions_by_expr_result&); - ThriftHiveMetastore_get_partitions_by_expr_result() { - } - - virtual ~ThriftHiveMetastore_get_partitions_by_expr_result() noexcept; - PartitionsByExprResult success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partitions_by_expr_result__isset __isset; - - void __set_success(const PartitionsByExprResult& val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_partitions_by_expr_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_by_expr_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_by_expr_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_by_expr_presult__isset { - _ThriftHiveMetastore_get_partitions_by_expr_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_by_expr_presult__isset; - -class ThriftHiveMetastore_get_partitions_by_expr_presult { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_by_expr_presult() noexcept; - PartitionsByExprResult* success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partitions_by_expr_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_num_partitions_by_filter_args__isset { - _ThriftHiveMetastore_get_num_partitions_by_filter_args__isset() : db_name(false), tbl_name(false), filter(false) {} - bool db_name :1; - bool tbl_name :1; - bool filter :1; -} _ThriftHiveMetastore_get_num_partitions_by_filter_args__isset; - -class ThriftHiveMetastore_get_num_partitions_by_filter_args { - public: - - ThriftHiveMetastore_get_num_partitions_by_filter_args(const ThriftHiveMetastore_get_num_partitions_by_filter_args&); - ThriftHiveMetastore_get_num_partitions_by_filter_args& operator=(const ThriftHiveMetastore_get_num_partitions_by_filter_args&); - ThriftHiveMetastore_get_num_partitions_by_filter_args() : db_name(), tbl_name(), filter() { - } - - virtual ~ThriftHiveMetastore_get_num_partitions_by_filter_args() noexcept; - std::string db_name; - std::string tbl_name; - std::string filter; - - _ThriftHiveMetastore_get_num_partitions_by_filter_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_filter(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_num_partitions_by_filter_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(filter == rhs.filter)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_num_partitions_by_filter_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_num_partitions_by_filter_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_num_partitions_by_filter_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_num_partitions_by_filter_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::string* filter; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_num_partitions_by_filter_result__isset { - _ThriftHiveMetastore_get_num_partitions_by_filter_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_num_partitions_by_filter_result__isset; - -class ThriftHiveMetastore_get_num_partitions_by_filter_result { - public: - - ThriftHiveMetastore_get_num_partitions_by_filter_result(const ThriftHiveMetastore_get_num_partitions_by_filter_result&); - ThriftHiveMetastore_get_num_partitions_by_filter_result& operator=(const ThriftHiveMetastore_get_num_partitions_by_filter_result&); - ThriftHiveMetastore_get_num_partitions_by_filter_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_get_num_partitions_by_filter_result() noexcept; - int32_t success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_num_partitions_by_filter_result__isset __isset; - - void __set_success(const int32_t val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_num_partitions_by_filter_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_num_partitions_by_filter_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_num_partitions_by_filter_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_num_partitions_by_filter_presult__isset { - _ThriftHiveMetastore_get_num_partitions_by_filter_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_num_partitions_by_filter_presult__isset; - -class ThriftHiveMetastore_get_num_partitions_by_filter_presult { - public: - - - virtual ~ThriftHiveMetastore_get_num_partitions_by_filter_presult() noexcept; - int32_t* success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_num_partitions_by_filter_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_by_names_args__isset { - _ThriftHiveMetastore_get_partitions_by_names_args__isset() : db_name(false), tbl_name(false), names(false) {} - bool db_name :1; - bool tbl_name :1; - bool names :1; -} _ThriftHiveMetastore_get_partitions_by_names_args__isset; - -class ThriftHiveMetastore_get_partitions_by_names_args { - public: - - ThriftHiveMetastore_get_partitions_by_names_args(const ThriftHiveMetastore_get_partitions_by_names_args&); - ThriftHiveMetastore_get_partitions_by_names_args& operator=(const ThriftHiveMetastore_get_partitions_by_names_args&); - ThriftHiveMetastore_get_partitions_by_names_args() : db_name(), tbl_name() { - } - - virtual ~ThriftHiveMetastore_get_partitions_by_names_args() noexcept; - std::string db_name; - std::string tbl_name; - std::vector names; - - _ThriftHiveMetastore_get_partitions_by_names_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_names(const std::vector & val); - - bool operator == (const ThriftHiveMetastore_get_partitions_by_names_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(names == rhs.names)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_by_names_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_by_names_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_partitions_by_names_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_by_names_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::vector * names; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_by_names_result__isset { - _ThriftHiveMetastore_get_partitions_by_names_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_by_names_result__isset; - -class ThriftHiveMetastore_get_partitions_by_names_result { - public: - - ThriftHiveMetastore_get_partitions_by_names_result(const ThriftHiveMetastore_get_partitions_by_names_result&); - ThriftHiveMetastore_get_partitions_by_names_result& operator=(const ThriftHiveMetastore_get_partitions_by_names_result&); - ThriftHiveMetastore_get_partitions_by_names_result() { - } - - virtual ~ThriftHiveMetastore_get_partitions_by_names_result() noexcept; - std::vector success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partitions_by_names_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_partitions_by_names_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_by_names_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_by_names_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_by_names_presult__isset { - _ThriftHiveMetastore_get_partitions_by_names_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_by_names_presult__isset; - -class ThriftHiveMetastore_get_partitions_by_names_presult { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_by_names_presult() noexcept; - std::vector * success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_partitions_by_names_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_alter_partition_args__isset { - _ThriftHiveMetastore_alter_partition_args__isset() : db_name(false), tbl_name(false), new_part(false) {} - bool db_name :1; - bool tbl_name :1; - bool new_part :1; -} _ThriftHiveMetastore_alter_partition_args__isset; - -class ThriftHiveMetastore_alter_partition_args { - public: - - ThriftHiveMetastore_alter_partition_args(const ThriftHiveMetastore_alter_partition_args&); - ThriftHiveMetastore_alter_partition_args& operator=(const ThriftHiveMetastore_alter_partition_args&); - ThriftHiveMetastore_alter_partition_args() : db_name(), tbl_name() { - } - - virtual ~ThriftHiveMetastore_alter_partition_args() noexcept; - std::string db_name; - std::string tbl_name; - Partition new_part; - - _ThriftHiveMetastore_alter_partition_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_new_part(const Partition& val); - - bool operator == (const ThriftHiveMetastore_alter_partition_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(new_part == rhs.new_part)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_partition_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_partition_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_alter_partition_pargs { - public: - - - virtual ~ThriftHiveMetastore_alter_partition_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const Partition* new_part; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_partition_result__isset { - _ThriftHiveMetastore_alter_partition_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_partition_result__isset; - -class ThriftHiveMetastore_alter_partition_result { - public: - - ThriftHiveMetastore_alter_partition_result(const ThriftHiveMetastore_alter_partition_result&); - ThriftHiveMetastore_alter_partition_result& operator=(const ThriftHiveMetastore_alter_partition_result&); - ThriftHiveMetastore_alter_partition_result() { - } - - virtual ~ThriftHiveMetastore_alter_partition_result() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_partition_result__isset __isset; - - void __set_o1(const InvalidOperationException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_alter_partition_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_partition_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_partition_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_partition_presult__isset { - _ThriftHiveMetastore_alter_partition_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_partition_presult__isset; - -class ThriftHiveMetastore_alter_partition_presult { - public: - - - virtual ~ThriftHiveMetastore_alter_partition_presult() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_partition_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_alter_partitions_args__isset { - _ThriftHiveMetastore_alter_partitions_args__isset() : db_name(false), tbl_name(false), new_parts(false) {} - bool db_name :1; - bool tbl_name :1; - bool new_parts :1; -} _ThriftHiveMetastore_alter_partitions_args__isset; - -class ThriftHiveMetastore_alter_partitions_args { - public: - - ThriftHiveMetastore_alter_partitions_args(const ThriftHiveMetastore_alter_partitions_args&); - ThriftHiveMetastore_alter_partitions_args& operator=(const ThriftHiveMetastore_alter_partitions_args&); - ThriftHiveMetastore_alter_partitions_args() : db_name(), tbl_name() { - } - - virtual ~ThriftHiveMetastore_alter_partitions_args() noexcept; - std::string db_name; - std::string tbl_name; - std::vector new_parts; - - _ThriftHiveMetastore_alter_partitions_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_new_parts(const std::vector & val); - - bool operator == (const ThriftHiveMetastore_alter_partitions_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(new_parts == rhs.new_parts)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_partitions_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_partitions_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_alter_partitions_pargs { - public: - - - virtual ~ThriftHiveMetastore_alter_partitions_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::vector * new_parts; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_partitions_result__isset { - _ThriftHiveMetastore_alter_partitions_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_partitions_result__isset; - -class ThriftHiveMetastore_alter_partitions_result { - public: - - ThriftHiveMetastore_alter_partitions_result(const ThriftHiveMetastore_alter_partitions_result&); - ThriftHiveMetastore_alter_partitions_result& operator=(const ThriftHiveMetastore_alter_partitions_result&); - ThriftHiveMetastore_alter_partitions_result() { - } - - virtual ~ThriftHiveMetastore_alter_partitions_result() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_partitions_result__isset __isset; - - void __set_o1(const InvalidOperationException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_alter_partitions_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_partitions_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_partitions_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_partitions_presult__isset { - _ThriftHiveMetastore_alter_partitions_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_partitions_presult__isset; - -class ThriftHiveMetastore_alter_partitions_presult { - public: - - - virtual ~ThriftHiveMetastore_alter_partitions_presult() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_partitions_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_alter_partitions_with_environment_context_args__isset { - _ThriftHiveMetastore_alter_partitions_with_environment_context_args__isset() : db_name(false), tbl_name(false), new_parts(false), environment_context(false) {} - bool db_name :1; - bool tbl_name :1; - bool new_parts :1; - bool environment_context :1; -} _ThriftHiveMetastore_alter_partitions_with_environment_context_args__isset; - -class ThriftHiveMetastore_alter_partitions_with_environment_context_args { - public: - - ThriftHiveMetastore_alter_partitions_with_environment_context_args(const ThriftHiveMetastore_alter_partitions_with_environment_context_args&); - ThriftHiveMetastore_alter_partitions_with_environment_context_args& operator=(const ThriftHiveMetastore_alter_partitions_with_environment_context_args&); - ThriftHiveMetastore_alter_partitions_with_environment_context_args() : db_name(), tbl_name() { - } - - virtual ~ThriftHiveMetastore_alter_partitions_with_environment_context_args() noexcept; - std::string db_name; - std::string tbl_name; - std::vector new_parts; - EnvironmentContext environment_context; - - _ThriftHiveMetastore_alter_partitions_with_environment_context_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_new_parts(const std::vector & val); - - void __set_environment_context(const EnvironmentContext& val); - - bool operator == (const ThriftHiveMetastore_alter_partitions_with_environment_context_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(new_parts == rhs.new_parts)) - return false; - if (!(environment_context == rhs.environment_context)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_partitions_with_environment_context_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_partitions_with_environment_context_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_alter_partitions_with_environment_context_pargs { - public: - - - virtual ~ThriftHiveMetastore_alter_partitions_with_environment_context_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::vector * new_parts; - const EnvironmentContext* environment_context; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_partitions_with_environment_context_result__isset { - _ThriftHiveMetastore_alter_partitions_with_environment_context_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_partitions_with_environment_context_result__isset; - -class ThriftHiveMetastore_alter_partitions_with_environment_context_result { - public: - - ThriftHiveMetastore_alter_partitions_with_environment_context_result(const ThriftHiveMetastore_alter_partitions_with_environment_context_result&); - ThriftHiveMetastore_alter_partitions_with_environment_context_result& operator=(const ThriftHiveMetastore_alter_partitions_with_environment_context_result&); - ThriftHiveMetastore_alter_partitions_with_environment_context_result() { - } - - virtual ~ThriftHiveMetastore_alter_partitions_with_environment_context_result() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_partitions_with_environment_context_result__isset __isset; - - void __set_o1(const InvalidOperationException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_alter_partitions_with_environment_context_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_partitions_with_environment_context_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_partitions_with_environment_context_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_partitions_with_environment_context_presult__isset { - _ThriftHiveMetastore_alter_partitions_with_environment_context_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_partitions_with_environment_context_presult__isset; - -class ThriftHiveMetastore_alter_partitions_with_environment_context_presult { - public: - - - virtual ~ThriftHiveMetastore_alter_partitions_with_environment_context_presult() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_partitions_with_environment_context_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_alter_partition_with_environment_context_args__isset { - _ThriftHiveMetastore_alter_partition_with_environment_context_args__isset() : db_name(false), tbl_name(false), new_part(false), environment_context(false) {} - bool db_name :1; - bool tbl_name :1; - bool new_part :1; - bool environment_context :1; -} _ThriftHiveMetastore_alter_partition_with_environment_context_args__isset; - -class ThriftHiveMetastore_alter_partition_with_environment_context_args { - public: - - ThriftHiveMetastore_alter_partition_with_environment_context_args(const ThriftHiveMetastore_alter_partition_with_environment_context_args&); - ThriftHiveMetastore_alter_partition_with_environment_context_args& operator=(const ThriftHiveMetastore_alter_partition_with_environment_context_args&); - ThriftHiveMetastore_alter_partition_with_environment_context_args() : db_name(), tbl_name() { - } - - virtual ~ThriftHiveMetastore_alter_partition_with_environment_context_args() noexcept; - std::string db_name; - std::string tbl_name; - Partition new_part; - EnvironmentContext environment_context; - - _ThriftHiveMetastore_alter_partition_with_environment_context_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_new_part(const Partition& val); - - void __set_environment_context(const EnvironmentContext& val); - - bool operator == (const ThriftHiveMetastore_alter_partition_with_environment_context_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(new_part == rhs.new_part)) - return false; - if (!(environment_context == rhs.environment_context)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_partition_with_environment_context_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_partition_with_environment_context_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_alter_partition_with_environment_context_pargs { - public: - - - virtual ~ThriftHiveMetastore_alter_partition_with_environment_context_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const Partition* new_part; - const EnvironmentContext* environment_context; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_partition_with_environment_context_result__isset { - _ThriftHiveMetastore_alter_partition_with_environment_context_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_partition_with_environment_context_result__isset; - -class ThriftHiveMetastore_alter_partition_with_environment_context_result { - public: - - ThriftHiveMetastore_alter_partition_with_environment_context_result(const ThriftHiveMetastore_alter_partition_with_environment_context_result&); - ThriftHiveMetastore_alter_partition_with_environment_context_result& operator=(const ThriftHiveMetastore_alter_partition_with_environment_context_result&); - ThriftHiveMetastore_alter_partition_with_environment_context_result() { - } - - virtual ~ThriftHiveMetastore_alter_partition_with_environment_context_result() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_partition_with_environment_context_result__isset __isset; - - void __set_o1(const InvalidOperationException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_alter_partition_with_environment_context_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_partition_with_environment_context_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_partition_with_environment_context_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_partition_with_environment_context_presult__isset { - _ThriftHiveMetastore_alter_partition_with_environment_context_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_partition_with_environment_context_presult__isset; - -class ThriftHiveMetastore_alter_partition_with_environment_context_presult { - public: - - - virtual ~ThriftHiveMetastore_alter_partition_with_environment_context_presult() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_partition_with_environment_context_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_rename_partition_args__isset { - _ThriftHiveMetastore_rename_partition_args__isset() : db_name(false), tbl_name(false), part_vals(false), new_part(false) {} - bool db_name :1; - bool tbl_name :1; - bool part_vals :1; - bool new_part :1; -} _ThriftHiveMetastore_rename_partition_args__isset; - -class ThriftHiveMetastore_rename_partition_args { - public: - - ThriftHiveMetastore_rename_partition_args(const ThriftHiveMetastore_rename_partition_args&); - ThriftHiveMetastore_rename_partition_args& operator=(const ThriftHiveMetastore_rename_partition_args&); - ThriftHiveMetastore_rename_partition_args() : db_name(), tbl_name() { - } - - virtual ~ThriftHiveMetastore_rename_partition_args() noexcept; - std::string db_name; - std::string tbl_name; - std::vector part_vals; - Partition new_part; - - _ThriftHiveMetastore_rename_partition_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_vals(const std::vector & val); - - void __set_new_part(const Partition& val); - - bool operator == (const ThriftHiveMetastore_rename_partition_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_vals == rhs.part_vals)) - return false; - if (!(new_part == rhs.new_part)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_rename_partition_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_rename_partition_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_rename_partition_pargs { - public: - - - virtual ~ThriftHiveMetastore_rename_partition_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::vector * part_vals; - const Partition* new_part; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_rename_partition_result__isset { - _ThriftHiveMetastore_rename_partition_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_rename_partition_result__isset; - -class ThriftHiveMetastore_rename_partition_result { - public: - - ThriftHiveMetastore_rename_partition_result(const ThriftHiveMetastore_rename_partition_result&); - ThriftHiveMetastore_rename_partition_result& operator=(const ThriftHiveMetastore_rename_partition_result&); - ThriftHiveMetastore_rename_partition_result() { - } - - virtual ~ThriftHiveMetastore_rename_partition_result() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_rename_partition_result__isset __isset; - - void __set_o1(const InvalidOperationException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_rename_partition_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_rename_partition_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_rename_partition_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_rename_partition_presult__isset { - _ThriftHiveMetastore_rename_partition_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_rename_partition_presult__isset; - -class ThriftHiveMetastore_rename_partition_presult { - public: - - - virtual ~ThriftHiveMetastore_rename_partition_presult() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_rename_partition_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_partition_name_has_valid_characters_args__isset { - _ThriftHiveMetastore_partition_name_has_valid_characters_args__isset() : part_vals(false), throw_exception(false) {} - bool part_vals :1; - bool throw_exception :1; -} _ThriftHiveMetastore_partition_name_has_valid_characters_args__isset; - -class ThriftHiveMetastore_partition_name_has_valid_characters_args { - public: - - ThriftHiveMetastore_partition_name_has_valid_characters_args(const ThriftHiveMetastore_partition_name_has_valid_characters_args&); - ThriftHiveMetastore_partition_name_has_valid_characters_args& operator=(const ThriftHiveMetastore_partition_name_has_valid_characters_args&); - ThriftHiveMetastore_partition_name_has_valid_characters_args() : throw_exception(0) { - } - - virtual ~ThriftHiveMetastore_partition_name_has_valid_characters_args() noexcept; - std::vector part_vals; - bool throw_exception; - - _ThriftHiveMetastore_partition_name_has_valid_characters_args__isset __isset; - - void __set_part_vals(const std::vector & val); - - void __set_throw_exception(const bool val); - - bool operator == (const ThriftHiveMetastore_partition_name_has_valid_characters_args & rhs) const - { - if (!(part_vals == rhs.part_vals)) - return false; - if (!(throw_exception == rhs.throw_exception)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_partition_name_has_valid_characters_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_partition_name_has_valid_characters_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_partition_name_has_valid_characters_pargs { - public: - - - virtual ~ThriftHiveMetastore_partition_name_has_valid_characters_pargs() noexcept; - const std::vector * part_vals; - const bool* throw_exception; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_partition_name_has_valid_characters_result__isset { - _ThriftHiveMetastore_partition_name_has_valid_characters_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_partition_name_has_valid_characters_result__isset; - -class ThriftHiveMetastore_partition_name_has_valid_characters_result { - public: - - ThriftHiveMetastore_partition_name_has_valid_characters_result(const ThriftHiveMetastore_partition_name_has_valid_characters_result&); - ThriftHiveMetastore_partition_name_has_valid_characters_result& operator=(const ThriftHiveMetastore_partition_name_has_valid_characters_result&); - ThriftHiveMetastore_partition_name_has_valid_characters_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_partition_name_has_valid_characters_result() noexcept; - bool success; - MetaException o1; - - _ThriftHiveMetastore_partition_name_has_valid_characters_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_partition_name_has_valid_characters_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_partition_name_has_valid_characters_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_partition_name_has_valid_characters_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_partition_name_has_valid_characters_presult__isset { - _ThriftHiveMetastore_partition_name_has_valid_characters_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_partition_name_has_valid_characters_presult__isset; - -class ThriftHiveMetastore_partition_name_has_valid_characters_presult { - public: - - - virtual ~ThriftHiveMetastore_partition_name_has_valid_characters_presult() noexcept; - bool* success; - MetaException o1; - - _ThriftHiveMetastore_partition_name_has_valid_characters_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_config_value_args__isset { - _ThriftHiveMetastore_get_config_value_args__isset() : name(false), defaultValue(false) {} - bool name :1; - bool defaultValue :1; -} _ThriftHiveMetastore_get_config_value_args__isset; - -class ThriftHiveMetastore_get_config_value_args { - public: - - ThriftHiveMetastore_get_config_value_args(const ThriftHiveMetastore_get_config_value_args&); - ThriftHiveMetastore_get_config_value_args& operator=(const ThriftHiveMetastore_get_config_value_args&); - ThriftHiveMetastore_get_config_value_args() : name(), defaultValue() { - } - - virtual ~ThriftHiveMetastore_get_config_value_args() noexcept; - std::string name; - std::string defaultValue; - - _ThriftHiveMetastore_get_config_value_args__isset __isset; - - void __set_name(const std::string& val); - - void __set_defaultValue(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_config_value_args & rhs) const - { - if (!(name == rhs.name)) - return false; - if (!(defaultValue == rhs.defaultValue)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_config_value_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_config_value_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_config_value_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_config_value_pargs() noexcept; - const std::string* name; - const std::string* defaultValue; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_config_value_result__isset { - _ThriftHiveMetastore_get_config_value_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_config_value_result__isset; - -class ThriftHiveMetastore_get_config_value_result { - public: - - ThriftHiveMetastore_get_config_value_result(const ThriftHiveMetastore_get_config_value_result&); - ThriftHiveMetastore_get_config_value_result& operator=(const ThriftHiveMetastore_get_config_value_result&); - ThriftHiveMetastore_get_config_value_result() : success() { - } - - virtual ~ThriftHiveMetastore_get_config_value_result() noexcept; - std::string success; - ConfigValSecurityException o1; - - _ThriftHiveMetastore_get_config_value_result__isset __isset; - - void __set_success(const std::string& val); - - void __set_o1(const ConfigValSecurityException& val); - - bool operator == (const ThriftHiveMetastore_get_config_value_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_config_value_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_config_value_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_config_value_presult__isset { - _ThriftHiveMetastore_get_config_value_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_config_value_presult__isset; - -class ThriftHiveMetastore_get_config_value_presult { - public: - - - virtual ~ThriftHiveMetastore_get_config_value_presult() noexcept; - std::string* success; - ConfigValSecurityException o1; - - _ThriftHiveMetastore_get_config_value_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_partition_name_to_vals_args__isset { - _ThriftHiveMetastore_partition_name_to_vals_args__isset() : part_name(false) {} - bool part_name :1; -} _ThriftHiveMetastore_partition_name_to_vals_args__isset; - -class ThriftHiveMetastore_partition_name_to_vals_args { - public: - - ThriftHiveMetastore_partition_name_to_vals_args(const ThriftHiveMetastore_partition_name_to_vals_args&); - ThriftHiveMetastore_partition_name_to_vals_args& operator=(const ThriftHiveMetastore_partition_name_to_vals_args&); - ThriftHiveMetastore_partition_name_to_vals_args() : part_name() { - } - - virtual ~ThriftHiveMetastore_partition_name_to_vals_args() noexcept; - std::string part_name; - - _ThriftHiveMetastore_partition_name_to_vals_args__isset __isset; - - void __set_part_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_partition_name_to_vals_args & rhs) const - { - if (!(part_name == rhs.part_name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_partition_name_to_vals_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_partition_name_to_vals_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_partition_name_to_vals_pargs { - public: - - - virtual ~ThriftHiveMetastore_partition_name_to_vals_pargs() noexcept; - const std::string* part_name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_partition_name_to_vals_result__isset { - _ThriftHiveMetastore_partition_name_to_vals_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_partition_name_to_vals_result__isset; - -class ThriftHiveMetastore_partition_name_to_vals_result { - public: - - ThriftHiveMetastore_partition_name_to_vals_result(const ThriftHiveMetastore_partition_name_to_vals_result&); - ThriftHiveMetastore_partition_name_to_vals_result& operator=(const ThriftHiveMetastore_partition_name_to_vals_result&); - ThriftHiveMetastore_partition_name_to_vals_result() { - } - - virtual ~ThriftHiveMetastore_partition_name_to_vals_result() noexcept; - std::vector success; - MetaException o1; - - _ThriftHiveMetastore_partition_name_to_vals_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_partition_name_to_vals_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_partition_name_to_vals_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_partition_name_to_vals_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_partition_name_to_vals_presult__isset { - _ThriftHiveMetastore_partition_name_to_vals_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_partition_name_to_vals_presult__isset; - -class ThriftHiveMetastore_partition_name_to_vals_presult { - public: - - - virtual ~ThriftHiveMetastore_partition_name_to_vals_presult() noexcept; - std::vector * success; - MetaException o1; - - _ThriftHiveMetastore_partition_name_to_vals_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_partition_name_to_spec_args__isset { - _ThriftHiveMetastore_partition_name_to_spec_args__isset() : part_name(false) {} - bool part_name :1; -} _ThriftHiveMetastore_partition_name_to_spec_args__isset; - -class ThriftHiveMetastore_partition_name_to_spec_args { - public: - - ThriftHiveMetastore_partition_name_to_spec_args(const ThriftHiveMetastore_partition_name_to_spec_args&); - ThriftHiveMetastore_partition_name_to_spec_args& operator=(const ThriftHiveMetastore_partition_name_to_spec_args&); - ThriftHiveMetastore_partition_name_to_spec_args() : part_name() { - } - - virtual ~ThriftHiveMetastore_partition_name_to_spec_args() noexcept; - std::string part_name; - - _ThriftHiveMetastore_partition_name_to_spec_args__isset __isset; - - void __set_part_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_partition_name_to_spec_args & rhs) const - { - if (!(part_name == rhs.part_name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_partition_name_to_spec_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_partition_name_to_spec_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_partition_name_to_spec_pargs { - public: - - - virtual ~ThriftHiveMetastore_partition_name_to_spec_pargs() noexcept; - const std::string* part_name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_partition_name_to_spec_result__isset { - _ThriftHiveMetastore_partition_name_to_spec_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_partition_name_to_spec_result__isset; - -class ThriftHiveMetastore_partition_name_to_spec_result { - public: - - ThriftHiveMetastore_partition_name_to_spec_result(const ThriftHiveMetastore_partition_name_to_spec_result&); - ThriftHiveMetastore_partition_name_to_spec_result& operator=(const ThriftHiveMetastore_partition_name_to_spec_result&); - ThriftHiveMetastore_partition_name_to_spec_result() { - } - - virtual ~ThriftHiveMetastore_partition_name_to_spec_result() noexcept; - std::map success; - MetaException o1; - - _ThriftHiveMetastore_partition_name_to_spec_result__isset __isset; - - void __set_success(const std::map & val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_partition_name_to_spec_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_partition_name_to_spec_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_partition_name_to_spec_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_partition_name_to_spec_presult__isset { - _ThriftHiveMetastore_partition_name_to_spec_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_partition_name_to_spec_presult__isset; - -class ThriftHiveMetastore_partition_name_to_spec_presult { - public: - - - virtual ~ThriftHiveMetastore_partition_name_to_spec_presult() noexcept; - std::map * success; - MetaException o1; - - _ThriftHiveMetastore_partition_name_to_spec_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_markPartitionForEvent_args__isset { - _ThriftHiveMetastore_markPartitionForEvent_args__isset() : db_name(false), tbl_name(false), part_vals(false), eventType(false) {} - bool db_name :1; - bool tbl_name :1; - bool part_vals :1; - bool eventType :1; -} _ThriftHiveMetastore_markPartitionForEvent_args__isset; - -class ThriftHiveMetastore_markPartitionForEvent_args { - public: - - ThriftHiveMetastore_markPartitionForEvent_args(const ThriftHiveMetastore_markPartitionForEvent_args&); - ThriftHiveMetastore_markPartitionForEvent_args& operator=(const ThriftHiveMetastore_markPartitionForEvent_args&); - ThriftHiveMetastore_markPartitionForEvent_args() : db_name(), tbl_name(), eventType((PartitionEventType::type)0) { - } - - virtual ~ThriftHiveMetastore_markPartitionForEvent_args() noexcept; - std::string db_name; - std::string tbl_name; - std::map part_vals; - PartitionEventType::type eventType; - - _ThriftHiveMetastore_markPartitionForEvent_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_vals(const std::map & val); - - void __set_eventType(const PartitionEventType::type val); - - bool operator == (const ThriftHiveMetastore_markPartitionForEvent_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_vals == rhs.part_vals)) - return false; - if (!(eventType == rhs.eventType)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_markPartitionForEvent_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_markPartitionForEvent_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_markPartitionForEvent_pargs { - public: - - - virtual ~ThriftHiveMetastore_markPartitionForEvent_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::map * part_vals; - const PartitionEventType::type* eventType; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_markPartitionForEvent_result__isset { - _ThriftHiveMetastore_markPartitionForEvent_result__isset() : o1(false), o2(false), o3(false), o4(false), o5(false), o6(false) {} - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; - bool o5 :1; - bool o6 :1; -} _ThriftHiveMetastore_markPartitionForEvent_result__isset; - -class ThriftHiveMetastore_markPartitionForEvent_result { - public: - - ThriftHiveMetastore_markPartitionForEvent_result(const ThriftHiveMetastore_markPartitionForEvent_result&); - ThriftHiveMetastore_markPartitionForEvent_result& operator=(const ThriftHiveMetastore_markPartitionForEvent_result&); - ThriftHiveMetastore_markPartitionForEvent_result() { - } - - virtual ~ThriftHiveMetastore_markPartitionForEvent_result() noexcept; - MetaException o1; - NoSuchObjectException o2; - UnknownDBException o3; - UnknownTableException o4; - UnknownPartitionException o5; - InvalidPartitionException o6; - - _ThriftHiveMetastore_markPartitionForEvent_result__isset __isset; - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - void __set_o3(const UnknownDBException& val); - - void __set_o4(const UnknownTableException& val); - - void __set_o5(const UnknownPartitionException& val); - - void __set_o6(const InvalidPartitionException& val); - - bool operator == (const ThriftHiveMetastore_markPartitionForEvent_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - if (!(o4 == rhs.o4)) - return false; - if (!(o5 == rhs.o5)) - return false; - if (!(o6 == rhs.o6)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_markPartitionForEvent_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_markPartitionForEvent_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_markPartitionForEvent_presult__isset { - _ThriftHiveMetastore_markPartitionForEvent_presult__isset() : o1(false), o2(false), o3(false), o4(false), o5(false), o6(false) {} - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; - bool o5 :1; - bool o6 :1; -} _ThriftHiveMetastore_markPartitionForEvent_presult__isset; - -class ThriftHiveMetastore_markPartitionForEvent_presult { - public: - - - virtual ~ThriftHiveMetastore_markPartitionForEvent_presult() noexcept; - MetaException o1; - NoSuchObjectException o2; - UnknownDBException o3; - UnknownTableException o4; - UnknownPartitionException o5; - InvalidPartitionException o6; - - _ThriftHiveMetastore_markPartitionForEvent_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_isPartitionMarkedForEvent_args__isset { - _ThriftHiveMetastore_isPartitionMarkedForEvent_args__isset() : db_name(false), tbl_name(false), part_vals(false), eventType(false) {} - bool db_name :1; - bool tbl_name :1; - bool part_vals :1; - bool eventType :1; -} _ThriftHiveMetastore_isPartitionMarkedForEvent_args__isset; - -class ThriftHiveMetastore_isPartitionMarkedForEvent_args { - public: - - ThriftHiveMetastore_isPartitionMarkedForEvent_args(const ThriftHiveMetastore_isPartitionMarkedForEvent_args&); - ThriftHiveMetastore_isPartitionMarkedForEvent_args& operator=(const ThriftHiveMetastore_isPartitionMarkedForEvent_args&); - ThriftHiveMetastore_isPartitionMarkedForEvent_args() : db_name(), tbl_name(), eventType((PartitionEventType::type)0) { - } - - virtual ~ThriftHiveMetastore_isPartitionMarkedForEvent_args() noexcept; - std::string db_name; - std::string tbl_name; - std::map part_vals; - PartitionEventType::type eventType; - - _ThriftHiveMetastore_isPartitionMarkedForEvent_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_vals(const std::map & val); - - void __set_eventType(const PartitionEventType::type val); - - bool operator == (const ThriftHiveMetastore_isPartitionMarkedForEvent_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_vals == rhs.part_vals)) - return false; - if (!(eventType == rhs.eventType)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_isPartitionMarkedForEvent_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_isPartitionMarkedForEvent_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_isPartitionMarkedForEvent_pargs { - public: - - - virtual ~ThriftHiveMetastore_isPartitionMarkedForEvent_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::map * part_vals; - const PartitionEventType::type* eventType; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_isPartitionMarkedForEvent_result__isset { - _ThriftHiveMetastore_isPartitionMarkedForEvent_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false), o5(false), o6(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; - bool o5 :1; - bool o6 :1; -} _ThriftHiveMetastore_isPartitionMarkedForEvent_result__isset; - -class ThriftHiveMetastore_isPartitionMarkedForEvent_result { - public: - - ThriftHiveMetastore_isPartitionMarkedForEvent_result(const ThriftHiveMetastore_isPartitionMarkedForEvent_result&); - ThriftHiveMetastore_isPartitionMarkedForEvent_result& operator=(const ThriftHiveMetastore_isPartitionMarkedForEvent_result&); - ThriftHiveMetastore_isPartitionMarkedForEvent_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_isPartitionMarkedForEvent_result() noexcept; - bool success; - MetaException o1; - NoSuchObjectException o2; - UnknownDBException o3; - UnknownTableException o4; - UnknownPartitionException o5; - InvalidPartitionException o6; - - _ThriftHiveMetastore_isPartitionMarkedForEvent_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - void __set_o3(const UnknownDBException& val); - - void __set_o4(const UnknownTableException& val); - - void __set_o5(const UnknownPartitionException& val); - - void __set_o6(const InvalidPartitionException& val); - - bool operator == (const ThriftHiveMetastore_isPartitionMarkedForEvent_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - if (!(o4 == rhs.o4)) - return false; - if (!(o5 == rhs.o5)) - return false; - if (!(o6 == rhs.o6)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_isPartitionMarkedForEvent_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_isPartitionMarkedForEvent_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_isPartitionMarkedForEvent_presult__isset { - _ThriftHiveMetastore_isPartitionMarkedForEvent_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false), o5(false), o6(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; - bool o5 :1; - bool o6 :1; -} _ThriftHiveMetastore_isPartitionMarkedForEvent_presult__isset; - -class ThriftHiveMetastore_isPartitionMarkedForEvent_presult { - public: - - - virtual ~ThriftHiveMetastore_isPartitionMarkedForEvent_presult() noexcept; - bool* success; - MetaException o1; - NoSuchObjectException o2; - UnknownDBException o3; - UnknownTableException o4; - UnknownPartitionException o5; - InvalidPartitionException o6; - - _ThriftHiveMetastore_isPartitionMarkedForEvent_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_add_index_args__isset { - _ThriftHiveMetastore_add_index_args__isset() : new_index(false), index_table(false) {} - bool new_index :1; - bool index_table :1; -} _ThriftHiveMetastore_add_index_args__isset; - -class ThriftHiveMetastore_add_index_args { - public: - - ThriftHiveMetastore_add_index_args(const ThriftHiveMetastore_add_index_args&); - ThriftHiveMetastore_add_index_args& operator=(const ThriftHiveMetastore_add_index_args&); - ThriftHiveMetastore_add_index_args() { - } - - virtual ~ThriftHiveMetastore_add_index_args() noexcept; - Index new_index; - Table index_table; - - _ThriftHiveMetastore_add_index_args__isset __isset; - - void __set_new_index(const Index& val); - - void __set_index_table(const Table& val); - - bool operator == (const ThriftHiveMetastore_add_index_args & rhs) const - { - if (!(new_index == rhs.new_index)) - return false; - if (!(index_table == rhs.index_table)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_index_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_index_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_add_index_pargs { - public: - - - virtual ~ThriftHiveMetastore_add_index_pargs() noexcept; - const Index* new_index; - const Table* index_table; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_index_result__isset { - _ThriftHiveMetastore_add_index_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_add_index_result__isset; - -class ThriftHiveMetastore_add_index_result { - public: - - ThriftHiveMetastore_add_index_result(const ThriftHiveMetastore_add_index_result&); - ThriftHiveMetastore_add_index_result& operator=(const ThriftHiveMetastore_add_index_result&); - ThriftHiveMetastore_add_index_result() { - } - - virtual ~ThriftHiveMetastore_add_index_result() noexcept; - Index success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_add_index_result__isset __isset; - - void __set_success(const Index& val); - - void __set_o1(const InvalidObjectException& val); - - void __set_o2(const AlreadyExistsException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_add_index_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_index_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_index_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_index_presult__isset { - _ThriftHiveMetastore_add_index_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_add_index_presult__isset; - -class ThriftHiveMetastore_add_index_presult { - public: - - - virtual ~ThriftHiveMetastore_add_index_presult() noexcept; - Index* success; - InvalidObjectException o1; - AlreadyExistsException o2; - MetaException o3; - - _ThriftHiveMetastore_add_index_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_alter_index_args__isset { - _ThriftHiveMetastore_alter_index_args__isset() : dbname(false), base_tbl_name(false), idx_name(false), new_idx(false) {} - bool dbname :1; - bool base_tbl_name :1; - bool idx_name :1; - bool new_idx :1; -} _ThriftHiveMetastore_alter_index_args__isset; - -class ThriftHiveMetastore_alter_index_args { - public: - - ThriftHiveMetastore_alter_index_args(const ThriftHiveMetastore_alter_index_args&); - ThriftHiveMetastore_alter_index_args& operator=(const ThriftHiveMetastore_alter_index_args&); - ThriftHiveMetastore_alter_index_args() : dbname(), base_tbl_name(), idx_name() { - } - - virtual ~ThriftHiveMetastore_alter_index_args() noexcept; - std::string dbname; - std::string base_tbl_name; - std::string idx_name; - Index new_idx; - - _ThriftHiveMetastore_alter_index_args__isset __isset; - - void __set_dbname(const std::string& val); - - void __set_base_tbl_name(const std::string& val); - - void __set_idx_name(const std::string& val); - - void __set_new_idx(const Index& val); - - bool operator == (const ThriftHiveMetastore_alter_index_args & rhs) const - { - if (!(dbname == rhs.dbname)) - return false; - if (!(base_tbl_name == rhs.base_tbl_name)) - return false; - if (!(idx_name == rhs.idx_name)) - return false; - if (!(new_idx == rhs.new_idx)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_index_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_index_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_alter_index_pargs { - public: - - - virtual ~ThriftHiveMetastore_alter_index_pargs() noexcept; - const std::string* dbname; - const std::string* base_tbl_name; - const std::string* idx_name; - const Index* new_idx; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_index_result__isset { - _ThriftHiveMetastore_alter_index_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_index_result__isset; - -class ThriftHiveMetastore_alter_index_result { - public: - - ThriftHiveMetastore_alter_index_result(const ThriftHiveMetastore_alter_index_result&); - ThriftHiveMetastore_alter_index_result& operator=(const ThriftHiveMetastore_alter_index_result&); - ThriftHiveMetastore_alter_index_result() { - } - - virtual ~ThriftHiveMetastore_alter_index_result() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_index_result__isset __isset; - - void __set_o1(const InvalidOperationException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_alter_index_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_index_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_index_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_index_presult__isset { - _ThriftHiveMetastore_alter_index_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_index_presult__isset; - -class ThriftHiveMetastore_alter_index_presult { - public: - - - virtual ~ThriftHiveMetastore_alter_index_presult() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_index_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_drop_index_by_name_args__isset { - _ThriftHiveMetastore_drop_index_by_name_args__isset() : db_name(false), tbl_name(false), index_name(false), deleteData(false) {} - bool db_name :1; - bool tbl_name :1; - bool index_name :1; - bool deleteData :1; -} _ThriftHiveMetastore_drop_index_by_name_args__isset; - -class ThriftHiveMetastore_drop_index_by_name_args { - public: - - ThriftHiveMetastore_drop_index_by_name_args(const ThriftHiveMetastore_drop_index_by_name_args&); - ThriftHiveMetastore_drop_index_by_name_args& operator=(const ThriftHiveMetastore_drop_index_by_name_args&); - ThriftHiveMetastore_drop_index_by_name_args() : db_name(), tbl_name(), index_name(), deleteData(0) { - } - - virtual ~ThriftHiveMetastore_drop_index_by_name_args() noexcept; - std::string db_name; - std::string tbl_name; - std::string index_name; - bool deleteData; - - _ThriftHiveMetastore_drop_index_by_name_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_index_name(const std::string& val); - - void __set_deleteData(const bool val); - - bool operator == (const ThriftHiveMetastore_drop_index_by_name_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(index_name == rhs.index_name)) - return false; - if (!(deleteData == rhs.deleteData)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_index_by_name_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_index_by_name_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_drop_index_by_name_pargs { - public: - - - virtual ~ThriftHiveMetastore_drop_index_by_name_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::string* index_name; - const bool* deleteData; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_index_by_name_result__isset { - _ThriftHiveMetastore_drop_index_by_name_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_drop_index_by_name_result__isset; - -class ThriftHiveMetastore_drop_index_by_name_result { - public: - - ThriftHiveMetastore_drop_index_by_name_result(const ThriftHiveMetastore_drop_index_by_name_result&); - ThriftHiveMetastore_drop_index_by_name_result& operator=(const ThriftHiveMetastore_drop_index_by_name_result&); - ThriftHiveMetastore_drop_index_by_name_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_drop_index_by_name_result() noexcept; - bool success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_drop_index_by_name_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_drop_index_by_name_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_index_by_name_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_index_by_name_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_index_by_name_presult__isset { - _ThriftHiveMetastore_drop_index_by_name_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_drop_index_by_name_presult__isset; - -class ThriftHiveMetastore_drop_index_by_name_presult { - public: - - - virtual ~ThriftHiveMetastore_drop_index_by_name_presult() noexcept; - bool* success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_drop_index_by_name_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_index_by_name_args__isset { - _ThriftHiveMetastore_get_index_by_name_args__isset() : db_name(false), tbl_name(false), index_name(false) {} - bool db_name :1; - bool tbl_name :1; - bool index_name :1; -} _ThriftHiveMetastore_get_index_by_name_args__isset; - -class ThriftHiveMetastore_get_index_by_name_args { - public: - - ThriftHiveMetastore_get_index_by_name_args(const ThriftHiveMetastore_get_index_by_name_args&); - ThriftHiveMetastore_get_index_by_name_args& operator=(const ThriftHiveMetastore_get_index_by_name_args&); - ThriftHiveMetastore_get_index_by_name_args() : db_name(), tbl_name(), index_name() { - } - - virtual ~ThriftHiveMetastore_get_index_by_name_args() noexcept; - std::string db_name; - std::string tbl_name; - std::string index_name; - - _ThriftHiveMetastore_get_index_by_name_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_index_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_index_by_name_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(index_name == rhs.index_name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_index_by_name_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_index_by_name_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_index_by_name_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_index_by_name_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::string* index_name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_index_by_name_result__isset { - _ThriftHiveMetastore_get_index_by_name_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_index_by_name_result__isset; - -class ThriftHiveMetastore_get_index_by_name_result { - public: - - ThriftHiveMetastore_get_index_by_name_result(const ThriftHiveMetastore_get_index_by_name_result&); - ThriftHiveMetastore_get_index_by_name_result& operator=(const ThriftHiveMetastore_get_index_by_name_result&); - ThriftHiveMetastore_get_index_by_name_result() { - } - - virtual ~ThriftHiveMetastore_get_index_by_name_result() noexcept; - Index success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_index_by_name_result__isset __isset; - - void __set_success(const Index& val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_index_by_name_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_index_by_name_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_index_by_name_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_index_by_name_presult__isset { - _ThriftHiveMetastore_get_index_by_name_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_index_by_name_presult__isset; - -class ThriftHiveMetastore_get_index_by_name_presult { - public: - - - virtual ~ThriftHiveMetastore_get_index_by_name_presult() noexcept; - Index* success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_index_by_name_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_indexes_args__isset { - _ThriftHiveMetastore_get_indexes_args__isset() : db_name(false), tbl_name(false), max_indexes(true) {} - bool db_name :1; - bool tbl_name :1; - bool max_indexes :1; -} _ThriftHiveMetastore_get_indexes_args__isset; - -class ThriftHiveMetastore_get_indexes_args { - public: - - ThriftHiveMetastore_get_indexes_args(const ThriftHiveMetastore_get_indexes_args&); - ThriftHiveMetastore_get_indexes_args& operator=(const ThriftHiveMetastore_get_indexes_args&); - ThriftHiveMetastore_get_indexes_args() : db_name(), tbl_name(), max_indexes(-1) { - } - - virtual ~ThriftHiveMetastore_get_indexes_args() noexcept; - std::string db_name; - std::string tbl_name; - int16_t max_indexes; - - _ThriftHiveMetastore_get_indexes_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_max_indexes(const int16_t val); - - bool operator == (const ThriftHiveMetastore_get_indexes_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(max_indexes == rhs.max_indexes)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_indexes_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_indexes_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_indexes_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_indexes_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const int16_t* max_indexes; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_indexes_result__isset { - _ThriftHiveMetastore_get_indexes_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_indexes_result__isset; - -class ThriftHiveMetastore_get_indexes_result { - public: - - ThriftHiveMetastore_get_indexes_result(const ThriftHiveMetastore_get_indexes_result&); - ThriftHiveMetastore_get_indexes_result& operator=(const ThriftHiveMetastore_get_indexes_result&); - ThriftHiveMetastore_get_indexes_result() { - } - - virtual ~ThriftHiveMetastore_get_indexes_result() noexcept; - std::vector success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_indexes_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_indexes_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_indexes_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_indexes_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_indexes_presult__isset { - _ThriftHiveMetastore_get_indexes_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_indexes_presult__isset; - -class ThriftHiveMetastore_get_indexes_presult { - public: - - - virtual ~ThriftHiveMetastore_get_indexes_presult() noexcept; - std::vector * success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_indexes_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_index_names_args__isset { - _ThriftHiveMetastore_get_index_names_args__isset() : db_name(false), tbl_name(false), max_indexes(true) {} - bool db_name :1; - bool tbl_name :1; - bool max_indexes :1; -} _ThriftHiveMetastore_get_index_names_args__isset; - -class ThriftHiveMetastore_get_index_names_args { - public: - - ThriftHiveMetastore_get_index_names_args(const ThriftHiveMetastore_get_index_names_args&); - ThriftHiveMetastore_get_index_names_args& operator=(const ThriftHiveMetastore_get_index_names_args&); - ThriftHiveMetastore_get_index_names_args() : db_name(), tbl_name(), max_indexes(-1) { - } - - virtual ~ThriftHiveMetastore_get_index_names_args() noexcept; - std::string db_name; - std::string tbl_name; - int16_t max_indexes; - - _ThriftHiveMetastore_get_index_names_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_max_indexes(const int16_t val); - - bool operator == (const ThriftHiveMetastore_get_index_names_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(max_indexes == rhs.max_indexes)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_index_names_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_index_names_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_index_names_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_index_names_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const int16_t* max_indexes; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_index_names_result__isset { - _ThriftHiveMetastore_get_index_names_result__isset() : success(false), o2(false) {} - bool success :1; - bool o2 :1; -} _ThriftHiveMetastore_get_index_names_result__isset; - -class ThriftHiveMetastore_get_index_names_result { - public: - - ThriftHiveMetastore_get_index_names_result(const ThriftHiveMetastore_get_index_names_result&); - ThriftHiveMetastore_get_index_names_result& operator=(const ThriftHiveMetastore_get_index_names_result&); - ThriftHiveMetastore_get_index_names_result() { - } - - virtual ~ThriftHiveMetastore_get_index_names_result() noexcept; - std::vector success; - MetaException o2; - - _ThriftHiveMetastore_get_index_names_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_index_names_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_index_names_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_index_names_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_index_names_presult__isset { - _ThriftHiveMetastore_get_index_names_presult__isset() : success(false), o2(false) {} - bool success :1; - bool o2 :1; -} _ThriftHiveMetastore_get_index_names_presult__isset; - -class ThriftHiveMetastore_get_index_names_presult { - public: - - - virtual ~ThriftHiveMetastore_get_index_names_presult() noexcept; - std::vector * success; - MetaException o2; - - _ThriftHiveMetastore_get_index_names_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_primary_keys_args__isset { - _ThriftHiveMetastore_get_primary_keys_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_get_primary_keys_args__isset; - -class ThriftHiveMetastore_get_primary_keys_args { - public: - - ThriftHiveMetastore_get_primary_keys_args(const ThriftHiveMetastore_get_primary_keys_args&); - ThriftHiveMetastore_get_primary_keys_args& operator=(const ThriftHiveMetastore_get_primary_keys_args&); - ThriftHiveMetastore_get_primary_keys_args() { - } - - virtual ~ThriftHiveMetastore_get_primary_keys_args() noexcept; - PrimaryKeysRequest request; - - _ThriftHiveMetastore_get_primary_keys_args__isset __isset; - - void __set_request(const PrimaryKeysRequest& val); - - bool operator == (const ThriftHiveMetastore_get_primary_keys_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_primary_keys_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_primary_keys_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_primary_keys_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_primary_keys_pargs() noexcept; - const PrimaryKeysRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_primary_keys_result__isset { - _ThriftHiveMetastore_get_primary_keys_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_primary_keys_result__isset; - -class ThriftHiveMetastore_get_primary_keys_result { - public: - - ThriftHiveMetastore_get_primary_keys_result(const ThriftHiveMetastore_get_primary_keys_result&); - ThriftHiveMetastore_get_primary_keys_result& operator=(const ThriftHiveMetastore_get_primary_keys_result&); - ThriftHiveMetastore_get_primary_keys_result() { - } - - virtual ~ThriftHiveMetastore_get_primary_keys_result() noexcept; - PrimaryKeysResponse success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_primary_keys_result__isset __isset; - - void __set_success(const PrimaryKeysResponse& val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_primary_keys_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_primary_keys_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_primary_keys_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_primary_keys_presult__isset { - _ThriftHiveMetastore_get_primary_keys_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_primary_keys_presult__isset; - -class ThriftHiveMetastore_get_primary_keys_presult { - public: - - - virtual ~ThriftHiveMetastore_get_primary_keys_presult() noexcept; - PrimaryKeysResponse* success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_primary_keys_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_foreign_keys_args__isset { - _ThriftHiveMetastore_get_foreign_keys_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_get_foreign_keys_args__isset; - -class ThriftHiveMetastore_get_foreign_keys_args { - public: - - ThriftHiveMetastore_get_foreign_keys_args(const ThriftHiveMetastore_get_foreign_keys_args&); - ThriftHiveMetastore_get_foreign_keys_args& operator=(const ThriftHiveMetastore_get_foreign_keys_args&); - ThriftHiveMetastore_get_foreign_keys_args() { - } - - virtual ~ThriftHiveMetastore_get_foreign_keys_args() noexcept; - ForeignKeysRequest request; - - _ThriftHiveMetastore_get_foreign_keys_args__isset __isset; - - void __set_request(const ForeignKeysRequest& val); - - bool operator == (const ThriftHiveMetastore_get_foreign_keys_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_foreign_keys_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_foreign_keys_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_foreign_keys_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_foreign_keys_pargs() noexcept; - const ForeignKeysRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_foreign_keys_result__isset { - _ThriftHiveMetastore_get_foreign_keys_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_foreign_keys_result__isset; - -class ThriftHiveMetastore_get_foreign_keys_result { - public: - - ThriftHiveMetastore_get_foreign_keys_result(const ThriftHiveMetastore_get_foreign_keys_result&); - ThriftHiveMetastore_get_foreign_keys_result& operator=(const ThriftHiveMetastore_get_foreign_keys_result&); - ThriftHiveMetastore_get_foreign_keys_result() { - } - - virtual ~ThriftHiveMetastore_get_foreign_keys_result() noexcept; - ForeignKeysResponse success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_foreign_keys_result__isset __isset; - - void __set_success(const ForeignKeysResponse& val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_foreign_keys_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_foreign_keys_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_foreign_keys_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_foreign_keys_presult__isset { - _ThriftHiveMetastore_get_foreign_keys_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_foreign_keys_presult__isset; - -class ThriftHiveMetastore_get_foreign_keys_presult { - public: - - - virtual ~ThriftHiveMetastore_get_foreign_keys_presult() noexcept; - ForeignKeysResponse* success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_foreign_keys_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_unique_constraints_args__isset { - _ThriftHiveMetastore_get_unique_constraints_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_get_unique_constraints_args__isset; - -class ThriftHiveMetastore_get_unique_constraints_args { - public: - - ThriftHiveMetastore_get_unique_constraints_args(const ThriftHiveMetastore_get_unique_constraints_args&); - ThriftHiveMetastore_get_unique_constraints_args& operator=(const ThriftHiveMetastore_get_unique_constraints_args&); - ThriftHiveMetastore_get_unique_constraints_args() { - } - - virtual ~ThriftHiveMetastore_get_unique_constraints_args() noexcept; - UniqueConstraintsRequest request; - - _ThriftHiveMetastore_get_unique_constraints_args__isset __isset; - - void __set_request(const UniqueConstraintsRequest& val); - - bool operator == (const ThriftHiveMetastore_get_unique_constraints_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_unique_constraints_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_unique_constraints_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_unique_constraints_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_unique_constraints_pargs() noexcept; - const UniqueConstraintsRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_unique_constraints_result__isset { - _ThriftHiveMetastore_get_unique_constraints_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_unique_constraints_result__isset; - -class ThriftHiveMetastore_get_unique_constraints_result { - public: - - ThriftHiveMetastore_get_unique_constraints_result(const ThriftHiveMetastore_get_unique_constraints_result&); - ThriftHiveMetastore_get_unique_constraints_result& operator=(const ThriftHiveMetastore_get_unique_constraints_result&); - ThriftHiveMetastore_get_unique_constraints_result() { - } - - virtual ~ThriftHiveMetastore_get_unique_constraints_result() noexcept; - UniqueConstraintsResponse success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_unique_constraints_result__isset __isset; - - void __set_success(const UniqueConstraintsResponse& val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_unique_constraints_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_unique_constraints_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_unique_constraints_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_unique_constraints_presult__isset { - _ThriftHiveMetastore_get_unique_constraints_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_unique_constraints_presult__isset; - -class ThriftHiveMetastore_get_unique_constraints_presult { - public: - - - virtual ~ThriftHiveMetastore_get_unique_constraints_presult() noexcept; - UniqueConstraintsResponse* success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_unique_constraints_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_not_null_constraints_args__isset { - _ThriftHiveMetastore_get_not_null_constraints_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_get_not_null_constraints_args__isset; - -class ThriftHiveMetastore_get_not_null_constraints_args { - public: - - ThriftHiveMetastore_get_not_null_constraints_args(const ThriftHiveMetastore_get_not_null_constraints_args&); - ThriftHiveMetastore_get_not_null_constraints_args& operator=(const ThriftHiveMetastore_get_not_null_constraints_args&); - ThriftHiveMetastore_get_not_null_constraints_args() { - } - - virtual ~ThriftHiveMetastore_get_not_null_constraints_args() noexcept; - NotNullConstraintsRequest request; - - _ThriftHiveMetastore_get_not_null_constraints_args__isset __isset; - - void __set_request(const NotNullConstraintsRequest& val); - - bool operator == (const ThriftHiveMetastore_get_not_null_constraints_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_not_null_constraints_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_not_null_constraints_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_not_null_constraints_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_not_null_constraints_pargs() noexcept; - const NotNullConstraintsRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_not_null_constraints_result__isset { - _ThriftHiveMetastore_get_not_null_constraints_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_not_null_constraints_result__isset; - -class ThriftHiveMetastore_get_not_null_constraints_result { - public: - - ThriftHiveMetastore_get_not_null_constraints_result(const ThriftHiveMetastore_get_not_null_constraints_result&); - ThriftHiveMetastore_get_not_null_constraints_result& operator=(const ThriftHiveMetastore_get_not_null_constraints_result&); - ThriftHiveMetastore_get_not_null_constraints_result() { - } - - virtual ~ThriftHiveMetastore_get_not_null_constraints_result() noexcept; - NotNullConstraintsResponse success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_not_null_constraints_result__isset __isset; - - void __set_success(const NotNullConstraintsResponse& val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_not_null_constraints_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_not_null_constraints_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_not_null_constraints_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_not_null_constraints_presult__isset { - _ThriftHiveMetastore_get_not_null_constraints_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_not_null_constraints_presult__isset; - -class ThriftHiveMetastore_get_not_null_constraints_presult { - public: - - - virtual ~ThriftHiveMetastore_get_not_null_constraints_presult() noexcept; - NotNullConstraintsResponse* success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_not_null_constraints_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_update_table_column_statistics_args__isset { - _ThriftHiveMetastore_update_table_column_statistics_args__isset() : stats_obj(false) {} - bool stats_obj :1; -} _ThriftHiveMetastore_update_table_column_statistics_args__isset; - -class ThriftHiveMetastore_update_table_column_statistics_args { - public: - - ThriftHiveMetastore_update_table_column_statistics_args(const ThriftHiveMetastore_update_table_column_statistics_args&); - ThriftHiveMetastore_update_table_column_statistics_args& operator=(const ThriftHiveMetastore_update_table_column_statistics_args&); - ThriftHiveMetastore_update_table_column_statistics_args() { - } - - virtual ~ThriftHiveMetastore_update_table_column_statistics_args() noexcept; - ColumnStatistics stats_obj; - - _ThriftHiveMetastore_update_table_column_statistics_args__isset __isset; - - void __set_stats_obj(const ColumnStatistics& val); - - bool operator == (const ThriftHiveMetastore_update_table_column_statistics_args & rhs) const - { - if (!(stats_obj == rhs.stats_obj)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_update_table_column_statistics_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_update_table_column_statistics_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_update_table_column_statistics_pargs { - public: - - - virtual ~ThriftHiveMetastore_update_table_column_statistics_pargs() noexcept; - const ColumnStatistics* stats_obj; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_update_table_column_statistics_result__isset { - _ThriftHiveMetastore_update_table_column_statistics_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_update_table_column_statistics_result__isset; - -class ThriftHiveMetastore_update_table_column_statistics_result { - public: - - ThriftHiveMetastore_update_table_column_statistics_result(const ThriftHiveMetastore_update_table_column_statistics_result&); - ThriftHiveMetastore_update_table_column_statistics_result& operator=(const ThriftHiveMetastore_update_table_column_statistics_result&); - ThriftHiveMetastore_update_table_column_statistics_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_update_table_column_statistics_result() noexcept; - bool success; - NoSuchObjectException o1; - InvalidObjectException o2; - MetaException o3; - InvalidInputException o4; - - _ThriftHiveMetastore_update_table_column_statistics_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const InvalidObjectException& val); - - void __set_o3(const MetaException& val); - - void __set_o4(const InvalidInputException& val); - - bool operator == (const ThriftHiveMetastore_update_table_column_statistics_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - if (!(o4 == rhs.o4)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_update_table_column_statistics_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_update_table_column_statistics_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_update_table_column_statistics_presult__isset { - _ThriftHiveMetastore_update_table_column_statistics_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_update_table_column_statistics_presult__isset; - -class ThriftHiveMetastore_update_table_column_statistics_presult { - public: - - - virtual ~ThriftHiveMetastore_update_table_column_statistics_presult() noexcept; - bool* success; - NoSuchObjectException o1; - InvalidObjectException o2; - MetaException o3; - InvalidInputException o4; - - _ThriftHiveMetastore_update_table_column_statistics_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_update_partition_column_statistics_args__isset { - _ThriftHiveMetastore_update_partition_column_statistics_args__isset() : stats_obj(false) {} - bool stats_obj :1; -} _ThriftHiveMetastore_update_partition_column_statistics_args__isset; - -class ThriftHiveMetastore_update_partition_column_statistics_args { - public: - - ThriftHiveMetastore_update_partition_column_statistics_args(const ThriftHiveMetastore_update_partition_column_statistics_args&); - ThriftHiveMetastore_update_partition_column_statistics_args& operator=(const ThriftHiveMetastore_update_partition_column_statistics_args&); - ThriftHiveMetastore_update_partition_column_statistics_args() { - } - - virtual ~ThriftHiveMetastore_update_partition_column_statistics_args() noexcept; - ColumnStatistics stats_obj; - - _ThriftHiveMetastore_update_partition_column_statistics_args__isset __isset; - - void __set_stats_obj(const ColumnStatistics& val); - - bool operator == (const ThriftHiveMetastore_update_partition_column_statistics_args & rhs) const - { - if (!(stats_obj == rhs.stats_obj)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_update_partition_column_statistics_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_update_partition_column_statistics_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_update_partition_column_statistics_pargs { - public: - - - virtual ~ThriftHiveMetastore_update_partition_column_statistics_pargs() noexcept; - const ColumnStatistics* stats_obj; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_update_partition_column_statistics_result__isset { - _ThriftHiveMetastore_update_partition_column_statistics_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_update_partition_column_statistics_result__isset; - -class ThriftHiveMetastore_update_partition_column_statistics_result { - public: - - ThriftHiveMetastore_update_partition_column_statistics_result(const ThriftHiveMetastore_update_partition_column_statistics_result&); - ThriftHiveMetastore_update_partition_column_statistics_result& operator=(const ThriftHiveMetastore_update_partition_column_statistics_result&); - ThriftHiveMetastore_update_partition_column_statistics_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_update_partition_column_statistics_result() noexcept; - bool success; - NoSuchObjectException o1; - InvalidObjectException o2; - MetaException o3; - InvalidInputException o4; - - _ThriftHiveMetastore_update_partition_column_statistics_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const InvalidObjectException& val); - - void __set_o3(const MetaException& val); - - void __set_o4(const InvalidInputException& val); - - bool operator == (const ThriftHiveMetastore_update_partition_column_statistics_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - if (!(o4 == rhs.o4)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_update_partition_column_statistics_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_update_partition_column_statistics_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_update_partition_column_statistics_presult__isset { - _ThriftHiveMetastore_update_partition_column_statistics_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_update_partition_column_statistics_presult__isset; - -class ThriftHiveMetastore_update_partition_column_statistics_presult { - public: - - - virtual ~ThriftHiveMetastore_update_partition_column_statistics_presult() noexcept; - bool* success; - NoSuchObjectException o1; - InvalidObjectException o2; - MetaException o3; - InvalidInputException o4; - - _ThriftHiveMetastore_update_partition_column_statistics_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_table_column_statistics_args__isset { - _ThriftHiveMetastore_get_table_column_statistics_args__isset() : db_name(false), tbl_name(false), col_name(false) {} - bool db_name :1; - bool tbl_name :1; - bool col_name :1; -} _ThriftHiveMetastore_get_table_column_statistics_args__isset; - -class ThriftHiveMetastore_get_table_column_statistics_args { - public: - - ThriftHiveMetastore_get_table_column_statistics_args(const ThriftHiveMetastore_get_table_column_statistics_args&); - ThriftHiveMetastore_get_table_column_statistics_args& operator=(const ThriftHiveMetastore_get_table_column_statistics_args&); - ThriftHiveMetastore_get_table_column_statistics_args() : db_name(), tbl_name(), col_name() { - } - - virtual ~ThriftHiveMetastore_get_table_column_statistics_args() noexcept; - std::string db_name; - std::string tbl_name; - std::string col_name; - - _ThriftHiveMetastore_get_table_column_statistics_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_col_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_table_column_statistics_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(col_name == rhs.col_name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_table_column_statistics_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_table_column_statistics_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_table_column_statistics_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_table_column_statistics_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::string* col_name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_table_column_statistics_result__isset { - _ThriftHiveMetastore_get_table_column_statistics_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_get_table_column_statistics_result__isset; - -class ThriftHiveMetastore_get_table_column_statistics_result { - public: - - ThriftHiveMetastore_get_table_column_statistics_result(const ThriftHiveMetastore_get_table_column_statistics_result&); - ThriftHiveMetastore_get_table_column_statistics_result& operator=(const ThriftHiveMetastore_get_table_column_statistics_result&); - ThriftHiveMetastore_get_table_column_statistics_result() { - } - - virtual ~ThriftHiveMetastore_get_table_column_statistics_result() noexcept; - ColumnStatistics success; - NoSuchObjectException o1; - MetaException o2; - InvalidInputException o3; - InvalidObjectException o4; - - _ThriftHiveMetastore_get_table_column_statistics_result__isset __isset; - - void __set_success(const ColumnStatistics& val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - void __set_o3(const InvalidInputException& val); - - void __set_o4(const InvalidObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_table_column_statistics_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - if (!(o4 == rhs.o4)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_table_column_statistics_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_table_column_statistics_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_table_column_statistics_presult__isset { - _ThriftHiveMetastore_get_table_column_statistics_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_get_table_column_statistics_presult__isset; - -class ThriftHiveMetastore_get_table_column_statistics_presult { - public: - - - virtual ~ThriftHiveMetastore_get_table_column_statistics_presult() noexcept; - ColumnStatistics* success; - NoSuchObjectException o1; - MetaException o2; - InvalidInputException o3; - InvalidObjectException o4; - - _ThriftHiveMetastore_get_table_column_statistics_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_partition_column_statistics_args__isset { - _ThriftHiveMetastore_get_partition_column_statistics_args__isset() : db_name(false), tbl_name(false), part_name(false), col_name(false) {} - bool db_name :1; - bool tbl_name :1; - bool part_name :1; - bool col_name :1; -} _ThriftHiveMetastore_get_partition_column_statistics_args__isset; - -class ThriftHiveMetastore_get_partition_column_statistics_args { - public: - - ThriftHiveMetastore_get_partition_column_statistics_args(const ThriftHiveMetastore_get_partition_column_statistics_args&); - ThriftHiveMetastore_get_partition_column_statistics_args& operator=(const ThriftHiveMetastore_get_partition_column_statistics_args&); - ThriftHiveMetastore_get_partition_column_statistics_args() : db_name(), tbl_name(), part_name(), col_name() { - } - - virtual ~ThriftHiveMetastore_get_partition_column_statistics_args() noexcept; - std::string db_name; - std::string tbl_name; - std::string part_name; - std::string col_name; - - _ThriftHiveMetastore_get_partition_column_statistics_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_name(const std::string& val); - - void __set_col_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_partition_column_statistics_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_name == rhs.part_name)) - return false; - if (!(col_name == rhs.col_name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partition_column_statistics_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partition_column_statistics_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_partition_column_statistics_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_partition_column_statistics_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::string* part_name; - const std::string* col_name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partition_column_statistics_result__isset { - _ThriftHiveMetastore_get_partition_column_statistics_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_get_partition_column_statistics_result__isset; - -class ThriftHiveMetastore_get_partition_column_statistics_result { - public: - - ThriftHiveMetastore_get_partition_column_statistics_result(const ThriftHiveMetastore_get_partition_column_statistics_result&); - ThriftHiveMetastore_get_partition_column_statistics_result& operator=(const ThriftHiveMetastore_get_partition_column_statistics_result&); - ThriftHiveMetastore_get_partition_column_statistics_result() { - } - - virtual ~ThriftHiveMetastore_get_partition_column_statistics_result() noexcept; - ColumnStatistics success; - NoSuchObjectException o1; - MetaException o2; - InvalidInputException o3; - InvalidObjectException o4; - - _ThriftHiveMetastore_get_partition_column_statistics_result__isset __isset; - - void __set_success(const ColumnStatistics& val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - void __set_o3(const InvalidInputException& val); - - void __set_o4(const InvalidObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_partition_column_statistics_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - if (!(o4 == rhs.o4)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partition_column_statistics_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partition_column_statistics_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partition_column_statistics_presult__isset { - _ThriftHiveMetastore_get_partition_column_statistics_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_get_partition_column_statistics_presult__isset; - -class ThriftHiveMetastore_get_partition_column_statistics_presult { - public: - - - virtual ~ThriftHiveMetastore_get_partition_column_statistics_presult() noexcept; - ColumnStatistics* success; - NoSuchObjectException o1; - MetaException o2; - InvalidInputException o3; - InvalidObjectException o4; - - _ThriftHiveMetastore_get_partition_column_statistics_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_table_statistics_req_args__isset { - _ThriftHiveMetastore_get_table_statistics_req_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_get_table_statistics_req_args__isset; - -class ThriftHiveMetastore_get_table_statistics_req_args { - public: - - ThriftHiveMetastore_get_table_statistics_req_args(const ThriftHiveMetastore_get_table_statistics_req_args&); - ThriftHiveMetastore_get_table_statistics_req_args& operator=(const ThriftHiveMetastore_get_table_statistics_req_args&); - ThriftHiveMetastore_get_table_statistics_req_args() { - } - - virtual ~ThriftHiveMetastore_get_table_statistics_req_args() noexcept; - TableStatsRequest request; - - _ThriftHiveMetastore_get_table_statistics_req_args__isset __isset; - - void __set_request(const TableStatsRequest& val); - - bool operator == (const ThriftHiveMetastore_get_table_statistics_req_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_table_statistics_req_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_table_statistics_req_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_table_statistics_req_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_table_statistics_req_pargs() noexcept; - const TableStatsRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_table_statistics_req_result__isset { - _ThriftHiveMetastore_get_table_statistics_req_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_table_statistics_req_result__isset; - -class ThriftHiveMetastore_get_table_statistics_req_result { - public: - - ThriftHiveMetastore_get_table_statistics_req_result(const ThriftHiveMetastore_get_table_statistics_req_result&); - ThriftHiveMetastore_get_table_statistics_req_result& operator=(const ThriftHiveMetastore_get_table_statistics_req_result&); - ThriftHiveMetastore_get_table_statistics_req_result() { - } - - virtual ~ThriftHiveMetastore_get_table_statistics_req_result() noexcept; - TableStatsResult success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_table_statistics_req_result__isset __isset; - - void __set_success(const TableStatsResult& val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_table_statistics_req_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_table_statistics_req_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_table_statistics_req_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_table_statistics_req_presult__isset { - _ThriftHiveMetastore_get_table_statistics_req_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_table_statistics_req_presult__isset; - -class ThriftHiveMetastore_get_table_statistics_req_presult { - public: - - - virtual ~ThriftHiveMetastore_get_table_statistics_req_presult() noexcept; - TableStatsResult* success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_table_statistics_req_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_statistics_req_args__isset { - _ThriftHiveMetastore_get_partitions_statistics_req_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_get_partitions_statistics_req_args__isset; - -class ThriftHiveMetastore_get_partitions_statistics_req_args { - public: - - ThriftHiveMetastore_get_partitions_statistics_req_args(const ThriftHiveMetastore_get_partitions_statistics_req_args&); - ThriftHiveMetastore_get_partitions_statistics_req_args& operator=(const ThriftHiveMetastore_get_partitions_statistics_req_args&); - ThriftHiveMetastore_get_partitions_statistics_req_args() { - } - - virtual ~ThriftHiveMetastore_get_partitions_statistics_req_args() noexcept; - PartitionsStatsRequest request; - - _ThriftHiveMetastore_get_partitions_statistics_req_args__isset __isset; - - void __set_request(const PartitionsStatsRequest& val); - - bool operator == (const ThriftHiveMetastore_get_partitions_statistics_req_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_statistics_req_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_statistics_req_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_partitions_statistics_req_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_statistics_req_pargs() noexcept; - const PartitionsStatsRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_statistics_req_result__isset { - _ThriftHiveMetastore_get_partitions_statistics_req_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_statistics_req_result__isset; - -class ThriftHiveMetastore_get_partitions_statistics_req_result { - public: - - ThriftHiveMetastore_get_partitions_statistics_req_result(const ThriftHiveMetastore_get_partitions_statistics_req_result&); - ThriftHiveMetastore_get_partitions_statistics_req_result& operator=(const ThriftHiveMetastore_get_partitions_statistics_req_result&); - ThriftHiveMetastore_get_partitions_statistics_req_result() { - } - - virtual ~ThriftHiveMetastore_get_partitions_statistics_req_result() noexcept; - PartitionsStatsResult success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_partitions_statistics_req_result__isset __isset; - - void __set_success(const PartitionsStatsResult& val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_partitions_statistics_req_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_partitions_statistics_req_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_partitions_statistics_req_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_partitions_statistics_req_presult__isset { - _ThriftHiveMetastore_get_partitions_statistics_req_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_partitions_statistics_req_presult__isset; - -class ThriftHiveMetastore_get_partitions_statistics_req_presult { - public: - - - virtual ~ThriftHiveMetastore_get_partitions_statistics_req_presult() noexcept; - PartitionsStatsResult* success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_partitions_statistics_req_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_aggr_stats_for_args__isset { - _ThriftHiveMetastore_get_aggr_stats_for_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_get_aggr_stats_for_args__isset; - -class ThriftHiveMetastore_get_aggr_stats_for_args { - public: - - ThriftHiveMetastore_get_aggr_stats_for_args(const ThriftHiveMetastore_get_aggr_stats_for_args&); - ThriftHiveMetastore_get_aggr_stats_for_args& operator=(const ThriftHiveMetastore_get_aggr_stats_for_args&); - ThriftHiveMetastore_get_aggr_stats_for_args() { - } - - virtual ~ThriftHiveMetastore_get_aggr_stats_for_args() noexcept; - PartitionsStatsRequest request; - - _ThriftHiveMetastore_get_aggr_stats_for_args__isset __isset; - - void __set_request(const PartitionsStatsRequest& val); - - bool operator == (const ThriftHiveMetastore_get_aggr_stats_for_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_aggr_stats_for_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_aggr_stats_for_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_aggr_stats_for_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_aggr_stats_for_pargs() noexcept; - const PartitionsStatsRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_aggr_stats_for_result__isset { - _ThriftHiveMetastore_get_aggr_stats_for_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_aggr_stats_for_result__isset; - -class ThriftHiveMetastore_get_aggr_stats_for_result { - public: - - ThriftHiveMetastore_get_aggr_stats_for_result(const ThriftHiveMetastore_get_aggr_stats_for_result&); - ThriftHiveMetastore_get_aggr_stats_for_result& operator=(const ThriftHiveMetastore_get_aggr_stats_for_result&); - ThriftHiveMetastore_get_aggr_stats_for_result() { - } - - virtual ~ThriftHiveMetastore_get_aggr_stats_for_result() noexcept; - AggrStats success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_aggr_stats_for_result__isset __isset; - - void __set_success(const AggrStats& val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_aggr_stats_for_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_aggr_stats_for_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_aggr_stats_for_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_aggr_stats_for_presult__isset { - _ThriftHiveMetastore_get_aggr_stats_for_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_aggr_stats_for_presult__isset; - -class ThriftHiveMetastore_get_aggr_stats_for_presult { - public: - - - virtual ~ThriftHiveMetastore_get_aggr_stats_for_presult() noexcept; - AggrStats* success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_aggr_stats_for_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_set_aggr_stats_for_args__isset { - _ThriftHiveMetastore_set_aggr_stats_for_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_set_aggr_stats_for_args__isset; - -class ThriftHiveMetastore_set_aggr_stats_for_args { - public: - - ThriftHiveMetastore_set_aggr_stats_for_args(const ThriftHiveMetastore_set_aggr_stats_for_args&); - ThriftHiveMetastore_set_aggr_stats_for_args& operator=(const ThriftHiveMetastore_set_aggr_stats_for_args&); - ThriftHiveMetastore_set_aggr_stats_for_args() { - } - - virtual ~ThriftHiveMetastore_set_aggr_stats_for_args() noexcept; - SetPartitionsStatsRequest request; - - _ThriftHiveMetastore_set_aggr_stats_for_args__isset __isset; - - void __set_request(const SetPartitionsStatsRequest& val); - - bool operator == (const ThriftHiveMetastore_set_aggr_stats_for_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_set_aggr_stats_for_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_set_aggr_stats_for_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_set_aggr_stats_for_pargs { - public: - - - virtual ~ThriftHiveMetastore_set_aggr_stats_for_pargs() noexcept; - const SetPartitionsStatsRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_set_aggr_stats_for_result__isset { - _ThriftHiveMetastore_set_aggr_stats_for_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_set_aggr_stats_for_result__isset; - -class ThriftHiveMetastore_set_aggr_stats_for_result { - public: - - ThriftHiveMetastore_set_aggr_stats_for_result(const ThriftHiveMetastore_set_aggr_stats_for_result&); - ThriftHiveMetastore_set_aggr_stats_for_result& operator=(const ThriftHiveMetastore_set_aggr_stats_for_result&); - ThriftHiveMetastore_set_aggr_stats_for_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_set_aggr_stats_for_result() noexcept; - bool success; - NoSuchObjectException o1; - InvalidObjectException o2; - MetaException o3; - InvalidInputException o4; - - _ThriftHiveMetastore_set_aggr_stats_for_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const InvalidObjectException& val); - - void __set_o3(const MetaException& val); - - void __set_o4(const InvalidInputException& val); - - bool operator == (const ThriftHiveMetastore_set_aggr_stats_for_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - if (!(o4 == rhs.o4)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_set_aggr_stats_for_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_set_aggr_stats_for_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_set_aggr_stats_for_presult__isset { - _ThriftHiveMetastore_set_aggr_stats_for_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_set_aggr_stats_for_presult__isset; - -class ThriftHiveMetastore_set_aggr_stats_for_presult { - public: - - - virtual ~ThriftHiveMetastore_set_aggr_stats_for_presult() noexcept; - bool* success; - NoSuchObjectException o1; - InvalidObjectException o2; - MetaException o3; - InvalidInputException o4; - - _ThriftHiveMetastore_set_aggr_stats_for_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_delete_partition_column_statistics_args__isset { - _ThriftHiveMetastore_delete_partition_column_statistics_args__isset() : db_name(false), tbl_name(false), part_name(false), col_name(false) {} - bool db_name :1; - bool tbl_name :1; - bool part_name :1; - bool col_name :1; -} _ThriftHiveMetastore_delete_partition_column_statistics_args__isset; - -class ThriftHiveMetastore_delete_partition_column_statistics_args { - public: - - ThriftHiveMetastore_delete_partition_column_statistics_args(const ThriftHiveMetastore_delete_partition_column_statistics_args&); - ThriftHiveMetastore_delete_partition_column_statistics_args& operator=(const ThriftHiveMetastore_delete_partition_column_statistics_args&); - ThriftHiveMetastore_delete_partition_column_statistics_args() : db_name(), tbl_name(), part_name(), col_name() { - } - - virtual ~ThriftHiveMetastore_delete_partition_column_statistics_args() noexcept; - std::string db_name; - std::string tbl_name; - std::string part_name; - std::string col_name; - - _ThriftHiveMetastore_delete_partition_column_statistics_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_part_name(const std::string& val); - - void __set_col_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_delete_partition_column_statistics_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(part_name == rhs.part_name)) - return false; - if (!(col_name == rhs.col_name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_delete_partition_column_statistics_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_delete_partition_column_statistics_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_delete_partition_column_statistics_pargs { - public: - - - virtual ~ThriftHiveMetastore_delete_partition_column_statistics_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::string* part_name; - const std::string* col_name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_delete_partition_column_statistics_result__isset { - _ThriftHiveMetastore_delete_partition_column_statistics_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_delete_partition_column_statistics_result__isset; - -class ThriftHiveMetastore_delete_partition_column_statistics_result { - public: - - ThriftHiveMetastore_delete_partition_column_statistics_result(const ThriftHiveMetastore_delete_partition_column_statistics_result&); - ThriftHiveMetastore_delete_partition_column_statistics_result& operator=(const ThriftHiveMetastore_delete_partition_column_statistics_result&); - ThriftHiveMetastore_delete_partition_column_statistics_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_delete_partition_column_statistics_result() noexcept; - bool success; - NoSuchObjectException o1; - MetaException o2; - InvalidObjectException o3; - InvalidInputException o4; - - _ThriftHiveMetastore_delete_partition_column_statistics_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - void __set_o3(const InvalidObjectException& val); - - void __set_o4(const InvalidInputException& val); - - bool operator == (const ThriftHiveMetastore_delete_partition_column_statistics_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - if (!(o4 == rhs.o4)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_delete_partition_column_statistics_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_delete_partition_column_statistics_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_delete_partition_column_statistics_presult__isset { - _ThriftHiveMetastore_delete_partition_column_statistics_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_delete_partition_column_statistics_presult__isset; - -class ThriftHiveMetastore_delete_partition_column_statistics_presult { - public: - - - virtual ~ThriftHiveMetastore_delete_partition_column_statistics_presult() noexcept; - bool* success; - NoSuchObjectException o1; - MetaException o2; - InvalidObjectException o3; - InvalidInputException o4; - - _ThriftHiveMetastore_delete_partition_column_statistics_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_delete_table_column_statistics_args__isset { - _ThriftHiveMetastore_delete_table_column_statistics_args__isset() : db_name(false), tbl_name(false), col_name(false) {} - bool db_name :1; - bool tbl_name :1; - bool col_name :1; -} _ThriftHiveMetastore_delete_table_column_statistics_args__isset; - -class ThriftHiveMetastore_delete_table_column_statistics_args { - public: - - ThriftHiveMetastore_delete_table_column_statistics_args(const ThriftHiveMetastore_delete_table_column_statistics_args&); - ThriftHiveMetastore_delete_table_column_statistics_args& operator=(const ThriftHiveMetastore_delete_table_column_statistics_args&); - ThriftHiveMetastore_delete_table_column_statistics_args() : db_name(), tbl_name(), col_name() { - } - - virtual ~ThriftHiveMetastore_delete_table_column_statistics_args() noexcept; - std::string db_name; - std::string tbl_name; - std::string col_name; - - _ThriftHiveMetastore_delete_table_column_statistics_args__isset __isset; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - void __set_col_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_delete_table_column_statistics_args & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - if (!(col_name == rhs.col_name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_delete_table_column_statistics_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_delete_table_column_statistics_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_delete_table_column_statistics_pargs { - public: - - - virtual ~ThriftHiveMetastore_delete_table_column_statistics_pargs() noexcept; - const std::string* db_name; - const std::string* tbl_name; - const std::string* col_name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_delete_table_column_statistics_result__isset { - _ThriftHiveMetastore_delete_table_column_statistics_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_delete_table_column_statistics_result__isset; - -class ThriftHiveMetastore_delete_table_column_statistics_result { - public: - - ThriftHiveMetastore_delete_table_column_statistics_result(const ThriftHiveMetastore_delete_table_column_statistics_result&); - ThriftHiveMetastore_delete_table_column_statistics_result& operator=(const ThriftHiveMetastore_delete_table_column_statistics_result&); - ThriftHiveMetastore_delete_table_column_statistics_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_delete_table_column_statistics_result() noexcept; - bool success; - NoSuchObjectException o1; - MetaException o2; - InvalidObjectException o3; - InvalidInputException o4; - - _ThriftHiveMetastore_delete_table_column_statistics_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - void __set_o3(const InvalidObjectException& val); - - void __set_o4(const InvalidInputException& val); - - bool operator == (const ThriftHiveMetastore_delete_table_column_statistics_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - if (!(o4 == rhs.o4)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_delete_table_column_statistics_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_delete_table_column_statistics_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_delete_table_column_statistics_presult__isset { - _ThriftHiveMetastore_delete_table_column_statistics_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_delete_table_column_statistics_presult__isset; - -class ThriftHiveMetastore_delete_table_column_statistics_presult { - public: - - - virtual ~ThriftHiveMetastore_delete_table_column_statistics_presult() noexcept; - bool* success; - NoSuchObjectException o1; - MetaException o2; - InvalidObjectException o3; - InvalidInputException o4; - - _ThriftHiveMetastore_delete_table_column_statistics_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_create_function_args__isset { - _ThriftHiveMetastore_create_function_args__isset() : func(false) {} - bool func :1; -} _ThriftHiveMetastore_create_function_args__isset; - -class ThriftHiveMetastore_create_function_args { - public: - - ThriftHiveMetastore_create_function_args(const ThriftHiveMetastore_create_function_args&); - ThriftHiveMetastore_create_function_args& operator=(const ThriftHiveMetastore_create_function_args&); - ThriftHiveMetastore_create_function_args() { - } - - virtual ~ThriftHiveMetastore_create_function_args() noexcept; - Function func; - - _ThriftHiveMetastore_create_function_args__isset __isset; - - void __set_func(const Function& val); - - bool operator == (const ThriftHiveMetastore_create_function_args & rhs) const - { - if (!(func == rhs.func)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_function_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_function_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_create_function_pargs { - public: - - - virtual ~ThriftHiveMetastore_create_function_pargs() noexcept; - const Function* func; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_function_result__isset { - _ThriftHiveMetastore_create_function_result__isset() : o1(false), o2(false), o3(false), o4(false) {} - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_create_function_result__isset; - -class ThriftHiveMetastore_create_function_result { - public: - - ThriftHiveMetastore_create_function_result(const ThriftHiveMetastore_create_function_result&); - ThriftHiveMetastore_create_function_result& operator=(const ThriftHiveMetastore_create_function_result&); - ThriftHiveMetastore_create_function_result() { - } - - virtual ~ThriftHiveMetastore_create_function_result() noexcept; - AlreadyExistsException o1; - InvalidObjectException o2; - MetaException o3; - NoSuchObjectException o4; - - _ThriftHiveMetastore_create_function_result__isset __isset; - - void __set_o1(const AlreadyExistsException& val); - - void __set_o2(const InvalidObjectException& val); - - void __set_o3(const MetaException& val); - - void __set_o4(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_create_function_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - if (!(o4 == rhs.o4)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_function_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_function_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_function_presult__isset { - _ThriftHiveMetastore_create_function_presult__isset() : o1(false), o2(false), o3(false), o4(false) {} - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_create_function_presult__isset; - -class ThriftHiveMetastore_create_function_presult { - public: - - - virtual ~ThriftHiveMetastore_create_function_presult() noexcept; - AlreadyExistsException o1; - InvalidObjectException o2; - MetaException o3; - NoSuchObjectException o4; - - _ThriftHiveMetastore_create_function_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_drop_function_args__isset { - _ThriftHiveMetastore_drop_function_args__isset() : dbName(false), funcName(false) {} - bool dbName :1; - bool funcName :1; -} _ThriftHiveMetastore_drop_function_args__isset; - -class ThriftHiveMetastore_drop_function_args { - public: - - ThriftHiveMetastore_drop_function_args(const ThriftHiveMetastore_drop_function_args&); - ThriftHiveMetastore_drop_function_args& operator=(const ThriftHiveMetastore_drop_function_args&); - ThriftHiveMetastore_drop_function_args() : dbName(), funcName() { - } - - virtual ~ThriftHiveMetastore_drop_function_args() noexcept; - std::string dbName; - std::string funcName; - - _ThriftHiveMetastore_drop_function_args__isset __isset; - - void __set_dbName(const std::string& val); - - void __set_funcName(const std::string& val); - - bool operator == (const ThriftHiveMetastore_drop_function_args & rhs) const - { - if (!(dbName == rhs.dbName)) - return false; - if (!(funcName == rhs.funcName)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_function_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_function_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_drop_function_pargs { - public: - - - virtual ~ThriftHiveMetastore_drop_function_pargs() noexcept; - const std::string* dbName; - const std::string* funcName; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_function_result__isset { - _ThriftHiveMetastore_drop_function_result__isset() : o1(false), o3(false) {} - bool o1 :1; - bool o3 :1; -} _ThriftHiveMetastore_drop_function_result__isset; - -class ThriftHiveMetastore_drop_function_result { - public: - - ThriftHiveMetastore_drop_function_result(const ThriftHiveMetastore_drop_function_result&); - ThriftHiveMetastore_drop_function_result& operator=(const ThriftHiveMetastore_drop_function_result&); - ThriftHiveMetastore_drop_function_result() { - } - - virtual ~ThriftHiveMetastore_drop_function_result() noexcept; - NoSuchObjectException o1; - MetaException o3; - - _ThriftHiveMetastore_drop_function_result__isset __isset; - - void __set_o1(const NoSuchObjectException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_drop_function_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_function_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_function_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_function_presult__isset { - _ThriftHiveMetastore_drop_function_presult__isset() : o1(false), o3(false) {} - bool o1 :1; - bool o3 :1; -} _ThriftHiveMetastore_drop_function_presult__isset; - -class ThriftHiveMetastore_drop_function_presult { - public: - - - virtual ~ThriftHiveMetastore_drop_function_presult() noexcept; - NoSuchObjectException o1; - MetaException o3; - - _ThriftHiveMetastore_drop_function_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_alter_function_args__isset { - _ThriftHiveMetastore_alter_function_args__isset() : dbName(false), funcName(false), newFunc(false) {} - bool dbName :1; - bool funcName :1; - bool newFunc :1; -} _ThriftHiveMetastore_alter_function_args__isset; - -class ThriftHiveMetastore_alter_function_args { - public: - - ThriftHiveMetastore_alter_function_args(const ThriftHiveMetastore_alter_function_args&); - ThriftHiveMetastore_alter_function_args& operator=(const ThriftHiveMetastore_alter_function_args&); - ThriftHiveMetastore_alter_function_args() : dbName(), funcName() { - } - - virtual ~ThriftHiveMetastore_alter_function_args() noexcept; - std::string dbName; - std::string funcName; - Function newFunc; - - _ThriftHiveMetastore_alter_function_args__isset __isset; - - void __set_dbName(const std::string& val); - - void __set_funcName(const std::string& val); - - void __set_newFunc(const Function& val); - - bool operator == (const ThriftHiveMetastore_alter_function_args & rhs) const - { - if (!(dbName == rhs.dbName)) - return false; - if (!(funcName == rhs.funcName)) - return false; - if (!(newFunc == rhs.newFunc)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_function_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_function_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_alter_function_pargs { - public: - - - virtual ~ThriftHiveMetastore_alter_function_pargs() noexcept; - const std::string* dbName; - const std::string* funcName; - const Function* newFunc; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_function_result__isset { - _ThriftHiveMetastore_alter_function_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_function_result__isset; - -class ThriftHiveMetastore_alter_function_result { - public: - - ThriftHiveMetastore_alter_function_result(const ThriftHiveMetastore_alter_function_result&); - ThriftHiveMetastore_alter_function_result& operator=(const ThriftHiveMetastore_alter_function_result&); - ThriftHiveMetastore_alter_function_result() { - } - - virtual ~ThriftHiveMetastore_alter_function_result() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_function_result__isset __isset; - - void __set_o1(const InvalidOperationException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_alter_function_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_function_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_function_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_function_presult__isset { - _ThriftHiveMetastore_alter_function_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_alter_function_presult__isset; - -class ThriftHiveMetastore_alter_function_presult { - public: - - - virtual ~ThriftHiveMetastore_alter_function_presult() noexcept; - InvalidOperationException o1; - MetaException o2; - - _ThriftHiveMetastore_alter_function_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_functions_args__isset { - _ThriftHiveMetastore_get_functions_args__isset() : dbName(false), pattern(false) {} - bool dbName :1; - bool pattern :1; -} _ThriftHiveMetastore_get_functions_args__isset; - -class ThriftHiveMetastore_get_functions_args { - public: - - ThriftHiveMetastore_get_functions_args(const ThriftHiveMetastore_get_functions_args&); - ThriftHiveMetastore_get_functions_args& operator=(const ThriftHiveMetastore_get_functions_args&); - ThriftHiveMetastore_get_functions_args() : dbName(), pattern() { - } - - virtual ~ThriftHiveMetastore_get_functions_args() noexcept; - std::string dbName; - std::string pattern; - - _ThriftHiveMetastore_get_functions_args__isset __isset; - - void __set_dbName(const std::string& val); - - void __set_pattern(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_functions_args & rhs) const - { - if (!(dbName == rhs.dbName)) - return false; - if (!(pattern == rhs.pattern)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_functions_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_functions_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_functions_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_functions_pargs() noexcept; - const std::string* dbName; - const std::string* pattern; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_functions_result__isset { - _ThriftHiveMetastore_get_functions_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_functions_result__isset; - -class ThriftHiveMetastore_get_functions_result { - public: - - ThriftHiveMetastore_get_functions_result(const ThriftHiveMetastore_get_functions_result&); - ThriftHiveMetastore_get_functions_result& operator=(const ThriftHiveMetastore_get_functions_result&); - ThriftHiveMetastore_get_functions_result() { - } - - virtual ~ThriftHiveMetastore_get_functions_result() noexcept; - std::vector success; - MetaException o1; - - _ThriftHiveMetastore_get_functions_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_functions_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_functions_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_functions_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_functions_presult__isset { - _ThriftHiveMetastore_get_functions_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_functions_presult__isset; - -class ThriftHiveMetastore_get_functions_presult { - public: - - - virtual ~ThriftHiveMetastore_get_functions_presult() noexcept; - std::vector * success; - MetaException o1; - - _ThriftHiveMetastore_get_functions_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_function_args__isset { - _ThriftHiveMetastore_get_function_args__isset() : dbName(false), funcName(false) {} - bool dbName :1; - bool funcName :1; -} _ThriftHiveMetastore_get_function_args__isset; - -class ThriftHiveMetastore_get_function_args { - public: - - ThriftHiveMetastore_get_function_args(const ThriftHiveMetastore_get_function_args&); - ThriftHiveMetastore_get_function_args& operator=(const ThriftHiveMetastore_get_function_args&); - ThriftHiveMetastore_get_function_args() : dbName(), funcName() { - } - - virtual ~ThriftHiveMetastore_get_function_args() noexcept; - std::string dbName; - std::string funcName; - - _ThriftHiveMetastore_get_function_args__isset __isset; - - void __set_dbName(const std::string& val); - - void __set_funcName(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_function_args & rhs) const - { - if (!(dbName == rhs.dbName)) - return false; - if (!(funcName == rhs.funcName)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_function_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_function_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_function_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_function_pargs() noexcept; - const std::string* dbName; - const std::string* funcName; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_function_result__isset { - _ThriftHiveMetastore_get_function_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_function_result__isset; - -class ThriftHiveMetastore_get_function_result { - public: - - ThriftHiveMetastore_get_function_result(const ThriftHiveMetastore_get_function_result&); - ThriftHiveMetastore_get_function_result& operator=(const ThriftHiveMetastore_get_function_result&); - ThriftHiveMetastore_get_function_result() { - } - - virtual ~ThriftHiveMetastore_get_function_result() noexcept; - Function success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_function_result__isset __isset; - - void __set_success(const Function& val); - - void __set_o1(const MetaException& val); - - void __set_o2(const NoSuchObjectException& val); - - bool operator == (const ThriftHiveMetastore_get_function_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_function_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_function_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_function_presult__isset { - _ThriftHiveMetastore_get_function_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_function_presult__isset; - -class ThriftHiveMetastore_get_function_presult { - public: - - - virtual ~ThriftHiveMetastore_get_function_presult() noexcept; - Function* success; - MetaException o1; - NoSuchObjectException o2; - - _ThriftHiveMetastore_get_function_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - - -class ThriftHiveMetastore_get_all_functions_args { - public: - - ThriftHiveMetastore_get_all_functions_args(const ThriftHiveMetastore_get_all_functions_args&); - ThriftHiveMetastore_get_all_functions_args& operator=(const ThriftHiveMetastore_get_all_functions_args&); - ThriftHiveMetastore_get_all_functions_args() { - } - - virtual ~ThriftHiveMetastore_get_all_functions_args() noexcept; - - bool operator == (const ThriftHiveMetastore_get_all_functions_args & /* rhs */) const - { - return true; - } - bool operator != (const ThriftHiveMetastore_get_all_functions_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_all_functions_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_all_functions_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_all_functions_pargs() noexcept; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_all_functions_result__isset { - _ThriftHiveMetastore_get_all_functions_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_all_functions_result__isset; - -class ThriftHiveMetastore_get_all_functions_result { - public: - - ThriftHiveMetastore_get_all_functions_result(const ThriftHiveMetastore_get_all_functions_result&); - ThriftHiveMetastore_get_all_functions_result& operator=(const ThriftHiveMetastore_get_all_functions_result&); - ThriftHiveMetastore_get_all_functions_result() { - } - - virtual ~ThriftHiveMetastore_get_all_functions_result() noexcept; - GetAllFunctionsResponse success; - MetaException o1; - - _ThriftHiveMetastore_get_all_functions_result__isset __isset; - - void __set_success(const GetAllFunctionsResponse& val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_all_functions_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_all_functions_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_all_functions_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_all_functions_presult__isset { - _ThriftHiveMetastore_get_all_functions_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_all_functions_presult__isset; - -class ThriftHiveMetastore_get_all_functions_presult { - public: - - - virtual ~ThriftHiveMetastore_get_all_functions_presult() noexcept; - GetAllFunctionsResponse* success; - MetaException o1; - - _ThriftHiveMetastore_get_all_functions_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_create_role_args__isset { - _ThriftHiveMetastore_create_role_args__isset() : role(false) {} - bool role :1; -} _ThriftHiveMetastore_create_role_args__isset; - -class ThriftHiveMetastore_create_role_args { - public: - - ThriftHiveMetastore_create_role_args(const ThriftHiveMetastore_create_role_args&); - ThriftHiveMetastore_create_role_args& operator=(const ThriftHiveMetastore_create_role_args&); - ThriftHiveMetastore_create_role_args() { - } - - virtual ~ThriftHiveMetastore_create_role_args() noexcept; - Role role; - - _ThriftHiveMetastore_create_role_args__isset __isset; - - void __set_role(const Role& val); - - bool operator == (const ThriftHiveMetastore_create_role_args & rhs) const - { - if (!(role == rhs.role)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_role_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_role_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_create_role_pargs { - public: - - - virtual ~ThriftHiveMetastore_create_role_pargs() noexcept; - const Role* role; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_role_result__isset { - _ThriftHiveMetastore_create_role_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_create_role_result__isset; - -class ThriftHiveMetastore_create_role_result { - public: - - ThriftHiveMetastore_create_role_result(const ThriftHiveMetastore_create_role_result&); - ThriftHiveMetastore_create_role_result& operator=(const ThriftHiveMetastore_create_role_result&); - ThriftHiveMetastore_create_role_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_create_role_result() noexcept; - bool success; - MetaException o1; - - _ThriftHiveMetastore_create_role_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_create_role_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_role_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_role_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_role_presult__isset { - _ThriftHiveMetastore_create_role_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_create_role_presult__isset; - -class ThriftHiveMetastore_create_role_presult { - public: - - - virtual ~ThriftHiveMetastore_create_role_presult() noexcept; - bool* success; - MetaException o1; - - _ThriftHiveMetastore_create_role_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_drop_role_args__isset { - _ThriftHiveMetastore_drop_role_args__isset() : role_name(false) {} - bool role_name :1; -} _ThriftHiveMetastore_drop_role_args__isset; - -class ThriftHiveMetastore_drop_role_args { - public: - - ThriftHiveMetastore_drop_role_args(const ThriftHiveMetastore_drop_role_args&); - ThriftHiveMetastore_drop_role_args& operator=(const ThriftHiveMetastore_drop_role_args&); - ThriftHiveMetastore_drop_role_args() : role_name() { - } - - virtual ~ThriftHiveMetastore_drop_role_args() noexcept; - std::string role_name; - - _ThriftHiveMetastore_drop_role_args__isset __isset; - - void __set_role_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_drop_role_args & rhs) const - { - if (!(role_name == rhs.role_name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_role_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_role_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_drop_role_pargs { - public: - - - virtual ~ThriftHiveMetastore_drop_role_pargs() noexcept; - const std::string* role_name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_role_result__isset { - _ThriftHiveMetastore_drop_role_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_drop_role_result__isset; - -class ThriftHiveMetastore_drop_role_result { - public: - - ThriftHiveMetastore_drop_role_result(const ThriftHiveMetastore_drop_role_result&); - ThriftHiveMetastore_drop_role_result& operator=(const ThriftHiveMetastore_drop_role_result&); - ThriftHiveMetastore_drop_role_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_drop_role_result() noexcept; - bool success; - MetaException o1; - - _ThriftHiveMetastore_drop_role_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_drop_role_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_role_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_role_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_role_presult__isset { - _ThriftHiveMetastore_drop_role_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_drop_role_presult__isset; - -class ThriftHiveMetastore_drop_role_presult { - public: - - - virtual ~ThriftHiveMetastore_drop_role_presult() noexcept; - bool* success; - MetaException o1; - - _ThriftHiveMetastore_drop_role_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - - -class ThriftHiveMetastore_get_role_names_args { - public: - - ThriftHiveMetastore_get_role_names_args(const ThriftHiveMetastore_get_role_names_args&); - ThriftHiveMetastore_get_role_names_args& operator=(const ThriftHiveMetastore_get_role_names_args&); - ThriftHiveMetastore_get_role_names_args() { - } - - virtual ~ThriftHiveMetastore_get_role_names_args() noexcept; - - bool operator == (const ThriftHiveMetastore_get_role_names_args & /* rhs */) const - { - return true; - } - bool operator != (const ThriftHiveMetastore_get_role_names_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_role_names_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_role_names_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_role_names_pargs() noexcept; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_role_names_result__isset { - _ThriftHiveMetastore_get_role_names_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_role_names_result__isset; - -class ThriftHiveMetastore_get_role_names_result { - public: - - ThriftHiveMetastore_get_role_names_result(const ThriftHiveMetastore_get_role_names_result&); - ThriftHiveMetastore_get_role_names_result& operator=(const ThriftHiveMetastore_get_role_names_result&); - ThriftHiveMetastore_get_role_names_result() { - } - - virtual ~ThriftHiveMetastore_get_role_names_result() noexcept; - std::vector success; - MetaException o1; - - _ThriftHiveMetastore_get_role_names_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_role_names_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_role_names_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_role_names_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_role_names_presult__isset { - _ThriftHiveMetastore_get_role_names_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_role_names_presult__isset; - -class ThriftHiveMetastore_get_role_names_presult { - public: - - - virtual ~ThriftHiveMetastore_get_role_names_presult() noexcept; - std::vector * success; - MetaException o1; - - _ThriftHiveMetastore_get_role_names_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_grant_role_args__isset { - _ThriftHiveMetastore_grant_role_args__isset() : role_name(false), principal_name(false), principal_type(false), grantor(false), grantorType(false), grant_option(false) {} - bool role_name :1; - bool principal_name :1; - bool principal_type :1; - bool grantor :1; - bool grantorType :1; - bool grant_option :1; -} _ThriftHiveMetastore_grant_role_args__isset; - -class ThriftHiveMetastore_grant_role_args { - public: - - ThriftHiveMetastore_grant_role_args(const ThriftHiveMetastore_grant_role_args&); - ThriftHiveMetastore_grant_role_args& operator=(const ThriftHiveMetastore_grant_role_args&); - ThriftHiveMetastore_grant_role_args() : role_name(), principal_name(), principal_type((PrincipalType::type)0), grantor(), grantorType((PrincipalType::type)0), grant_option(0) { - } - - virtual ~ThriftHiveMetastore_grant_role_args() noexcept; - std::string role_name; - std::string principal_name; - PrincipalType::type principal_type; - std::string grantor; - PrincipalType::type grantorType; - bool grant_option; - - _ThriftHiveMetastore_grant_role_args__isset __isset; - - void __set_role_name(const std::string& val); - - void __set_principal_name(const std::string& val); - - void __set_principal_type(const PrincipalType::type val); - - void __set_grantor(const std::string& val); - - void __set_grantorType(const PrincipalType::type val); - - void __set_grant_option(const bool val); - - bool operator == (const ThriftHiveMetastore_grant_role_args & rhs) const - { - if (!(role_name == rhs.role_name)) - return false; - if (!(principal_name == rhs.principal_name)) - return false; - if (!(principal_type == rhs.principal_type)) - return false; - if (!(grantor == rhs.grantor)) - return false; - if (!(grantorType == rhs.grantorType)) - return false; - if (!(grant_option == rhs.grant_option)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_grant_role_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_grant_role_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_grant_role_pargs { - public: - - - virtual ~ThriftHiveMetastore_grant_role_pargs() noexcept; - const std::string* role_name; - const std::string* principal_name; - const PrincipalType::type* principal_type; - const std::string* grantor; - const PrincipalType::type* grantorType; - const bool* grant_option; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_grant_role_result__isset { - _ThriftHiveMetastore_grant_role_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_grant_role_result__isset; - -class ThriftHiveMetastore_grant_role_result { - public: - - ThriftHiveMetastore_grant_role_result(const ThriftHiveMetastore_grant_role_result&); - ThriftHiveMetastore_grant_role_result& operator=(const ThriftHiveMetastore_grant_role_result&); - ThriftHiveMetastore_grant_role_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_grant_role_result() noexcept; - bool success; - MetaException o1; - - _ThriftHiveMetastore_grant_role_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_grant_role_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_grant_role_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_grant_role_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_grant_role_presult__isset { - _ThriftHiveMetastore_grant_role_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_grant_role_presult__isset; - -class ThriftHiveMetastore_grant_role_presult { - public: - - - virtual ~ThriftHiveMetastore_grant_role_presult() noexcept; - bool* success; - MetaException o1; - - _ThriftHiveMetastore_grant_role_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_revoke_role_args__isset { - _ThriftHiveMetastore_revoke_role_args__isset() : role_name(false), principal_name(false), principal_type(false) {} - bool role_name :1; - bool principal_name :1; - bool principal_type :1; -} _ThriftHiveMetastore_revoke_role_args__isset; - -class ThriftHiveMetastore_revoke_role_args { - public: - - ThriftHiveMetastore_revoke_role_args(const ThriftHiveMetastore_revoke_role_args&); - ThriftHiveMetastore_revoke_role_args& operator=(const ThriftHiveMetastore_revoke_role_args&); - ThriftHiveMetastore_revoke_role_args() : role_name(), principal_name(), principal_type((PrincipalType::type)0) { - } - - virtual ~ThriftHiveMetastore_revoke_role_args() noexcept; - std::string role_name; - std::string principal_name; - PrincipalType::type principal_type; - - _ThriftHiveMetastore_revoke_role_args__isset __isset; - - void __set_role_name(const std::string& val); - - void __set_principal_name(const std::string& val); - - void __set_principal_type(const PrincipalType::type val); - - bool operator == (const ThriftHiveMetastore_revoke_role_args & rhs) const - { - if (!(role_name == rhs.role_name)) - return false; - if (!(principal_name == rhs.principal_name)) - return false; - if (!(principal_type == rhs.principal_type)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_revoke_role_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_revoke_role_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_revoke_role_pargs { - public: - - - virtual ~ThriftHiveMetastore_revoke_role_pargs() noexcept; - const std::string* role_name; - const std::string* principal_name; - const PrincipalType::type* principal_type; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_revoke_role_result__isset { - _ThriftHiveMetastore_revoke_role_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_revoke_role_result__isset; - -class ThriftHiveMetastore_revoke_role_result { - public: - - ThriftHiveMetastore_revoke_role_result(const ThriftHiveMetastore_revoke_role_result&); - ThriftHiveMetastore_revoke_role_result& operator=(const ThriftHiveMetastore_revoke_role_result&); - ThriftHiveMetastore_revoke_role_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_revoke_role_result() noexcept; - bool success; - MetaException o1; - - _ThriftHiveMetastore_revoke_role_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_revoke_role_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_revoke_role_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_revoke_role_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_revoke_role_presult__isset { - _ThriftHiveMetastore_revoke_role_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_revoke_role_presult__isset; - -class ThriftHiveMetastore_revoke_role_presult { - public: - - - virtual ~ThriftHiveMetastore_revoke_role_presult() noexcept; - bool* success; - MetaException o1; - - _ThriftHiveMetastore_revoke_role_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_list_roles_args__isset { - _ThriftHiveMetastore_list_roles_args__isset() : principal_name(false), principal_type(false) {} - bool principal_name :1; - bool principal_type :1; -} _ThriftHiveMetastore_list_roles_args__isset; - -class ThriftHiveMetastore_list_roles_args { - public: - - ThriftHiveMetastore_list_roles_args(const ThriftHiveMetastore_list_roles_args&); - ThriftHiveMetastore_list_roles_args& operator=(const ThriftHiveMetastore_list_roles_args&); - ThriftHiveMetastore_list_roles_args() : principal_name(), principal_type((PrincipalType::type)0) { - } - - virtual ~ThriftHiveMetastore_list_roles_args() noexcept; - std::string principal_name; - PrincipalType::type principal_type; - - _ThriftHiveMetastore_list_roles_args__isset __isset; - - void __set_principal_name(const std::string& val); - - void __set_principal_type(const PrincipalType::type val); - - bool operator == (const ThriftHiveMetastore_list_roles_args & rhs) const - { - if (!(principal_name == rhs.principal_name)) - return false; - if (!(principal_type == rhs.principal_type)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_list_roles_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_list_roles_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_list_roles_pargs { - public: - - - virtual ~ThriftHiveMetastore_list_roles_pargs() noexcept; - const std::string* principal_name; - const PrincipalType::type* principal_type; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_list_roles_result__isset { - _ThriftHiveMetastore_list_roles_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_list_roles_result__isset; - -class ThriftHiveMetastore_list_roles_result { - public: - - ThriftHiveMetastore_list_roles_result(const ThriftHiveMetastore_list_roles_result&); - ThriftHiveMetastore_list_roles_result& operator=(const ThriftHiveMetastore_list_roles_result&); - ThriftHiveMetastore_list_roles_result() { - } - - virtual ~ThriftHiveMetastore_list_roles_result() noexcept; - std::vector success; - MetaException o1; - - _ThriftHiveMetastore_list_roles_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_list_roles_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_list_roles_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_list_roles_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_list_roles_presult__isset { - _ThriftHiveMetastore_list_roles_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_list_roles_presult__isset; - -class ThriftHiveMetastore_list_roles_presult { - public: - - - virtual ~ThriftHiveMetastore_list_roles_presult() noexcept; - std::vector * success; - MetaException o1; - - _ThriftHiveMetastore_list_roles_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_grant_revoke_role_args__isset { - _ThriftHiveMetastore_grant_revoke_role_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_grant_revoke_role_args__isset; - -class ThriftHiveMetastore_grant_revoke_role_args { - public: - - ThriftHiveMetastore_grant_revoke_role_args(const ThriftHiveMetastore_grant_revoke_role_args&); - ThriftHiveMetastore_grant_revoke_role_args& operator=(const ThriftHiveMetastore_grant_revoke_role_args&); - ThriftHiveMetastore_grant_revoke_role_args() { - } - - virtual ~ThriftHiveMetastore_grant_revoke_role_args() noexcept; - GrantRevokeRoleRequest request; - - _ThriftHiveMetastore_grant_revoke_role_args__isset __isset; - - void __set_request(const GrantRevokeRoleRequest& val); - - bool operator == (const ThriftHiveMetastore_grant_revoke_role_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_grant_revoke_role_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_grant_revoke_role_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_grant_revoke_role_pargs { - public: - - - virtual ~ThriftHiveMetastore_grant_revoke_role_pargs() noexcept; - const GrantRevokeRoleRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_grant_revoke_role_result__isset { - _ThriftHiveMetastore_grant_revoke_role_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_grant_revoke_role_result__isset; - -class ThriftHiveMetastore_grant_revoke_role_result { - public: - - ThriftHiveMetastore_grant_revoke_role_result(const ThriftHiveMetastore_grant_revoke_role_result&); - ThriftHiveMetastore_grant_revoke_role_result& operator=(const ThriftHiveMetastore_grant_revoke_role_result&); - ThriftHiveMetastore_grant_revoke_role_result() { - } - - virtual ~ThriftHiveMetastore_grant_revoke_role_result() noexcept; - GrantRevokeRoleResponse success; - MetaException o1; - - _ThriftHiveMetastore_grant_revoke_role_result__isset __isset; - - void __set_success(const GrantRevokeRoleResponse& val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_grant_revoke_role_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_grant_revoke_role_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_grant_revoke_role_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_grant_revoke_role_presult__isset { - _ThriftHiveMetastore_grant_revoke_role_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_grant_revoke_role_presult__isset; - -class ThriftHiveMetastore_grant_revoke_role_presult { - public: - - - virtual ~ThriftHiveMetastore_grant_revoke_role_presult() noexcept; - GrantRevokeRoleResponse* success; - MetaException o1; - - _ThriftHiveMetastore_grant_revoke_role_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_principals_in_role_args__isset { - _ThriftHiveMetastore_get_principals_in_role_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_get_principals_in_role_args__isset; - -class ThriftHiveMetastore_get_principals_in_role_args { - public: - - ThriftHiveMetastore_get_principals_in_role_args(const ThriftHiveMetastore_get_principals_in_role_args&); - ThriftHiveMetastore_get_principals_in_role_args& operator=(const ThriftHiveMetastore_get_principals_in_role_args&); - ThriftHiveMetastore_get_principals_in_role_args() { - } - - virtual ~ThriftHiveMetastore_get_principals_in_role_args() noexcept; - GetPrincipalsInRoleRequest request; - - _ThriftHiveMetastore_get_principals_in_role_args__isset __isset; - - void __set_request(const GetPrincipalsInRoleRequest& val); - - bool operator == (const ThriftHiveMetastore_get_principals_in_role_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_principals_in_role_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_principals_in_role_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_principals_in_role_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_principals_in_role_pargs() noexcept; - const GetPrincipalsInRoleRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_principals_in_role_result__isset { - _ThriftHiveMetastore_get_principals_in_role_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_principals_in_role_result__isset; - -class ThriftHiveMetastore_get_principals_in_role_result { - public: - - ThriftHiveMetastore_get_principals_in_role_result(const ThriftHiveMetastore_get_principals_in_role_result&); - ThriftHiveMetastore_get_principals_in_role_result& operator=(const ThriftHiveMetastore_get_principals_in_role_result&); - ThriftHiveMetastore_get_principals_in_role_result() { - } - - virtual ~ThriftHiveMetastore_get_principals_in_role_result() noexcept; - GetPrincipalsInRoleResponse success; - MetaException o1; - - _ThriftHiveMetastore_get_principals_in_role_result__isset __isset; - - void __set_success(const GetPrincipalsInRoleResponse& val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_principals_in_role_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_principals_in_role_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_principals_in_role_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_principals_in_role_presult__isset { - _ThriftHiveMetastore_get_principals_in_role_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_principals_in_role_presult__isset; - -class ThriftHiveMetastore_get_principals_in_role_presult { - public: - - - virtual ~ThriftHiveMetastore_get_principals_in_role_presult() noexcept; - GetPrincipalsInRoleResponse* success; - MetaException o1; - - _ThriftHiveMetastore_get_principals_in_role_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_role_grants_for_principal_args__isset { - _ThriftHiveMetastore_get_role_grants_for_principal_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_get_role_grants_for_principal_args__isset; - -class ThriftHiveMetastore_get_role_grants_for_principal_args { - public: - - ThriftHiveMetastore_get_role_grants_for_principal_args(const ThriftHiveMetastore_get_role_grants_for_principal_args&); - ThriftHiveMetastore_get_role_grants_for_principal_args& operator=(const ThriftHiveMetastore_get_role_grants_for_principal_args&); - ThriftHiveMetastore_get_role_grants_for_principal_args() { - } - - virtual ~ThriftHiveMetastore_get_role_grants_for_principal_args() noexcept; - GetRoleGrantsForPrincipalRequest request; - - _ThriftHiveMetastore_get_role_grants_for_principal_args__isset __isset; - - void __set_request(const GetRoleGrantsForPrincipalRequest& val); - - bool operator == (const ThriftHiveMetastore_get_role_grants_for_principal_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_role_grants_for_principal_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_role_grants_for_principal_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_role_grants_for_principal_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_role_grants_for_principal_pargs() noexcept; - const GetRoleGrantsForPrincipalRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_role_grants_for_principal_result__isset { - _ThriftHiveMetastore_get_role_grants_for_principal_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_role_grants_for_principal_result__isset; - -class ThriftHiveMetastore_get_role_grants_for_principal_result { - public: - - ThriftHiveMetastore_get_role_grants_for_principal_result(const ThriftHiveMetastore_get_role_grants_for_principal_result&); - ThriftHiveMetastore_get_role_grants_for_principal_result& operator=(const ThriftHiveMetastore_get_role_grants_for_principal_result&); - ThriftHiveMetastore_get_role_grants_for_principal_result() { - } - - virtual ~ThriftHiveMetastore_get_role_grants_for_principal_result() noexcept; - GetRoleGrantsForPrincipalResponse success; - MetaException o1; - - _ThriftHiveMetastore_get_role_grants_for_principal_result__isset __isset; - - void __set_success(const GetRoleGrantsForPrincipalResponse& val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_role_grants_for_principal_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_role_grants_for_principal_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_role_grants_for_principal_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_role_grants_for_principal_presult__isset { - _ThriftHiveMetastore_get_role_grants_for_principal_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_role_grants_for_principal_presult__isset; - -class ThriftHiveMetastore_get_role_grants_for_principal_presult { - public: - - - virtual ~ThriftHiveMetastore_get_role_grants_for_principal_presult() noexcept; - GetRoleGrantsForPrincipalResponse* success; - MetaException o1; - - _ThriftHiveMetastore_get_role_grants_for_principal_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_privilege_set_args__isset { - _ThriftHiveMetastore_get_privilege_set_args__isset() : hiveObject(false), user_name(false), group_names(false) {} - bool hiveObject :1; - bool user_name :1; - bool group_names :1; -} _ThriftHiveMetastore_get_privilege_set_args__isset; - -class ThriftHiveMetastore_get_privilege_set_args { - public: - - ThriftHiveMetastore_get_privilege_set_args(const ThriftHiveMetastore_get_privilege_set_args&); - ThriftHiveMetastore_get_privilege_set_args& operator=(const ThriftHiveMetastore_get_privilege_set_args&); - ThriftHiveMetastore_get_privilege_set_args() : user_name() { - } - - virtual ~ThriftHiveMetastore_get_privilege_set_args() noexcept; - HiveObjectRef hiveObject; - std::string user_name; - std::vector group_names; - - _ThriftHiveMetastore_get_privilege_set_args__isset __isset; - - void __set_hiveObject(const HiveObjectRef& val); - - void __set_user_name(const std::string& val); - - void __set_group_names(const std::vector & val); - - bool operator == (const ThriftHiveMetastore_get_privilege_set_args & rhs) const - { - if (!(hiveObject == rhs.hiveObject)) - return false; - if (!(user_name == rhs.user_name)) - return false; - if (!(group_names == rhs.group_names)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_privilege_set_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_privilege_set_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_privilege_set_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_privilege_set_pargs() noexcept; - const HiveObjectRef* hiveObject; - const std::string* user_name; - const std::vector * group_names; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_privilege_set_result__isset { - _ThriftHiveMetastore_get_privilege_set_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_privilege_set_result__isset; - -class ThriftHiveMetastore_get_privilege_set_result { - public: - - ThriftHiveMetastore_get_privilege_set_result(const ThriftHiveMetastore_get_privilege_set_result&); - ThriftHiveMetastore_get_privilege_set_result& operator=(const ThriftHiveMetastore_get_privilege_set_result&); - ThriftHiveMetastore_get_privilege_set_result() { - } - - virtual ~ThriftHiveMetastore_get_privilege_set_result() noexcept; - PrincipalPrivilegeSet success; - MetaException o1; - - _ThriftHiveMetastore_get_privilege_set_result__isset __isset; - - void __set_success(const PrincipalPrivilegeSet& val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_privilege_set_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_privilege_set_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_privilege_set_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_privilege_set_presult__isset { - _ThriftHiveMetastore_get_privilege_set_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_privilege_set_presult__isset; - -class ThriftHiveMetastore_get_privilege_set_presult { - public: - - - virtual ~ThriftHiveMetastore_get_privilege_set_presult() noexcept; - PrincipalPrivilegeSet* success; - MetaException o1; - - _ThriftHiveMetastore_get_privilege_set_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_list_privileges_args__isset { - _ThriftHiveMetastore_list_privileges_args__isset() : principal_name(false), principal_type(false), hiveObject(false) {} - bool principal_name :1; - bool principal_type :1; - bool hiveObject :1; -} _ThriftHiveMetastore_list_privileges_args__isset; - -class ThriftHiveMetastore_list_privileges_args { - public: - - ThriftHiveMetastore_list_privileges_args(const ThriftHiveMetastore_list_privileges_args&); - ThriftHiveMetastore_list_privileges_args& operator=(const ThriftHiveMetastore_list_privileges_args&); - ThriftHiveMetastore_list_privileges_args() : principal_name(), principal_type((PrincipalType::type)0) { - } - - virtual ~ThriftHiveMetastore_list_privileges_args() noexcept; - std::string principal_name; - PrincipalType::type principal_type; - HiveObjectRef hiveObject; - - _ThriftHiveMetastore_list_privileges_args__isset __isset; - - void __set_principal_name(const std::string& val); - - void __set_principal_type(const PrincipalType::type val); - - void __set_hiveObject(const HiveObjectRef& val); - - bool operator == (const ThriftHiveMetastore_list_privileges_args & rhs) const - { - if (!(principal_name == rhs.principal_name)) - return false; - if (!(principal_type == rhs.principal_type)) - return false; - if (!(hiveObject == rhs.hiveObject)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_list_privileges_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_list_privileges_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_list_privileges_pargs { - public: - - - virtual ~ThriftHiveMetastore_list_privileges_pargs() noexcept; - const std::string* principal_name; - const PrincipalType::type* principal_type; - const HiveObjectRef* hiveObject; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_list_privileges_result__isset { - _ThriftHiveMetastore_list_privileges_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_list_privileges_result__isset; - -class ThriftHiveMetastore_list_privileges_result { - public: - - ThriftHiveMetastore_list_privileges_result(const ThriftHiveMetastore_list_privileges_result&); - ThriftHiveMetastore_list_privileges_result& operator=(const ThriftHiveMetastore_list_privileges_result&); - ThriftHiveMetastore_list_privileges_result() { - } - - virtual ~ThriftHiveMetastore_list_privileges_result() noexcept; - std::vector success; - MetaException o1; - - _ThriftHiveMetastore_list_privileges_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_list_privileges_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_list_privileges_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_list_privileges_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_list_privileges_presult__isset { - _ThriftHiveMetastore_list_privileges_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_list_privileges_presult__isset; - -class ThriftHiveMetastore_list_privileges_presult { - public: - - - virtual ~ThriftHiveMetastore_list_privileges_presult() noexcept; - std::vector * success; - MetaException o1; - - _ThriftHiveMetastore_list_privileges_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_grant_privileges_args__isset { - _ThriftHiveMetastore_grant_privileges_args__isset() : privileges(false) {} - bool privileges :1; -} _ThriftHiveMetastore_grant_privileges_args__isset; - -class ThriftHiveMetastore_grant_privileges_args { - public: - - ThriftHiveMetastore_grant_privileges_args(const ThriftHiveMetastore_grant_privileges_args&); - ThriftHiveMetastore_grant_privileges_args& operator=(const ThriftHiveMetastore_grant_privileges_args&); - ThriftHiveMetastore_grant_privileges_args() { - } - - virtual ~ThriftHiveMetastore_grant_privileges_args() noexcept; - PrivilegeBag privileges; - - _ThriftHiveMetastore_grant_privileges_args__isset __isset; - - void __set_privileges(const PrivilegeBag& val); - - bool operator == (const ThriftHiveMetastore_grant_privileges_args & rhs) const - { - if (!(privileges == rhs.privileges)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_grant_privileges_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_grant_privileges_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_grant_privileges_pargs { - public: - - - virtual ~ThriftHiveMetastore_grant_privileges_pargs() noexcept; - const PrivilegeBag* privileges; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_grant_privileges_result__isset { - _ThriftHiveMetastore_grant_privileges_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_grant_privileges_result__isset; - -class ThriftHiveMetastore_grant_privileges_result { - public: - - ThriftHiveMetastore_grant_privileges_result(const ThriftHiveMetastore_grant_privileges_result&); - ThriftHiveMetastore_grant_privileges_result& operator=(const ThriftHiveMetastore_grant_privileges_result&); - ThriftHiveMetastore_grant_privileges_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_grant_privileges_result() noexcept; - bool success; - MetaException o1; - - _ThriftHiveMetastore_grant_privileges_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_grant_privileges_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_grant_privileges_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_grant_privileges_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_grant_privileges_presult__isset { - _ThriftHiveMetastore_grant_privileges_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_grant_privileges_presult__isset; - -class ThriftHiveMetastore_grant_privileges_presult { - public: - - - virtual ~ThriftHiveMetastore_grant_privileges_presult() noexcept; - bool* success; - MetaException o1; - - _ThriftHiveMetastore_grant_privileges_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_revoke_privileges_args__isset { - _ThriftHiveMetastore_revoke_privileges_args__isset() : privileges(false) {} - bool privileges :1; -} _ThriftHiveMetastore_revoke_privileges_args__isset; - -class ThriftHiveMetastore_revoke_privileges_args { - public: - - ThriftHiveMetastore_revoke_privileges_args(const ThriftHiveMetastore_revoke_privileges_args&); - ThriftHiveMetastore_revoke_privileges_args& operator=(const ThriftHiveMetastore_revoke_privileges_args&); - ThriftHiveMetastore_revoke_privileges_args() { - } - - virtual ~ThriftHiveMetastore_revoke_privileges_args() noexcept; - PrivilegeBag privileges; - - _ThriftHiveMetastore_revoke_privileges_args__isset __isset; - - void __set_privileges(const PrivilegeBag& val); - - bool operator == (const ThriftHiveMetastore_revoke_privileges_args & rhs) const - { - if (!(privileges == rhs.privileges)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_revoke_privileges_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_revoke_privileges_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_revoke_privileges_pargs { - public: - - - virtual ~ThriftHiveMetastore_revoke_privileges_pargs() noexcept; - const PrivilegeBag* privileges; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_revoke_privileges_result__isset { - _ThriftHiveMetastore_revoke_privileges_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_revoke_privileges_result__isset; - -class ThriftHiveMetastore_revoke_privileges_result { - public: - - ThriftHiveMetastore_revoke_privileges_result(const ThriftHiveMetastore_revoke_privileges_result&); - ThriftHiveMetastore_revoke_privileges_result& operator=(const ThriftHiveMetastore_revoke_privileges_result&); - ThriftHiveMetastore_revoke_privileges_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_revoke_privileges_result() noexcept; - bool success; - MetaException o1; - - _ThriftHiveMetastore_revoke_privileges_result__isset __isset; - - void __set_success(const bool val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_revoke_privileges_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_revoke_privileges_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_revoke_privileges_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_revoke_privileges_presult__isset { - _ThriftHiveMetastore_revoke_privileges_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_revoke_privileges_presult__isset; - -class ThriftHiveMetastore_revoke_privileges_presult { - public: - - - virtual ~ThriftHiveMetastore_revoke_privileges_presult() noexcept; - bool* success; - MetaException o1; - - _ThriftHiveMetastore_revoke_privileges_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_grant_revoke_privileges_args__isset { - _ThriftHiveMetastore_grant_revoke_privileges_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_grant_revoke_privileges_args__isset; - -class ThriftHiveMetastore_grant_revoke_privileges_args { - public: - - ThriftHiveMetastore_grant_revoke_privileges_args(const ThriftHiveMetastore_grant_revoke_privileges_args&); - ThriftHiveMetastore_grant_revoke_privileges_args& operator=(const ThriftHiveMetastore_grant_revoke_privileges_args&); - ThriftHiveMetastore_grant_revoke_privileges_args() { - } - - virtual ~ThriftHiveMetastore_grant_revoke_privileges_args() noexcept; - GrantRevokePrivilegeRequest request; - - _ThriftHiveMetastore_grant_revoke_privileges_args__isset __isset; - - void __set_request(const GrantRevokePrivilegeRequest& val); - - bool operator == (const ThriftHiveMetastore_grant_revoke_privileges_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_grant_revoke_privileges_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_grant_revoke_privileges_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_grant_revoke_privileges_pargs { - public: - - - virtual ~ThriftHiveMetastore_grant_revoke_privileges_pargs() noexcept; - const GrantRevokePrivilegeRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_grant_revoke_privileges_result__isset { - _ThriftHiveMetastore_grant_revoke_privileges_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_grant_revoke_privileges_result__isset; - -class ThriftHiveMetastore_grant_revoke_privileges_result { - public: - - ThriftHiveMetastore_grant_revoke_privileges_result(const ThriftHiveMetastore_grant_revoke_privileges_result&); - ThriftHiveMetastore_grant_revoke_privileges_result& operator=(const ThriftHiveMetastore_grant_revoke_privileges_result&); - ThriftHiveMetastore_grant_revoke_privileges_result() { - } - - virtual ~ThriftHiveMetastore_grant_revoke_privileges_result() noexcept; - GrantRevokePrivilegeResponse success; - MetaException o1; - - _ThriftHiveMetastore_grant_revoke_privileges_result__isset __isset; - - void __set_success(const GrantRevokePrivilegeResponse& val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_grant_revoke_privileges_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_grant_revoke_privileges_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_grant_revoke_privileges_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_grant_revoke_privileges_presult__isset { - _ThriftHiveMetastore_grant_revoke_privileges_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_grant_revoke_privileges_presult__isset; - -class ThriftHiveMetastore_grant_revoke_privileges_presult { - public: - - - virtual ~ThriftHiveMetastore_grant_revoke_privileges_presult() noexcept; - GrantRevokePrivilegeResponse* success; - MetaException o1; - - _ThriftHiveMetastore_grant_revoke_privileges_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_set_ugi_args__isset { - _ThriftHiveMetastore_set_ugi_args__isset() : user_name(false), group_names(false) {} - bool user_name :1; - bool group_names :1; -} _ThriftHiveMetastore_set_ugi_args__isset; - -class ThriftHiveMetastore_set_ugi_args { - public: - - ThriftHiveMetastore_set_ugi_args(const ThriftHiveMetastore_set_ugi_args&); - ThriftHiveMetastore_set_ugi_args& operator=(const ThriftHiveMetastore_set_ugi_args&); - ThriftHiveMetastore_set_ugi_args() : user_name() { - } - - virtual ~ThriftHiveMetastore_set_ugi_args() noexcept; - std::string user_name; - std::vector group_names; - - _ThriftHiveMetastore_set_ugi_args__isset __isset; - - void __set_user_name(const std::string& val); - - void __set_group_names(const std::vector & val); - - bool operator == (const ThriftHiveMetastore_set_ugi_args & rhs) const - { - if (!(user_name == rhs.user_name)) - return false; - if (!(group_names == rhs.group_names)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_set_ugi_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_set_ugi_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_set_ugi_pargs { - public: - - - virtual ~ThriftHiveMetastore_set_ugi_pargs() noexcept; - const std::string* user_name; - const std::vector * group_names; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_set_ugi_result__isset { - _ThriftHiveMetastore_set_ugi_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_set_ugi_result__isset; - -class ThriftHiveMetastore_set_ugi_result { - public: - - ThriftHiveMetastore_set_ugi_result(const ThriftHiveMetastore_set_ugi_result&); - ThriftHiveMetastore_set_ugi_result& operator=(const ThriftHiveMetastore_set_ugi_result&); - ThriftHiveMetastore_set_ugi_result() { - } - - virtual ~ThriftHiveMetastore_set_ugi_result() noexcept; - std::vector success; - MetaException o1; - - _ThriftHiveMetastore_set_ugi_result__isset __isset; - - void __set_success(const std::vector & val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_set_ugi_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_set_ugi_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_set_ugi_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_set_ugi_presult__isset { - _ThriftHiveMetastore_set_ugi_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_set_ugi_presult__isset; - -class ThriftHiveMetastore_set_ugi_presult { - public: - - - virtual ~ThriftHiveMetastore_set_ugi_presult() noexcept; - std::vector * success; - MetaException o1; - - _ThriftHiveMetastore_set_ugi_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_delegation_token_args__isset { - _ThriftHiveMetastore_get_delegation_token_args__isset() : token_owner(false), renewer_kerberos_principal_name(false) {} - bool token_owner :1; - bool renewer_kerberos_principal_name :1; -} _ThriftHiveMetastore_get_delegation_token_args__isset; - -class ThriftHiveMetastore_get_delegation_token_args { - public: - - ThriftHiveMetastore_get_delegation_token_args(const ThriftHiveMetastore_get_delegation_token_args&); - ThriftHiveMetastore_get_delegation_token_args& operator=(const ThriftHiveMetastore_get_delegation_token_args&); - ThriftHiveMetastore_get_delegation_token_args() : token_owner(), renewer_kerberos_principal_name() { - } - - virtual ~ThriftHiveMetastore_get_delegation_token_args() noexcept; - std::string token_owner; - std::string renewer_kerberos_principal_name; - - _ThriftHiveMetastore_get_delegation_token_args__isset __isset; - - void __set_token_owner(const std::string& val); - - void __set_renewer_kerberos_principal_name(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_delegation_token_args & rhs) const - { - if (!(token_owner == rhs.token_owner)) - return false; - if (!(renewer_kerberos_principal_name == rhs.renewer_kerberos_principal_name)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_delegation_token_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_delegation_token_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_delegation_token_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_delegation_token_pargs() noexcept; - const std::string* token_owner; - const std::string* renewer_kerberos_principal_name; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_delegation_token_result__isset { - _ThriftHiveMetastore_get_delegation_token_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_delegation_token_result__isset; - -class ThriftHiveMetastore_get_delegation_token_result { - public: - - ThriftHiveMetastore_get_delegation_token_result(const ThriftHiveMetastore_get_delegation_token_result&); - ThriftHiveMetastore_get_delegation_token_result& operator=(const ThriftHiveMetastore_get_delegation_token_result&); - ThriftHiveMetastore_get_delegation_token_result() : success() { - } - - virtual ~ThriftHiveMetastore_get_delegation_token_result() noexcept; - std::string success; - MetaException o1; - - _ThriftHiveMetastore_get_delegation_token_result__isset __isset; - - void __set_success(const std::string& val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_delegation_token_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_delegation_token_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_delegation_token_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_delegation_token_presult__isset { - _ThriftHiveMetastore_get_delegation_token_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_delegation_token_presult__isset; - -class ThriftHiveMetastore_get_delegation_token_presult { - public: - - - virtual ~ThriftHiveMetastore_get_delegation_token_presult() noexcept; - std::string* success; - MetaException o1; - - _ThriftHiveMetastore_get_delegation_token_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_renew_delegation_token_args__isset { - _ThriftHiveMetastore_renew_delegation_token_args__isset() : token_str_form(false) {} - bool token_str_form :1; -} _ThriftHiveMetastore_renew_delegation_token_args__isset; - -class ThriftHiveMetastore_renew_delegation_token_args { - public: - - ThriftHiveMetastore_renew_delegation_token_args(const ThriftHiveMetastore_renew_delegation_token_args&); - ThriftHiveMetastore_renew_delegation_token_args& operator=(const ThriftHiveMetastore_renew_delegation_token_args&); - ThriftHiveMetastore_renew_delegation_token_args() : token_str_form() { - } - - virtual ~ThriftHiveMetastore_renew_delegation_token_args() noexcept; - std::string token_str_form; - - _ThriftHiveMetastore_renew_delegation_token_args__isset __isset; - - void __set_token_str_form(const std::string& val); - - bool operator == (const ThriftHiveMetastore_renew_delegation_token_args & rhs) const - { - if (!(token_str_form == rhs.token_str_form)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_renew_delegation_token_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_renew_delegation_token_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_renew_delegation_token_pargs { - public: - - - virtual ~ThriftHiveMetastore_renew_delegation_token_pargs() noexcept; - const std::string* token_str_form; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_renew_delegation_token_result__isset { - _ThriftHiveMetastore_renew_delegation_token_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_renew_delegation_token_result__isset; - -class ThriftHiveMetastore_renew_delegation_token_result { - public: - - ThriftHiveMetastore_renew_delegation_token_result(const ThriftHiveMetastore_renew_delegation_token_result&); - ThriftHiveMetastore_renew_delegation_token_result& operator=(const ThriftHiveMetastore_renew_delegation_token_result&); - ThriftHiveMetastore_renew_delegation_token_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_renew_delegation_token_result() noexcept; - int64_t success; - MetaException o1; - - _ThriftHiveMetastore_renew_delegation_token_result__isset __isset; - - void __set_success(const int64_t val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_renew_delegation_token_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_renew_delegation_token_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_renew_delegation_token_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_renew_delegation_token_presult__isset { - _ThriftHiveMetastore_renew_delegation_token_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_renew_delegation_token_presult__isset; - -class ThriftHiveMetastore_renew_delegation_token_presult { - public: - - - virtual ~ThriftHiveMetastore_renew_delegation_token_presult() noexcept; - int64_t* success; - MetaException o1; - - _ThriftHiveMetastore_renew_delegation_token_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_cancel_delegation_token_args__isset { - _ThriftHiveMetastore_cancel_delegation_token_args__isset() : token_str_form(false) {} - bool token_str_form :1; -} _ThriftHiveMetastore_cancel_delegation_token_args__isset; - -class ThriftHiveMetastore_cancel_delegation_token_args { - public: - - ThriftHiveMetastore_cancel_delegation_token_args(const ThriftHiveMetastore_cancel_delegation_token_args&); - ThriftHiveMetastore_cancel_delegation_token_args& operator=(const ThriftHiveMetastore_cancel_delegation_token_args&); - ThriftHiveMetastore_cancel_delegation_token_args() : token_str_form() { - } - - virtual ~ThriftHiveMetastore_cancel_delegation_token_args() noexcept; - std::string token_str_form; - - _ThriftHiveMetastore_cancel_delegation_token_args__isset __isset; - - void __set_token_str_form(const std::string& val); - - bool operator == (const ThriftHiveMetastore_cancel_delegation_token_args & rhs) const - { - if (!(token_str_form == rhs.token_str_form)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_cancel_delegation_token_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_cancel_delegation_token_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_cancel_delegation_token_pargs { - public: - - - virtual ~ThriftHiveMetastore_cancel_delegation_token_pargs() noexcept; - const std::string* token_str_form; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_cancel_delegation_token_result__isset { - _ThriftHiveMetastore_cancel_delegation_token_result__isset() : o1(false) {} - bool o1 :1; -} _ThriftHiveMetastore_cancel_delegation_token_result__isset; - -class ThriftHiveMetastore_cancel_delegation_token_result { - public: - - ThriftHiveMetastore_cancel_delegation_token_result(const ThriftHiveMetastore_cancel_delegation_token_result&); - ThriftHiveMetastore_cancel_delegation_token_result& operator=(const ThriftHiveMetastore_cancel_delegation_token_result&); - ThriftHiveMetastore_cancel_delegation_token_result() { - } - - virtual ~ThriftHiveMetastore_cancel_delegation_token_result() noexcept; - MetaException o1; - - _ThriftHiveMetastore_cancel_delegation_token_result__isset __isset; - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_cancel_delegation_token_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_cancel_delegation_token_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_cancel_delegation_token_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_cancel_delegation_token_presult__isset { - _ThriftHiveMetastore_cancel_delegation_token_presult__isset() : o1(false) {} - bool o1 :1; -} _ThriftHiveMetastore_cancel_delegation_token_presult__isset; - -class ThriftHiveMetastore_cancel_delegation_token_presult { - public: - - - virtual ~ThriftHiveMetastore_cancel_delegation_token_presult() noexcept; - MetaException o1; - - _ThriftHiveMetastore_cancel_delegation_token_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_add_token_args__isset { - _ThriftHiveMetastore_add_token_args__isset() : token_identifier(false), delegation_token(false) {} - bool token_identifier :1; - bool delegation_token :1; -} _ThriftHiveMetastore_add_token_args__isset; - -class ThriftHiveMetastore_add_token_args { - public: - - ThriftHiveMetastore_add_token_args(const ThriftHiveMetastore_add_token_args&); - ThriftHiveMetastore_add_token_args& operator=(const ThriftHiveMetastore_add_token_args&); - ThriftHiveMetastore_add_token_args() : token_identifier(), delegation_token() { - } - - virtual ~ThriftHiveMetastore_add_token_args() noexcept; - std::string token_identifier; - std::string delegation_token; - - _ThriftHiveMetastore_add_token_args__isset __isset; - - void __set_token_identifier(const std::string& val); - - void __set_delegation_token(const std::string& val); - - bool operator == (const ThriftHiveMetastore_add_token_args & rhs) const - { - if (!(token_identifier == rhs.token_identifier)) - return false; - if (!(delegation_token == rhs.delegation_token)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_token_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_token_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_add_token_pargs { - public: - - - virtual ~ThriftHiveMetastore_add_token_pargs() noexcept; - const std::string* token_identifier; - const std::string* delegation_token; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_token_result__isset { - _ThriftHiveMetastore_add_token_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_add_token_result__isset; - -class ThriftHiveMetastore_add_token_result { - public: - - ThriftHiveMetastore_add_token_result(const ThriftHiveMetastore_add_token_result&); - ThriftHiveMetastore_add_token_result& operator=(const ThriftHiveMetastore_add_token_result&); - ThriftHiveMetastore_add_token_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_add_token_result() noexcept; - bool success; - - _ThriftHiveMetastore_add_token_result__isset __isset; - - void __set_success(const bool val); - - bool operator == (const ThriftHiveMetastore_add_token_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_token_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_token_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_token_presult__isset { - _ThriftHiveMetastore_add_token_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_add_token_presult__isset; - -class ThriftHiveMetastore_add_token_presult { - public: - - - virtual ~ThriftHiveMetastore_add_token_presult() noexcept; - bool* success; - - _ThriftHiveMetastore_add_token_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_remove_token_args__isset { - _ThriftHiveMetastore_remove_token_args__isset() : token_identifier(false) {} - bool token_identifier :1; -} _ThriftHiveMetastore_remove_token_args__isset; - -class ThriftHiveMetastore_remove_token_args { - public: - - ThriftHiveMetastore_remove_token_args(const ThriftHiveMetastore_remove_token_args&); - ThriftHiveMetastore_remove_token_args& operator=(const ThriftHiveMetastore_remove_token_args&); - ThriftHiveMetastore_remove_token_args() : token_identifier() { - } - - virtual ~ThriftHiveMetastore_remove_token_args() noexcept; - std::string token_identifier; - - _ThriftHiveMetastore_remove_token_args__isset __isset; - - void __set_token_identifier(const std::string& val); - - bool operator == (const ThriftHiveMetastore_remove_token_args & rhs) const - { - if (!(token_identifier == rhs.token_identifier)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_remove_token_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_remove_token_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_remove_token_pargs { - public: - - - virtual ~ThriftHiveMetastore_remove_token_pargs() noexcept; - const std::string* token_identifier; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_remove_token_result__isset { - _ThriftHiveMetastore_remove_token_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_remove_token_result__isset; - -class ThriftHiveMetastore_remove_token_result { - public: - - ThriftHiveMetastore_remove_token_result(const ThriftHiveMetastore_remove_token_result&); - ThriftHiveMetastore_remove_token_result& operator=(const ThriftHiveMetastore_remove_token_result&); - ThriftHiveMetastore_remove_token_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_remove_token_result() noexcept; - bool success; - - _ThriftHiveMetastore_remove_token_result__isset __isset; - - void __set_success(const bool val); - - bool operator == (const ThriftHiveMetastore_remove_token_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_remove_token_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_remove_token_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_remove_token_presult__isset { - _ThriftHiveMetastore_remove_token_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_remove_token_presult__isset; - -class ThriftHiveMetastore_remove_token_presult { - public: - - - virtual ~ThriftHiveMetastore_remove_token_presult() noexcept; - bool* success; - - _ThriftHiveMetastore_remove_token_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_token_args__isset { - _ThriftHiveMetastore_get_token_args__isset() : token_identifier(false) {} - bool token_identifier :1; -} _ThriftHiveMetastore_get_token_args__isset; - -class ThriftHiveMetastore_get_token_args { - public: - - ThriftHiveMetastore_get_token_args(const ThriftHiveMetastore_get_token_args&); - ThriftHiveMetastore_get_token_args& operator=(const ThriftHiveMetastore_get_token_args&); - ThriftHiveMetastore_get_token_args() : token_identifier() { - } - - virtual ~ThriftHiveMetastore_get_token_args() noexcept; - std::string token_identifier; - - _ThriftHiveMetastore_get_token_args__isset __isset; - - void __set_token_identifier(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_token_args & rhs) const - { - if (!(token_identifier == rhs.token_identifier)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_token_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_token_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_token_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_token_pargs() noexcept; - const std::string* token_identifier; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_token_result__isset { - _ThriftHiveMetastore_get_token_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_token_result__isset; - -class ThriftHiveMetastore_get_token_result { - public: - - ThriftHiveMetastore_get_token_result(const ThriftHiveMetastore_get_token_result&); - ThriftHiveMetastore_get_token_result& operator=(const ThriftHiveMetastore_get_token_result&); - ThriftHiveMetastore_get_token_result() : success() { - } - - virtual ~ThriftHiveMetastore_get_token_result() noexcept; - std::string success; - - _ThriftHiveMetastore_get_token_result__isset __isset; - - void __set_success(const std::string& val); - - bool operator == (const ThriftHiveMetastore_get_token_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_token_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_token_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_token_presult__isset { - _ThriftHiveMetastore_get_token_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_token_presult__isset; - -class ThriftHiveMetastore_get_token_presult { - public: - - - virtual ~ThriftHiveMetastore_get_token_presult() noexcept; - std::string* success; - - _ThriftHiveMetastore_get_token_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - - -class ThriftHiveMetastore_get_all_token_identifiers_args { - public: - - ThriftHiveMetastore_get_all_token_identifiers_args(const ThriftHiveMetastore_get_all_token_identifiers_args&); - ThriftHiveMetastore_get_all_token_identifiers_args& operator=(const ThriftHiveMetastore_get_all_token_identifiers_args&); - ThriftHiveMetastore_get_all_token_identifiers_args() { - } - - virtual ~ThriftHiveMetastore_get_all_token_identifiers_args() noexcept; - - bool operator == (const ThriftHiveMetastore_get_all_token_identifiers_args & /* rhs */) const - { - return true; - } - bool operator != (const ThriftHiveMetastore_get_all_token_identifiers_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_all_token_identifiers_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_all_token_identifiers_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_all_token_identifiers_pargs() noexcept; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_all_token_identifiers_result__isset { - _ThriftHiveMetastore_get_all_token_identifiers_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_all_token_identifiers_result__isset; - -class ThriftHiveMetastore_get_all_token_identifiers_result { - public: - - ThriftHiveMetastore_get_all_token_identifiers_result(const ThriftHiveMetastore_get_all_token_identifiers_result&); - ThriftHiveMetastore_get_all_token_identifiers_result& operator=(const ThriftHiveMetastore_get_all_token_identifiers_result&); - ThriftHiveMetastore_get_all_token_identifiers_result() { - } - - virtual ~ThriftHiveMetastore_get_all_token_identifiers_result() noexcept; - std::vector success; - - _ThriftHiveMetastore_get_all_token_identifiers_result__isset __isset; - - void __set_success(const std::vector & val); - - bool operator == (const ThriftHiveMetastore_get_all_token_identifiers_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_all_token_identifiers_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_all_token_identifiers_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_all_token_identifiers_presult__isset { - _ThriftHiveMetastore_get_all_token_identifiers_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_all_token_identifiers_presult__isset; - -class ThriftHiveMetastore_get_all_token_identifiers_presult { - public: - - - virtual ~ThriftHiveMetastore_get_all_token_identifiers_presult() noexcept; - std::vector * success; - - _ThriftHiveMetastore_get_all_token_identifiers_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_add_master_key_args__isset { - _ThriftHiveMetastore_add_master_key_args__isset() : key(false) {} - bool key :1; -} _ThriftHiveMetastore_add_master_key_args__isset; - -class ThriftHiveMetastore_add_master_key_args { - public: - - ThriftHiveMetastore_add_master_key_args(const ThriftHiveMetastore_add_master_key_args&); - ThriftHiveMetastore_add_master_key_args& operator=(const ThriftHiveMetastore_add_master_key_args&); - ThriftHiveMetastore_add_master_key_args() : key() { - } - - virtual ~ThriftHiveMetastore_add_master_key_args() noexcept; - std::string key; - - _ThriftHiveMetastore_add_master_key_args__isset __isset; - - void __set_key(const std::string& val); - - bool operator == (const ThriftHiveMetastore_add_master_key_args & rhs) const - { - if (!(key == rhs.key)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_master_key_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_master_key_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_add_master_key_pargs { - public: - - - virtual ~ThriftHiveMetastore_add_master_key_pargs() noexcept; - const std::string* key; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_master_key_result__isset { - _ThriftHiveMetastore_add_master_key_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_add_master_key_result__isset; - -class ThriftHiveMetastore_add_master_key_result { - public: - - ThriftHiveMetastore_add_master_key_result(const ThriftHiveMetastore_add_master_key_result&); - ThriftHiveMetastore_add_master_key_result& operator=(const ThriftHiveMetastore_add_master_key_result&); - ThriftHiveMetastore_add_master_key_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_add_master_key_result() noexcept; - int32_t success; - MetaException o1; - - _ThriftHiveMetastore_add_master_key_result__isset __isset; - - void __set_success(const int32_t val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_add_master_key_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_master_key_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_master_key_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_master_key_presult__isset { - _ThriftHiveMetastore_add_master_key_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_add_master_key_presult__isset; - -class ThriftHiveMetastore_add_master_key_presult { - public: - - - virtual ~ThriftHiveMetastore_add_master_key_presult() noexcept; - int32_t* success; - MetaException o1; - - _ThriftHiveMetastore_add_master_key_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_update_master_key_args__isset { - _ThriftHiveMetastore_update_master_key_args__isset() : seq_number(false), key(false) {} - bool seq_number :1; - bool key :1; -} _ThriftHiveMetastore_update_master_key_args__isset; - -class ThriftHiveMetastore_update_master_key_args { - public: - - ThriftHiveMetastore_update_master_key_args(const ThriftHiveMetastore_update_master_key_args&); - ThriftHiveMetastore_update_master_key_args& operator=(const ThriftHiveMetastore_update_master_key_args&); - ThriftHiveMetastore_update_master_key_args() : seq_number(0), key() { - } - - virtual ~ThriftHiveMetastore_update_master_key_args() noexcept; - int32_t seq_number; - std::string key; - - _ThriftHiveMetastore_update_master_key_args__isset __isset; - - void __set_seq_number(const int32_t val); - - void __set_key(const std::string& val); - - bool operator == (const ThriftHiveMetastore_update_master_key_args & rhs) const - { - if (!(seq_number == rhs.seq_number)) - return false; - if (!(key == rhs.key)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_update_master_key_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_update_master_key_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_update_master_key_pargs { - public: - - - virtual ~ThriftHiveMetastore_update_master_key_pargs() noexcept; - const int32_t* seq_number; - const std::string* key; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_update_master_key_result__isset { - _ThriftHiveMetastore_update_master_key_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_update_master_key_result__isset; - -class ThriftHiveMetastore_update_master_key_result { - public: - - ThriftHiveMetastore_update_master_key_result(const ThriftHiveMetastore_update_master_key_result&); - ThriftHiveMetastore_update_master_key_result& operator=(const ThriftHiveMetastore_update_master_key_result&); - ThriftHiveMetastore_update_master_key_result() { - } - - virtual ~ThriftHiveMetastore_update_master_key_result() noexcept; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_update_master_key_result__isset __isset; - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_update_master_key_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_update_master_key_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_update_master_key_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_update_master_key_presult__isset { - _ThriftHiveMetastore_update_master_key_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_update_master_key_presult__isset; - -class ThriftHiveMetastore_update_master_key_presult { - public: - - - virtual ~ThriftHiveMetastore_update_master_key_presult() noexcept; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_update_master_key_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_remove_master_key_args__isset { - _ThriftHiveMetastore_remove_master_key_args__isset() : key_seq(false) {} - bool key_seq :1; -} _ThriftHiveMetastore_remove_master_key_args__isset; - -class ThriftHiveMetastore_remove_master_key_args { - public: - - ThriftHiveMetastore_remove_master_key_args(const ThriftHiveMetastore_remove_master_key_args&); - ThriftHiveMetastore_remove_master_key_args& operator=(const ThriftHiveMetastore_remove_master_key_args&); - ThriftHiveMetastore_remove_master_key_args() : key_seq(0) { - } - - virtual ~ThriftHiveMetastore_remove_master_key_args() noexcept; - int32_t key_seq; - - _ThriftHiveMetastore_remove_master_key_args__isset __isset; - - void __set_key_seq(const int32_t val); - - bool operator == (const ThriftHiveMetastore_remove_master_key_args & rhs) const - { - if (!(key_seq == rhs.key_seq)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_remove_master_key_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_remove_master_key_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_remove_master_key_pargs { - public: - - - virtual ~ThriftHiveMetastore_remove_master_key_pargs() noexcept; - const int32_t* key_seq; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_remove_master_key_result__isset { - _ThriftHiveMetastore_remove_master_key_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_remove_master_key_result__isset; - -class ThriftHiveMetastore_remove_master_key_result { - public: - - ThriftHiveMetastore_remove_master_key_result(const ThriftHiveMetastore_remove_master_key_result&); - ThriftHiveMetastore_remove_master_key_result& operator=(const ThriftHiveMetastore_remove_master_key_result&); - ThriftHiveMetastore_remove_master_key_result() : success(0) { - } - - virtual ~ThriftHiveMetastore_remove_master_key_result() noexcept; - bool success; - - _ThriftHiveMetastore_remove_master_key_result__isset __isset; - - void __set_success(const bool val); - - bool operator == (const ThriftHiveMetastore_remove_master_key_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_remove_master_key_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_remove_master_key_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_remove_master_key_presult__isset { - _ThriftHiveMetastore_remove_master_key_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_remove_master_key_presult__isset; - -class ThriftHiveMetastore_remove_master_key_presult { - public: - - - virtual ~ThriftHiveMetastore_remove_master_key_presult() noexcept; - bool* success; - - _ThriftHiveMetastore_remove_master_key_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - - -class ThriftHiveMetastore_get_master_keys_args { - public: - - ThriftHiveMetastore_get_master_keys_args(const ThriftHiveMetastore_get_master_keys_args&); - ThriftHiveMetastore_get_master_keys_args& operator=(const ThriftHiveMetastore_get_master_keys_args&); - ThriftHiveMetastore_get_master_keys_args() { - } - - virtual ~ThriftHiveMetastore_get_master_keys_args() noexcept; - - bool operator == (const ThriftHiveMetastore_get_master_keys_args & /* rhs */) const - { - return true; - } - bool operator != (const ThriftHiveMetastore_get_master_keys_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_master_keys_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_master_keys_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_master_keys_pargs() noexcept; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_master_keys_result__isset { - _ThriftHiveMetastore_get_master_keys_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_master_keys_result__isset; - -class ThriftHiveMetastore_get_master_keys_result { - public: - - ThriftHiveMetastore_get_master_keys_result(const ThriftHiveMetastore_get_master_keys_result&); - ThriftHiveMetastore_get_master_keys_result& operator=(const ThriftHiveMetastore_get_master_keys_result&); - ThriftHiveMetastore_get_master_keys_result() { - } - - virtual ~ThriftHiveMetastore_get_master_keys_result() noexcept; - std::vector success; - - _ThriftHiveMetastore_get_master_keys_result__isset __isset; - - void __set_success(const std::vector & val); - - bool operator == (const ThriftHiveMetastore_get_master_keys_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_master_keys_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_master_keys_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_master_keys_presult__isset { - _ThriftHiveMetastore_get_master_keys_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_master_keys_presult__isset; - -class ThriftHiveMetastore_get_master_keys_presult { - public: - - - virtual ~ThriftHiveMetastore_get_master_keys_presult() noexcept; - std::vector * success; - - _ThriftHiveMetastore_get_master_keys_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - - -class ThriftHiveMetastore_get_open_txns_args { - public: - - ThriftHiveMetastore_get_open_txns_args(const ThriftHiveMetastore_get_open_txns_args&); - ThriftHiveMetastore_get_open_txns_args& operator=(const ThriftHiveMetastore_get_open_txns_args&); - ThriftHiveMetastore_get_open_txns_args() { - } - - virtual ~ThriftHiveMetastore_get_open_txns_args() noexcept; - - bool operator == (const ThriftHiveMetastore_get_open_txns_args & /* rhs */) const - { - return true; - } - bool operator != (const ThriftHiveMetastore_get_open_txns_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_open_txns_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_open_txns_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_open_txns_pargs() noexcept; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_open_txns_result__isset { - _ThriftHiveMetastore_get_open_txns_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_open_txns_result__isset; - -class ThriftHiveMetastore_get_open_txns_result { - public: - - ThriftHiveMetastore_get_open_txns_result(const ThriftHiveMetastore_get_open_txns_result&); - ThriftHiveMetastore_get_open_txns_result& operator=(const ThriftHiveMetastore_get_open_txns_result&); - ThriftHiveMetastore_get_open_txns_result() { - } - - virtual ~ThriftHiveMetastore_get_open_txns_result() noexcept; - GetOpenTxnsResponse success; - - _ThriftHiveMetastore_get_open_txns_result__isset __isset; - - void __set_success(const GetOpenTxnsResponse& val); - - bool operator == (const ThriftHiveMetastore_get_open_txns_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_open_txns_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_open_txns_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_open_txns_presult__isset { - _ThriftHiveMetastore_get_open_txns_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_open_txns_presult__isset; - -class ThriftHiveMetastore_get_open_txns_presult { - public: - - - virtual ~ThriftHiveMetastore_get_open_txns_presult() noexcept; - GetOpenTxnsResponse* success; - - _ThriftHiveMetastore_get_open_txns_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - - -class ThriftHiveMetastore_get_open_txns_info_args { - public: - - ThriftHiveMetastore_get_open_txns_info_args(const ThriftHiveMetastore_get_open_txns_info_args&); - ThriftHiveMetastore_get_open_txns_info_args& operator=(const ThriftHiveMetastore_get_open_txns_info_args&); - ThriftHiveMetastore_get_open_txns_info_args() { - } - - virtual ~ThriftHiveMetastore_get_open_txns_info_args() noexcept; - - bool operator == (const ThriftHiveMetastore_get_open_txns_info_args & /* rhs */) const - { - return true; - } - bool operator != (const ThriftHiveMetastore_get_open_txns_info_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_open_txns_info_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_open_txns_info_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_open_txns_info_pargs() noexcept; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_open_txns_info_result__isset { - _ThriftHiveMetastore_get_open_txns_info_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_open_txns_info_result__isset; - -class ThriftHiveMetastore_get_open_txns_info_result { - public: - - ThriftHiveMetastore_get_open_txns_info_result(const ThriftHiveMetastore_get_open_txns_info_result&); - ThriftHiveMetastore_get_open_txns_info_result& operator=(const ThriftHiveMetastore_get_open_txns_info_result&); - ThriftHiveMetastore_get_open_txns_info_result() { - } - - virtual ~ThriftHiveMetastore_get_open_txns_info_result() noexcept; - GetOpenTxnsInfoResponse success; - - _ThriftHiveMetastore_get_open_txns_info_result__isset __isset; - - void __set_success(const GetOpenTxnsInfoResponse& val); - - bool operator == (const ThriftHiveMetastore_get_open_txns_info_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_open_txns_info_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_open_txns_info_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_open_txns_info_presult__isset { - _ThriftHiveMetastore_get_open_txns_info_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_open_txns_info_presult__isset; - -class ThriftHiveMetastore_get_open_txns_info_presult { - public: - - - virtual ~ThriftHiveMetastore_get_open_txns_info_presult() noexcept; - GetOpenTxnsInfoResponse* success; - - _ThriftHiveMetastore_get_open_txns_info_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_open_txns_args__isset { - _ThriftHiveMetastore_open_txns_args__isset() : rqst(false) {} - bool rqst :1; -} _ThriftHiveMetastore_open_txns_args__isset; - -class ThriftHiveMetastore_open_txns_args { - public: - - ThriftHiveMetastore_open_txns_args(const ThriftHiveMetastore_open_txns_args&); - ThriftHiveMetastore_open_txns_args& operator=(const ThriftHiveMetastore_open_txns_args&); - ThriftHiveMetastore_open_txns_args() { - } - - virtual ~ThriftHiveMetastore_open_txns_args() noexcept; - OpenTxnRequest rqst; - - _ThriftHiveMetastore_open_txns_args__isset __isset; - - void __set_rqst(const OpenTxnRequest& val); - - bool operator == (const ThriftHiveMetastore_open_txns_args & rhs) const - { - if (!(rqst == rhs.rqst)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_open_txns_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_open_txns_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_open_txns_pargs { - public: - - - virtual ~ThriftHiveMetastore_open_txns_pargs() noexcept; - const OpenTxnRequest* rqst; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_open_txns_result__isset { - _ThriftHiveMetastore_open_txns_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_open_txns_result__isset; - -class ThriftHiveMetastore_open_txns_result { - public: - - ThriftHiveMetastore_open_txns_result(const ThriftHiveMetastore_open_txns_result&); - ThriftHiveMetastore_open_txns_result& operator=(const ThriftHiveMetastore_open_txns_result&); - ThriftHiveMetastore_open_txns_result() { - } - - virtual ~ThriftHiveMetastore_open_txns_result() noexcept; - OpenTxnsResponse success; - - _ThriftHiveMetastore_open_txns_result__isset __isset; - - void __set_success(const OpenTxnsResponse& val); - - bool operator == (const ThriftHiveMetastore_open_txns_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_open_txns_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_open_txns_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_open_txns_presult__isset { - _ThriftHiveMetastore_open_txns_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_open_txns_presult__isset; - -class ThriftHiveMetastore_open_txns_presult { - public: - - - virtual ~ThriftHiveMetastore_open_txns_presult() noexcept; - OpenTxnsResponse* success; - - _ThriftHiveMetastore_open_txns_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_abort_txn_args__isset { - _ThriftHiveMetastore_abort_txn_args__isset() : rqst(false) {} - bool rqst :1; -} _ThriftHiveMetastore_abort_txn_args__isset; - -class ThriftHiveMetastore_abort_txn_args { - public: - - ThriftHiveMetastore_abort_txn_args(const ThriftHiveMetastore_abort_txn_args&); - ThriftHiveMetastore_abort_txn_args& operator=(const ThriftHiveMetastore_abort_txn_args&); - ThriftHiveMetastore_abort_txn_args() { - } - - virtual ~ThriftHiveMetastore_abort_txn_args() noexcept; - AbortTxnRequest rqst; - - _ThriftHiveMetastore_abort_txn_args__isset __isset; - - void __set_rqst(const AbortTxnRequest& val); - - bool operator == (const ThriftHiveMetastore_abort_txn_args & rhs) const - { - if (!(rqst == rhs.rqst)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_abort_txn_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_abort_txn_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_abort_txn_pargs { - public: - - - virtual ~ThriftHiveMetastore_abort_txn_pargs() noexcept; - const AbortTxnRequest* rqst; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_abort_txn_result__isset { - _ThriftHiveMetastore_abort_txn_result__isset() : o1(false) {} - bool o1 :1; -} _ThriftHiveMetastore_abort_txn_result__isset; - -class ThriftHiveMetastore_abort_txn_result { - public: - - ThriftHiveMetastore_abort_txn_result(const ThriftHiveMetastore_abort_txn_result&); - ThriftHiveMetastore_abort_txn_result& operator=(const ThriftHiveMetastore_abort_txn_result&); - ThriftHiveMetastore_abort_txn_result() { - } - - virtual ~ThriftHiveMetastore_abort_txn_result() noexcept; - NoSuchTxnException o1; - - _ThriftHiveMetastore_abort_txn_result__isset __isset; - - void __set_o1(const NoSuchTxnException& val); - - bool operator == (const ThriftHiveMetastore_abort_txn_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_abort_txn_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_abort_txn_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_abort_txn_presult__isset { - _ThriftHiveMetastore_abort_txn_presult__isset() : o1(false) {} - bool o1 :1; -} _ThriftHiveMetastore_abort_txn_presult__isset; - -class ThriftHiveMetastore_abort_txn_presult { - public: - - - virtual ~ThriftHiveMetastore_abort_txn_presult() noexcept; - NoSuchTxnException o1; - - _ThriftHiveMetastore_abort_txn_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_abort_txns_args__isset { - _ThriftHiveMetastore_abort_txns_args__isset() : rqst(false) {} - bool rqst :1; -} _ThriftHiveMetastore_abort_txns_args__isset; - -class ThriftHiveMetastore_abort_txns_args { - public: - - ThriftHiveMetastore_abort_txns_args(const ThriftHiveMetastore_abort_txns_args&); - ThriftHiveMetastore_abort_txns_args& operator=(const ThriftHiveMetastore_abort_txns_args&); - ThriftHiveMetastore_abort_txns_args() { - } - - virtual ~ThriftHiveMetastore_abort_txns_args() noexcept; - AbortTxnsRequest rqst; - - _ThriftHiveMetastore_abort_txns_args__isset __isset; - - void __set_rqst(const AbortTxnsRequest& val); - - bool operator == (const ThriftHiveMetastore_abort_txns_args & rhs) const - { - if (!(rqst == rhs.rqst)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_abort_txns_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_abort_txns_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_abort_txns_pargs { - public: - - - virtual ~ThriftHiveMetastore_abort_txns_pargs() noexcept; - const AbortTxnsRequest* rqst; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_abort_txns_result__isset { - _ThriftHiveMetastore_abort_txns_result__isset() : o1(false) {} - bool o1 :1; -} _ThriftHiveMetastore_abort_txns_result__isset; - -class ThriftHiveMetastore_abort_txns_result { - public: - - ThriftHiveMetastore_abort_txns_result(const ThriftHiveMetastore_abort_txns_result&); - ThriftHiveMetastore_abort_txns_result& operator=(const ThriftHiveMetastore_abort_txns_result&); - ThriftHiveMetastore_abort_txns_result() { - } - - virtual ~ThriftHiveMetastore_abort_txns_result() noexcept; - NoSuchTxnException o1; - - _ThriftHiveMetastore_abort_txns_result__isset __isset; - - void __set_o1(const NoSuchTxnException& val); - - bool operator == (const ThriftHiveMetastore_abort_txns_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_abort_txns_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_abort_txns_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_abort_txns_presult__isset { - _ThriftHiveMetastore_abort_txns_presult__isset() : o1(false) {} - bool o1 :1; -} _ThriftHiveMetastore_abort_txns_presult__isset; - -class ThriftHiveMetastore_abort_txns_presult { - public: - - - virtual ~ThriftHiveMetastore_abort_txns_presult() noexcept; - NoSuchTxnException o1; - - _ThriftHiveMetastore_abort_txns_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_commit_txn_args__isset { - _ThriftHiveMetastore_commit_txn_args__isset() : rqst(false) {} - bool rqst :1; -} _ThriftHiveMetastore_commit_txn_args__isset; - -class ThriftHiveMetastore_commit_txn_args { - public: - - ThriftHiveMetastore_commit_txn_args(const ThriftHiveMetastore_commit_txn_args&); - ThriftHiveMetastore_commit_txn_args& operator=(const ThriftHiveMetastore_commit_txn_args&); - ThriftHiveMetastore_commit_txn_args() { - } - - virtual ~ThriftHiveMetastore_commit_txn_args() noexcept; - CommitTxnRequest rqst; - - _ThriftHiveMetastore_commit_txn_args__isset __isset; - - void __set_rqst(const CommitTxnRequest& val); - - bool operator == (const ThriftHiveMetastore_commit_txn_args & rhs) const - { - if (!(rqst == rhs.rqst)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_commit_txn_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_commit_txn_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_commit_txn_pargs { - public: - - - virtual ~ThriftHiveMetastore_commit_txn_pargs() noexcept; - const CommitTxnRequest* rqst; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_commit_txn_result__isset { - _ThriftHiveMetastore_commit_txn_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_commit_txn_result__isset; - -class ThriftHiveMetastore_commit_txn_result { - public: - - ThriftHiveMetastore_commit_txn_result(const ThriftHiveMetastore_commit_txn_result&); - ThriftHiveMetastore_commit_txn_result& operator=(const ThriftHiveMetastore_commit_txn_result&); - ThriftHiveMetastore_commit_txn_result() { - } - - virtual ~ThriftHiveMetastore_commit_txn_result() noexcept; - NoSuchTxnException o1; - TxnAbortedException o2; - - _ThriftHiveMetastore_commit_txn_result__isset __isset; - - void __set_o1(const NoSuchTxnException& val); - - void __set_o2(const TxnAbortedException& val); - - bool operator == (const ThriftHiveMetastore_commit_txn_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_commit_txn_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_commit_txn_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_commit_txn_presult__isset { - _ThriftHiveMetastore_commit_txn_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_commit_txn_presult__isset; - -class ThriftHiveMetastore_commit_txn_presult { - public: - - - virtual ~ThriftHiveMetastore_commit_txn_presult() noexcept; - NoSuchTxnException o1; - TxnAbortedException o2; - - _ThriftHiveMetastore_commit_txn_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_lock_args__isset { - _ThriftHiveMetastore_lock_args__isset() : rqst(false) {} - bool rqst :1; -} _ThriftHiveMetastore_lock_args__isset; - -class ThriftHiveMetastore_lock_args { - public: - - ThriftHiveMetastore_lock_args(const ThriftHiveMetastore_lock_args&); - ThriftHiveMetastore_lock_args& operator=(const ThriftHiveMetastore_lock_args&); - ThriftHiveMetastore_lock_args() { - } - - virtual ~ThriftHiveMetastore_lock_args() noexcept; - LockRequest rqst; - - _ThriftHiveMetastore_lock_args__isset __isset; - - void __set_rqst(const LockRequest& val); - - bool operator == (const ThriftHiveMetastore_lock_args & rhs) const - { - if (!(rqst == rhs.rqst)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_lock_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_lock_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_lock_pargs { - public: - - - virtual ~ThriftHiveMetastore_lock_pargs() noexcept; - const LockRequest* rqst; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_lock_result__isset { - _ThriftHiveMetastore_lock_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_lock_result__isset; - -class ThriftHiveMetastore_lock_result { - public: - - ThriftHiveMetastore_lock_result(const ThriftHiveMetastore_lock_result&); - ThriftHiveMetastore_lock_result& operator=(const ThriftHiveMetastore_lock_result&); - ThriftHiveMetastore_lock_result() { - } - - virtual ~ThriftHiveMetastore_lock_result() noexcept; - LockResponse success; - NoSuchTxnException o1; - TxnAbortedException o2; - - _ThriftHiveMetastore_lock_result__isset __isset; - - void __set_success(const LockResponse& val); - - void __set_o1(const NoSuchTxnException& val); - - void __set_o2(const TxnAbortedException& val); - - bool operator == (const ThriftHiveMetastore_lock_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_lock_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_lock_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_lock_presult__isset { - _ThriftHiveMetastore_lock_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_lock_presult__isset; - -class ThriftHiveMetastore_lock_presult { - public: - - - virtual ~ThriftHiveMetastore_lock_presult() noexcept; - LockResponse* success; - NoSuchTxnException o1; - TxnAbortedException o2; - - _ThriftHiveMetastore_lock_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_check_lock_args__isset { - _ThriftHiveMetastore_check_lock_args__isset() : rqst(false) {} - bool rqst :1; -} _ThriftHiveMetastore_check_lock_args__isset; - -class ThriftHiveMetastore_check_lock_args { - public: - - ThriftHiveMetastore_check_lock_args(const ThriftHiveMetastore_check_lock_args&); - ThriftHiveMetastore_check_lock_args& operator=(const ThriftHiveMetastore_check_lock_args&); - ThriftHiveMetastore_check_lock_args() { - } - - virtual ~ThriftHiveMetastore_check_lock_args() noexcept; - CheckLockRequest rqst; - - _ThriftHiveMetastore_check_lock_args__isset __isset; - - void __set_rqst(const CheckLockRequest& val); - - bool operator == (const ThriftHiveMetastore_check_lock_args & rhs) const - { - if (!(rqst == rhs.rqst)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_check_lock_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_check_lock_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_check_lock_pargs { - public: - - - virtual ~ThriftHiveMetastore_check_lock_pargs() noexcept; - const CheckLockRequest* rqst; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_check_lock_result__isset { - _ThriftHiveMetastore_check_lock_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_check_lock_result__isset; - -class ThriftHiveMetastore_check_lock_result { - public: - - ThriftHiveMetastore_check_lock_result(const ThriftHiveMetastore_check_lock_result&); - ThriftHiveMetastore_check_lock_result& operator=(const ThriftHiveMetastore_check_lock_result&); - ThriftHiveMetastore_check_lock_result() { - } - - virtual ~ThriftHiveMetastore_check_lock_result() noexcept; - LockResponse success; - NoSuchTxnException o1; - TxnAbortedException o2; - NoSuchLockException o3; - - _ThriftHiveMetastore_check_lock_result__isset __isset; - - void __set_success(const LockResponse& val); - - void __set_o1(const NoSuchTxnException& val); - - void __set_o2(const TxnAbortedException& val); - - void __set_o3(const NoSuchLockException& val); - - bool operator == (const ThriftHiveMetastore_check_lock_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_check_lock_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_check_lock_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_check_lock_presult__isset { - _ThriftHiveMetastore_check_lock_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_check_lock_presult__isset; - -class ThriftHiveMetastore_check_lock_presult { - public: - - - virtual ~ThriftHiveMetastore_check_lock_presult() noexcept; - LockResponse* success; - NoSuchTxnException o1; - TxnAbortedException o2; - NoSuchLockException o3; - - _ThriftHiveMetastore_check_lock_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_unlock_args__isset { - _ThriftHiveMetastore_unlock_args__isset() : rqst(false) {} - bool rqst :1; -} _ThriftHiveMetastore_unlock_args__isset; - -class ThriftHiveMetastore_unlock_args { - public: - - ThriftHiveMetastore_unlock_args(const ThriftHiveMetastore_unlock_args&); - ThriftHiveMetastore_unlock_args& operator=(const ThriftHiveMetastore_unlock_args&); - ThriftHiveMetastore_unlock_args() { - } - - virtual ~ThriftHiveMetastore_unlock_args() noexcept; - UnlockRequest rqst; - - _ThriftHiveMetastore_unlock_args__isset __isset; - - void __set_rqst(const UnlockRequest& val); - - bool operator == (const ThriftHiveMetastore_unlock_args & rhs) const - { - if (!(rqst == rhs.rqst)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_unlock_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_unlock_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_unlock_pargs { - public: - - - virtual ~ThriftHiveMetastore_unlock_pargs() noexcept; - const UnlockRequest* rqst; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_unlock_result__isset { - _ThriftHiveMetastore_unlock_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_unlock_result__isset; - -class ThriftHiveMetastore_unlock_result { - public: - - ThriftHiveMetastore_unlock_result(const ThriftHiveMetastore_unlock_result&); - ThriftHiveMetastore_unlock_result& operator=(const ThriftHiveMetastore_unlock_result&); - ThriftHiveMetastore_unlock_result() { - } - - virtual ~ThriftHiveMetastore_unlock_result() noexcept; - NoSuchLockException o1; - TxnOpenException o2; - - _ThriftHiveMetastore_unlock_result__isset __isset; - - void __set_o1(const NoSuchLockException& val); - - void __set_o2(const TxnOpenException& val); - - bool operator == (const ThriftHiveMetastore_unlock_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_unlock_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_unlock_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_unlock_presult__isset { - _ThriftHiveMetastore_unlock_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_unlock_presult__isset; - -class ThriftHiveMetastore_unlock_presult { - public: - - - virtual ~ThriftHiveMetastore_unlock_presult() noexcept; - NoSuchLockException o1; - TxnOpenException o2; - - _ThriftHiveMetastore_unlock_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_show_locks_args__isset { - _ThriftHiveMetastore_show_locks_args__isset() : rqst(false) {} - bool rqst :1; -} _ThriftHiveMetastore_show_locks_args__isset; - -class ThriftHiveMetastore_show_locks_args { - public: - - ThriftHiveMetastore_show_locks_args(const ThriftHiveMetastore_show_locks_args&); - ThriftHiveMetastore_show_locks_args& operator=(const ThriftHiveMetastore_show_locks_args&); - ThriftHiveMetastore_show_locks_args() { - } - - virtual ~ThriftHiveMetastore_show_locks_args() noexcept; - ShowLocksRequest rqst; - - _ThriftHiveMetastore_show_locks_args__isset __isset; - - void __set_rqst(const ShowLocksRequest& val); - - bool operator == (const ThriftHiveMetastore_show_locks_args & rhs) const - { - if (!(rqst == rhs.rqst)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_show_locks_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_show_locks_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_show_locks_pargs { - public: - - - virtual ~ThriftHiveMetastore_show_locks_pargs() noexcept; - const ShowLocksRequest* rqst; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_show_locks_result__isset { - _ThriftHiveMetastore_show_locks_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_show_locks_result__isset; - -class ThriftHiveMetastore_show_locks_result { - public: - - ThriftHiveMetastore_show_locks_result(const ThriftHiveMetastore_show_locks_result&); - ThriftHiveMetastore_show_locks_result& operator=(const ThriftHiveMetastore_show_locks_result&); - ThriftHiveMetastore_show_locks_result() { - } - - virtual ~ThriftHiveMetastore_show_locks_result() noexcept; - ShowLocksResponse success; - - _ThriftHiveMetastore_show_locks_result__isset __isset; - - void __set_success(const ShowLocksResponse& val); - - bool operator == (const ThriftHiveMetastore_show_locks_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_show_locks_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_show_locks_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_show_locks_presult__isset { - _ThriftHiveMetastore_show_locks_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_show_locks_presult__isset; - -class ThriftHiveMetastore_show_locks_presult { - public: - - - virtual ~ThriftHiveMetastore_show_locks_presult() noexcept; - ShowLocksResponse* success; - - _ThriftHiveMetastore_show_locks_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_heartbeat_args__isset { - _ThriftHiveMetastore_heartbeat_args__isset() : ids(false) {} - bool ids :1; -} _ThriftHiveMetastore_heartbeat_args__isset; - -class ThriftHiveMetastore_heartbeat_args { - public: - - ThriftHiveMetastore_heartbeat_args(const ThriftHiveMetastore_heartbeat_args&); - ThriftHiveMetastore_heartbeat_args& operator=(const ThriftHiveMetastore_heartbeat_args&); - ThriftHiveMetastore_heartbeat_args() { - } - - virtual ~ThriftHiveMetastore_heartbeat_args() noexcept; - HeartbeatRequest ids; - - _ThriftHiveMetastore_heartbeat_args__isset __isset; - - void __set_ids(const HeartbeatRequest& val); - - bool operator == (const ThriftHiveMetastore_heartbeat_args & rhs) const - { - if (!(ids == rhs.ids)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_heartbeat_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_heartbeat_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_heartbeat_pargs { - public: - - - virtual ~ThriftHiveMetastore_heartbeat_pargs() noexcept; - const HeartbeatRequest* ids; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_heartbeat_result__isset { - _ThriftHiveMetastore_heartbeat_result__isset() : o1(false), o2(false), o3(false) {} - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_heartbeat_result__isset; - -class ThriftHiveMetastore_heartbeat_result { - public: - - ThriftHiveMetastore_heartbeat_result(const ThriftHiveMetastore_heartbeat_result&); - ThriftHiveMetastore_heartbeat_result& operator=(const ThriftHiveMetastore_heartbeat_result&); - ThriftHiveMetastore_heartbeat_result() { - } - - virtual ~ThriftHiveMetastore_heartbeat_result() noexcept; - NoSuchLockException o1; - NoSuchTxnException o2; - TxnAbortedException o3; - - _ThriftHiveMetastore_heartbeat_result__isset __isset; - - void __set_o1(const NoSuchLockException& val); - - void __set_o2(const NoSuchTxnException& val); - - void __set_o3(const TxnAbortedException& val); - - bool operator == (const ThriftHiveMetastore_heartbeat_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_heartbeat_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_heartbeat_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_heartbeat_presult__isset { - _ThriftHiveMetastore_heartbeat_presult__isset() : o1(false), o2(false), o3(false) {} - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_heartbeat_presult__isset; - -class ThriftHiveMetastore_heartbeat_presult { - public: - - - virtual ~ThriftHiveMetastore_heartbeat_presult() noexcept; - NoSuchLockException o1; - NoSuchTxnException o2; - TxnAbortedException o3; - - _ThriftHiveMetastore_heartbeat_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_heartbeat_txn_range_args__isset { - _ThriftHiveMetastore_heartbeat_txn_range_args__isset() : txns(false) {} - bool txns :1; -} _ThriftHiveMetastore_heartbeat_txn_range_args__isset; - -class ThriftHiveMetastore_heartbeat_txn_range_args { - public: - - ThriftHiveMetastore_heartbeat_txn_range_args(const ThriftHiveMetastore_heartbeat_txn_range_args&); - ThriftHiveMetastore_heartbeat_txn_range_args& operator=(const ThriftHiveMetastore_heartbeat_txn_range_args&); - ThriftHiveMetastore_heartbeat_txn_range_args() { - } - - virtual ~ThriftHiveMetastore_heartbeat_txn_range_args() noexcept; - HeartbeatTxnRangeRequest txns; - - _ThriftHiveMetastore_heartbeat_txn_range_args__isset __isset; - - void __set_txns(const HeartbeatTxnRangeRequest& val); - - bool operator == (const ThriftHiveMetastore_heartbeat_txn_range_args & rhs) const - { - if (!(txns == rhs.txns)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_heartbeat_txn_range_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_heartbeat_txn_range_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_heartbeat_txn_range_pargs { - public: - - - virtual ~ThriftHiveMetastore_heartbeat_txn_range_pargs() noexcept; - const HeartbeatTxnRangeRequest* txns; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_heartbeat_txn_range_result__isset { - _ThriftHiveMetastore_heartbeat_txn_range_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_heartbeat_txn_range_result__isset; - -class ThriftHiveMetastore_heartbeat_txn_range_result { - public: - - ThriftHiveMetastore_heartbeat_txn_range_result(const ThriftHiveMetastore_heartbeat_txn_range_result&); - ThriftHiveMetastore_heartbeat_txn_range_result& operator=(const ThriftHiveMetastore_heartbeat_txn_range_result&); - ThriftHiveMetastore_heartbeat_txn_range_result() { - } - - virtual ~ThriftHiveMetastore_heartbeat_txn_range_result() noexcept; - HeartbeatTxnRangeResponse success; - - _ThriftHiveMetastore_heartbeat_txn_range_result__isset __isset; - - void __set_success(const HeartbeatTxnRangeResponse& val); - - bool operator == (const ThriftHiveMetastore_heartbeat_txn_range_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_heartbeat_txn_range_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_heartbeat_txn_range_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_heartbeat_txn_range_presult__isset { - _ThriftHiveMetastore_heartbeat_txn_range_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_heartbeat_txn_range_presult__isset; - -class ThriftHiveMetastore_heartbeat_txn_range_presult { - public: - - - virtual ~ThriftHiveMetastore_heartbeat_txn_range_presult() noexcept; - HeartbeatTxnRangeResponse* success; - - _ThriftHiveMetastore_heartbeat_txn_range_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_compact_args__isset { - _ThriftHiveMetastore_compact_args__isset() : rqst(false) {} - bool rqst :1; -} _ThriftHiveMetastore_compact_args__isset; - -class ThriftHiveMetastore_compact_args { - public: - - ThriftHiveMetastore_compact_args(const ThriftHiveMetastore_compact_args&); - ThriftHiveMetastore_compact_args& operator=(const ThriftHiveMetastore_compact_args&); - ThriftHiveMetastore_compact_args() { - } - - virtual ~ThriftHiveMetastore_compact_args() noexcept; - CompactionRequest rqst; - - _ThriftHiveMetastore_compact_args__isset __isset; - - void __set_rqst(const CompactionRequest& val); - - bool operator == (const ThriftHiveMetastore_compact_args & rhs) const - { - if (!(rqst == rhs.rqst)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_compact_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_compact_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_compact_pargs { - public: - - - virtual ~ThriftHiveMetastore_compact_pargs() noexcept; - const CompactionRequest* rqst; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_compact_result { - public: - - ThriftHiveMetastore_compact_result(const ThriftHiveMetastore_compact_result&); - ThriftHiveMetastore_compact_result& operator=(const ThriftHiveMetastore_compact_result&); - ThriftHiveMetastore_compact_result() { - } - - virtual ~ThriftHiveMetastore_compact_result() noexcept; - - bool operator == (const ThriftHiveMetastore_compact_result & /* rhs */) const - { - return true; - } - bool operator != (const ThriftHiveMetastore_compact_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_compact_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_compact_presult { - public: - - - virtual ~ThriftHiveMetastore_compact_presult() noexcept; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_compact2_args__isset { - _ThriftHiveMetastore_compact2_args__isset() : rqst(false) {} - bool rqst :1; -} _ThriftHiveMetastore_compact2_args__isset; - -class ThriftHiveMetastore_compact2_args { - public: - - ThriftHiveMetastore_compact2_args(const ThriftHiveMetastore_compact2_args&); - ThriftHiveMetastore_compact2_args& operator=(const ThriftHiveMetastore_compact2_args&); - ThriftHiveMetastore_compact2_args() { - } - - virtual ~ThriftHiveMetastore_compact2_args() noexcept; - CompactionRequest rqst; - - _ThriftHiveMetastore_compact2_args__isset __isset; - - void __set_rqst(const CompactionRequest& val); - - bool operator == (const ThriftHiveMetastore_compact2_args & rhs) const - { - if (!(rqst == rhs.rqst)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_compact2_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_compact2_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_compact2_pargs { - public: - - - virtual ~ThriftHiveMetastore_compact2_pargs() noexcept; - const CompactionRequest* rqst; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_compact2_result__isset { - _ThriftHiveMetastore_compact2_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_compact2_result__isset; - -class ThriftHiveMetastore_compact2_result { - public: - - ThriftHiveMetastore_compact2_result(const ThriftHiveMetastore_compact2_result&); - ThriftHiveMetastore_compact2_result& operator=(const ThriftHiveMetastore_compact2_result&); - ThriftHiveMetastore_compact2_result() { - } - - virtual ~ThriftHiveMetastore_compact2_result() noexcept; - CompactionResponse success; - - _ThriftHiveMetastore_compact2_result__isset __isset; - - void __set_success(const CompactionResponse& val); - - bool operator == (const ThriftHiveMetastore_compact2_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_compact2_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_compact2_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_compact2_presult__isset { - _ThriftHiveMetastore_compact2_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_compact2_presult__isset; - -class ThriftHiveMetastore_compact2_presult { - public: - - - virtual ~ThriftHiveMetastore_compact2_presult() noexcept; - CompactionResponse* success; - - _ThriftHiveMetastore_compact2_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_show_compact_args__isset { - _ThriftHiveMetastore_show_compact_args__isset() : rqst(false) {} - bool rqst :1; -} _ThriftHiveMetastore_show_compact_args__isset; - -class ThriftHiveMetastore_show_compact_args { - public: - - ThriftHiveMetastore_show_compact_args(const ThriftHiveMetastore_show_compact_args&); - ThriftHiveMetastore_show_compact_args& operator=(const ThriftHiveMetastore_show_compact_args&); - ThriftHiveMetastore_show_compact_args() { - } - - virtual ~ThriftHiveMetastore_show_compact_args() noexcept; - ShowCompactRequest rqst; - - _ThriftHiveMetastore_show_compact_args__isset __isset; - - void __set_rqst(const ShowCompactRequest& val); - - bool operator == (const ThriftHiveMetastore_show_compact_args & rhs) const - { - if (!(rqst == rhs.rqst)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_show_compact_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_show_compact_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_show_compact_pargs { - public: - - - virtual ~ThriftHiveMetastore_show_compact_pargs() noexcept; - const ShowCompactRequest* rqst; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_show_compact_result__isset { - _ThriftHiveMetastore_show_compact_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_show_compact_result__isset; - -class ThriftHiveMetastore_show_compact_result { - public: - - ThriftHiveMetastore_show_compact_result(const ThriftHiveMetastore_show_compact_result&); - ThriftHiveMetastore_show_compact_result& operator=(const ThriftHiveMetastore_show_compact_result&); - ThriftHiveMetastore_show_compact_result() { - } - - virtual ~ThriftHiveMetastore_show_compact_result() noexcept; - ShowCompactResponse success; - - _ThriftHiveMetastore_show_compact_result__isset __isset; - - void __set_success(const ShowCompactResponse& val); - - bool operator == (const ThriftHiveMetastore_show_compact_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_show_compact_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_show_compact_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_show_compact_presult__isset { - _ThriftHiveMetastore_show_compact_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_show_compact_presult__isset; - -class ThriftHiveMetastore_show_compact_presult { - public: - - - virtual ~ThriftHiveMetastore_show_compact_presult() noexcept; - ShowCompactResponse* success; - - _ThriftHiveMetastore_show_compact_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_add_dynamic_partitions_args__isset { - _ThriftHiveMetastore_add_dynamic_partitions_args__isset() : rqst(false) {} - bool rqst :1; -} _ThriftHiveMetastore_add_dynamic_partitions_args__isset; - -class ThriftHiveMetastore_add_dynamic_partitions_args { - public: - - ThriftHiveMetastore_add_dynamic_partitions_args(const ThriftHiveMetastore_add_dynamic_partitions_args&); - ThriftHiveMetastore_add_dynamic_partitions_args& operator=(const ThriftHiveMetastore_add_dynamic_partitions_args&); - ThriftHiveMetastore_add_dynamic_partitions_args() { - } - - virtual ~ThriftHiveMetastore_add_dynamic_partitions_args() noexcept; - AddDynamicPartitions rqst; - - _ThriftHiveMetastore_add_dynamic_partitions_args__isset __isset; - - void __set_rqst(const AddDynamicPartitions& val); - - bool operator == (const ThriftHiveMetastore_add_dynamic_partitions_args & rhs) const - { - if (!(rqst == rhs.rqst)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_dynamic_partitions_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_dynamic_partitions_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_add_dynamic_partitions_pargs { - public: - - - virtual ~ThriftHiveMetastore_add_dynamic_partitions_pargs() noexcept; - const AddDynamicPartitions* rqst; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_dynamic_partitions_result__isset { - _ThriftHiveMetastore_add_dynamic_partitions_result__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_add_dynamic_partitions_result__isset; - -class ThriftHiveMetastore_add_dynamic_partitions_result { - public: - - ThriftHiveMetastore_add_dynamic_partitions_result(const ThriftHiveMetastore_add_dynamic_partitions_result&); - ThriftHiveMetastore_add_dynamic_partitions_result& operator=(const ThriftHiveMetastore_add_dynamic_partitions_result&); - ThriftHiveMetastore_add_dynamic_partitions_result() { - } - - virtual ~ThriftHiveMetastore_add_dynamic_partitions_result() noexcept; - NoSuchTxnException o1; - TxnAbortedException o2; - - _ThriftHiveMetastore_add_dynamic_partitions_result__isset __isset; - - void __set_o1(const NoSuchTxnException& val); - - void __set_o2(const TxnAbortedException& val); - - bool operator == (const ThriftHiveMetastore_add_dynamic_partitions_result & rhs) const - { - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_add_dynamic_partitions_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_add_dynamic_partitions_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_add_dynamic_partitions_presult__isset { - _ThriftHiveMetastore_add_dynamic_partitions_presult__isset() : o1(false), o2(false) {} - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_add_dynamic_partitions_presult__isset; - -class ThriftHiveMetastore_add_dynamic_partitions_presult { - public: - - - virtual ~ThriftHiveMetastore_add_dynamic_partitions_presult() noexcept; - NoSuchTxnException o1; - TxnAbortedException o2; - - _ThriftHiveMetastore_add_dynamic_partitions_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_next_notification_args__isset { - _ThriftHiveMetastore_get_next_notification_args__isset() : rqst(false) {} - bool rqst :1; -} _ThriftHiveMetastore_get_next_notification_args__isset; - -class ThriftHiveMetastore_get_next_notification_args { - public: - - ThriftHiveMetastore_get_next_notification_args(const ThriftHiveMetastore_get_next_notification_args&); - ThriftHiveMetastore_get_next_notification_args& operator=(const ThriftHiveMetastore_get_next_notification_args&); - ThriftHiveMetastore_get_next_notification_args() { - } - - virtual ~ThriftHiveMetastore_get_next_notification_args() noexcept; - NotificationEventRequest rqst; - - _ThriftHiveMetastore_get_next_notification_args__isset __isset; - - void __set_rqst(const NotificationEventRequest& val); - - bool operator == (const ThriftHiveMetastore_get_next_notification_args & rhs) const - { - if (!(rqst == rhs.rqst)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_next_notification_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_next_notification_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_next_notification_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_next_notification_pargs() noexcept; - const NotificationEventRequest* rqst; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_next_notification_result__isset { - _ThriftHiveMetastore_get_next_notification_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_next_notification_result__isset; - -class ThriftHiveMetastore_get_next_notification_result { - public: - - ThriftHiveMetastore_get_next_notification_result(const ThriftHiveMetastore_get_next_notification_result&); - ThriftHiveMetastore_get_next_notification_result& operator=(const ThriftHiveMetastore_get_next_notification_result&); - ThriftHiveMetastore_get_next_notification_result() { - } - - virtual ~ThriftHiveMetastore_get_next_notification_result() noexcept; - NotificationEventResponse success; - - _ThriftHiveMetastore_get_next_notification_result__isset __isset; - - void __set_success(const NotificationEventResponse& val); - - bool operator == (const ThriftHiveMetastore_get_next_notification_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_next_notification_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_next_notification_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_next_notification_presult__isset { - _ThriftHiveMetastore_get_next_notification_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_next_notification_presult__isset; - -class ThriftHiveMetastore_get_next_notification_presult { - public: - - - virtual ~ThriftHiveMetastore_get_next_notification_presult() noexcept; - NotificationEventResponse* success; - - _ThriftHiveMetastore_get_next_notification_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - - -class ThriftHiveMetastore_get_current_notificationEventId_args { - public: - - ThriftHiveMetastore_get_current_notificationEventId_args(const ThriftHiveMetastore_get_current_notificationEventId_args&); - ThriftHiveMetastore_get_current_notificationEventId_args& operator=(const ThriftHiveMetastore_get_current_notificationEventId_args&); - ThriftHiveMetastore_get_current_notificationEventId_args() { - } - - virtual ~ThriftHiveMetastore_get_current_notificationEventId_args() noexcept; - - bool operator == (const ThriftHiveMetastore_get_current_notificationEventId_args & /* rhs */) const - { - return true; - } - bool operator != (const ThriftHiveMetastore_get_current_notificationEventId_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_current_notificationEventId_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_current_notificationEventId_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_current_notificationEventId_pargs() noexcept; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_current_notificationEventId_result__isset { - _ThriftHiveMetastore_get_current_notificationEventId_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_current_notificationEventId_result__isset; - -class ThriftHiveMetastore_get_current_notificationEventId_result { - public: - - ThriftHiveMetastore_get_current_notificationEventId_result(const ThriftHiveMetastore_get_current_notificationEventId_result&); - ThriftHiveMetastore_get_current_notificationEventId_result& operator=(const ThriftHiveMetastore_get_current_notificationEventId_result&); - ThriftHiveMetastore_get_current_notificationEventId_result() { - } - - virtual ~ThriftHiveMetastore_get_current_notificationEventId_result() noexcept; - CurrentNotificationEventId success; - - _ThriftHiveMetastore_get_current_notificationEventId_result__isset __isset; - - void __set_success(const CurrentNotificationEventId& val); - - bool operator == (const ThriftHiveMetastore_get_current_notificationEventId_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_current_notificationEventId_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_current_notificationEventId_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_current_notificationEventId_presult__isset { - _ThriftHiveMetastore_get_current_notificationEventId_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_current_notificationEventId_presult__isset; - -class ThriftHiveMetastore_get_current_notificationEventId_presult { - public: - - - virtual ~ThriftHiveMetastore_get_current_notificationEventId_presult() noexcept; - CurrentNotificationEventId* success; - - _ThriftHiveMetastore_get_current_notificationEventId_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_notification_events_count_args__isset { - _ThriftHiveMetastore_get_notification_events_count_args__isset() : rqst(false) {} - bool rqst :1; -} _ThriftHiveMetastore_get_notification_events_count_args__isset; - -class ThriftHiveMetastore_get_notification_events_count_args { - public: - - ThriftHiveMetastore_get_notification_events_count_args(const ThriftHiveMetastore_get_notification_events_count_args&); - ThriftHiveMetastore_get_notification_events_count_args& operator=(const ThriftHiveMetastore_get_notification_events_count_args&); - ThriftHiveMetastore_get_notification_events_count_args() { - } - - virtual ~ThriftHiveMetastore_get_notification_events_count_args() noexcept; - NotificationEventsCountRequest rqst; - - _ThriftHiveMetastore_get_notification_events_count_args__isset __isset; - - void __set_rqst(const NotificationEventsCountRequest& val); - - bool operator == (const ThriftHiveMetastore_get_notification_events_count_args & rhs) const - { - if (!(rqst == rhs.rqst)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_notification_events_count_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_notification_events_count_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_notification_events_count_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_notification_events_count_pargs() noexcept; - const NotificationEventsCountRequest* rqst; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_notification_events_count_result__isset { - _ThriftHiveMetastore_get_notification_events_count_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_notification_events_count_result__isset; - -class ThriftHiveMetastore_get_notification_events_count_result { - public: - - ThriftHiveMetastore_get_notification_events_count_result(const ThriftHiveMetastore_get_notification_events_count_result&); - ThriftHiveMetastore_get_notification_events_count_result& operator=(const ThriftHiveMetastore_get_notification_events_count_result&); - ThriftHiveMetastore_get_notification_events_count_result() { - } - - virtual ~ThriftHiveMetastore_get_notification_events_count_result() noexcept; - NotificationEventsCountResponse success; - - _ThriftHiveMetastore_get_notification_events_count_result__isset __isset; - - void __set_success(const NotificationEventsCountResponse& val); - - bool operator == (const ThriftHiveMetastore_get_notification_events_count_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_notification_events_count_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_notification_events_count_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_notification_events_count_presult__isset { - _ThriftHiveMetastore_get_notification_events_count_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_notification_events_count_presult__isset; - -class ThriftHiveMetastore_get_notification_events_count_presult { - public: - - - virtual ~ThriftHiveMetastore_get_notification_events_count_presult() noexcept; - NotificationEventsCountResponse* success; - - _ThriftHiveMetastore_get_notification_events_count_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_fire_listener_event_args__isset { - _ThriftHiveMetastore_fire_listener_event_args__isset() : rqst(false) {} - bool rqst :1; -} _ThriftHiveMetastore_fire_listener_event_args__isset; - -class ThriftHiveMetastore_fire_listener_event_args { - public: - - ThriftHiveMetastore_fire_listener_event_args(const ThriftHiveMetastore_fire_listener_event_args&); - ThriftHiveMetastore_fire_listener_event_args& operator=(const ThriftHiveMetastore_fire_listener_event_args&); - ThriftHiveMetastore_fire_listener_event_args() { - } - - virtual ~ThriftHiveMetastore_fire_listener_event_args() noexcept; - FireEventRequest rqst; - - _ThriftHiveMetastore_fire_listener_event_args__isset __isset; - - void __set_rqst(const FireEventRequest& val); - - bool operator == (const ThriftHiveMetastore_fire_listener_event_args & rhs) const - { - if (!(rqst == rhs.rqst)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_fire_listener_event_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_fire_listener_event_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_fire_listener_event_pargs { - public: - - - virtual ~ThriftHiveMetastore_fire_listener_event_pargs() noexcept; - const FireEventRequest* rqst; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_fire_listener_event_result__isset { - _ThriftHiveMetastore_fire_listener_event_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_fire_listener_event_result__isset; - -class ThriftHiveMetastore_fire_listener_event_result { - public: - - ThriftHiveMetastore_fire_listener_event_result(const ThriftHiveMetastore_fire_listener_event_result&); - ThriftHiveMetastore_fire_listener_event_result& operator=(const ThriftHiveMetastore_fire_listener_event_result&); - ThriftHiveMetastore_fire_listener_event_result() { - } - - virtual ~ThriftHiveMetastore_fire_listener_event_result() noexcept; - FireEventResponse success; - - _ThriftHiveMetastore_fire_listener_event_result__isset __isset; - - void __set_success(const FireEventResponse& val); - - bool operator == (const ThriftHiveMetastore_fire_listener_event_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_fire_listener_event_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_fire_listener_event_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_fire_listener_event_presult__isset { - _ThriftHiveMetastore_fire_listener_event_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_fire_listener_event_presult__isset; - -class ThriftHiveMetastore_fire_listener_event_presult { - public: - - - virtual ~ThriftHiveMetastore_fire_listener_event_presult() noexcept; - FireEventResponse* success; - - _ThriftHiveMetastore_fire_listener_event_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - - -class ThriftHiveMetastore_flushCache_args { - public: - - ThriftHiveMetastore_flushCache_args(const ThriftHiveMetastore_flushCache_args&); - ThriftHiveMetastore_flushCache_args& operator=(const ThriftHiveMetastore_flushCache_args&); - ThriftHiveMetastore_flushCache_args() { - } - - virtual ~ThriftHiveMetastore_flushCache_args() noexcept; - - bool operator == (const ThriftHiveMetastore_flushCache_args & /* rhs */) const - { - return true; - } - bool operator != (const ThriftHiveMetastore_flushCache_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_flushCache_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_flushCache_pargs { - public: - - - virtual ~ThriftHiveMetastore_flushCache_pargs() noexcept; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_flushCache_result { - public: - - ThriftHiveMetastore_flushCache_result(const ThriftHiveMetastore_flushCache_result&); - ThriftHiveMetastore_flushCache_result& operator=(const ThriftHiveMetastore_flushCache_result&); - ThriftHiveMetastore_flushCache_result() { - } - - virtual ~ThriftHiveMetastore_flushCache_result() noexcept; - - bool operator == (const ThriftHiveMetastore_flushCache_result & /* rhs */) const - { - return true; - } - bool operator != (const ThriftHiveMetastore_flushCache_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_flushCache_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_flushCache_presult { - public: - - - virtual ~ThriftHiveMetastore_flushCache_presult() noexcept; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_cm_recycle_args__isset { - _ThriftHiveMetastore_cm_recycle_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_cm_recycle_args__isset; - -class ThriftHiveMetastore_cm_recycle_args { - public: - - ThriftHiveMetastore_cm_recycle_args(const ThriftHiveMetastore_cm_recycle_args&); - ThriftHiveMetastore_cm_recycle_args& operator=(const ThriftHiveMetastore_cm_recycle_args&); - ThriftHiveMetastore_cm_recycle_args() { - } - - virtual ~ThriftHiveMetastore_cm_recycle_args() noexcept; - CmRecycleRequest request; - - _ThriftHiveMetastore_cm_recycle_args__isset __isset; - - void __set_request(const CmRecycleRequest& val); - - bool operator == (const ThriftHiveMetastore_cm_recycle_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_cm_recycle_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_cm_recycle_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_cm_recycle_pargs { - public: - - - virtual ~ThriftHiveMetastore_cm_recycle_pargs() noexcept; - const CmRecycleRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_cm_recycle_result__isset { - _ThriftHiveMetastore_cm_recycle_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_cm_recycle_result__isset; - -class ThriftHiveMetastore_cm_recycle_result { - public: - - ThriftHiveMetastore_cm_recycle_result(const ThriftHiveMetastore_cm_recycle_result&); - ThriftHiveMetastore_cm_recycle_result& operator=(const ThriftHiveMetastore_cm_recycle_result&); - ThriftHiveMetastore_cm_recycle_result() { - } - - virtual ~ThriftHiveMetastore_cm_recycle_result() noexcept; - CmRecycleResponse success; - MetaException o1; - - _ThriftHiveMetastore_cm_recycle_result__isset __isset; - - void __set_success(const CmRecycleResponse& val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_cm_recycle_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_cm_recycle_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_cm_recycle_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_cm_recycle_presult__isset { - _ThriftHiveMetastore_cm_recycle_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_cm_recycle_presult__isset; - -class ThriftHiveMetastore_cm_recycle_presult { - public: - - - virtual ~ThriftHiveMetastore_cm_recycle_presult() noexcept; - CmRecycleResponse* success; - MetaException o1; - - _ThriftHiveMetastore_cm_recycle_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_file_metadata_by_expr_args__isset { - _ThriftHiveMetastore_get_file_metadata_by_expr_args__isset() : req(false) {} - bool req :1; -} _ThriftHiveMetastore_get_file_metadata_by_expr_args__isset; - -class ThriftHiveMetastore_get_file_metadata_by_expr_args { - public: - - ThriftHiveMetastore_get_file_metadata_by_expr_args(const ThriftHiveMetastore_get_file_metadata_by_expr_args&); - ThriftHiveMetastore_get_file_metadata_by_expr_args& operator=(const ThriftHiveMetastore_get_file_metadata_by_expr_args&); - ThriftHiveMetastore_get_file_metadata_by_expr_args() { - } - - virtual ~ThriftHiveMetastore_get_file_metadata_by_expr_args() noexcept; - GetFileMetadataByExprRequest req; - - _ThriftHiveMetastore_get_file_metadata_by_expr_args__isset __isset; - - void __set_req(const GetFileMetadataByExprRequest& val); - - bool operator == (const ThriftHiveMetastore_get_file_metadata_by_expr_args & rhs) const - { - if (!(req == rhs.req)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_file_metadata_by_expr_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_file_metadata_by_expr_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_file_metadata_by_expr_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_file_metadata_by_expr_pargs() noexcept; - const GetFileMetadataByExprRequest* req; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_file_metadata_by_expr_result__isset { - _ThriftHiveMetastore_get_file_metadata_by_expr_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_file_metadata_by_expr_result__isset; - -class ThriftHiveMetastore_get_file_metadata_by_expr_result { - public: - - ThriftHiveMetastore_get_file_metadata_by_expr_result(const ThriftHiveMetastore_get_file_metadata_by_expr_result&); - ThriftHiveMetastore_get_file_metadata_by_expr_result& operator=(const ThriftHiveMetastore_get_file_metadata_by_expr_result&); - ThriftHiveMetastore_get_file_metadata_by_expr_result() { - } - - virtual ~ThriftHiveMetastore_get_file_metadata_by_expr_result() noexcept; - GetFileMetadataByExprResult success; - - _ThriftHiveMetastore_get_file_metadata_by_expr_result__isset __isset; - - void __set_success(const GetFileMetadataByExprResult& val); - - bool operator == (const ThriftHiveMetastore_get_file_metadata_by_expr_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_file_metadata_by_expr_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_file_metadata_by_expr_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_file_metadata_by_expr_presult__isset { - _ThriftHiveMetastore_get_file_metadata_by_expr_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_file_metadata_by_expr_presult__isset; - -class ThriftHiveMetastore_get_file_metadata_by_expr_presult { - public: - - - virtual ~ThriftHiveMetastore_get_file_metadata_by_expr_presult() noexcept; - GetFileMetadataByExprResult* success; - - _ThriftHiveMetastore_get_file_metadata_by_expr_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_file_metadata_args__isset { - _ThriftHiveMetastore_get_file_metadata_args__isset() : req(false) {} - bool req :1; -} _ThriftHiveMetastore_get_file_metadata_args__isset; - -class ThriftHiveMetastore_get_file_metadata_args { - public: - - ThriftHiveMetastore_get_file_metadata_args(const ThriftHiveMetastore_get_file_metadata_args&); - ThriftHiveMetastore_get_file_metadata_args& operator=(const ThriftHiveMetastore_get_file_metadata_args&); - ThriftHiveMetastore_get_file_metadata_args() { - } - - virtual ~ThriftHiveMetastore_get_file_metadata_args() noexcept; - GetFileMetadataRequest req; - - _ThriftHiveMetastore_get_file_metadata_args__isset __isset; - - void __set_req(const GetFileMetadataRequest& val); - - bool operator == (const ThriftHiveMetastore_get_file_metadata_args & rhs) const - { - if (!(req == rhs.req)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_file_metadata_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_file_metadata_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_file_metadata_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_file_metadata_pargs() noexcept; - const GetFileMetadataRequest* req; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_file_metadata_result__isset { - _ThriftHiveMetastore_get_file_metadata_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_file_metadata_result__isset; - -class ThriftHiveMetastore_get_file_metadata_result { - public: - - ThriftHiveMetastore_get_file_metadata_result(const ThriftHiveMetastore_get_file_metadata_result&); - ThriftHiveMetastore_get_file_metadata_result& operator=(const ThriftHiveMetastore_get_file_metadata_result&); - ThriftHiveMetastore_get_file_metadata_result() { - } - - virtual ~ThriftHiveMetastore_get_file_metadata_result() noexcept; - GetFileMetadataResult success; - - _ThriftHiveMetastore_get_file_metadata_result__isset __isset; - - void __set_success(const GetFileMetadataResult& val); - - bool operator == (const ThriftHiveMetastore_get_file_metadata_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_file_metadata_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_file_metadata_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_file_metadata_presult__isset { - _ThriftHiveMetastore_get_file_metadata_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_get_file_metadata_presult__isset; - -class ThriftHiveMetastore_get_file_metadata_presult { - public: - - - virtual ~ThriftHiveMetastore_get_file_metadata_presult() noexcept; - GetFileMetadataResult* success; - - _ThriftHiveMetastore_get_file_metadata_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_put_file_metadata_args__isset { - _ThriftHiveMetastore_put_file_metadata_args__isset() : req(false) {} - bool req :1; -} _ThriftHiveMetastore_put_file_metadata_args__isset; - -class ThriftHiveMetastore_put_file_metadata_args { - public: - - ThriftHiveMetastore_put_file_metadata_args(const ThriftHiveMetastore_put_file_metadata_args&); - ThriftHiveMetastore_put_file_metadata_args& operator=(const ThriftHiveMetastore_put_file_metadata_args&); - ThriftHiveMetastore_put_file_metadata_args() { - } - - virtual ~ThriftHiveMetastore_put_file_metadata_args() noexcept; - PutFileMetadataRequest req; - - _ThriftHiveMetastore_put_file_metadata_args__isset __isset; - - void __set_req(const PutFileMetadataRequest& val); - - bool operator == (const ThriftHiveMetastore_put_file_metadata_args & rhs) const - { - if (!(req == rhs.req)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_put_file_metadata_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_put_file_metadata_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_put_file_metadata_pargs { - public: - - - virtual ~ThriftHiveMetastore_put_file_metadata_pargs() noexcept; - const PutFileMetadataRequest* req; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_put_file_metadata_result__isset { - _ThriftHiveMetastore_put_file_metadata_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_put_file_metadata_result__isset; - -class ThriftHiveMetastore_put_file_metadata_result { - public: - - ThriftHiveMetastore_put_file_metadata_result(const ThriftHiveMetastore_put_file_metadata_result&); - ThriftHiveMetastore_put_file_metadata_result& operator=(const ThriftHiveMetastore_put_file_metadata_result&); - ThriftHiveMetastore_put_file_metadata_result() { - } - - virtual ~ThriftHiveMetastore_put_file_metadata_result() noexcept; - PutFileMetadataResult success; - - _ThriftHiveMetastore_put_file_metadata_result__isset __isset; - - void __set_success(const PutFileMetadataResult& val); - - bool operator == (const ThriftHiveMetastore_put_file_metadata_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_put_file_metadata_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_put_file_metadata_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_put_file_metadata_presult__isset { - _ThriftHiveMetastore_put_file_metadata_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_put_file_metadata_presult__isset; - -class ThriftHiveMetastore_put_file_metadata_presult { - public: - - - virtual ~ThriftHiveMetastore_put_file_metadata_presult() noexcept; - PutFileMetadataResult* success; - - _ThriftHiveMetastore_put_file_metadata_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_clear_file_metadata_args__isset { - _ThriftHiveMetastore_clear_file_metadata_args__isset() : req(false) {} - bool req :1; -} _ThriftHiveMetastore_clear_file_metadata_args__isset; - -class ThriftHiveMetastore_clear_file_metadata_args { - public: - - ThriftHiveMetastore_clear_file_metadata_args(const ThriftHiveMetastore_clear_file_metadata_args&); - ThriftHiveMetastore_clear_file_metadata_args& operator=(const ThriftHiveMetastore_clear_file_metadata_args&); - ThriftHiveMetastore_clear_file_metadata_args() { - } - - virtual ~ThriftHiveMetastore_clear_file_metadata_args() noexcept; - ClearFileMetadataRequest req; - - _ThriftHiveMetastore_clear_file_metadata_args__isset __isset; - - void __set_req(const ClearFileMetadataRequest& val); - - bool operator == (const ThriftHiveMetastore_clear_file_metadata_args & rhs) const - { - if (!(req == rhs.req)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_clear_file_metadata_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_clear_file_metadata_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_clear_file_metadata_pargs { - public: - - - virtual ~ThriftHiveMetastore_clear_file_metadata_pargs() noexcept; - const ClearFileMetadataRequest* req; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_clear_file_metadata_result__isset { - _ThriftHiveMetastore_clear_file_metadata_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_clear_file_metadata_result__isset; - -class ThriftHiveMetastore_clear_file_metadata_result { - public: - - ThriftHiveMetastore_clear_file_metadata_result(const ThriftHiveMetastore_clear_file_metadata_result&); - ThriftHiveMetastore_clear_file_metadata_result& operator=(const ThriftHiveMetastore_clear_file_metadata_result&); - ThriftHiveMetastore_clear_file_metadata_result() { - } - - virtual ~ThriftHiveMetastore_clear_file_metadata_result() noexcept; - ClearFileMetadataResult success; - - _ThriftHiveMetastore_clear_file_metadata_result__isset __isset; - - void __set_success(const ClearFileMetadataResult& val); - - bool operator == (const ThriftHiveMetastore_clear_file_metadata_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_clear_file_metadata_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_clear_file_metadata_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_clear_file_metadata_presult__isset { - _ThriftHiveMetastore_clear_file_metadata_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_clear_file_metadata_presult__isset; - -class ThriftHiveMetastore_clear_file_metadata_presult { - public: - - - virtual ~ThriftHiveMetastore_clear_file_metadata_presult() noexcept; - ClearFileMetadataResult* success; - - _ThriftHiveMetastore_clear_file_metadata_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_cache_file_metadata_args__isset { - _ThriftHiveMetastore_cache_file_metadata_args__isset() : req(false) {} - bool req :1; -} _ThriftHiveMetastore_cache_file_metadata_args__isset; - -class ThriftHiveMetastore_cache_file_metadata_args { - public: - - ThriftHiveMetastore_cache_file_metadata_args(const ThriftHiveMetastore_cache_file_metadata_args&); - ThriftHiveMetastore_cache_file_metadata_args& operator=(const ThriftHiveMetastore_cache_file_metadata_args&); - ThriftHiveMetastore_cache_file_metadata_args() { - } - - virtual ~ThriftHiveMetastore_cache_file_metadata_args() noexcept; - CacheFileMetadataRequest req; - - _ThriftHiveMetastore_cache_file_metadata_args__isset __isset; - - void __set_req(const CacheFileMetadataRequest& val); - - bool operator == (const ThriftHiveMetastore_cache_file_metadata_args & rhs) const - { - if (!(req == rhs.req)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_cache_file_metadata_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_cache_file_metadata_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_cache_file_metadata_pargs { - public: - - - virtual ~ThriftHiveMetastore_cache_file_metadata_pargs() noexcept; - const CacheFileMetadataRequest* req; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_cache_file_metadata_result__isset { - _ThriftHiveMetastore_cache_file_metadata_result__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_cache_file_metadata_result__isset; - -class ThriftHiveMetastore_cache_file_metadata_result { - public: - - ThriftHiveMetastore_cache_file_metadata_result(const ThriftHiveMetastore_cache_file_metadata_result&); - ThriftHiveMetastore_cache_file_metadata_result& operator=(const ThriftHiveMetastore_cache_file_metadata_result&); - ThriftHiveMetastore_cache_file_metadata_result() { - } - - virtual ~ThriftHiveMetastore_cache_file_metadata_result() noexcept; - CacheFileMetadataResult success; - - _ThriftHiveMetastore_cache_file_metadata_result__isset __isset; - - void __set_success(const CacheFileMetadataResult& val); - - bool operator == (const ThriftHiveMetastore_cache_file_metadata_result & rhs) const - { - if (!(success == rhs.success)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_cache_file_metadata_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_cache_file_metadata_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_cache_file_metadata_presult__isset { - _ThriftHiveMetastore_cache_file_metadata_presult__isset() : success(false) {} - bool success :1; -} _ThriftHiveMetastore_cache_file_metadata_presult__isset; - -class ThriftHiveMetastore_cache_file_metadata_presult { - public: - - - virtual ~ThriftHiveMetastore_cache_file_metadata_presult() noexcept; - CacheFileMetadataResult* success; - - _ThriftHiveMetastore_cache_file_metadata_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - - -class ThriftHiveMetastore_get_metastore_db_uuid_args { - public: - - ThriftHiveMetastore_get_metastore_db_uuid_args(const ThriftHiveMetastore_get_metastore_db_uuid_args&); - ThriftHiveMetastore_get_metastore_db_uuid_args& operator=(const ThriftHiveMetastore_get_metastore_db_uuid_args&); - ThriftHiveMetastore_get_metastore_db_uuid_args() { - } - - virtual ~ThriftHiveMetastore_get_metastore_db_uuid_args() noexcept; - - bool operator == (const ThriftHiveMetastore_get_metastore_db_uuid_args & /* rhs */) const - { - return true; - } - bool operator != (const ThriftHiveMetastore_get_metastore_db_uuid_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_metastore_db_uuid_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_metastore_db_uuid_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_metastore_db_uuid_pargs() noexcept; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_metastore_db_uuid_result__isset { - _ThriftHiveMetastore_get_metastore_db_uuid_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_metastore_db_uuid_result__isset; - -class ThriftHiveMetastore_get_metastore_db_uuid_result { - public: - - ThriftHiveMetastore_get_metastore_db_uuid_result(const ThriftHiveMetastore_get_metastore_db_uuid_result&); - ThriftHiveMetastore_get_metastore_db_uuid_result& operator=(const ThriftHiveMetastore_get_metastore_db_uuid_result&); - ThriftHiveMetastore_get_metastore_db_uuid_result() : success() { - } - - virtual ~ThriftHiveMetastore_get_metastore_db_uuid_result() noexcept; - std::string success; - MetaException o1; - - _ThriftHiveMetastore_get_metastore_db_uuid_result__isset __isset; - - void __set_success(const std::string& val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_metastore_db_uuid_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_metastore_db_uuid_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_metastore_db_uuid_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_metastore_db_uuid_presult__isset { - _ThriftHiveMetastore_get_metastore_db_uuid_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_metastore_db_uuid_presult__isset; - -class ThriftHiveMetastore_get_metastore_db_uuid_presult { - public: - - - virtual ~ThriftHiveMetastore_get_metastore_db_uuid_presult() noexcept; - std::string* success; - MetaException o1; - - _ThriftHiveMetastore_get_metastore_db_uuid_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_create_resource_plan_args__isset { - _ThriftHiveMetastore_create_resource_plan_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_create_resource_plan_args__isset; - -class ThriftHiveMetastore_create_resource_plan_args { - public: - - ThriftHiveMetastore_create_resource_plan_args(const ThriftHiveMetastore_create_resource_plan_args&); - ThriftHiveMetastore_create_resource_plan_args& operator=(const ThriftHiveMetastore_create_resource_plan_args&); - ThriftHiveMetastore_create_resource_plan_args() { - } - - virtual ~ThriftHiveMetastore_create_resource_plan_args() noexcept; - WMCreateResourcePlanRequest request; - - _ThriftHiveMetastore_create_resource_plan_args__isset __isset; - - void __set_request(const WMCreateResourcePlanRequest& val); - - bool operator == (const ThriftHiveMetastore_create_resource_plan_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_resource_plan_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_resource_plan_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_create_resource_plan_pargs { - public: - - - virtual ~ThriftHiveMetastore_create_resource_plan_pargs() noexcept; - const WMCreateResourcePlanRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_resource_plan_result__isset { - _ThriftHiveMetastore_create_resource_plan_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_create_resource_plan_result__isset; - -class ThriftHiveMetastore_create_resource_plan_result { - public: - - ThriftHiveMetastore_create_resource_plan_result(const ThriftHiveMetastore_create_resource_plan_result&); - ThriftHiveMetastore_create_resource_plan_result& operator=(const ThriftHiveMetastore_create_resource_plan_result&); - ThriftHiveMetastore_create_resource_plan_result() { - } - - virtual ~ThriftHiveMetastore_create_resource_plan_result() noexcept; - WMCreateResourcePlanResponse success; - AlreadyExistsException o1; - InvalidObjectException o2; - MetaException o3; - - _ThriftHiveMetastore_create_resource_plan_result__isset __isset; - - void __set_success(const WMCreateResourcePlanResponse& val); - - void __set_o1(const AlreadyExistsException& val); - - void __set_o2(const InvalidObjectException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_create_resource_plan_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_resource_plan_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_resource_plan_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_resource_plan_presult__isset { - _ThriftHiveMetastore_create_resource_plan_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_create_resource_plan_presult__isset; - -class ThriftHiveMetastore_create_resource_plan_presult { - public: - - - virtual ~ThriftHiveMetastore_create_resource_plan_presult() noexcept; - WMCreateResourcePlanResponse* success; - AlreadyExistsException o1; - InvalidObjectException o2; - MetaException o3; - - _ThriftHiveMetastore_create_resource_plan_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_resource_plan_args__isset { - _ThriftHiveMetastore_get_resource_plan_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_get_resource_plan_args__isset; - -class ThriftHiveMetastore_get_resource_plan_args { - public: - - ThriftHiveMetastore_get_resource_plan_args(const ThriftHiveMetastore_get_resource_plan_args&); - ThriftHiveMetastore_get_resource_plan_args& operator=(const ThriftHiveMetastore_get_resource_plan_args&); - ThriftHiveMetastore_get_resource_plan_args() { - } - - virtual ~ThriftHiveMetastore_get_resource_plan_args() noexcept; - WMGetResourcePlanRequest request; - - _ThriftHiveMetastore_get_resource_plan_args__isset __isset; - - void __set_request(const WMGetResourcePlanRequest& val); - - bool operator == (const ThriftHiveMetastore_get_resource_plan_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_resource_plan_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_resource_plan_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_resource_plan_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_resource_plan_pargs() noexcept; - const WMGetResourcePlanRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_resource_plan_result__isset { - _ThriftHiveMetastore_get_resource_plan_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_resource_plan_result__isset; - -class ThriftHiveMetastore_get_resource_plan_result { - public: - - ThriftHiveMetastore_get_resource_plan_result(const ThriftHiveMetastore_get_resource_plan_result&); - ThriftHiveMetastore_get_resource_plan_result& operator=(const ThriftHiveMetastore_get_resource_plan_result&); - ThriftHiveMetastore_get_resource_plan_result() { - } - - virtual ~ThriftHiveMetastore_get_resource_plan_result() noexcept; - WMGetResourcePlanResponse success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_resource_plan_result__isset __isset; - - void __set_success(const WMGetResourcePlanResponse& val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_resource_plan_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_resource_plan_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_resource_plan_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_resource_plan_presult__isset { - _ThriftHiveMetastore_get_resource_plan_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_resource_plan_presult__isset; - -class ThriftHiveMetastore_get_resource_plan_presult { - public: - - - virtual ~ThriftHiveMetastore_get_resource_plan_presult() noexcept; - WMGetResourcePlanResponse* success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_resource_plan_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_active_resource_plan_args__isset { - _ThriftHiveMetastore_get_active_resource_plan_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_get_active_resource_plan_args__isset; - -class ThriftHiveMetastore_get_active_resource_plan_args { - public: - - ThriftHiveMetastore_get_active_resource_plan_args(const ThriftHiveMetastore_get_active_resource_plan_args&); - ThriftHiveMetastore_get_active_resource_plan_args& operator=(const ThriftHiveMetastore_get_active_resource_plan_args&); - ThriftHiveMetastore_get_active_resource_plan_args() { - } - - virtual ~ThriftHiveMetastore_get_active_resource_plan_args() noexcept; - WMGetActiveResourcePlanRequest request; - - _ThriftHiveMetastore_get_active_resource_plan_args__isset __isset; - - void __set_request(const WMGetActiveResourcePlanRequest& val); - - bool operator == (const ThriftHiveMetastore_get_active_resource_plan_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_active_resource_plan_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_active_resource_plan_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_active_resource_plan_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_active_resource_plan_pargs() noexcept; - const WMGetActiveResourcePlanRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_active_resource_plan_result__isset { - _ThriftHiveMetastore_get_active_resource_plan_result__isset() : success(false), o2(false) {} - bool success :1; - bool o2 :1; -} _ThriftHiveMetastore_get_active_resource_plan_result__isset; - -class ThriftHiveMetastore_get_active_resource_plan_result { - public: - - ThriftHiveMetastore_get_active_resource_plan_result(const ThriftHiveMetastore_get_active_resource_plan_result&); - ThriftHiveMetastore_get_active_resource_plan_result& operator=(const ThriftHiveMetastore_get_active_resource_plan_result&); - ThriftHiveMetastore_get_active_resource_plan_result() { - } - - virtual ~ThriftHiveMetastore_get_active_resource_plan_result() noexcept; - WMGetActiveResourcePlanResponse success; - MetaException o2; - - _ThriftHiveMetastore_get_active_resource_plan_result__isset __isset; - - void __set_success(const WMGetActiveResourcePlanResponse& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_active_resource_plan_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_active_resource_plan_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_active_resource_plan_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_active_resource_plan_presult__isset { - _ThriftHiveMetastore_get_active_resource_plan_presult__isset() : success(false), o2(false) {} - bool success :1; - bool o2 :1; -} _ThriftHiveMetastore_get_active_resource_plan_presult__isset; - -class ThriftHiveMetastore_get_active_resource_plan_presult { - public: - - - virtual ~ThriftHiveMetastore_get_active_resource_plan_presult() noexcept; - WMGetActiveResourcePlanResponse* success; - MetaException o2; - - _ThriftHiveMetastore_get_active_resource_plan_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_all_resource_plans_args__isset { - _ThriftHiveMetastore_get_all_resource_plans_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_get_all_resource_plans_args__isset; - -class ThriftHiveMetastore_get_all_resource_plans_args { - public: - - ThriftHiveMetastore_get_all_resource_plans_args(const ThriftHiveMetastore_get_all_resource_plans_args&); - ThriftHiveMetastore_get_all_resource_plans_args& operator=(const ThriftHiveMetastore_get_all_resource_plans_args&); - ThriftHiveMetastore_get_all_resource_plans_args() { - } - - virtual ~ThriftHiveMetastore_get_all_resource_plans_args() noexcept; - WMGetAllResourcePlanRequest request; - - _ThriftHiveMetastore_get_all_resource_plans_args__isset __isset; - - void __set_request(const WMGetAllResourcePlanRequest& val); - - bool operator == (const ThriftHiveMetastore_get_all_resource_plans_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_all_resource_plans_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_all_resource_plans_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_all_resource_plans_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_all_resource_plans_pargs() noexcept; - const WMGetAllResourcePlanRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_all_resource_plans_result__isset { - _ThriftHiveMetastore_get_all_resource_plans_result__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_all_resource_plans_result__isset; - -class ThriftHiveMetastore_get_all_resource_plans_result { - public: - - ThriftHiveMetastore_get_all_resource_plans_result(const ThriftHiveMetastore_get_all_resource_plans_result&); - ThriftHiveMetastore_get_all_resource_plans_result& operator=(const ThriftHiveMetastore_get_all_resource_plans_result&); - ThriftHiveMetastore_get_all_resource_plans_result() { - } - - virtual ~ThriftHiveMetastore_get_all_resource_plans_result() noexcept; - WMGetAllResourcePlanResponse success; - MetaException o1; - - _ThriftHiveMetastore_get_all_resource_plans_result__isset __isset; - - void __set_success(const WMGetAllResourcePlanResponse& val); - - void __set_o1(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_all_resource_plans_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_all_resource_plans_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_all_resource_plans_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_all_resource_plans_presult__isset { - _ThriftHiveMetastore_get_all_resource_plans_presult__isset() : success(false), o1(false) {} - bool success :1; - bool o1 :1; -} _ThriftHiveMetastore_get_all_resource_plans_presult__isset; - -class ThriftHiveMetastore_get_all_resource_plans_presult { - public: - - - virtual ~ThriftHiveMetastore_get_all_resource_plans_presult() noexcept; - WMGetAllResourcePlanResponse* success; - MetaException o1; - - _ThriftHiveMetastore_get_all_resource_plans_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_alter_resource_plan_args__isset { - _ThriftHiveMetastore_alter_resource_plan_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_alter_resource_plan_args__isset; - -class ThriftHiveMetastore_alter_resource_plan_args { - public: - - ThriftHiveMetastore_alter_resource_plan_args(const ThriftHiveMetastore_alter_resource_plan_args&); - ThriftHiveMetastore_alter_resource_plan_args& operator=(const ThriftHiveMetastore_alter_resource_plan_args&); - ThriftHiveMetastore_alter_resource_plan_args() { - } - - virtual ~ThriftHiveMetastore_alter_resource_plan_args() noexcept; - WMAlterResourcePlanRequest request; - - _ThriftHiveMetastore_alter_resource_plan_args__isset __isset; - - void __set_request(const WMAlterResourcePlanRequest& val); - - bool operator == (const ThriftHiveMetastore_alter_resource_plan_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_resource_plan_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_resource_plan_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_alter_resource_plan_pargs { - public: - - - virtual ~ThriftHiveMetastore_alter_resource_plan_pargs() noexcept; - const WMAlterResourcePlanRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_resource_plan_result__isset { - _ThriftHiveMetastore_alter_resource_plan_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_alter_resource_plan_result__isset; - -class ThriftHiveMetastore_alter_resource_plan_result { - public: - - ThriftHiveMetastore_alter_resource_plan_result(const ThriftHiveMetastore_alter_resource_plan_result&); - ThriftHiveMetastore_alter_resource_plan_result& operator=(const ThriftHiveMetastore_alter_resource_plan_result&); - ThriftHiveMetastore_alter_resource_plan_result() { - } - - virtual ~ThriftHiveMetastore_alter_resource_plan_result() noexcept; - WMAlterResourcePlanResponse success; - NoSuchObjectException o1; - InvalidOperationException o2; - MetaException o3; - - _ThriftHiveMetastore_alter_resource_plan_result__isset __isset; - - void __set_success(const WMAlterResourcePlanResponse& val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const InvalidOperationException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_alter_resource_plan_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_resource_plan_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_resource_plan_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_resource_plan_presult__isset { - _ThriftHiveMetastore_alter_resource_plan_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_alter_resource_plan_presult__isset; - -class ThriftHiveMetastore_alter_resource_plan_presult { - public: - - - virtual ~ThriftHiveMetastore_alter_resource_plan_presult() noexcept; - WMAlterResourcePlanResponse* success; - NoSuchObjectException o1; - InvalidOperationException o2; - MetaException o3; - - _ThriftHiveMetastore_alter_resource_plan_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_validate_resource_plan_args__isset { - _ThriftHiveMetastore_validate_resource_plan_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_validate_resource_plan_args__isset; - -class ThriftHiveMetastore_validate_resource_plan_args { - public: - - ThriftHiveMetastore_validate_resource_plan_args(const ThriftHiveMetastore_validate_resource_plan_args&); - ThriftHiveMetastore_validate_resource_plan_args& operator=(const ThriftHiveMetastore_validate_resource_plan_args&); - ThriftHiveMetastore_validate_resource_plan_args() { - } - - virtual ~ThriftHiveMetastore_validate_resource_plan_args() noexcept; - WMValidateResourcePlanRequest request; - - _ThriftHiveMetastore_validate_resource_plan_args__isset __isset; - - void __set_request(const WMValidateResourcePlanRequest& val); - - bool operator == (const ThriftHiveMetastore_validate_resource_plan_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_validate_resource_plan_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_validate_resource_plan_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_validate_resource_plan_pargs { - public: - - - virtual ~ThriftHiveMetastore_validate_resource_plan_pargs() noexcept; - const WMValidateResourcePlanRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_validate_resource_plan_result__isset { - _ThriftHiveMetastore_validate_resource_plan_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_validate_resource_plan_result__isset; - -class ThriftHiveMetastore_validate_resource_plan_result { - public: - - ThriftHiveMetastore_validate_resource_plan_result(const ThriftHiveMetastore_validate_resource_plan_result&); - ThriftHiveMetastore_validate_resource_plan_result& operator=(const ThriftHiveMetastore_validate_resource_plan_result&); - ThriftHiveMetastore_validate_resource_plan_result() { - } - - virtual ~ThriftHiveMetastore_validate_resource_plan_result() noexcept; - WMValidateResourcePlanResponse success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_validate_resource_plan_result__isset __isset; - - void __set_success(const WMValidateResourcePlanResponse& val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_validate_resource_plan_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_validate_resource_plan_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_validate_resource_plan_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_validate_resource_plan_presult__isset { - _ThriftHiveMetastore_validate_resource_plan_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_validate_resource_plan_presult__isset; - -class ThriftHiveMetastore_validate_resource_plan_presult { - public: - - - virtual ~ThriftHiveMetastore_validate_resource_plan_presult() noexcept; - WMValidateResourcePlanResponse* success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_validate_resource_plan_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_drop_resource_plan_args__isset { - _ThriftHiveMetastore_drop_resource_plan_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_drop_resource_plan_args__isset; - -class ThriftHiveMetastore_drop_resource_plan_args { - public: - - ThriftHiveMetastore_drop_resource_plan_args(const ThriftHiveMetastore_drop_resource_plan_args&); - ThriftHiveMetastore_drop_resource_plan_args& operator=(const ThriftHiveMetastore_drop_resource_plan_args&); - ThriftHiveMetastore_drop_resource_plan_args() { - } - - virtual ~ThriftHiveMetastore_drop_resource_plan_args() noexcept; - WMDropResourcePlanRequest request; - - _ThriftHiveMetastore_drop_resource_plan_args__isset __isset; - - void __set_request(const WMDropResourcePlanRequest& val); - - bool operator == (const ThriftHiveMetastore_drop_resource_plan_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_resource_plan_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_resource_plan_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_drop_resource_plan_pargs { - public: - - - virtual ~ThriftHiveMetastore_drop_resource_plan_pargs() noexcept; - const WMDropResourcePlanRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_resource_plan_result__isset { - _ThriftHiveMetastore_drop_resource_plan_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_drop_resource_plan_result__isset; - -class ThriftHiveMetastore_drop_resource_plan_result { - public: - - ThriftHiveMetastore_drop_resource_plan_result(const ThriftHiveMetastore_drop_resource_plan_result&); - ThriftHiveMetastore_drop_resource_plan_result& operator=(const ThriftHiveMetastore_drop_resource_plan_result&); - ThriftHiveMetastore_drop_resource_plan_result() { - } - - virtual ~ThriftHiveMetastore_drop_resource_plan_result() noexcept; - WMDropResourcePlanResponse success; - NoSuchObjectException o1; - InvalidOperationException o2; - MetaException o3; - - _ThriftHiveMetastore_drop_resource_plan_result__isset __isset; - - void __set_success(const WMDropResourcePlanResponse& val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const InvalidOperationException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_drop_resource_plan_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_resource_plan_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_resource_plan_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_resource_plan_presult__isset { - _ThriftHiveMetastore_drop_resource_plan_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_drop_resource_plan_presult__isset; - -class ThriftHiveMetastore_drop_resource_plan_presult { - public: - - - virtual ~ThriftHiveMetastore_drop_resource_plan_presult() noexcept; - WMDropResourcePlanResponse* success; - NoSuchObjectException o1; - InvalidOperationException o2; - MetaException o3; - - _ThriftHiveMetastore_drop_resource_plan_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_create_wm_trigger_args__isset { - _ThriftHiveMetastore_create_wm_trigger_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_create_wm_trigger_args__isset; - -class ThriftHiveMetastore_create_wm_trigger_args { - public: - - ThriftHiveMetastore_create_wm_trigger_args(const ThriftHiveMetastore_create_wm_trigger_args&); - ThriftHiveMetastore_create_wm_trigger_args& operator=(const ThriftHiveMetastore_create_wm_trigger_args&); - ThriftHiveMetastore_create_wm_trigger_args() { - } - - virtual ~ThriftHiveMetastore_create_wm_trigger_args() noexcept; - WMCreateTriggerRequest request; - - _ThriftHiveMetastore_create_wm_trigger_args__isset __isset; - - void __set_request(const WMCreateTriggerRequest& val); - - bool operator == (const ThriftHiveMetastore_create_wm_trigger_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_wm_trigger_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_wm_trigger_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_create_wm_trigger_pargs { - public: - - - virtual ~ThriftHiveMetastore_create_wm_trigger_pargs() noexcept; - const WMCreateTriggerRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_wm_trigger_result__isset { - _ThriftHiveMetastore_create_wm_trigger_result__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_create_wm_trigger_result__isset; - -class ThriftHiveMetastore_create_wm_trigger_result { - public: - - ThriftHiveMetastore_create_wm_trigger_result(const ThriftHiveMetastore_create_wm_trigger_result&); - ThriftHiveMetastore_create_wm_trigger_result& operator=(const ThriftHiveMetastore_create_wm_trigger_result&); - ThriftHiveMetastore_create_wm_trigger_result() { - } - - virtual ~ThriftHiveMetastore_create_wm_trigger_result() noexcept; - WMCreateTriggerResponse success; - AlreadyExistsException o1; - NoSuchObjectException o2; - InvalidObjectException o3; - MetaException o4; - - _ThriftHiveMetastore_create_wm_trigger_result__isset __isset; - - void __set_success(const WMCreateTriggerResponse& val); - - void __set_o1(const AlreadyExistsException& val); - - void __set_o2(const NoSuchObjectException& val); - - void __set_o3(const InvalidObjectException& val); - - void __set_o4(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_create_wm_trigger_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - if (!(o4 == rhs.o4)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_create_wm_trigger_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_create_wm_trigger_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_create_wm_trigger_presult__isset { - _ThriftHiveMetastore_create_wm_trigger_presult__isset() : success(false), o1(false), o2(false), o3(false), o4(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; - bool o4 :1; -} _ThriftHiveMetastore_create_wm_trigger_presult__isset; - -class ThriftHiveMetastore_create_wm_trigger_presult { - public: - - - virtual ~ThriftHiveMetastore_create_wm_trigger_presult() noexcept; - WMCreateTriggerResponse* success; - AlreadyExistsException o1; - NoSuchObjectException o2; - InvalidObjectException o3; - MetaException o4; - - _ThriftHiveMetastore_create_wm_trigger_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_alter_wm_trigger_args__isset { - _ThriftHiveMetastore_alter_wm_trigger_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_alter_wm_trigger_args__isset; - -class ThriftHiveMetastore_alter_wm_trigger_args { - public: - - ThriftHiveMetastore_alter_wm_trigger_args(const ThriftHiveMetastore_alter_wm_trigger_args&); - ThriftHiveMetastore_alter_wm_trigger_args& operator=(const ThriftHiveMetastore_alter_wm_trigger_args&); - ThriftHiveMetastore_alter_wm_trigger_args() { - } - - virtual ~ThriftHiveMetastore_alter_wm_trigger_args() noexcept; - WMAlterTriggerRequest request; - - _ThriftHiveMetastore_alter_wm_trigger_args__isset __isset; - - void __set_request(const WMAlterTriggerRequest& val); - - bool operator == (const ThriftHiveMetastore_alter_wm_trigger_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_wm_trigger_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_wm_trigger_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_alter_wm_trigger_pargs { - public: - - - virtual ~ThriftHiveMetastore_alter_wm_trigger_pargs() noexcept; - const WMAlterTriggerRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_wm_trigger_result__isset { - _ThriftHiveMetastore_alter_wm_trigger_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_alter_wm_trigger_result__isset; - -class ThriftHiveMetastore_alter_wm_trigger_result { - public: - - ThriftHiveMetastore_alter_wm_trigger_result(const ThriftHiveMetastore_alter_wm_trigger_result&); - ThriftHiveMetastore_alter_wm_trigger_result& operator=(const ThriftHiveMetastore_alter_wm_trigger_result&); - ThriftHiveMetastore_alter_wm_trigger_result() { - } - - virtual ~ThriftHiveMetastore_alter_wm_trigger_result() noexcept; - WMAlterTriggerResponse success; - NoSuchObjectException o1; - InvalidObjectException o2; - MetaException o3; - - _ThriftHiveMetastore_alter_wm_trigger_result__isset __isset; - - void __set_success(const WMAlterTriggerResponse& val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const InvalidObjectException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_alter_wm_trigger_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_alter_wm_trigger_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_alter_wm_trigger_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_alter_wm_trigger_presult__isset { - _ThriftHiveMetastore_alter_wm_trigger_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_alter_wm_trigger_presult__isset; - -class ThriftHiveMetastore_alter_wm_trigger_presult { - public: - - - virtual ~ThriftHiveMetastore_alter_wm_trigger_presult() noexcept; - WMAlterTriggerResponse* success; - NoSuchObjectException o1; - InvalidObjectException o2; - MetaException o3; - - _ThriftHiveMetastore_alter_wm_trigger_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_drop_wm_trigger_args__isset { - _ThriftHiveMetastore_drop_wm_trigger_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_drop_wm_trigger_args__isset; - -class ThriftHiveMetastore_drop_wm_trigger_args { - public: - - ThriftHiveMetastore_drop_wm_trigger_args(const ThriftHiveMetastore_drop_wm_trigger_args&); - ThriftHiveMetastore_drop_wm_trigger_args& operator=(const ThriftHiveMetastore_drop_wm_trigger_args&); - ThriftHiveMetastore_drop_wm_trigger_args() { - } - - virtual ~ThriftHiveMetastore_drop_wm_trigger_args() noexcept; - WMDropTriggerRequest request; - - _ThriftHiveMetastore_drop_wm_trigger_args__isset __isset; - - void __set_request(const WMDropTriggerRequest& val); - - bool operator == (const ThriftHiveMetastore_drop_wm_trigger_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_wm_trigger_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_wm_trigger_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_drop_wm_trigger_pargs { - public: - - - virtual ~ThriftHiveMetastore_drop_wm_trigger_pargs() noexcept; - const WMDropTriggerRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_wm_trigger_result__isset { - _ThriftHiveMetastore_drop_wm_trigger_result__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_drop_wm_trigger_result__isset; - -class ThriftHiveMetastore_drop_wm_trigger_result { - public: - - ThriftHiveMetastore_drop_wm_trigger_result(const ThriftHiveMetastore_drop_wm_trigger_result&); - ThriftHiveMetastore_drop_wm_trigger_result& operator=(const ThriftHiveMetastore_drop_wm_trigger_result&); - ThriftHiveMetastore_drop_wm_trigger_result() { - } - - virtual ~ThriftHiveMetastore_drop_wm_trigger_result() noexcept; - WMDropTriggerResponse success; - NoSuchObjectException o1; - InvalidOperationException o2; - MetaException o3; - - _ThriftHiveMetastore_drop_wm_trigger_result__isset __isset; - - void __set_success(const WMDropTriggerResponse& val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const InvalidOperationException& val); - - void __set_o3(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_drop_wm_trigger_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - if (!(o3 == rhs.o3)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_drop_wm_trigger_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_drop_wm_trigger_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_drop_wm_trigger_presult__isset { - _ThriftHiveMetastore_drop_wm_trigger_presult__isset() : success(false), o1(false), o2(false), o3(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; - bool o3 :1; -} _ThriftHiveMetastore_drop_wm_trigger_presult__isset; - -class ThriftHiveMetastore_drop_wm_trigger_presult { - public: - - - virtual ~ThriftHiveMetastore_drop_wm_trigger_presult() noexcept; - WMDropTriggerResponse* success; - NoSuchObjectException o1; - InvalidOperationException o2; - MetaException o3; - - _ThriftHiveMetastore_drop_wm_trigger_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -typedef struct _ThriftHiveMetastore_get_triggers_for_resourceplan_args__isset { - _ThriftHiveMetastore_get_triggers_for_resourceplan_args__isset() : request(false) {} - bool request :1; -} _ThriftHiveMetastore_get_triggers_for_resourceplan_args__isset; - -class ThriftHiveMetastore_get_triggers_for_resourceplan_args { - public: - - ThriftHiveMetastore_get_triggers_for_resourceplan_args(const ThriftHiveMetastore_get_triggers_for_resourceplan_args&); - ThriftHiveMetastore_get_triggers_for_resourceplan_args& operator=(const ThriftHiveMetastore_get_triggers_for_resourceplan_args&); - ThriftHiveMetastore_get_triggers_for_resourceplan_args() { - } - - virtual ~ThriftHiveMetastore_get_triggers_for_resourceplan_args() noexcept; - WMGetTriggersForResourePlanRequest request; - - _ThriftHiveMetastore_get_triggers_for_resourceplan_args__isset __isset; - - void __set_request(const WMGetTriggersForResourePlanRequest& val); - - bool operator == (const ThriftHiveMetastore_get_triggers_for_resourceplan_args & rhs) const - { - if (!(request == rhs.request)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_triggers_for_resourceplan_args &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_triggers_for_resourceplan_args & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - - -class ThriftHiveMetastore_get_triggers_for_resourceplan_pargs { - public: - - - virtual ~ThriftHiveMetastore_get_triggers_for_resourceplan_pargs() noexcept; - const WMGetTriggersForResourePlanRequest* request; - - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_triggers_for_resourceplan_result__isset { - _ThriftHiveMetastore_get_triggers_for_resourceplan_result__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_triggers_for_resourceplan_result__isset; - -class ThriftHiveMetastore_get_triggers_for_resourceplan_result { - public: - - ThriftHiveMetastore_get_triggers_for_resourceplan_result(const ThriftHiveMetastore_get_triggers_for_resourceplan_result&); - ThriftHiveMetastore_get_triggers_for_resourceplan_result& operator=(const ThriftHiveMetastore_get_triggers_for_resourceplan_result&); - ThriftHiveMetastore_get_triggers_for_resourceplan_result() { - } - - virtual ~ThriftHiveMetastore_get_triggers_for_resourceplan_result() noexcept; - WMGetTriggersForResourePlanResponse success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_triggers_for_resourceplan_result__isset __isset; - - void __set_success(const WMGetTriggersForResourePlanResponse& val); - - void __set_o1(const NoSuchObjectException& val); - - void __set_o2(const MetaException& val); - - bool operator == (const ThriftHiveMetastore_get_triggers_for_resourceplan_result & rhs) const - { - if (!(success == rhs.success)) - return false; - if (!(o1 == rhs.o1)) - return false; - if (!(o2 == rhs.o2)) - return false; - return true; - } - bool operator != (const ThriftHiveMetastore_get_triggers_for_resourceplan_result &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ThriftHiveMetastore_get_triggers_for_resourceplan_result & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - -}; - -typedef struct _ThriftHiveMetastore_get_triggers_for_resourceplan_presult__isset { - _ThriftHiveMetastore_get_triggers_for_resourceplan_presult__isset() : success(false), o1(false), o2(false) {} - bool success :1; - bool o1 :1; - bool o2 :1; -} _ThriftHiveMetastore_get_triggers_for_resourceplan_presult__isset; - -class ThriftHiveMetastore_get_triggers_for_resourceplan_presult { - public: - - - virtual ~ThriftHiveMetastore_get_triggers_for_resourceplan_presult() noexcept; - WMGetTriggersForResourePlanResponse* success; - NoSuchObjectException o1; - MetaException o2; - - _ThriftHiveMetastore_get_triggers_for_resourceplan_presult__isset __isset; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - -}; - -class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf { - public: - ThriftHiveMetastoreClient(std::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) { - setProtocol(prot); - } - ThriftHiveMetastoreClient(std::shared_ptr< ::apache::thrift::protocol::TProtocol> iprot, std::shared_ptr< ::apache::thrift::protocol::TProtocol> oprot) { - setProtocol(iprot,oprot); - } - private: - void setProtocol(std::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) { - setProtocol(prot,prot); - } - void setProtocol(std::shared_ptr< ::apache::thrift::protocol::TProtocol> iprot, std::shared_ptr< ::apache::thrift::protocol::TProtocol> oprot) { - piprot_=iprot; - poprot_=oprot; - iprot_ = iprot.get(); - oprot_ = oprot.get(); - } - public: - std::shared_ptr< ::apache::thrift::protocol::TProtocol> getInputProtocol() { - return piprot_; - } - std::shared_ptr< ::apache::thrift::protocol::TProtocol> getOutputProtocol() { - return poprot_; - } - void getMetaConf(std::string& _return, const std::string& key); - void send_getMetaConf(const std::string& key); - void recv_getMetaConf(std::string& _return); - void setMetaConf(const std::string& key, const std::string& value); - void send_setMetaConf(const std::string& key, const std::string& value); - void recv_setMetaConf(); - void create_database(const Database& database); - void send_create_database(const Database& database); - void recv_create_database(); - void get_database(Database& _return, const std::string& name); - void send_get_database(const std::string& name); - void recv_get_database(Database& _return); - void drop_database(const std::string& name, const bool deleteData, const bool cascade); - void send_drop_database(const std::string& name, const bool deleteData, const bool cascade); - void recv_drop_database(); - void get_databases(std::vector & _return, const std::string& pattern); - void send_get_databases(const std::string& pattern); - void recv_get_databases(std::vector & _return); - void get_all_databases(std::vector & _return); - void send_get_all_databases(); - void recv_get_all_databases(std::vector & _return); - void alter_database(const std::string& dbname, const Database& db); - void send_alter_database(const std::string& dbname, const Database& db); - void recv_alter_database(); - void get_type(Type& _return, const std::string& name); - void send_get_type(const std::string& name); - void recv_get_type(Type& _return); - bool create_type(const Type& type); - void send_create_type(const Type& type); - bool recv_create_type(); - bool drop_type(const std::string& type); - void send_drop_type(const std::string& type); - bool recv_drop_type(); - void get_type_all(std::map & _return, const std::string& name); - void send_get_type_all(const std::string& name); - void recv_get_type_all(std::map & _return); - void get_fields(std::vector & _return, const std::string& db_name, const std::string& table_name); - void send_get_fields(const std::string& db_name, const std::string& table_name); - void recv_get_fields(std::vector & _return); - void get_fields_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context); - void send_get_fields_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context); - void recv_get_fields_with_environment_context(std::vector & _return); - void get_schema(std::vector & _return, const std::string& db_name, const std::string& table_name); - void send_get_schema(const std::string& db_name, const std::string& table_name); - void recv_get_schema(std::vector & _return); - void get_schema_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context); - void send_get_schema_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context); - void recv_get_schema_with_environment_context(std::vector & _return); - void create_table(const Table& tbl); - void send_create_table(const Table& tbl); - void recv_create_table(); - void create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context); - void send_create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context); - void recv_create_table_with_environment_context(); - void create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints); - void send_create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints); - void recv_create_table_with_constraints(); - void drop_constraint(const DropConstraintRequest& req); - void send_drop_constraint(const DropConstraintRequest& req); - void recv_drop_constraint(); - void add_primary_key(const AddPrimaryKeyRequest& req); - void send_add_primary_key(const AddPrimaryKeyRequest& req); - void recv_add_primary_key(); - void add_foreign_key(const AddForeignKeyRequest& req); - void send_add_foreign_key(const AddForeignKeyRequest& req); - void recv_add_foreign_key(); - void add_unique_constraint(const AddUniqueConstraintRequest& req); - void send_add_unique_constraint(const AddUniqueConstraintRequest& req); - void recv_add_unique_constraint(); - void add_not_null_constraint(const AddNotNullConstraintRequest& req); - void send_add_not_null_constraint(const AddNotNullConstraintRequest& req); - void recv_add_not_null_constraint(); - void drop_table(const std::string& dbname, const std::string& name, const bool deleteData); - void send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData); - void recv_drop_table(); - void drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context); - void send_drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context); - void recv_drop_table_with_environment_context(); - void truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames); - void send_truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames); - void recv_truncate_table(); - void get_tables(std::vector & _return, const std::string& db_name, const std::string& pattern); - void send_get_tables(const std::string& db_name, const std::string& pattern); - void recv_get_tables(std::vector & _return); - void get_tables_by_type(std::vector & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType); - void send_get_tables_by_type(const std::string& db_name, const std::string& pattern, const std::string& tableType); - void recv_get_tables_by_type(std::vector & _return); - void get_table_meta(std::vector & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types); - void send_get_table_meta(const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types); - void recv_get_table_meta(std::vector & _return); - void get_all_tables(std::vector & _return, const std::string& db_name); - void send_get_all_tables(const std::string& db_name); - void recv_get_all_tables(std::vector & _return); - void get_table(Table& _return, const std::string& dbname, const std::string& tbl_name); - void send_get_table(const std::string& dbname, const std::string& tbl_name); - void recv_get_table(Table& _return); - void get_table_objects_by_name(std::vector
& _return, const std::string& dbname, const std::vector & tbl_names); - void send_get_table_objects_by_name(const std::string& dbname, const std::vector & tbl_names); - void recv_get_table_objects_by_name(std::vector
& _return); - void get_table_req(GetTableResult& _return, const GetTableRequest& req); - void send_get_table_req(const GetTableRequest& req); - void recv_get_table_req(GetTableResult& _return); - void get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req); - void send_get_table_objects_by_name_req(const GetTablesRequest& req); - void recv_get_table_objects_by_name_req(GetTablesResult& _return); - void get_table_names_by_filter(std::vector & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables); - void send_get_table_names_by_filter(const std::string& dbname, const std::string& filter, const int16_t max_tables); - void recv_get_table_names_by_filter(std::vector & _return); - void alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl); - void send_alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl); - void recv_alter_table(); - void alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context); - void send_alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context); - void recv_alter_table_with_environment_context(); - void alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade); - void send_alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade); - void recv_alter_table_with_cascade(); - void add_partition(Partition& _return, const Partition& new_part); - void send_add_partition(const Partition& new_part); - void recv_add_partition(Partition& _return); - void add_partition_with_environment_context(Partition& _return, const Partition& new_part, const EnvironmentContext& environment_context); - void send_add_partition_with_environment_context(const Partition& new_part, const EnvironmentContext& environment_context); - void recv_add_partition_with_environment_context(Partition& _return); - int32_t add_partitions(const std::vector & new_parts); - void send_add_partitions(const std::vector & new_parts); - int32_t recv_add_partitions(); - int32_t add_partitions_pspec(const std::vector & new_parts); - void send_add_partitions_pspec(const std::vector & new_parts); - int32_t recv_add_partitions_pspec(); - void append_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals); - void send_append_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals); - void recv_append_partition(Partition& _return); - void add_partitions_req(AddPartitionsResult& _return, const AddPartitionsRequest& request); - void send_add_partitions_req(const AddPartitionsRequest& request); - void recv_add_partitions_req(AddPartitionsResult& _return); - void append_partition_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context); - void send_append_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context); - void recv_append_partition_with_environment_context(Partition& _return); - void append_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name); - void send_append_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name); - void recv_append_partition_by_name(Partition& _return); - void append_partition_by_name_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context); - void send_append_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context); - void recv_append_partition_by_name_with_environment_context(Partition& _return); - bool drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData); - void send_drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData); - bool recv_drop_partition(); - bool drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context); - void send_drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context); - bool recv_drop_partition_with_environment_context(); - bool drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData); - void send_drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData); - bool recv_drop_partition_by_name(); - bool drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context); - void send_drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context); - bool recv_drop_partition_by_name_with_environment_context(); - void drop_partitions_req(DropPartitionsResult& _return, const DropPartitionsRequest& req); - void send_drop_partitions_req(const DropPartitionsRequest& req); - void recv_drop_partitions_req(DropPartitionsResult& _return); - void get_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals); - void send_get_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals); - void recv_get_partition(Partition& _return); - void exchange_partition(Partition& _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name); - void send_exchange_partition(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name); - void recv_exchange_partition(Partition& _return); - void exchange_partitions(std::vector & _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name); - void send_exchange_partitions(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name); - void recv_exchange_partitions(std::vector & _return); - void get_partition_with_auth(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names); - void send_get_partition_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names); - void recv_get_partition_with_auth(Partition& _return); - void get_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name); - void send_get_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name); - void recv_get_partition_by_name(Partition& _return); - void get_partitions(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts); - void send_get_partitions(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts); - void recv_get_partitions(std::vector & _return); - void get_partitions_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names); - void send_get_partitions_with_auth(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names); - void recv_get_partitions_with_auth(std::vector & _return); - void get_partitions_pspec(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int32_t max_parts); - void send_get_partitions_pspec(const std::string& db_name, const std::string& tbl_name, const int32_t max_parts); - void recv_get_partitions_pspec(std::vector & _return); - void get_partition_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts); - void send_get_partition_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts); - void recv_get_partition_names(std::vector & _return); - void get_partition_values(PartitionValuesResponse& _return, const PartitionValuesRequest& request); - void send_get_partition_values(const PartitionValuesRequest& request); - void recv_get_partition_values(PartitionValuesResponse& _return); - void get_partitions_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts); - void send_get_partitions_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts); - void recv_get_partitions_ps(std::vector & _return); - void get_partitions_ps_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names); - void send_get_partitions_ps_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names); - void recv_get_partitions_ps_with_auth(std::vector & _return); - void get_partition_names_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts); - void send_get_partition_names_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts); - void recv_get_partition_names_ps(std::vector & _return); - void get_partitions_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts); - void send_get_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts); - void recv_get_partitions_by_filter(std::vector & _return); - void get_part_specs_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts); - void send_get_part_specs_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts); - void recv_get_part_specs_by_filter(std::vector & _return); - void get_partitions_by_expr(PartitionsByExprResult& _return, const PartitionsByExprRequest& req); - void send_get_partitions_by_expr(const PartitionsByExprRequest& req); - void recv_get_partitions_by_expr(PartitionsByExprResult& _return); - int32_t get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter); - void send_get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter); - int32_t recv_get_num_partitions_by_filter(); - void get_partitions_by_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & names); - void send_get_partitions_by_names(const std::string& db_name, const std::string& tbl_name, const std::vector & names); - void recv_get_partitions_by_names(std::vector & _return); - void alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part); - void send_alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part); - void recv_alter_partition(); - void alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts); - void send_alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts); - void recv_alter_partitions(); - void alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context); - void send_alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context); - void recv_alter_partitions_with_environment_context(); - void alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context); - void send_alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context); - void recv_alter_partition_with_environment_context(); - void rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part); - void send_rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part); - void recv_rename_partition(); - bool partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception); - void send_partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception); - bool recv_partition_name_has_valid_characters(); - void get_config_value(std::string& _return, const std::string& name, const std::string& defaultValue); - void send_get_config_value(const std::string& name, const std::string& defaultValue); - void recv_get_config_value(std::string& _return); - void partition_name_to_vals(std::vector & _return, const std::string& part_name); - void send_partition_name_to_vals(const std::string& part_name); - void recv_partition_name_to_vals(std::vector & _return); - void partition_name_to_spec(std::map & _return, const std::string& part_name); - void send_partition_name_to_spec(const std::string& part_name); - void recv_partition_name_to_spec(std::map & _return); - void markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType); - void send_markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType); - void recv_markPartitionForEvent(); - bool isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType); - void send_isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType); - bool recv_isPartitionMarkedForEvent(); - void add_index(Index& _return, const Index& new_index, const Table& index_table); - void send_add_index(const Index& new_index, const Table& index_table); - void recv_add_index(Index& _return); - void alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx); - void send_alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx); - void recv_alter_index(); - bool drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData); - void send_drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData); - bool recv_drop_index_by_name(); - void get_index_by_name(Index& _return, const std::string& db_name, const std::string& tbl_name, const std::string& index_name); - void send_get_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name); - void recv_get_index_by_name(Index& _return); - void get_indexes(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes); - void send_get_indexes(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes); - void recv_get_indexes(std::vector & _return); - void get_index_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes); - void send_get_index_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes); - void recv_get_index_names(std::vector & _return); - void get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request); - void send_get_primary_keys(const PrimaryKeysRequest& request); - void recv_get_primary_keys(PrimaryKeysResponse& _return); - void get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request); - void send_get_foreign_keys(const ForeignKeysRequest& request); - void recv_get_foreign_keys(ForeignKeysResponse& _return); - void get_unique_constraints(UniqueConstraintsResponse& _return, const UniqueConstraintsRequest& request); - void send_get_unique_constraints(const UniqueConstraintsRequest& request); - void recv_get_unique_constraints(UniqueConstraintsResponse& _return); - void get_not_null_constraints(NotNullConstraintsResponse& _return, const NotNullConstraintsRequest& request); - void send_get_not_null_constraints(const NotNullConstraintsRequest& request); - void recv_get_not_null_constraints(NotNullConstraintsResponse& _return); - bool update_table_column_statistics(const ColumnStatistics& stats_obj); - void send_update_table_column_statistics(const ColumnStatistics& stats_obj); - bool recv_update_table_column_statistics(); - bool update_partition_column_statistics(const ColumnStatistics& stats_obj); - void send_update_partition_column_statistics(const ColumnStatistics& stats_obj); - bool recv_update_partition_column_statistics(); - void get_table_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& col_name); - void send_get_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name); - void recv_get_table_column_statistics(ColumnStatistics& _return); - void get_partition_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name); - void send_get_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name); - void recv_get_partition_column_statistics(ColumnStatistics& _return); - void get_table_statistics_req(TableStatsResult& _return, const TableStatsRequest& request); - void send_get_table_statistics_req(const TableStatsRequest& request); - void recv_get_table_statistics_req(TableStatsResult& _return); - void get_partitions_statistics_req(PartitionsStatsResult& _return, const PartitionsStatsRequest& request); - void send_get_partitions_statistics_req(const PartitionsStatsRequest& request); - void recv_get_partitions_statistics_req(PartitionsStatsResult& _return); - void get_aggr_stats_for(AggrStats& _return, const PartitionsStatsRequest& request); - void send_get_aggr_stats_for(const PartitionsStatsRequest& request); - void recv_get_aggr_stats_for(AggrStats& _return); - bool set_aggr_stats_for(const SetPartitionsStatsRequest& request); - void send_set_aggr_stats_for(const SetPartitionsStatsRequest& request); - bool recv_set_aggr_stats_for(); - bool delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name); - void send_delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name); - bool recv_delete_partition_column_statistics(); - bool delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name); - void send_delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name); - bool recv_delete_table_column_statistics(); - void create_function(const Function& func); - void send_create_function(const Function& func); - void recv_create_function(); - void drop_function(const std::string& dbName, const std::string& funcName); - void send_drop_function(const std::string& dbName, const std::string& funcName); - void recv_drop_function(); - void alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc); - void send_alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc); - void recv_alter_function(); - void get_functions(std::vector & _return, const std::string& dbName, const std::string& pattern); - void send_get_functions(const std::string& dbName, const std::string& pattern); - void recv_get_functions(std::vector & _return); - void get_function(Function& _return, const std::string& dbName, const std::string& funcName); - void send_get_function(const std::string& dbName, const std::string& funcName); - void recv_get_function(Function& _return); - void get_all_functions(GetAllFunctionsResponse& _return); - void send_get_all_functions(); - void recv_get_all_functions(GetAllFunctionsResponse& _return); - bool create_role(const Role& role); - void send_create_role(const Role& role); - bool recv_create_role(); - bool drop_role(const std::string& role_name); - void send_drop_role(const std::string& role_name); - bool recv_drop_role(); - void get_role_names(std::vector & _return); - void send_get_role_names(); - void recv_get_role_names(std::vector & _return); - bool grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option); - void send_grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option); - bool recv_grant_role(); - bool revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type); - void send_revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type); - bool recv_revoke_role(); - void list_roles(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type); - void send_list_roles(const std::string& principal_name, const PrincipalType::type principal_type); - void recv_list_roles(std::vector & _return); - void grant_revoke_role(GrantRevokeRoleResponse& _return, const GrantRevokeRoleRequest& request); - void send_grant_revoke_role(const GrantRevokeRoleRequest& request); - void recv_grant_revoke_role(GrantRevokeRoleResponse& _return); - void get_principals_in_role(GetPrincipalsInRoleResponse& _return, const GetPrincipalsInRoleRequest& request); - void send_get_principals_in_role(const GetPrincipalsInRoleRequest& request); - void recv_get_principals_in_role(GetPrincipalsInRoleResponse& _return); - void get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return, const GetRoleGrantsForPrincipalRequest& request); - void send_get_role_grants_for_principal(const GetRoleGrantsForPrincipalRequest& request); - void recv_get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return); - void get_privilege_set(PrincipalPrivilegeSet& _return, const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names); - void send_get_privilege_set(const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names); - void recv_get_privilege_set(PrincipalPrivilegeSet& _return); - void list_privileges(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject); - void send_list_privileges(const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject); - void recv_list_privileges(std::vector & _return); - bool grant_privileges(const PrivilegeBag& privileges); - void send_grant_privileges(const PrivilegeBag& privileges); - bool recv_grant_privileges(); - bool revoke_privileges(const PrivilegeBag& privileges); - void send_revoke_privileges(const PrivilegeBag& privileges); - bool recv_revoke_privileges(); - void grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const GrantRevokePrivilegeRequest& request); - void send_grant_revoke_privileges(const GrantRevokePrivilegeRequest& request); - void recv_grant_revoke_privileges(GrantRevokePrivilegeResponse& _return); - void set_ugi(std::vector & _return, const std::string& user_name, const std::vector & group_names); - void send_set_ugi(const std::string& user_name, const std::vector & group_names); - void recv_set_ugi(std::vector & _return); - void get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name); - void send_get_delegation_token(const std::string& token_owner, const std::string& renewer_kerberos_principal_name); - void recv_get_delegation_token(std::string& _return); - int64_t renew_delegation_token(const std::string& token_str_form); - void send_renew_delegation_token(const std::string& token_str_form); - int64_t recv_renew_delegation_token(); - void cancel_delegation_token(const std::string& token_str_form); - void send_cancel_delegation_token(const std::string& token_str_form); - void recv_cancel_delegation_token(); - bool add_token(const std::string& token_identifier, const std::string& delegation_token); - void send_add_token(const std::string& token_identifier, const std::string& delegation_token); - bool recv_add_token(); - bool remove_token(const std::string& token_identifier); - void send_remove_token(const std::string& token_identifier); - bool recv_remove_token(); - void get_token(std::string& _return, const std::string& token_identifier); - void send_get_token(const std::string& token_identifier); - void recv_get_token(std::string& _return); - void get_all_token_identifiers(std::vector & _return); - void send_get_all_token_identifiers(); - void recv_get_all_token_identifiers(std::vector & _return); - int32_t add_master_key(const std::string& key); - void send_add_master_key(const std::string& key); - int32_t recv_add_master_key(); - void update_master_key(const int32_t seq_number, const std::string& key); - void send_update_master_key(const int32_t seq_number, const std::string& key); - void recv_update_master_key(); - bool remove_master_key(const int32_t key_seq); - void send_remove_master_key(const int32_t key_seq); - bool recv_remove_master_key(); - void get_master_keys(std::vector & _return); - void send_get_master_keys(); - void recv_get_master_keys(std::vector & _return); - void get_open_txns(GetOpenTxnsResponse& _return); - void send_get_open_txns(); - void recv_get_open_txns(GetOpenTxnsResponse& _return); - void get_open_txns_info(GetOpenTxnsInfoResponse& _return); - void send_get_open_txns_info(); - void recv_get_open_txns_info(GetOpenTxnsInfoResponse& _return); - void open_txns(OpenTxnsResponse& _return, const OpenTxnRequest& rqst); - void send_open_txns(const OpenTxnRequest& rqst); - void recv_open_txns(OpenTxnsResponse& _return); - void abort_txn(const AbortTxnRequest& rqst); - void send_abort_txn(const AbortTxnRequest& rqst); - void recv_abort_txn(); - void abort_txns(const AbortTxnsRequest& rqst); - void send_abort_txns(const AbortTxnsRequest& rqst); - void recv_abort_txns(); - void commit_txn(const CommitTxnRequest& rqst); - void send_commit_txn(const CommitTxnRequest& rqst); - void recv_commit_txn(); - void lock(LockResponse& _return, const LockRequest& rqst); - void send_lock(const LockRequest& rqst); - void recv_lock(LockResponse& _return); - void check_lock(LockResponse& _return, const CheckLockRequest& rqst); - void send_check_lock(const CheckLockRequest& rqst); - void recv_check_lock(LockResponse& _return); - void unlock(const UnlockRequest& rqst); - void send_unlock(const UnlockRequest& rqst); - void recv_unlock(); - void show_locks(ShowLocksResponse& _return, const ShowLocksRequest& rqst); - void send_show_locks(const ShowLocksRequest& rqst); - void recv_show_locks(ShowLocksResponse& _return); - void heartbeat(const HeartbeatRequest& ids); - void send_heartbeat(const HeartbeatRequest& ids); - void recv_heartbeat(); - void heartbeat_txn_range(HeartbeatTxnRangeResponse& _return, const HeartbeatTxnRangeRequest& txns); - void send_heartbeat_txn_range(const HeartbeatTxnRangeRequest& txns); - void recv_heartbeat_txn_range(HeartbeatTxnRangeResponse& _return); - void compact(const CompactionRequest& rqst); - void send_compact(const CompactionRequest& rqst); - void recv_compact(); - void compact2(CompactionResponse& _return, const CompactionRequest& rqst); - void send_compact2(const CompactionRequest& rqst); - void recv_compact2(CompactionResponse& _return); - void show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst); - void send_show_compact(const ShowCompactRequest& rqst); - void recv_show_compact(ShowCompactResponse& _return); - void add_dynamic_partitions(const AddDynamicPartitions& rqst); - void send_add_dynamic_partitions(const AddDynamicPartitions& rqst); - void recv_add_dynamic_partitions(); - void get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst); - void send_get_next_notification(const NotificationEventRequest& rqst); - void recv_get_next_notification(NotificationEventResponse& _return); - void get_current_notificationEventId(CurrentNotificationEventId& _return); - void send_get_current_notificationEventId(); - void recv_get_current_notificationEventId(CurrentNotificationEventId& _return); - void get_notification_events_count(NotificationEventsCountResponse& _return, const NotificationEventsCountRequest& rqst); - void send_get_notification_events_count(const NotificationEventsCountRequest& rqst); - void recv_get_notification_events_count(NotificationEventsCountResponse& _return); - void fire_listener_event(FireEventResponse& _return, const FireEventRequest& rqst); - void send_fire_listener_event(const FireEventRequest& rqst); - void recv_fire_listener_event(FireEventResponse& _return); - void flushCache(); - void send_flushCache(); - void recv_flushCache(); - void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request); - void send_cm_recycle(const CmRecycleRequest& request); - void recv_cm_recycle(CmRecycleResponse& _return); - void get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req); - void send_get_file_metadata_by_expr(const GetFileMetadataByExprRequest& req); - void recv_get_file_metadata_by_expr(GetFileMetadataByExprResult& _return); - void get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req); - void send_get_file_metadata(const GetFileMetadataRequest& req); - void recv_get_file_metadata(GetFileMetadataResult& _return); - void put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req); - void send_put_file_metadata(const PutFileMetadataRequest& req); - void recv_put_file_metadata(PutFileMetadataResult& _return); - void clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req); - void send_clear_file_metadata(const ClearFileMetadataRequest& req); - void recv_clear_file_metadata(ClearFileMetadataResult& _return); - void cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req); - void send_cache_file_metadata(const CacheFileMetadataRequest& req); - void recv_cache_file_metadata(CacheFileMetadataResult& _return); - void get_metastore_db_uuid(std::string& _return); - void send_get_metastore_db_uuid(); - void recv_get_metastore_db_uuid(std::string& _return); - void create_resource_plan(WMCreateResourcePlanResponse& _return, const WMCreateResourcePlanRequest& request); - void send_create_resource_plan(const WMCreateResourcePlanRequest& request); - void recv_create_resource_plan(WMCreateResourcePlanResponse& _return); - void get_resource_plan(WMGetResourcePlanResponse& _return, const WMGetResourcePlanRequest& request); - void send_get_resource_plan(const WMGetResourcePlanRequest& request); - void recv_get_resource_plan(WMGetResourcePlanResponse& _return); - void get_active_resource_plan(WMGetActiveResourcePlanResponse& _return, const WMGetActiveResourcePlanRequest& request); - void send_get_active_resource_plan(const WMGetActiveResourcePlanRequest& request); - void recv_get_active_resource_plan(WMGetActiveResourcePlanResponse& _return); - void get_all_resource_plans(WMGetAllResourcePlanResponse& _return, const WMGetAllResourcePlanRequest& request); - void send_get_all_resource_plans(const WMGetAllResourcePlanRequest& request); - void recv_get_all_resource_plans(WMGetAllResourcePlanResponse& _return); - void alter_resource_plan(WMAlterResourcePlanResponse& _return, const WMAlterResourcePlanRequest& request); - void send_alter_resource_plan(const WMAlterResourcePlanRequest& request); - void recv_alter_resource_plan(WMAlterResourcePlanResponse& _return); - void validate_resource_plan(WMValidateResourcePlanResponse& _return, const WMValidateResourcePlanRequest& request); - void send_validate_resource_plan(const WMValidateResourcePlanRequest& request); - void recv_validate_resource_plan(WMValidateResourcePlanResponse& _return); - void drop_resource_plan(WMDropResourcePlanResponse& _return, const WMDropResourcePlanRequest& request); - void send_drop_resource_plan(const WMDropResourcePlanRequest& request); - void recv_drop_resource_plan(WMDropResourcePlanResponse& _return); - void create_wm_trigger(WMCreateTriggerResponse& _return, const WMCreateTriggerRequest& request); - void send_create_wm_trigger(const WMCreateTriggerRequest& request); - void recv_create_wm_trigger(WMCreateTriggerResponse& _return); - void alter_wm_trigger(WMAlterTriggerResponse& _return, const WMAlterTriggerRequest& request); - void send_alter_wm_trigger(const WMAlterTriggerRequest& request); - void recv_alter_wm_trigger(WMAlterTriggerResponse& _return); - void drop_wm_trigger(WMDropTriggerResponse& _return, const WMDropTriggerRequest& request); - void send_drop_wm_trigger(const WMDropTriggerRequest& request); - void recv_drop_wm_trigger(WMDropTriggerResponse& _return); - void get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return, const WMGetTriggersForResourePlanRequest& request); - void send_get_triggers_for_resourceplan(const WMGetTriggersForResourePlanRequest& request); - void recv_get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return); - protected: - std::shared_ptr< ::apache::thrift::protocol::TProtocol> piprot_; - std::shared_ptr< ::apache::thrift::protocol::TProtocol> poprot_; - ::apache::thrift::protocol::TProtocol* iprot_; - ::apache::thrift::protocol::TProtocol* oprot_; -}; - -class ThriftHiveMetastoreProcessor : public ::apache::thrift::TDispatchProcessor { - protected: - ::std::shared_ptr iface_; - virtual bool dispatchCall(::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, const std::string& fname, int32_t seqid, void* callContext); - private: - typedef void (ThriftHiveMetastoreProcessor::*ProcessFunction)(int32_t, ::apache::thrift::protocol::TProtocol*, ::apache::thrift::protocol::TProtocol*, void*); - typedef std::map ProcessMap; - ProcessMap processMap_; - void process_getMetaConf(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_setMetaConf(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_create_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_drop_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_databases(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_all_databases(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_alter_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_create_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_drop_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_type_all(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_fields(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_fields_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_schema(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_schema_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_create_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_create_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_create_table_with_constraints(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_drop_constraint(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_add_primary_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_add_foreign_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_add_unique_constraint(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_add_not_null_constraint(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_drop_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_drop_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_truncate_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_tables_by_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_table_meta(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_all_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_table_objects_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_table_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_table_objects_by_name_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_table_names_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_alter_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_alter_table_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_alter_table_with_cascade(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_add_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_add_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_add_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_add_partitions_pspec(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_append_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_add_partitions_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_append_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_append_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_append_partition_by_name_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_drop_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_drop_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_drop_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_drop_partition_by_name_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_drop_partitions_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_exchange_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_exchange_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_partition_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_partitions_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_partitions_pspec(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_partition_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_partition_values(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_partitions_ps(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_partitions_ps_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_partition_names_ps(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_partitions_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_part_specs_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_partitions_by_expr(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_num_partitions_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_partitions_by_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_alter_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_alter_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_alter_partitions_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_alter_partition_with_environment_context(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_rename_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_partition_name_has_valid_characters(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_config_value(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_partition_name_to_vals(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_partition_name_to_spec(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_markPartitionForEvent(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_isPartitionMarkedForEvent(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_add_index(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_alter_index(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_drop_index_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_index_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_indexes(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_index_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_primary_keys(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_foreign_keys(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_unique_constraints(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_not_null_constraints(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_update_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_update_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_table_statistics_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_partitions_statistics_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_aggr_stats_for(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_set_aggr_stats_for(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_delete_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_delete_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_create_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_drop_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_alter_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_functions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_all_functions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_create_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_drop_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_role_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_grant_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_revoke_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_list_roles(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_grant_revoke_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_principals_in_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_role_grants_for_principal(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_privilege_set(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_list_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_grant_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_revoke_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_grant_revoke_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_set_ugi(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_renew_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_cancel_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_add_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_remove_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_all_token_identifiers(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_add_master_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_update_master_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_remove_master_key(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_master_keys(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_open_txns(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_open_txns_info(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_open_txns(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_abort_txn(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_abort_txns(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_commit_txn(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_lock(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_check_lock(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_unlock(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_show_locks(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_heartbeat(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_heartbeat_txn_range(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_compact(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_compact2(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_show_compact(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_add_dynamic_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_next_notification(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_current_notificationEventId(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_notification_events_count(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_fire_listener_event(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_flushCache(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_cm_recycle(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_file_metadata_by_expr(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_put_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_clear_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_cache_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_metastore_db_uuid(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_create_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_active_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_all_resource_plans(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_alter_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_validate_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_drop_resource_plan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_create_wm_trigger(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_alter_wm_trigger(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_drop_wm_trigger(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - void process_get_triggers_for_resourceplan(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); - public: - ThriftHiveMetastoreProcessor(::std::shared_ptr iface) : - iface_(iface) { - processMap_["getMetaConf"] = &ThriftHiveMetastoreProcessor::process_getMetaConf; - processMap_["setMetaConf"] = &ThriftHiveMetastoreProcessor::process_setMetaConf; - processMap_["create_database"] = &ThriftHiveMetastoreProcessor::process_create_database; - processMap_["get_database"] = &ThriftHiveMetastoreProcessor::process_get_database; - processMap_["drop_database"] = &ThriftHiveMetastoreProcessor::process_drop_database; - processMap_["get_databases"] = &ThriftHiveMetastoreProcessor::process_get_databases; - processMap_["get_all_databases"] = &ThriftHiveMetastoreProcessor::process_get_all_databases; - processMap_["alter_database"] = &ThriftHiveMetastoreProcessor::process_alter_database; - processMap_["get_type"] = &ThriftHiveMetastoreProcessor::process_get_type; - processMap_["create_type"] = &ThriftHiveMetastoreProcessor::process_create_type; - processMap_["drop_type"] = &ThriftHiveMetastoreProcessor::process_drop_type; - processMap_["get_type_all"] = &ThriftHiveMetastoreProcessor::process_get_type_all; - processMap_["get_fields"] = &ThriftHiveMetastoreProcessor::process_get_fields; - processMap_["get_fields_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_get_fields_with_environment_context; - processMap_["get_schema"] = &ThriftHiveMetastoreProcessor::process_get_schema; - processMap_["get_schema_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_get_schema_with_environment_context; - processMap_["create_table"] = &ThriftHiveMetastoreProcessor::process_create_table; - processMap_["create_table_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_create_table_with_environment_context; - processMap_["create_table_with_constraints"] = &ThriftHiveMetastoreProcessor::process_create_table_with_constraints; - processMap_["drop_constraint"] = &ThriftHiveMetastoreProcessor::process_drop_constraint; - processMap_["add_primary_key"] = &ThriftHiveMetastoreProcessor::process_add_primary_key; - processMap_["add_foreign_key"] = &ThriftHiveMetastoreProcessor::process_add_foreign_key; - processMap_["add_unique_constraint"] = &ThriftHiveMetastoreProcessor::process_add_unique_constraint; - processMap_["add_not_null_constraint"] = &ThriftHiveMetastoreProcessor::process_add_not_null_constraint; - processMap_["drop_table"] = &ThriftHiveMetastoreProcessor::process_drop_table; - processMap_["drop_table_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_drop_table_with_environment_context; - processMap_["truncate_table"] = &ThriftHiveMetastoreProcessor::process_truncate_table; - processMap_["get_tables"] = &ThriftHiveMetastoreProcessor::process_get_tables; - processMap_["get_tables_by_type"] = &ThriftHiveMetastoreProcessor::process_get_tables_by_type; - processMap_["get_table_meta"] = &ThriftHiveMetastoreProcessor::process_get_table_meta; - processMap_["get_all_tables"] = &ThriftHiveMetastoreProcessor::process_get_all_tables; - processMap_["get_table"] = &ThriftHiveMetastoreProcessor::process_get_table; - processMap_["get_table_objects_by_name"] = &ThriftHiveMetastoreProcessor::process_get_table_objects_by_name; - processMap_["get_table_req"] = &ThriftHiveMetastoreProcessor::process_get_table_req; - processMap_["get_table_objects_by_name_req"] = &ThriftHiveMetastoreProcessor::process_get_table_objects_by_name_req; - processMap_["get_table_names_by_filter"] = &ThriftHiveMetastoreProcessor::process_get_table_names_by_filter; - processMap_["alter_table"] = &ThriftHiveMetastoreProcessor::process_alter_table; - processMap_["alter_table_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_alter_table_with_environment_context; - processMap_["alter_table_with_cascade"] = &ThriftHiveMetastoreProcessor::process_alter_table_with_cascade; - processMap_["add_partition"] = &ThriftHiveMetastoreProcessor::process_add_partition; - processMap_["add_partition_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_add_partition_with_environment_context; - processMap_["add_partitions"] = &ThriftHiveMetastoreProcessor::process_add_partitions; - processMap_["add_partitions_pspec"] = &ThriftHiveMetastoreProcessor::process_add_partitions_pspec; - processMap_["append_partition"] = &ThriftHiveMetastoreProcessor::process_append_partition; - processMap_["add_partitions_req"] = &ThriftHiveMetastoreProcessor::process_add_partitions_req; - processMap_["append_partition_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_append_partition_with_environment_context; - processMap_["append_partition_by_name"] = &ThriftHiveMetastoreProcessor::process_append_partition_by_name; - processMap_["append_partition_by_name_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_append_partition_by_name_with_environment_context; - processMap_["drop_partition"] = &ThriftHiveMetastoreProcessor::process_drop_partition; - processMap_["drop_partition_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_drop_partition_with_environment_context; - processMap_["drop_partition_by_name"] = &ThriftHiveMetastoreProcessor::process_drop_partition_by_name; - processMap_["drop_partition_by_name_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_drop_partition_by_name_with_environment_context; - processMap_["drop_partitions_req"] = &ThriftHiveMetastoreProcessor::process_drop_partitions_req; - processMap_["get_partition"] = &ThriftHiveMetastoreProcessor::process_get_partition; - processMap_["exchange_partition"] = &ThriftHiveMetastoreProcessor::process_exchange_partition; - processMap_["exchange_partitions"] = &ThriftHiveMetastoreProcessor::process_exchange_partitions; - processMap_["get_partition_with_auth"] = &ThriftHiveMetastoreProcessor::process_get_partition_with_auth; - processMap_["get_partition_by_name"] = &ThriftHiveMetastoreProcessor::process_get_partition_by_name; - processMap_["get_partitions"] = &ThriftHiveMetastoreProcessor::process_get_partitions; - processMap_["get_partitions_with_auth"] = &ThriftHiveMetastoreProcessor::process_get_partitions_with_auth; - processMap_["get_partitions_pspec"] = &ThriftHiveMetastoreProcessor::process_get_partitions_pspec; - processMap_["get_partition_names"] = &ThriftHiveMetastoreProcessor::process_get_partition_names; - processMap_["get_partition_values"] = &ThriftHiveMetastoreProcessor::process_get_partition_values; - processMap_["get_partitions_ps"] = &ThriftHiveMetastoreProcessor::process_get_partitions_ps; - processMap_["get_partitions_ps_with_auth"] = &ThriftHiveMetastoreProcessor::process_get_partitions_ps_with_auth; - processMap_["get_partition_names_ps"] = &ThriftHiveMetastoreProcessor::process_get_partition_names_ps; - processMap_["get_partitions_by_filter"] = &ThriftHiveMetastoreProcessor::process_get_partitions_by_filter; - processMap_["get_part_specs_by_filter"] = &ThriftHiveMetastoreProcessor::process_get_part_specs_by_filter; - processMap_["get_partitions_by_expr"] = &ThriftHiveMetastoreProcessor::process_get_partitions_by_expr; - processMap_["get_num_partitions_by_filter"] = &ThriftHiveMetastoreProcessor::process_get_num_partitions_by_filter; - processMap_["get_partitions_by_names"] = &ThriftHiveMetastoreProcessor::process_get_partitions_by_names; - processMap_["alter_partition"] = &ThriftHiveMetastoreProcessor::process_alter_partition; - processMap_["alter_partitions"] = &ThriftHiveMetastoreProcessor::process_alter_partitions; - processMap_["alter_partitions_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_alter_partitions_with_environment_context; - processMap_["alter_partition_with_environment_context"] = &ThriftHiveMetastoreProcessor::process_alter_partition_with_environment_context; - processMap_["rename_partition"] = &ThriftHiveMetastoreProcessor::process_rename_partition; - processMap_["partition_name_has_valid_characters"] = &ThriftHiveMetastoreProcessor::process_partition_name_has_valid_characters; - processMap_["get_config_value"] = &ThriftHiveMetastoreProcessor::process_get_config_value; - processMap_["partition_name_to_vals"] = &ThriftHiveMetastoreProcessor::process_partition_name_to_vals; - processMap_["partition_name_to_spec"] = &ThriftHiveMetastoreProcessor::process_partition_name_to_spec; - processMap_["markPartitionForEvent"] = &ThriftHiveMetastoreProcessor::process_markPartitionForEvent; - processMap_["isPartitionMarkedForEvent"] = &ThriftHiveMetastoreProcessor::process_isPartitionMarkedForEvent; - processMap_["add_index"] = &ThriftHiveMetastoreProcessor::process_add_index; - processMap_["alter_index"] = &ThriftHiveMetastoreProcessor::process_alter_index; - processMap_["drop_index_by_name"] = &ThriftHiveMetastoreProcessor::process_drop_index_by_name; - processMap_["get_index_by_name"] = &ThriftHiveMetastoreProcessor::process_get_index_by_name; - processMap_["get_indexes"] = &ThriftHiveMetastoreProcessor::process_get_indexes; - processMap_["get_index_names"] = &ThriftHiveMetastoreProcessor::process_get_index_names; - processMap_["get_primary_keys"] = &ThriftHiveMetastoreProcessor::process_get_primary_keys; - processMap_["get_foreign_keys"] = &ThriftHiveMetastoreProcessor::process_get_foreign_keys; - processMap_["get_unique_constraints"] = &ThriftHiveMetastoreProcessor::process_get_unique_constraints; - processMap_["get_not_null_constraints"] = &ThriftHiveMetastoreProcessor::process_get_not_null_constraints; - processMap_["update_table_column_statistics"] = &ThriftHiveMetastoreProcessor::process_update_table_column_statistics; - processMap_["update_partition_column_statistics"] = &ThriftHiveMetastoreProcessor::process_update_partition_column_statistics; - processMap_["get_table_column_statistics"] = &ThriftHiveMetastoreProcessor::process_get_table_column_statistics; - processMap_["get_partition_column_statistics"] = &ThriftHiveMetastoreProcessor::process_get_partition_column_statistics; - processMap_["get_table_statistics_req"] = &ThriftHiveMetastoreProcessor::process_get_table_statistics_req; - processMap_["get_partitions_statistics_req"] = &ThriftHiveMetastoreProcessor::process_get_partitions_statistics_req; - processMap_["get_aggr_stats_for"] = &ThriftHiveMetastoreProcessor::process_get_aggr_stats_for; - processMap_["set_aggr_stats_for"] = &ThriftHiveMetastoreProcessor::process_set_aggr_stats_for; - processMap_["delete_partition_column_statistics"] = &ThriftHiveMetastoreProcessor::process_delete_partition_column_statistics; - processMap_["delete_table_column_statistics"] = &ThriftHiveMetastoreProcessor::process_delete_table_column_statistics; - processMap_["create_function"] = &ThriftHiveMetastoreProcessor::process_create_function; - processMap_["drop_function"] = &ThriftHiveMetastoreProcessor::process_drop_function; - processMap_["alter_function"] = &ThriftHiveMetastoreProcessor::process_alter_function; - processMap_["get_functions"] = &ThriftHiveMetastoreProcessor::process_get_functions; - processMap_["get_function"] = &ThriftHiveMetastoreProcessor::process_get_function; - processMap_["get_all_functions"] = &ThriftHiveMetastoreProcessor::process_get_all_functions; - processMap_["create_role"] = &ThriftHiveMetastoreProcessor::process_create_role; - processMap_["drop_role"] = &ThriftHiveMetastoreProcessor::process_drop_role; - processMap_["get_role_names"] = &ThriftHiveMetastoreProcessor::process_get_role_names; - processMap_["grant_role"] = &ThriftHiveMetastoreProcessor::process_grant_role; - processMap_["revoke_role"] = &ThriftHiveMetastoreProcessor::process_revoke_role; - processMap_["list_roles"] = &ThriftHiveMetastoreProcessor::process_list_roles; - processMap_["grant_revoke_role"] = &ThriftHiveMetastoreProcessor::process_grant_revoke_role; - processMap_["get_principals_in_role"] = &ThriftHiveMetastoreProcessor::process_get_principals_in_role; - processMap_["get_role_grants_for_principal"] = &ThriftHiveMetastoreProcessor::process_get_role_grants_for_principal; - processMap_["get_privilege_set"] = &ThriftHiveMetastoreProcessor::process_get_privilege_set; - processMap_["list_privileges"] = &ThriftHiveMetastoreProcessor::process_list_privileges; - processMap_["grant_privileges"] = &ThriftHiveMetastoreProcessor::process_grant_privileges; - processMap_["revoke_privileges"] = &ThriftHiveMetastoreProcessor::process_revoke_privileges; - processMap_["grant_revoke_privileges"] = &ThriftHiveMetastoreProcessor::process_grant_revoke_privileges; - processMap_["set_ugi"] = &ThriftHiveMetastoreProcessor::process_set_ugi; - processMap_["get_delegation_token"] = &ThriftHiveMetastoreProcessor::process_get_delegation_token; - processMap_["renew_delegation_token"] = &ThriftHiveMetastoreProcessor::process_renew_delegation_token; - processMap_["cancel_delegation_token"] = &ThriftHiveMetastoreProcessor::process_cancel_delegation_token; - processMap_["add_token"] = &ThriftHiveMetastoreProcessor::process_add_token; - processMap_["remove_token"] = &ThriftHiveMetastoreProcessor::process_remove_token; - processMap_["get_token"] = &ThriftHiveMetastoreProcessor::process_get_token; - processMap_["get_all_token_identifiers"] = &ThriftHiveMetastoreProcessor::process_get_all_token_identifiers; - processMap_["add_master_key"] = &ThriftHiveMetastoreProcessor::process_add_master_key; - processMap_["update_master_key"] = &ThriftHiveMetastoreProcessor::process_update_master_key; - processMap_["remove_master_key"] = &ThriftHiveMetastoreProcessor::process_remove_master_key; - processMap_["get_master_keys"] = &ThriftHiveMetastoreProcessor::process_get_master_keys; - processMap_["get_open_txns"] = &ThriftHiveMetastoreProcessor::process_get_open_txns; - processMap_["get_open_txns_info"] = &ThriftHiveMetastoreProcessor::process_get_open_txns_info; - processMap_["open_txns"] = &ThriftHiveMetastoreProcessor::process_open_txns; - processMap_["abort_txn"] = &ThriftHiveMetastoreProcessor::process_abort_txn; - processMap_["abort_txns"] = &ThriftHiveMetastoreProcessor::process_abort_txns; - processMap_["commit_txn"] = &ThriftHiveMetastoreProcessor::process_commit_txn; - processMap_["lock"] = &ThriftHiveMetastoreProcessor::process_lock; - processMap_["check_lock"] = &ThriftHiveMetastoreProcessor::process_check_lock; - processMap_["unlock"] = &ThriftHiveMetastoreProcessor::process_unlock; - processMap_["show_locks"] = &ThriftHiveMetastoreProcessor::process_show_locks; - processMap_["heartbeat"] = &ThriftHiveMetastoreProcessor::process_heartbeat; - processMap_["heartbeat_txn_range"] = &ThriftHiveMetastoreProcessor::process_heartbeat_txn_range; - processMap_["compact"] = &ThriftHiveMetastoreProcessor::process_compact; - processMap_["compact2"] = &ThriftHiveMetastoreProcessor::process_compact2; - processMap_["show_compact"] = &ThriftHiveMetastoreProcessor::process_show_compact; - processMap_["add_dynamic_partitions"] = &ThriftHiveMetastoreProcessor::process_add_dynamic_partitions; - processMap_["get_next_notification"] = &ThriftHiveMetastoreProcessor::process_get_next_notification; - processMap_["get_current_notificationEventId"] = &ThriftHiveMetastoreProcessor::process_get_current_notificationEventId; - processMap_["get_notification_events_count"] = &ThriftHiveMetastoreProcessor::process_get_notification_events_count; - processMap_["fire_listener_event"] = &ThriftHiveMetastoreProcessor::process_fire_listener_event; - processMap_["flushCache"] = &ThriftHiveMetastoreProcessor::process_flushCache; - processMap_["cm_recycle"] = &ThriftHiveMetastoreProcessor::process_cm_recycle; - processMap_["get_file_metadata_by_expr"] = &ThriftHiveMetastoreProcessor::process_get_file_metadata_by_expr; - processMap_["get_file_metadata"] = &ThriftHiveMetastoreProcessor::process_get_file_metadata; - processMap_["put_file_metadata"] = &ThriftHiveMetastoreProcessor::process_put_file_metadata; - processMap_["clear_file_metadata"] = &ThriftHiveMetastoreProcessor::process_clear_file_metadata; - processMap_["cache_file_metadata"] = &ThriftHiveMetastoreProcessor::process_cache_file_metadata; - processMap_["get_metastore_db_uuid"] = &ThriftHiveMetastoreProcessor::process_get_metastore_db_uuid; - processMap_["create_resource_plan"] = &ThriftHiveMetastoreProcessor::process_create_resource_plan; - processMap_["get_resource_plan"] = &ThriftHiveMetastoreProcessor::process_get_resource_plan; - processMap_["get_active_resource_plan"] = &ThriftHiveMetastoreProcessor::process_get_active_resource_plan; - processMap_["get_all_resource_plans"] = &ThriftHiveMetastoreProcessor::process_get_all_resource_plans; - processMap_["alter_resource_plan"] = &ThriftHiveMetastoreProcessor::process_alter_resource_plan; - processMap_["validate_resource_plan"] = &ThriftHiveMetastoreProcessor::process_validate_resource_plan; - processMap_["drop_resource_plan"] = &ThriftHiveMetastoreProcessor::process_drop_resource_plan; - processMap_["create_wm_trigger"] = &ThriftHiveMetastoreProcessor::process_create_wm_trigger; - processMap_["alter_wm_trigger"] = &ThriftHiveMetastoreProcessor::process_alter_wm_trigger; - processMap_["drop_wm_trigger"] = &ThriftHiveMetastoreProcessor::process_drop_wm_trigger; - processMap_["get_triggers_for_resourceplan"] = &ThriftHiveMetastoreProcessor::process_get_triggers_for_resourceplan; - } - - virtual ~ThriftHiveMetastoreProcessor() {} -}; - -class ThriftHiveMetastoreProcessorFactory : public ::apache::thrift::TProcessorFactory { - public: - ThriftHiveMetastoreProcessorFactory(const ::std::shared_ptr< ThriftHiveMetastoreIfFactory >& handlerFactory) : - handlerFactory_(handlerFactory) {} - - ::std::shared_ptr< ::apache::thrift::TProcessor > getProcessor(const ::apache::thrift::TConnectionInfo& connInfo); - - protected: - ::std::shared_ptr< ThriftHiveMetastoreIfFactory > handlerFactory_; -}; - -class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf { - public: - ThriftHiveMetastoreMultiface(std::vector >& ifaces) : ifaces_(ifaces) { - } - virtual ~ThriftHiveMetastoreMultiface() {} - protected: - std::vector > ifaces_; - ThriftHiveMetastoreMultiface() {} - void add(::std::shared_ptr iface) { - ifaces_.push_back(iface); - } - public: - void getMetaConf(std::string& _return, const std::string& key) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->getMetaConf(_return, key); - } - ifaces_[i]->getMetaConf(_return, key); - return; - } - - void setMetaConf(const std::string& key, const std::string& value) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->setMetaConf(key, value); - } - ifaces_[i]->setMetaConf(key, value); - } - - void create_database(const Database& database) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->create_database(database); - } - ifaces_[i]->create_database(database); - } - - void get_database(Database& _return, const std::string& name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_database(_return, name); - } - ifaces_[i]->get_database(_return, name); - return; - } - - void drop_database(const std::string& name, const bool deleteData, const bool cascade) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->drop_database(name, deleteData, cascade); - } - ifaces_[i]->drop_database(name, deleteData, cascade); - } - - void get_databases(std::vector & _return, const std::string& pattern) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_databases(_return, pattern); - } - ifaces_[i]->get_databases(_return, pattern); - return; - } - - void get_all_databases(std::vector & _return) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_all_databases(_return); - } - ifaces_[i]->get_all_databases(_return); - return; - } - - void alter_database(const std::string& dbname, const Database& db) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->alter_database(dbname, db); - } - ifaces_[i]->alter_database(dbname, db); - } - - void get_type(Type& _return, const std::string& name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_type(_return, name); - } - ifaces_[i]->get_type(_return, name); - return; - } - - bool create_type(const Type& type) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->create_type(type); - } - return ifaces_[i]->create_type(type); - } - - bool drop_type(const std::string& type) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->drop_type(type); - } - return ifaces_[i]->drop_type(type); - } - - void get_type_all(std::map & _return, const std::string& name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_type_all(_return, name); - } - ifaces_[i]->get_type_all(_return, name); - return; - } - - void get_fields(std::vector & _return, const std::string& db_name, const std::string& table_name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_fields(_return, db_name, table_name); - } - ifaces_[i]->get_fields(_return, db_name, table_name); - return; - } - - void get_fields_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_fields_with_environment_context(_return, db_name, table_name, environment_context); - } - ifaces_[i]->get_fields_with_environment_context(_return, db_name, table_name, environment_context); - return; - } - - void get_schema(std::vector & _return, const std::string& db_name, const std::string& table_name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_schema(_return, db_name, table_name); - } - ifaces_[i]->get_schema(_return, db_name, table_name); - return; - } - - void get_schema_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_schema_with_environment_context(_return, db_name, table_name, environment_context); - } - ifaces_[i]->get_schema_with_environment_context(_return, db_name, table_name, environment_context); - return; - } - - void create_table(const Table& tbl) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->create_table(tbl); - } - ifaces_[i]->create_table(tbl); - } - - void create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->create_table_with_environment_context(tbl, environment_context); - } - ifaces_[i]->create_table_with_environment_context(tbl, environment_context); - } - - void create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints); - } - ifaces_[i]->create_table_with_constraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints); - } - - void drop_constraint(const DropConstraintRequest& req) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->drop_constraint(req); - } - ifaces_[i]->drop_constraint(req); - } - - void add_primary_key(const AddPrimaryKeyRequest& req) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->add_primary_key(req); - } - ifaces_[i]->add_primary_key(req); - } - - void add_foreign_key(const AddForeignKeyRequest& req) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->add_foreign_key(req); - } - ifaces_[i]->add_foreign_key(req); - } - - void add_unique_constraint(const AddUniqueConstraintRequest& req) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->add_unique_constraint(req); - } - ifaces_[i]->add_unique_constraint(req); - } - - void add_not_null_constraint(const AddNotNullConstraintRequest& req) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->add_not_null_constraint(req); - } - ifaces_[i]->add_not_null_constraint(req); - } - - void drop_table(const std::string& dbname, const std::string& name, const bool deleteData) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->drop_table(dbname, name, deleteData); - } - ifaces_[i]->drop_table(dbname, name, deleteData); - } - - void drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->drop_table_with_environment_context(dbname, name, deleteData, environment_context); - } - ifaces_[i]->drop_table_with_environment_context(dbname, name, deleteData, environment_context); - } - - void truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->truncate_table(dbName, tableName, partNames); - } - ifaces_[i]->truncate_table(dbName, tableName, partNames); - } - - void get_tables(std::vector & _return, const std::string& db_name, const std::string& pattern) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_tables(_return, db_name, pattern); - } - ifaces_[i]->get_tables(_return, db_name, pattern); - return; - } - - void get_tables_by_type(std::vector & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_tables_by_type(_return, db_name, pattern, tableType); - } - ifaces_[i]->get_tables_by_type(_return, db_name, pattern, tableType); - return; - } - - void get_table_meta(std::vector & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_table_meta(_return, db_patterns, tbl_patterns, tbl_types); - } - ifaces_[i]->get_table_meta(_return, db_patterns, tbl_patterns, tbl_types); - return; - } - - void get_all_tables(std::vector & _return, const std::string& db_name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_all_tables(_return, db_name); - } - ifaces_[i]->get_all_tables(_return, db_name); - return; - } - - void get_table(Table& _return, const std::string& dbname, const std::string& tbl_name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_table(_return, dbname, tbl_name); - } - ifaces_[i]->get_table(_return, dbname, tbl_name); - return; - } - - void get_table_objects_by_name(std::vector
& _return, const std::string& dbname, const std::vector & tbl_names) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_table_objects_by_name(_return, dbname, tbl_names); - } - ifaces_[i]->get_table_objects_by_name(_return, dbname, tbl_names); - return; - } - - void get_table_req(GetTableResult& _return, const GetTableRequest& req) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_table_req(_return, req); - } - ifaces_[i]->get_table_req(_return, req); - return; - } - - void get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_table_objects_by_name_req(_return, req); - } - ifaces_[i]->get_table_objects_by_name_req(_return, req); - return; - } - - void get_table_names_by_filter(std::vector & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_table_names_by_filter(_return, dbname, filter, max_tables); - } - ifaces_[i]->get_table_names_by_filter(_return, dbname, filter, max_tables); - return; - } - - void alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->alter_table(dbname, tbl_name, new_tbl); - } - ifaces_[i]->alter_table(dbname, tbl_name, new_tbl); - } - - void alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->alter_table_with_environment_context(dbname, tbl_name, new_tbl, environment_context); - } - ifaces_[i]->alter_table_with_environment_context(dbname, tbl_name, new_tbl, environment_context); - } - - void alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->alter_table_with_cascade(dbname, tbl_name, new_tbl, cascade); - } - ifaces_[i]->alter_table_with_cascade(dbname, tbl_name, new_tbl, cascade); - } - - void add_partition(Partition& _return, const Partition& new_part) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->add_partition(_return, new_part); - } - ifaces_[i]->add_partition(_return, new_part); - return; - } - - void add_partition_with_environment_context(Partition& _return, const Partition& new_part, const EnvironmentContext& environment_context) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->add_partition_with_environment_context(_return, new_part, environment_context); - } - ifaces_[i]->add_partition_with_environment_context(_return, new_part, environment_context); - return; - } - - int32_t add_partitions(const std::vector & new_parts) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->add_partitions(new_parts); - } - return ifaces_[i]->add_partitions(new_parts); - } - - int32_t add_partitions_pspec(const std::vector & new_parts) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->add_partitions_pspec(new_parts); - } - return ifaces_[i]->add_partitions_pspec(new_parts); - } - - void append_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->append_partition(_return, db_name, tbl_name, part_vals); - } - ifaces_[i]->append_partition(_return, db_name, tbl_name, part_vals); - return; - } - - void add_partitions_req(AddPartitionsResult& _return, const AddPartitionsRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->add_partitions_req(_return, request); - } - ifaces_[i]->add_partitions_req(_return, request); - return; - } - - void append_partition_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->append_partition_with_environment_context(_return, db_name, tbl_name, part_vals, environment_context); - } - ifaces_[i]->append_partition_with_environment_context(_return, db_name, tbl_name, part_vals, environment_context); - return; - } - - void append_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->append_partition_by_name(_return, db_name, tbl_name, part_name); - } - ifaces_[i]->append_partition_by_name(_return, db_name, tbl_name, part_name); - return; - } - - void append_partition_by_name_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->append_partition_by_name_with_environment_context(_return, db_name, tbl_name, part_name, environment_context); - } - ifaces_[i]->append_partition_by_name_with_environment_context(_return, db_name, tbl_name, part_name, environment_context); - return; - } - - bool drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->drop_partition(db_name, tbl_name, part_vals, deleteData); - } - return ifaces_[i]->drop_partition(db_name, tbl_name, part_vals, deleteData); - } - - bool drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->drop_partition_with_environment_context(db_name, tbl_name, part_vals, deleteData, environment_context); - } - return ifaces_[i]->drop_partition_with_environment_context(db_name, tbl_name, part_vals, deleteData, environment_context); - } - - bool drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->drop_partition_by_name(db_name, tbl_name, part_name, deleteData); - } - return ifaces_[i]->drop_partition_by_name(db_name, tbl_name, part_name, deleteData); - } - - bool drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->drop_partition_by_name_with_environment_context(db_name, tbl_name, part_name, deleteData, environment_context); - } - return ifaces_[i]->drop_partition_by_name_with_environment_context(db_name, tbl_name, part_name, deleteData, environment_context); - } - - void drop_partitions_req(DropPartitionsResult& _return, const DropPartitionsRequest& req) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->drop_partitions_req(_return, req); - } - ifaces_[i]->drop_partitions_req(_return, req); - return; - } - - void get_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_partition(_return, db_name, tbl_name, part_vals); - } - ifaces_[i]->get_partition(_return, db_name, tbl_name, part_vals); - return; - } - - void exchange_partition(Partition& _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->exchange_partition(_return, partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); - } - ifaces_[i]->exchange_partition(_return, partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); - return; - } - - void exchange_partitions(std::vector & _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->exchange_partitions(_return, partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); - } - ifaces_[i]->exchange_partitions(_return, partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); - return; - } - - void get_partition_with_auth(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_partition_with_auth(_return, db_name, tbl_name, part_vals, user_name, group_names); - } - ifaces_[i]->get_partition_with_auth(_return, db_name, tbl_name, part_vals, user_name, group_names); - return; - } - - void get_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_partition_by_name(_return, db_name, tbl_name, part_name); - } - ifaces_[i]->get_partition_by_name(_return, db_name, tbl_name, part_name); - return; - } - - void get_partitions(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_partitions(_return, db_name, tbl_name, max_parts); - } - ifaces_[i]->get_partitions(_return, db_name, tbl_name, max_parts); - return; - } - - void get_partitions_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_partitions_with_auth(_return, db_name, tbl_name, max_parts, user_name, group_names); - } - ifaces_[i]->get_partitions_with_auth(_return, db_name, tbl_name, max_parts, user_name, group_names); - return; - } - - void get_partitions_pspec(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int32_t max_parts) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_partitions_pspec(_return, db_name, tbl_name, max_parts); - } - ifaces_[i]->get_partitions_pspec(_return, db_name, tbl_name, max_parts); - return; - } - - void get_partition_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_partition_names(_return, db_name, tbl_name, max_parts); - } - ifaces_[i]->get_partition_names(_return, db_name, tbl_name, max_parts); - return; - } - - void get_partition_values(PartitionValuesResponse& _return, const PartitionValuesRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_partition_values(_return, request); - } - ifaces_[i]->get_partition_values(_return, request); - return; - } - - void get_partitions_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_partitions_ps(_return, db_name, tbl_name, part_vals, max_parts); - } - ifaces_[i]->get_partitions_ps(_return, db_name, tbl_name, part_vals, max_parts); - return; - } - - void get_partitions_ps_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_partitions_ps_with_auth(_return, db_name, tbl_name, part_vals, max_parts, user_name, group_names); - } - ifaces_[i]->get_partitions_ps_with_auth(_return, db_name, tbl_name, part_vals, max_parts, user_name, group_names); - return; - } - - void get_partition_names_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_partition_names_ps(_return, db_name, tbl_name, part_vals, max_parts); - } - ifaces_[i]->get_partition_names_ps(_return, db_name, tbl_name, part_vals, max_parts); - return; - } - - void get_partitions_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_partitions_by_filter(_return, db_name, tbl_name, filter, max_parts); - } - ifaces_[i]->get_partitions_by_filter(_return, db_name, tbl_name, filter, max_parts); - return; - } - - void get_part_specs_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_part_specs_by_filter(_return, db_name, tbl_name, filter, max_parts); - } - ifaces_[i]->get_part_specs_by_filter(_return, db_name, tbl_name, filter, max_parts); - return; - } - - void get_partitions_by_expr(PartitionsByExprResult& _return, const PartitionsByExprRequest& req) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_partitions_by_expr(_return, req); - } - ifaces_[i]->get_partitions_by_expr(_return, req); - return; - } - - int32_t get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_num_partitions_by_filter(db_name, tbl_name, filter); - } - return ifaces_[i]->get_num_partitions_by_filter(db_name, tbl_name, filter); - } - - void get_partitions_by_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & names) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_partitions_by_names(_return, db_name, tbl_name, names); - } - ifaces_[i]->get_partitions_by_names(_return, db_name, tbl_name, names); - return; - } - - void alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->alter_partition(db_name, tbl_name, new_part); - } - ifaces_[i]->alter_partition(db_name, tbl_name, new_part); - } - - void alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->alter_partitions(db_name, tbl_name, new_parts); - } - ifaces_[i]->alter_partitions(db_name, tbl_name, new_parts); - } - - void alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->alter_partitions_with_environment_context(db_name, tbl_name, new_parts, environment_context); - } - ifaces_[i]->alter_partitions_with_environment_context(db_name, tbl_name, new_parts, environment_context); - } - - void alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->alter_partition_with_environment_context(db_name, tbl_name, new_part, environment_context); - } - ifaces_[i]->alter_partition_with_environment_context(db_name, tbl_name, new_part, environment_context); - } - - void rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->rename_partition(db_name, tbl_name, part_vals, new_part); - } - ifaces_[i]->rename_partition(db_name, tbl_name, part_vals, new_part); - } - - bool partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->partition_name_has_valid_characters(part_vals, throw_exception); - } - return ifaces_[i]->partition_name_has_valid_characters(part_vals, throw_exception); - } - - void get_config_value(std::string& _return, const std::string& name, const std::string& defaultValue) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_config_value(_return, name, defaultValue); - } - ifaces_[i]->get_config_value(_return, name, defaultValue); - return; - } - - void partition_name_to_vals(std::vector & _return, const std::string& part_name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->partition_name_to_vals(_return, part_name); - } - ifaces_[i]->partition_name_to_vals(_return, part_name); - return; - } - - void partition_name_to_spec(std::map & _return, const std::string& part_name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->partition_name_to_spec(_return, part_name); - } - ifaces_[i]->partition_name_to_spec(_return, part_name); - return; - } - - void markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->markPartitionForEvent(db_name, tbl_name, part_vals, eventType); - } - ifaces_[i]->markPartitionForEvent(db_name, tbl_name, part_vals, eventType); - } - - bool isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->isPartitionMarkedForEvent(db_name, tbl_name, part_vals, eventType); - } - return ifaces_[i]->isPartitionMarkedForEvent(db_name, tbl_name, part_vals, eventType); - } - - void add_index(Index& _return, const Index& new_index, const Table& index_table) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->add_index(_return, new_index, index_table); - } - ifaces_[i]->add_index(_return, new_index, index_table); - return; - } - - void alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->alter_index(dbname, base_tbl_name, idx_name, new_idx); - } - ifaces_[i]->alter_index(dbname, base_tbl_name, idx_name, new_idx); - } - - bool drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->drop_index_by_name(db_name, tbl_name, index_name, deleteData); - } - return ifaces_[i]->drop_index_by_name(db_name, tbl_name, index_name, deleteData); - } - - void get_index_by_name(Index& _return, const std::string& db_name, const std::string& tbl_name, const std::string& index_name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_index_by_name(_return, db_name, tbl_name, index_name); - } - ifaces_[i]->get_index_by_name(_return, db_name, tbl_name, index_name); - return; - } - - void get_indexes(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_indexes(_return, db_name, tbl_name, max_indexes); - } - ifaces_[i]->get_indexes(_return, db_name, tbl_name, max_indexes); - return; - } - - void get_index_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_index_names(_return, db_name, tbl_name, max_indexes); - } - ifaces_[i]->get_index_names(_return, db_name, tbl_name, max_indexes); - return; - } - - void get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_primary_keys(_return, request); - } - ifaces_[i]->get_primary_keys(_return, request); - return; - } - - void get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_foreign_keys(_return, request); - } - ifaces_[i]->get_foreign_keys(_return, request); - return; - } - - void get_unique_constraints(UniqueConstraintsResponse& _return, const UniqueConstraintsRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_unique_constraints(_return, request); - } - ifaces_[i]->get_unique_constraints(_return, request); - return; - } - - void get_not_null_constraints(NotNullConstraintsResponse& _return, const NotNullConstraintsRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_not_null_constraints(_return, request); - } - ifaces_[i]->get_not_null_constraints(_return, request); - return; - } - - bool update_table_column_statistics(const ColumnStatistics& stats_obj) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->update_table_column_statistics(stats_obj); - } - return ifaces_[i]->update_table_column_statistics(stats_obj); - } - - bool update_partition_column_statistics(const ColumnStatistics& stats_obj) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->update_partition_column_statistics(stats_obj); - } - return ifaces_[i]->update_partition_column_statistics(stats_obj); - } - - void get_table_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& col_name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_table_column_statistics(_return, db_name, tbl_name, col_name); - } - ifaces_[i]->get_table_column_statistics(_return, db_name, tbl_name, col_name); - return; - } - - void get_partition_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_partition_column_statistics(_return, db_name, tbl_name, part_name, col_name); - } - ifaces_[i]->get_partition_column_statistics(_return, db_name, tbl_name, part_name, col_name); - return; - } - - void get_table_statistics_req(TableStatsResult& _return, const TableStatsRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_table_statistics_req(_return, request); - } - ifaces_[i]->get_table_statistics_req(_return, request); - return; - } - - void get_partitions_statistics_req(PartitionsStatsResult& _return, const PartitionsStatsRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_partitions_statistics_req(_return, request); - } - ifaces_[i]->get_partitions_statistics_req(_return, request); - return; - } - - void get_aggr_stats_for(AggrStats& _return, const PartitionsStatsRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_aggr_stats_for(_return, request); - } - ifaces_[i]->get_aggr_stats_for(_return, request); - return; - } - - bool set_aggr_stats_for(const SetPartitionsStatsRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->set_aggr_stats_for(request); - } - return ifaces_[i]->set_aggr_stats_for(request); - } - - bool delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->delete_partition_column_statistics(db_name, tbl_name, part_name, col_name); - } - return ifaces_[i]->delete_partition_column_statistics(db_name, tbl_name, part_name, col_name); - } - - bool delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->delete_table_column_statistics(db_name, tbl_name, col_name); - } - return ifaces_[i]->delete_table_column_statistics(db_name, tbl_name, col_name); - } - - void create_function(const Function& func) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->create_function(func); - } - ifaces_[i]->create_function(func); - } - - void drop_function(const std::string& dbName, const std::string& funcName) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->drop_function(dbName, funcName); - } - ifaces_[i]->drop_function(dbName, funcName); - } - - void alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->alter_function(dbName, funcName, newFunc); - } - ifaces_[i]->alter_function(dbName, funcName, newFunc); - } - - void get_functions(std::vector & _return, const std::string& dbName, const std::string& pattern) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_functions(_return, dbName, pattern); - } - ifaces_[i]->get_functions(_return, dbName, pattern); - return; - } - - void get_function(Function& _return, const std::string& dbName, const std::string& funcName) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_function(_return, dbName, funcName); - } - ifaces_[i]->get_function(_return, dbName, funcName); - return; - } - - void get_all_functions(GetAllFunctionsResponse& _return) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_all_functions(_return); - } - ifaces_[i]->get_all_functions(_return); - return; - } - - bool create_role(const Role& role) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->create_role(role); - } - return ifaces_[i]->create_role(role); - } - - bool drop_role(const std::string& role_name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->drop_role(role_name); - } - return ifaces_[i]->drop_role(role_name); - } - - void get_role_names(std::vector & _return) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_role_names(_return); - } - ifaces_[i]->get_role_names(_return); - return; - } - - bool grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->grant_role(role_name, principal_name, principal_type, grantor, grantorType, grant_option); - } - return ifaces_[i]->grant_role(role_name, principal_name, principal_type, grantor, grantorType, grant_option); - } - - bool revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->revoke_role(role_name, principal_name, principal_type); - } - return ifaces_[i]->revoke_role(role_name, principal_name, principal_type); - } - - void list_roles(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->list_roles(_return, principal_name, principal_type); - } - ifaces_[i]->list_roles(_return, principal_name, principal_type); - return; - } - - void grant_revoke_role(GrantRevokeRoleResponse& _return, const GrantRevokeRoleRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->grant_revoke_role(_return, request); - } - ifaces_[i]->grant_revoke_role(_return, request); - return; - } - - void get_principals_in_role(GetPrincipalsInRoleResponse& _return, const GetPrincipalsInRoleRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_principals_in_role(_return, request); - } - ifaces_[i]->get_principals_in_role(_return, request); - return; - } - - void get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return, const GetRoleGrantsForPrincipalRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_role_grants_for_principal(_return, request); - } - ifaces_[i]->get_role_grants_for_principal(_return, request); - return; - } - - void get_privilege_set(PrincipalPrivilegeSet& _return, const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_privilege_set(_return, hiveObject, user_name, group_names); - } - ifaces_[i]->get_privilege_set(_return, hiveObject, user_name, group_names); - return; - } - - void list_privileges(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->list_privileges(_return, principal_name, principal_type, hiveObject); - } - ifaces_[i]->list_privileges(_return, principal_name, principal_type, hiveObject); - return; - } - - bool grant_privileges(const PrivilegeBag& privileges) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->grant_privileges(privileges); - } - return ifaces_[i]->grant_privileges(privileges); - } - - bool revoke_privileges(const PrivilegeBag& privileges) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->revoke_privileges(privileges); - } - return ifaces_[i]->revoke_privileges(privileges); - } - - void grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const GrantRevokePrivilegeRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->grant_revoke_privileges(_return, request); - } - ifaces_[i]->grant_revoke_privileges(_return, request); - return; - } - - void set_ugi(std::vector & _return, const std::string& user_name, const std::vector & group_names) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->set_ugi(_return, user_name, group_names); - } - ifaces_[i]->set_ugi(_return, user_name, group_names); - return; - } - - void get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_delegation_token(_return, token_owner, renewer_kerberos_principal_name); - } - ifaces_[i]->get_delegation_token(_return, token_owner, renewer_kerberos_principal_name); - return; - } - - int64_t renew_delegation_token(const std::string& token_str_form) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->renew_delegation_token(token_str_form); - } - return ifaces_[i]->renew_delegation_token(token_str_form); - } - - void cancel_delegation_token(const std::string& token_str_form) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->cancel_delegation_token(token_str_form); - } - ifaces_[i]->cancel_delegation_token(token_str_form); - } - - bool add_token(const std::string& token_identifier, const std::string& delegation_token) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->add_token(token_identifier, delegation_token); - } - return ifaces_[i]->add_token(token_identifier, delegation_token); - } - - bool remove_token(const std::string& token_identifier) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->remove_token(token_identifier); - } - return ifaces_[i]->remove_token(token_identifier); - } - - void get_token(std::string& _return, const std::string& token_identifier) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_token(_return, token_identifier); - } - ifaces_[i]->get_token(_return, token_identifier); - return; - } - - void get_all_token_identifiers(std::vector & _return) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_all_token_identifiers(_return); - } - ifaces_[i]->get_all_token_identifiers(_return); - return; - } - - int32_t add_master_key(const std::string& key) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->add_master_key(key); - } - return ifaces_[i]->add_master_key(key); - } - - void update_master_key(const int32_t seq_number, const std::string& key) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->update_master_key(seq_number, key); - } - ifaces_[i]->update_master_key(seq_number, key); - } - - bool remove_master_key(const int32_t key_seq) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->remove_master_key(key_seq); - } - return ifaces_[i]->remove_master_key(key_seq); - } - - void get_master_keys(std::vector & _return) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_master_keys(_return); - } - ifaces_[i]->get_master_keys(_return); - return; - } - - void get_open_txns(GetOpenTxnsResponse& _return) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_open_txns(_return); - } - ifaces_[i]->get_open_txns(_return); - return; - } - - void get_open_txns_info(GetOpenTxnsInfoResponse& _return) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_open_txns_info(_return); - } - ifaces_[i]->get_open_txns_info(_return); - return; - } - - void open_txns(OpenTxnsResponse& _return, const OpenTxnRequest& rqst) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->open_txns(_return, rqst); - } - ifaces_[i]->open_txns(_return, rqst); - return; - } - - void abort_txn(const AbortTxnRequest& rqst) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->abort_txn(rqst); - } - ifaces_[i]->abort_txn(rqst); - } - - void abort_txns(const AbortTxnsRequest& rqst) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->abort_txns(rqst); - } - ifaces_[i]->abort_txns(rqst); - } - - void commit_txn(const CommitTxnRequest& rqst) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->commit_txn(rqst); - } - ifaces_[i]->commit_txn(rqst); - } - - void lock(LockResponse& _return, const LockRequest& rqst) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->lock(_return, rqst); - } - ifaces_[i]->lock(_return, rqst); - return; - } - - void check_lock(LockResponse& _return, const CheckLockRequest& rqst) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->check_lock(_return, rqst); - } - ifaces_[i]->check_lock(_return, rqst); - return; - } - - void unlock(const UnlockRequest& rqst) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->unlock(rqst); - } - ifaces_[i]->unlock(rqst); - } - - void show_locks(ShowLocksResponse& _return, const ShowLocksRequest& rqst) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->show_locks(_return, rqst); - } - ifaces_[i]->show_locks(_return, rqst); - return; - } - - void heartbeat(const HeartbeatRequest& ids) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->heartbeat(ids); - } - ifaces_[i]->heartbeat(ids); - } - - void heartbeat_txn_range(HeartbeatTxnRangeResponse& _return, const HeartbeatTxnRangeRequest& txns) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->heartbeat_txn_range(_return, txns); - } - ifaces_[i]->heartbeat_txn_range(_return, txns); - return; - } - - void compact(const CompactionRequest& rqst) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->compact(rqst); - } - ifaces_[i]->compact(rqst); - } - - void compact2(CompactionResponse& _return, const CompactionRequest& rqst) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->compact2(_return, rqst); - } - ifaces_[i]->compact2(_return, rqst); - return; - } - - void show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->show_compact(_return, rqst); - } - ifaces_[i]->show_compact(_return, rqst); - return; - } - - void add_dynamic_partitions(const AddDynamicPartitions& rqst) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->add_dynamic_partitions(rqst); - } - ifaces_[i]->add_dynamic_partitions(rqst); - } - - void get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_next_notification(_return, rqst); - } - ifaces_[i]->get_next_notification(_return, rqst); - return; - } - - void get_current_notificationEventId(CurrentNotificationEventId& _return) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_current_notificationEventId(_return); - } - ifaces_[i]->get_current_notificationEventId(_return); - return; - } - - void get_notification_events_count(NotificationEventsCountResponse& _return, const NotificationEventsCountRequest& rqst) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_notification_events_count(_return, rqst); - } - ifaces_[i]->get_notification_events_count(_return, rqst); - return; - } - - void fire_listener_event(FireEventResponse& _return, const FireEventRequest& rqst) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->fire_listener_event(_return, rqst); - } - ifaces_[i]->fire_listener_event(_return, rqst); - return; - } - - void flushCache() { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->flushCache(); - } - ifaces_[i]->flushCache(); - } - - void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->cm_recycle(_return, request); - } - ifaces_[i]->cm_recycle(_return, request); - return; - } - - void get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_file_metadata_by_expr(_return, req); - } - ifaces_[i]->get_file_metadata_by_expr(_return, req); - return; - } - - void get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_file_metadata(_return, req); - } - ifaces_[i]->get_file_metadata(_return, req); - return; - } - - void put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->put_file_metadata(_return, req); - } - ifaces_[i]->put_file_metadata(_return, req); - return; - } - - void clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->clear_file_metadata(_return, req); - } - ifaces_[i]->clear_file_metadata(_return, req); - return; - } - - void cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->cache_file_metadata(_return, req); - } - ifaces_[i]->cache_file_metadata(_return, req); - return; - } - - void get_metastore_db_uuid(std::string& _return) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_metastore_db_uuid(_return); - } - ifaces_[i]->get_metastore_db_uuid(_return); - return; - } - - void create_resource_plan(WMCreateResourcePlanResponse& _return, const WMCreateResourcePlanRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->create_resource_plan(_return, request); - } - ifaces_[i]->create_resource_plan(_return, request); - return; - } - - void get_resource_plan(WMGetResourcePlanResponse& _return, const WMGetResourcePlanRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_resource_plan(_return, request); - } - ifaces_[i]->get_resource_plan(_return, request); - return; - } - - void get_active_resource_plan(WMGetActiveResourcePlanResponse& _return, const WMGetActiveResourcePlanRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_active_resource_plan(_return, request); - } - ifaces_[i]->get_active_resource_plan(_return, request); - return; - } - - void get_all_resource_plans(WMGetAllResourcePlanResponse& _return, const WMGetAllResourcePlanRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_all_resource_plans(_return, request); - } - ifaces_[i]->get_all_resource_plans(_return, request); - return; - } - - void alter_resource_plan(WMAlterResourcePlanResponse& _return, const WMAlterResourcePlanRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->alter_resource_plan(_return, request); - } - ifaces_[i]->alter_resource_plan(_return, request); - return; - } - - void validate_resource_plan(WMValidateResourcePlanResponse& _return, const WMValidateResourcePlanRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->validate_resource_plan(_return, request); - } - ifaces_[i]->validate_resource_plan(_return, request); - return; - } - - void drop_resource_plan(WMDropResourcePlanResponse& _return, const WMDropResourcePlanRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->drop_resource_plan(_return, request); - } - ifaces_[i]->drop_resource_plan(_return, request); - return; - } - - void create_wm_trigger(WMCreateTriggerResponse& _return, const WMCreateTriggerRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->create_wm_trigger(_return, request); - } - ifaces_[i]->create_wm_trigger(_return, request); - return; - } - - void alter_wm_trigger(WMAlterTriggerResponse& _return, const WMAlterTriggerRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->alter_wm_trigger(_return, request); - } - ifaces_[i]->alter_wm_trigger(_return, request); - return; - } - - void drop_wm_trigger(WMDropTriggerResponse& _return, const WMDropTriggerRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->drop_wm_trigger(_return, request); - } - ifaces_[i]->drop_wm_trigger(_return, request); - return; - } - - void get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return, const WMGetTriggersForResourePlanRequest& request) { - size_t sz = ifaces_.size(); - size_t i = 0; - for (; i < (sz - 1); ++i) { - ifaces_[i]->get_triggers_for_resourceplan(_return, request); - } - ifaces_[i]->get_triggers_for_resourceplan(_return, request); - return; - } - -}; - -// The 'concurrent' client is a thread safe client that correctly handles -// out of order responses. It is slower than the regular client, so should -// only be used when you need to share a connection among multiple threads -class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf { - public: - ThriftHiveMetastoreConcurrentClient(std::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) { - setProtocol(prot); - } - ThriftHiveMetastoreConcurrentClient(std::shared_ptr< ::apache::thrift::protocol::TProtocol> iprot, std::shared_ptr< ::apache::thrift::protocol::TProtocol> oprot) { - setProtocol(iprot,oprot); - } - private: - void setProtocol(std::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) { - setProtocol(prot,prot); - } - void setProtocol(std::shared_ptr< ::apache::thrift::protocol::TProtocol> iprot, std::shared_ptr< ::apache::thrift::protocol::TProtocol> oprot) { - piprot_=iprot; - poprot_=oprot; - iprot_ = iprot.get(); - oprot_ = oprot.get(); - } - public: - std::shared_ptr< ::apache::thrift::protocol::TProtocol> getInputProtocol() { - return piprot_; - } - std::shared_ptr< ::apache::thrift::protocol::TProtocol> getOutputProtocol() { - return poprot_; - } - void getMetaConf(std::string& _return, const std::string& key); - int32_t send_getMetaConf(const std::string& key); - void recv_getMetaConf(std::string& _return, const int32_t seqid); - void setMetaConf(const std::string& key, const std::string& value); - int32_t send_setMetaConf(const std::string& key, const std::string& value); - void recv_setMetaConf(const int32_t seqid); - void create_database(const Database& database); - int32_t send_create_database(const Database& database); - void recv_create_database(const int32_t seqid); - void get_database(Database& _return, const std::string& name); - int32_t send_get_database(const std::string& name); - void recv_get_database(Database& _return, const int32_t seqid); - void drop_database(const std::string& name, const bool deleteData, const bool cascade); - int32_t send_drop_database(const std::string& name, const bool deleteData, const bool cascade); - void recv_drop_database(const int32_t seqid); - void get_databases(std::vector & _return, const std::string& pattern); - int32_t send_get_databases(const std::string& pattern); - void recv_get_databases(std::vector & _return, const int32_t seqid); - void get_all_databases(std::vector & _return); - int32_t send_get_all_databases(); - void recv_get_all_databases(std::vector & _return, const int32_t seqid); - void alter_database(const std::string& dbname, const Database& db); - int32_t send_alter_database(const std::string& dbname, const Database& db); - void recv_alter_database(const int32_t seqid); - void get_type(Type& _return, const std::string& name); - int32_t send_get_type(const std::string& name); - void recv_get_type(Type& _return, const int32_t seqid); - bool create_type(const Type& type); - int32_t send_create_type(const Type& type); - bool recv_create_type(const int32_t seqid); - bool drop_type(const std::string& type); - int32_t send_drop_type(const std::string& type); - bool recv_drop_type(const int32_t seqid); - void get_type_all(std::map & _return, const std::string& name); - int32_t send_get_type_all(const std::string& name); - void recv_get_type_all(std::map & _return, const int32_t seqid); - void get_fields(std::vector & _return, const std::string& db_name, const std::string& table_name); - int32_t send_get_fields(const std::string& db_name, const std::string& table_name); - void recv_get_fields(std::vector & _return, const int32_t seqid); - void get_fields_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context); - int32_t send_get_fields_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context); - void recv_get_fields_with_environment_context(std::vector & _return, const int32_t seqid); - void get_schema(std::vector & _return, const std::string& db_name, const std::string& table_name); - int32_t send_get_schema(const std::string& db_name, const std::string& table_name); - void recv_get_schema(std::vector & _return, const int32_t seqid); - void get_schema_with_environment_context(std::vector & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context); - int32_t send_get_schema_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context); - void recv_get_schema_with_environment_context(std::vector & _return, const int32_t seqid); - void create_table(const Table& tbl); - int32_t send_create_table(const Table& tbl); - void recv_create_table(const int32_t seqid); - void create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context); - int32_t send_create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context); - void recv_create_table_with_environment_context(const int32_t seqid); - void create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints); - int32_t send_create_table_with_constraints(const Table& tbl, const std::vector & primaryKeys, const std::vector & foreignKeys, const std::vector & uniqueConstraints, const std::vector & notNullConstraints); - void recv_create_table_with_constraints(const int32_t seqid); - void drop_constraint(const DropConstraintRequest& req); - int32_t send_drop_constraint(const DropConstraintRequest& req); - void recv_drop_constraint(const int32_t seqid); - void add_primary_key(const AddPrimaryKeyRequest& req); - int32_t send_add_primary_key(const AddPrimaryKeyRequest& req); - void recv_add_primary_key(const int32_t seqid); - void add_foreign_key(const AddForeignKeyRequest& req); - int32_t send_add_foreign_key(const AddForeignKeyRequest& req); - void recv_add_foreign_key(const int32_t seqid); - void add_unique_constraint(const AddUniqueConstraintRequest& req); - int32_t send_add_unique_constraint(const AddUniqueConstraintRequest& req); - void recv_add_unique_constraint(const int32_t seqid); - void add_not_null_constraint(const AddNotNullConstraintRequest& req); - int32_t send_add_not_null_constraint(const AddNotNullConstraintRequest& req); - void recv_add_not_null_constraint(const int32_t seqid); - void drop_table(const std::string& dbname, const std::string& name, const bool deleteData); - int32_t send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData); - void recv_drop_table(const int32_t seqid); - void drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context); - int32_t send_drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context); - void recv_drop_table_with_environment_context(const int32_t seqid); - void truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames); - int32_t send_truncate_table(const std::string& dbName, const std::string& tableName, const std::vector & partNames); - void recv_truncate_table(const int32_t seqid); - void get_tables(std::vector & _return, const std::string& db_name, const std::string& pattern); - int32_t send_get_tables(const std::string& db_name, const std::string& pattern); - void recv_get_tables(std::vector & _return, const int32_t seqid); - void get_tables_by_type(std::vector & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType); - int32_t send_get_tables_by_type(const std::string& db_name, const std::string& pattern, const std::string& tableType); - void recv_get_tables_by_type(std::vector & _return, const int32_t seqid); - void get_table_meta(std::vector & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types); - int32_t send_get_table_meta(const std::string& db_patterns, const std::string& tbl_patterns, const std::vector & tbl_types); - void recv_get_table_meta(std::vector & _return, const int32_t seqid); - void get_all_tables(std::vector & _return, const std::string& db_name); - int32_t send_get_all_tables(const std::string& db_name); - void recv_get_all_tables(std::vector & _return, const int32_t seqid); - void get_table(Table& _return, const std::string& dbname, const std::string& tbl_name); - int32_t send_get_table(const std::string& dbname, const std::string& tbl_name); - void recv_get_table(Table& _return, const int32_t seqid); - void get_table_objects_by_name(std::vector
& _return, const std::string& dbname, const std::vector & tbl_names); - int32_t send_get_table_objects_by_name(const std::string& dbname, const std::vector & tbl_names); - void recv_get_table_objects_by_name(std::vector
& _return, const int32_t seqid); - void get_table_req(GetTableResult& _return, const GetTableRequest& req); - int32_t send_get_table_req(const GetTableRequest& req); - void recv_get_table_req(GetTableResult& _return, const int32_t seqid); - void get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req); - int32_t send_get_table_objects_by_name_req(const GetTablesRequest& req); - void recv_get_table_objects_by_name_req(GetTablesResult& _return, const int32_t seqid); - void get_table_names_by_filter(std::vector & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables); - int32_t send_get_table_names_by_filter(const std::string& dbname, const std::string& filter, const int16_t max_tables); - void recv_get_table_names_by_filter(std::vector & _return, const int32_t seqid); - void alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl); - int32_t send_alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl); - void recv_alter_table(const int32_t seqid); - void alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context); - int32_t send_alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context); - void recv_alter_table_with_environment_context(const int32_t seqid); - void alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade); - int32_t send_alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade); - void recv_alter_table_with_cascade(const int32_t seqid); - void add_partition(Partition& _return, const Partition& new_part); - int32_t send_add_partition(const Partition& new_part); - void recv_add_partition(Partition& _return, const int32_t seqid); - void add_partition_with_environment_context(Partition& _return, const Partition& new_part, const EnvironmentContext& environment_context); - int32_t send_add_partition_with_environment_context(const Partition& new_part, const EnvironmentContext& environment_context); - void recv_add_partition_with_environment_context(Partition& _return, const int32_t seqid); - int32_t add_partitions(const std::vector & new_parts); - int32_t send_add_partitions(const std::vector & new_parts); - int32_t recv_add_partitions(const int32_t seqid); - int32_t add_partitions_pspec(const std::vector & new_parts); - int32_t send_add_partitions_pspec(const std::vector & new_parts); - int32_t recv_add_partitions_pspec(const int32_t seqid); - void append_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals); - int32_t send_append_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals); - void recv_append_partition(Partition& _return, const int32_t seqid); - void add_partitions_req(AddPartitionsResult& _return, const AddPartitionsRequest& request); - int32_t send_add_partitions_req(const AddPartitionsRequest& request); - void recv_add_partitions_req(AddPartitionsResult& _return, const int32_t seqid); - void append_partition_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context); - int32_t send_append_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const EnvironmentContext& environment_context); - void recv_append_partition_with_environment_context(Partition& _return, const int32_t seqid); - void append_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name); - int32_t send_append_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name); - void recv_append_partition_by_name(Partition& _return, const int32_t seqid); - void append_partition_by_name_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context); - int32_t send_append_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context); - void recv_append_partition_by_name_with_environment_context(Partition& _return, const int32_t seqid); - bool drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData); - int32_t send_drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData); - bool recv_drop_partition(const int32_t seqid); - bool drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context); - int32_t send_drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData, const EnvironmentContext& environment_context); - bool recv_drop_partition_with_environment_context(const int32_t seqid); - bool drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData); - int32_t send_drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData); - bool recv_drop_partition_by_name(const int32_t seqid); - bool drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context); - int32_t send_drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context); - bool recv_drop_partition_by_name_with_environment_context(const int32_t seqid); - void drop_partitions_req(DropPartitionsResult& _return, const DropPartitionsRequest& req); - int32_t send_drop_partitions_req(const DropPartitionsRequest& req); - void recv_drop_partitions_req(DropPartitionsResult& _return, const int32_t seqid); - void get_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals); - int32_t send_get_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals); - void recv_get_partition(Partition& _return, const int32_t seqid); - void exchange_partition(Partition& _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name); - int32_t send_exchange_partition(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name); - void recv_exchange_partition(Partition& _return, const int32_t seqid); - void exchange_partitions(std::vector & _return, const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name); - int32_t send_exchange_partitions(const std::map & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name); - void recv_exchange_partitions(std::vector & _return, const int32_t seqid); - void get_partition_with_auth(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names); - int32_t send_get_partition_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const std::string& user_name, const std::vector & group_names); - void recv_get_partition_with_auth(Partition& _return, const int32_t seqid); - void get_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name); - int32_t send_get_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name); - void recv_get_partition_by_name(Partition& _return, const int32_t seqid); - void get_partitions(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts); - int32_t send_get_partitions(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts); - void recv_get_partitions(std::vector & _return, const int32_t seqid); - void get_partitions_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names); - int32_t send_get_partitions_with_auth(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector & group_names); - void recv_get_partitions_with_auth(std::vector & _return, const int32_t seqid); - void get_partitions_pspec(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int32_t max_parts); - int32_t send_get_partitions_pspec(const std::string& db_name, const std::string& tbl_name, const int32_t max_parts); - void recv_get_partitions_pspec(std::vector & _return, const int32_t seqid); - void get_partition_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts); - int32_t send_get_partition_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts); - void recv_get_partition_names(std::vector & _return, const int32_t seqid); - void get_partition_values(PartitionValuesResponse& _return, const PartitionValuesRequest& request); - int32_t send_get_partition_values(const PartitionValuesRequest& request); - void recv_get_partition_values(PartitionValuesResponse& _return, const int32_t seqid); - void get_partitions_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts); - int32_t send_get_partitions_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts); - void recv_get_partitions_ps(std::vector & _return, const int32_t seqid); - void get_partitions_ps_with_auth(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names); - int32_t send_get_partitions_ps_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector & group_names); - void recv_get_partitions_ps_with_auth(std::vector & _return, const int32_t seqid); - void get_partition_names_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts); - int32_t send_get_partition_names_ps(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const int16_t max_parts); - void recv_get_partition_names_ps(std::vector & _return, const int32_t seqid); - void get_partitions_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts); - int32_t send_get_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts); - void recv_get_partitions_by_filter(std::vector & _return, const int32_t seqid); - void get_part_specs_by_filter(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts); - int32_t send_get_part_specs_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int32_t max_parts); - void recv_get_part_specs_by_filter(std::vector & _return, const int32_t seqid); - void get_partitions_by_expr(PartitionsByExprResult& _return, const PartitionsByExprRequest& req); - int32_t send_get_partitions_by_expr(const PartitionsByExprRequest& req); - void recv_get_partitions_by_expr(PartitionsByExprResult& _return, const int32_t seqid); - int32_t get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter); - int32_t send_get_num_partitions_by_filter(const std::string& db_name, const std::string& tbl_name, const std::string& filter); - int32_t recv_get_num_partitions_by_filter(const int32_t seqid); - void get_partitions_by_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & names); - int32_t send_get_partitions_by_names(const std::string& db_name, const std::string& tbl_name, const std::vector & names); - void recv_get_partitions_by_names(std::vector & _return, const int32_t seqid); - void alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part); - int32_t send_alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part); - void recv_alter_partition(const int32_t seqid); - void alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts); - int32_t send_alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts); - void recv_alter_partitions(const int32_t seqid); - void alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context); - int32_t send_alter_partitions_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts, const EnvironmentContext& environment_context); - void recv_alter_partitions_with_environment_context(const int32_t seqid); - void alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context); - int32_t send_alter_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const Partition& new_part, const EnvironmentContext& environment_context); - void recv_alter_partition_with_environment_context(const int32_t seqid); - void rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part); - int32_t send_rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part); - void recv_rename_partition(const int32_t seqid); - bool partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception); - int32_t send_partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception); - bool recv_partition_name_has_valid_characters(const int32_t seqid); - void get_config_value(std::string& _return, const std::string& name, const std::string& defaultValue); - int32_t send_get_config_value(const std::string& name, const std::string& defaultValue); - void recv_get_config_value(std::string& _return, const int32_t seqid); - void partition_name_to_vals(std::vector & _return, const std::string& part_name); - int32_t send_partition_name_to_vals(const std::string& part_name); - void recv_partition_name_to_vals(std::vector & _return, const int32_t seqid); - void partition_name_to_spec(std::map & _return, const std::string& part_name); - int32_t send_partition_name_to_spec(const std::string& part_name); - void recv_partition_name_to_spec(std::map & _return, const int32_t seqid); - void markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType); - int32_t send_markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType); - void recv_markPartitionForEvent(const int32_t seqid); - bool isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType); - int32_t send_isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType); - bool recv_isPartitionMarkedForEvent(const int32_t seqid); - void add_index(Index& _return, const Index& new_index, const Table& index_table); - int32_t send_add_index(const Index& new_index, const Table& index_table); - void recv_add_index(Index& _return, const int32_t seqid); - void alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx); - int32_t send_alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx); - void recv_alter_index(const int32_t seqid); - bool drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData); - int32_t send_drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData); - bool recv_drop_index_by_name(const int32_t seqid); - void get_index_by_name(Index& _return, const std::string& db_name, const std::string& tbl_name, const std::string& index_name); - int32_t send_get_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name); - void recv_get_index_by_name(Index& _return, const int32_t seqid); - void get_indexes(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes); - int32_t send_get_indexes(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes); - void recv_get_indexes(std::vector & _return, const int32_t seqid); - void get_index_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes); - int32_t send_get_index_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes); - void recv_get_index_names(std::vector & _return, const int32_t seqid); - void get_primary_keys(PrimaryKeysResponse& _return, const PrimaryKeysRequest& request); - int32_t send_get_primary_keys(const PrimaryKeysRequest& request); - void recv_get_primary_keys(PrimaryKeysResponse& _return, const int32_t seqid); - void get_foreign_keys(ForeignKeysResponse& _return, const ForeignKeysRequest& request); - int32_t send_get_foreign_keys(const ForeignKeysRequest& request); - void recv_get_foreign_keys(ForeignKeysResponse& _return, const int32_t seqid); - void get_unique_constraints(UniqueConstraintsResponse& _return, const UniqueConstraintsRequest& request); - int32_t send_get_unique_constraints(const UniqueConstraintsRequest& request); - void recv_get_unique_constraints(UniqueConstraintsResponse& _return, const int32_t seqid); - void get_not_null_constraints(NotNullConstraintsResponse& _return, const NotNullConstraintsRequest& request); - int32_t send_get_not_null_constraints(const NotNullConstraintsRequest& request); - void recv_get_not_null_constraints(NotNullConstraintsResponse& _return, const int32_t seqid); - bool update_table_column_statistics(const ColumnStatistics& stats_obj); - int32_t send_update_table_column_statistics(const ColumnStatistics& stats_obj); - bool recv_update_table_column_statistics(const int32_t seqid); - bool update_partition_column_statistics(const ColumnStatistics& stats_obj); - int32_t send_update_partition_column_statistics(const ColumnStatistics& stats_obj); - bool recv_update_partition_column_statistics(const int32_t seqid); - void get_table_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& col_name); - int32_t send_get_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name); - void recv_get_table_column_statistics(ColumnStatistics& _return, const int32_t seqid); - void get_partition_column_statistics(ColumnStatistics& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name); - int32_t send_get_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name); - void recv_get_partition_column_statistics(ColumnStatistics& _return, const int32_t seqid); - void get_table_statistics_req(TableStatsResult& _return, const TableStatsRequest& request); - int32_t send_get_table_statistics_req(const TableStatsRequest& request); - void recv_get_table_statistics_req(TableStatsResult& _return, const int32_t seqid); - void get_partitions_statistics_req(PartitionsStatsResult& _return, const PartitionsStatsRequest& request); - int32_t send_get_partitions_statistics_req(const PartitionsStatsRequest& request); - void recv_get_partitions_statistics_req(PartitionsStatsResult& _return, const int32_t seqid); - void get_aggr_stats_for(AggrStats& _return, const PartitionsStatsRequest& request); - int32_t send_get_aggr_stats_for(const PartitionsStatsRequest& request); - void recv_get_aggr_stats_for(AggrStats& _return, const int32_t seqid); - bool set_aggr_stats_for(const SetPartitionsStatsRequest& request); - int32_t send_set_aggr_stats_for(const SetPartitionsStatsRequest& request); - bool recv_set_aggr_stats_for(const int32_t seqid); - bool delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name); - int32_t send_delete_partition_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const std::string& col_name); - bool recv_delete_partition_column_statistics(const int32_t seqid); - bool delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name); - int32_t send_delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name); - bool recv_delete_table_column_statistics(const int32_t seqid); - void create_function(const Function& func); - int32_t send_create_function(const Function& func); - void recv_create_function(const int32_t seqid); - void drop_function(const std::string& dbName, const std::string& funcName); - int32_t send_drop_function(const std::string& dbName, const std::string& funcName); - void recv_drop_function(const int32_t seqid); - void alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc); - int32_t send_alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc); - void recv_alter_function(const int32_t seqid); - void get_functions(std::vector & _return, const std::string& dbName, const std::string& pattern); - int32_t send_get_functions(const std::string& dbName, const std::string& pattern); - void recv_get_functions(std::vector & _return, const int32_t seqid); - void get_function(Function& _return, const std::string& dbName, const std::string& funcName); - int32_t send_get_function(const std::string& dbName, const std::string& funcName); - void recv_get_function(Function& _return, const int32_t seqid); - void get_all_functions(GetAllFunctionsResponse& _return); - int32_t send_get_all_functions(); - void recv_get_all_functions(GetAllFunctionsResponse& _return, const int32_t seqid); - bool create_role(const Role& role); - int32_t send_create_role(const Role& role); - bool recv_create_role(const int32_t seqid); - bool drop_role(const std::string& role_name); - int32_t send_drop_role(const std::string& role_name); - bool recv_drop_role(const int32_t seqid); - void get_role_names(std::vector & _return); - int32_t send_get_role_names(); - void recv_get_role_names(std::vector & _return, const int32_t seqid); - bool grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option); - int32_t send_grant_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type, const std::string& grantor, const PrincipalType::type grantorType, const bool grant_option); - bool recv_grant_role(const int32_t seqid); - bool revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type); - int32_t send_revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type); - bool recv_revoke_role(const int32_t seqid); - void list_roles(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type); - int32_t send_list_roles(const std::string& principal_name, const PrincipalType::type principal_type); - void recv_list_roles(std::vector & _return, const int32_t seqid); - void grant_revoke_role(GrantRevokeRoleResponse& _return, const GrantRevokeRoleRequest& request); - int32_t send_grant_revoke_role(const GrantRevokeRoleRequest& request); - void recv_grant_revoke_role(GrantRevokeRoleResponse& _return, const int32_t seqid); - void get_principals_in_role(GetPrincipalsInRoleResponse& _return, const GetPrincipalsInRoleRequest& request); - int32_t send_get_principals_in_role(const GetPrincipalsInRoleRequest& request); - void recv_get_principals_in_role(GetPrincipalsInRoleResponse& _return, const int32_t seqid); - void get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return, const GetRoleGrantsForPrincipalRequest& request); - int32_t send_get_role_grants_for_principal(const GetRoleGrantsForPrincipalRequest& request); - void recv_get_role_grants_for_principal(GetRoleGrantsForPrincipalResponse& _return, const int32_t seqid); - void get_privilege_set(PrincipalPrivilegeSet& _return, const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names); - int32_t send_get_privilege_set(const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names); - void recv_get_privilege_set(PrincipalPrivilegeSet& _return, const int32_t seqid); - void list_privileges(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject); - int32_t send_list_privileges(const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject); - void recv_list_privileges(std::vector & _return, const int32_t seqid); - bool grant_privileges(const PrivilegeBag& privileges); - int32_t send_grant_privileges(const PrivilegeBag& privileges); - bool recv_grant_privileges(const int32_t seqid); - bool revoke_privileges(const PrivilegeBag& privileges); - int32_t send_revoke_privileges(const PrivilegeBag& privileges); - bool recv_revoke_privileges(const int32_t seqid); - void grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const GrantRevokePrivilegeRequest& request); - int32_t send_grant_revoke_privileges(const GrantRevokePrivilegeRequest& request); - void recv_grant_revoke_privileges(GrantRevokePrivilegeResponse& _return, const int32_t seqid); - void set_ugi(std::vector & _return, const std::string& user_name, const std::vector & group_names); - int32_t send_set_ugi(const std::string& user_name, const std::vector & group_names); - void recv_set_ugi(std::vector & _return, const int32_t seqid); - void get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name); - int32_t send_get_delegation_token(const std::string& token_owner, const std::string& renewer_kerberos_principal_name); - void recv_get_delegation_token(std::string& _return, const int32_t seqid); - int64_t renew_delegation_token(const std::string& token_str_form); - int32_t send_renew_delegation_token(const std::string& token_str_form); - int64_t recv_renew_delegation_token(const int32_t seqid); - void cancel_delegation_token(const std::string& token_str_form); - int32_t send_cancel_delegation_token(const std::string& token_str_form); - void recv_cancel_delegation_token(const int32_t seqid); - bool add_token(const std::string& token_identifier, const std::string& delegation_token); - int32_t send_add_token(const std::string& token_identifier, const std::string& delegation_token); - bool recv_add_token(const int32_t seqid); - bool remove_token(const std::string& token_identifier); - int32_t send_remove_token(const std::string& token_identifier); - bool recv_remove_token(const int32_t seqid); - void get_token(std::string& _return, const std::string& token_identifier); - int32_t send_get_token(const std::string& token_identifier); - void recv_get_token(std::string& _return, const int32_t seqid); - void get_all_token_identifiers(std::vector & _return); - int32_t send_get_all_token_identifiers(); - void recv_get_all_token_identifiers(std::vector & _return, const int32_t seqid); - int32_t add_master_key(const std::string& key); - int32_t send_add_master_key(const std::string& key); - int32_t recv_add_master_key(const int32_t seqid); - void update_master_key(const int32_t seq_number, const std::string& key); - int32_t send_update_master_key(const int32_t seq_number, const std::string& key); - void recv_update_master_key(const int32_t seqid); - bool remove_master_key(const int32_t key_seq); - int32_t send_remove_master_key(const int32_t key_seq); - bool recv_remove_master_key(const int32_t seqid); - void get_master_keys(std::vector & _return); - int32_t send_get_master_keys(); - void recv_get_master_keys(std::vector & _return, const int32_t seqid); - void get_open_txns(GetOpenTxnsResponse& _return); - int32_t send_get_open_txns(); - void recv_get_open_txns(GetOpenTxnsResponse& _return, const int32_t seqid); - void get_open_txns_info(GetOpenTxnsInfoResponse& _return); - int32_t send_get_open_txns_info(); - void recv_get_open_txns_info(GetOpenTxnsInfoResponse& _return, const int32_t seqid); - void open_txns(OpenTxnsResponse& _return, const OpenTxnRequest& rqst); - int32_t send_open_txns(const OpenTxnRequest& rqst); - void recv_open_txns(OpenTxnsResponse& _return, const int32_t seqid); - void abort_txn(const AbortTxnRequest& rqst); - int32_t send_abort_txn(const AbortTxnRequest& rqst); - void recv_abort_txn(const int32_t seqid); - void abort_txns(const AbortTxnsRequest& rqst); - int32_t send_abort_txns(const AbortTxnsRequest& rqst); - void recv_abort_txns(const int32_t seqid); - void commit_txn(const CommitTxnRequest& rqst); - int32_t send_commit_txn(const CommitTxnRequest& rqst); - void recv_commit_txn(const int32_t seqid); - void lock(LockResponse& _return, const LockRequest& rqst); - int32_t send_lock(const LockRequest& rqst); - void recv_lock(LockResponse& _return, const int32_t seqid); - void check_lock(LockResponse& _return, const CheckLockRequest& rqst); - int32_t send_check_lock(const CheckLockRequest& rqst); - void recv_check_lock(LockResponse& _return, const int32_t seqid); - void unlock(const UnlockRequest& rqst); - int32_t send_unlock(const UnlockRequest& rqst); - void recv_unlock(const int32_t seqid); - void show_locks(ShowLocksResponse& _return, const ShowLocksRequest& rqst); - int32_t send_show_locks(const ShowLocksRequest& rqst); - void recv_show_locks(ShowLocksResponse& _return, const int32_t seqid); - void heartbeat(const HeartbeatRequest& ids); - int32_t send_heartbeat(const HeartbeatRequest& ids); - void recv_heartbeat(const int32_t seqid); - void heartbeat_txn_range(HeartbeatTxnRangeResponse& _return, const HeartbeatTxnRangeRequest& txns); - int32_t send_heartbeat_txn_range(const HeartbeatTxnRangeRequest& txns); - void recv_heartbeat_txn_range(HeartbeatTxnRangeResponse& _return, const int32_t seqid); - void compact(const CompactionRequest& rqst); - int32_t send_compact(const CompactionRequest& rqst); - void recv_compact(const int32_t seqid); - void compact2(CompactionResponse& _return, const CompactionRequest& rqst); - int32_t send_compact2(const CompactionRequest& rqst); - void recv_compact2(CompactionResponse& _return, const int32_t seqid); - void show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst); - int32_t send_show_compact(const ShowCompactRequest& rqst); - void recv_show_compact(ShowCompactResponse& _return, const int32_t seqid); - void add_dynamic_partitions(const AddDynamicPartitions& rqst); - int32_t send_add_dynamic_partitions(const AddDynamicPartitions& rqst); - void recv_add_dynamic_partitions(const int32_t seqid); - void get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst); - int32_t send_get_next_notification(const NotificationEventRequest& rqst); - void recv_get_next_notification(NotificationEventResponse& _return, const int32_t seqid); - void get_current_notificationEventId(CurrentNotificationEventId& _return); - int32_t send_get_current_notificationEventId(); - void recv_get_current_notificationEventId(CurrentNotificationEventId& _return, const int32_t seqid); - void get_notification_events_count(NotificationEventsCountResponse& _return, const NotificationEventsCountRequest& rqst); - int32_t send_get_notification_events_count(const NotificationEventsCountRequest& rqst); - void recv_get_notification_events_count(NotificationEventsCountResponse& _return, const int32_t seqid); - void fire_listener_event(FireEventResponse& _return, const FireEventRequest& rqst); - int32_t send_fire_listener_event(const FireEventRequest& rqst); - void recv_fire_listener_event(FireEventResponse& _return, const int32_t seqid); - void flushCache(); - int32_t send_flushCache(); - void recv_flushCache(const int32_t seqid); - void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request); - int32_t send_cm_recycle(const CmRecycleRequest& request); - void recv_cm_recycle(CmRecycleResponse& _return, const int32_t seqid); - void get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req); - int32_t send_get_file_metadata_by_expr(const GetFileMetadataByExprRequest& req); - void recv_get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const int32_t seqid); - void get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req); - int32_t send_get_file_metadata(const GetFileMetadataRequest& req); - void recv_get_file_metadata(GetFileMetadataResult& _return, const int32_t seqid); - void put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req); - int32_t send_put_file_metadata(const PutFileMetadataRequest& req); - void recv_put_file_metadata(PutFileMetadataResult& _return, const int32_t seqid); - void clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req); - int32_t send_clear_file_metadata(const ClearFileMetadataRequest& req); - void recv_clear_file_metadata(ClearFileMetadataResult& _return, const int32_t seqid); - void cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req); - int32_t send_cache_file_metadata(const CacheFileMetadataRequest& req); - void recv_cache_file_metadata(CacheFileMetadataResult& _return, const int32_t seqid); - void get_metastore_db_uuid(std::string& _return); - int32_t send_get_metastore_db_uuid(); - void recv_get_metastore_db_uuid(std::string& _return, const int32_t seqid); - void create_resource_plan(WMCreateResourcePlanResponse& _return, const WMCreateResourcePlanRequest& request); - int32_t send_create_resource_plan(const WMCreateResourcePlanRequest& request); - void recv_create_resource_plan(WMCreateResourcePlanResponse& _return, const int32_t seqid); - void get_resource_plan(WMGetResourcePlanResponse& _return, const WMGetResourcePlanRequest& request); - int32_t send_get_resource_plan(const WMGetResourcePlanRequest& request); - void recv_get_resource_plan(WMGetResourcePlanResponse& _return, const int32_t seqid); - void get_active_resource_plan(WMGetActiveResourcePlanResponse& _return, const WMGetActiveResourcePlanRequest& request); - int32_t send_get_active_resource_plan(const WMGetActiveResourcePlanRequest& request); - void recv_get_active_resource_plan(WMGetActiveResourcePlanResponse& _return, const int32_t seqid); - void get_all_resource_plans(WMGetAllResourcePlanResponse& _return, const WMGetAllResourcePlanRequest& request); - int32_t send_get_all_resource_plans(const WMGetAllResourcePlanRequest& request); - void recv_get_all_resource_plans(WMGetAllResourcePlanResponse& _return, const int32_t seqid); - void alter_resource_plan(WMAlterResourcePlanResponse& _return, const WMAlterResourcePlanRequest& request); - int32_t send_alter_resource_plan(const WMAlterResourcePlanRequest& request); - void recv_alter_resource_plan(WMAlterResourcePlanResponse& _return, const int32_t seqid); - void validate_resource_plan(WMValidateResourcePlanResponse& _return, const WMValidateResourcePlanRequest& request); - int32_t send_validate_resource_plan(const WMValidateResourcePlanRequest& request); - void recv_validate_resource_plan(WMValidateResourcePlanResponse& _return, const int32_t seqid); - void drop_resource_plan(WMDropResourcePlanResponse& _return, const WMDropResourcePlanRequest& request); - int32_t send_drop_resource_plan(const WMDropResourcePlanRequest& request); - void recv_drop_resource_plan(WMDropResourcePlanResponse& _return, const int32_t seqid); - void create_wm_trigger(WMCreateTriggerResponse& _return, const WMCreateTriggerRequest& request); - int32_t send_create_wm_trigger(const WMCreateTriggerRequest& request); - void recv_create_wm_trigger(WMCreateTriggerResponse& _return, const int32_t seqid); - void alter_wm_trigger(WMAlterTriggerResponse& _return, const WMAlterTriggerRequest& request); - int32_t send_alter_wm_trigger(const WMAlterTriggerRequest& request); - void recv_alter_wm_trigger(WMAlterTriggerResponse& _return, const int32_t seqid); - void drop_wm_trigger(WMDropTriggerResponse& _return, const WMDropTriggerRequest& request); - int32_t send_drop_wm_trigger(const WMDropTriggerRequest& request); - void recv_drop_wm_trigger(WMDropTriggerResponse& _return, const int32_t seqid); - void get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return, const WMGetTriggersForResourePlanRequest& request); - int32_t send_get_triggers_for_resourceplan(const WMGetTriggersForResourePlanRequest& request); - void recv_get_triggers_for_resourceplan(WMGetTriggersForResourePlanResponse& _return, const int32_t seqid); - protected: - std::shared_ptr< ::apache::thrift::protocol::TProtocol> piprot_; - std::shared_ptr< ::apache::thrift::protocol::TProtocol> poprot_; - ::apache::thrift::protocol::TProtocol* iprot_; - ::apache::thrift::protocol::TProtocol* oprot_; - ::apache::thrift::async::TConcurrentClientSyncInfo sync_; -}; - -#ifdef _MSC_VER - #pragma warning( pop ) -#endif - -}}} // namespace - -#endif diff --git a/contrib/hive-metastore/hive_metastore.thrift b/contrib/hive-metastore/hive_metastore.thrift deleted file mode 100644 index 60d91b56492..00000000000 --- a/contrib/hive-metastore/hive_metastore.thrift +++ /dev/null @@ -1,1807 +0,0 @@ -#!/usr/local/bin/thrift -java - -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -# -# Thrift Service that the MetaStore is built on -# - -namespace java org.apache.hadoop.hive.metastore.api -namespace php metastore -namespace cpp Apache.Hadoop.Hive - -const string DDL_TIME = "transient_lastDdlTime" - -struct Version { - 1: string version, - 2: string comments -} - -struct FieldSchema { - 1: string name, // name of the field - 2: string type, // type of the field. primitive types defined above, specify list, map for lists & maps - 3: string comment -} - -struct SQLPrimaryKey { - 1: string table_db, // table schema - 2: string table_name, // table name - 3: string column_name, // column name - 4: i32 key_seq, // sequence number within primary key - 5: string pk_name, // primary key name - 6: bool enable_cstr, // Enable/Disable - 7: bool validate_cstr, // Validate/No validate - 8: bool rely_cstr // Rely/No Rely -} - -struct SQLForeignKey { - 1: string pktable_db, // primary key table schema - 2: string pktable_name, // primary key table name - 3: string pkcolumn_name, // primary key column name - 4: string fktable_db, // foreign key table schema - 5: string fktable_name, // foreign key table name - 6: string fkcolumn_name, // foreign key column name - 7: i32 key_seq, // sequence within foreign key - 8: i32 update_rule, // what happens to foreign key when parent key is updated - 9: i32 delete_rule, // what happens to foreign key when parent key is deleted - 10: string fk_name, // foreign key name - 11: string pk_name, // primary key name - 12: bool enable_cstr, // Enable/Disable - 13: bool validate_cstr, // Validate/No validate - 14: bool rely_cstr // Rely/No Rely -} - -struct SQLUniqueConstraint { - 1: string table_db, // table schema - 2: string table_name, // table name - 3: string column_name, // column name - 4: i32 key_seq, // sequence number within unique constraint - 5: string uk_name, // unique key name - 6: bool enable_cstr, // Enable/Disable - 7: bool validate_cstr, // Validate/No validate - 8: bool rely_cstr // Rely/No Rely -} - -struct SQLNotNullConstraint { - 1: string table_db, // table schema - 2: string table_name, // table name - 3: string column_name, // column name - 4: string nn_name, // not null name - 5: bool enable_cstr, // Enable/Disable - 6: bool validate_cstr, // Validate/No validate - 7: bool rely_cstr // Rely/No Rely -} - -struct Type { - 1: string name, // one of the types in PrimitiveTypes or CollectionTypes or User defined types - 2: optional string type1, // object type if the name is 'list' (LIST_TYPE), key type if the name is 'map' (MAP_TYPE) - 3: optional string type2, // val type if the name is 'map' (MAP_TYPE) - 4: optional list fields // if the name is one of the user defined types -} - -enum HiveObjectType { - GLOBAL = 1, - DATABASE = 2, - TABLE = 3, - PARTITION = 4, - COLUMN = 5, -} - -enum PrincipalType { - USER = 1, - ROLE = 2, - GROUP = 3, -} - -const string HIVE_FILTER_FIELD_OWNER = "hive_filter_field_owner__" -const string HIVE_FILTER_FIELD_PARAMS = "hive_filter_field_params__" -const string HIVE_FILTER_FIELD_LAST_ACCESS = "hive_filter_field_last_access__" - -enum PartitionEventType { - LOAD_DONE = 1, -} - -// Enums for transaction and lock management -enum TxnState { - COMMITTED = 1, - ABORTED = 2, - OPEN = 3, -} - -enum LockLevel { - DB = 1, - TABLE = 2, - PARTITION = 3, -} - -enum LockState { - ACQUIRED = 1, // requester has the lock - WAITING = 2, // requester is waiting for the lock and should call checklock at a later point to see if the lock has been obtained. - ABORT = 3, // the lock has been aborted, most likely due to timeout - NOT_ACQUIRED = 4, // returned only with lockNoWait, indicates the lock was not available and was not acquired -} - -enum LockType { - SHARED_READ = 1, - SHARED_WRITE = 2, - EXCLUSIVE = 3, -} - -enum CompactionType { - MINOR = 1, - MAJOR = 2, -} - -enum GrantRevokeType { - GRANT = 1, - REVOKE = 2, -} - -enum DataOperationType { - SELECT = 1, - INSERT = 2 - UPDATE = 3, - DELETE = 4, - UNSET = 5,//this is the default to distinguish from NULL from old clients - NO_TXN = 6,//drop table, insert overwrite, etc - something non-transactional -} - -// Types of events the client can request that the metastore fire. For now just support DML operations, as the metastore knows -// about DDL operations and there's no reason for the client to request such an event. -enum EventRequestType { - INSERT = 1, - UPDATE = 2, - DELETE = 3, -} - -struct HiveObjectRef{ - 1: HiveObjectType objectType, - 2: string dbName, - 3: string objectName, - 4: list partValues, - 5: string columnName, -} - -struct PrivilegeGrantInfo { - 1: string privilege, - 2: i32 createTime, - 3: string grantor, - 4: PrincipalType grantorType, - 5: bool grantOption, -} - -struct HiveObjectPrivilege { - 1: HiveObjectRef hiveObject, - 2: string principalName, - 3: PrincipalType principalType, - 4: PrivilegeGrantInfo grantInfo, -} - -struct PrivilegeBag { - 1: list privileges, -} - -struct PrincipalPrivilegeSet { - 1: map> userPrivileges, // user name -> privilege grant info - 2: map> groupPrivileges, // group name -> privilege grant info - 3: map> rolePrivileges, //role name -> privilege grant info -} - -struct GrantRevokePrivilegeRequest { - 1: GrantRevokeType requestType; - 2: PrivilegeBag privileges; - 3: optional bool revokeGrantOption; // Only for revoke request -} - -struct GrantRevokePrivilegeResponse { - 1: optional bool success; -} - -struct Role { - 1: string roleName, - 2: i32 createTime, - 3: string ownerName, -} - -// Representation of a grant for a principal to a role -struct RolePrincipalGrant { - 1: string roleName, - 2: string principalName, - 3: PrincipalType principalType, - 4: bool grantOption, - 5: i32 grantTime, - 6: string grantorName, - 7: PrincipalType grantorPrincipalType -} - -struct GetRoleGrantsForPrincipalRequest { - 1: required string principal_name, - 2: required PrincipalType principal_type -} - -struct GetRoleGrantsForPrincipalResponse { - 1: required list principalGrants; -} - -struct GetPrincipalsInRoleRequest { - 1: required string roleName; -} - -struct GetPrincipalsInRoleResponse { - 1: required list principalGrants; -} - -struct GrantRevokeRoleRequest { - 1: GrantRevokeType requestType; - 2: string roleName; - 3: string principalName; - 4: PrincipalType principalType; - 5: optional string grantor; // Needed for grant - 6: optional PrincipalType grantorType; // Needed for grant - 7: optional bool grantOption; -} - -struct GrantRevokeRoleResponse { - 1: optional bool success; -} - -// namespace for tables -struct Database { - 1: string name, - 2: string description, - 3: string locationUri, - 4: map parameters, // properties associated with the database - 5: optional PrincipalPrivilegeSet privileges, - 6: optional string ownerName, - 7: optional PrincipalType ownerType -} - -// This object holds the information needed by SerDes -struct SerDeInfo { - 1: string name, // name of the serde, table name by default - 2: string serializationLib, // usually the class that implements the extractor & loader - 3: map parameters // initialization parameters -} - -// sort order of a column (column name along with asc(1)/desc(0)) -struct Order { - 1: string col, // sort column name - 2: i32 order // asc(1) or desc(0) -} - -// this object holds all the information about skewed table -struct SkewedInfo { - 1: list skewedColNames, // skewed column names - 2: list> skewedColValues, //skewed values - 3: map skewedColValueLocationMaps, //skewed value to location mappings -} - -// this object holds all the information about physical storage of the data belonging to a table -struct StorageDescriptor { - 1: list cols, // required (refer to types defined above) - 2: string location, // defaults to //tablename - 3: string inputFormat, // SequenceFileInputFormat (binary) or TextInputFormat` or custom format - 4: string outputFormat, // SequenceFileOutputFormat (binary) or IgnoreKeyTextOutputFormat or custom format - 5: bool compressed, // compressed or not - 6: i32 numBuckets, // this must be specified if there are any dimension columns - 7: SerDeInfo serdeInfo, // serialization and deserialization information - 8: list bucketCols, // reducer grouping columns and clustering columns and bucketing columns` - 9: list sortCols, // sort order of the data in each bucket - 10: map parameters, // any user supplied key value hash - 11: optional SkewedInfo skewedInfo, // skewed information - 12: optional bool storedAsSubDirectories // stored as subdirectories or not -} - -// table information -struct Table { - 1: string tableName, // name of the table - 2: string dbName, // database name ('default') - 3: string owner, // owner of this table - 4: i32 createTime, // creation time of the table - 5: i32 lastAccessTime, // last access time (usually this will be filled from HDFS and shouldn't be relied on) - 6: i32 retention, // retention time - 7: StorageDescriptor sd, // storage descriptor of the table - 8: list partitionKeys, // partition keys of the table. only primitive types are supported - 9: map parameters, // to store comments or any other user level parameters - 10: string viewOriginalText, // original view text, null for non-view - 11: string viewExpandedText, // expanded view text, null for non-view - 12: string tableType, // table type enum, e.g. EXTERNAL_TABLE - 13: optional PrincipalPrivilegeSet privileges, - 14: optional bool temporary=false, - 15: optional bool rewriteEnabled, // rewrite enabled or not -} - -struct Partition { - 1: list values // string value is converted to appropriate partition key type - 2: string dbName, - 3: string tableName, - 4: i32 createTime, - 5: i32 lastAccessTime, - 6: StorageDescriptor sd, - 7: map parameters, - 8: optional PrincipalPrivilegeSet privileges -} - -struct PartitionWithoutSD { - 1: list values // string value is converted to appropriate partition key type - 2: i32 createTime, - 3: i32 lastAccessTime, - 4: string relativePath, - 5: map parameters, - 6: optional PrincipalPrivilegeSet privileges -} - -struct PartitionSpecWithSharedSD { - 1: list partitions, - 2: StorageDescriptor sd, -} - -struct PartitionListComposingSpec { - 1: list partitions -} - -struct PartitionSpec { - 1: string dbName, - 2: string tableName, - 3: string rootPath, - 4: optional PartitionSpecWithSharedSD sharedSDPartitionSpec, - 5: optional PartitionListComposingSpec partitionList -} - -struct Index { - 1: string indexName, // unique with in the whole database namespace - 2: string indexHandlerClass, // reserved - 3: string dbName, - 4: string origTableName, - 5: i32 createTime, - 6: i32 lastAccessTime, - 7: string indexTableName, - 8: StorageDescriptor sd, - 9: map parameters, - 10: bool deferredRebuild -} - -// column statistics -struct BooleanColumnStatsData { -1: required i64 numTrues, -2: required i64 numFalses, -3: required i64 numNulls, -4: optional binary bitVectors -} - -struct DoubleColumnStatsData { -1: optional double lowValue, -2: optional double highValue, -3: required i64 numNulls, -4: required i64 numDVs, -5: optional binary bitVectors -} - -struct LongColumnStatsData { -1: optional i64 lowValue, -2: optional i64 highValue, -3: required i64 numNulls, -4: required i64 numDVs, -5: optional binary bitVectors -} - -struct StringColumnStatsData { -1: required i64 maxColLen, -2: required double avgColLen, -3: required i64 numNulls, -4: required i64 numDVs, -5: optional binary bitVectors -} - -struct BinaryColumnStatsData { -1: required i64 maxColLen, -2: required double avgColLen, -3: required i64 numNulls, -4: optional binary bitVectors -} - - -struct Decimal { -1: required binary unscaled, -3: required i16 scale -} - -struct DecimalColumnStatsData { -1: optional Decimal lowValue, -2: optional Decimal highValue, -3: required i64 numNulls, -4: required i64 numDVs, -5: optional binary bitVectors -} - -struct Date { -1: required i64 daysSinceEpoch -} - -struct DateColumnStatsData { -1: optional Date lowValue, -2: optional Date highValue, -3: required i64 numNulls, -4: required i64 numDVs, -5: optional binary bitVectors -} - -union ColumnStatisticsData { -1: BooleanColumnStatsData booleanStats, -2: LongColumnStatsData longStats, -3: DoubleColumnStatsData doubleStats, -4: StringColumnStatsData stringStats, -5: BinaryColumnStatsData binaryStats, -6: DecimalColumnStatsData decimalStats, -7: DateColumnStatsData dateStats -} - -struct ColumnStatisticsObj { -1: required string colName, -2: required string colType, -3: required ColumnStatisticsData statsData -} - -struct ColumnStatisticsDesc { -1: required bool isTblLevel, -2: required string dbName, -3: required string tableName, -4: optional string partName, -5: optional i64 lastAnalyzed -} - -struct ColumnStatistics { -1: required ColumnStatisticsDesc statsDesc, -2: required list statsObj; -} - -struct AggrStats { -1: required list colStats, -2: required i64 partsFound // number of partitions for which stats were found -} - -struct SetPartitionsStatsRequest { -1: required list colStats, -2: optional bool needMerge //stats need to be merged with the existing stats -} - -// schema of the table/query results etc. -struct Schema { - // column names, types, comments - 1: list fieldSchemas, // delimiters etc - 2: map properties -} - -// Key-value store to be used with selected -// Metastore APIs (create, alter methods). -// The client can pass environment properties / configs that can be -// accessed in hooks. -struct EnvironmentContext { - 1: map properties -} - -struct PrimaryKeysRequest { - 1: required string db_name, - 2: required string tbl_name -} - -struct PrimaryKeysResponse { - 1: required list primaryKeys -} - -struct ForeignKeysRequest { - 1: string parent_db_name, - 2: string parent_tbl_name, - 3: string foreign_db_name, - 4: string foreign_tbl_name -} - -struct ForeignKeysResponse { - 1: required list foreignKeys -} - -struct UniqueConstraintsRequest { - 1: required string db_name, - 2: required string tbl_name -} - -struct UniqueConstraintsResponse { - 1: required list uniqueConstraints -} - -struct NotNullConstraintsRequest { - 1: required string db_name, - 2: required string tbl_name -} - -struct NotNullConstraintsResponse { - 1: required list notNullConstraints -} - -struct DropConstraintRequest { - 1: required string dbname, - 2: required string tablename, - 3: required string constraintname -} - -struct AddPrimaryKeyRequest { - 1: required list primaryKeyCols -} - -struct AddForeignKeyRequest { - 1: required list foreignKeyCols -} - -struct AddUniqueConstraintRequest { - 1: required list uniqueConstraintCols -} - -struct AddNotNullConstraintRequest { - 1: required list notNullConstraintCols -} - -// Return type for get_partitions_by_expr -struct PartitionsByExprResult { - 1: required list partitions, - // Whether the results has any (currently, all) partitions which may or may not match - 2: required bool hasUnknownPartitions -} - -struct PartitionsByExprRequest { - 1: required string dbName, - 2: required string tblName, - 3: required binary expr, - 4: optional string defaultPartitionName, - 5: optional i16 maxParts=-1 -} - -struct TableStatsResult { - 1: required list tableStats -} - -struct PartitionsStatsResult { - 1: required map> partStats -} - -struct TableStatsRequest { - 1: required string dbName, - 2: required string tblName, - 3: required list colNames -} - -struct PartitionsStatsRequest { - 1: required string dbName, - 2: required string tblName, - 3: required list colNames, - 4: required list partNames -} - -// Return type for add_partitions_req -struct AddPartitionsResult { - 1: optional list partitions, -} - -// Request type for add_partitions_req -struct AddPartitionsRequest { - 1: required string dbName, - 2: required string tblName, - 3: required list parts, - 4: required bool ifNotExists, - 5: optional bool needResult=true -} - -// Return type for drop_partitions_req -struct DropPartitionsResult { - 1: optional list partitions, -} - -struct DropPartitionsExpr { - 1: required binary expr; - 2: optional i32 partArchiveLevel; -} - -union RequestPartsSpec { - 1: list names; - 2: list exprs; -} - -// Request type for drop_partitions_req -// TODO: we might want to add "bestEffort" flag; where a subset can fail -struct DropPartitionsRequest { - 1: required string dbName, - 2: required string tblName, - 3: required RequestPartsSpec parts, - 4: optional bool deleteData, - 5: optional bool ifExists=true, // currently verified on client - 6: optional bool ignoreProtection, - 7: optional EnvironmentContext environmentContext, - 8: optional bool needResult=true -} - -struct PartitionValuesRequest { - 1: required string dbName, - 2: required string tblName, - 3: required list partitionKeys; - 4: optional bool applyDistinct = true; - 5: optional string filter; - 6: optional list partitionOrder; - 7: optional bool ascending = true; - 8: optional i64 maxParts = -1; -} - -struct PartitionValuesRow { - 1: required list row; -} - -struct PartitionValuesResponse { - 1: required list partitionValues; -} - -enum FunctionType { - JAVA = 1, -} - -enum ResourceType { - JAR = 1, - FILE = 2, - ARCHIVE = 3, -} - -struct ResourceUri { - 1: ResourceType resourceType, - 2: string uri, -} - -// User-defined function -struct Function { - 1: string functionName, - 2: string dbName, - 3: string className, - 4: string ownerName, - 5: PrincipalType ownerType, - 6: i32 createTime, - 7: FunctionType functionType, - 8: list resourceUris, -} - -// Structs for transaction and locks -struct TxnInfo { - 1: required i64 id, - 2: required TxnState state, - 3: required string user, // used in 'show transactions' to help admins find who has open transactions - 4: required string hostname, // used in 'show transactions' to help admins find who has open transactions - 5: optional string agentInfo = "Unknown", - 6: optional i32 heartbeatCount=0, - 7: optional string metaInfo, - 8: optional i64 startedTime, - 9: optional i64 lastHeartbeatTime, -} - -struct GetOpenTxnsInfoResponse { - 1: required i64 txn_high_water_mark, - 2: required list open_txns, -} - -struct GetOpenTxnsResponse { - 1: required i64 txn_high_water_mark, - 2: required list open_txns, // set changed to list since 3.0 - 3: optional i64 min_open_txn, //since 1.3,2.2 - 4: required binary abortedBits, // since 3.0 -} - -struct OpenTxnRequest { - 1: required i32 num_txns, - 2: required string user, - 3: required string hostname, - 4: optional string agentInfo = "Unknown", -} - -struct OpenTxnsResponse { - 1: required list txn_ids, -} - -struct AbortTxnRequest { - 1: required i64 txnid, -} - -struct AbortTxnsRequest { - 1: required list txn_ids, -} - -struct CommitTxnRequest { - 1: required i64 txnid, -} - -struct LockComponent { - 1: required LockType type, - 2: required LockLevel level, - 3: required string dbname, - 4: optional string tablename, - 5: optional string partitionname, - 6: optional DataOperationType operationType = DataOperationType.UNSET, - 7: optional bool isAcid = false, - 8: optional bool isDynamicPartitionWrite = false -} - -struct LockRequest { - 1: required list component, - 2: optional i64 txnid, - 3: required string user, // used in 'show locks' to help admins find who has open locks - 4: required string hostname, // used in 'show locks' to help admins find who has open locks - 5: optional string agentInfo = "Unknown", -} - -struct LockResponse { - 1: required i64 lockid, - 2: required LockState state, -} - -struct CheckLockRequest { - 1: required i64 lockid, - 2: optional i64 txnid, - 3: optional i64 elapsed_ms, -} - -struct UnlockRequest { - 1: required i64 lockid, -} - -struct ShowLocksRequest { - 1: optional string dbname, - 2: optional string tablename, - 3: optional string partname, - 4: optional bool isExtended=false, -} - -struct ShowLocksResponseElement { - 1: required i64 lockid, - 2: required string dbname, - 3: optional string tablename, - 4: optional string partname, - 5: required LockState state, - 6: required LockType type, - 7: optional i64 txnid, - 8: required i64 lastheartbeat, - 9: optional i64 acquiredat, - 10: required string user, - 11: required string hostname, - 12: optional i32 heartbeatCount = 0, - 13: optional string agentInfo, - 14: optional i64 blockedByExtId, - 15: optional i64 blockedByIntId, - 16: optional i64 lockIdInternal, -} - -struct ShowLocksResponse { - 1: list locks, -} - -struct HeartbeatRequest { - 1: optional i64 lockid, - 2: optional i64 txnid -} - -struct HeartbeatTxnRangeRequest { - 1: required i64 min, - 2: required i64 max -} - -struct HeartbeatTxnRangeResponse { - 1: required set aborted, - 2: required set nosuch -} - -struct CompactionRequest { - 1: required string dbname, - 2: required string tablename, - 3: optional string partitionname, - 4: required CompactionType type, - 5: optional string runas, - 6: optional map properties -} - -struct CompactionResponse { - 1: required i64 id, - 2: required string state, - 3: required bool accepted -} - -struct ShowCompactRequest { -} - -struct ShowCompactResponseElement { - 1: required string dbname, - 2: required string tablename, - 3: optional string partitionname, - 4: required CompactionType type, - 5: required string state, - 6: optional string workerid, - 7: optional i64 start, - 8: optional string runAs, - 9: optional i64 hightestTxnId, // Highest Txn ID handled by this compaction - 10: optional string metaInfo, - 11: optional i64 endTime, - 12: optional string hadoopJobId = "None", - 13: optional i64 id, -} - -struct ShowCompactResponse { - 1: required list compacts, -} - -struct AddDynamicPartitions { - 1: required i64 txnid, - 2: required string dbname, - 3: required string tablename, - 4: required list partitionnames, - 5: optional DataOperationType operationType = DataOperationType.UNSET -} - -struct NotificationEventRequest { - 1: required i64 lastEvent, - 2: optional i32 maxEvents, -} - -struct NotificationEvent { - 1: required i64 eventId, - 2: required i32 eventTime, - 3: required string eventType, - 4: optional string dbName, - 5: optional string tableName, - 6: required string message, - 7: optional string messageFormat, -} - -struct NotificationEventResponse { - 1: required list events, -} - -struct CurrentNotificationEventId { - 1: required i64 eventId, -} - -struct NotificationEventsCountRequest { - 1: required i64 fromEventId, - 2: required string dbName, -} - -struct NotificationEventsCountResponse { - 1: required i64 eventsCount, -} - -struct InsertEventRequestData { - 1: optional bool replace, - 2: required list filesAdded, - // Checksum of files (hex string of checksum byte payload) - 3: optional list filesAddedChecksum, -} - -union FireEventRequestData { - 1: InsertEventRequestData insertData -} - -struct FireEventRequest { - 1: required bool successful, - 2: required FireEventRequestData data - // dbname, tablename, and partition vals are included as optional in the top level event rather than placed in each type of - // subevent as I assume they'll be used across most event types. - 3: optional string dbName, - 4: optional string tableName, - 5: optional list partitionVals, -} - -struct FireEventResponse { - // NOP for now, this is just a place holder for future responses -} - -struct MetadataPpdResult { - 1: optional binary metadata, - 2: optional binary includeBitset -} - -// Return type for get_file_metadata_by_expr -struct GetFileMetadataByExprResult { - 1: required map metadata, - 2: required bool isSupported -} - -enum FileMetadataExprType { - ORC_SARG = 1 -} - - -// Request type for get_file_metadata_by_expr -struct GetFileMetadataByExprRequest { - 1: required list fileIds, - 2: required binary expr, - 3: optional bool doGetFooters, - 4: optional FileMetadataExprType type -} - -// Return type for get_file_metadata -struct GetFileMetadataResult { - 1: required map metadata, - 2: required bool isSupported -} - -// Request type for get_file_metadata -struct GetFileMetadataRequest { - 1: required list fileIds -} - -// Return type for put_file_metadata -struct PutFileMetadataResult { -} - -// Request type for put_file_metadata -struct PutFileMetadataRequest { - 1: required list fileIds, - 2: required list metadata, - 3: optional FileMetadataExprType type -} - -// Return type for clear_file_metadata -struct ClearFileMetadataResult { -} - -// Request type for clear_file_metadata -struct ClearFileMetadataRequest { - 1: required list fileIds -} - -// Return type for cache_file_metadata -struct CacheFileMetadataResult { - 1: required bool isSupported -} - -// Request type for cache_file_metadata -struct CacheFileMetadataRequest { - 1: required string dbName, - 2: required string tblName, - 3: optional string partName, - 4: optional bool isAllParts -} - -struct GetAllFunctionsResponse { - 1: optional list functions -} - -enum ClientCapability { - TEST_CAPABILITY = 1, - INSERT_ONLY_TABLES = 2 -} - - -struct ClientCapabilities { - 1: required list values -} - -struct GetTableRequest { - 1: required string dbName, - 2: required string tblName, - 3: optional ClientCapabilities capabilities -} - -struct GetTableResult { - 1: required Table table -} - -struct GetTablesRequest { - 1: required string dbName, - 2: optional list tblNames, - 3: optional ClientCapabilities capabilities -} - -struct GetTablesResult { - 1: required list
tables -} - -// Request type for cm_recycle -struct CmRecycleRequest { - 1: required string dataPath, - 2: required bool purge -} - -// Response type for cm_recycle -struct CmRecycleResponse { -} - -struct TableMeta { - 1: required string dbName; - 2: required string tableName; - 3: required string tableType; - 4: optional string comments; -} - -// Data types for workload management. - -enum WMResourcePlanStatus { - ACTIVE = 1, - ENABLED = 2, - DISABLED = 3 -} - -struct WMResourcePlan { - 1: required string name; - 2: optional WMResourcePlanStatus status; - 3: optional i32 queryParallelism; - 4: optional string defaultPoolPath; -} - -struct WMPool { - 1: required string resourcePlanName; - 2: required string poolPath; - 3: optional double allocFraction; - 4: optional i32 queryParallelism; - 5: optional string schedulingPolicy; -} - -struct WMTrigger { - 1: required string resourcePlanName; - 2: required string triggerName; - 3: optional string triggerExpression; - 4: optional string actionExpression; -} - -struct WMMapping { - 1: required string resourcePlanName; - 2: required string entityType; - 3: required string entityName; - 4: optional string poolName; - 5: optional i32 ordering; -} - -struct WMPoolTrigger { - 1: required string pool; - 2: required string trigger; -} - -struct WMFullResourcePlan { - 1: required WMResourcePlan plan; - 2: required list pools; - 3: optional list mappings; - 4: optional list triggers; - 5: optional list poolTriggers; -} - -// Request response for workload management API's. - -struct WMCreateResourcePlanRequest { - 1: optional WMResourcePlan resourcePlan; -} - -struct WMCreateResourcePlanResponse { -} - -struct WMGetActiveResourcePlanRequest { -} - -struct WMGetActiveResourcePlanResponse { - 1: optional WMFullResourcePlan resourcePlan; -} - -struct WMGetResourcePlanRequest { - 1: optional string resourcePlanName; -} - -struct WMGetResourcePlanResponse { - 1: optional WMResourcePlan resourcePlan; -} - -struct WMGetAllResourcePlanRequest { -} - -struct WMGetAllResourcePlanResponse { - 1: optional list resourcePlans; -} - -struct WMAlterResourcePlanRequest { - 1: optional string resourcePlanName; - 2: optional WMResourcePlan resourcePlan; - 3: optional bool isEnableAndActivate; -} - -struct WMAlterResourcePlanResponse { - 1: optional WMFullResourcePlan fullResourcePlan; -} - -struct WMValidateResourcePlanRequest { - 1: optional string resourcePlanName; -} - -struct WMValidateResourcePlanResponse { - 1: optional bool isValid; -} - -struct WMDropResourcePlanRequest { - 1: optional string resourcePlanName; -} - -struct WMDropResourcePlanResponse { -} - -struct WMCreateTriggerRequest { - 1: optional WMTrigger trigger; -} - -struct WMCreateTriggerResponse { -} - -struct WMAlterTriggerRequest { - 1: optional WMTrigger trigger; -} - -struct WMAlterTriggerResponse { -} - -struct WMDropTriggerRequest { - 1: optional string resourcePlanName; - 2: optional string triggerName; -} - -struct WMDropTriggerResponse { -} - -struct WMGetTriggersForResourePlanRequest { - 1: optional string resourcePlanName; -} - -struct WMGetTriggersForResourePlanResponse { - 1: optional list triggers; -} - - -// Exceptions. - -exception MetaException { - 1: string message -} - -exception UnknownTableException { - 1: string message -} - -exception UnknownDBException { - 1: string message -} - -exception AlreadyExistsException { - 1: string message -} - -exception InvalidPartitionException { - 1: string message -} - -exception UnknownPartitionException { - 1: string message -} - -exception InvalidObjectException { - 1: string message -} - -exception NoSuchObjectException { - 1: string message -} - -exception IndexAlreadyExistsException { - 1: string message -} - -exception InvalidOperationException { - 1: string message -} - -exception ConfigValSecurityException { - 1: string message -} - -exception InvalidInputException { - 1: string message -} - -// Transaction and lock exceptions -exception NoSuchTxnException { - 1: string message -} - -exception TxnAbortedException { - 1: string message -} - -exception TxnOpenException { - 1: string message -} - -exception NoSuchLockException { - 1: string message -} - -/** -* This interface is live. -*/ -service ThriftHiveMetastore -{ - string getMetaConf(1:string key) throws(1:MetaException o1) - void setMetaConf(1:string key, 2:string value) throws(1:MetaException o1) - - void create_database(1:Database database) throws(1:AlreadyExistsException o1, 2:InvalidObjectException o2, 3:MetaException o3) - Database get_database(1:string name) throws(1:NoSuchObjectException o1, 2:MetaException o2) - void drop_database(1:string name, 2:bool deleteData, 3:bool cascade) throws(1:NoSuchObjectException o1, 2:InvalidOperationException o2, 3:MetaException o3) - list get_databases(1:string pattern) throws(1:MetaException o1) - list get_all_databases() throws(1:MetaException o1) - void alter_database(1:string dbname, 2:Database db) throws(1:MetaException o1, 2:NoSuchObjectException o2) - - // returns the type with given name (make seperate calls for the dependent types if needed) - Type get_type(1:string name) throws(1:MetaException o1, 2:NoSuchObjectException o2) - bool create_type(1:Type type) throws(1:AlreadyExistsException o1, 2:InvalidObjectException o2, 3:MetaException o3) - bool drop_type(1:string type) throws(1:MetaException o1, 2:NoSuchObjectException o2) - map get_type_all(1:string name) - throws(1:MetaException o2) - - // Gets a list of FieldSchemas describing the columns of a particular table - list get_fields(1: string db_name, 2: string table_name) throws (1: MetaException o1, 2: UnknownTableException o2, 3: UnknownDBException o3), - list get_fields_with_environment_context(1: string db_name, 2: string table_name, 3:EnvironmentContext environment_context) throws (1: MetaException o1, 2: UnknownTableException o2, 3: UnknownDBException o3) - - // Gets a list of FieldSchemas describing both the columns and the partition keys of a particular table - list get_schema(1: string db_name, 2: string table_name) throws (1: MetaException o1, 2: UnknownTableException o2, 3: UnknownDBException o3) - list get_schema_with_environment_context(1: string db_name, 2: string table_name, 3:EnvironmentContext environment_context) throws (1: MetaException o1, 2: UnknownTableException o2, 3: UnknownDBException o3) - - // create a Hive table. Following fields must be set - // tableName - // database (only 'default' for now until Hive QL supports databases) - // owner (not needed, but good to have for tracking purposes) - // sd.cols (list of field schemas) - // sd.inputFormat (SequenceFileInputFormat (binary like falcon tables or u_full) or TextInputFormat) - // sd.outputFormat (SequenceFileInputFormat (binary) or TextInputFormat) - // sd.serdeInfo.serializationLib (SerDe class name eg org.apache.hadoop.hive.serde.simple_meta.MetadataTypedColumnsetSerDe - // * See notes on DDL_TIME - void create_table(1:Table tbl) throws(1:AlreadyExistsException o1, 2:InvalidObjectException o2, 3:MetaException o3, 4:NoSuchObjectException o4) - void create_table_with_environment_context(1:Table tbl, - 2:EnvironmentContext environment_context) - throws (1:AlreadyExistsException o1, - 2:InvalidObjectException o2, 3:MetaException o3, - 4:NoSuchObjectException o4) - void create_table_with_constraints(1:Table tbl, 2: list primaryKeys, 3: list foreignKeys, - 4: list uniqueConstraints, 5: list notNullConstraints) - throws (1:AlreadyExistsException o1, - 2:InvalidObjectException o2, 3:MetaException o3, - 4:NoSuchObjectException o4) - void drop_constraint(1:DropConstraintRequest req) - throws(1:NoSuchObjectException o1, 2:MetaException o3) - void add_primary_key(1:AddPrimaryKeyRequest req) - throws(1:NoSuchObjectException o1, 2:MetaException o2) - void add_foreign_key(1:AddForeignKeyRequest req) - throws(1:NoSuchObjectException o1, 2:MetaException o2) - void add_unique_constraint(1:AddUniqueConstraintRequest req) - throws(1:NoSuchObjectException o1, 2:MetaException o2) - void add_not_null_constraint(1:AddNotNullConstraintRequest req) - throws(1:NoSuchObjectException o1, 2:MetaException o2) - - // drops the table and all the partitions associated with it if the table has partitions - // delete data (including partitions) if deleteData is set to true - void drop_table(1:string dbname, 2:string name, 3:bool deleteData) - throws(1:NoSuchObjectException o1, 2:MetaException o3) - void drop_table_with_environment_context(1:string dbname, 2:string name, 3:bool deleteData, - 4:EnvironmentContext environment_context) - throws(1:NoSuchObjectException o1, 2:MetaException o3) - void truncate_table(1:string dbName, 2:string tableName, 3:list partNames) - throws(1:MetaException o1) - list get_tables(1: string db_name, 2: string pattern) throws (1: MetaException o1) - list get_tables_by_type(1: string db_name, 2: string pattern, 3: string tableType) throws (1: MetaException o1) - list get_table_meta(1: string db_patterns, 2: string tbl_patterns, 3: list tbl_types) - throws (1: MetaException o1) - list get_all_tables(1: string db_name) throws (1: MetaException o1) - - Table get_table(1:string dbname, 2:string tbl_name) - throws (1:MetaException o1, 2:NoSuchObjectException o2) - list
get_table_objects_by_name(1:string dbname, 2:list tbl_names) - GetTableResult get_table_req(1:GetTableRequest req) throws (1:MetaException o1, 2:NoSuchObjectException o2) - GetTablesResult get_table_objects_by_name_req(1:GetTablesRequest req) - throws (1:MetaException o1, 2:InvalidOperationException o2, 3:UnknownDBException o3) - - // Get a list of table names that match a filter. - // The filter operators are LIKE, <, <=, >, >=, =, <> - // - // In the filter statement, values interpreted as strings must be enclosed in quotes, - // while values interpreted as integers should not be. Strings and integers are the only - // supported value types. - // - // The currently supported key names in the filter are: - // Constants.HIVE_FILTER_FIELD_OWNER, which filters on the tables' owner's name - // and supports all filter operators - // Constants.HIVE_FILTER_FIELD_LAST_ACCESS, which filters on the last access times - // and supports all filter operators except LIKE - // Constants.HIVE_FILTER_FIELD_PARAMS, which filters on the tables' parameter keys and values - // and only supports the filter operators = and <>. - // Append the parameter key name to HIVE_FILTER_FIELD_PARAMS in the filter statement. - // For example, to filter on parameter keys called "retention", the key name in the filter - // statement should be Constants.HIVE_FILTER_FIELD_PARAMS + "retention" - // Also, = and <> only work for keys that exist - // in the tables. E.g., if you are looking for tables where key1 <> value, it will only - // look at tables that have a value for the parameter key1. - // Some example filter statements include: - // filter = Constants.HIVE_FILTER_FIELD_OWNER + " like \".*test.*\" and " + - // Constants.HIVE_FILTER_FIELD_LAST_ACCESS + " = 0"; - // filter = Constants.HIVE_FILTER_FIELD_PARAMS + "retention = \"30\" or " + - // Constants.HIVE_FILTER_FIELD_PARAMS + "retention = \"90\"" - // @param dbName - // The name of the database from which you will retrieve the table names - // @param filterType - // The type of filter - // @param filter - // The filter string - // @param max_tables - // The maximum number of tables returned - // @return A list of table names that match the desired filter - list get_table_names_by_filter(1:string dbname, 2:string filter, 3:i16 max_tables=-1) - throws (1:MetaException o1, 2:InvalidOperationException o2, 3:UnknownDBException o3) - - // alter table applies to only future partitions not for existing partitions - // * See notes on DDL_TIME - void alter_table(1:string dbname, 2:string tbl_name, 3:Table new_tbl) - throws (1:InvalidOperationException o1, 2:MetaException o2) - void alter_table_with_environment_context(1:string dbname, 2:string tbl_name, - 3:Table new_tbl, 4:EnvironmentContext environment_context) - throws (1:InvalidOperationException o1, 2:MetaException o2) - // alter table not only applies to future partitions but also cascade to existing partitions - void alter_table_with_cascade(1:string dbname, 2:string tbl_name, 3:Table new_tbl, 4:bool cascade) - throws (1:InvalidOperationException o1, 2:MetaException o2) - // the following applies to only tables that have partitions - // * See notes on DDL_TIME - Partition add_partition(1:Partition new_part) - throws(1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) - Partition add_partition_with_environment_context(1:Partition new_part, - 2:EnvironmentContext environment_context) - throws (1:InvalidObjectException o1, 2:AlreadyExistsException o2, - 3:MetaException o3) - i32 add_partitions(1:list new_parts) - throws(1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) - i32 add_partitions_pspec(1:list new_parts) - throws(1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) - Partition append_partition(1:string db_name, 2:string tbl_name, 3:list part_vals) - throws (1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) - AddPartitionsResult add_partitions_req(1:AddPartitionsRequest request) - throws(1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) - Partition append_partition_with_environment_context(1:string db_name, 2:string tbl_name, - 3:list part_vals, 4:EnvironmentContext environment_context) - throws (1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) - Partition append_partition_by_name(1:string db_name, 2:string tbl_name, 3:string part_name) - throws (1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) - Partition append_partition_by_name_with_environment_context(1:string db_name, 2:string tbl_name, - 3:string part_name, 4:EnvironmentContext environment_context) - throws (1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) - bool drop_partition(1:string db_name, 2:string tbl_name, 3:list part_vals, 4:bool deleteData) - throws(1:NoSuchObjectException o1, 2:MetaException o2) - bool drop_partition_with_environment_context(1:string db_name, 2:string tbl_name, - 3:list part_vals, 4:bool deleteData, 5:EnvironmentContext environment_context) - throws(1:NoSuchObjectException o1, 2:MetaException o2) - bool drop_partition_by_name(1:string db_name, 2:string tbl_name, 3:string part_name, 4:bool deleteData) - throws(1:NoSuchObjectException o1, 2:MetaException o2) - bool drop_partition_by_name_with_environment_context(1:string db_name, 2:string tbl_name, - 3:string part_name, 4:bool deleteData, 5:EnvironmentContext environment_context) - throws(1:NoSuchObjectException o1, 2:MetaException o2) - DropPartitionsResult drop_partitions_req(1: DropPartitionsRequest req) - throws(1:NoSuchObjectException o1, 2:MetaException o2) - - Partition get_partition(1:string db_name, 2:string tbl_name, 3:list part_vals) - throws(1:MetaException o1, 2:NoSuchObjectException o2) - Partition exchange_partition(1:map partitionSpecs, 2:string source_db, - 3:string source_table_name, 4:string dest_db, 5:string dest_table_name) - throws(1:MetaException o1, 2:NoSuchObjectException o2, 3:InvalidObjectException o3, - 4:InvalidInputException o4) - - list exchange_partitions(1:map partitionSpecs, 2:string source_db, - 3:string source_table_name, 4:string dest_db, 5:string dest_table_name) - throws(1:MetaException o1, 2:NoSuchObjectException o2, 3:InvalidObjectException o3, - 4:InvalidInputException o4) - - Partition get_partition_with_auth(1:string db_name, 2:string tbl_name, 3:list part_vals, - 4: string user_name, 5: list group_names) throws(1:MetaException o1, 2:NoSuchObjectException o2) - - Partition get_partition_by_name(1:string db_name 2:string tbl_name, 3:string part_name) - throws(1:MetaException o1, 2:NoSuchObjectException o2) - - // returns all the partitions for this table in reverse chronological order. - // If max parts is given then it will return only that many. - list get_partitions(1:string db_name, 2:string tbl_name, 3:i16 max_parts=-1) - throws(1:NoSuchObjectException o1, 2:MetaException o2) - list get_partitions_with_auth(1:string db_name, 2:string tbl_name, 3:i16 max_parts=-1, - 4: string user_name, 5: list group_names) throws(1:NoSuchObjectException o1, 2:MetaException o2) - - list get_partitions_pspec(1:string db_name, 2:string tbl_name, 3:i32 max_parts=-1) - throws(1:NoSuchObjectException o1, 2:MetaException o2) - - list get_partition_names(1:string db_name, 2:string tbl_name, 3:i16 max_parts=-1) - throws(1:NoSuchObjectException o1, 2:MetaException o2) - - PartitionValuesResponse get_partition_values(1:PartitionValuesRequest request) - throws(1:MetaException o1, 2:NoSuchObjectException o2); - - // get_partition*_ps methods allow filtering by a partial partition specification, - // as needed for dynamic partitions. The values that are not restricted should - // be empty strings. Nulls were considered (instead of "") but caused errors in - // generated Python code. The size of part_vals may be smaller than the - // number of partition columns - the unspecified values are considered the same - // as "". - list get_partitions_ps(1:string db_name 2:string tbl_name - 3:list part_vals, 4:i16 max_parts=-1) - throws(1:MetaException o1, 2:NoSuchObjectException o2) - list get_partitions_ps_with_auth(1:string db_name, 2:string tbl_name, 3:list part_vals, 4:i16 max_parts=-1, - 5: string user_name, 6: list group_names) throws(1:NoSuchObjectException o1, 2:MetaException o2) - - list get_partition_names_ps(1:string db_name, - 2:string tbl_name, 3:list part_vals, 4:i16 max_parts=-1) - throws(1:MetaException o1, 2:NoSuchObjectException o2) - - // get the partitions matching the given partition filter - list get_partitions_by_filter(1:string db_name 2:string tbl_name - 3:string filter, 4:i16 max_parts=-1) - throws(1:MetaException o1, 2:NoSuchObjectException o2) - - // List partitions as PartitionSpec instances. - list get_part_specs_by_filter(1:string db_name 2:string tbl_name - 3:string filter, 4:i32 max_parts=-1) - throws(1:MetaException o1, 2:NoSuchObjectException o2) - - // get the partitions matching the given partition filter - // unlike get_partitions_by_filter, takes serialized hive expression, and with that can work - // with any filter (get_partitions_by_filter only works if the filter can be pushed down to JDOQL. - PartitionsByExprResult get_partitions_by_expr(1:PartitionsByExprRequest req) - throws(1:MetaException o1, 2:NoSuchObjectException o2) - - // get the partitions matching the given partition filter - i32 get_num_partitions_by_filter(1:string db_name 2:string tbl_name 3:string filter) - throws(1:MetaException o1, 2:NoSuchObjectException o2) - - // get partitions give a list of partition names - list get_partitions_by_names(1:string db_name 2:string tbl_name 3:list names) - throws(1:MetaException o1, 2:NoSuchObjectException o2) - - // changes the partition to the new partition object. partition is identified from the part values - // in the new_part - // * See notes on DDL_TIME - void alter_partition(1:string db_name, 2:string tbl_name, 3:Partition new_part) - throws (1:InvalidOperationException o1, 2:MetaException o2) - - // change a list of partitions. All partitions are altered atomically and all - // prehooks are fired together followed by all post hooks - void alter_partitions(1:string db_name, 2:string tbl_name, 3:list new_parts) - throws (1:InvalidOperationException o1, 2:MetaException o2) - void alter_partitions_with_environment_context(1:string db_name, 2:string tbl_name, 3:list new_parts, 4:EnvironmentContext environment_context) throws (1:InvalidOperationException o1, 2:MetaException o2) - - void alter_partition_with_environment_context(1:string db_name, - 2:string tbl_name, 3:Partition new_part, - 4:EnvironmentContext environment_context) - throws (1:InvalidOperationException o1, 2:MetaException o2) - - // rename the old partition to the new partition object by changing old part values to the part values - // in the new_part. old partition is identified from part_vals. - // partition keys in new_part should be the same as those in old partition. - void rename_partition(1:string db_name, 2:string tbl_name, 3:list part_vals, 4:Partition new_part) - throws (1:InvalidOperationException o1, 2:MetaException o2) - - // returns whether or not the partition name is valid based on the value of the config - // hive.metastore.partition.name.whitelist.pattern - bool partition_name_has_valid_characters(1:list part_vals, 2:bool throw_exception) - throws(1: MetaException o1) - - // gets the value of the configuration key in the metastore server. returns - // defaultValue if the key does not exist. if the configuration key does not - // begin with "hive", "mapred", or "hdfs", a ConfigValSecurityException is - // thrown. - string get_config_value(1:string name, 2:string defaultValue) - throws(1:ConfigValSecurityException o1) - - // converts a partition name into a partition values array - list partition_name_to_vals(1: string part_name) - throws(1: MetaException o1) - // converts a partition name into a partition specification (a mapping from - // the partition cols to the values) - map partition_name_to_spec(1: string part_name) - throws(1: MetaException o1) - - void markPartitionForEvent(1:string db_name, 2:string tbl_name, 3:map part_vals, - 4:PartitionEventType eventType) throws (1: MetaException o1, 2: NoSuchObjectException o2, - 3: UnknownDBException o3, 4: UnknownTableException o4, 5: UnknownPartitionException o5, - 6: InvalidPartitionException o6) - bool isPartitionMarkedForEvent(1:string db_name, 2:string tbl_name, 3:map part_vals, - 4: PartitionEventType eventType) throws (1: MetaException o1, 2:NoSuchObjectException o2, - 3: UnknownDBException o3, 4: UnknownTableException o4, 5: UnknownPartitionException o5, - 6: InvalidPartitionException o6) - - //index - Index add_index(1:Index new_index, 2: Table index_table) - throws(1:InvalidObjectException o1, 2:AlreadyExistsException o2, 3:MetaException o3) - void alter_index(1:string dbname, 2:string base_tbl_name, 3:string idx_name, 4:Index new_idx) - throws (1:InvalidOperationException o1, 2:MetaException o2) - bool drop_index_by_name(1:string db_name, 2:string tbl_name, 3:string index_name, 4:bool deleteData) - throws(1:NoSuchObjectException o1, 2:MetaException o2) - Index get_index_by_name(1:string db_name 2:string tbl_name, 3:string index_name) - throws(1:MetaException o1, 2:NoSuchObjectException o2) - - list get_indexes(1:string db_name, 2:string tbl_name, 3:i16 max_indexes=-1) - throws(1:NoSuchObjectException o1, 2:MetaException o2) - list get_index_names(1:string db_name, 2:string tbl_name, 3:i16 max_indexes=-1) - throws(1:MetaException o2) - - //primary keys and foreign keys - PrimaryKeysResponse get_primary_keys(1:PrimaryKeysRequest request) - throws(1:MetaException o1, 2:NoSuchObjectException o2) - ForeignKeysResponse get_foreign_keys(1:ForeignKeysRequest request) - throws(1:MetaException o1, 2:NoSuchObjectException o2) - // other constraints - UniqueConstraintsResponse get_unique_constraints(1:UniqueConstraintsRequest request) - throws(1:MetaException o1, 2:NoSuchObjectException o2) - NotNullConstraintsResponse get_not_null_constraints(1:NotNullConstraintsRequest request) - throws(1:MetaException o1, 2:NoSuchObjectException o2) - - // column statistics interfaces - - // update APIs persist the column statistics object(s) that are passed in. If statistics already - // exists for one or more columns, the existing statistics will be overwritten. The update APIs - // validate that the dbName, tableName, partName, colName[] passed in as part of the ColumnStatistics - // struct are valid, throws InvalidInputException/NoSuchObjectException if found to be invalid - bool update_table_column_statistics(1:ColumnStatistics stats_obj) throws (1:NoSuchObjectException o1, - 2:InvalidObjectException o2, 3:MetaException o3, 4:InvalidInputException o4) - bool update_partition_column_statistics(1:ColumnStatistics stats_obj) throws (1:NoSuchObjectException o1, - 2:InvalidObjectException o2, 3:MetaException o3, 4:InvalidInputException o4) - - // get APIs return the column statistics corresponding to db_name, tbl_name, [part_name], col_name if - // such statistics exists. If the required statistics doesn't exist, get APIs throw NoSuchObjectException - // For instance, if get_table_column_statistics is called on a partitioned table for which only - // partition level column stats exist, get_table_column_statistics will throw NoSuchObjectException - ColumnStatistics get_table_column_statistics(1:string db_name, 2:string tbl_name, 3:string col_name) throws - (1:NoSuchObjectException o1, 2:MetaException o2, 3:InvalidInputException o3, 4:InvalidObjectException o4) - ColumnStatistics get_partition_column_statistics(1:string db_name, 2:string tbl_name, 3:string part_name, - 4:string col_name) throws (1:NoSuchObjectException o1, 2:MetaException o2, - 3:InvalidInputException o3, 4:InvalidObjectException o4) - TableStatsResult get_table_statistics_req(1:TableStatsRequest request) throws - (1:NoSuchObjectException o1, 2:MetaException o2) - PartitionsStatsResult get_partitions_statistics_req(1:PartitionsStatsRequest request) throws - (1:NoSuchObjectException o1, 2:MetaException o2) - AggrStats get_aggr_stats_for(1:PartitionsStatsRequest request) throws - (1:NoSuchObjectException o1, 2:MetaException o2) - bool set_aggr_stats_for(1:SetPartitionsStatsRequest request) throws - (1:NoSuchObjectException o1, 2:InvalidObjectException o2, 3:MetaException o3, 4:InvalidInputException o4) - - - // delete APIs attempt to delete column statistics, if found, associated with a given db_name, tbl_name, [part_name] - // and col_name. If the delete API doesn't find the statistics record in the metastore, throws NoSuchObjectException - // Delete API validates the input and if the input is invalid throws InvalidInputException/InvalidObjectException. - bool delete_partition_column_statistics(1:string db_name, 2:string tbl_name, 3:string part_name, 4:string col_name) throws - (1:NoSuchObjectException o1, 2:MetaException o2, 3:InvalidObjectException o3, - 4:InvalidInputException o4) - bool delete_table_column_statistics(1:string db_name, 2:string tbl_name, 3:string col_name) throws - (1:NoSuchObjectException o1, 2:MetaException o2, 3:InvalidObjectException o3, - 4:InvalidInputException o4) - - // - // user-defined functions - // - - void create_function(1:Function func) - throws (1:AlreadyExistsException o1, - 2:InvalidObjectException o2, - 3:MetaException o3, - 4:NoSuchObjectException o4) - - void drop_function(1:string dbName, 2:string funcName) - throws (1:NoSuchObjectException o1, 2:MetaException o3) - - void alter_function(1:string dbName, 2:string funcName, 3:Function newFunc) - throws (1:InvalidOperationException o1, 2:MetaException o2) - - list get_functions(1:string dbName, 2:string pattern) - throws (1:MetaException o1) - Function get_function(1:string dbName, 2:string funcName) - throws (1:MetaException o1, 2:NoSuchObjectException o2) - - GetAllFunctionsResponse get_all_functions() throws (1:MetaException o1) - - //authorization privileges - - bool create_role(1:Role role) throws(1:MetaException o1) - bool drop_role(1:string role_name) throws(1:MetaException o1) - list get_role_names() throws(1:MetaException o1) - // Deprecated, use grant_revoke_role() - bool grant_role(1:string role_name, 2:string principal_name, 3:PrincipalType principal_type, - 4:string grantor, 5:PrincipalType grantorType, 6:bool grant_option) throws(1:MetaException o1) - // Deprecated, use grant_revoke_role() - bool revoke_role(1:string role_name, 2:string principal_name, 3:PrincipalType principal_type) - throws(1:MetaException o1) - list list_roles(1:string principal_name, 2:PrincipalType principal_type) throws(1:MetaException o1) - GrantRevokeRoleResponse grant_revoke_role(1:GrantRevokeRoleRequest request) throws(1:MetaException o1) - - // get all role-grants for users/roles that have been granted the given role - // Note that in the returned list of RolePrincipalGrants, the roleName is - // redundant as it would match the role_name argument of this function - GetPrincipalsInRoleResponse get_principals_in_role(1: GetPrincipalsInRoleRequest request) throws(1:MetaException o1) - - // get grant information of all roles granted to the given principal - // Note that in the returned list of RolePrincipalGrants, the principal name,type is - // redundant as it would match the principal name,type arguments of this function - GetRoleGrantsForPrincipalResponse get_role_grants_for_principal(1: GetRoleGrantsForPrincipalRequest request) throws(1:MetaException o1) - - PrincipalPrivilegeSet get_privilege_set(1:HiveObjectRef hiveObject, 2:string user_name, - 3: list group_names) throws(1:MetaException o1) - list list_privileges(1:string principal_name, 2:PrincipalType principal_type, - 3: HiveObjectRef hiveObject) throws(1:MetaException o1) - - // Deprecated, use grant_revoke_privileges() - bool grant_privileges(1:PrivilegeBag privileges) throws(1:MetaException o1) - // Deprecated, use grant_revoke_privileges() - bool revoke_privileges(1:PrivilegeBag privileges) throws(1:MetaException o1) - GrantRevokePrivilegeResponse grant_revoke_privileges(1:GrantRevokePrivilegeRequest request) throws(1:MetaException o1); - - // this is used by metastore client to send UGI information to metastore server immediately - // after setting up a connection. - list set_ugi(1:string user_name, 2:list group_names) throws (1:MetaException o1) - - //Authentication (delegation token) interfaces - - // get metastore server delegation token for use from the map/reduce tasks to authenticate - // to metastore server - string get_delegation_token(1:string token_owner, 2:string renewer_kerberos_principal_name) - throws (1:MetaException o1) - - // method to renew delegation token obtained from metastore server - i64 renew_delegation_token(1:string token_str_form) throws (1:MetaException o1) - - // method to cancel delegation token obtained from metastore server - void cancel_delegation_token(1:string token_str_form) throws (1:MetaException o1) - - // add a delegation token - bool add_token(1:string token_identifier, 2:string delegation_token) - - // remove a delegation token - bool remove_token(1:string token_identifier) - - // get a delegation token by identifier - string get_token(1:string token_identifier) - - // get all delegation token identifiers - list get_all_token_identifiers() - - // add master key - i32 add_master_key(1:string key) throws (1:MetaException o1) - - // update master key - void update_master_key(1:i32 seq_number, 2:string key) throws (1:NoSuchObjectException o1, 2:MetaException o2) - - // remove master key - bool remove_master_key(1:i32 key_seq) - - // get master keys - list get_master_keys() - - // Transaction and lock management calls - // Get just list of open transactions - GetOpenTxnsResponse get_open_txns() - // Get list of open transactions with state (open, aborted) - GetOpenTxnsInfoResponse get_open_txns_info() - OpenTxnsResponse open_txns(1:OpenTxnRequest rqst) - void abort_txn(1:AbortTxnRequest rqst) throws (1:NoSuchTxnException o1) - void abort_txns(1:AbortTxnsRequest rqst) throws (1:NoSuchTxnException o1) - void commit_txn(1:CommitTxnRequest rqst) throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2) - LockResponse lock(1:LockRequest rqst) throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2) - LockResponse check_lock(1:CheckLockRequest rqst) - throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2, 3:NoSuchLockException o3) - void unlock(1:UnlockRequest rqst) throws (1:NoSuchLockException o1, 2:TxnOpenException o2) - ShowLocksResponse show_locks(1:ShowLocksRequest rqst) - void heartbeat(1:HeartbeatRequest ids) throws (1:NoSuchLockException o1, 2:NoSuchTxnException o2, 3:TxnAbortedException o3) - HeartbeatTxnRangeResponse heartbeat_txn_range(1:HeartbeatTxnRangeRequest txns) - void compact(1:CompactionRequest rqst) - CompactionResponse compact2(1:CompactionRequest rqst) - ShowCompactResponse show_compact(1:ShowCompactRequest rqst) - void add_dynamic_partitions(1:AddDynamicPartitions rqst) throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2) - - // Notification logging calls - NotificationEventResponse get_next_notification(1:NotificationEventRequest rqst) - CurrentNotificationEventId get_current_notificationEventId() - NotificationEventsCountResponse get_notification_events_count(NotificationEventsCountRequest rqst) - FireEventResponse fire_listener_event(1:FireEventRequest rqst) - void flushCache() - - // Repl Change Management api - CmRecycleResponse cm_recycle(1:CmRecycleRequest request) throws(1:MetaException o1) - - GetFileMetadataByExprResult get_file_metadata_by_expr(1:GetFileMetadataByExprRequest req) - GetFileMetadataResult get_file_metadata(1:GetFileMetadataRequest req) - PutFileMetadataResult put_file_metadata(1:PutFileMetadataRequest req) - ClearFileMetadataResult clear_file_metadata(1:ClearFileMetadataRequest req) - CacheFileMetadataResult cache_file_metadata(1:CacheFileMetadataRequest req) - - // Metastore DB properties - string get_metastore_db_uuid() throws (1:MetaException o1) - - // Workload management API's - WMCreateResourcePlanResponse create_resource_plan(1:WMCreateResourcePlanRequest request) - throws(1:AlreadyExistsException o1, 2:InvalidObjectException o2, 3:MetaException o3) - - WMGetResourcePlanResponse get_resource_plan(1:WMGetResourcePlanRequest request) - throws(1:NoSuchObjectException o1, 2:MetaException o2) - - WMGetActiveResourcePlanResponse get_active_resource_plan(1:WMGetActiveResourcePlanRequest request) - throws(1:MetaException o2) - - WMGetAllResourcePlanResponse get_all_resource_plans(1:WMGetAllResourcePlanRequest request) - throws(1:MetaException o1) - - WMAlterResourcePlanResponse alter_resource_plan(1:WMAlterResourcePlanRequest request) - throws(1:NoSuchObjectException o1, 2:InvalidOperationException o2, 3:MetaException o3) - - WMValidateResourcePlanResponse validate_resource_plan(1:WMValidateResourcePlanRequest request) - throws(1:NoSuchObjectException o1, 2:MetaException o2) - - WMDropResourcePlanResponse drop_resource_plan(1:WMDropResourcePlanRequest request) - throws(1:NoSuchObjectException o1, 2:InvalidOperationException o2, 3:MetaException o3) - - WMCreateTriggerResponse create_wm_trigger(1:WMCreateTriggerRequest request) - throws(1:AlreadyExistsException o1, 2:NoSuchObjectException o2, 3:InvalidObjectException o3, 4:MetaException o4) - - WMAlterTriggerResponse alter_wm_trigger(1:WMAlterTriggerRequest request) - throws(1:NoSuchObjectException o1, 2:InvalidObjectException o2, 3:MetaException o3) - - WMDropTriggerResponse drop_wm_trigger(1:WMDropTriggerRequest request) - throws(1:NoSuchObjectException o1, 2:InvalidOperationException o2, 3:MetaException o3) - - WMGetTriggersForResourePlanResponse get_triggers_for_resourceplan(1:WMGetTriggersForResourePlanRequest request) - throws(1:NoSuchObjectException o1, 2:MetaException o2) -} - -// * Note about the DDL_TIME: When creating or altering a table or a partition, -// if the DDL_TIME is not set, the current time will be used. - -// For storing info about archived partitions in parameters - -// Whether the partition is archived -const string IS_ARCHIVED = "is_archived", -// The original location of the partition, before archiving. After archiving, -// this directory will contain the archive. When the partition -// is dropped, this directory will be deleted -const string ORIGINAL_LOCATION = "original_location", - -// Whether or not the table is considered immutable - immutable tables can only be -// overwritten or created if unpartitioned, or if partitioned, partitions inside them -// can only be overwritten or created. Immutability supports write-once and replace -// semantics, but not append. -const string IS_IMMUTABLE = "immutable", - -// these should be needed only for backward compatibility with filestore -const string META_TABLE_COLUMNS = "columns", -const string META_TABLE_COLUMN_TYPES = "columns.types", -const string BUCKET_FIELD_NAME = "bucket_field_name", -const string BUCKET_COUNT = "bucket_count", -const string FIELD_TO_DIMENSION = "field_to_dimension", -const string META_TABLE_NAME = "name", -const string META_TABLE_DB = "db", -const string META_TABLE_LOCATION = "location", -const string META_TABLE_SERDE = "serde", -const string META_TABLE_PARTITION_COLUMNS = "partition_columns", -const string META_TABLE_PARTITION_COLUMN_TYPES = "partition_columns.types", -const string FILE_INPUT_FORMAT = "file.inputformat", -const string FILE_OUTPUT_FORMAT = "file.outputformat", -const string META_TABLE_STORAGE = "storage_handler", -const string TABLE_IS_TRANSACTIONAL = "transactional", -const string TABLE_NO_AUTO_COMPACT = "no_auto_compaction", -const string TABLE_TRANSACTIONAL_PROPERTIES = "transactional_properties", - - diff --git a/contrib/hive-metastore/hive_metastore_constants.cpp b/contrib/hive-metastore/hive_metastore_constants.cpp deleted file mode 100644 index 517d771efce..00000000000 --- a/contrib/hive-metastore/hive_metastore_constants.cpp +++ /dev/null @@ -1,65 +0,0 @@ -/** - * Autogenerated by Thrift Compiler () - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -#include "hive_metastore_constants.h" - -namespace Apache { namespace Hadoop { namespace Hive { - -const hive_metastoreConstants g_hive_metastore_constants; - -hive_metastoreConstants::hive_metastoreConstants() { - DDL_TIME = "transient_lastDdlTime"; - - HIVE_FILTER_FIELD_OWNER = "hive_filter_field_owner__"; - - HIVE_FILTER_FIELD_PARAMS = "hive_filter_field_params__"; - - HIVE_FILTER_FIELD_LAST_ACCESS = "hive_filter_field_last_access__"; - - IS_ARCHIVED = "is_archived"; - - ORIGINAL_LOCATION = "original_location"; - - IS_IMMUTABLE = "immutable"; - - META_TABLE_COLUMNS = "columns"; - - META_TABLE_COLUMN_TYPES = "columns.types"; - - BUCKET_FIELD_NAME = "bucket_field_name"; - - BUCKET_COUNT = "bucket_count"; - - FIELD_TO_DIMENSION = "field_to_dimension"; - - META_TABLE_NAME = "name"; - - META_TABLE_DB = "db"; - - META_TABLE_LOCATION = "location"; - - META_TABLE_SERDE = "serde"; - - META_TABLE_PARTITION_COLUMNS = "partition_columns"; - - META_TABLE_PARTITION_COLUMN_TYPES = "partition_columns.types"; - - FILE_INPUT_FORMAT = "file.inputformat"; - - FILE_OUTPUT_FORMAT = "file.outputformat"; - - META_TABLE_STORAGE = "storage_handler"; - - TABLE_IS_TRANSACTIONAL = "transactional"; - - TABLE_NO_AUTO_COMPACT = "no_auto_compaction"; - - TABLE_TRANSACTIONAL_PROPERTIES = "transactional_properties"; - -} - -}}} // namespace - diff --git a/contrib/hive-metastore/hive_metastore_constants.h b/contrib/hive-metastore/hive_metastore_constants.h deleted file mode 100644 index 68e8bea72e2..00000000000 --- a/contrib/hive-metastore/hive_metastore_constants.h +++ /dev/null @@ -1,48 +0,0 @@ -/** - * Autogenerated by Thrift Compiler () - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -#ifndef hive_metastore_CONSTANTS_H -#define hive_metastore_CONSTANTS_H - -#include "hive_metastore_types.h" - -namespace Apache { namespace Hadoop { namespace Hive { - -class hive_metastoreConstants { - public: - hive_metastoreConstants(); - - std::string DDL_TIME; - std::string HIVE_FILTER_FIELD_OWNER; - std::string HIVE_FILTER_FIELD_PARAMS; - std::string HIVE_FILTER_FIELD_LAST_ACCESS; - std::string IS_ARCHIVED; - std::string ORIGINAL_LOCATION; - std::string IS_IMMUTABLE; - std::string META_TABLE_COLUMNS; - std::string META_TABLE_COLUMN_TYPES; - std::string BUCKET_FIELD_NAME; - std::string BUCKET_COUNT; - std::string FIELD_TO_DIMENSION; - std::string META_TABLE_NAME; - std::string META_TABLE_DB; - std::string META_TABLE_LOCATION; - std::string META_TABLE_SERDE; - std::string META_TABLE_PARTITION_COLUMNS; - std::string META_TABLE_PARTITION_COLUMN_TYPES; - std::string FILE_INPUT_FORMAT; - std::string FILE_OUTPUT_FORMAT; - std::string META_TABLE_STORAGE; - std::string TABLE_IS_TRANSACTIONAL; - std::string TABLE_NO_AUTO_COMPACT; - std::string TABLE_TRANSACTIONAL_PROPERTIES; -}; - -extern const hive_metastoreConstants g_hive_metastore_constants; - -}}} // namespace - -#endif diff --git a/contrib/hive-metastore/hive_metastore_types.cpp b/contrib/hive-metastore/hive_metastore_types.cpp deleted file mode 100644 index e4d820f1863..00000000000 --- a/contrib/hive-metastore/hive_metastore_types.cpp +++ /dev/null @@ -1,26465 +0,0 @@ -/** - * Autogenerated by Thrift Compiler () - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -#include "hive_metastore_types.h" - -#include -#include - -#include - -namespace Apache { namespace Hadoop { namespace Hive { - -int _kHiveObjectTypeValues[] = { - HiveObjectType::GLOBAL, - HiveObjectType::DATABASE, - HiveObjectType::TABLE, - HiveObjectType::PARTITION, - HiveObjectType::COLUMN -}; -const char* _kHiveObjectTypeNames[] = { - "GLOBAL", - "DATABASE", - "TABLE", - "PARTITION", - "COLUMN" -}; -const std::map _HiveObjectType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(5, _kHiveObjectTypeValues, _kHiveObjectTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const HiveObjectType::type& val) { - std::map::const_iterator it = _HiveObjectType_VALUES_TO_NAMES.find(val); - if (it != _HiveObjectType_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kPrincipalTypeValues[] = { - PrincipalType::USER, - PrincipalType::ROLE, - PrincipalType::GROUP -}; -const char* _kPrincipalTypeNames[] = { - "USER", - "ROLE", - "GROUP" -}; -const std::map _PrincipalType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kPrincipalTypeValues, _kPrincipalTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const PrincipalType::type& val) { - std::map::const_iterator it = _PrincipalType_VALUES_TO_NAMES.find(val); - if (it != _PrincipalType_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kPartitionEventTypeValues[] = { - PartitionEventType::LOAD_DONE -}; -const char* _kPartitionEventTypeNames[] = { - "LOAD_DONE" -}; -const std::map _PartitionEventType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(1, _kPartitionEventTypeValues, _kPartitionEventTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const PartitionEventType::type& val) { - std::map::const_iterator it = _PartitionEventType_VALUES_TO_NAMES.find(val); - if (it != _PartitionEventType_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kTxnStateValues[] = { - TxnState::COMMITTED, - TxnState::ABORTED, - TxnState::OPEN -}; -const char* _kTxnStateNames[] = { - "COMMITTED", - "ABORTED", - "OPEN" -}; -const std::map _TxnState_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kTxnStateValues, _kTxnStateNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const TxnState::type& val) { - std::map::const_iterator it = _TxnState_VALUES_TO_NAMES.find(val); - if (it != _TxnState_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kLockLevelValues[] = { - LockLevel::DB, - LockLevel::TABLE, - LockLevel::PARTITION -}; -const char* _kLockLevelNames[] = { - "DB", - "TABLE", - "PARTITION" -}; -const std::map _LockLevel_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kLockLevelValues, _kLockLevelNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const LockLevel::type& val) { - std::map::const_iterator it = _LockLevel_VALUES_TO_NAMES.find(val); - if (it != _LockLevel_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kLockStateValues[] = { - LockState::ACQUIRED, - LockState::WAITING, - LockState::ABORT, - LockState::NOT_ACQUIRED -}; -const char* _kLockStateNames[] = { - "ACQUIRED", - "WAITING", - "ABORT", - "NOT_ACQUIRED" -}; -const std::map _LockState_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(4, _kLockStateValues, _kLockStateNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const LockState::type& val) { - std::map::const_iterator it = _LockState_VALUES_TO_NAMES.find(val); - if (it != _LockState_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kLockTypeValues[] = { - LockType::SHARED_READ, - LockType::SHARED_WRITE, - LockType::EXCLUSIVE -}; -const char* _kLockTypeNames[] = { - "SHARED_READ", - "SHARED_WRITE", - "EXCLUSIVE" -}; -const std::map _LockType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kLockTypeValues, _kLockTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const LockType::type& val) { - std::map::const_iterator it = _LockType_VALUES_TO_NAMES.find(val); - if (it != _LockType_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kCompactionTypeValues[] = { - CompactionType::MINOR, - CompactionType::MAJOR -}; -const char* _kCompactionTypeNames[] = { - "MINOR", - "MAJOR" -}; -const std::map _CompactionType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(2, _kCompactionTypeValues, _kCompactionTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const CompactionType::type& val) { - std::map::const_iterator it = _CompactionType_VALUES_TO_NAMES.find(val); - if (it != _CompactionType_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kGrantRevokeTypeValues[] = { - GrantRevokeType::GRANT, - GrantRevokeType::REVOKE -}; -const char* _kGrantRevokeTypeNames[] = { - "GRANT", - "REVOKE" -}; -const std::map _GrantRevokeType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(2, _kGrantRevokeTypeValues, _kGrantRevokeTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const GrantRevokeType::type& val) { - std::map::const_iterator it = _GrantRevokeType_VALUES_TO_NAMES.find(val); - if (it != _GrantRevokeType_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kDataOperationTypeValues[] = { - DataOperationType::SELECT, - DataOperationType::INSERT, - DataOperationType::UPDATE, - DataOperationType::DELETE, - DataOperationType::UNSET, - DataOperationType::NO_TXN -}; -const char* _kDataOperationTypeNames[] = { - "SELECT", - "INSERT", - "UPDATE", - "DELETE", - "UNSET", - "NO_TXN" -}; -const std::map _DataOperationType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(6, _kDataOperationTypeValues, _kDataOperationTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const DataOperationType::type& val) { - std::map::const_iterator it = _DataOperationType_VALUES_TO_NAMES.find(val); - if (it != _DataOperationType_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kEventRequestTypeValues[] = { - EventRequestType::INSERT, - EventRequestType::UPDATE, - EventRequestType::DELETE -}; -const char* _kEventRequestTypeNames[] = { - "INSERT", - "UPDATE", - "DELETE" -}; -const std::map _EventRequestType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kEventRequestTypeValues, _kEventRequestTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const EventRequestType::type& val) { - std::map::const_iterator it = _EventRequestType_VALUES_TO_NAMES.find(val); - if (it != _EventRequestType_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kFunctionTypeValues[] = { - FunctionType::JAVA -}; -const char* _kFunctionTypeNames[] = { - "JAVA" -}; -const std::map _FunctionType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(1, _kFunctionTypeValues, _kFunctionTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const FunctionType::type& val) { - std::map::const_iterator it = _FunctionType_VALUES_TO_NAMES.find(val); - if (it != _FunctionType_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kResourceTypeValues[] = { - ResourceType::JAR, - ResourceType::FILE, - ResourceType::ARCHIVE -}; -const char* _kResourceTypeNames[] = { - "JAR", - "FILE", - "ARCHIVE" -}; -const std::map _ResourceType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kResourceTypeValues, _kResourceTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const ResourceType::type& val) { - std::map::const_iterator it = _ResourceType_VALUES_TO_NAMES.find(val); - if (it != _ResourceType_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kFileMetadataExprTypeValues[] = { - FileMetadataExprType::ORC_SARG -}; -const char* _kFileMetadataExprTypeNames[] = { - "ORC_SARG" -}; -const std::map _FileMetadataExprType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(1, _kFileMetadataExprTypeValues, _kFileMetadataExprTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const FileMetadataExprType::type& val) { - std::map::const_iterator it = _FileMetadataExprType_VALUES_TO_NAMES.find(val); - if (it != _FileMetadataExprType_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kClientCapabilityValues[] = { - ClientCapability::TEST_CAPABILITY, - ClientCapability::INSERT_ONLY_TABLES -}; -const char* _kClientCapabilityNames[] = { - "TEST_CAPABILITY", - "INSERT_ONLY_TABLES" -}; -const std::map _ClientCapability_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(2, _kClientCapabilityValues, _kClientCapabilityNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const ClientCapability::type& val) { - std::map::const_iterator it = _ClientCapability_VALUES_TO_NAMES.find(val); - if (it != _ClientCapability_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kWMResourcePlanStatusValues[] = { - WMResourcePlanStatus::ACTIVE, - WMResourcePlanStatus::ENABLED, - WMResourcePlanStatus::DISABLED -}; -const char* _kWMResourcePlanStatusNames[] = { - "ACTIVE", - "ENABLED", - "DISABLED" -}; -const std::map _WMResourcePlanStatus_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kWMResourcePlanStatusValues, _kWMResourcePlanStatusNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const WMResourcePlanStatus::type& val) { - std::map::const_iterator it = _WMResourcePlanStatus_VALUES_TO_NAMES.find(val); - if (it != _WMResourcePlanStatus_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - - -Version::~Version() noexcept { -} - - -void Version::__set_version(const std::string& val) { - this->version = val; -} - -void Version::__set_comments(const std::string& val) { - this->comments = val; -} -std::ostream& operator<<(std::ostream& out, const Version& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t Version::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->version); - this->__isset.version = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->comments); - this->__isset.comments = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t Version::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("Version"); - - xfer += oprot->writeFieldBegin("version", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->version); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("comments", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->comments); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(Version &a, Version &b) { - using ::std::swap; - swap(a.version, b.version); - swap(a.comments, b.comments); - swap(a.__isset, b.__isset); -} - -Version::Version(const Version& other0) { - version = other0.version; - comments = other0.comments; - __isset = other0.__isset; -} -Version& Version::operator=(const Version& other1) { - version = other1.version; - comments = other1.comments; - __isset = other1.__isset; - return *this; -} -void Version::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "Version("; - out << "version=" << to_string(version); - out << ", " << "comments=" << to_string(comments); - out << ")"; -} - - -FieldSchema::~FieldSchema() noexcept { -} - - -void FieldSchema::__set_name(const std::string& val) { - this->name = val; -} - -void FieldSchema::__set_type(const std::string& val) { - this->type = val; -} - -void FieldSchema::__set_comment(const std::string& val) { - this->comment = val; -} -std::ostream& operator<<(std::ostream& out, const FieldSchema& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t FieldSchema::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->name); - this->__isset.name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->type); - this->__isset.type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->comment); - this->__isset.comment = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t FieldSchema::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("FieldSchema"); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->type); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("comment", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->comment); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(FieldSchema &a, FieldSchema &b) { - using ::std::swap; - swap(a.name, b.name); - swap(a.type, b.type); - swap(a.comment, b.comment); - swap(a.__isset, b.__isset); -} - -FieldSchema::FieldSchema(const FieldSchema& other2) { - name = other2.name; - type = other2.type; - comment = other2.comment; - __isset = other2.__isset; -} -FieldSchema& FieldSchema::operator=(const FieldSchema& other3) { - name = other3.name; - type = other3.type; - comment = other3.comment; - __isset = other3.__isset; - return *this; -} -void FieldSchema::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "FieldSchema("; - out << "name=" << to_string(name); - out << ", " << "type=" << to_string(type); - out << ", " << "comment=" << to_string(comment); - out << ")"; -} - - -SQLPrimaryKey::~SQLPrimaryKey() noexcept { -} - - -void SQLPrimaryKey::__set_table_db(const std::string& val) { - this->table_db = val; -} - -void SQLPrimaryKey::__set_table_name(const std::string& val) { - this->table_name = val; -} - -void SQLPrimaryKey::__set_column_name(const std::string& val) { - this->column_name = val; -} - -void SQLPrimaryKey::__set_key_seq(const int32_t val) { - this->key_seq = val; -} - -void SQLPrimaryKey::__set_pk_name(const std::string& val) { - this->pk_name = val; -} - -void SQLPrimaryKey::__set_enable_cstr(const bool val) { - this->enable_cstr = val; -} - -void SQLPrimaryKey::__set_validate_cstr(const bool val) { - this->validate_cstr = val; -} - -void SQLPrimaryKey::__set_rely_cstr(const bool val) { - this->rely_cstr = val; -} -std::ostream& operator<<(std::ostream& out, const SQLPrimaryKey& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t SQLPrimaryKey::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->table_db); - this->__isset.table_db = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->table_name); - this->__isset.table_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->column_name); - this->__isset.column_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->key_seq); - this->__isset.key_seq = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->pk_name); - this->__isset.pk_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->enable_cstr); - this->__isset.enable_cstr = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->validate_cstr); - this->__isset.validate_cstr = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->rely_cstr); - this->__isset.rely_cstr = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t SQLPrimaryKey::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("SQLPrimaryKey"); - - xfer += oprot->writeFieldBegin("table_db", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->table_db); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->table_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("column_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->column_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("key_seq", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32(this->key_seq); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("pk_name", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString(this->pk_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 6); - xfer += oprot->writeBool(this->enable_cstr); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 7); - xfer += oprot->writeBool(this->validate_cstr); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 8); - xfer += oprot->writeBool(this->rely_cstr); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(SQLPrimaryKey &a, SQLPrimaryKey &b) { - using ::std::swap; - swap(a.table_db, b.table_db); - swap(a.table_name, b.table_name); - swap(a.column_name, b.column_name); - swap(a.key_seq, b.key_seq); - swap(a.pk_name, b.pk_name); - swap(a.enable_cstr, b.enable_cstr); - swap(a.validate_cstr, b.validate_cstr); - swap(a.rely_cstr, b.rely_cstr); - swap(a.__isset, b.__isset); -} - -SQLPrimaryKey::SQLPrimaryKey(const SQLPrimaryKey& other4) { - table_db = other4.table_db; - table_name = other4.table_name; - column_name = other4.column_name; - key_seq = other4.key_seq; - pk_name = other4.pk_name; - enable_cstr = other4.enable_cstr; - validate_cstr = other4.validate_cstr; - rely_cstr = other4.rely_cstr; - __isset = other4.__isset; -} -SQLPrimaryKey& SQLPrimaryKey::operator=(const SQLPrimaryKey& other5) { - table_db = other5.table_db; - table_name = other5.table_name; - column_name = other5.column_name; - key_seq = other5.key_seq; - pk_name = other5.pk_name; - enable_cstr = other5.enable_cstr; - validate_cstr = other5.validate_cstr; - rely_cstr = other5.rely_cstr; - __isset = other5.__isset; - return *this; -} -void SQLPrimaryKey::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "SQLPrimaryKey("; - out << "table_db=" << to_string(table_db); - out << ", " << "table_name=" << to_string(table_name); - out << ", " << "column_name=" << to_string(column_name); - out << ", " << "key_seq=" << to_string(key_seq); - out << ", " << "pk_name=" << to_string(pk_name); - out << ", " << "enable_cstr=" << to_string(enable_cstr); - out << ", " << "validate_cstr=" << to_string(validate_cstr); - out << ", " << "rely_cstr=" << to_string(rely_cstr); - out << ")"; -} - - -SQLForeignKey::~SQLForeignKey() noexcept { -} - - -void SQLForeignKey::__set_pktable_db(const std::string& val) { - this->pktable_db = val; -} - -void SQLForeignKey::__set_pktable_name(const std::string& val) { - this->pktable_name = val; -} - -void SQLForeignKey::__set_pkcolumn_name(const std::string& val) { - this->pkcolumn_name = val; -} - -void SQLForeignKey::__set_fktable_db(const std::string& val) { - this->fktable_db = val; -} - -void SQLForeignKey::__set_fktable_name(const std::string& val) { - this->fktable_name = val; -} - -void SQLForeignKey::__set_fkcolumn_name(const std::string& val) { - this->fkcolumn_name = val; -} - -void SQLForeignKey::__set_key_seq(const int32_t val) { - this->key_seq = val; -} - -void SQLForeignKey::__set_update_rule(const int32_t val) { - this->update_rule = val; -} - -void SQLForeignKey::__set_delete_rule(const int32_t val) { - this->delete_rule = val; -} - -void SQLForeignKey::__set_fk_name(const std::string& val) { - this->fk_name = val; -} - -void SQLForeignKey::__set_pk_name(const std::string& val) { - this->pk_name = val; -} - -void SQLForeignKey::__set_enable_cstr(const bool val) { - this->enable_cstr = val; -} - -void SQLForeignKey::__set_validate_cstr(const bool val) { - this->validate_cstr = val; -} - -void SQLForeignKey::__set_rely_cstr(const bool val) { - this->rely_cstr = val; -} -std::ostream& operator<<(std::ostream& out, const SQLForeignKey& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t SQLForeignKey::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->pktable_db); - this->__isset.pktable_db = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->pktable_name); - this->__isset.pktable_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->pkcolumn_name); - this->__isset.pkcolumn_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->fktable_db); - this->__isset.fktable_db = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->fktable_name); - this->__isset.fktable_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->fkcolumn_name); - this->__isset.fkcolumn_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->key_seq); - this->__isset.key_seq = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->update_rule); - this->__isset.update_rule = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 9: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->delete_rule); - this->__isset.delete_rule = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 10: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->fk_name); - this->__isset.fk_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 11: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->pk_name); - this->__isset.pk_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 12: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->enable_cstr); - this->__isset.enable_cstr = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 13: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->validate_cstr); - this->__isset.validate_cstr = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 14: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->rely_cstr); - this->__isset.rely_cstr = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t SQLForeignKey::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("SQLForeignKey"); - - xfer += oprot->writeFieldBegin("pktable_db", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->pktable_db); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("pktable_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->pktable_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("pkcolumn_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->pkcolumn_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("fktable_db", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->fktable_db); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("fktable_name", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString(this->fktable_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("fkcolumn_name", ::apache::thrift::protocol::T_STRING, 6); - xfer += oprot->writeString(this->fkcolumn_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("key_seq", ::apache::thrift::protocol::T_I32, 7); - xfer += oprot->writeI32(this->key_seq); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("update_rule", ::apache::thrift::protocol::T_I32, 8); - xfer += oprot->writeI32(this->update_rule); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("delete_rule", ::apache::thrift::protocol::T_I32, 9); - xfer += oprot->writeI32(this->delete_rule); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("fk_name", ::apache::thrift::protocol::T_STRING, 10); - xfer += oprot->writeString(this->fk_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("pk_name", ::apache::thrift::protocol::T_STRING, 11); - xfer += oprot->writeString(this->pk_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 12); - xfer += oprot->writeBool(this->enable_cstr); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 13); - xfer += oprot->writeBool(this->validate_cstr); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 14); - xfer += oprot->writeBool(this->rely_cstr); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(SQLForeignKey &a, SQLForeignKey &b) { - using ::std::swap; - swap(a.pktable_db, b.pktable_db); - swap(a.pktable_name, b.pktable_name); - swap(a.pkcolumn_name, b.pkcolumn_name); - swap(a.fktable_db, b.fktable_db); - swap(a.fktable_name, b.fktable_name); - swap(a.fkcolumn_name, b.fkcolumn_name); - swap(a.key_seq, b.key_seq); - swap(a.update_rule, b.update_rule); - swap(a.delete_rule, b.delete_rule); - swap(a.fk_name, b.fk_name); - swap(a.pk_name, b.pk_name); - swap(a.enable_cstr, b.enable_cstr); - swap(a.validate_cstr, b.validate_cstr); - swap(a.rely_cstr, b.rely_cstr); - swap(a.__isset, b.__isset); -} - -SQLForeignKey::SQLForeignKey(const SQLForeignKey& other6) { - pktable_db = other6.pktable_db; - pktable_name = other6.pktable_name; - pkcolumn_name = other6.pkcolumn_name; - fktable_db = other6.fktable_db; - fktable_name = other6.fktable_name; - fkcolumn_name = other6.fkcolumn_name; - key_seq = other6.key_seq; - update_rule = other6.update_rule; - delete_rule = other6.delete_rule; - fk_name = other6.fk_name; - pk_name = other6.pk_name; - enable_cstr = other6.enable_cstr; - validate_cstr = other6.validate_cstr; - rely_cstr = other6.rely_cstr; - __isset = other6.__isset; -} -SQLForeignKey& SQLForeignKey::operator=(const SQLForeignKey& other7) { - pktable_db = other7.pktable_db; - pktable_name = other7.pktable_name; - pkcolumn_name = other7.pkcolumn_name; - fktable_db = other7.fktable_db; - fktable_name = other7.fktable_name; - fkcolumn_name = other7.fkcolumn_name; - key_seq = other7.key_seq; - update_rule = other7.update_rule; - delete_rule = other7.delete_rule; - fk_name = other7.fk_name; - pk_name = other7.pk_name; - enable_cstr = other7.enable_cstr; - validate_cstr = other7.validate_cstr; - rely_cstr = other7.rely_cstr; - __isset = other7.__isset; - return *this; -} -void SQLForeignKey::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "SQLForeignKey("; - out << "pktable_db=" << to_string(pktable_db); - out << ", " << "pktable_name=" << to_string(pktable_name); - out << ", " << "pkcolumn_name=" << to_string(pkcolumn_name); - out << ", " << "fktable_db=" << to_string(fktable_db); - out << ", " << "fktable_name=" << to_string(fktable_name); - out << ", " << "fkcolumn_name=" << to_string(fkcolumn_name); - out << ", " << "key_seq=" << to_string(key_seq); - out << ", " << "update_rule=" << to_string(update_rule); - out << ", " << "delete_rule=" << to_string(delete_rule); - out << ", " << "fk_name=" << to_string(fk_name); - out << ", " << "pk_name=" << to_string(pk_name); - out << ", " << "enable_cstr=" << to_string(enable_cstr); - out << ", " << "validate_cstr=" << to_string(validate_cstr); - out << ", " << "rely_cstr=" << to_string(rely_cstr); - out << ")"; -} - - -SQLUniqueConstraint::~SQLUniqueConstraint() noexcept { -} - - -void SQLUniqueConstraint::__set_table_db(const std::string& val) { - this->table_db = val; -} - -void SQLUniqueConstraint::__set_table_name(const std::string& val) { - this->table_name = val; -} - -void SQLUniqueConstraint::__set_column_name(const std::string& val) { - this->column_name = val; -} - -void SQLUniqueConstraint::__set_key_seq(const int32_t val) { - this->key_seq = val; -} - -void SQLUniqueConstraint::__set_uk_name(const std::string& val) { - this->uk_name = val; -} - -void SQLUniqueConstraint::__set_enable_cstr(const bool val) { - this->enable_cstr = val; -} - -void SQLUniqueConstraint::__set_validate_cstr(const bool val) { - this->validate_cstr = val; -} - -void SQLUniqueConstraint::__set_rely_cstr(const bool val) { - this->rely_cstr = val; -} -std::ostream& operator<<(std::ostream& out, const SQLUniqueConstraint& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t SQLUniqueConstraint::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->table_db); - this->__isset.table_db = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->table_name); - this->__isset.table_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->column_name); - this->__isset.column_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->key_seq); - this->__isset.key_seq = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->uk_name); - this->__isset.uk_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->enable_cstr); - this->__isset.enable_cstr = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->validate_cstr); - this->__isset.validate_cstr = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->rely_cstr); - this->__isset.rely_cstr = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t SQLUniqueConstraint::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("SQLUniqueConstraint"); - - xfer += oprot->writeFieldBegin("table_db", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->table_db); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->table_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("column_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->column_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("key_seq", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32(this->key_seq); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("uk_name", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString(this->uk_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 6); - xfer += oprot->writeBool(this->enable_cstr); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 7); - xfer += oprot->writeBool(this->validate_cstr); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 8); - xfer += oprot->writeBool(this->rely_cstr); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(SQLUniqueConstraint &a, SQLUniqueConstraint &b) { - using ::std::swap; - swap(a.table_db, b.table_db); - swap(a.table_name, b.table_name); - swap(a.column_name, b.column_name); - swap(a.key_seq, b.key_seq); - swap(a.uk_name, b.uk_name); - swap(a.enable_cstr, b.enable_cstr); - swap(a.validate_cstr, b.validate_cstr); - swap(a.rely_cstr, b.rely_cstr); - swap(a.__isset, b.__isset); -} - -SQLUniqueConstraint::SQLUniqueConstraint(const SQLUniqueConstraint& other8) { - table_db = other8.table_db; - table_name = other8.table_name; - column_name = other8.column_name; - key_seq = other8.key_seq; - uk_name = other8.uk_name; - enable_cstr = other8.enable_cstr; - validate_cstr = other8.validate_cstr; - rely_cstr = other8.rely_cstr; - __isset = other8.__isset; -} -SQLUniqueConstraint& SQLUniqueConstraint::operator=(const SQLUniqueConstraint& other9) { - table_db = other9.table_db; - table_name = other9.table_name; - column_name = other9.column_name; - key_seq = other9.key_seq; - uk_name = other9.uk_name; - enable_cstr = other9.enable_cstr; - validate_cstr = other9.validate_cstr; - rely_cstr = other9.rely_cstr; - __isset = other9.__isset; - return *this; -} -void SQLUniqueConstraint::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "SQLUniqueConstraint("; - out << "table_db=" << to_string(table_db); - out << ", " << "table_name=" << to_string(table_name); - out << ", " << "column_name=" << to_string(column_name); - out << ", " << "key_seq=" << to_string(key_seq); - out << ", " << "uk_name=" << to_string(uk_name); - out << ", " << "enable_cstr=" << to_string(enable_cstr); - out << ", " << "validate_cstr=" << to_string(validate_cstr); - out << ", " << "rely_cstr=" << to_string(rely_cstr); - out << ")"; -} - - -SQLNotNullConstraint::~SQLNotNullConstraint() noexcept { -} - - -void SQLNotNullConstraint::__set_table_db(const std::string& val) { - this->table_db = val; -} - -void SQLNotNullConstraint::__set_table_name(const std::string& val) { - this->table_name = val; -} - -void SQLNotNullConstraint::__set_column_name(const std::string& val) { - this->column_name = val; -} - -void SQLNotNullConstraint::__set_nn_name(const std::string& val) { - this->nn_name = val; -} - -void SQLNotNullConstraint::__set_enable_cstr(const bool val) { - this->enable_cstr = val; -} - -void SQLNotNullConstraint::__set_validate_cstr(const bool val) { - this->validate_cstr = val; -} - -void SQLNotNullConstraint::__set_rely_cstr(const bool val) { - this->rely_cstr = val; -} -std::ostream& operator<<(std::ostream& out, const SQLNotNullConstraint& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t SQLNotNullConstraint::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->table_db); - this->__isset.table_db = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->table_name); - this->__isset.table_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->column_name); - this->__isset.column_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->nn_name); - this->__isset.nn_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->enable_cstr); - this->__isset.enable_cstr = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->validate_cstr); - this->__isset.validate_cstr = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->rely_cstr); - this->__isset.rely_cstr = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t SQLNotNullConstraint::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("SQLNotNullConstraint"); - - xfer += oprot->writeFieldBegin("table_db", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->table_db); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->table_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("column_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->column_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("nn_name", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->nn_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 5); - xfer += oprot->writeBool(this->enable_cstr); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 6); - xfer += oprot->writeBool(this->validate_cstr); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 7); - xfer += oprot->writeBool(this->rely_cstr); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(SQLNotNullConstraint &a, SQLNotNullConstraint &b) { - using ::std::swap; - swap(a.table_db, b.table_db); - swap(a.table_name, b.table_name); - swap(a.column_name, b.column_name); - swap(a.nn_name, b.nn_name); - swap(a.enable_cstr, b.enable_cstr); - swap(a.validate_cstr, b.validate_cstr); - swap(a.rely_cstr, b.rely_cstr); - swap(a.__isset, b.__isset); -} - -SQLNotNullConstraint::SQLNotNullConstraint(const SQLNotNullConstraint& other10) { - table_db = other10.table_db; - table_name = other10.table_name; - column_name = other10.column_name; - nn_name = other10.nn_name; - enable_cstr = other10.enable_cstr; - validate_cstr = other10.validate_cstr; - rely_cstr = other10.rely_cstr; - __isset = other10.__isset; -} -SQLNotNullConstraint& SQLNotNullConstraint::operator=(const SQLNotNullConstraint& other11) { - table_db = other11.table_db; - table_name = other11.table_name; - column_name = other11.column_name; - nn_name = other11.nn_name; - enable_cstr = other11.enable_cstr; - validate_cstr = other11.validate_cstr; - rely_cstr = other11.rely_cstr; - __isset = other11.__isset; - return *this; -} -void SQLNotNullConstraint::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "SQLNotNullConstraint("; - out << "table_db=" << to_string(table_db); - out << ", " << "table_name=" << to_string(table_name); - out << ", " << "column_name=" << to_string(column_name); - out << ", " << "nn_name=" << to_string(nn_name); - out << ", " << "enable_cstr=" << to_string(enable_cstr); - out << ", " << "validate_cstr=" << to_string(validate_cstr); - out << ", " << "rely_cstr=" << to_string(rely_cstr); - out << ")"; -} - - -Type::~Type() noexcept { -} - - -void Type::__set_name(const std::string& val) { - this->name = val; -} - -void Type::__set_type1(const std::string& val) { - this->type1 = val; -__isset.type1 = true; -} - -void Type::__set_type2(const std::string& val) { - this->type2 = val; -__isset.type2 = true; -} - -void Type::__set_fields(const std::vector & val) { - this->fields = val; -__isset.fields = true; -} -std::ostream& operator<<(std::ostream& out, const Type& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t Type::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->name); - this->__isset.name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->type1); - this->__isset.type1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->type2); - this->__isset.type2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->fields.clear(); - uint32_t _size12; - ::apache::thrift::protocol::TType _etype15; - xfer += iprot->readListBegin(_etype15, _size12); - this->fields.resize(_size12); - uint32_t _i16; - for (_i16 = 0; _i16 < _size12; ++_i16) - { - xfer += this->fields[_i16].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.fields = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t Type::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("Type"); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->name); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.type1) { - xfer += oprot->writeFieldBegin("type1", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->type1); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.type2) { - xfer += oprot->writeFieldBegin("type2", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->type2); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.fields) { - xfer += oprot->writeFieldBegin("fields", ::apache::thrift::protocol::T_LIST, 4); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->fields.size())); - std::vector ::const_iterator _iter17; - for (_iter17 = this->fields.begin(); _iter17 != this->fields.end(); ++_iter17) - { - xfer += (*_iter17).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(Type &a, Type &b) { - using ::std::swap; - swap(a.name, b.name); - swap(a.type1, b.type1); - swap(a.type2, b.type2); - swap(a.fields, b.fields); - swap(a.__isset, b.__isset); -} - -Type::Type(const Type& other18) { - name = other18.name; - type1 = other18.type1; - type2 = other18.type2; - fields = other18.fields; - __isset = other18.__isset; -} -Type& Type::operator=(const Type& other19) { - name = other19.name; - type1 = other19.type1; - type2 = other19.type2; - fields = other19.fields; - __isset = other19.__isset; - return *this; -} -void Type::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "Type("; - out << "name=" << to_string(name); - out << ", " << "type1="; (__isset.type1 ? (out << to_string(type1)) : (out << "")); - out << ", " << "type2="; (__isset.type2 ? (out << to_string(type2)) : (out << "")); - out << ", " << "fields="; (__isset.fields ? (out << to_string(fields)) : (out << "")); - out << ")"; -} - - -HiveObjectRef::~HiveObjectRef() noexcept { -} - - -void HiveObjectRef::__set_objectType(const HiveObjectType::type val) { - this->objectType = val; -} - -void HiveObjectRef::__set_dbName(const std::string& val) { - this->dbName = val; -} - -void HiveObjectRef::__set_objectName(const std::string& val) { - this->objectName = val; -} - -void HiveObjectRef::__set_partValues(const std::vector & val) { - this->partValues = val; -} - -void HiveObjectRef::__set_columnName(const std::string& val) { - this->columnName = val; -} -std::ostream& operator<<(std::ostream& out, const HiveObjectRef& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t HiveObjectRef::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast20; - xfer += iprot->readI32(ecast20); - this->objectType = (HiveObjectType::type)ecast20; - this->__isset.objectType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbName); - this->__isset.dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->objectName); - this->__isset.objectName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->partValues.clear(); - uint32_t _size21; - ::apache::thrift::protocol::TType _etype24; - xfer += iprot->readListBegin(_etype24, _size21); - this->partValues.resize(_size21); - uint32_t _i25; - for (_i25 = 0; _i25 < _size21; ++_i25) - { - xfer += iprot->readString(this->partValues[_i25]); - } - xfer += iprot->readListEnd(); - } - this->__isset.partValues = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->columnName); - this->__isset.columnName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t HiveObjectRef::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("HiveObjectRef"); - - xfer += oprot->writeFieldBegin("objectType", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32((int32_t)this->objectType); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("objectName", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->objectName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("partValues", ::apache::thrift::protocol::T_LIST, 4); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partValues.size())); - std::vector ::const_iterator _iter26; - for (_iter26 = this->partValues.begin(); _iter26 != this->partValues.end(); ++_iter26) - { - xfer += oprot->writeString((*_iter26)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("columnName", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString(this->columnName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(HiveObjectRef &a, HiveObjectRef &b) { - using ::std::swap; - swap(a.objectType, b.objectType); - swap(a.dbName, b.dbName); - swap(a.objectName, b.objectName); - swap(a.partValues, b.partValues); - swap(a.columnName, b.columnName); - swap(a.__isset, b.__isset); -} - -HiveObjectRef::HiveObjectRef(const HiveObjectRef& other27) { - objectType = other27.objectType; - dbName = other27.dbName; - objectName = other27.objectName; - partValues = other27.partValues; - columnName = other27.columnName; - __isset = other27.__isset; -} -HiveObjectRef& HiveObjectRef::operator=(const HiveObjectRef& other28) { - objectType = other28.objectType; - dbName = other28.dbName; - objectName = other28.objectName; - partValues = other28.partValues; - columnName = other28.columnName; - __isset = other28.__isset; - return *this; -} -void HiveObjectRef::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "HiveObjectRef("; - out << "objectType=" << to_string(objectType); - out << ", " << "dbName=" << to_string(dbName); - out << ", " << "objectName=" << to_string(objectName); - out << ", " << "partValues=" << to_string(partValues); - out << ", " << "columnName=" << to_string(columnName); - out << ")"; -} - - -PrivilegeGrantInfo::~PrivilegeGrantInfo() noexcept { -} - - -void PrivilegeGrantInfo::__set_privilege(const std::string& val) { - this->privilege = val; -} - -void PrivilegeGrantInfo::__set_createTime(const int32_t val) { - this->createTime = val; -} - -void PrivilegeGrantInfo::__set_grantor(const std::string& val) { - this->grantor = val; -} - -void PrivilegeGrantInfo::__set_grantorType(const PrincipalType::type val) { - this->grantorType = val; -} - -void PrivilegeGrantInfo::__set_grantOption(const bool val) { - this->grantOption = val; -} -std::ostream& operator<<(std::ostream& out, const PrivilegeGrantInfo& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PrivilegeGrantInfo::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->privilege); - this->__isset.privilege = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->createTime); - this->__isset.createTime = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->grantor); - this->__isset.grantor = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast29; - xfer += iprot->readI32(ecast29); - this->grantorType = (PrincipalType::type)ecast29; - this->__isset.grantorType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->grantOption); - this->__isset.grantOption = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t PrivilegeGrantInfo::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PrivilegeGrantInfo"); - - xfer += oprot->writeFieldBegin("privilege", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->privilege); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32(this->createTime); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("grantor", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->grantor); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("grantorType", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32((int32_t)this->grantorType); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("grantOption", ::apache::thrift::protocol::T_BOOL, 5); - xfer += oprot->writeBool(this->grantOption); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PrivilegeGrantInfo &a, PrivilegeGrantInfo &b) { - using ::std::swap; - swap(a.privilege, b.privilege); - swap(a.createTime, b.createTime); - swap(a.grantor, b.grantor); - swap(a.grantorType, b.grantorType); - swap(a.grantOption, b.grantOption); - swap(a.__isset, b.__isset); -} - -PrivilegeGrantInfo::PrivilegeGrantInfo(const PrivilegeGrantInfo& other30) { - privilege = other30.privilege; - createTime = other30.createTime; - grantor = other30.grantor; - grantorType = other30.grantorType; - grantOption = other30.grantOption; - __isset = other30.__isset; -} -PrivilegeGrantInfo& PrivilegeGrantInfo::operator=(const PrivilegeGrantInfo& other31) { - privilege = other31.privilege; - createTime = other31.createTime; - grantor = other31.grantor; - grantorType = other31.grantorType; - grantOption = other31.grantOption; - __isset = other31.__isset; - return *this; -} -void PrivilegeGrantInfo::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PrivilegeGrantInfo("; - out << "privilege=" << to_string(privilege); - out << ", " << "createTime=" << to_string(createTime); - out << ", " << "grantor=" << to_string(grantor); - out << ", " << "grantorType=" << to_string(grantorType); - out << ", " << "grantOption=" << to_string(grantOption); - out << ")"; -} - - -HiveObjectPrivilege::~HiveObjectPrivilege() noexcept { -} - - -void HiveObjectPrivilege::__set_hiveObject(const HiveObjectRef& val) { - this->hiveObject = val; -} - -void HiveObjectPrivilege::__set_principalName(const std::string& val) { - this->principalName = val; -} - -void HiveObjectPrivilege::__set_principalType(const PrincipalType::type val) { - this->principalType = val; -} - -void HiveObjectPrivilege::__set_grantInfo(const PrivilegeGrantInfo& val) { - this->grantInfo = val; -} -std::ostream& operator<<(std::ostream& out, const HiveObjectPrivilege& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t HiveObjectPrivilege::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->hiveObject.read(iprot); - this->__isset.hiveObject = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->principalName); - this->__isset.principalName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast32; - xfer += iprot->readI32(ecast32); - this->principalType = (PrincipalType::type)ecast32; - this->__isset.principalType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->grantInfo.read(iprot); - this->__isset.grantInfo = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t HiveObjectPrivilege::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("HiveObjectPrivilege"); - - xfer += oprot->writeFieldBegin("hiveObject", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->hiveObject.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principalName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->principalName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principalType", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32((int32_t)this->principalType); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("grantInfo", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->grantInfo.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(HiveObjectPrivilege &a, HiveObjectPrivilege &b) { - using ::std::swap; - swap(a.hiveObject, b.hiveObject); - swap(a.principalName, b.principalName); - swap(a.principalType, b.principalType); - swap(a.grantInfo, b.grantInfo); - swap(a.__isset, b.__isset); -} - -HiveObjectPrivilege::HiveObjectPrivilege(const HiveObjectPrivilege& other33) { - hiveObject = other33.hiveObject; - principalName = other33.principalName; - principalType = other33.principalType; - grantInfo = other33.grantInfo; - __isset = other33.__isset; -} -HiveObjectPrivilege& HiveObjectPrivilege::operator=(const HiveObjectPrivilege& other34) { - hiveObject = other34.hiveObject; - principalName = other34.principalName; - principalType = other34.principalType; - grantInfo = other34.grantInfo; - __isset = other34.__isset; - return *this; -} -void HiveObjectPrivilege::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "HiveObjectPrivilege("; - out << "hiveObject=" << to_string(hiveObject); - out << ", " << "principalName=" << to_string(principalName); - out << ", " << "principalType=" << to_string(principalType); - out << ", " << "grantInfo=" << to_string(grantInfo); - out << ")"; -} - - -PrivilegeBag::~PrivilegeBag() noexcept { -} - - -void PrivilegeBag::__set_privileges(const std::vector & val) { - this->privileges = val; -} -std::ostream& operator<<(std::ostream& out, const PrivilegeBag& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PrivilegeBag::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->privileges.clear(); - uint32_t _size35; - ::apache::thrift::protocol::TType _etype38; - xfer += iprot->readListBegin(_etype38, _size35); - this->privileges.resize(_size35); - uint32_t _i39; - for (_i39 = 0; _i39 < _size35; ++_i39) - { - xfer += this->privileges[_i39].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.privileges = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t PrivilegeBag::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PrivilegeBag"); - - xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->privileges.size())); - std::vector ::const_iterator _iter40; - for (_iter40 = this->privileges.begin(); _iter40 != this->privileges.end(); ++_iter40) - { - xfer += (*_iter40).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PrivilegeBag &a, PrivilegeBag &b) { - using ::std::swap; - swap(a.privileges, b.privileges); - swap(a.__isset, b.__isset); -} - -PrivilegeBag::PrivilegeBag(const PrivilegeBag& other41) { - privileges = other41.privileges; - __isset = other41.__isset; -} -PrivilegeBag& PrivilegeBag::operator=(const PrivilegeBag& other42) { - privileges = other42.privileges; - __isset = other42.__isset; - return *this; -} -void PrivilegeBag::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PrivilegeBag("; - out << "privileges=" << to_string(privileges); - out << ")"; -} - - -PrincipalPrivilegeSet::~PrincipalPrivilegeSet() noexcept { -} - - -void PrincipalPrivilegeSet::__set_userPrivileges(const std::map > & val) { - this->userPrivileges = val; -} - -void PrincipalPrivilegeSet::__set_groupPrivileges(const std::map > & val) { - this->groupPrivileges = val; -} - -void PrincipalPrivilegeSet::__set_rolePrivileges(const std::map > & val) { - this->rolePrivileges = val; -} -std::ostream& operator<<(std::ostream& out, const PrincipalPrivilegeSet& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PrincipalPrivilegeSet::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->userPrivileges.clear(); - uint32_t _size43; - ::apache::thrift::protocol::TType _ktype44; - ::apache::thrift::protocol::TType _vtype45; - xfer += iprot->readMapBegin(_ktype44, _vtype45, _size43); - uint32_t _i47; - for (_i47 = 0; _i47 < _size43; ++_i47) - { - std::string _key48; - xfer += iprot->readString(_key48); - std::vector & _val49 = this->userPrivileges[_key48]; - { - _val49.clear(); - uint32_t _size50; - ::apache::thrift::protocol::TType _etype53; - xfer += iprot->readListBegin(_etype53, _size50); - _val49.resize(_size50); - uint32_t _i54; - for (_i54 = 0; _i54 < _size50; ++_i54) - { - xfer += _val49[_i54].read(iprot); - } - xfer += iprot->readListEnd(); - } - } - xfer += iprot->readMapEnd(); - } - this->__isset.userPrivileges = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->groupPrivileges.clear(); - uint32_t _size55; - ::apache::thrift::protocol::TType _ktype56; - ::apache::thrift::protocol::TType _vtype57; - xfer += iprot->readMapBegin(_ktype56, _vtype57, _size55); - uint32_t _i59; - for (_i59 = 0; _i59 < _size55; ++_i59) - { - std::string _key60; - xfer += iprot->readString(_key60); - std::vector & _val61 = this->groupPrivileges[_key60]; - { - _val61.clear(); - uint32_t _size62; - ::apache::thrift::protocol::TType _etype65; - xfer += iprot->readListBegin(_etype65, _size62); - _val61.resize(_size62); - uint32_t _i66; - for (_i66 = 0; _i66 < _size62; ++_i66) - { - xfer += _val61[_i66].read(iprot); - } - xfer += iprot->readListEnd(); - } - } - xfer += iprot->readMapEnd(); - } - this->__isset.groupPrivileges = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->rolePrivileges.clear(); - uint32_t _size67; - ::apache::thrift::protocol::TType _ktype68; - ::apache::thrift::protocol::TType _vtype69; - xfer += iprot->readMapBegin(_ktype68, _vtype69, _size67); - uint32_t _i71; - for (_i71 = 0; _i71 < _size67; ++_i71) - { - std::string _key72; - xfer += iprot->readString(_key72); - std::vector & _val73 = this->rolePrivileges[_key72]; - { - _val73.clear(); - uint32_t _size74; - ::apache::thrift::protocol::TType _etype77; - xfer += iprot->readListBegin(_etype77, _size74); - _val73.resize(_size74); - uint32_t _i78; - for (_i78 = 0; _i78 < _size74; ++_i78) - { - xfer += _val73[_i78].read(iprot); - } - xfer += iprot->readListEnd(); - } - } - xfer += iprot->readMapEnd(); - } - this->__isset.rolePrivileges = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t PrincipalPrivilegeSet::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PrincipalPrivilegeSet"); - - xfer += oprot->writeFieldBegin("userPrivileges", ::apache::thrift::protocol::T_MAP, 1); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast(this->userPrivileges.size())); - std::map > ::const_iterator _iter79; - for (_iter79 = this->userPrivileges.begin(); _iter79 != this->userPrivileges.end(); ++_iter79) - { - xfer += oprot->writeString(_iter79->first); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(_iter79->second.size())); - std::vector ::const_iterator _iter80; - for (_iter80 = _iter79->second.begin(); _iter80 != _iter79->second.end(); ++_iter80) - { - xfer += (*_iter80).write(oprot); - } - xfer += oprot->writeListEnd(); - } - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("groupPrivileges", ::apache::thrift::protocol::T_MAP, 2); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast(this->groupPrivileges.size())); - std::map > ::const_iterator _iter81; - for (_iter81 = this->groupPrivileges.begin(); _iter81 != this->groupPrivileges.end(); ++_iter81) - { - xfer += oprot->writeString(_iter81->first); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(_iter81->second.size())); - std::vector ::const_iterator _iter82; - for (_iter82 = _iter81->second.begin(); _iter82 != _iter81->second.end(); ++_iter82) - { - xfer += (*_iter82).write(oprot); - } - xfer += oprot->writeListEnd(); - } - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("rolePrivileges", ::apache::thrift::protocol::T_MAP, 3); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast(this->rolePrivileges.size())); - std::map > ::const_iterator _iter83; - for (_iter83 = this->rolePrivileges.begin(); _iter83 != this->rolePrivileges.end(); ++_iter83) - { - xfer += oprot->writeString(_iter83->first); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(_iter83->second.size())); - std::vector ::const_iterator _iter84; - for (_iter84 = _iter83->second.begin(); _iter84 != _iter83->second.end(); ++_iter84) - { - xfer += (*_iter84).write(oprot); - } - xfer += oprot->writeListEnd(); - } - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PrincipalPrivilegeSet &a, PrincipalPrivilegeSet &b) { - using ::std::swap; - swap(a.userPrivileges, b.userPrivileges); - swap(a.groupPrivileges, b.groupPrivileges); - swap(a.rolePrivileges, b.rolePrivileges); - swap(a.__isset, b.__isset); -} - -PrincipalPrivilegeSet::PrincipalPrivilegeSet(const PrincipalPrivilegeSet& other85) { - userPrivileges = other85.userPrivileges; - groupPrivileges = other85.groupPrivileges; - rolePrivileges = other85.rolePrivileges; - __isset = other85.__isset; -} -PrincipalPrivilegeSet& PrincipalPrivilegeSet::operator=(const PrincipalPrivilegeSet& other86) { - userPrivileges = other86.userPrivileges; - groupPrivileges = other86.groupPrivileges; - rolePrivileges = other86.rolePrivileges; - __isset = other86.__isset; - return *this; -} -void PrincipalPrivilegeSet::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PrincipalPrivilegeSet("; - out << "userPrivileges=" << to_string(userPrivileges); - out << ", " << "groupPrivileges=" << to_string(groupPrivileges); - out << ", " << "rolePrivileges=" << to_string(rolePrivileges); - out << ")"; -} - - -GrantRevokePrivilegeRequest::~GrantRevokePrivilegeRequest() noexcept { -} - - -void GrantRevokePrivilegeRequest::__set_requestType(const GrantRevokeType::type val) { - this->requestType = val; -} - -void GrantRevokePrivilegeRequest::__set_privileges(const PrivilegeBag& val) { - this->privileges = val; -} - -void GrantRevokePrivilegeRequest::__set_revokeGrantOption(const bool val) { - this->revokeGrantOption = val; -__isset.revokeGrantOption = true; -} -std::ostream& operator<<(std::ostream& out, const GrantRevokePrivilegeRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GrantRevokePrivilegeRequest::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast87; - xfer += iprot->readI32(ecast87); - this->requestType = (GrantRevokeType::type)ecast87; - this->__isset.requestType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->privileges.read(iprot); - this->__isset.privileges = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->revokeGrantOption); - this->__isset.revokeGrantOption = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t GrantRevokePrivilegeRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GrantRevokePrivilegeRequest"); - - xfer += oprot->writeFieldBegin("requestType", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32((int32_t)this->requestType); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->privileges.write(oprot); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.revokeGrantOption) { - xfer += oprot->writeFieldBegin("revokeGrantOption", ::apache::thrift::protocol::T_BOOL, 3); - xfer += oprot->writeBool(this->revokeGrantOption); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GrantRevokePrivilegeRequest &a, GrantRevokePrivilegeRequest &b) { - using ::std::swap; - swap(a.requestType, b.requestType); - swap(a.privileges, b.privileges); - swap(a.revokeGrantOption, b.revokeGrantOption); - swap(a.__isset, b.__isset); -} - -GrantRevokePrivilegeRequest::GrantRevokePrivilegeRequest(const GrantRevokePrivilegeRequest& other88) { - requestType = other88.requestType; - privileges = other88.privileges; - revokeGrantOption = other88.revokeGrantOption; - __isset = other88.__isset; -} -GrantRevokePrivilegeRequest& GrantRevokePrivilegeRequest::operator=(const GrantRevokePrivilegeRequest& other89) { - requestType = other89.requestType; - privileges = other89.privileges; - revokeGrantOption = other89.revokeGrantOption; - __isset = other89.__isset; - return *this; -} -void GrantRevokePrivilegeRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GrantRevokePrivilegeRequest("; - out << "requestType=" << to_string(requestType); - out << ", " << "privileges=" << to_string(privileges); - out << ", " << "revokeGrantOption="; (__isset.revokeGrantOption ? (out << to_string(revokeGrantOption)) : (out << "")); - out << ")"; -} - - -GrantRevokePrivilegeResponse::~GrantRevokePrivilegeResponse() noexcept { -} - - -void GrantRevokePrivilegeResponse::__set_success(const bool val) { - this->success = val; -__isset.success = true; -} -std::ostream& operator<<(std::ostream& out, const GrantRevokePrivilegeResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GrantRevokePrivilegeResponse::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t GrantRevokePrivilegeResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GrantRevokePrivilegeResponse"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 1); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GrantRevokePrivilegeResponse &a, GrantRevokePrivilegeResponse &b) { - using ::std::swap; - swap(a.success, b.success); - swap(a.__isset, b.__isset); -} - -GrantRevokePrivilegeResponse::GrantRevokePrivilegeResponse(const GrantRevokePrivilegeResponse& other90) { - success = other90.success; - __isset = other90.__isset; -} -GrantRevokePrivilegeResponse& GrantRevokePrivilegeResponse::operator=(const GrantRevokePrivilegeResponse& other91) { - success = other91.success; - __isset = other91.__isset; - return *this; -} -void GrantRevokePrivilegeResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GrantRevokePrivilegeResponse("; - out << "success="; (__isset.success ? (out << to_string(success)) : (out << "")); - out << ")"; -} - - -Role::~Role() noexcept { -} - - -void Role::__set_roleName(const std::string& val) { - this->roleName = val; -} - -void Role::__set_createTime(const int32_t val) { - this->createTime = val; -} - -void Role::__set_ownerName(const std::string& val) { - this->ownerName = val; -} -std::ostream& operator<<(std::ostream& out, const Role& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t Role::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->roleName); - this->__isset.roleName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->createTime); - this->__isset.createTime = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->ownerName); - this->__isset.ownerName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t Role::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("Role"); - - xfer += oprot->writeFieldBegin("roleName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->roleName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32(this->createTime); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("ownerName", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->ownerName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(Role &a, Role &b) { - using ::std::swap; - swap(a.roleName, b.roleName); - swap(a.createTime, b.createTime); - swap(a.ownerName, b.ownerName); - swap(a.__isset, b.__isset); -} - -Role::Role(const Role& other92) { - roleName = other92.roleName; - createTime = other92.createTime; - ownerName = other92.ownerName; - __isset = other92.__isset; -} -Role& Role::operator=(const Role& other93) { - roleName = other93.roleName; - createTime = other93.createTime; - ownerName = other93.ownerName; - __isset = other93.__isset; - return *this; -} -void Role::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "Role("; - out << "roleName=" << to_string(roleName); - out << ", " << "createTime=" << to_string(createTime); - out << ", " << "ownerName=" << to_string(ownerName); - out << ")"; -} - - -RolePrincipalGrant::~RolePrincipalGrant() noexcept { -} - - -void RolePrincipalGrant::__set_roleName(const std::string& val) { - this->roleName = val; -} - -void RolePrincipalGrant::__set_principalName(const std::string& val) { - this->principalName = val; -} - -void RolePrincipalGrant::__set_principalType(const PrincipalType::type val) { - this->principalType = val; -} - -void RolePrincipalGrant::__set_grantOption(const bool val) { - this->grantOption = val; -} - -void RolePrincipalGrant::__set_grantTime(const int32_t val) { - this->grantTime = val; -} - -void RolePrincipalGrant::__set_grantorName(const std::string& val) { - this->grantorName = val; -} - -void RolePrincipalGrant::__set_grantorPrincipalType(const PrincipalType::type val) { - this->grantorPrincipalType = val; -} -std::ostream& operator<<(std::ostream& out, const RolePrincipalGrant& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t RolePrincipalGrant::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->roleName); - this->__isset.roleName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->principalName); - this->__isset.principalName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast94; - xfer += iprot->readI32(ecast94); - this->principalType = (PrincipalType::type)ecast94; - this->__isset.principalType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->grantOption); - this->__isset.grantOption = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->grantTime); - this->__isset.grantTime = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->grantorName); - this->__isset.grantorName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast95; - xfer += iprot->readI32(ecast95); - this->grantorPrincipalType = (PrincipalType::type)ecast95; - this->__isset.grantorPrincipalType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t RolePrincipalGrant::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("RolePrincipalGrant"); - - xfer += oprot->writeFieldBegin("roleName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->roleName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principalName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->principalName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principalType", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32((int32_t)this->principalType); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("grantOption", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool(this->grantOption); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("grantTime", ::apache::thrift::protocol::T_I32, 5); - xfer += oprot->writeI32(this->grantTime); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("grantorName", ::apache::thrift::protocol::T_STRING, 6); - xfer += oprot->writeString(this->grantorName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("grantorPrincipalType", ::apache::thrift::protocol::T_I32, 7); - xfer += oprot->writeI32((int32_t)this->grantorPrincipalType); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(RolePrincipalGrant &a, RolePrincipalGrant &b) { - using ::std::swap; - swap(a.roleName, b.roleName); - swap(a.principalName, b.principalName); - swap(a.principalType, b.principalType); - swap(a.grantOption, b.grantOption); - swap(a.grantTime, b.grantTime); - swap(a.grantorName, b.grantorName); - swap(a.grantorPrincipalType, b.grantorPrincipalType); - swap(a.__isset, b.__isset); -} - -RolePrincipalGrant::RolePrincipalGrant(const RolePrincipalGrant& other96) { - roleName = other96.roleName; - principalName = other96.principalName; - principalType = other96.principalType; - grantOption = other96.grantOption; - grantTime = other96.grantTime; - grantorName = other96.grantorName; - grantorPrincipalType = other96.grantorPrincipalType; - __isset = other96.__isset; -} -RolePrincipalGrant& RolePrincipalGrant::operator=(const RolePrincipalGrant& other97) { - roleName = other97.roleName; - principalName = other97.principalName; - principalType = other97.principalType; - grantOption = other97.grantOption; - grantTime = other97.grantTime; - grantorName = other97.grantorName; - grantorPrincipalType = other97.grantorPrincipalType; - __isset = other97.__isset; - return *this; -} -void RolePrincipalGrant::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "RolePrincipalGrant("; - out << "roleName=" << to_string(roleName); - out << ", " << "principalName=" << to_string(principalName); - out << ", " << "principalType=" << to_string(principalType); - out << ", " << "grantOption=" << to_string(grantOption); - out << ", " << "grantTime=" << to_string(grantTime); - out << ", " << "grantorName=" << to_string(grantorName); - out << ", " << "grantorPrincipalType=" << to_string(grantorPrincipalType); - out << ")"; -} - - -GetRoleGrantsForPrincipalRequest::~GetRoleGrantsForPrincipalRequest() noexcept { -} - - -void GetRoleGrantsForPrincipalRequest::__set_principal_name(const std::string& val) { - this->principal_name = val; -} - -void GetRoleGrantsForPrincipalRequest::__set_principal_type(const PrincipalType::type val) { - this->principal_type = val; -} -std::ostream& operator<<(std::ostream& out, const GetRoleGrantsForPrincipalRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GetRoleGrantsForPrincipalRequest::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_principal_name = false; - bool isset_principal_type = 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_STRING) { - xfer += iprot->readString(this->principal_name); - isset_principal_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast98; - xfer += iprot->readI32(ecast98); - this->principal_type = (PrincipalType::type)ecast98; - isset_principal_type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_principal_name) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_principal_type) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t GetRoleGrantsForPrincipalRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GetRoleGrantsForPrincipalRequest"); - - xfer += oprot->writeFieldBegin("principal_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->principal_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principal_type", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32((int32_t)this->principal_type); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GetRoleGrantsForPrincipalRequest &a, GetRoleGrantsForPrincipalRequest &b) { - using ::std::swap; - swap(a.principal_name, b.principal_name); - swap(a.principal_type, b.principal_type); -} - -GetRoleGrantsForPrincipalRequest::GetRoleGrantsForPrincipalRequest(const GetRoleGrantsForPrincipalRequest& other99) { - principal_name = other99.principal_name; - principal_type = other99.principal_type; -} -GetRoleGrantsForPrincipalRequest& GetRoleGrantsForPrincipalRequest::operator=(const GetRoleGrantsForPrincipalRequest& other100) { - principal_name = other100.principal_name; - principal_type = other100.principal_type; - return *this; -} -void GetRoleGrantsForPrincipalRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GetRoleGrantsForPrincipalRequest("; - out << "principal_name=" << to_string(principal_name); - out << ", " << "principal_type=" << to_string(principal_type); - out << ")"; -} - - -GetRoleGrantsForPrincipalResponse::~GetRoleGrantsForPrincipalResponse() noexcept { -} - - -void GetRoleGrantsForPrincipalResponse::__set_principalGrants(const std::vector & val) { - this->principalGrants = val; -} -std::ostream& operator<<(std::ostream& out, const GetRoleGrantsForPrincipalResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GetRoleGrantsForPrincipalResponse::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_principalGrants = 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_LIST) { - { - this->principalGrants.clear(); - uint32_t _size101; - ::apache::thrift::protocol::TType _etype104; - xfer += iprot->readListBegin(_etype104, _size101); - this->principalGrants.resize(_size101); - uint32_t _i105; - for (_i105 = 0; _i105 < _size101; ++_i105) - { - xfer += this->principalGrants[_i105].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_principalGrants = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_principalGrants) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t GetRoleGrantsForPrincipalResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GetRoleGrantsForPrincipalResponse"); - - xfer += oprot->writeFieldBegin("principalGrants", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->principalGrants.size())); - std::vector ::const_iterator _iter106; - for (_iter106 = this->principalGrants.begin(); _iter106 != this->principalGrants.end(); ++_iter106) - { - xfer += (*_iter106).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GetRoleGrantsForPrincipalResponse &a, GetRoleGrantsForPrincipalResponse &b) { - using ::std::swap; - swap(a.principalGrants, b.principalGrants); -} - -GetRoleGrantsForPrincipalResponse::GetRoleGrantsForPrincipalResponse(const GetRoleGrantsForPrincipalResponse& other107) { - principalGrants = other107.principalGrants; -} -GetRoleGrantsForPrincipalResponse& GetRoleGrantsForPrincipalResponse::operator=(const GetRoleGrantsForPrincipalResponse& other108) { - principalGrants = other108.principalGrants; - return *this; -} -void GetRoleGrantsForPrincipalResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GetRoleGrantsForPrincipalResponse("; - out << "principalGrants=" << to_string(principalGrants); - out << ")"; -} - - -GetPrincipalsInRoleRequest::~GetPrincipalsInRoleRequest() noexcept { -} - - -void GetPrincipalsInRoleRequest::__set_roleName(const std::string& val) { - this->roleName = val; -} -std::ostream& operator<<(std::ostream& out, const GetPrincipalsInRoleRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GetPrincipalsInRoleRequest::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_roleName = 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_STRING) { - xfer += iprot->readString(this->roleName); - isset_roleName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_roleName) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t GetPrincipalsInRoleRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GetPrincipalsInRoleRequest"); - - xfer += oprot->writeFieldBegin("roleName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->roleName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GetPrincipalsInRoleRequest &a, GetPrincipalsInRoleRequest &b) { - using ::std::swap; - swap(a.roleName, b.roleName); -} - -GetPrincipalsInRoleRequest::GetPrincipalsInRoleRequest(const GetPrincipalsInRoleRequest& other109) { - roleName = other109.roleName; -} -GetPrincipalsInRoleRequest& GetPrincipalsInRoleRequest::operator=(const GetPrincipalsInRoleRequest& other110) { - roleName = other110.roleName; - return *this; -} -void GetPrincipalsInRoleRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GetPrincipalsInRoleRequest("; - out << "roleName=" << to_string(roleName); - out << ")"; -} - - -GetPrincipalsInRoleResponse::~GetPrincipalsInRoleResponse() noexcept { -} - - -void GetPrincipalsInRoleResponse::__set_principalGrants(const std::vector & val) { - this->principalGrants = val; -} -std::ostream& operator<<(std::ostream& out, const GetPrincipalsInRoleResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GetPrincipalsInRoleResponse::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_principalGrants = 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_LIST) { - { - this->principalGrants.clear(); - uint32_t _size111; - ::apache::thrift::protocol::TType _etype114; - xfer += iprot->readListBegin(_etype114, _size111); - this->principalGrants.resize(_size111); - uint32_t _i115; - for (_i115 = 0; _i115 < _size111; ++_i115) - { - xfer += this->principalGrants[_i115].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_principalGrants = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_principalGrants) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t GetPrincipalsInRoleResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GetPrincipalsInRoleResponse"); - - xfer += oprot->writeFieldBegin("principalGrants", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->principalGrants.size())); - std::vector ::const_iterator _iter116; - for (_iter116 = this->principalGrants.begin(); _iter116 != this->principalGrants.end(); ++_iter116) - { - xfer += (*_iter116).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GetPrincipalsInRoleResponse &a, GetPrincipalsInRoleResponse &b) { - using ::std::swap; - swap(a.principalGrants, b.principalGrants); -} - -GetPrincipalsInRoleResponse::GetPrincipalsInRoleResponse(const GetPrincipalsInRoleResponse& other117) { - principalGrants = other117.principalGrants; -} -GetPrincipalsInRoleResponse& GetPrincipalsInRoleResponse::operator=(const GetPrincipalsInRoleResponse& other118) { - principalGrants = other118.principalGrants; - return *this; -} -void GetPrincipalsInRoleResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GetPrincipalsInRoleResponse("; - out << "principalGrants=" << to_string(principalGrants); - out << ")"; -} - - -GrantRevokeRoleRequest::~GrantRevokeRoleRequest() noexcept { -} - - -void GrantRevokeRoleRequest::__set_requestType(const GrantRevokeType::type val) { - this->requestType = val; -} - -void GrantRevokeRoleRequest::__set_roleName(const std::string& val) { - this->roleName = val; -} - -void GrantRevokeRoleRequest::__set_principalName(const std::string& val) { - this->principalName = val; -} - -void GrantRevokeRoleRequest::__set_principalType(const PrincipalType::type val) { - this->principalType = val; -} - -void GrantRevokeRoleRequest::__set_grantor(const std::string& val) { - this->grantor = val; -__isset.grantor = true; -} - -void GrantRevokeRoleRequest::__set_grantorType(const PrincipalType::type val) { - this->grantorType = val; -__isset.grantorType = true; -} - -void GrantRevokeRoleRequest::__set_grantOption(const bool val) { - this->grantOption = val; -__isset.grantOption = true; -} -std::ostream& operator<<(std::ostream& out, const GrantRevokeRoleRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GrantRevokeRoleRequest::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast119; - xfer += iprot->readI32(ecast119); - this->requestType = (GrantRevokeType::type)ecast119; - this->__isset.requestType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->roleName); - this->__isset.roleName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->principalName); - this->__isset.principalName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast120; - xfer += iprot->readI32(ecast120); - this->principalType = (PrincipalType::type)ecast120; - this->__isset.principalType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->grantor); - this->__isset.grantor = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast121; - xfer += iprot->readI32(ecast121); - this->grantorType = (PrincipalType::type)ecast121; - this->__isset.grantorType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->grantOption); - this->__isset.grantOption = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t GrantRevokeRoleRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GrantRevokeRoleRequest"); - - xfer += oprot->writeFieldBegin("requestType", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32((int32_t)this->requestType); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("roleName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->roleName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principalName", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->principalName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("principalType", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32((int32_t)this->principalType); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.grantor) { - xfer += oprot->writeFieldBegin("grantor", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString(this->grantor); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.grantorType) { - xfer += oprot->writeFieldBegin("grantorType", ::apache::thrift::protocol::T_I32, 6); - xfer += oprot->writeI32((int32_t)this->grantorType); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.grantOption) { - xfer += oprot->writeFieldBegin("grantOption", ::apache::thrift::protocol::T_BOOL, 7); - xfer += oprot->writeBool(this->grantOption); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GrantRevokeRoleRequest &a, GrantRevokeRoleRequest &b) { - using ::std::swap; - swap(a.requestType, b.requestType); - swap(a.roleName, b.roleName); - swap(a.principalName, b.principalName); - swap(a.principalType, b.principalType); - swap(a.grantor, b.grantor); - swap(a.grantorType, b.grantorType); - swap(a.grantOption, b.grantOption); - swap(a.__isset, b.__isset); -} - -GrantRevokeRoleRequest::GrantRevokeRoleRequest(const GrantRevokeRoleRequest& other122) { - requestType = other122.requestType; - roleName = other122.roleName; - principalName = other122.principalName; - principalType = other122.principalType; - grantor = other122.grantor; - grantorType = other122.grantorType; - grantOption = other122.grantOption; - __isset = other122.__isset; -} -GrantRevokeRoleRequest& GrantRevokeRoleRequest::operator=(const GrantRevokeRoleRequest& other123) { - requestType = other123.requestType; - roleName = other123.roleName; - principalName = other123.principalName; - principalType = other123.principalType; - grantor = other123.grantor; - grantorType = other123.grantorType; - grantOption = other123.grantOption; - __isset = other123.__isset; - return *this; -} -void GrantRevokeRoleRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GrantRevokeRoleRequest("; - out << "requestType=" << to_string(requestType); - out << ", " << "roleName=" << to_string(roleName); - out << ", " << "principalName=" << to_string(principalName); - out << ", " << "principalType=" << to_string(principalType); - out << ", " << "grantor="; (__isset.grantor ? (out << to_string(grantor)) : (out << "")); - out << ", " << "grantorType="; (__isset.grantorType ? (out << to_string(grantorType)) : (out << "")); - out << ", " << "grantOption="; (__isset.grantOption ? (out << to_string(grantOption)) : (out << "")); - out << ")"; -} - - -GrantRevokeRoleResponse::~GrantRevokeRoleResponse() noexcept { -} - - -void GrantRevokeRoleResponse::__set_success(const bool val) { - this->success = val; -__isset.success = true; -} -std::ostream& operator<<(std::ostream& out, const GrantRevokeRoleResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GrantRevokeRoleResponse::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->success); - this->__isset.success = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t GrantRevokeRoleResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GrantRevokeRoleResponse"); - - if (this->__isset.success) { - xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 1); - xfer += oprot->writeBool(this->success); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GrantRevokeRoleResponse &a, GrantRevokeRoleResponse &b) { - using ::std::swap; - swap(a.success, b.success); - swap(a.__isset, b.__isset); -} - -GrantRevokeRoleResponse::GrantRevokeRoleResponse(const GrantRevokeRoleResponse& other124) { - success = other124.success; - __isset = other124.__isset; -} -GrantRevokeRoleResponse& GrantRevokeRoleResponse::operator=(const GrantRevokeRoleResponse& other125) { - success = other125.success; - __isset = other125.__isset; - return *this; -} -void GrantRevokeRoleResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GrantRevokeRoleResponse("; - out << "success="; (__isset.success ? (out << to_string(success)) : (out << "")); - out << ")"; -} - - -Database::~Database() noexcept { -} - - -void Database::__set_name(const std::string& val) { - this->name = val; -} - -void Database::__set_description(const std::string& val) { - this->description = val; -} - -void Database::__set_locationUri(const std::string& val) { - this->locationUri = val; -} - -void Database::__set_parameters(const std::map & val) { - this->parameters = val; -} - -void Database::__set_privileges(const PrincipalPrivilegeSet& val) { - this->privileges = val; -__isset.privileges = true; -} - -void Database::__set_ownerName(const std::string& val) { - this->ownerName = val; -__isset.ownerName = true; -} - -void Database::__set_ownerType(const PrincipalType::type val) { - this->ownerType = val; -__isset.ownerType = true; -} -std::ostream& operator<<(std::ostream& out, const Database& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t Database::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->name); - this->__isset.name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->description); - this->__isset.description = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->locationUri); - this->__isset.locationUri = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->parameters.clear(); - uint32_t _size126; - ::apache::thrift::protocol::TType _ktype127; - ::apache::thrift::protocol::TType _vtype128; - xfer += iprot->readMapBegin(_ktype127, _vtype128, _size126); - uint32_t _i130; - for (_i130 = 0; _i130 < _size126; ++_i130) - { - std::string _key131; - xfer += iprot->readString(_key131); - std::string& _val132 = this->parameters[_key131]; - xfer += iprot->readString(_val132); - } - xfer += iprot->readMapEnd(); - } - this->__isset.parameters = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->privileges.read(iprot); - this->__isset.privileges = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->ownerName); - this->__isset.ownerName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast133; - xfer += iprot->readI32(ecast133); - this->ownerType = (PrincipalType::type)ecast133; - this->__isset.ownerType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t Database::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("Database"); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("description", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->description); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("locationUri", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->locationUri); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 4); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->parameters.size())); - std::map ::const_iterator _iter134; - for (_iter134 = this->parameters.begin(); _iter134 != this->parameters.end(); ++_iter134) - { - xfer += oprot->writeString(_iter134->first); - xfer += oprot->writeString(_iter134->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - if (this->__isset.privileges) { - xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 5); - xfer += this->privileges.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.ownerName) { - xfer += oprot->writeFieldBegin("ownerName", ::apache::thrift::protocol::T_STRING, 6); - xfer += oprot->writeString(this->ownerName); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.ownerType) { - xfer += oprot->writeFieldBegin("ownerType", ::apache::thrift::protocol::T_I32, 7); - xfer += oprot->writeI32((int32_t)this->ownerType); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(Database &a, Database &b) { - using ::std::swap; - swap(a.name, b.name); - swap(a.description, b.description); - swap(a.locationUri, b.locationUri); - swap(a.parameters, b.parameters); - swap(a.privileges, b.privileges); - swap(a.ownerName, b.ownerName); - swap(a.ownerType, b.ownerType); - swap(a.__isset, b.__isset); -} - -Database::Database(const Database& other135) { - name = other135.name; - description = other135.description; - locationUri = other135.locationUri; - parameters = other135.parameters; - privileges = other135.privileges; - ownerName = other135.ownerName; - ownerType = other135.ownerType; - __isset = other135.__isset; -} -Database& Database::operator=(const Database& other136) { - name = other136.name; - description = other136.description; - locationUri = other136.locationUri; - parameters = other136.parameters; - privileges = other136.privileges; - ownerName = other136.ownerName; - ownerType = other136.ownerType; - __isset = other136.__isset; - return *this; -} -void Database::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "Database("; - out << "name=" << to_string(name); - out << ", " << "description=" << to_string(description); - out << ", " << "locationUri=" << to_string(locationUri); - out << ", " << "parameters=" << to_string(parameters); - out << ", " << "privileges="; (__isset.privileges ? (out << to_string(privileges)) : (out << "")); - out << ", " << "ownerName="; (__isset.ownerName ? (out << to_string(ownerName)) : (out << "")); - out << ", " << "ownerType="; (__isset.ownerType ? (out << to_string(ownerType)) : (out << "")); - out << ")"; -} - - -SerDeInfo::~SerDeInfo() noexcept { -} - - -void SerDeInfo::__set_name(const std::string& val) { - this->name = val; -} - -void SerDeInfo::__set_serializationLib(const std::string& val) { - this->serializationLib = val; -} - -void SerDeInfo::__set_parameters(const std::map & val) { - this->parameters = val; -} -std::ostream& operator<<(std::ostream& out, const SerDeInfo& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t SerDeInfo::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->name); - this->__isset.name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->serializationLib); - this->__isset.serializationLib = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->parameters.clear(); - uint32_t _size137; - ::apache::thrift::protocol::TType _ktype138; - ::apache::thrift::protocol::TType _vtype139; - xfer += iprot->readMapBegin(_ktype138, _vtype139, _size137); - uint32_t _i141; - for (_i141 = 0; _i141 < _size137; ++_i141) - { - std::string _key142; - xfer += iprot->readString(_key142); - std::string& _val143 = this->parameters[_key142]; - xfer += iprot->readString(_val143); - } - xfer += iprot->readMapEnd(); - } - this->__isset.parameters = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t SerDeInfo::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("SerDeInfo"); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("serializationLib", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->serializationLib); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 3); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->parameters.size())); - std::map ::const_iterator _iter144; - for (_iter144 = this->parameters.begin(); _iter144 != this->parameters.end(); ++_iter144) - { - xfer += oprot->writeString(_iter144->first); - xfer += oprot->writeString(_iter144->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(SerDeInfo &a, SerDeInfo &b) { - using ::std::swap; - swap(a.name, b.name); - swap(a.serializationLib, b.serializationLib); - swap(a.parameters, b.parameters); - swap(a.__isset, b.__isset); -} - -SerDeInfo::SerDeInfo(const SerDeInfo& other145) { - name = other145.name; - serializationLib = other145.serializationLib; - parameters = other145.parameters; - __isset = other145.__isset; -} -SerDeInfo& SerDeInfo::operator=(const SerDeInfo& other146) { - name = other146.name; - serializationLib = other146.serializationLib; - parameters = other146.parameters; - __isset = other146.__isset; - return *this; -} -void SerDeInfo::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "SerDeInfo("; - out << "name=" << to_string(name); - out << ", " << "serializationLib=" << to_string(serializationLib); - out << ", " << "parameters=" << to_string(parameters); - out << ")"; -} - - -Order::~Order() noexcept { -} - - -void Order::__set_col(const std::string& val) { - this->col = val; -} - -void Order::__set_order(const int32_t val) { - this->order = val; -} -std::ostream& operator<<(std::ostream& out, const Order& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t Order::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->col); - this->__isset.col = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->order); - this->__isset.order = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t Order::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("Order"); - - xfer += oprot->writeFieldBegin("col", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->col); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("order", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32(this->order); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(Order &a, Order &b) { - using ::std::swap; - swap(a.col, b.col); - swap(a.order, b.order); - swap(a.__isset, b.__isset); -} - -Order::Order(const Order& other147) { - col = other147.col; - order = other147.order; - __isset = other147.__isset; -} -Order& Order::operator=(const Order& other148) { - col = other148.col; - order = other148.order; - __isset = other148.__isset; - return *this; -} -void Order::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "Order("; - out << "col=" << to_string(col); - out << ", " << "order=" << to_string(order); - out << ")"; -} - - -SkewedInfo::~SkewedInfo() noexcept { -} - - -void SkewedInfo::__set_skewedColNames(const std::vector & val) { - this->skewedColNames = val; -} - -void SkewedInfo::__set_skewedColValues(const std::vector > & val) { - this->skewedColValues = val; -} - -void SkewedInfo::__set_skewedColValueLocationMaps(const std::map & val) { - this->skewedColValueLocationMaps = val; -} -std::ostream& operator<<(std::ostream& out, const SkewedInfo& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t SkewedInfo::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->skewedColNames.clear(); - uint32_t _size149; - ::apache::thrift::protocol::TType _etype152; - xfer += iprot->readListBegin(_etype152, _size149); - this->skewedColNames.resize(_size149); - uint32_t _i153; - for (_i153 = 0; _i153 < _size149; ++_i153) - { - xfer += iprot->readString(this->skewedColNames[_i153]); - } - xfer += iprot->readListEnd(); - } - this->__isset.skewedColNames = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->skewedColValues.clear(); - uint32_t _size154; - ::apache::thrift::protocol::TType _etype157; - xfer += iprot->readListBegin(_etype157, _size154); - this->skewedColValues.resize(_size154); - uint32_t _i158; - for (_i158 = 0; _i158 < _size154; ++_i158) - { - { - this->skewedColValues[_i158].clear(); - uint32_t _size159; - ::apache::thrift::protocol::TType _etype162; - xfer += iprot->readListBegin(_etype162, _size159); - this->skewedColValues[_i158].resize(_size159); - uint32_t _i163; - for (_i163 = 0; _i163 < _size159; ++_i163) - { - xfer += iprot->readString(this->skewedColValues[_i158][_i163]); - } - xfer += iprot->readListEnd(); - } - } - xfer += iprot->readListEnd(); - } - this->__isset.skewedColValues = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->skewedColValueLocationMaps.clear(); - uint32_t _size164; - ::apache::thrift::protocol::TType _ktype165; - ::apache::thrift::protocol::TType _vtype166; - xfer += iprot->readMapBegin(_ktype165, _vtype166, _size164); - uint32_t _i168; - for (_i168 = 0; _i168 < _size164; ++_i168) - { - std::string _key169; - xfer += iprot->readString(_key169); - std::string& _val170 = this->skewedColValueLocationMaps[_key169]; - xfer += iprot->readString(_val170); - } - xfer += iprot->readMapEnd(); - } - this->__isset.skewedColValueLocationMaps = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t SkewedInfo::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("SkewedInfo"); - - xfer += oprot->writeFieldBegin("skewedColNames", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->skewedColNames.size())); - std::vector ::const_iterator _iter171; - for (_iter171 = this->skewedColNames.begin(); _iter171 != this->skewedColNames.end(); ++_iter171) - { - xfer += oprot->writeString((*_iter171)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("skewedColValues", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_LIST, static_cast(this->skewedColValues.size())); - std::vector > ::const_iterator _iter172; - for (_iter172 = this->skewedColValues.begin(); _iter172 != this->skewedColValues.end(); ++_iter172) - { - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*_iter172).size())); - std::vector ::const_iterator _iter173; - for (_iter173 = (*_iter172).begin(); _iter173 != (*_iter172).end(); ++_iter173) - { - xfer += oprot->writeString((*_iter173)); - } - xfer += oprot->writeListEnd(); - } - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("skewedColValueLocationMaps", ::apache::thrift::protocol::T_MAP, 3); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->skewedColValueLocationMaps.size())); - std::map ::const_iterator _iter174; - for (_iter174 = this->skewedColValueLocationMaps.begin(); _iter174 != this->skewedColValueLocationMaps.end(); ++_iter174) - { - xfer += oprot->writeString(_iter174->first); - xfer += oprot->writeString(_iter174->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(SkewedInfo &a, SkewedInfo &b) { - using ::std::swap; - swap(a.skewedColNames, b.skewedColNames); - swap(a.skewedColValues, b.skewedColValues); - swap(a.skewedColValueLocationMaps, b.skewedColValueLocationMaps); - swap(a.__isset, b.__isset); -} - -SkewedInfo::SkewedInfo(const SkewedInfo& other175) { - skewedColNames = other175.skewedColNames; - skewedColValues = other175.skewedColValues; - skewedColValueLocationMaps = other175.skewedColValueLocationMaps; - __isset = other175.__isset; -} -SkewedInfo& SkewedInfo::operator=(const SkewedInfo& other176) { - skewedColNames = other176.skewedColNames; - skewedColValues = other176.skewedColValues; - skewedColValueLocationMaps = other176.skewedColValueLocationMaps; - __isset = other176.__isset; - return *this; -} -void SkewedInfo::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "SkewedInfo("; - out << "skewedColNames=" << to_string(skewedColNames); - out << ", " << "skewedColValues=" << to_string(skewedColValues); - out << ", " << "skewedColValueLocationMaps=" << to_string(skewedColValueLocationMaps); - out << ")"; -} - - -StorageDescriptor::~StorageDescriptor() noexcept { -} - - -void StorageDescriptor::__set_cols(const std::vector & val) { - this->cols = val; -} - -void StorageDescriptor::__set_location(const std::string& val) { - this->location = val; -} - -void StorageDescriptor::__set_inputFormat(const std::string& val) { - this->inputFormat = val; -} - -void StorageDescriptor::__set_outputFormat(const std::string& val) { - this->outputFormat = val; -} - -void StorageDescriptor::__set_compressed(const bool val) { - this->compressed = val; -} - -void StorageDescriptor::__set_numBuckets(const int32_t val) { - this->numBuckets = val; -} - -void StorageDescriptor::__set_serdeInfo(const SerDeInfo& val) { - this->serdeInfo = val; -} - -void StorageDescriptor::__set_bucketCols(const std::vector & val) { - this->bucketCols = val; -} - -void StorageDescriptor::__set_sortCols(const std::vector & val) { - this->sortCols = val; -} - -void StorageDescriptor::__set_parameters(const std::map & val) { - this->parameters = val; -} - -void StorageDescriptor::__set_skewedInfo(const SkewedInfo& val) { - this->skewedInfo = val; -__isset.skewedInfo = true; -} - -void StorageDescriptor::__set_storedAsSubDirectories(const bool val) { - this->storedAsSubDirectories = val; -__isset.storedAsSubDirectories = true; -} -std::ostream& operator<<(std::ostream& out, const StorageDescriptor& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t StorageDescriptor::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->cols.clear(); - uint32_t _size177; - ::apache::thrift::protocol::TType _etype180; - xfer += iprot->readListBegin(_etype180, _size177); - this->cols.resize(_size177); - uint32_t _i181; - for (_i181 = 0; _i181 < _size177; ++_i181) - { - xfer += this->cols[_i181].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.cols = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->location); - this->__isset.location = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->inputFormat); - this->__isset.inputFormat = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->outputFormat); - this->__isset.outputFormat = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->compressed); - this->__isset.compressed = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->numBuckets); - this->__isset.numBuckets = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->serdeInfo.read(iprot); - this->__isset.serdeInfo = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->bucketCols.clear(); - uint32_t _size182; - ::apache::thrift::protocol::TType _etype185; - xfer += iprot->readListBegin(_etype185, _size182); - this->bucketCols.resize(_size182); - uint32_t _i186; - for (_i186 = 0; _i186 < _size182; ++_i186) - { - xfer += iprot->readString(this->bucketCols[_i186]); - } - xfer += iprot->readListEnd(); - } - this->__isset.bucketCols = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 9: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->sortCols.clear(); - uint32_t _size187; - ::apache::thrift::protocol::TType _etype190; - xfer += iprot->readListBegin(_etype190, _size187); - this->sortCols.resize(_size187); - uint32_t _i191; - for (_i191 = 0; _i191 < _size187; ++_i191) - { - xfer += this->sortCols[_i191].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.sortCols = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 10: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->parameters.clear(); - uint32_t _size192; - ::apache::thrift::protocol::TType _ktype193; - ::apache::thrift::protocol::TType _vtype194; - xfer += iprot->readMapBegin(_ktype193, _vtype194, _size192); - uint32_t _i196; - for (_i196 = 0; _i196 < _size192; ++_i196) - { - std::string _key197; - xfer += iprot->readString(_key197); - std::string& _val198 = this->parameters[_key197]; - xfer += iprot->readString(_val198); - } - xfer += iprot->readMapEnd(); - } - this->__isset.parameters = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 11: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->skewedInfo.read(iprot); - this->__isset.skewedInfo = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 12: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->storedAsSubDirectories); - this->__isset.storedAsSubDirectories = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t StorageDescriptor::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("StorageDescriptor"); - - xfer += oprot->writeFieldBegin("cols", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->cols.size())); - std::vector ::const_iterator _iter199; - for (_iter199 = this->cols.begin(); _iter199 != this->cols.end(); ++_iter199) - { - xfer += (*_iter199).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("location", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->location); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("inputFormat", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->inputFormat); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("outputFormat", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->outputFormat); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("compressed", ::apache::thrift::protocol::T_BOOL, 5); - xfer += oprot->writeBool(this->compressed); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("numBuckets", ::apache::thrift::protocol::T_I32, 6); - xfer += oprot->writeI32(this->numBuckets); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("serdeInfo", ::apache::thrift::protocol::T_STRUCT, 7); - xfer += this->serdeInfo.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("bucketCols", ::apache::thrift::protocol::T_LIST, 8); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->bucketCols.size())); - std::vector ::const_iterator _iter200; - for (_iter200 = this->bucketCols.begin(); _iter200 != this->bucketCols.end(); ++_iter200) - { - xfer += oprot->writeString((*_iter200)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("sortCols", ::apache::thrift::protocol::T_LIST, 9); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->sortCols.size())); - std::vector ::const_iterator _iter201; - for (_iter201 = this->sortCols.begin(); _iter201 != this->sortCols.end(); ++_iter201) - { - xfer += (*_iter201).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 10); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->parameters.size())); - std::map ::const_iterator _iter202; - for (_iter202 = this->parameters.begin(); _iter202 != this->parameters.end(); ++_iter202) - { - xfer += oprot->writeString(_iter202->first); - xfer += oprot->writeString(_iter202->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - if (this->__isset.skewedInfo) { - xfer += oprot->writeFieldBegin("skewedInfo", ::apache::thrift::protocol::T_STRUCT, 11); - xfer += this->skewedInfo.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.storedAsSubDirectories) { - xfer += oprot->writeFieldBegin("storedAsSubDirectories", ::apache::thrift::protocol::T_BOOL, 12); - xfer += oprot->writeBool(this->storedAsSubDirectories); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(StorageDescriptor &a, StorageDescriptor &b) { - using ::std::swap; - swap(a.cols, b.cols); - swap(a.location, b.location); - swap(a.inputFormat, b.inputFormat); - swap(a.outputFormat, b.outputFormat); - swap(a.compressed, b.compressed); - swap(a.numBuckets, b.numBuckets); - swap(a.serdeInfo, b.serdeInfo); - swap(a.bucketCols, b.bucketCols); - swap(a.sortCols, b.sortCols); - swap(a.parameters, b.parameters); - swap(a.skewedInfo, b.skewedInfo); - swap(a.storedAsSubDirectories, b.storedAsSubDirectories); - swap(a.__isset, b.__isset); -} - -StorageDescriptor::StorageDescriptor(const StorageDescriptor& other203) { - cols = other203.cols; - location = other203.location; - inputFormat = other203.inputFormat; - outputFormat = other203.outputFormat; - compressed = other203.compressed; - numBuckets = other203.numBuckets; - serdeInfo = other203.serdeInfo; - bucketCols = other203.bucketCols; - sortCols = other203.sortCols; - parameters = other203.parameters; - skewedInfo = other203.skewedInfo; - storedAsSubDirectories = other203.storedAsSubDirectories; - __isset = other203.__isset; -} -StorageDescriptor& StorageDescriptor::operator=(const StorageDescriptor& other204) { - cols = other204.cols; - location = other204.location; - inputFormat = other204.inputFormat; - outputFormat = other204.outputFormat; - compressed = other204.compressed; - numBuckets = other204.numBuckets; - serdeInfo = other204.serdeInfo; - bucketCols = other204.bucketCols; - sortCols = other204.sortCols; - parameters = other204.parameters; - skewedInfo = other204.skewedInfo; - storedAsSubDirectories = other204.storedAsSubDirectories; - __isset = other204.__isset; - return *this; -} -void StorageDescriptor::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "StorageDescriptor("; - out << "cols=" << to_string(cols); - out << ", " << "location=" << to_string(location); - out << ", " << "inputFormat=" << to_string(inputFormat); - out << ", " << "outputFormat=" << to_string(outputFormat); - out << ", " << "compressed=" << to_string(compressed); - out << ", " << "numBuckets=" << to_string(numBuckets); - out << ", " << "serdeInfo=" << to_string(serdeInfo); - out << ", " << "bucketCols=" << to_string(bucketCols); - out << ", " << "sortCols=" << to_string(sortCols); - out << ", " << "parameters=" << to_string(parameters); - out << ", " << "skewedInfo="; (__isset.skewedInfo ? (out << to_string(skewedInfo)) : (out << "")); - out << ", " << "storedAsSubDirectories="; (__isset.storedAsSubDirectories ? (out << to_string(storedAsSubDirectories)) : (out << "")); - out << ")"; -} - - -Table::~Table() noexcept { -} - - -void Table::__set_tableName(const std::string& val) { - this->tableName = val; -} - -void Table::__set_dbName(const std::string& val) { - this->dbName = val; -} - -void Table::__set_owner(const std::string& val) { - this->owner = val; -} - -void Table::__set_createTime(const int32_t val) { - this->createTime = val; -} - -void Table::__set_lastAccessTime(const int32_t val) { - this->lastAccessTime = val; -} - -void Table::__set_retention(const int32_t val) { - this->retention = val; -} - -void Table::__set_sd(const StorageDescriptor& val) { - this->sd = val; -} - -void Table::__set_partitionKeys(const std::vector & val) { - this->partitionKeys = val; -} - -void Table::__set_parameters(const std::map & val) { - this->parameters = val; -} - -void Table::__set_viewOriginalText(const std::string& val) { - this->viewOriginalText = val; -} - -void Table::__set_viewExpandedText(const std::string& val) { - this->viewExpandedText = val; -} - -void Table::__set_tableType(const std::string& val) { - this->tableType = val; -} - -void Table::__set_privileges(const PrincipalPrivilegeSet& val) { - this->privileges = val; -__isset.privileges = true; -} - -void Table::__set_temporary(const bool val) { - this->temporary = val; -__isset.temporary = true; -} - -void Table::__set_rewriteEnabled(const bool val) { - this->rewriteEnabled = val; -__isset.rewriteEnabled = true; -} -std::ostream& operator<<(std::ostream& out, const Table& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t Table::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tableName); - this->__isset.tableName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbName); - this->__isset.dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->owner); - this->__isset.owner = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->createTime); - this->__isset.createTime = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->lastAccessTime); - this->__isset.lastAccessTime = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->retention); - this->__isset.retention = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->sd.read(iprot); - this->__isset.sd = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->partitionKeys.clear(); - uint32_t _size205; - ::apache::thrift::protocol::TType _etype208; - xfer += iprot->readListBegin(_etype208, _size205); - this->partitionKeys.resize(_size205); - uint32_t _i209; - for (_i209 = 0; _i209 < _size205; ++_i209) - { - xfer += this->partitionKeys[_i209].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.partitionKeys = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 9: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->parameters.clear(); - uint32_t _size210; - ::apache::thrift::protocol::TType _ktype211; - ::apache::thrift::protocol::TType _vtype212; - xfer += iprot->readMapBegin(_ktype211, _vtype212, _size210); - uint32_t _i214; - for (_i214 = 0; _i214 < _size210; ++_i214) - { - std::string _key215; - xfer += iprot->readString(_key215); - std::string& _val216 = this->parameters[_key215]; - xfer += iprot->readString(_val216); - } - xfer += iprot->readMapEnd(); - } - this->__isset.parameters = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 10: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->viewOriginalText); - this->__isset.viewOriginalText = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 11: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->viewExpandedText); - this->__isset.viewExpandedText = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 12: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tableType); - this->__isset.tableType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 13: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->privileges.read(iprot); - this->__isset.privileges = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 14: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->temporary); - this->__isset.temporary = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 15: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->rewriteEnabled); - this->__isset.rewriteEnabled = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t Table::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("Table"); - - xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->tableName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("owner", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->owner); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32(this->createTime); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("lastAccessTime", ::apache::thrift::protocol::T_I32, 5); - xfer += oprot->writeI32(this->lastAccessTime); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("retention", ::apache::thrift::protocol::T_I32, 6); - xfer += oprot->writeI32(this->retention); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("sd", ::apache::thrift::protocol::T_STRUCT, 7); - xfer += this->sd.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("partitionKeys", ::apache::thrift::protocol::T_LIST, 8); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitionKeys.size())); - std::vector ::const_iterator _iter217; - for (_iter217 = this->partitionKeys.begin(); _iter217 != this->partitionKeys.end(); ++_iter217) - { - xfer += (*_iter217).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 9); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->parameters.size())); - std::map ::const_iterator _iter218; - for (_iter218 = this->parameters.begin(); _iter218 != this->parameters.end(); ++_iter218) - { - xfer += oprot->writeString(_iter218->first); - xfer += oprot->writeString(_iter218->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("viewOriginalText", ::apache::thrift::protocol::T_STRING, 10); - xfer += oprot->writeString(this->viewOriginalText); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("viewExpandedText", ::apache::thrift::protocol::T_STRING, 11); - xfer += oprot->writeString(this->viewExpandedText); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 12); - xfer += oprot->writeString(this->tableType); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.privileges) { - xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 13); - xfer += this->privileges.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.temporary) { - xfer += oprot->writeFieldBegin("temporary", ::apache::thrift::protocol::T_BOOL, 14); - xfer += oprot->writeBool(this->temporary); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.rewriteEnabled) { - xfer += oprot->writeFieldBegin("rewriteEnabled", ::apache::thrift::protocol::T_BOOL, 15); - xfer += oprot->writeBool(this->rewriteEnabled); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(Table &a, Table &b) { - using ::std::swap; - swap(a.tableName, b.tableName); - swap(a.dbName, b.dbName); - swap(a.owner, b.owner); - swap(a.createTime, b.createTime); - swap(a.lastAccessTime, b.lastAccessTime); - swap(a.retention, b.retention); - swap(a.sd, b.sd); - swap(a.partitionKeys, b.partitionKeys); - swap(a.parameters, b.parameters); - swap(a.viewOriginalText, b.viewOriginalText); - swap(a.viewExpandedText, b.viewExpandedText); - swap(a.tableType, b.tableType); - swap(a.privileges, b.privileges); - swap(a.temporary, b.temporary); - swap(a.rewriteEnabled, b.rewriteEnabled); - swap(a.__isset, b.__isset); -} - -Table::Table(const Table& other219) { - tableName = other219.tableName; - dbName = other219.dbName; - owner = other219.owner; - createTime = other219.createTime; - lastAccessTime = other219.lastAccessTime; - retention = other219.retention; - sd = other219.sd; - partitionKeys = other219.partitionKeys; - parameters = other219.parameters; - viewOriginalText = other219.viewOriginalText; - viewExpandedText = other219.viewExpandedText; - tableType = other219.tableType; - privileges = other219.privileges; - temporary = other219.temporary; - rewriteEnabled = other219.rewriteEnabled; - __isset = other219.__isset; -} -Table& Table::operator=(const Table& other220) { - tableName = other220.tableName; - dbName = other220.dbName; - owner = other220.owner; - createTime = other220.createTime; - lastAccessTime = other220.lastAccessTime; - retention = other220.retention; - sd = other220.sd; - partitionKeys = other220.partitionKeys; - parameters = other220.parameters; - viewOriginalText = other220.viewOriginalText; - viewExpandedText = other220.viewExpandedText; - tableType = other220.tableType; - privileges = other220.privileges; - temporary = other220.temporary; - rewriteEnabled = other220.rewriteEnabled; - __isset = other220.__isset; - return *this; -} -void Table::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "Table("; - out << "tableName=" << to_string(tableName); - out << ", " << "dbName=" << to_string(dbName); - out << ", " << "owner=" << to_string(owner); - out << ", " << "createTime=" << to_string(createTime); - out << ", " << "lastAccessTime=" << to_string(lastAccessTime); - out << ", " << "retention=" << to_string(retention); - out << ", " << "sd=" << to_string(sd); - out << ", " << "partitionKeys=" << to_string(partitionKeys); - out << ", " << "parameters=" << to_string(parameters); - out << ", " << "viewOriginalText=" << to_string(viewOriginalText); - out << ", " << "viewExpandedText=" << to_string(viewExpandedText); - out << ", " << "tableType=" << to_string(tableType); - out << ", " << "privileges="; (__isset.privileges ? (out << to_string(privileges)) : (out << "")); - out << ", " << "temporary="; (__isset.temporary ? (out << to_string(temporary)) : (out << "")); - out << ", " << "rewriteEnabled="; (__isset.rewriteEnabled ? (out << to_string(rewriteEnabled)) : (out << "")); - out << ")"; -} - - -Partition::~Partition() noexcept { -} - - -void Partition::__set_values(const std::vector & val) { - this->values = val; -} - -void Partition::__set_dbName(const std::string& val) { - this->dbName = val; -} - -void Partition::__set_tableName(const std::string& val) { - this->tableName = val; -} - -void Partition::__set_createTime(const int32_t val) { - this->createTime = val; -} - -void Partition::__set_lastAccessTime(const int32_t val) { - this->lastAccessTime = val; -} - -void Partition::__set_sd(const StorageDescriptor& val) { - this->sd = val; -} - -void Partition::__set_parameters(const std::map & val) { - this->parameters = val; -} - -void Partition::__set_privileges(const PrincipalPrivilegeSet& val) { - this->privileges = val; -__isset.privileges = true; -} -std::ostream& operator<<(std::ostream& out, const Partition& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t Partition::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->values.clear(); - uint32_t _size221; - ::apache::thrift::protocol::TType _etype224; - xfer += iprot->readListBegin(_etype224, _size221); - this->values.resize(_size221); - uint32_t _i225; - for (_i225 = 0; _i225 < _size221; ++_i225) - { - xfer += iprot->readString(this->values[_i225]); - } - xfer += iprot->readListEnd(); - } - this->__isset.values = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbName); - this->__isset.dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tableName); - this->__isset.tableName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->createTime); - this->__isset.createTime = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->lastAccessTime); - this->__isset.lastAccessTime = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->sd.read(iprot); - this->__isset.sd = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->parameters.clear(); - uint32_t _size226; - ::apache::thrift::protocol::TType _ktype227; - ::apache::thrift::protocol::TType _vtype228; - xfer += iprot->readMapBegin(_ktype227, _vtype228, _size226); - uint32_t _i230; - for (_i230 = 0; _i230 < _size226; ++_i230) - { - std::string _key231; - xfer += iprot->readString(_key231); - std::string& _val232 = this->parameters[_key231]; - xfer += iprot->readString(_val232); - } - xfer += iprot->readMapEnd(); - } - this->__isset.parameters = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->privileges.read(iprot); - this->__isset.privileges = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t Partition::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("Partition"); - - xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->values.size())); - std::vector ::const_iterator _iter233; - for (_iter233 = this->values.begin(); _iter233 != this->values.end(); ++_iter233) - { - xfer += oprot->writeString((*_iter233)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->tableName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32(this->createTime); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("lastAccessTime", ::apache::thrift::protocol::T_I32, 5); - xfer += oprot->writeI32(this->lastAccessTime); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("sd", ::apache::thrift::protocol::T_STRUCT, 6); - xfer += this->sd.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 7); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->parameters.size())); - std::map ::const_iterator _iter234; - for (_iter234 = this->parameters.begin(); _iter234 != this->parameters.end(); ++_iter234) - { - xfer += oprot->writeString(_iter234->first); - xfer += oprot->writeString(_iter234->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - if (this->__isset.privileges) { - xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 8); - xfer += this->privileges.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(Partition &a, Partition &b) { - using ::std::swap; - swap(a.values, b.values); - swap(a.dbName, b.dbName); - swap(a.tableName, b.tableName); - swap(a.createTime, b.createTime); - swap(a.lastAccessTime, b.lastAccessTime); - swap(a.sd, b.sd); - swap(a.parameters, b.parameters); - swap(a.privileges, b.privileges); - swap(a.__isset, b.__isset); -} - -Partition::Partition(const Partition& other235) { - values = other235.values; - dbName = other235.dbName; - tableName = other235.tableName; - createTime = other235.createTime; - lastAccessTime = other235.lastAccessTime; - sd = other235.sd; - parameters = other235.parameters; - privileges = other235.privileges; - __isset = other235.__isset; -} -Partition& Partition::operator=(const Partition& other236) { - values = other236.values; - dbName = other236.dbName; - tableName = other236.tableName; - createTime = other236.createTime; - lastAccessTime = other236.lastAccessTime; - sd = other236.sd; - parameters = other236.parameters; - privileges = other236.privileges; - __isset = other236.__isset; - return *this; -} -void Partition::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "Partition("; - out << "values=" << to_string(values); - out << ", " << "dbName=" << to_string(dbName); - out << ", " << "tableName=" << to_string(tableName); - out << ", " << "createTime=" << to_string(createTime); - out << ", " << "lastAccessTime=" << to_string(lastAccessTime); - out << ", " << "sd=" << to_string(sd); - out << ", " << "parameters=" << to_string(parameters); - out << ", " << "privileges="; (__isset.privileges ? (out << to_string(privileges)) : (out << "")); - out << ")"; -} - - -PartitionWithoutSD::~PartitionWithoutSD() noexcept { -} - - -void PartitionWithoutSD::__set_values(const std::vector & val) { - this->values = val; -} - -void PartitionWithoutSD::__set_createTime(const int32_t val) { - this->createTime = val; -} - -void PartitionWithoutSD::__set_lastAccessTime(const int32_t val) { - this->lastAccessTime = val; -} - -void PartitionWithoutSD::__set_relativePath(const std::string& val) { - this->relativePath = val; -} - -void PartitionWithoutSD::__set_parameters(const std::map & val) { - this->parameters = val; -} - -void PartitionWithoutSD::__set_privileges(const PrincipalPrivilegeSet& val) { - this->privileges = val; -__isset.privileges = true; -} -std::ostream& operator<<(std::ostream& out, const PartitionWithoutSD& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PartitionWithoutSD::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->values.clear(); - uint32_t _size237; - ::apache::thrift::protocol::TType _etype240; - xfer += iprot->readListBegin(_etype240, _size237); - this->values.resize(_size237); - uint32_t _i241; - for (_i241 = 0; _i241 < _size237; ++_i241) - { - xfer += iprot->readString(this->values[_i241]); - } - xfer += iprot->readListEnd(); - } - this->__isset.values = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->createTime); - this->__isset.createTime = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->lastAccessTime); - this->__isset.lastAccessTime = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->relativePath); - this->__isset.relativePath = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->parameters.clear(); - uint32_t _size242; - ::apache::thrift::protocol::TType _ktype243; - ::apache::thrift::protocol::TType _vtype244; - xfer += iprot->readMapBegin(_ktype243, _vtype244, _size242); - uint32_t _i246; - for (_i246 = 0; _i246 < _size242; ++_i246) - { - std::string _key247; - xfer += iprot->readString(_key247); - std::string& _val248 = this->parameters[_key247]; - xfer += iprot->readString(_val248); - } - xfer += iprot->readMapEnd(); - } - this->__isset.parameters = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->privileges.read(iprot); - this->__isset.privileges = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t PartitionWithoutSD::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PartitionWithoutSD"); - - xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->values.size())); - std::vector ::const_iterator _iter249; - for (_iter249 = this->values.begin(); _iter249 != this->values.end(); ++_iter249) - { - xfer += oprot->writeString((*_iter249)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32(this->createTime); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("lastAccessTime", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32(this->lastAccessTime); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("relativePath", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->relativePath); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 5); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->parameters.size())); - std::map ::const_iterator _iter250; - for (_iter250 = this->parameters.begin(); _iter250 != this->parameters.end(); ++_iter250) - { - xfer += oprot->writeString(_iter250->first); - xfer += oprot->writeString(_iter250->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - if (this->__isset.privileges) { - xfer += oprot->writeFieldBegin("privileges", ::apache::thrift::protocol::T_STRUCT, 6); - xfer += this->privileges.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PartitionWithoutSD &a, PartitionWithoutSD &b) { - using ::std::swap; - swap(a.values, b.values); - swap(a.createTime, b.createTime); - swap(a.lastAccessTime, b.lastAccessTime); - swap(a.relativePath, b.relativePath); - swap(a.parameters, b.parameters); - swap(a.privileges, b.privileges); - swap(a.__isset, b.__isset); -} - -PartitionWithoutSD::PartitionWithoutSD(const PartitionWithoutSD& other251) { - values = other251.values; - createTime = other251.createTime; - lastAccessTime = other251.lastAccessTime; - relativePath = other251.relativePath; - parameters = other251.parameters; - privileges = other251.privileges; - __isset = other251.__isset; -} -PartitionWithoutSD& PartitionWithoutSD::operator=(const PartitionWithoutSD& other252) { - values = other252.values; - createTime = other252.createTime; - lastAccessTime = other252.lastAccessTime; - relativePath = other252.relativePath; - parameters = other252.parameters; - privileges = other252.privileges; - __isset = other252.__isset; - return *this; -} -void PartitionWithoutSD::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PartitionWithoutSD("; - out << "values=" << to_string(values); - out << ", " << "createTime=" << to_string(createTime); - out << ", " << "lastAccessTime=" << to_string(lastAccessTime); - out << ", " << "relativePath=" << to_string(relativePath); - out << ", " << "parameters=" << to_string(parameters); - out << ", " << "privileges="; (__isset.privileges ? (out << to_string(privileges)) : (out << "")); - out << ")"; -} - - -PartitionSpecWithSharedSD::~PartitionSpecWithSharedSD() noexcept { -} - - -void PartitionSpecWithSharedSD::__set_partitions(const std::vector & val) { - this->partitions = val; -} - -void PartitionSpecWithSharedSD::__set_sd(const StorageDescriptor& val) { - this->sd = val; -} -std::ostream& operator<<(std::ostream& out, const PartitionSpecWithSharedSD& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PartitionSpecWithSharedSD::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->partitions.clear(); - uint32_t _size253; - ::apache::thrift::protocol::TType _etype256; - xfer += iprot->readListBegin(_etype256, _size253); - this->partitions.resize(_size253); - uint32_t _i257; - for (_i257 = 0; _i257 < _size253; ++_i257) - { - xfer += this->partitions[_i257].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.partitions = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->sd.read(iprot); - this->__isset.sd = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t PartitionSpecWithSharedSD::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PartitionSpecWithSharedSD"); - - xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitions.size())); - std::vector ::const_iterator _iter258; - for (_iter258 = this->partitions.begin(); _iter258 != this->partitions.end(); ++_iter258) - { - xfer += (*_iter258).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("sd", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->sd.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PartitionSpecWithSharedSD &a, PartitionSpecWithSharedSD &b) { - using ::std::swap; - swap(a.partitions, b.partitions); - swap(a.sd, b.sd); - swap(a.__isset, b.__isset); -} - -PartitionSpecWithSharedSD::PartitionSpecWithSharedSD(const PartitionSpecWithSharedSD& other259) { - partitions = other259.partitions; - sd = other259.sd; - __isset = other259.__isset; -} -PartitionSpecWithSharedSD& PartitionSpecWithSharedSD::operator=(const PartitionSpecWithSharedSD& other260) { - partitions = other260.partitions; - sd = other260.sd; - __isset = other260.__isset; - return *this; -} -void PartitionSpecWithSharedSD::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PartitionSpecWithSharedSD("; - out << "partitions=" << to_string(partitions); - out << ", " << "sd=" << to_string(sd); - out << ")"; -} - - -PartitionListComposingSpec::~PartitionListComposingSpec() noexcept { -} - - -void PartitionListComposingSpec::__set_partitions(const std::vector & val) { - this->partitions = val; -} -std::ostream& operator<<(std::ostream& out, const PartitionListComposingSpec& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PartitionListComposingSpec::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->partitions.clear(); - uint32_t _size261; - ::apache::thrift::protocol::TType _etype264; - xfer += iprot->readListBegin(_etype264, _size261); - this->partitions.resize(_size261); - uint32_t _i265; - for (_i265 = 0; _i265 < _size261; ++_i265) - { - xfer += this->partitions[_i265].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.partitions = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t PartitionListComposingSpec::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PartitionListComposingSpec"); - - xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitions.size())); - std::vector ::const_iterator _iter266; - for (_iter266 = this->partitions.begin(); _iter266 != this->partitions.end(); ++_iter266) - { - xfer += (*_iter266).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PartitionListComposingSpec &a, PartitionListComposingSpec &b) { - using ::std::swap; - swap(a.partitions, b.partitions); - swap(a.__isset, b.__isset); -} - -PartitionListComposingSpec::PartitionListComposingSpec(const PartitionListComposingSpec& other267) { - partitions = other267.partitions; - __isset = other267.__isset; -} -PartitionListComposingSpec& PartitionListComposingSpec::operator=(const PartitionListComposingSpec& other268) { - partitions = other268.partitions; - __isset = other268.__isset; - return *this; -} -void PartitionListComposingSpec::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PartitionListComposingSpec("; - out << "partitions=" << to_string(partitions); - out << ")"; -} - - -PartitionSpec::~PartitionSpec() noexcept { -} - - -void PartitionSpec::__set_dbName(const std::string& val) { - this->dbName = val; -} - -void PartitionSpec::__set_tableName(const std::string& val) { - this->tableName = val; -} - -void PartitionSpec::__set_rootPath(const std::string& val) { - this->rootPath = val; -} - -void PartitionSpec::__set_sharedSDPartitionSpec(const PartitionSpecWithSharedSD& val) { - this->sharedSDPartitionSpec = val; -__isset.sharedSDPartitionSpec = true; -} - -void PartitionSpec::__set_partitionList(const PartitionListComposingSpec& val) { - this->partitionList = val; -__isset.partitionList = true; -} -std::ostream& operator<<(std::ostream& out, const PartitionSpec& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PartitionSpec::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbName); - this->__isset.dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tableName); - this->__isset.tableName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->rootPath); - this->__isset.rootPath = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->sharedSDPartitionSpec.read(iprot); - this->__isset.sharedSDPartitionSpec = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->partitionList.read(iprot); - this->__isset.partitionList = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t PartitionSpec::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PartitionSpec"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tableName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("rootPath", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->rootPath); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.sharedSDPartitionSpec) { - xfer += oprot->writeFieldBegin("sharedSDPartitionSpec", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->sharedSDPartitionSpec.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.partitionList) { - xfer += oprot->writeFieldBegin("partitionList", ::apache::thrift::protocol::T_STRUCT, 5); - xfer += this->partitionList.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PartitionSpec &a, PartitionSpec &b) { - using ::std::swap; - swap(a.dbName, b.dbName); - swap(a.tableName, b.tableName); - swap(a.rootPath, b.rootPath); - swap(a.sharedSDPartitionSpec, b.sharedSDPartitionSpec); - swap(a.partitionList, b.partitionList); - swap(a.__isset, b.__isset); -} - -PartitionSpec::PartitionSpec(const PartitionSpec& other269) { - dbName = other269.dbName; - tableName = other269.tableName; - rootPath = other269.rootPath; - sharedSDPartitionSpec = other269.sharedSDPartitionSpec; - partitionList = other269.partitionList; - __isset = other269.__isset; -} -PartitionSpec& PartitionSpec::operator=(const PartitionSpec& other270) { - dbName = other270.dbName; - tableName = other270.tableName; - rootPath = other270.rootPath; - sharedSDPartitionSpec = other270.sharedSDPartitionSpec; - partitionList = other270.partitionList; - __isset = other270.__isset; - return *this; -} -void PartitionSpec::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PartitionSpec("; - out << "dbName=" << to_string(dbName); - out << ", " << "tableName=" << to_string(tableName); - out << ", " << "rootPath=" << to_string(rootPath); - out << ", " << "sharedSDPartitionSpec="; (__isset.sharedSDPartitionSpec ? (out << to_string(sharedSDPartitionSpec)) : (out << "")); - out << ", " << "partitionList="; (__isset.partitionList ? (out << to_string(partitionList)) : (out << "")); - out << ")"; -} - - -Index::~Index() noexcept { -} - - -void Index::__set_indexName(const std::string& val) { - this->indexName = val; -} - -void Index::__set_indexHandlerClass(const std::string& val) { - this->indexHandlerClass = val; -} - -void Index::__set_dbName(const std::string& val) { - this->dbName = val; -} - -void Index::__set_origTableName(const std::string& val) { - this->origTableName = val; -} - -void Index::__set_createTime(const int32_t val) { - this->createTime = val; -} - -void Index::__set_lastAccessTime(const int32_t val) { - this->lastAccessTime = val; -} - -void Index::__set_indexTableName(const std::string& val) { - this->indexTableName = val; -} - -void Index::__set_sd(const StorageDescriptor& val) { - this->sd = val; -} - -void Index::__set_parameters(const std::map & val) { - this->parameters = val; -} - -void Index::__set_deferredRebuild(const bool val) { - this->deferredRebuild = val; -} -std::ostream& operator<<(std::ostream& out, const Index& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t Index::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->indexName); - this->__isset.indexName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->indexHandlerClass); - this->__isset.indexHandlerClass = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbName); - this->__isset.dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->origTableName); - this->__isset.origTableName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->createTime); - this->__isset.createTime = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->lastAccessTime); - this->__isset.lastAccessTime = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->indexTableName); - this->__isset.indexTableName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->sd.read(iprot); - this->__isset.sd = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 9: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->parameters.clear(); - uint32_t _size271; - ::apache::thrift::protocol::TType _ktype272; - ::apache::thrift::protocol::TType _vtype273; - xfer += iprot->readMapBegin(_ktype272, _vtype273, _size271); - uint32_t _i275; - for (_i275 = 0; _i275 < _size271; ++_i275) - { - std::string _key276; - xfer += iprot->readString(_key276); - std::string& _val277 = this->parameters[_key276]; - xfer += iprot->readString(_val277); - } - xfer += iprot->readMapEnd(); - } - this->__isset.parameters = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 10: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->deferredRebuild); - this->__isset.deferredRebuild = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t Index::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("Index"); - - xfer += oprot->writeFieldBegin("indexName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->indexName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("indexHandlerClass", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->indexHandlerClass); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("origTableName", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->origTableName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 5); - xfer += oprot->writeI32(this->createTime); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("lastAccessTime", ::apache::thrift::protocol::T_I32, 6); - xfer += oprot->writeI32(this->lastAccessTime); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("indexTableName", ::apache::thrift::protocol::T_STRING, 7); - xfer += oprot->writeString(this->indexTableName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("sd", ::apache::thrift::protocol::T_STRUCT, 8); - xfer += this->sd.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 9); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->parameters.size())); - std::map ::const_iterator _iter278; - for (_iter278 = this->parameters.begin(); _iter278 != this->parameters.end(); ++_iter278) - { - xfer += oprot->writeString(_iter278->first); - xfer += oprot->writeString(_iter278->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("deferredRebuild", ::apache::thrift::protocol::T_BOOL, 10); - xfer += oprot->writeBool(this->deferredRebuild); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(Index &a, Index &b) { - using ::std::swap; - swap(a.indexName, b.indexName); - swap(a.indexHandlerClass, b.indexHandlerClass); - swap(a.dbName, b.dbName); - swap(a.origTableName, b.origTableName); - swap(a.createTime, b.createTime); - swap(a.lastAccessTime, b.lastAccessTime); - swap(a.indexTableName, b.indexTableName); - swap(a.sd, b.sd); - swap(a.parameters, b.parameters); - swap(a.deferredRebuild, b.deferredRebuild); - swap(a.__isset, b.__isset); -} - -Index::Index(const Index& other279) { - indexName = other279.indexName; - indexHandlerClass = other279.indexHandlerClass; - dbName = other279.dbName; - origTableName = other279.origTableName; - createTime = other279.createTime; - lastAccessTime = other279.lastAccessTime; - indexTableName = other279.indexTableName; - sd = other279.sd; - parameters = other279.parameters; - deferredRebuild = other279.deferredRebuild; - __isset = other279.__isset; -} -Index& Index::operator=(const Index& other280) { - indexName = other280.indexName; - indexHandlerClass = other280.indexHandlerClass; - dbName = other280.dbName; - origTableName = other280.origTableName; - createTime = other280.createTime; - lastAccessTime = other280.lastAccessTime; - indexTableName = other280.indexTableName; - sd = other280.sd; - parameters = other280.parameters; - deferredRebuild = other280.deferredRebuild; - __isset = other280.__isset; - return *this; -} -void Index::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "Index("; - out << "indexName=" << to_string(indexName); - out << ", " << "indexHandlerClass=" << to_string(indexHandlerClass); - out << ", " << "dbName=" << to_string(dbName); - out << ", " << "origTableName=" << to_string(origTableName); - out << ", " << "createTime=" << to_string(createTime); - out << ", " << "lastAccessTime=" << to_string(lastAccessTime); - out << ", " << "indexTableName=" << to_string(indexTableName); - out << ", " << "sd=" << to_string(sd); - out << ", " << "parameters=" << to_string(parameters); - out << ", " << "deferredRebuild=" << to_string(deferredRebuild); - out << ")"; -} - - -BooleanColumnStatsData::~BooleanColumnStatsData() noexcept { -} - - -void BooleanColumnStatsData::__set_numTrues(const int64_t val) { - this->numTrues = val; -} - -void BooleanColumnStatsData::__set_numFalses(const int64_t val) { - this->numFalses = val; -} - -void BooleanColumnStatsData::__set_numNulls(const int64_t val) { - this->numNulls = val; -} - -void BooleanColumnStatsData::__set_bitVectors(const std::string& val) { - this->bitVectors = val; -__isset.bitVectors = true; -} -std::ostream& operator<<(std::ostream& out, const BooleanColumnStatsData& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t BooleanColumnStatsData::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_numTrues = false; - bool isset_numFalses = false; - bool isset_numNulls = 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_I64) { - xfer += iprot->readI64(this->numTrues); - isset_numTrues = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->numFalses); - isset_numFalses = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->numNulls); - isset_numNulls = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->bitVectors); - this->__isset.bitVectors = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_numTrues) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_numFalses) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_numNulls) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t BooleanColumnStatsData::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("BooleanColumnStatsData"); - - xfer += oprot->writeFieldBegin("numTrues", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->numTrues); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("numFalses", ::apache::thrift::protocol::T_I64, 2); - xfer += oprot->writeI64(this->numFalses); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("numNulls", ::apache::thrift::protocol::T_I64, 3); - xfer += oprot->writeI64(this->numNulls); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.bitVectors) { - xfer += oprot->writeFieldBegin("bitVectors", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeBinary(this->bitVectors); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(BooleanColumnStatsData &a, BooleanColumnStatsData &b) { - using ::std::swap; - swap(a.numTrues, b.numTrues); - swap(a.numFalses, b.numFalses); - swap(a.numNulls, b.numNulls); - swap(a.bitVectors, b.bitVectors); - swap(a.__isset, b.__isset); -} - -BooleanColumnStatsData::BooleanColumnStatsData(const BooleanColumnStatsData& other281) { - numTrues = other281.numTrues; - numFalses = other281.numFalses; - numNulls = other281.numNulls; - bitVectors = other281.bitVectors; - __isset = other281.__isset; -} -BooleanColumnStatsData& BooleanColumnStatsData::operator=(const BooleanColumnStatsData& other282) { - numTrues = other282.numTrues; - numFalses = other282.numFalses; - numNulls = other282.numNulls; - bitVectors = other282.bitVectors; - __isset = other282.__isset; - return *this; -} -void BooleanColumnStatsData::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "BooleanColumnStatsData("; - out << "numTrues=" << to_string(numTrues); - out << ", " << "numFalses=" << to_string(numFalses); - out << ", " << "numNulls=" << to_string(numNulls); - out << ", " << "bitVectors="; (__isset.bitVectors ? (out << to_string(bitVectors)) : (out << "")); - out << ")"; -} - - -DoubleColumnStatsData::~DoubleColumnStatsData() noexcept { -} - - -void DoubleColumnStatsData::__set_lowValue(const double val) { - this->lowValue = val; -__isset.lowValue = true; -} - -void DoubleColumnStatsData::__set_highValue(const double val) { - this->highValue = val; -__isset.highValue = true; -} - -void DoubleColumnStatsData::__set_numNulls(const int64_t val) { - this->numNulls = val; -} - -void DoubleColumnStatsData::__set_numDVs(const int64_t val) { - this->numDVs = val; -} - -void DoubleColumnStatsData::__set_bitVectors(const std::string& val) { - this->bitVectors = val; -__isset.bitVectors = true; -} -std::ostream& operator<<(std::ostream& out, const DoubleColumnStatsData& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t DoubleColumnStatsData::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_numNulls = false; - bool isset_numDVs = 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_DOUBLE) { - xfer += iprot->readDouble(this->lowValue); - this->__isset.lowValue = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_DOUBLE) { - xfer += iprot->readDouble(this->highValue); - this->__isset.highValue = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->numNulls); - isset_numNulls = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->numDVs); - isset_numDVs = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->bitVectors); - this->__isset.bitVectors = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_numNulls) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_numDVs) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t DoubleColumnStatsData::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("DoubleColumnStatsData"); - - if (this->__isset.lowValue) { - xfer += oprot->writeFieldBegin("lowValue", ::apache::thrift::protocol::T_DOUBLE, 1); - xfer += oprot->writeDouble(this->lowValue); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.highValue) { - xfer += oprot->writeFieldBegin("highValue", ::apache::thrift::protocol::T_DOUBLE, 2); - xfer += oprot->writeDouble(this->highValue); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldBegin("numNulls", ::apache::thrift::protocol::T_I64, 3); - xfer += oprot->writeI64(this->numNulls); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("numDVs", ::apache::thrift::protocol::T_I64, 4); - xfer += oprot->writeI64(this->numDVs); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.bitVectors) { - xfer += oprot->writeFieldBegin("bitVectors", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeBinary(this->bitVectors); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(DoubleColumnStatsData &a, DoubleColumnStatsData &b) { - using ::std::swap; - swap(a.lowValue, b.lowValue); - swap(a.highValue, b.highValue); - swap(a.numNulls, b.numNulls); - swap(a.numDVs, b.numDVs); - swap(a.bitVectors, b.bitVectors); - swap(a.__isset, b.__isset); -} - -DoubleColumnStatsData::DoubleColumnStatsData(const DoubleColumnStatsData& other283) { - lowValue = other283.lowValue; - highValue = other283.highValue; - numNulls = other283.numNulls; - numDVs = other283.numDVs; - bitVectors = other283.bitVectors; - __isset = other283.__isset; -} -DoubleColumnStatsData& DoubleColumnStatsData::operator=(const DoubleColumnStatsData& other284) { - lowValue = other284.lowValue; - highValue = other284.highValue; - numNulls = other284.numNulls; - numDVs = other284.numDVs; - bitVectors = other284.bitVectors; - __isset = other284.__isset; - return *this; -} -void DoubleColumnStatsData::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "DoubleColumnStatsData("; - out << "lowValue="; (__isset.lowValue ? (out << to_string(lowValue)) : (out << "")); - out << ", " << "highValue="; (__isset.highValue ? (out << to_string(highValue)) : (out << "")); - out << ", " << "numNulls=" << to_string(numNulls); - out << ", " << "numDVs=" << to_string(numDVs); - out << ", " << "bitVectors="; (__isset.bitVectors ? (out << to_string(bitVectors)) : (out << "")); - out << ")"; -} - - -LongColumnStatsData::~LongColumnStatsData() noexcept { -} - - -void LongColumnStatsData::__set_lowValue(const int64_t val) { - this->lowValue = val; -__isset.lowValue = true; -} - -void LongColumnStatsData::__set_highValue(const int64_t val) { - this->highValue = val; -__isset.highValue = true; -} - -void LongColumnStatsData::__set_numNulls(const int64_t val) { - this->numNulls = val; -} - -void LongColumnStatsData::__set_numDVs(const int64_t val) { - this->numDVs = val; -} - -void LongColumnStatsData::__set_bitVectors(const std::string& val) { - this->bitVectors = val; -__isset.bitVectors = true; -} -std::ostream& operator<<(std::ostream& out, const LongColumnStatsData& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t LongColumnStatsData::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_numNulls = false; - bool isset_numDVs = 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_I64) { - xfer += iprot->readI64(this->lowValue); - this->__isset.lowValue = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->highValue); - this->__isset.highValue = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->numNulls); - isset_numNulls = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->numDVs); - isset_numDVs = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->bitVectors); - this->__isset.bitVectors = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_numNulls) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_numDVs) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t LongColumnStatsData::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("LongColumnStatsData"); - - if (this->__isset.lowValue) { - xfer += oprot->writeFieldBegin("lowValue", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->lowValue); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.highValue) { - xfer += oprot->writeFieldBegin("highValue", ::apache::thrift::protocol::T_I64, 2); - xfer += oprot->writeI64(this->highValue); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldBegin("numNulls", ::apache::thrift::protocol::T_I64, 3); - xfer += oprot->writeI64(this->numNulls); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("numDVs", ::apache::thrift::protocol::T_I64, 4); - xfer += oprot->writeI64(this->numDVs); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.bitVectors) { - xfer += oprot->writeFieldBegin("bitVectors", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeBinary(this->bitVectors); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(LongColumnStatsData &a, LongColumnStatsData &b) { - using ::std::swap; - swap(a.lowValue, b.lowValue); - swap(a.highValue, b.highValue); - swap(a.numNulls, b.numNulls); - swap(a.numDVs, b.numDVs); - swap(a.bitVectors, b.bitVectors); - swap(a.__isset, b.__isset); -} - -LongColumnStatsData::LongColumnStatsData(const LongColumnStatsData& other285) { - lowValue = other285.lowValue; - highValue = other285.highValue; - numNulls = other285.numNulls; - numDVs = other285.numDVs; - bitVectors = other285.bitVectors; - __isset = other285.__isset; -} -LongColumnStatsData& LongColumnStatsData::operator=(const LongColumnStatsData& other286) { - lowValue = other286.lowValue; - highValue = other286.highValue; - numNulls = other286.numNulls; - numDVs = other286.numDVs; - bitVectors = other286.bitVectors; - __isset = other286.__isset; - return *this; -} -void LongColumnStatsData::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "LongColumnStatsData("; - out << "lowValue="; (__isset.lowValue ? (out << to_string(lowValue)) : (out << "")); - out << ", " << "highValue="; (__isset.highValue ? (out << to_string(highValue)) : (out << "")); - out << ", " << "numNulls=" << to_string(numNulls); - out << ", " << "numDVs=" << to_string(numDVs); - out << ", " << "bitVectors="; (__isset.bitVectors ? (out << to_string(bitVectors)) : (out << "")); - out << ")"; -} - - -StringColumnStatsData::~StringColumnStatsData() noexcept { -} - - -void StringColumnStatsData::__set_maxColLen(const int64_t val) { - this->maxColLen = val; -} - -void StringColumnStatsData::__set_avgColLen(const double val) { - this->avgColLen = val; -} - -void StringColumnStatsData::__set_numNulls(const int64_t val) { - this->numNulls = val; -} - -void StringColumnStatsData::__set_numDVs(const int64_t val) { - this->numDVs = val; -} - -void StringColumnStatsData::__set_bitVectors(const std::string& val) { - this->bitVectors = val; -__isset.bitVectors = true; -} -std::ostream& operator<<(std::ostream& out, const StringColumnStatsData& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t StringColumnStatsData::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_maxColLen = false; - bool isset_avgColLen = false; - bool isset_numNulls = false; - bool isset_numDVs = 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_I64) { - xfer += iprot->readI64(this->maxColLen); - isset_maxColLen = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_DOUBLE) { - xfer += iprot->readDouble(this->avgColLen); - isset_avgColLen = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->numNulls); - isset_numNulls = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->numDVs); - isset_numDVs = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->bitVectors); - this->__isset.bitVectors = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_maxColLen) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_avgColLen) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_numNulls) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_numDVs) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t StringColumnStatsData::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("StringColumnStatsData"); - - xfer += oprot->writeFieldBegin("maxColLen", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->maxColLen); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("avgColLen", ::apache::thrift::protocol::T_DOUBLE, 2); - xfer += oprot->writeDouble(this->avgColLen); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("numNulls", ::apache::thrift::protocol::T_I64, 3); - xfer += oprot->writeI64(this->numNulls); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("numDVs", ::apache::thrift::protocol::T_I64, 4); - xfer += oprot->writeI64(this->numDVs); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.bitVectors) { - xfer += oprot->writeFieldBegin("bitVectors", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeBinary(this->bitVectors); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(StringColumnStatsData &a, StringColumnStatsData &b) { - using ::std::swap; - swap(a.maxColLen, b.maxColLen); - swap(a.avgColLen, b.avgColLen); - swap(a.numNulls, b.numNulls); - swap(a.numDVs, b.numDVs); - swap(a.bitVectors, b.bitVectors); - swap(a.__isset, b.__isset); -} - -StringColumnStatsData::StringColumnStatsData(const StringColumnStatsData& other287) { - maxColLen = other287.maxColLen; - avgColLen = other287.avgColLen; - numNulls = other287.numNulls; - numDVs = other287.numDVs; - bitVectors = other287.bitVectors; - __isset = other287.__isset; -} -StringColumnStatsData& StringColumnStatsData::operator=(const StringColumnStatsData& other288) { - maxColLen = other288.maxColLen; - avgColLen = other288.avgColLen; - numNulls = other288.numNulls; - numDVs = other288.numDVs; - bitVectors = other288.bitVectors; - __isset = other288.__isset; - return *this; -} -void StringColumnStatsData::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "StringColumnStatsData("; - out << "maxColLen=" << to_string(maxColLen); - out << ", " << "avgColLen=" << to_string(avgColLen); - out << ", " << "numNulls=" << to_string(numNulls); - out << ", " << "numDVs=" << to_string(numDVs); - out << ", " << "bitVectors="; (__isset.bitVectors ? (out << to_string(bitVectors)) : (out << "")); - out << ")"; -} - - -BinaryColumnStatsData::~BinaryColumnStatsData() noexcept { -} - - -void BinaryColumnStatsData::__set_maxColLen(const int64_t val) { - this->maxColLen = val; -} - -void BinaryColumnStatsData::__set_avgColLen(const double val) { - this->avgColLen = val; -} - -void BinaryColumnStatsData::__set_numNulls(const int64_t val) { - this->numNulls = val; -} - -void BinaryColumnStatsData::__set_bitVectors(const std::string& val) { - this->bitVectors = val; -__isset.bitVectors = true; -} -std::ostream& operator<<(std::ostream& out, const BinaryColumnStatsData& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t BinaryColumnStatsData::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_maxColLen = false; - bool isset_avgColLen = false; - bool isset_numNulls = 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_I64) { - xfer += iprot->readI64(this->maxColLen); - isset_maxColLen = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_DOUBLE) { - xfer += iprot->readDouble(this->avgColLen); - isset_avgColLen = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->numNulls); - isset_numNulls = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->bitVectors); - this->__isset.bitVectors = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_maxColLen) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_avgColLen) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_numNulls) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t BinaryColumnStatsData::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("BinaryColumnStatsData"); - - xfer += oprot->writeFieldBegin("maxColLen", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->maxColLen); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("avgColLen", ::apache::thrift::protocol::T_DOUBLE, 2); - xfer += oprot->writeDouble(this->avgColLen); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("numNulls", ::apache::thrift::protocol::T_I64, 3); - xfer += oprot->writeI64(this->numNulls); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.bitVectors) { - xfer += oprot->writeFieldBegin("bitVectors", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeBinary(this->bitVectors); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(BinaryColumnStatsData &a, BinaryColumnStatsData &b) { - using ::std::swap; - swap(a.maxColLen, b.maxColLen); - swap(a.avgColLen, b.avgColLen); - swap(a.numNulls, b.numNulls); - swap(a.bitVectors, b.bitVectors); - swap(a.__isset, b.__isset); -} - -BinaryColumnStatsData::BinaryColumnStatsData(const BinaryColumnStatsData& other289) { - maxColLen = other289.maxColLen; - avgColLen = other289.avgColLen; - numNulls = other289.numNulls; - bitVectors = other289.bitVectors; - __isset = other289.__isset; -} -BinaryColumnStatsData& BinaryColumnStatsData::operator=(const BinaryColumnStatsData& other290) { - maxColLen = other290.maxColLen; - avgColLen = other290.avgColLen; - numNulls = other290.numNulls; - bitVectors = other290.bitVectors; - __isset = other290.__isset; - return *this; -} -void BinaryColumnStatsData::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "BinaryColumnStatsData("; - out << "maxColLen=" << to_string(maxColLen); - out << ", " << "avgColLen=" << to_string(avgColLen); - out << ", " << "numNulls=" << to_string(numNulls); - out << ", " << "bitVectors="; (__isset.bitVectors ? (out << to_string(bitVectors)) : (out << "")); - out << ")"; -} - - -Decimal::~Decimal() noexcept { -} - - -void Decimal::__set_unscaled(const std::string& val) { - this->unscaled = val; -} - -void Decimal::__set_scale(const int16_t val) { - this->scale = val; -} -std::ostream& operator<<(std::ostream& out, const Decimal& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t Decimal::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_unscaled = false; - bool isset_scale = 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_STRING) { - xfer += iprot->readBinary(this->unscaled); - isset_unscaled = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I16) { - xfer += iprot->readI16(this->scale); - isset_scale = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_unscaled) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_scale) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t Decimal::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("Decimal"); - - xfer += oprot->writeFieldBegin("unscaled", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeBinary(this->unscaled); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("scale", ::apache::thrift::protocol::T_I16, 3); - xfer += oprot->writeI16(this->scale); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(Decimal &a, Decimal &b) { - using ::std::swap; - swap(a.unscaled, b.unscaled); - swap(a.scale, b.scale); -} - -Decimal::Decimal(const Decimal& other291) { - unscaled = other291.unscaled; - scale = other291.scale; -} -Decimal& Decimal::operator=(const Decimal& other292) { - unscaled = other292.unscaled; - scale = other292.scale; - return *this; -} -void Decimal::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "Decimal("; - out << "unscaled=" << to_string(unscaled); - out << ", " << "scale=" << to_string(scale); - out << ")"; -} - - -DecimalColumnStatsData::~DecimalColumnStatsData() noexcept { -} - - -void DecimalColumnStatsData::__set_lowValue(const Decimal& val) { - this->lowValue = val; -__isset.lowValue = true; -} - -void DecimalColumnStatsData::__set_highValue(const Decimal& val) { - this->highValue = val; -__isset.highValue = true; -} - -void DecimalColumnStatsData::__set_numNulls(const int64_t val) { - this->numNulls = val; -} - -void DecimalColumnStatsData::__set_numDVs(const int64_t val) { - this->numDVs = val; -} - -void DecimalColumnStatsData::__set_bitVectors(const std::string& val) { - this->bitVectors = val; -__isset.bitVectors = true; -} -std::ostream& operator<<(std::ostream& out, const DecimalColumnStatsData& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t DecimalColumnStatsData::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_numNulls = false; - bool isset_numDVs = 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_STRUCT) { - xfer += this->lowValue.read(iprot); - this->__isset.lowValue = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->highValue.read(iprot); - this->__isset.highValue = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->numNulls); - isset_numNulls = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->numDVs); - isset_numDVs = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->bitVectors); - this->__isset.bitVectors = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_numNulls) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_numDVs) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t DecimalColumnStatsData::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("DecimalColumnStatsData"); - - if (this->__isset.lowValue) { - xfer += oprot->writeFieldBegin("lowValue", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->lowValue.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.highValue) { - xfer += oprot->writeFieldBegin("highValue", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->highValue.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldBegin("numNulls", ::apache::thrift::protocol::T_I64, 3); - xfer += oprot->writeI64(this->numNulls); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("numDVs", ::apache::thrift::protocol::T_I64, 4); - xfer += oprot->writeI64(this->numDVs); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.bitVectors) { - xfer += oprot->writeFieldBegin("bitVectors", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeBinary(this->bitVectors); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(DecimalColumnStatsData &a, DecimalColumnStatsData &b) { - using ::std::swap; - swap(a.lowValue, b.lowValue); - swap(a.highValue, b.highValue); - swap(a.numNulls, b.numNulls); - swap(a.numDVs, b.numDVs); - swap(a.bitVectors, b.bitVectors); - swap(a.__isset, b.__isset); -} - -DecimalColumnStatsData::DecimalColumnStatsData(const DecimalColumnStatsData& other293) { - lowValue = other293.lowValue; - highValue = other293.highValue; - numNulls = other293.numNulls; - numDVs = other293.numDVs; - bitVectors = other293.bitVectors; - __isset = other293.__isset; -} -DecimalColumnStatsData& DecimalColumnStatsData::operator=(const DecimalColumnStatsData& other294) { - lowValue = other294.lowValue; - highValue = other294.highValue; - numNulls = other294.numNulls; - numDVs = other294.numDVs; - bitVectors = other294.bitVectors; - __isset = other294.__isset; - return *this; -} -void DecimalColumnStatsData::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "DecimalColumnStatsData("; - out << "lowValue="; (__isset.lowValue ? (out << to_string(lowValue)) : (out << "")); - out << ", " << "highValue="; (__isset.highValue ? (out << to_string(highValue)) : (out << "")); - out << ", " << "numNulls=" << to_string(numNulls); - out << ", " << "numDVs=" << to_string(numDVs); - out << ", " << "bitVectors="; (__isset.bitVectors ? (out << to_string(bitVectors)) : (out << "")); - out << ")"; -} - - -Date::~Date() noexcept { -} - - -void Date::__set_daysSinceEpoch(const int64_t val) { - this->daysSinceEpoch = val; -} -std::ostream& operator<<(std::ostream& out, const Date& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t Date::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_daysSinceEpoch = 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_I64) { - xfer += iprot->readI64(this->daysSinceEpoch); - isset_daysSinceEpoch = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_daysSinceEpoch) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t Date::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("Date"); - - xfer += oprot->writeFieldBegin("daysSinceEpoch", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->daysSinceEpoch); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(Date &a, Date &b) { - using ::std::swap; - swap(a.daysSinceEpoch, b.daysSinceEpoch); -} - -Date::Date(const Date& other295) { - daysSinceEpoch = other295.daysSinceEpoch; -} -Date& Date::operator=(const Date& other296) { - daysSinceEpoch = other296.daysSinceEpoch; - return *this; -} -void Date::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "Date("; - out << "daysSinceEpoch=" << to_string(daysSinceEpoch); - out << ")"; -} - - -DateColumnStatsData::~DateColumnStatsData() noexcept { -} - - -void DateColumnStatsData::__set_lowValue(const Date& val) { - this->lowValue = val; -__isset.lowValue = true; -} - -void DateColumnStatsData::__set_highValue(const Date& val) { - this->highValue = val; -__isset.highValue = true; -} - -void DateColumnStatsData::__set_numNulls(const int64_t val) { - this->numNulls = val; -} - -void DateColumnStatsData::__set_numDVs(const int64_t val) { - this->numDVs = val; -} - -void DateColumnStatsData::__set_bitVectors(const std::string& val) { - this->bitVectors = val; -__isset.bitVectors = true; -} -std::ostream& operator<<(std::ostream& out, const DateColumnStatsData& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t DateColumnStatsData::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_numNulls = false; - bool isset_numDVs = 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_STRUCT) { - xfer += this->lowValue.read(iprot); - this->__isset.lowValue = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->highValue.read(iprot); - this->__isset.highValue = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->numNulls); - isset_numNulls = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->numDVs); - isset_numDVs = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->bitVectors); - this->__isset.bitVectors = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_numNulls) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_numDVs) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t DateColumnStatsData::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("DateColumnStatsData"); - - if (this->__isset.lowValue) { - xfer += oprot->writeFieldBegin("lowValue", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->lowValue.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.highValue) { - xfer += oprot->writeFieldBegin("highValue", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->highValue.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldBegin("numNulls", ::apache::thrift::protocol::T_I64, 3); - xfer += oprot->writeI64(this->numNulls); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("numDVs", ::apache::thrift::protocol::T_I64, 4); - xfer += oprot->writeI64(this->numDVs); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.bitVectors) { - xfer += oprot->writeFieldBegin("bitVectors", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeBinary(this->bitVectors); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(DateColumnStatsData &a, DateColumnStatsData &b) { - using ::std::swap; - swap(a.lowValue, b.lowValue); - swap(a.highValue, b.highValue); - swap(a.numNulls, b.numNulls); - swap(a.numDVs, b.numDVs); - swap(a.bitVectors, b.bitVectors); - swap(a.__isset, b.__isset); -} - -DateColumnStatsData::DateColumnStatsData(const DateColumnStatsData& other297) { - lowValue = other297.lowValue; - highValue = other297.highValue; - numNulls = other297.numNulls; - numDVs = other297.numDVs; - bitVectors = other297.bitVectors; - __isset = other297.__isset; -} -DateColumnStatsData& DateColumnStatsData::operator=(const DateColumnStatsData& other298) { - lowValue = other298.lowValue; - highValue = other298.highValue; - numNulls = other298.numNulls; - numDVs = other298.numDVs; - bitVectors = other298.bitVectors; - __isset = other298.__isset; - return *this; -} -void DateColumnStatsData::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "DateColumnStatsData("; - out << "lowValue="; (__isset.lowValue ? (out << to_string(lowValue)) : (out << "")); - out << ", " << "highValue="; (__isset.highValue ? (out << to_string(highValue)) : (out << "")); - out << ", " << "numNulls=" << to_string(numNulls); - out << ", " << "numDVs=" << to_string(numDVs); - out << ", " << "bitVectors="; (__isset.bitVectors ? (out << to_string(bitVectors)) : (out << "")); - out << ")"; -} - - -ColumnStatisticsData::~ColumnStatisticsData() noexcept { -} - - -void ColumnStatisticsData::__set_booleanStats(const BooleanColumnStatsData& val) { - this->booleanStats = val; -__isset.booleanStats = true; -} - -void ColumnStatisticsData::__set_longStats(const LongColumnStatsData& val) { - this->longStats = val; -__isset.longStats = true; -} - -void ColumnStatisticsData::__set_doubleStats(const DoubleColumnStatsData& val) { - this->doubleStats = val; -__isset.doubleStats = true; -} - -void ColumnStatisticsData::__set_stringStats(const StringColumnStatsData& val) { - this->stringStats = val; -__isset.stringStats = true; -} - -void ColumnStatisticsData::__set_binaryStats(const BinaryColumnStatsData& val) { - this->binaryStats = val; -__isset.binaryStats = true; -} - -void ColumnStatisticsData::__set_decimalStats(const DecimalColumnStatsData& val) { - this->decimalStats = val; -__isset.decimalStats = true; -} - -void ColumnStatisticsData::__set_dateStats(const DateColumnStatsData& val) { - this->dateStats = val; -__isset.dateStats = true; -} -std::ostream& operator<<(std::ostream& out, const ColumnStatisticsData& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ColumnStatisticsData::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->booleanStats.read(iprot); - this->__isset.booleanStats = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->longStats.read(iprot); - this->__isset.longStats = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->doubleStats.read(iprot); - this->__isset.doubleStats = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->stringStats.read(iprot); - this->__isset.stringStats = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->binaryStats.read(iprot); - this->__isset.binaryStats = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->decimalStats.read(iprot); - this->__isset.decimalStats = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->dateStats.read(iprot); - this->__isset.dateStats = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ColumnStatisticsData::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ColumnStatisticsData"); - - if (this->__isset.booleanStats) { - xfer += oprot->writeFieldBegin("booleanStats", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->booleanStats.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.longStats) { - xfer += oprot->writeFieldBegin("longStats", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->longStats.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.doubleStats) { - xfer += oprot->writeFieldBegin("doubleStats", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->doubleStats.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.stringStats) { - xfer += oprot->writeFieldBegin("stringStats", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->stringStats.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.binaryStats) { - xfer += oprot->writeFieldBegin("binaryStats", ::apache::thrift::protocol::T_STRUCT, 5); - xfer += this->binaryStats.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.decimalStats) { - xfer += oprot->writeFieldBegin("decimalStats", ::apache::thrift::protocol::T_STRUCT, 6); - xfer += this->decimalStats.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.dateStats) { - xfer += oprot->writeFieldBegin("dateStats", ::apache::thrift::protocol::T_STRUCT, 7); - xfer += this->dateStats.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ColumnStatisticsData &a, ColumnStatisticsData &b) { - using ::std::swap; - swap(a.booleanStats, b.booleanStats); - swap(a.longStats, b.longStats); - swap(a.doubleStats, b.doubleStats); - swap(a.stringStats, b.stringStats); - swap(a.binaryStats, b.binaryStats); - swap(a.decimalStats, b.decimalStats); - swap(a.dateStats, b.dateStats); - swap(a.__isset, b.__isset); -} - -ColumnStatisticsData::ColumnStatisticsData(const ColumnStatisticsData& other299) { - booleanStats = other299.booleanStats; - longStats = other299.longStats; - doubleStats = other299.doubleStats; - stringStats = other299.stringStats; - binaryStats = other299.binaryStats; - decimalStats = other299.decimalStats; - dateStats = other299.dateStats; - __isset = other299.__isset; -} -ColumnStatisticsData& ColumnStatisticsData::operator=(const ColumnStatisticsData& other300) { - booleanStats = other300.booleanStats; - longStats = other300.longStats; - doubleStats = other300.doubleStats; - stringStats = other300.stringStats; - binaryStats = other300.binaryStats; - decimalStats = other300.decimalStats; - dateStats = other300.dateStats; - __isset = other300.__isset; - return *this; -} -void ColumnStatisticsData::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ColumnStatisticsData("; - out << "booleanStats="; (__isset.booleanStats ? (out << to_string(booleanStats)) : (out << "")); - out << ", " << "longStats="; (__isset.longStats ? (out << to_string(longStats)) : (out << "")); - out << ", " << "doubleStats="; (__isset.doubleStats ? (out << to_string(doubleStats)) : (out << "")); - out << ", " << "stringStats="; (__isset.stringStats ? (out << to_string(stringStats)) : (out << "")); - out << ", " << "binaryStats="; (__isset.binaryStats ? (out << to_string(binaryStats)) : (out << "")); - out << ", " << "decimalStats="; (__isset.decimalStats ? (out << to_string(decimalStats)) : (out << "")); - out << ", " << "dateStats="; (__isset.dateStats ? (out << to_string(dateStats)) : (out << "")); - out << ")"; -} - - -ColumnStatisticsObj::~ColumnStatisticsObj() noexcept { -} - - -void ColumnStatisticsObj::__set_colName(const std::string& val) { - this->colName = val; -} - -void ColumnStatisticsObj::__set_colType(const std::string& val) { - this->colType = val; -} - -void ColumnStatisticsObj::__set_statsData(const ColumnStatisticsData& val) { - this->statsData = val; -} -std::ostream& operator<<(std::ostream& out, const ColumnStatisticsObj& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ColumnStatisticsObj::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_colName = false; - bool isset_colType = false; - bool isset_statsData = 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_STRING) { - xfer += iprot->readString(this->colName); - isset_colName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->colType); - isset_colType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->statsData.read(iprot); - isset_statsData = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_colName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_colType) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_statsData) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t ColumnStatisticsObj::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ColumnStatisticsObj"); - - xfer += oprot->writeFieldBegin("colName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->colName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("colType", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->colType); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("statsData", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->statsData.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ColumnStatisticsObj &a, ColumnStatisticsObj &b) { - using ::std::swap; - swap(a.colName, b.colName); - swap(a.colType, b.colType); - swap(a.statsData, b.statsData); -} - -ColumnStatisticsObj::ColumnStatisticsObj(const ColumnStatisticsObj& other301) { - colName = other301.colName; - colType = other301.colType; - statsData = other301.statsData; -} -ColumnStatisticsObj& ColumnStatisticsObj::operator=(const ColumnStatisticsObj& other302) { - colName = other302.colName; - colType = other302.colType; - statsData = other302.statsData; - return *this; -} -void ColumnStatisticsObj::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ColumnStatisticsObj("; - out << "colName=" << to_string(colName); - out << ", " << "colType=" << to_string(colType); - out << ", " << "statsData=" << to_string(statsData); - out << ")"; -} - - -ColumnStatisticsDesc::~ColumnStatisticsDesc() noexcept { -} - - -void ColumnStatisticsDesc::__set_isTblLevel(const bool val) { - this->isTblLevel = val; -} - -void ColumnStatisticsDesc::__set_dbName(const std::string& val) { - this->dbName = val; -} - -void ColumnStatisticsDesc::__set_tableName(const std::string& val) { - this->tableName = val; -} - -void ColumnStatisticsDesc::__set_partName(const std::string& val) { - this->partName = val; -__isset.partName = true; -} - -void ColumnStatisticsDesc::__set_lastAnalyzed(const int64_t val) { - this->lastAnalyzed = val; -__isset.lastAnalyzed = true; -} -std::ostream& operator<<(std::ostream& out, const ColumnStatisticsDesc& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ColumnStatisticsDesc::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_isTblLevel = false; - bool isset_dbName = false; - bool isset_tableName = 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_BOOL) { - xfer += iprot->readBool(this->isTblLevel); - isset_isTblLevel = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbName); - isset_dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tableName); - isset_tableName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->partName); - this->__isset.partName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->lastAnalyzed); - this->__isset.lastAnalyzed = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_isTblLevel) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_dbName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tableName) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t ColumnStatisticsDesc::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ColumnStatisticsDesc"); - - xfer += oprot->writeFieldBegin("isTblLevel", ::apache::thrift::protocol::T_BOOL, 1); - xfer += oprot->writeBool(this->isTblLevel); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->tableName); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.partName) { - xfer += oprot->writeFieldBegin("partName", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->partName); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.lastAnalyzed) { - xfer += oprot->writeFieldBegin("lastAnalyzed", ::apache::thrift::protocol::T_I64, 5); - xfer += oprot->writeI64(this->lastAnalyzed); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ColumnStatisticsDesc &a, ColumnStatisticsDesc &b) { - using ::std::swap; - swap(a.isTblLevel, b.isTblLevel); - swap(a.dbName, b.dbName); - swap(a.tableName, b.tableName); - swap(a.partName, b.partName); - swap(a.lastAnalyzed, b.lastAnalyzed); - swap(a.__isset, b.__isset); -} - -ColumnStatisticsDesc::ColumnStatisticsDesc(const ColumnStatisticsDesc& other303) { - isTblLevel = other303.isTblLevel; - dbName = other303.dbName; - tableName = other303.tableName; - partName = other303.partName; - lastAnalyzed = other303.lastAnalyzed; - __isset = other303.__isset; -} -ColumnStatisticsDesc& ColumnStatisticsDesc::operator=(const ColumnStatisticsDesc& other304) { - isTblLevel = other304.isTblLevel; - dbName = other304.dbName; - tableName = other304.tableName; - partName = other304.partName; - lastAnalyzed = other304.lastAnalyzed; - __isset = other304.__isset; - return *this; -} -void ColumnStatisticsDesc::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ColumnStatisticsDesc("; - out << "isTblLevel=" << to_string(isTblLevel); - out << ", " << "dbName=" << to_string(dbName); - out << ", " << "tableName=" << to_string(tableName); - out << ", " << "partName="; (__isset.partName ? (out << to_string(partName)) : (out << "")); - out << ", " << "lastAnalyzed="; (__isset.lastAnalyzed ? (out << to_string(lastAnalyzed)) : (out << "")); - out << ")"; -} - - -ColumnStatistics::~ColumnStatistics() noexcept { -} - - -void ColumnStatistics::__set_statsDesc(const ColumnStatisticsDesc& val) { - this->statsDesc = val; -} - -void ColumnStatistics::__set_statsObj(const std::vector & val) { - this->statsObj = val; -} -std::ostream& operator<<(std::ostream& out, const ColumnStatistics& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ColumnStatistics::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_statsDesc = false; - bool isset_statsObj = 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_STRUCT) { - xfer += this->statsDesc.read(iprot); - isset_statsDesc = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->statsObj.clear(); - uint32_t _size305; - ::apache::thrift::protocol::TType _etype308; - xfer += iprot->readListBegin(_etype308, _size305); - this->statsObj.resize(_size305); - uint32_t _i309; - for (_i309 = 0; _i309 < _size305; ++_i309) - { - xfer += this->statsObj[_i309].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_statsObj = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_statsDesc) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_statsObj) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t ColumnStatistics::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ColumnStatistics"); - - xfer += oprot->writeFieldBegin("statsDesc", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->statsDesc.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("statsObj", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->statsObj.size())); - std::vector ::const_iterator _iter310; - for (_iter310 = this->statsObj.begin(); _iter310 != this->statsObj.end(); ++_iter310) - { - xfer += (*_iter310).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ColumnStatistics &a, ColumnStatistics &b) { - using ::std::swap; - swap(a.statsDesc, b.statsDesc); - swap(a.statsObj, b.statsObj); -} - -ColumnStatistics::ColumnStatistics(const ColumnStatistics& other311) { - statsDesc = other311.statsDesc; - statsObj = other311.statsObj; -} -ColumnStatistics& ColumnStatistics::operator=(const ColumnStatistics& other312) { - statsDesc = other312.statsDesc; - statsObj = other312.statsObj; - return *this; -} -void ColumnStatistics::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ColumnStatistics("; - out << "statsDesc=" << to_string(statsDesc); - out << ", " << "statsObj=" << to_string(statsObj); - out << ")"; -} - - -AggrStats::~AggrStats() noexcept { -} - - -void AggrStats::__set_colStats(const std::vector & val) { - this->colStats = val; -} - -void AggrStats::__set_partsFound(const int64_t val) { - this->partsFound = val; -} -std::ostream& operator<<(std::ostream& out, const AggrStats& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t AggrStats::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_colStats = false; - bool isset_partsFound = 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_LIST) { - { - this->colStats.clear(); - uint32_t _size313; - ::apache::thrift::protocol::TType _etype316; - xfer += iprot->readListBegin(_etype316, _size313); - this->colStats.resize(_size313); - uint32_t _i317; - for (_i317 = 0; _i317 < _size313; ++_i317) - { - xfer += this->colStats[_i317].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_colStats = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->partsFound); - isset_partsFound = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_colStats) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_partsFound) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t AggrStats::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("AggrStats"); - - xfer += oprot->writeFieldBegin("colStats", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->colStats.size())); - std::vector ::const_iterator _iter318; - for (_iter318 = this->colStats.begin(); _iter318 != this->colStats.end(); ++_iter318) - { - xfer += (*_iter318).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("partsFound", ::apache::thrift::protocol::T_I64, 2); - xfer += oprot->writeI64(this->partsFound); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(AggrStats &a, AggrStats &b) { - using ::std::swap; - swap(a.colStats, b.colStats); - swap(a.partsFound, b.partsFound); -} - -AggrStats::AggrStats(const AggrStats& other319) { - colStats = other319.colStats; - partsFound = other319.partsFound; -} -AggrStats& AggrStats::operator=(const AggrStats& other320) { - colStats = other320.colStats; - partsFound = other320.partsFound; - return *this; -} -void AggrStats::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "AggrStats("; - out << "colStats=" << to_string(colStats); - out << ", " << "partsFound=" << to_string(partsFound); - out << ")"; -} - - -SetPartitionsStatsRequest::~SetPartitionsStatsRequest() noexcept { -} - - -void SetPartitionsStatsRequest::__set_colStats(const std::vector & val) { - this->colStats = val; -} - -void SetPartitionsStatsRequest::__set_needMerge(const bool val) { - this->needMerge = val; -__isset.needMerge = true; -} -std::ostream& operator<<(std::ostream& out, const SetPartitionsStatsRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t SetPartitionsStatsRequest::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_colStats = 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_LIST) { - { - this->colStats.clear(); - uint32_t _size321; - ::apache::thrift::protocol::TType _etype324; - xfer += iprot->readListBegin(_etype324, _size321); - this->colStats.resize(_size321); - uint32_t _i325; - for (_i325 = 0; _i325 < _size321; ++_i325) - { - xfer += this->colStats[_i325].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_colStats = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->needMerge); - this->__isset.needMerge = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_colStats) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t SetPartitionsStatsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("SetPartitionsStatsRequest"); - - xfer += oprot->writeFieldBegin("colStats", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->colStats.size())); - std::vector ::const_iterator _iter326; - for (_iter326 = this->colStats.begin(); _iter326 != this->colStats.end(); ++_iter326) - { - xfer += (*_iter326).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - if (this->__isset.needMerge) { - xfer += oprot->writeFieldBegin("needMerge", ::apache::thrift::protocol::T_BOOL, 2); - xfer += oprot->writeBool(this->needMerge); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(SetPartitionsStatsRequest &a, SetPartitionsStatsRequest &b) { - using ::std::swap; - swap(a.colStats, b.colStats); - swap(a.needMerge, b.needMerge); - swap(a.__isset, b.__isset); -} - -SetPartitionsStatsRequest::SetPartitionsStatsRequest(const SetPartitionsStatsRequest& other327) { - colStats = other327.colStats; - needMerge = other327.needMerge; - __isset = other327.__isset; -} -SetPartitionsStatsRequest& SetPartitionsStatsRequest::operator=(const SetPartitionsStatsRequest& other328) { - colStats = other328.colStats; - needMerge = other328.needMerge; - __isset = other328.__isset; - return *this; -} -void SetPartitionsStatsRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "SetPartitionsStatsRequest("; - out << "colStats=" << to_string(colStats); - out << ", " << "needMerge="; (__isset.needMerge ? (out << to_string(needMerge)) : (out << "")); - out << ")"; -} - - -Schema::~Schema() noexcept { -} - - -void Schema::__set_fieldSchemas(const std::vector & val) { - this->fieldSchemas = val; -} - -void Schema::__set_properties(const std::map & val) { - this->properties = val; -} -std::ostream& operator<<(std::ostream& out, const Schema& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t Schema::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->fieldSchemas.clear(); - uint32_t _size329; - ::apache::thrift::protocol::TType _etype332; - xfer += iprot->readListBegin(_etype332, _size329); - this->fieldSchemas.resize(_size329); - uint32_t _i333; - for (_i333 = 0; _i333 < _size329; ++_i333) - { - xfer += this->fieldSchemas[_i333].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.fieldSchemas = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->properties.clear(); - uint32_t _size334; - ::apache::thrift::protocol::TType _ktype335; - ::apache::thrift::protocol::TType _vtype336; - xfer += iprot->readMapBegin(_ktype335, _vtype336, _size334); - uint32_t _i338; - for (_i338 = 0; _i338 < _size334; ++_i338) - { - std::string _key339; - xfer += iprot->readString(_key339); - std::string& _val340 = this->properties[_key339]; - xfer += iprot->readString(_val340); - } - xfer += iprot->readMapEnd(); - } - this->__isset.properties = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t Schema::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("Schema"); - - xfer += oprot->writeFieldBegin("fieldSchemas", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->fieldSchemas.size())); - std::vector ::const_iterator _iter341; - for (_iter341 = this->fieldSchemas.begin(); _iter341 != this->fieldSchemas.end(); ++_iter341) - { - xfer += (*_iter341).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 2); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->properties.size())); - std::map ::const_iterator _iter342; - for (_iter342 = this->properties.begin(); _iter342 != this->properties.end(); ++_iter342) - { - xfer += oprot->writeString(_iter342->first); - xfer += oprot->writeString(_iter342->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(Schema &a, Schema &b) { - using ::std::swap; - swap(a.fieldSchemas, b.fieldSchemas); - swap(a.properties, b.properties); - swap(a.__isset, b.__isset); -} - -Schema::Schema(const Schema& other343) { - fieldSchemas = other343.fieldSchemas; - properties = other343.properties; - __isset = other343.__isset; -} -Schema& Schema::operator=(const Schema& other344) { - fieldSchemas = other344.fieldSchemas; - properties = other344.properties; - __isset = other344.__isset; - return *this; -} -void Schema::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "Schema("; - out << "fieldSchemas=" << to_string(fieldSchemas); - out << ", " << "properties=" << to_string(properties); - out << ")"; -} - - -EnvironmentContext::~EnvironmentContext() noexcept { -} - - -void EnvironmentContext::__set_properties(const std::map & val) { - this->properties = val; -} -std::ostream& operator<<(std::ostream& out, const EnvironmentContext& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t EnvironmentContext::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->properties.clear(); - uint32_t _size345; - ::apache::thrift::protocol::TType _ktype346; - ::apache::thrift::protocol::TType _vtype347; - xfer += iprot->readMapBegin(_ktype346, _vtype347, _size345); - uint32_t _i349; - for (_i349 = 0; _i349 < _size345; ++_i349) - { - std::string _key350; - xfer += iprot->readString(_key350); - std::string& _val351 = this->properties[_key350]; - xfer += iprot->readString(_val351); - } - xfer += iprot->readMapEnd(); - } - this->__isset.properties = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t EnvironmentContext::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("EnvironmentContext"); - - xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 1); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->properties.size())); - std::map ::const_iterator _iter352; - for (_iter352 = this->properties.begin(); _iter352 != this->properties.end(); ++_iter352) - { - xfer += oprot->writeString(_iter352->first); - xfer += oprot->writeString(_iter352->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(EnvironmentContext &a, EnvironmentContext &b) { - using ::std::swap; - swap(a.properties, b.properties); - swap(a.__isset, b.__isset); -} - -EnvironmentContext::EnvironmentContext(const EnvironmentContext& other353) { - properties = other353.properties; - __isset = other353.__isset; -} -EnvironmentContext& EnvironmentContext::operator=(const EnvironmentContext& other354) { - properties = other354.properties; - __isset = other354.__isset; - return *this; -} -void EnvironmentContext::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "EnvironmentContext("; - out << "properties=" << to_string(properties); - out << ")"; -} - - -PrimaryKeysRequest::~PrimaryKeysRequest() noexcept { -} - - -void PrimaryKeysRequest::__set_db_name(const std::string& val) { - this->db_name = val; -} - -void PrimaryKeysRequest::__set_tbl_name(const std::string& val) { - this->tbl_name = val; -} -std::ostream& operator<<(std::ostream& out, const PrimaryKeysRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PrimaryKeysRequest::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_db_name = false; - bool isset_tbl_name = 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_STRING) { - xfer += iprot->readString(this->db_name); - isset_db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - isset_tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_db_name) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tbl_name) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t PrimaryKeysRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PrimaryKeysRequest"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PrimaryKeysRequest &a, PrimaryKeysRequest &b) { - using ::std::swap; - swap(a.db_name, b.db_name); - swap(a.tbl_name, b.tbl_name); -} - -PrimaryKeysRequest::PrimaryKeysRequest(const PrimaryKeysRequest& other355) { - db_name = other355.db_name; - tbl_name = other355.tbl_name; -} -PrimaryKeysRequest& PrimaryKeysRequest::operator=(const PrimaryKeysRequest& other356) { - db_name = other356.db_name; - tbl_name = other356.tbl_name; - return *this; -} -void PrimaryKeysRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PrimaryKeysRequest("; - out << "db_name=" << to_string(db_name); - out << ", " << "tbl_name=" << to_string(tbl_name); - out << ")"; -} - - -PrimaryKeysResponse::~PrimaryKeysResponse() noexcept { -} - - -void PrimaryKeysResponse::__set_primaryKeys(const std::vector & val) { - this->primaryKeys = val; -} -std::ostream& operator<<(std::ostream& out, const PrimaryKeysResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PrimaryKeysResponse::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_primaryKeys = 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_LIST) { - { - this->primaryKeys.clear(); - uint32_t _size357; - ::apache::thrift::protocol::TType _etype360; - xfer += iprot->readListBegin(_etype360, _size357); - this->primaryKeys.resize(_size357); - uint32_t _i361; - for (_i361 = 0; _i361 < _size357; ++_i361) - { - xfer += this->primaryKeys[_i361].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_primaryKeys = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_primaryKeys) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t PrimaryKeysResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PrimaryKeysResponse"); - - xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->primaryKeys.size())); - std::vector ::const_iterator _iter362; - for (_iter362 = this->primaryKeys.begin(); _iter362 != this->primaryKeys.end(); ++_iter362) - { - xfer += (*_iter362).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PrimaryKeysResponse &a, PrimaryKeysResponse &b) { - using ::std::swap; - swap(a.primaryKeys, b.primaryKeys); -} - -PrimaryKeysResponse::PrimaryKeysResponse(const PrimaryKeysResponse& other363) { - primaryKeys = other363.primaryKeys; -} -PrimaryKeysResponse& PrimaryKeysResponse::operator=(const PrimaryKeysResponse& other364) { - primaryKeys = other364.primaryKeys; - return *this; -} -void PrimaryKeysResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PrimaryKeysResponse("; - out << "primaryKeys=" << to_string(primaryKeys); - out << ")"; -} - - -ForeignKeysRequest::~ForeignKeysRequest() noexcept { -} - - -void ForeignKeysRequest::__set_parent_db_name(const std::string& val) { - this->parent_db_name = val; -} - -void ForeignKeysRequest::__set_parent_tbl_name(const std::string& val) { - this->parent_tbl_name = val; -} - -void ForeignKeysRequest::__set_foreign_db_name(const std::string& val) { - this->foreign_db_name = val; -} - -void ForeignKeysRequest::__set_foreign_tbl_name(const std::string& val) { - this->foreign_tbl_name = val; -} -std::ostream& operator<<(std::ostream& out, const ForeignKeysRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ForeignKeysRequest::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->parent_db_name); - this->__isset.parent_db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->parent_tbl_name); - this->__isset.parent_tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->foreign_db_name); - this->__isset.foreign_db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->foreign_tbl_name); - this->__isset.foreign_tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ForeignKeysRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ForeignKeysRequest"); - - xfer += oprot->writeFieldBegin("parent_db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->parent_db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("parent_tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->parent_tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("foreign_db_name", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->foreign_db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("foreign_tbl_name", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->foreign_tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ForeignKeysRequest &a, ForeignKeysRequest &b) { - using ::std::swap; - swap(a.parent_db_name, b.parent_db_name); - swap(a.parent_tbl_name, b.parent_tbl_name); - swap(a.foreign_db_name, b.foreign_db_name); - swap(a.foreign_tbl_name, b.foreign_tbl_name); - swap(a.__isset, b.__isset); -} - -ForeignKeysRequest::ForeignKeysRequest(const ForeignKeysRequest& other365) { - parent_db_name = other365.parent_db_name; - parent_tbl_name = other365.parent_tbl_name; - foreign_db_name = other365.foreign_db_name; - foreign_tbl_name = other365.foreign_tbl_name; - __isset = other365.__isset; -} -ForeignKeysRequest& ForeignKeysRequest::operator=(const ForeignKeysRequest& other366) { - parent_db_name = other366.parent_db_name; - parent_tbl_name = other366.parent_tbl_name; - foreign_db_name = other366.foreign_db_name; - foreign_tbl_name = other366.foreign_tbl_name; - __isset = other366.__isset; - return *this; -} -void ForeignKeysRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ForeignKeysRequest("; - out << "parent_db_name=" << to_string(parent_db_name); - out << ", " << "parent_tbl_name=" << to_string(parent_tbl_name); - out << ", " << "foreign_db_name=" << to_string(foreign_db_name); - out << ", " << "foreign_tbl_name=" << to_string(foreign_tbl_name); - out << ")"; -} - - -ForeignKeysResponse::~ForeignKeysResponse() noexcept { -} - - -void ForeignKeysResponse::__set_foreignKeys(const std::vector & val) { - this->foreignKeys = val; -} -std::ostream& operator<<(std::ostream& out, const ForeignKeysResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ForeignKeysResponse::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_foreignKeys = 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_LIST) { - { - this->foreignKeys.clear(); - uint32_t _size367; - ::apache::thrift::protocol::TType _etype370; - xfer += iprot->readListBegin(_etype370, _size367); - this->foreignKeys.resize(_size367); - uint32_t _i371; - for (_i371 = 0; _i371 < _size367; ++_i371) - { - xfer += this->foreignKeys[_i371].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_foreignKeys = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_foreignKeys) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t ForeignKeysResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ForeignKeysResponse"); - - xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->foreignKeys.size())); - std::vector ::const_iterator _iter372; - for (_iter372 = this->foreignKeys.begin(); _iter372 != this->foreignKeys.end(); ++_iter372) - { - xfer += (*_iter372).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ForeignKeysResponse &a, ForeignKeysResponse &b) { - using ::std::swap; - swap(a.foreignKeys, b.foreignKeys); -} - -ForeignKeysResponse::ForeignKeysResponse(const ForeignKeysResponse& other373) { - foreignKeys = other373.foreignKeys; -} -ForeignKeysResponse& ForeignKeysResponse::operator=(const ForeignKeysResponse& other374) { - foreignKeys = other374.foreignKeys; - return *this; -} -void ForeignKeysResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ForeignKeysResponse("; - out << "foreignKeys=" << to_string(foreignKeys); - out << ")"; -} - - -UniqueConstraintsRequest::~UniqueConstraintsRequest() noexcept { -} - - -void UniqueConstraintsRequest::__set_db_name(const std::string& val) { - this->db_name = val; -} - -void UniqueConstraintsRequest::__set_tbl_name(const std::string& val) { - this->tbl_name = val; -} -std::ostream& operator<<(std::ostream& out, const UniqueConstraintsRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t UniqueConstraintsRequest::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_db_name = false; - bool isset_tbl_name = 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_STRING) { - xfer += iprot->readString(this->db_name); - isset_db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - isset_tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_db_name) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tbl_name) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t UniqueConstraintsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("UniqueConstraintsRequest"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(UniqueConstraintsRequest &a, UniqueConstraintsRequest &b) { - using ::std::swap; - swap(a.db_name, b.db_name); - swap(a.tbl_name, b.tbl_name); -} - -UniqueConstraintsRequest::UniqueConstraintsRequest(const UniqueConstraintsRequest& other375) { - db_name = other375.db_name; - tbl_name = other375.tbl_name; -} -UniqueConstraintsRequest& UniqueConstraintsRequest::operator=(const UniqueConstraintsRequest& other376) { - db_name = other376.db_name; - tbl_name = other376.tbl_name; - return *this; -} -void UniqueConstraintsRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "UniqueConstraintsRequest("; - out << "db_name=" << to_string(db_name); - out << ", " << "tbl_name=" << to_string(tbl_name); - out << ")"; -} - - -UniqueConstraintsResponse::~UniqueConstraintsResponse() noexcept { -} - - -void UniqueConstraintsResponse::__set_uniqueConstraints(const std::vector & val) { - this->uniqueConstraints = val; -} -std::ostream& operator<<(std::ostream& out, const UniqueConstraintsResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t UniqueConstraintsResponse::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_uniqueConstraints = 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_LIST) { - { - this->uniqueConstraints.clear(); - uint32_t _size377; - ::apache::thrift::protocol::TType _etype380; - xfer += iprot->readListBegin(_etype380, _size377); - this->uniqueConstraints.resize(_size377); - uint32_t _i381; - for (_i381 = 0; _i381 < _size377; ++_i381) - { - xfer += this->uniqueConstraints[_i381].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_uniqueConstraints = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_uniqueConstraints) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t UniqueConstraintsResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("UniqueConstraintsResponse"); - - xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->uniqueConstraints.size())); - std::vector ::const_iterator _iter382; - for (_iter382 = this->uniqueConstraints.begin(); _iter382 != this->uniqueConstraints.end(); ++_iter382) - { - xfer += (*_iter382).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(UniqueConstraintsResponse &a, UniqueConstraintsResponse &b) { - using ::std::swap; - swap(a.uniqueConstraints, b.uniqueConstraints); -} - -UniqueConstraintsResponse::UniqueConstraintsResponse(const UniqueConstraintsResponse& other383) { - uniqueConstraints = other383.uniqueConstraints; -} -UniqueConstraintsResponse& UniqueConstraintsResponse::operator=(const UniqueConstraintsResponse& other384) { - uniqueConstraints = other384.uniqueConstraints; - return *this; -} -void UniqueConstraintsResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "UniqueConstraintsResponse("; - out << "uniqueConstraints=" << to_string(uniqueConstraints); - out << ")"; -} - - -NotNullConstraintsRequest::~NotNullConstraintsRequest() noexcept { -} - - -void NotNullConstraintsRequest::__set_db_name(const std::string& val) { - this->db_name = val; -} - -void NotNullConstraintsRequest::__set_tbl_name(const std::string& val) { - this->tbl_name = val; -} -std::ostream& operator<<(std::ostream& out, const NotNullConstraintsRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t NotNullConstraintsRequest::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_db_name = false; - bool isset_tbl_name = 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_STRING) { - xfer += iprot->readString(this->db_name); - isset_db_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tbl_name); - isset_tbl_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_db_name) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tbl_name) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t NotNullConstraintsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("NotNullConstraintsRequest"); - - xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->db_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tbl_name", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tbl_name); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(NotNullConstraintsRequest &a, NotNullConstraintsRequest &b) { - using ::std::swap; - swap(a.db_name, b.db_name); - swap(a.tbl_name, b.tbl_name); -} - -NotNullConstraintsRequest::NotNullConstraintsRequest(const NotNullConstraintsRequest& other385) { - db_name = other385.db_name; - tbl_name = other385.tbl_name; -} -NotNullConstraintsRequest& NotNullConstraintsRequest::operator=(const NotNullConstraintsRequest& other386) { - db_name = other386.db_name; - tbl_name = other386.tbl_name; - return *this; -} -void NotNullConstraintsRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "NotNullConstraintsRequest("; - out << "db_name=" << to_string(db_name); - out << ", " << "tbl_name=" << to_string(tbl_name); - out << ")"; -} - - -NotNullConstraintsResponse::~NotNullConstraintsResponse() noexcept { -} - - -void NotNullConstraintsResponse::__set_notNullConstraints(const std::vector & val) { - this->notNullConstraints = val; -} -std::ostream& operator<<(std::ostream& out, const NotNullConstraintsResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t NotNullConstraintsResponse::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_notNullConstraints = 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_LIST) { - { - this->notNullConstraints.clear(); - uint32_t _size387; - ::apache::thrift::protocol::TType _etype390; - xfer += iprot->readListBegin(_etype390, _size387); - this->notNullConstraints.resize(_size387); - uint32_t _i391; - for (_i391 = 0; _i391 < _size387; ++_i391) - { - xfer += this->notNullConstraints[_i391].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_notNullConstraints = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_notNullConstraints) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t NotNullConstraintsResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("NotNullConstraintsResponse"); - - xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->notNullConstraints.size())); - std::vector ::const_iterator _iter392; - for (_iter392 = this->notNullConstraints.begin(); _iter392 != this->notNullConstraints.end(); ++_iter392) - { - xfer += (*_iter392).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(NotNullConstraintsResponse &a, NotNullConstraintsResponse &b) { - using ::std::swap; - swap(a.notNullConstraints, b.notNullConstraints); -} - -NotNullConstraintsResponse::NotNullConstraintsResponse(const NotNullConstraintsResponse& other393) { - notNullConstraints = other393.notNullConstraints; -} -NotNullConstraintsResponse& NotNullConstraintsResponse::operator=(const NotNullConstraintsResponse& other394) { - notNullConstraints = other394.notNullConstraints; - return *this; -} -void NotNullConstraintsResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "NotNullConstraintsResponse("; - out << "notNullConstraints=" << to_string(notNullConstraints); - out << ")"; -} - - -DropConstraintRequest::~DropConstraintRequest() noexcept { -} - - -void DropConstraintRequest::__set_dbname(const std::string& val) { - this->dbname = val; -} - -void DropConstraintRequest::__set_tablename(const std::string& val) { - this->tablename = val; -} - -void DropConstraintRequest::__set_constraintname(const std::string& val) { - this->constraintname = val; -} -std::ostream& operator<<(std::ostream& out, const DropConstraintRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t DropConstraintRequest::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_dbname = false; - bool isset_tablename = false; - bool isset_constraintname = 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_STRING) { - xfer += iprot->readString(this->dbname); - isset_dbname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tablename); - isset_tablename = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->constraintname); - isset_constraintname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_dbname) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tablename) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_constraintname) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t DropConstraintRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("DropConstraintRequest"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbname); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tablename", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tablename); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("constraintname", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->constraintname); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(DropConstraintRequest &a, DropConstraintRequest &b) { - using ::std::swap; - swap(a.dbname, b.dbname); - swap(a.tablename, b.tablename); - swap(a.constraintname, b.constraintname); -} - -DropConstraintRequest::DropConstraintRequest(const DropConstraintRequest& other395) { - dbname = other395.dbname; - tablename = other395.tablename; - constraintname = other395.constraintname; -} -DropConstraintRequest& DropConstraintRequest::operator=(const DropConstraintRequest& other396) { - dbname = other396.dbname; - tablename = other396.tablename; - constraintname = other396.constraintname; - return *this; -} -void DropConstraintRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "DropConstraintRequest("; - out << "dbname=" << to_string(dbname); - out << ", " << "tablename=" << to_string(tablename); - out << ", " << "constraintname=" << to_string(constraintname); - out << ")"; -} - - -AddPrimaryKeyRequest::~AddPrimaryKeyRequest() noexcept { -} - - -void AddPrimaryKeyRequest::__set_primaryKeyCols(const std::vector & val) { - this->primaryKeyCols = val; -} -std::ostream& operator<<(std::ostream& out, const AddPrimaryKeyRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t AddPrimaryKeyRequest::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_primaryKeyCols = 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_LIST) { - { - this->primaryKeyCols.clear(); - uint32_t _size397; - ::apache::thrift::protocol::TType _etype400; - xfer += iprot->readListBegin(_etype400, _size397); - this->primaryKeyCols.resize(_size397); - uint32_t _i401; - for (_i401 = 0; _i401 < _size397; ++_i401) - { - xfer += this->primaryKeyCols[_i401].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_primaryKeyCols = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_primaryKeyCols) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t AddPrimaryKeyRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("AddPrimaryKeyRequest"); - - xfer += oprot->writeFieldBegin("primaryKeyCols", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->primaryKeyCols.size())); - std::vector ::const_iterator _iter402; - for (_iter402 = this->primaryKeyCols.begin(); _iter402 != this->primaryKeyCols.end(); ++_iter402) - { - xfer += (*_iter402).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(AddPrimaryKeyRequest &a, AddPrimaryKeyRequest &b) { - using ::std::swap; - swap(a.primaryKeyCols, b.primaryKeyCols); -} - -AddPrimaryKeyRequest::AddPrimaryKeyRequest(const AddPrimaryKeyRequest& other403) { - primaryKeyCols = other403.primaryKeyCols; -} -AddPrimaryKeyRequest& AddPrimaryKeyRequest::operator=(const AddPrimaryKeyRequest& other404) { - primaryKeyCols = other404.primaryKeyCols; - return *this; -} -void AddPrimaryKeyRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "AddPrimaryKeyRequest("; - out << "primaryKeyCols=" << to_string(primaryKeyCols); - out << ")"; -} - - -AddForeignKeyRequest::~AddForeignKeyRequest() noexcept { -} - - -void AddForeignKeyRequest::__set_foreignKeyCols(const std::vector & val) { - this->foreignKeyCols = val; -} -std::ostream& operator<<(std::ostream& out, const AddForeignKeyRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t AddForeignKeyRequest::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_foreignKeyCols = 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_LIST) { - { - this->foreignKeyCols.clear(); - uint32_t _size405; - ::apache::thrift::protocol::TType _etype408; - xfer += iprot->readListBegin(_etype408, _size405); - this->foreignKeyCols.resize(_size405); - uint32_t _i409; - for (_i409 = 0; _i409 < _size405; ++_i409) - { - xfer += this->foreignKeyCols[_i409].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_foreignKeyCols = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_foreignKeyCols) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t AddForeignKeyRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("AddForeignKeyRequest"); - - xfer += oprot->writeFieldBegin("foreignKeyCols", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->foreignKeyCols.size())); - std::vector ::const_iterator _iter410; - for (_iter410 = this->foreignKeyCols.begin(); _iter410 != this->foreignKeyCols.end(); ++_iter410) - { - xfer += (*_iter410).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(AddForeignKeyRequest &a, AddForeignKeyRequest &b) { - using ::std::swap; - swap(a.foreignKeyCols, b.foreignKeyCols); -} - -AddForeignKeyRequest::AddForeignKeyRequest(const AddForeignKeyRequest& other411) { - foreignKeyCols = other411.foreignKeyCols; -} -AddForeignKeyRequest& AddForeignKeyRequest::operator=(const AddForeignKeyRequest& other412) { - foreignKeyCols = other412.foreignKeyCols; - return *this; -} -void AddForeignKeyRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "AddForeignKeyRequest("; - out << "foreignKeyCols=" << to_string(foreignKeyCols); - out << ")"; -} - - -AddUniqueConstraintRequest::~AddUniqueConstraintRequest() noexcept { -} - - -void AddUniqueConstraintRequest::__set_uniqueConstraintCols(const std::vector & val) { - this->uniqueConstraintCols = val; -} -std::ostream& operator<<(std::ostream& out, const AddUniqueConstraintRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t AddUniqueConstraintRequest::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_uniqueConstraintCols = 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_LIST) { - { - this->uniqueConstraintCols.clear(); - uint32_t _size413; - ::apache::thrift::protocol::TType _etype416; - xfer += iprot->readListBegin(_etype416, _size413); - this->uniqueConstraintCols.resize(_size413); - uint32_t _i417; - for (_i417 = 0; _i417 < _size413; ++_i417) - { - xfer += this->uniqueConstraintCols[_i417].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_uniqueConstraintCols = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_uniqueConstraintCols) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t AddUniqueConstraintRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("AddUniqueConstraintRequest"); - - xfer += oprot->writeFieldBegin("uniqueConstraintCols", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->uniqueConstraintCols.size())); - std::vector ::const_iterator _iter418; - for (_iter418 = this->uniqueConstraintCols.begin(); _iter418 != this->uniqueConstraintCols.end(); ++_iter418) - { - xfer += (*_iter418).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(AddUniqueConstraintRequest &a, AddUniqueConstraintRequest &b) { - using ::std::swap; - swap(a.uniqueConstraintCols, b.uniqueConstraintCols); -} - -AddUniqueConstraintRequest::AddUniqueConstraintRequest(const AddUniqueConstraintRequest& other419) { - uniqueConstraintCols = other419.uniqueConstraintCols; -} -AddUniqueConstraintRequest& AddUniqueConstraintRequest::operator=(const AddUniqueConstraintRequest& other420) { - uniqueConstraintCols = other420.uniqueConstraintCols; - return *this; -} -void AddUniqueConstraintRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "AddUniqueConstraintRequest("; - out << "uniqueConstraintCols=" << to_string(uniqueConstraintCols); - out << ")"; -} - - -AddNotNullConstraintRequest::~AddNotNullConstraintRequest() noexcept { -} - - -void AddNotNullConstraintRequest::__set_notNullConstraintCols(const std::vector & val) { - this->notNullConstraintCols = val; -} -std::ostream& operator<<(std::ostream& out, const AddNotNullConstraintRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t AddNotNullConstraintRequest::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_notNullConstraintCols = 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_LIST) { - { - this->notNullConstraintCols.clear(); - uint32_t _size421; - ::apache::thrift::protocol::TType _etype424; - xfer += iprot->readListBegin(_etype424, _size421); - this->notNullConstraintCols.resize(_size421); - uint32_t _i425; - for (_i425 = 0; _i425 < _size421; ++_i425) - { - xfer += this->notNullConstraintCols[_i425].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_notNullConstraintCols = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_notNullConstraintCols) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t AddNotNullConstraintRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("AddNotNullConstraintRequest"); - - xfer += oprot->writeFieldBegin("notNullConstraintCols", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->notNullConstraintCols.size())); - std::vector ::const_iterator _iter426; - for (_iter426 = this->notNullConstraintCols.begin(); _iter426 != this->notNullConstraintCols.end(); ++_iter426) - { - xfer += (*_iter426).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(AddNotNullConstraintRequest &a, AddNotNullConstraintRequest &b) { - using ::std::swap; - swap(a.notNullConstraintCols, b.notNullConstraintCols); -} - -AddNotNullConstraintRequest::AddNotNullConstraintRequest(const AddNotNullConstraintRequest& other427) { - notNullConstraintCols = other427.notNullConstraintCols; -} -AddNotNullConstraintRequest& AddNotNullConstraintRequest::operator=(const AddNotNullConstraintRequest& other428) { - notNullConstraintCols = other428.notNullConstraintCols; - return *this; -} -void AddNotNullConstraintRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "AddNotNullConstraintRequest("; - out << "notNullConstraintCols=" << to_string(notNullConstraintCols); - out << ")"; -} - - -PartitionsByExprResult::~PartitionsByExprResult() noexcept { -} - - -void PartitionsByExprResult::__set_partitions(const std::vector & val) { - this->partitions = val; -} - -void PartitionsByExprResult::__set_hasUnknownPartitions(const bool val) { - this->hasUnknownPartitions = val; -} -std::ostream& operator<<(std::ostream& out, const PartitionsByExprResult& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PartitionsByExprResult::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_partitions = false; - bool isset_hasUnknownPartitions = 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_LIST) { - { - this->partitions.clear(); - uint32_t _size429; - ::apache::thrift::protocol::TType _etype432; - xfer += iprot->readListBegin(_etype432, _size429); - this->partitions.resize(_size429); - uint32_t _i433; - for (_i433 = 0; _i433 < _size429; ++_i433) - { - xfer += this->partitions[_i433].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_partitions = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->hasUnknownPartitions); - isset_hasUnknownPartitions = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_partitions) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_hasUnknownPartitions) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t PartitionsByExprResult::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PartitionsByExprResult"); - - xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitions.size())); - std::vector ::const_iterator _iter434; - for (_iter434 = this->partitions.begin(); _iter434 != this->partitions.end(); ++_iter434) - { - xfer += (*_iter434).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("hasUnknownPartitions", ::apache::thrift::protocol::T_BOOL, 2); - xfer += oprot->writeBool(this->hasUnknownPartitions); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PartitionsByExprResult &a, PartitionsByExprResult &b) { - using ::std::swap; - swap(a.partitions, b.partitions); - swap(a.hasUnknownPartitions, b.hasUnknownPartitions); -} - -PartitionsByExprResult::PartitionsByExprResult(const PartitionsByExprResult& other435) { - partitions = other435.partitions; - hasUnknownPartitions = other435.hasUnknownPartitions; -} -PartitionsByExprResult& PartitionsByExprResult::operator=(const PartitionsByExprResult& other436) { - partitions = other436.partitions; - hasUnknownPartitions = other436.hasUnknownPartitions; - return *this; -} -void PartitionsByExprResult::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PartitionsByExprResult("; - out << "partitions=" << to_string(partitions); - out << ", " << "hasUnknownPartitions=" << to_string(hasUnknownPartitions); - out << ")"; -} - - -PartitionsByExprRequest::~PartitionsByExprRequest() noexcept { -} - - -void PartitionsByExprRequest::__set_dbName(const std::string& val) { - this->dbName = val; -} - -void PartitionsByExprRequest::__set_tblName(const std::string& val) { - this->tblName = val; -} - -void PartitionsByExprRequest::__set_expr(const std::string& val) { - this->expr = val; -} - -void PartitionsByExprRequest::__set_defaultPartitionName(const std::string& val) { - this->defaultPartitionName = val; -__isset.defaultPartitionName = true; -} - -void PartitionsByExprRequest::__set_maxParts(const int16_t val) { - this->maxParts = val; -__isset.maxParts = true; -} -std::ostream& operator<<(std::ostream& out, const PartitionsByExprRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PartitionsByExprRequest::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_dbName = false; - bool isset_tblName = false; - bool isset_expr = 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_STRING) { - xfer += iprot->readString(this->dbName); - isset_dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tblName); - isset_tblName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->expr); - isset_expr = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->defaultPartitionName); - this->__isset.defaultPartitionName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_I16) { - xfer += iprot->readI16(this->maxParts); - this->__isset.maxParts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_dbName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tblName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_expr) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t PartitionsByExprRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PartitionsByExprRequest"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tblName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("expr", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeBinary(this->expr); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.defaultPartitionName) { - xfer += oprot->writeFieldBegin("defaultPartitionName", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->defaultPartitionName); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.maxParts) { - xfer += oprot->writeFieldBegin("maxParts", ::apache::thrift::protocol::T_I16, 5); - xfer += oprot->writeI16(this->maxParts); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PartitionsByExprRequest &a, PartitionsByExprRequest &b) { - using ::std::swap; - swap(a.dbName, b.dbName); - swap(a.tblName, b.tblName); - swap(a.expr, b.expr); - swap(a.defaultPartitionName, b.defaultPartitionName); - swap(a.maxParts, b.maxParts); - swap(a.__isset, b.__isset); -} - -PartitionsByExprRequest::PartitionsByExprRequest(const PartitionsByExprRequest& other437) { - dbName = other437.dbName; - tblName = other437.tblName; - expr = other437.expr; - defaultPartitionName = other437.defaultPartitionName; - maxParts = other437.maxParts; - __isset = other437.__isset; -} -PartitionsByExprRequest& PartitionsByExprRequest::operator=(const PartitionsByExprRequest& other438) { - dbName = other438.dbName; - tblName = other438.tblName; - expr = other438.expr; - defaultPartitionName = other438.defaultPartitionName; - maxParts = other438.maxParts; - __isset = other438.__isset; - return *this; -} -void PartitionsByExprRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PartitionsByExprRequest("; - out << "dbName=" << to_string(dbName); - out << ", " << "tblName=" << to_string(tblName); - out << ", " << "expr=" << to_string(expr); - out << ", " << "defaultPartitionName="; (__isset.defaultPartitionName ? (out << to_string(defaultPartitionName)) : (out << "")); - out << ", " << "maxParts="; (__isset.maxParts ? (out << to_string(maxParts)) : (out << "")); - out << ")"; -} - - -TableStatsResult::~TableStatsResult() noexcept { -} - - -void TableStatsResult::__set_tableStats(const std::vector & val) { - this->tableStats = val; -} -std::ostream& operator<<(std::ostream& out, const TableStatsResult& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t TableStatsResult::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_tableStats = 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_LIST) { - { - this->tableStats.clear(); - uint32_t _size439; - ::apache::thrift::protocol::TType _etype442; - xfer += iprot->readListBegin(_etype442, _size439); - this->tableStats.resize(_size439); - uint32_t _i443; - for (_i443 = 0; _i443 < _size439; ++_i443) - { - xfer += this->tableStats[_i443].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_tableStats = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_tableStats) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t TableStatsResult::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("TableStatsResult"); - - xfer += oprot->writeFieldBegin("tableStats", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->tableStats.size())); - std::vector ::const_iterator _iter444; - for (_iter444 = this->tableStats.begin(); _iter444 != this->tableStats.end(); ++_iter444) - { - xfer += (*_iter444).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(TableStatsResult &a, TableStatsResult &b) { - using ::std::swap; - swap(a.tableStats, b.tableStats); -} - -TableStatsResult::TableStatsResult(const TableStatsResult& other445) { - tableStats = other445.tableStats; -} -TableStatsResult& TableStatsResult::operator=(const TableStatsResult& other446) { - tableStats = other446.tableStats; - return *this; -} -void TableStatsResult::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "TableStatsResult("; - out << "tableStats=" << to_string(tableStats); - out << ")"; -} - - -PartitionsStatsResult::~PartitionsStatsResult() noexcept { -} - - -void PartitionsStatsResult::__set_partStats(const std::map > & val) { - this->partStats = val; -} -std::ostream& operator<<(std::ostream& out, const PartitionsStatsResult& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PartitionsStatsResult::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_partStats = 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_MAP) { - { - this->partStats.clear(); - uint32_t _size447; - ::apache::thrift::protocol::TType _ktype448; - ::apache::thrift::protocol::TType _vtype449; - xfer += iprot->readMapBegin(_ktype448, _vtype449, _size447); - uint32_t _i451; - for (_i451 = 0; _i451 < _size447; ++_i451) - { - std::string _key452; - xfer += iprot->readString(_key452); - std::vector & _val453 = this->partStats[_key452]; - { - _val453.clear(); - uint32_t _size454; - ::apache::thrift::protocol::TType _etype457; - xfer += iprot->readListBegin(_etype457, _size454); - _val453.resize(_size454); - uint32_t _i458; - for (_i458 = 0; _i458 < _size454; ++_i458) - { - xfer += _val453[_i458].read(iprot); - } - xfer += iprot->readListEnd(); - } - } - xfer += iprot->readMapEnd(); - } - isset_partStats = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_partStats) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t PartitionsStatsResult::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PartitionsStatsResult"); - - xfer += oprot->writeFieldBegin("partStats", ::apache::thrift::protocol::T_MAP, 1); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast(this->partStats.size())); - std::map > ::const_iterator _iter459; - for (_iter459 = this->partStats.begin(); _iter459 != this->partStats.end(); ++_iter459) - { - xfer += oprot->writeString(_iter459->first); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(_iter459->second.size())); - std::vector ::const_iterator _iter460; - for (_iter460 = _iter459->second.begin(); _iter460 != _iter459->second.end(); ++_iter460) - { - xfer += (*_iter460).write(oprot); - } - xfer += oprot->writeListEnd(); - } - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PartitionsStatsResult &a, PartitionsStatsResult &b) { - using ::std::swap; - swap(a.partStats, b.partStats); -} - -PartitionsStatsResult::PartitionsStatsResult(const PartitionsStatsResult& other461) { - partStats = other461.partStats; -} -PartitionsStatsResult& PartitionsStatsResult::operator=(const PartitionsStatsResult& other462) { - partStats = other462.partStats; - return *this; -} -void PartitionsStatsResult::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PartitionsStatsResult("; - out << "partStats=" << to_string(partStats); - out << ")"; -} - - -TableStatsRequest::~TableStatsRequest() noexcept { -} - - -void TableStatsRequest::__set_dbName(const std::string& val) { - this->dbName = val; -} - -void TableStatsRequest::__set_tblName(const std::string& val) { - this->tblName = val; -} - -void TableStatsRequest::__set_colNames(const std::vector & val) { - this->colNames = val; -} -std::ostream& operator<<(std::ostream& out, const TableStatsRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t TableStatsRequest::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_dbName = false; - bool isset_tblName = false; - bool isset_colNames = 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_STRING) { - xfer += iprot->readString(this->dbName); - isset_dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tblName); - isset_tblName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->colNames.clear(); - uint32_t _size463; - ::apache::thrift::protocol::TType _etype466; - xfer += iprot->readListBegin(_etype466, _size463); - this->colNames.resize(_size463); - uint32_t _i467; - for (_i467 = 0; _i467 < _size463; ++_i467) - { - xfer += iprot->readString(this->colNames[_i467]); - } - xfer += iprot->readListEnd(); - } - isset_colNames = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_dbName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tblName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_colNames) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t TableStatsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("TableStatsRequest"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tblName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("colNames", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->colNames.size())); - std::vector ::const_iterator _iter468; - for (_iter468 = this->colNames.begin(); _iter468 != this->colNames.end(); ++_iter468) - { - xfer += oprot->writeString((*_iter468)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(TableStatsRequest &a, TableStatsRequest &b) { - using ::std::swap; - swap(a.dbName, b.dbName); - swap(a.tblName, b.tblName); - swap(a.colNames, b.colNames); -} - -TableStatsRequest::TableStatsRequest(const TableStatsRequest& other469) { - dbName = other469.dbName; - tblName = other469.tblName; - colNames = other469.colNames; -} -TableStatsRequest& TableStatsRequest::operator=(const TableStatsRequest& other470) { - dbName = other470.dbName; - tblName = other470.tblName; - colNames = other470.colNames; - return *this; -} -void TableStatsRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "TableStatsRequest("; - out << "dbName=" << to_string(dbName); - out << ", " << "tblName=" << to_string(tblName); - out << ", " << "colNames=" << to_string(colNames); - out << ")"; -} - - -PartitionsStatsRequest::~PartitionsStatsRequest() noexcept { -} - - -void PartitionsStatsRequest::__set_dbName(const std::string& val) { - this->dbName = val; -} - -void PartitionsStatsRequest::__set_tblName(const std::string& val) { - this->tblName = val; -} - -void PartitionsStatsRequest::__set_colNames(const std::vector & val) { - this->colNames = val; -} - -void PartitionsStatsRequest::__set_partNames(const std::vector & val) { - this->partNames = val; -} -std::ostream& operator<<(std::ostream& out, const PartitionsStatsRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PartitionsStatsRequest::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_dbName = false; - bool isset_tblName = false; - bool isset_colNames = false; - bool isset_partNames = 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_STRING) { - xfer += iprot->readString(this->dbName); - isset_dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tblName); - isset_tblName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->colNames.clear(); - uint32_t _size471; - ::apache::thrift::protocol::TType _etype474; - xfer += iprot->readListBegin(_etype474, _size471); - this->colNames.resize(_size471); - uint32_t _i475; - for (_i475 = 0; _i475 < _size471; ++_i475) - { - xfer += iprot->readString(this->colNames[_i475]); - } - xfer += iprot->readListEnd(); - } - isset_colNames = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->partNames.clear(); - uint32_t _size476; - ::apache::thrift::protocol::TType _etype479; - xfer += iprot->readListBegin(_etype479, _size476); - this->partNames.resize(_size476); - uint32_t _i480; - for (_i480 = 0; _i480 < _size476; ++_i480) - { - xfer += iprot->readString(this->partNames[_i480]); - } - xfer += iprot->readListEnd(); - } - isset_partNames = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_dbName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tblName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_colNames) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_partNames) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t PartitionsStatsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PartitionsStatsRequest"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tblName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("colNames", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->colNames.size())); - std::vector ::const_iterator _iter481; - for (_iter481 = this->colNames.begin(); _iter481 != this->colNames.end(); ++_iter481) - { - xfer += oprot->writeString((*_iter481)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 4); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partNames.size())); - std::vector ::const_iterator _iter482; - for (_iter482 = this->partNames.begin(); _iter482 != this->partNames.end(); ++_iter482) - { - xfer += oprot->writeString((*_iter482)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PartitionsStatsRequest &a, PartitionsStatsRequest &b) { - using ::std::swap; - swap(a.dbName, b.dbName); - swap(a.tblName, b.tblName); - swap(a.colNames, b.colNames); - swap(a.partNames, b.partNames); -} - -PartitionsStatsRequest::PartitionsStatsRequest(const PartitionsStatsRequest& other483) { - dbName = other483.dbName; - tblName = other483.tblName; - colNames = other483.colNames; - partNames = other483.partNames; -} -PartitionsStatsRequest& PartitionsStatsRequest::operator=(const PartitionsStatsRequest& other484) { - dbName = other484.dbName; - tblName = other484.tblName; - colNames = other484.colNames; - partNames = other484.partNames; - return *this; -} -void PartitionsStatsRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PartitionsStatsRequest("; - out << "dbName=" << to_string(dbName); - out << ", " << "tblName=" << to_string(tblName); - out << ", " << "colNames=" << to_string(colNames); - out << ", " << "partNames=" << to_string(partNames); - out << ")"; -} - - -AddPartitionsResult::~AddPartitionsResult() noexcept { -} - - -void AddPartitionsResult::__set_partitions(const std::vector & val) { - this->partitions = val; -__isset.partitions = true; -} -std::ostream& operator<<(std::ostream& out, const AddPartitionsResult& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t AddPartitionsResult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->partitions.clear(); - uint32_t _size485; - ::apache::thrift::protocol::TType _etype488; - xfer += iprot->readListBegin(_etype488, _size485); - this->partitions.resize(_size485); - uint32_t _i489; - for (_i489 = 0; _i489 < _size485; ++_i489) - { - xfer += this->partitions[_i489].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.partitions = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t AddPartitionsResult::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("AddPartitionsResult"); - - if (this->__isset.partitions) { - xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitions.size())); - std::vector ::const_iterator _iter490; - for (_iter490 = this->partitions.begin(); _iter490 != this->partitions.end(); ++_iter490) - { - xfer += (*_iter490).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(AddPartitionsResult &a, AddPartitionsResult &b) { - using ::std::swap; - swap(a.partitions, b.partitions); - swap(a.__isset, b.__isset); -} - -AddPartitionsResult::AddPartitionsResult(const AddPartitionsResult& other491) { - partitions = other491.partitions; - __isset = other491.__isset; -} -AddPartitionsResult& AddPartitionsResult::operator=(const AddPartitionsResult& other492) { - partitions = other492.partitions; - __isset = other492.__isset; - return *this; -} -void AddPartitionsResult::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "AddPartitionsResult("; - out << "partitions="; (__isset.partitions ? (out << to_string(partitions)) : (out << "")); - out << ")"; -} - - -AddPartitionsRequest::~AddPartitionsRequest() noexcept { -} - - -void AddPartitionsRequest::__set_dbName(const std::string& val) { - this->dbName = val; -} - -void AddPartitionsRequest::__set_tblName(const std::string& val) { - this->tblName = val; -} - -void AddPartitionsRequest::__set_parts(const std::vector & val) { - this->parts = val; -} - -void AddPartitionsRequest::__set_ifNotExists(const bool val) { - this->ifNotExists = val; -} - -void AddPartitionsRequest::__set_needResult(const bool val) { - this->needResult = val; -__isset.needResult = true; -} -std::ostream& operator<<(std::ostream& out, const AddPartitionsRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t AddPartitionsRequest::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_dbName = false; - bool isset_tblName = false; - bool isset_parts = false; - bool isset_ifNotExists = 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_STRING) { - xfer += iprot->readString(this->dbName); - isset_dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tblName); - isset_tblName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->parts.clear(); - uint32_t _size493; - ::apache::thrift::protocol::TType _etype496; - xfer += iprot->readListBegin(_etype496, _size493); - this->parts.resize(_size493); - uint32_t _i497; - for (_i497 = 0; _i497 < _size493; ++_i497) - { - xfer += this->parts[_i497].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_parts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->ifNotExists); - isset_ifNotExists = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->needResult); - this->__isset.needResult = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_dbName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tblName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_parts) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_ifNotExists) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t AddPartitionsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("AddPartitionsRequest"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tblName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("parts", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->parts.size())); - std::vector ::const_iterator _iter498; - for (_iter498 = this->parts.begin(); _iter498 != this->parts.end(); ++_iter498) - { - xfer += (*_iter498).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("ifNotExists", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool(this->ifNotExists); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.needResult) { - xfer += oprot->writeFieldBegin("needResult", ::apache::thrift::protocol::T_BOOL, 5); - xfer += oprot->writeBool(this->needResult); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(AddPartitionsRequest &a, AddPartitionsRequest &b) { - using ::std::swap; - swap(a.dbName, b.dbName); - swap(a.tblName, b.tblName); - swap(a.parts, b.parts); - swap(a.ifNotExists, b.ifNotExists); - swap(a.needResult, b.needResult); - swap(a.__isset, b.__isset); -} - -AddPartitionsRequest::AddPartitionsRequest(const AddPartitionsRequest& other499) { - dbName = other499.dbName; - tblName = other499.tblName; - parts = other499.parts; - ifNotExists = other499.ifNotExists; - needResult = other499.needResult; - __isset = other499.__isset; -} -AddPartitionsRequest& AddPartitionsRequest::operator=(const AddPartitionsRequest& other500) { - dbName = other500.dbName; - tblName = other500.tblName; - parts = other500.parts; - ifNotExists = other500.ifNotExists; - needResult = other500.needResult; - __isset = other500.__isset; - return *this; -} -void AddPartitionsRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "AddPartitionsRequest("; - out << "dbName=" << to_string(dbName); - out << ", " << "tblName=" << to_string(tblName); - out << ", " << "parts=" << to_string(parts); - out << ", " << "ifNotExists=" << to_string(ifNotExists); - out << ", " << "needResult="; (__isset.needResult ? (out << to_string(needResult)) : (out << "")); - out << ")"; -} - - -DropPartitionsResult::~DropPartitionsResult() noexcept { -} - - -void DropPartitionsResult::__set_partitions(const std::vector & val) { - this->partitions = val; -__isset.partitions = true; -} -std::ostream& operator<<(std::ostream& out, const DropPartitionsResult& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t DropPartitionsResult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->partitions.clear(); - uint32_t _size501; - ::apache::thrift::protocol::TType _etype504; - xfer += iprot->readListBegin(_etype504, _size501); - this->partitions.resize(_size501); - uint32_t _i505; - for (_i505 = 0; _i505 < _size501; ++_i505) - { - xfer += this->partitions[_i505].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.partitions = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t DropPartitionsResult::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("DropPartitionsResult"); - - if (this->__isset.partitions) { - xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitions.size())); - std::vector ::const_iterator _iter506; - for (_iter506 = this->partitions.begin(); _iter506 != this->partitions.end(); ++_iter506) - { - xfer += (*_iter506).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(DropPartitionsResult &a, DropPartitionsResult &b) { - using ::std::swap; - swap(a.partitions, b.partitions); - swap(a.__isset, b.__isset); -} - -DropPartitionsResult::DropPartitionsResult(const DropPartitionsResult& other507) { - partitions = other507.partitions; - __isset = other507.__isset; -} -DropPartitionsResult& DropPartitionsResult::operator=(const DropPartitionsResult& other508) { - partitions = other508.partitions; - __isset = other508.__isset; - return *this; -} -void DropPartitionsResult::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "DropPartitionsResult("; - out << "partitions="; (__isset.partitions ? (out << to_string(partitions)) : (out << "")); - out << ")"; -} - - -DropPartitionsExpr::~DropPartitionsExpr() noexcept { -} - - -void DropPartitionsExpr::__set_expr(const std::string& val) { - this->expr = val; -} - -void DropPartitionsExpr::__set_partArchiveLevel(const int32_t val) { - this->partArchiveLevel = val; -__isset.partArchiveLevel = true; -} -std::ostream& operator<<(std::ostream& out, const DropPartitionsExpr& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t DropPartitionsExpr::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_expr = 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_STRING) { - xfer += iprot->readBinary(this->expr); - isset_expr = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->partArchiveLevel); - this->__isset.partArchiveLevel = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_expr) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t DropPartitionsExpr::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("DropPartitionsExpr"); - - xfer += oprot->writeFieldBegin("expr", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeBinary(this->expr); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.partArchiveLevel) { - xfer += oprot->writeFieldBegin("partArchiveLevel", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32(this->partArchiveLevel); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(DropPartitionsExpr &a, DropPartitionsExpr &b) { - using ::std::swap; - swap(a.expr, b.expr); - swap(a.partArchiveLevel, b.partArchiveLevel); - swap(a.__isset, b.__isset); -} - -DropPartitionsExpr::DropPartitionsExpr(const DropPartitionsExpr& other509) { - expr = other509.expr; - partArchiveLevel = other509.partArchiveLevel; - __isset = other509.__isset; -} -DropPartitionsExpr& DropPartitionsExpr::operator=(const DropPartitionsExpr& other510) { - expr = other510.expr; - partArchiveLevel = other510.partArchiveLevel; - __isset = other510.__isset; - return *this; -} -void DropPartitionsExpr::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "DropPartitionsExpr("; - out << "expr=" << to_string(expr); - out << ", " << "partArchiveLevel="; (__isset.partArchiveLevel ? (out << to_string(partArchiveLevel)) : (out << "")); - out << ")"; -} - - -RequestPartsSpec::~RequestPartsSpec() noexcept { -} - - -void RequestPartsSpec::__set_names(const std::vector & val) { - this->names = val; -__isset.names = true; -} - -void RequestPartsSpec::__set_exprs(const std::vector & val) { - this->exprs = val; -__isset.exprs = true; -} -std::ostream& operator<<(std::ostream& out, const RequestPartsSpec& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t RequestPartsSpec::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->names.clear(); - uint32_t _size511; - ::apache::thrift::protocol::TType _etype514; - xfer += iprot->readListBegin(_etype514, _size511); - this->names.resize(_size511); - uint32_t _i515; - for (_i515 = 0; _i515 < _size511; ++_i515) - { - xfer += iprot->readString(this->names[_i515]); - } - xfer += iprot->readListEnd(); - } - this->__isset.names = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->exprs.clear(); - uint32_t _size516; - ::apache::thrift::protocol::TType _etype519; - xfer += iprot->readListBegin(_etype519, _size516); - this->exprs.resize(_size516); - uint32_t _i520; - for (_i520 = 0; _i520 < _size516; ++_i520) - { - xfer += this->exprs[_i520].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.exprs = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t RequestPartsSpec::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("RequestPartsSpec"); - - if (this->__isset.names) { - xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->names.size())); - std::vector ::const_iterator _iter521; - for (_iter521 = this->names.begin(); _iter521 != this->names.end(); ++_iter521) - { - xfer += oprot->writeString((*_iter521)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.exprs) { - xfer += oprot->writeFieldBegin("exprs", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->exprs.size())); - std::vector ::const_iterator _iter522; - for (_iter522 = this->exprs.begin(); _iter522 != this->exprs.end(); ++_iter522) - { - xfer += (*_iter522).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(RequestPartsSpec &a, RequestPartsSpec &b) { - using ::std::swap; - swap(a.names, b.names); - swap(a.exprs, b.exprs); - swap(a.__isset, b.__isset); -} - -RequestPartsSpec::RequestPartsSpec(const RequestPartsSpec& other523) { - names = other523.names; - exprs = other523.exprs; - __isset = other523.__isset; -} -RequestPartsSpec& RequestPartsSpec::operator=(const RequestPartsSpec& other524) { - names = other524.names; - exprs = other524.exprs; - __isset = other524.__isset; - return *this; -} -void RequestPartsSpec::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "RequestPartsSpec("; - out << "names="; (__isset.names ? (out << to_string(names)) : (out << "")); - out << ", " << "exprs="; (__isset.exprs ? (out << to_string(exprs)) : (out << "")); - out << ")"; -} - - -DropPartitionsRequest::~DropPartitionsRequest() noexcept { -} - - -void DropPartitionsRequest::__set_dbName(const std::string& val) { - this->dbName = val; -} - -void DropPartitionsRequest::__set_tblName(const std::string& val) { - this->tblName = val; -} - -void DropPartitionsRequest::__set_parts(const RequestPartsSpec& val) { - this->parts = val; -} - -void DropPartitionsRequest::__set_deleteData(const bool val) { - this->deleteData = val; -__isset.deleteData = true; -} - -void DropPartitionsRequest::__set_ifExists(const bool val) { - this->ifExists = val; -__isset.ifExists = true; -} - -void DropPartitionsRequest::__set_ignoreProtection(const bool val) { - this->ignoreProtection = val; -__isset.ignoreProtection = true; -} - -void DropPartitionsRequest::__set_environmentContext(const EnvironmentContext& val) { - this->environmentContext = val; -__isset.environmentContext = true; -} - -void DropPartitionsRequest::__set_needResult(const bool val) { - this->needResult = val; -__isset.needResult = true; -} -std::ostream& operator<<(std::ostream& out, const DropPartitionsRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t DropPartitionsRequest::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_dbName = false; - bool isset_tblName = false; - bool isset_parts = 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_STRING) { - xfer += iprot->readString(this->dbName); - isset_dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tblName); - isset_tblName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->parts.read(iprot); - isset_parts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->deleteData); - this->__isset.deleteData = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->ifExists); - this->__isset.ifExists = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->ignoreProtection); - this->__isset.ignoreProtection = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->environmentContext.read(iprot); - this->__isset.environmentContext = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->needResult); - this->__isset.needResult = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_dbName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tblName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_parts) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t DropPartitionsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("DropPartitionsRequest"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tblName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("parts", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->parts.write(oprot); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.deleteData) { - xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool(this->deleteData); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.ifExists) { - xfer += oprot->writeFieldBegin("ifExists", ::apache::thrift::protocol::T_BOOL, 5); - xfer += oprot->writeBool(this->ifExists); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.ignoreProtection) { - xfer += oprot->writeFieldBegin("ignoreProtection", ::apache::thrift::protocol::T_BOOL, 6); - xfer += oprot->writeBool(this->ignoreProtection); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.environmentContext) { - xfer += oprot->writeFieldBegin("environmentContext", ::apache::thrift::protocol::T_STRUCT, 7); - xfer += this->environmentContext.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.needResult) { - xfer += oprot->writeFieldBegin("needResult", ::apache::thrift::protocol::T_BOOL, 8); - xfer += oprot->writeBool(this->needResult); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(DropPartitionsRequest &a, DropPartitionsRequest &b) { - using ::std::swap; - swap(a.dbName, b.dbName); - swap(a.tblName, b.tblName); - swap(a.parts, b.parts); - swap(a.deleteData, b.deleteData); - swap(a.ifExists, b.ifExists); - swap(a.ignoreProtection, b.ignoreProtection); - swap(a.environmentContext, b.environmentContext); - swap(a.needResult, b.needResult); - swap(a.__isset, b.__isset); -} - -DropPartitionsRequest::DropPartitionsRequest(const DropPartitionsRequest& other525) { - dbName = other525.dbName; - tblName = other525.tblName; - parts = other525.parts; - deleteData = other525.deleteData; - ifExists = other525.ifExists; - ignoreProtection = other525.ignoreProtection; - environmentContext = other525.environmentContext; - needResult = other525.needResult; - __isset = other525.__isset; -} -DropPartitionsRequest& DropPartitionsRequest::operator=(const DropPartitionsRequest& other526) { - dbName = other526.dbName; - tblName = other526.tblName; - parts = other526.parts; - deleteData = other526.deleteData; - ifExists = other526.ifExists; - ignoreProtection = other526.ignoreProtection; - environmentContext = other526.environmentContext; - needResult = other526.needResult; - __isset = other526.__isset; - return *this; -} -void DropPartitionsRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "DropPartitionsRequest("; - out << "dbName=" << to_string(dbName); - out << ", " << "tblName=" << to_string(tblName); - out << ", " << "parts=" << to_string(parts); - out << ", " << "deleteData="; (__isset.deleteData ? (out << to_string(deleteData)) : (out << "")); - out << ", " << "ifExists="; (__isset.ifExists ? (out << to_string(ifExists)) : (out << "")); - out << ", " << "ignoreProtection="; (__isset.ignoreProtection ? (out << to_string(ignoreProtection)) : (out << "")); - out << ", " << "environmentContext="; (__isset.environmentContext ? (out << to_string(environmentContext)) : (out << "")); - out << ", " << "needResult="; (__isset.needResult ? (out << to_string(needResult)) : (out << "")); - out << ")"; -} - - -PartitionValuesRequest::~PartitionValuesRequest() noexcept { -} - - -void PartitionValuesRequest::__set_dbName(const std::string& val) { - this->dbName = val; -} - -void PartitionValuesRequest::__set_tblName(const std::string& val) { - this->tblName = val; -} - -void PartitionValuesRequest::__set_partitionKeys(const std::vector & val) { - this->partitionKeys = val; -} - -void PartitionValuesRequest::__set_applyDistinct(const bool val) { - this->applyDistinct = val; -__isset.applyDistinct = true; -} - -void PartitionValuesRequest::__set_filter(const std::string& val) { - this->filter = val; -__isset.filter = true; -} - -void PartitionValuesRequest::__set_partitionOrder(const std::vector & val) { - this->partitionOrder = val; -__isset.partitionOrder = true; -} - -void PartitionValuesRequest::__set_ascending(const bool val) { - this->ascending = val; -__isset.ascending = true; -} - -void PartitionValuesRequest::__set_maxParts(const int64_t val) { - this->maxParts = val; -__isset.maxParts = true; -} -std::ostream& operator<<(std::ostream& out, const PartitionValuesRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PartitionValuesRequest::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_dbName = false; - bool isset_tblName = false; - bool isset_partitionKeys = 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_STRING) { - xfer += iprot->readString(this->dbName); - isset_dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tblName); - isset_tblName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->partitionKeys.clear(); - uint32_t _size527; - ::apache::thrift::protocol::TType _etype530; - xfer += iprot->readListBegin(_etype530, _size527); - this->partitionKeys.resize(_size527); - uint32_t _i531; - for (_i531 = 0; _i531 < _size527; ++_i531) - { - xfer += this->partitionKeys[_i531].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_partitionKeys = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->applyDistinct); - this->__isset.applyDistinct = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->filter); - this->__isset.filter = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->partitionOrder.clear(); - uint32_t _size532; - ::apache::thrift::protocol::TType _etype535; - xfer += iprot->readListBegin(_etype535, _size532); - this->partitionOrder.resize(_size532); - uint32_t _i536; - for (_i536 = 0; _i536 < _size532; ++_i536) - { - xfer += this->partitionOrder[_i536].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.partitionOrder = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->ascending); - this->__isset.ascending = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->maxParts); - this->__isset.maxParts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_dbName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tblName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_partitionKeys) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t PartitionValuesRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PartitionValuesRequest"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tblName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("partitionKeys", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitionKeys.size())); - std::vector ::const_iterator _iter537; - for (_iter537 = this->partitionKeys.begin(); _iter537 != this->partitionKeys.end(); ++_iter537) - { - xfer += (*_iter537).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - if (this->__isset.applyDistinct) { - xfer += oprot->writeFieldBegin("applyDistinct", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool(this->applyDistinct); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.filter) { - xfer += oprot->writeFieldBegin("filter", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString(this->filter); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.partitionOrder) { - xfer += oprot->writeFieldBegin("partitionOrder", ::apache::thrift::protocol::T_LIST, 6); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitionOrder.size())); - std::vector ::const_iterator _iter538; - for (_iter538 = this->partitionOrder.begin(); _iter538 != this->partitionOrder.end(); ++_iter538) - { - xfer += (*_iter538).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.ascending) { - xfer += oprot->writeFieldBegin("ascending", ::apache::thrift::protocol::T_BOOL, 7); - xfer += oprot->writeBool(this->ascending); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.maxParts) { - xfer += oprot->writeFieldBegin("maxParts", ::apache::thrift::protocol::T_I64, 8); - xfer += oprot->writeI64(this->maxParts); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PartitionValuesRequest &a, PartitionValuesRequest &b) { - using ::std::swap; - swap(a.dbName, b.dbName); - swap(a.tblName, b.tblName); - swap(a.partitionKeys, b.partitionKeys); - swap(a.applyDistinct, b.applyDistinct); - swap(a.filter, b.filter); - swap(a.partitionOrder, b.partitionOrder); - swap(a.ascending, b.ascending); - swap(a.maxParts, b.maxParts); - swap(a.__isset, b.__isset); -} - -PartitionValuesRequest::PartitionValuesRequest(const PartitionValuesRequest& other539) { - dbName = other539.dbName; - tblName = other539.tblName; - partitionKeys = other539.partitionKeys; - applyDistinct = other539.applyDistinct; - filter = other539.filter; - partitionOrder = other539.partitionOrder; - ascending = other539.ascending; - maxParts = other539.maxParts; - __isset = other539.__isset; -} -PartitionValuesRequest& PartitionValuesRequest::operator=(const PartitionValuesRequest& other540) { - dbName = other540.dbName; - tblName = other540.tblName; - partitionKeys = other540.partitionKeys; - applyDistinct = other540.applyDistinct; - filter = other540.filter; - partitionOrder = other540.partitionOrder; - ascending = other540.ascending; - maxParts = other540.maxParts; - __isset = other540.__isset; - return *this; -} -void PartitionValuesRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PartitionValuesRequest("; - out << "dbName=" << to_string(dbName); - out << ", " << "tblName=" << to_string(tblName); - out << ", " << "partitionKeys=" << to_string(partitionKeys); - out << ", " << "applyDistinct="; (__isset.applyDistinct ? (out << to_string(applyDistinct)) : (out << "")); - out << ", " << "filter="; (__isset.filter ? (out << to_string(filter)) : (out << "")); - out << ", " << "partitionOrder="; (__isset.partitionOrder ? (out << to_string(partitionOrder)) : (out << "")); - out << ", " << "ascending="; (__isset.ascending ? (out << to_string(ascending)) : (out << "")); - out << ", " << "maxParts="; (__isset.maxParts ? (out << to_string(maxParts)) : (out << "")); - out << ")"; -} - - -PartitionValuesRow::~PartitionValuesRow() noexcept { -} - - -void PartitionValuesRow::__set_row(const std::vector & val) { - this->row = val; -} -std::ostream& operator<<(std::ostream& out, const PartitionValuesRow& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PartitionValuesRow::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_row = 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_LIST) { - { - this->row.clear(); - uint32_t _size541; - ::apache::thrift::protocol::TType _etype544; - xfer += iprot->readListBegin(_etype544, _size541); - this->row.resize(_size541); - uint32_t _i545; - for (_i545 = 0; _i545 < _size541; ++_i545) - { - xfer += iprot->readString(this->row[_i545]); - } - xfer += iprot->readListEnd(); - } - isset_row = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_row) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t PartitionValuesRow::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PartitionValuesRow"); - - xfer += oprot->writeFieldBegin("row", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->row.size())); - std::vector ::const_iterator _iter546; - for (_iter546 = this->row.begin(); _iter546 != this->row.end(); ++_iter546) - { - xfer += oprot->writeString((*_iter546)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PartitionValuesRow &a, PartitionValuesRow &b) { - using ::std::swap; - swap(a.row, b.row); -} - -PartitionValuesRow::PartitionValuesRow(const PartitionValuesRow& other547) { - row = other547.row; -} -PartitionValuesRow& PartitionValuesRow::operator=(const PartitionValuesRow& other548) { - row = other548.row; - return *this; -} -void PartitionValuesRow::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PartitionValuesRow("; - out << "row=" << to_string(row); - out << ")"; -} - - -PartitionValuesResponse::~PartitionValuesResponse() noexcept { -} - - -void PartitionValuesResponse::__set_partitionValues(const std::vector & val) { - this->partitionValues = val; -} -std::ostream& operator<<(std::ostream& out, const PartitionValuesResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PartitionValuesResponse::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_partitionValues = 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_LIST) { - { - this->partitionValues.clear(); - uint32_t _size549; - ::apache::thrift::protocol::TType _etype552; - xfer += iprot->readListBegin(_etype552, _size549); - this->partitionValues.resize(_size549); - uint32_t _i553; - for (_i553 = 0; _i553 < _size549; ++_i553) - { - xfer += this->partitionValues[_i553].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_partitionValues = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_partitionValues) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t PartitionValuesResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PartitionValuesResponse"); - - xfer += oprot->writeFieldBegin("partitionValues", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->partitionValues.size())); - std::vector ::const_iterator _iter554; - for (_iter554 = this->partitionValues.begin(); _iter554 != this->partitionValues.end(); ++_iter554) - { - xfer += (*_iter554).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PartitionValuesResponse &a, PartitionValuesResponse &b) { - using ::std::swap; - swap(a.partitionValues, b.partitionValues); -} - -PartitionValuesResponse::PartitionValuesResponse(const PartitionValuesResponse& other555) { - partitionValues = other555.partitionValues; -} -PartitionValuesResponse& PartitionValuesResponse::operator=(const PartitionValuesResponse& other556) { - partitionValues = other556.partitionValues; - return *this; -} -void PartitionValuesResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PartitionValuesResponse("; - out << "partitionValues=" << to_string(partitionValues); - out << ")"; -} - - -ResourceUri::~ResourceUri() noexcept { -} - - -void ResourceUri::__set_resourceType(const ResourceType::type val) { - this->resourceType = val; -} - -void ResourceUri::__set_uri(const std::string& val) { - this->uri = val; -} -std::ostream& operator<<(std::ostream& out, const ResourceUri& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ResourceUri::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast557; - xfer += iprot->readI32(ecast557); - this->resourceType = (ResourceType::type)ecast557; - this->__isset.resourceType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->uri); - this->__isset.uri = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ResourceUri::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ResourceUri"); - - xfer += oprot->writeFieldBegin("resourceType", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32((int32_t)this->resourceType); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("uri", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->uri); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ResourceUri &a, ResourceUri &b) { - using ::std::swap; - swap(a.resourceType, b.resourceType); - swap(a.uri, b.uri); - swap(a.__isset, b.__isset); -} - -ResourceUri::ResourceUri(const ResourceUri& other558) { - resourceType = other558.resourceType; - uri = other558.uri; - __isset = other558.__isset; -} -ResourceUri& ResourceUri::operator=(const ResourceUri& other559) { - resourceType = other559.resourceType; - uri = other559.uri; - __isset = other559.__isset; - return *this; -} -void ResourceUri::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ResourceUri("; - out << "resourceType=" << to_string(resourceType); - out << ", " << "uri=" << to_string(uri); - out << ")"; -} - - -Function::~Function() noexcept { -} - - -void Function::__set_functionName(const std::string& val) { - this->functionName = val; -} - -void Function::__set_dbName(const std::string& val) { - this->dbName = val; -} - -void Function::__set_className(const std::string& val) { - this->className = val; -} - -void Function::__set_ownerName(const std::string& val) { - this->ownerName = val; -} - -void Function::__set_ownerType(const PrincipalType::type val) { - this->ownerType = val; -} - -void Function::__set_createTime(const int32_t val) { - this->createTime = val; -} - -void Function::__set_functionType(const FunctionType::type val) { - this->functionType = val; -} - -void Function::__set_resourceUris(const std::vector & val) { - this->resourceUris = val; -} -std::ostream& operator<<(std::ostream& out, const Function& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t Function::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->functionName); - this->__isset.functionName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbName); - this->__isset.dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->className); - this->__isset.className = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->ownerName); - this->__isset.ownerName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast560; - xfer += iprot->readI32(ecast560); - this->ownerType = (PrincipalType::type)ecast560; - this->__isset.ownerType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->createTime); - this->__isset.createTime = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast561; - xfer += iprot->readI32(ecast561); - this->functionType = (FunctionType::type)ecast561; - this->__isset.functionType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->resourceUris.clear(); - uint32_t _size562; - ::apache::thrift::protocol::TType _etype565; - xfer += iprot->readListBegin(_etype565, _size562); - this->resourceUris.resize(_size562); - uint32_t _i566; - for (_i566 = 0; _i566 < _size562; ++_i566) - { - xfer += this->resourceUris[_i566].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.resourceUris = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t Function::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("Function"); - - xfer += oprot->writeFieldBegin("functionName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->functionName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("className", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->className); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("ownerName", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->ownerName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("ownerType", ::apache::thrift::protocol::T_I32, 5); - xfer += oprot->writeI32((int32_t)this->ownerType); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 6); - xfer += oprot->writeI32(this->createTime); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("functionType", ::apache::thrift::protocol::T_I32, 7); - xfer += oprot->writeI32((int32_t)this->functionType); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("resourceUris", ::apache::thrift::protocol::T_LIST, 8); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->resourceUris.size())); - std::vector ::const_iterator _iter567; - for (_iter567 = this->resourceUris.begin(); _iter567 != this->resourceUris.end(); ++_iter567) - { - xfer += (*_iter567).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(Function &a, Function &b) { - using ::std::swap; - swap(a.functionName, b.functionName); - swap(a.dbName, b.dbName); - swap(a.className, b.className); - swap(a.ownerName, b.ownerName); - swap(a.ownerType, b.ownerType); - swap(a.createTime, b.createTime); - swap(a.functionType, b.functionType); - swap(a.resourceUris, b.resourceUris); - swap(a.__isset, b.__isset); -} - -Function::Function(const Function& other568) { - functionName = other568.functionName; - dbName = other568.dbName; - className = other568.className; - ownerName = other568.ownerName; - ownerType = other568.ownerType; - createTime = other568.createTime; - functionType = other568.functionType; - resourceUris = other568.resourceUris; - __isset = other568.__isset; -} -Function& Function::operator=(const Function& other569) { - functionName = other569.functionName; - dbName = other569.dbName; - className = other569.className; - ownerName = other569.ownerName; - ownerType = other569.ownerType; - createTime = other569.createTime; - functionType = other569.functionType; - resourceUris = other569.resourceUris; - __isset = other569.__isset; - return *this; -} -void Function::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "Function("; - out << "functionName=" << to_string(functionName); - out << ", " << "dbName=" << to_string(dbName); - out << ", " << "className=" << to_string(className); - out << ", " << "ownerName=" << to_string(ownerName); - out << ", " << "ownerType=" << to_string(ownerType); - out << ", " << "createTime=" << to_string(createTime); - out << ", " << "functionType=" << to_string(functionType); - out << ", " << "resourceUris=" << to_string(resourceUris); - out << ")"; -} - - -TxnInfo::~TxnInfo() noexcept { -} - - -void TxnInfo::__set_id(const int64_t val) { - this->id = val; -} - -void TxnInfo::__set_state(const TxnState::type val) { - this->state = val; -} - -void TxnInfo::__set_user(const std::string& val) { - this->user = val; -} - -void TxnInfo::__set_hostname(const std::string& val) { - this->hostname = val; -} - -void TxnInfo::__set_agentInfo(const std::string& val) { - this->agentInfo = val; -__isset.agentInfo = true; -} - -void TxnInfo::__set_heartbeatCount(const int32_t val) { - this->heartbeatCount = val; -__isset.heartbeatCount = true; -} - -void TxnInfo::__set_metaInfo(const std::string& val) { - this->metaInfo = val; -__isset.metaInfo = true; -} - -void TxnInfo::__set_startedTime(const int64_t val) { - this->startedTime = val; -__isset.startedTime = true; -} - -void TxnInfo::__set_lastHeartbeatTime(const int64_t val) { - this->lastHeartbeatTime = val; -__isset.lastHeartbeatTime = true; -} -std::ostream& operator<<(std::ostream& out, const TxnInfo& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t TxnInfo::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_id = false; - bool isset_state = false; - bool isset_user = false; - bool isset_hostname = 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_I64) { - xfer += iprot->readI64(this->id); - isset_id = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast570; - xfer += iprot->readI32(ecast570); - this->state = (TxnState::type)ecast570; - isset_state = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->user); - isset_user = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->hostname); - isset_hostname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->agentInfo); - this->__isset.agentInfo = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->heartbeatCount); - this->__isset.heartbeatCount = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->metaInfo); - this->__isset.metaInfo = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->startedTime); - this->__isset.startedTime = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 9: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->lastHeartbeatTime); - this->__isset.lastHeartbeatTime = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_id) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_state) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_user) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_hostname) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t TxnInfo::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("TxnInfo"); - - xfer += oprot->writeFieldBegin("id", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->id); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("state", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32((int32_t)this->state); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("user", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->user); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("hostname", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->hostname); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.agentInfo) { - xfer += oprot->writeFieldBegin("agentInfo", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString(this->agentInfo); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.heartbeatCount) { - xfer += oprot->writeFieldBegin("heartbeatCount", ::apache::thrift::protocol::T_I32, 6); - xfer += oprot->writeI32(this->heartbeatCount); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.metaInfo) { - xfer += oprot->writeFieldBegin("metaInfo", ::apache::thrift::protocol::T_STRING, 7); - xfer += oprot->writeString(this->metaInfo); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.startedTime) { - xfer += oprot->writeFieldBegin("startedTime", ::apache::thrift::protocol::T_I64, 8); - xfer += oprot->writeI64(this->startedTime); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.lastHeartbeatTime) { - xfer += oprot->writeFieldBegin("lastHeartbeatTime", ::apache::thrift::protocol::T_I64, 9); - xfer += oprot->writeI64(this->lastHeartbeatTime); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(TxnInfo &a, TxnInfo &b) { - using ::std::swap; - swap(a.id, b.id); - swap(a.state, b.state); - swap(a.user, b.user); - swap(a.hostname, b.hostname); - swap(a.agentInfo, b.agentInfo); - swap(a.heartbeatCount, b.heartbeatCount); - swap(a.metaInfo, b.metaInfo); - swap(a.startedTime, b.startedTime); - swap(a.lastHeartbeatTime, b.lastHeartbeatTime); - swap(a.__isset, b.__isset); -} - -TxnInfo::TxnInfo(const TxnInfo& other571) { - id = other571.id; - state = other571.state; - user = other571.user; - hostname = other571.hostname; - agentInfo = other571.agentInfo; - heartbeatCount = other571.heartbeatCount; - metaInfo = other571.metaInfo; - startedTime = other571.startedTime; - lastHeartbeatTime = other571.lastHeartbeatTime; - __isset = other571.__isset; -} -TxnInfo& TxnInfo::operator=(const TxnInfo& other572) { - id = other572.id; - state = other572.state; - user = other572.user; - hostname = other572.hostname; - agentInfo = other572.agentInfo; - heartbeatCount = other572.heartbeatCount; - metaInfo = other572.metaInfo; - startedTime = other572.startedTime; - lastHeartbeatTime = other572.lastHeartbeatTime; - __isset = other572.__isset; - return *this; -} -void TxnInfo::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "TxnInfo("; - out << "id=" << to_string(id); - out << ", " << "state=" << to_string(state); - out << ", " << "user=" << to_string(user); - out << ", " << "hostname=" << to_string(hostname); - out << ", " << "agentInfo="; (__isset.agentInfo ? (out << to_string(agentInfo)) : (out << "")); - out << ", " << "heartbeatCount="; (__isset.heartbeatCount ? (out << to_string(heartbeatCount)) : (out << "")); - out << ", " << "metaInfo="; (__isset.metaInfo ? (out << to_string(metaInfo)) : (out << "")); - out << ", " << "startedTime="; (__isset.startedTime ? (out << to_string(startedTime)) : (out << "")); - out << ", " << "lastHeartbeatTime="; (__isset.lastHeartbeatTime ? (out << to_string(lastHeartbeatTime)) : (out << "")); - out << ")"; -} - - -GetOpenTxnsInfoResponse::~GetOpenTxnsInfoResponse() noexcept { -} - - -void GetOpenTxnsInfoResponse::__set_txn_high_water_mark(const int64_t val) { - this->txn_high_water_mark = val; -} - -void GetOpenTxnsInfoResponse::__set_open_txns(const std::vector & val) { - this->open_txns = val; -} -std::ostream& operator<<(std::ostream& out, const GetOpenTxnsInfoResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GetOpenTxnsInfoResponse::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_txn_high_water_mark = false; - bool isset_open_txns = 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_I64) { - xfer += iprot->readI64(this->txn_high_water_mark); - isset_txn_high_water_mark = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->open_txns.clear(); - uint32_t _size573; - ::apache::thrift::protocol::TType _etype576; - xfer += iprot->readListBegin(_etype576, _size573); - this->open_txns.resize(_size573); - uint32_t _i577; - for (_i577 = 0; _i577 < _size573; ++_i577) - { - xfer += this->open_txns[_i577].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_open_txns = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_txn_high_water_mark) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_open_txns) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t GetOpenTxnsInfoResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GetOpenTxnsInfoResponse"); - - xfer += oprot->writeFieldBegin("txn_high_water_mark", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->txn_high_water_mark); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("open_txns", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->open_txns.size())); - std::vector ::const_iterator _iter578; - for (_iter578 = this->open_txns.begin(); _iter578 != this->open_txns.end(); ++_iter578) - { - xfer += (*_iter578).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GetOpenTxnsInfoResponse &a, GetOpenTxnsInfoResponse &b) { - using ::std::swap; - swap(a.txn_high_water_mark, b.txn_high_water_mark); - swap(a.open_txns, b.open_txns); -} - -GetOpenTxnsInfoResponse::GetOpenTxnsInfoResponse(const GetOpenTxnsInfoResponse& other579) { - txn_high_water_mark = other579.txn_high_water_mark; - open_txns = other579.open_txns; -} -GetOpenTxnsInfoResponse& GetOpenTxnsInfoResponse::operator=(const GetOpenTxnsInfoResponse& other580) { - txn_high_water_mark = other580.txn_high_water_mark; - open_txns = other580.open_txns; - return *this; -} -void GetOpenTxnsInfoResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GetOpenTxnsInfoResponse("; - out << "txn_high_water_mark=" << to_string(txn_high_water_mark); - out << ", " << "open_txns=" << to_string(open_txns); - out << ")"; -} - - -GetOpenTxnsResponse::~GetOpenTxnsResponse() noexcept { -} - - -void GetOpenTxnsResponse::__set_txn_high_water_mark(const int64_t val) { - this->txn_high_water_mark = val; -} - -void GetOpenTxnsResponse::__set_open_txns(const std::vector & val) { - this->open_txns = val; -} - -void GetOpenTxnsResponse::__set_min_open_txn(const int64_t val) { - this->min_open_txn = val; -__isset.min_open_txn = true; -} - -void GetOpenTxnsResponse::__set_abortedBits(const std::string& val) { - this->abortedBits = val; -} -std::ostream& operator<<(std::ostream& out, const GetOpenTxnsResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GetOpenTxnsResponse::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_txn_high_water_mark = false; - bool isset_open_txns = false; - bool isset_abortedBits = 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_I64) { - xfer += iprot->readI64(this->txn_high_water_mark); - isset_txn_high_water_mark = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->open_txns.clear(); - uint32_t _size581; - ::apache::thrift::protocol::TType _etype584; - xfer += iprot->readListBegin(_etype584, _size581); - this->open_txns.resize(_size581); - uint32_t _i585; - for (_i585 = 0; _i585 < _size581; ++_i585) - { - xfer += iprot->readI64(this->open_txns[_i585]); - } - xfer += iprot->readListEnd(); - } - isset_open_txns = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->min_open_txn); - this->__isset.min_open_txn = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->abortedBits); - isset_abortedBits = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_txn_high_water_mark) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_open_txns) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_abortedBits) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t GetOpenTxnsResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GetOpenTxnsResponse"); - - xfer += oprot->writeFieldBegin("txn_high_water_mark", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->txn_high_water_mark); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("open_txns", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->open_txns.size())); - std::vector ::const_iterator _iter586; - for (_iter586 = this->open_txns.begin(); _iter586 != this->open_txns.end(); ++_iter586) - { - xfer += oprot->writeI64((*_iter586)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - if (this->__isset.min_open_txn) { - xfer += oprot->writeFieldBegin("min_open_txn", ::apache::thrift::protocol::T_I64, 3); - xfer += oprot->writeI64(this->min_open_txn); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldBegin("abortedBits", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeBinary(this->abortedBits); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GetOpenTxnsResponse &a, GetOpenTxnsResponse &b) { - using ::std::swap; - swap(a.txn_high_water_mark, b.txn_high_water_mark); - swap(a.open_txns, b.open_txns); - swap(a.min_open_txn, b.min_open_txn); - swap(a.abortedBits, b.abortedBits); - swap(a.__isset, b.__isset); -} - -GetOpenTxnsResponse::GetOpenTxnsResponse(const GetOpenTxnsResponse& other587) { - txn_high_water_mark = other587.txn_high_water_mark; - open_txns = other587.open_txns; - min_open_txn = other587.min_open_txn; - abortedBits = other587.abortedBits; - __isset = other587.__isset; -} -GetOpenTxnsResponse& GetOpenTxnsResponse::operator=(const GetOpenTxnsResponse& other588) { - txn_high_water_mark = other588.txn_high_water_mark; - open_txns = other588.open_txns; - min_open_txn = other588.min_open_txn; - abortedBits = other588.abortedBits; - __isset = other588.__isset; - return *this; -} -void GetOpenTxnsResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GetOpenTxnsResponse("; - out << "txn_high_water_mark=" << to_string(txn_high_water_mark); - out << ", " << "open_txns=" << to_string(open_txns); - out << ", " << "min_open_txn="; (__isset.min_open_txn ? (out << to_string(min_open_txn)) : (out << "")); - out << ", " << "abortedBits=" << to_string(abortedBits); - out << ")"; -} - - -OpenTxnRequest::~OpenTxnRequest() noexcept { -} - - -void OpenTxnRequest::__set_num_txns(const int32_t val) { - this->num_txns = val; -} - -void OpenTxnRequest::__set_user(const std::string& val) { - this->user = val; -} - -void OpenTxnRequest::__set_hostname(const std::string& val) { - this->hostname = val; -} - -void OpenTxnRequest::__set_agentInfo(const std::string& val) { - this->agentInfo = val; -__isset.agentInfo = true; -} -std::ostream& operator<<(std::ostream& out, const OpenTxnRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t OpenTxnRequest::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_num_txns = false; - bool isset_user = false; - bool isset_hostname = 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->num_txns); - isset_num_txns = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->user); - isset_user = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->hostname); - isset_hostname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->agentInfo); - this->__isset.agentInfo = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_num_txns) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_user) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_hostname) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t OpenTxnRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("OpenTxnRequest"); - - xfer += oprot->writeFieldBegin("num_txns", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32(this->num_txns); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("user", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->user); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("hostname", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->hostname); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.agentInfo) { - xfer += oprot->writeFieldBegin("agentInfo", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->agentInfo); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(OpenTxnRequest &a, OpenTxnRequest &b) { - using ::std::swap; - swap(a.num_txns, b.num_txns); - swap(a.user, b.user); - swap(a.hostname, b.hostname); - swap(a.agentInfo, b.agentInfo); - swap(a.__isset, b.__isset); -} - -OpenTxnRequest::OpenTxnRequest(const OpenTxnRequest& other589) { - num_txns = other589.num_txns; - user = other589.user; - hostname = other589.hostname; - agentInfo = other589.agentInfo; - __isset = other589.__isset; -} -OpenTxnRequest& OpenTxnRequest::operator=(const OpenTxnRequest& other590) { - num_txns = other590.num_txns; - user = other590.user; - hostname = other590.hostname; - agentInfo = other590.agentInfo; - __isset = other590.__isset; - return *this; -} -void OpenTxnRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "OpenTxnRequest("; - out << "num_txns=" << to_string(num_txns); - out << ", " << "user=" << to_string(user); - out << ", " << "hostname=" << to_string(hostname); - out << ", " << "agentInfo="; (__isset.agentInfo ? (out << to_string(agentInfo)) : (out << "")); - out << ")"; -} - - -OpenTxnsResponse::~OpenTxnsResponse() noexcept { -} - - -void OpenTxnsResponse::__set_txn_ids(const std::vector & val) { - this->txn_ids = val; -} -std::ostream& operator<<(std::ostream& out, const OpenTxnsResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t OpenTxnsResponse::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_txn_ids = 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_LIST) { - { - this->txn_ids.clear(); - uint32_t _size591; - ::apache::thrift::protocol::TType _etype594; - xfer += iprot->readListBegin(_etype594, _size591); - this->txn_ids.resize(_size591); - uint32_t _i595; - for (_i595 = 0; _i595 < _size591; ++_i595) - { - xfer += iprot->readI64(this->txn_ids[_i595]); - } - xfer += iprot->readListEnd(); - } - isset_txn_ids = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_txn_ids) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t OpenTxnsResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("OpenTxnsResponse"); - - xfer += oprot->writeFieldBegin("txn_ids", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->txn_ids.size())); - std::vector ::const_iterator _iter596; - for (_iter596 = this->txn_ids.begin(); _iter596 != this->txn_ids.end(); ++_iter596) - { - xfer += oprot->writeI64((*_iter596)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(OpenTxnsResponse &a, OpenTxnsResponse &b) { - using ::std::swap; - swap(a.txn_ids, b.txn_ids); -} - -OpenTxnsResponse::OpenTxnsResponse(const OpenTxnsResponse& other597) { - txn_ids = other597.txn_ids; -} -OpenTxnsResponse& OpenTxnsResponse::operator=(const OpenTxnsResponse& other598) { - txn_ids = other598.txn_ids; - return *this; -} -void OpenTxnsResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "OpenTxnsResponse("; - out << "txn_ids=" << to_string(txn_ids); - out << ")"; -} - - -AbortTxnRequest::~AbortTxnRequest() noexcept { -} - - -void AbortTxnRequest::__set_txnid(const int64_t val) { - this->txnid = val; -} -std::ostream& operator<<(std::ostream& out, const AbortTxnRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t AbortTxnRequest::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_txnid = 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_I64) { - xfer += iprot->readI64(this->txnid); - isset_txnid = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_txnid) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t AbortTxnRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("AbortTxnRequest"); - - xfer += oprot->writeFieldBegin("txnid", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->txnid); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(AbortTxnRequest &a, AbortTxnRequest &b) { - using ::std::swap; - swap(a.txnid, b.txnid); -} - -AbortTxnRequest::AbortTxnRequest(const AbortTxnRequest& other599) { - txnid = other599.txnid; -} -AbortTxnRequest& AbortTxnRequest::operator=(const AbortTxnRequest& other600) { - txnid = other600.txnid; - return *this; -} -void AbortTxnRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "AbortTxnRequest("; - out << "txnid=" << to_string(txnid); - out << ")"; -} - - -AbortTxnsRequest::~AbortTxnsRequest() noexcept { -} - - -void AbortTxnsRequest::__set_txn_ids(const std::vector & val) { - this->txn_ids = val; -} -std::ostream& operator<<(std::ostream& out, const AbortTxnsRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t AbortTxnsRequest::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_txn_ids = 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_LIST) { - { - this->txn_ids.clear(); - uint32_t _size601; - ::apache::thrift::protocol::TType _etype604; - xfer += iprot->readListBegin(_etype604, _size601); - this->txn_ids.resize(_size601); - uint32_t _i605; - for (_i605 = 0; _i605 < _size601; ++_i605) - { - xfer += iprot->readI64(this->txn_ids[_i605]); - } - xfer += iprot->readListEnd(); - } - isset_txn_ids = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_txn_ids) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t AbortTxnsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("AbortTxnsRequest"); - - xfer += oprot->writeFieldBegin("txn_ids", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->txn_ids.size())); - std::vector ::const_iterator _iter606; - for (_iter606 = this->txn_ids.begin(); _iter606 != this->txn_ids.end(); ++_iter606) - { - xfer += oprot->writeI64((*_iter606)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(AbortTxnsRequest &a, AbortTxnsRequest &b) { - using ::std::swap; - swap(a.txn_ids, b.txn_ids); -} - -AbortTxnsRequest::AbortTxnsRequest(const AbortTxnsRequest& other607) { - txn_ids = other607.txn_ids; -} -AbortTxnsRequest& AbortTxnsRequest::operator=(const AbortTxnsRequest& other608) { - txn_ids = other608.txn_ids; - return *this; -} -void AbortTxnsRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "AbortTxnsRequest("; - out << "txn_ids=" << to_string(txn_ids); - out << ")"; -} - - -CommitTxnRequest::~CommitTxnRequest() noexcept { -} - - -void CommitTxnRequest::__set_txnid(const int64_t val) { - this->txnid = val; -} -std::ostream& operator<<(std::ostream& out, const CommitTxnRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t CommitTxnRequest::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_txnid = 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_I64) { - xfer += iprot->readI64(this->txnid); - isset_txnid = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_txnid) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t CommitTxnRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("CommitTxnRequest"); - - xfer += oprot->writeFieldBegin("txnid", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->txnid); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(CommitTxnRequest &a, CommitTxnRequest &b) { - using ::std::swap; - swap(a.txnid, b.txnid); -} - -CommitTxnRequest::CommitTxnRequest(const CommitTxnRequest& other609) { - txnid = other609.txnid; -} -CommitTxnRequest& CommitTxnRequest::operator=(const CommitTxnRequest& other610) { - txnid = other610.txnid; - return *this; -} -void CommitTxnRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "CommitTxnRequest("; - out << "txnid=" << to_string(txnid); - out << ")"; -} - - -LockComponent::~LockComponent() noexcept { -} - - -void LockComponent::__set_type(const LockType::type val) { - this->type = val; -} - -void LockComponent::__set_level(const LockLevel::type val) { - this->level = val; -} - -void LockComponent::__set_dbname(const std::string& val) { - this->dbname = val; -} - -void LockComponent::__set_tablename(const std::string& val) { - this->tablename = val; -__isset.tablename = true; -} - -void LockComponent::__set_partitionname(const std::string& val) { - this->partitionname = val; -__isset.partitionname = true; -} - -void LockComponent::__set_operationType(const DataOperationType::type val) { - this->operationType = val; -__isset.operationType = true; -} - -void LockComponent::__set_isAcid(const bool val) { - this->isAcid = val; -__isset.isAcid = true; -} - -void LockComponent::__set_isDynamicPartitionWrite(const bool val) { - this->isDynamicPartitionWrite = val; -__isset.isDynamicPartitionWrite = true; -} -std::ostream& operator<<(std::ostream& out, const LockComponent& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t LockComponent::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_type = false; - bool isset_level = false; - bool isset_dbname = 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) { - int32_t ecast611; - xfer += iprot->readI32(ecast611); - this->type = (LockType::type)ecast611; - isset_type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast612; - xfer += iprot->readI32(ecast612); - this->level = (LockLevel::type)ecast612; - isset_level = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbname); - isset_dbname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tablename); - this->__isset.tablename = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->partitionname); - this->__isset.partitionname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast613; - xfer += iprot->readI32(ecast613); - this->operationType = (DataOperationType::type)ecast613; - this->__isset.operationType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->isAcid); - this->__isset.isAcid = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->isDynamicPartitionWrite); - this->__isset.isDynamicPartitionWrite = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_type) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_level) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_dbname) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t LockComponent::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("LockComponent"); - - xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32((int32_t)this->type); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("level", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32((int32_t)this->level); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->dbname); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.tablename) { - xfer += oprot->writeFieldBegin("tablename", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->tablename); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.partitionname) { - xfer += oprot->writeFieldBegin("partitionname", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString(this->partitionname); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.operationType) { - xfer += oprot->writeFieldBegin("operationType", ::apache::thrift::protocol::T_I32, 6); - xfer += oprot->writeI32((int32_t)this->operationType); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.isAcid) { - xfer += oprot->writeFieldBegin("isAcid", ::apache::thrift::protocol::T_BOOL, 7); - xfer += oprot->writeBool(this->isAcid); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.isDynamicPartitionWrite) { - xfer += oprot->writeFieldBegin("isDynamicPartitionWrite", ::apache::thrift::protocol::T_BOOL, 8); - xfer += oprot->writeBool(this->isDynamicPartitionWrite); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(LockComponent &a, LockComponent &b) { - using ::std::swap; - swap(a.type, b.type); - swap(a.level, b.level); - swap(a.dbname, b.dbname); - swap(a.tablename, b.tablename); - swap(a.partitionname, b.partitionname); - swap(a.operationType, b.operationType); - swap(a.isAcid, b.isAcid); - swap(a.isDynamicPartitionWrite, b.isDynamicPartitionWrite); - swap(a.__isset, b.__isset); -} - -LockComponent::LockComponent(const LockComponent& other614) { - type = other614.type; - level = other614.level; - dbname = other614.dbname; - tablename = other614.tablename; - partitionname = other614.partitionname; - operationType = other614.operationType; - isAcid = other614.isAcid; - isDynamicPartitionWrite = other614.isDynamicPartitionWrite; - __isset = other614.__isset; -} -LockComponent& LockComponent::operator=(const LockComponent& other615) { - type = other615.type; - level = other615.level; - dbname = other615.dbname; - tablename = other615.tablename; - partitionname = other615.partitionname; - operationType = other615.operationType; - isAcid = other615.isAcid; - isDynamicPartitionWrite = other615.isDynamicPartitionWrite; - __isset = other615.__isset; - return *this; -} -void LockComponent::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "LockComponent("; - out << "type=" << to_string(type); - out << ", " << "level=" << to_string(level); - out << ", " << "dbname=" << to_string(dbname); - out << ", " << "tablename="; (__isset.tablename ? (out << to_string(tablename)) : (out << "")); - out << ", " << "partitionname="; (__isset.partitionname ? (out << to_string(partitionname)) : (out << "")); - out << ", " << "operationType="; (__isset.operationType ? (out << to_string(operationType)) : (out << "")); - out << ", " << "isAcid="; (__isset.isAcid ? (out << to_string(isAcid)) : (out << "")); - out << ", " << "isDynamicPartitionWrite="; (__isset.isDynamicPartitionWrite ? (out << to_string(isDynamicPartitionWrite)) : (out << "")); - out << ")"; -} - - -LockRequest::~LockRequest() noexcept { -} - - -void LockRequest::__set_component(const std::vector & val) { - this->component = val; -} - -void LockRequest::__set_txnid(const int64_t val) { - this->txnid = val; -__isset.txnid = true; -} - -void LockRequest::__set_user(const std::string& val) { - this->user = val; -} - -void LockRequest::__set_hostname(const std::string& val) { - this->hostname = val; -} - -void LockRequest::__set_agentInfo(const std::string& val) { - this->agentInfo = val; -__isset.agentInfo = true; -} -std::ostream& operator<<(std::ostream& out, const LockRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t LockRequest::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_component = false; - bool isset_user = false; - bool isset_hostname = 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_LIST) { - { - this->component.clear(); - uint32_t _size616; - ::apache::thrift::protocol::TType _etype619; - xfer += iprot->readListBegin(_etype619, _size616); - this->component.resize(_size616); - uint32_t _i620; - for (_i620 = 0; _i620 < _size616; ++_i620) - { - xfer += this->component[_i620].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_component = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->txnid); - this->__isset.txnid = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->user); - isset_user = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->hostname); - isset_hostname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->agentInfo); - this->__isset.agentInfo = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_component) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_user) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_hostname) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t LockRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("LockRequest"); - - xfer += oprot->writeFieldBegin("component", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->component.size())); - std::vector ::const_iterator _iter621; - for (_iter621 = this->component.begin(); _iter621 != this->component.end(); ++_iter621) - { - xfer += (*_iter621).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - if (this->__isset.txnid) { - xfer += oprot->writeFieldBegin("txnid", ::apache::thrift::protocol::T_I64, 2); - xfer += oprot->writeI64(this->txnid); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldBegin("user", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->user); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("hostname", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->hostname); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.agentInfo) { - xfer += oprot->writeFieldBegin("agentInfo", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString(this->agentInfo); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(LockRequest &a, LockRequest &b) { - using ::std::swap; - swap(a.component, b.component); - swap(a.txnid, b.txnid); - swap(a.user, b.user); - swap(a.hostname, b.hostname); - swap(a.agentInfo, b.agentInfo); - swap(a.__isset, b.__isset); -} - -LockRequest::LockRequest(const LockRequest& other622) { - component = other622.component; - txnid = other622.txnid; - user = other622.user; - hostname = other622.hostname; - agentInfo = other622.agentInfo; - __isset = other622.__isset; -} -LockRequest& LockRequest::operator=(const LockRequest& other623) { - component = other623.component; - txnid = other623.txnid; - user = other623.user; - hostname = other623.hostname; - agentInfo = other623.agentInfo; - __isset = other623.__isset; - return *this; -} -void LockRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "LockRequest("; - out << "component=" << to_string(component); - out << ", " << "txnid="; (__isset.txnid ? (out << to_string(txnid)) : (out << "")); - out << ", " << "user=" << to_string(user); - out << ", " << "hostname=" << to_string(hostname); - out << ", " << "agentInfo="; (__isset.agentInfo ? (out << to_string(agentInfo)) : (out << "")); - out << ")"; -} - - -LockResponse::~LockResponse() noexcept { -} - - -void LockResponse::__set_lockid(const int64_t val) { - this->lockid = val; -} - -void LockResponse::__set_state(const LockState::type val) { - this->state = val; -} -std::ostream& operator<<(std::ostream& out, const LockResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t LockResponse::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_lockid = false; - bool isset_state = 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_I64) { - xfer += iprot->readI64(this->lockid); - isset_lockid = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast624; - xfer += iprot->readI32(ecast624); - this->state = (LockState::type)ecast624; - isset_state = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_lockid) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_state) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t LockResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("LockResponse"); - - xfer += oprot->writeFieldBegin("lockid", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->lockid); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("state", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32((int32_t)this->state); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(LockResponse &a, LockResponse &b) { - using ::std::swap; - swap(a.lockid, b.lockid); - swap(a.state, b.state); -} - -LockResponse::LockResponse(const LockResponse& other625) { - lockid = other625.lockid; - state = other625.state; -} -LockResponse& LockResponse::operator=(const LockResponse& other626) { - lockid = other626.lockid; - state = other626.state; - return *this; -} -void LockResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "LockResponse("; - out << "lockid=" << to_string(lockid); - out << ", " << "state=" << to_string(state); - out << ")"; -} - - -CheckLockRequest::~CheckLockRequest() noexcept { -} - - -void CheckLockRequest::__set_lockid(const int64_t val) { - this->lockid = val; -} - -void CheckLockRequest::__set_txnid(const int64_t val) { - this->txnid = val; -__isset.txnid = true; -} - -void CheckLockRequest::__set_elapsed_ms(const int64_t val) { - this->elapsed_ms = val; -__isset.elapsed_ms = true; -} -std::ostream& operator<<(std::ostream& out, const CheckLockRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t CheckLockRequest::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_lockid = 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_I64) { - xfer += iprot->readI64(this->lockid); - isset_lockid = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->txnid); - this->__isset.txnid = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->elapsed_ms); - this->__isset.elapsed_ms = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_lockid) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t CheckLockRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("CheckLockRequest"); - - xfer += oprot->writeFieldBegin("lockid", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->lockid); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.txnid) { - xfer += oprot->writeFieldBegin("txnid", ::apache::thrift::protocol::T_I64, 2); - xfer += oprot->writeI64(this->txnid); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.elapsed_ms) { - xfer += oprot->writeFieldBegin("elapsed_ms", ::apache::thrift::protocol::T_I64, 3); - xfer += oprot->writeI64(this->elapsed_ms); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(CheckLockRequest &a, CheckLockRequest &b) { - using ::std::swap; - swap(a.lockid, b.lockid); - swap(a.txnid, b.txnid); - swap(a.elapsed_ms, b.elapsed_ms); - swap(a.__isset, b.__isset); -} - -CheckLockRequest::CheckLockRequest(const CheckLockRequest& other627) { - lockid = other627.lockid; - txnid = other627.txnid; - elapsed_ms = other627.elapsed_ms; - __isset = other627.__isset; -} -CheckLockRequest& CheckLockRequest::operator=(const CheckLockRequest& other628) { - lockid = other628.lockid; - txnid = other628.txnid; - elapsed_ms = other628.elapsed_ms; - __isset = other628.__isset; - return *this; -} -void CheckLockRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "CheckLockRequest("; - out << "lockid=" << to_string(lockid); - out << ", " << "txnid="; (__isset.txnid ? (out << to_string(txnid)) : (out << "")); - out << ", " << "elapsed_ms="; (__isset.elapsed_ms ? (out << to_string(elapsed_ms)) : (out << "")); - out << ")"; -} - - -UnlockRequest::~UnlockRequest() noexcept { -} - - -void UnlockRequest::__set_lockid(const int64_t val) { - this->lockid = val; -} -std::ostream& operator<<(std::ostream& out, const UnlockRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t UnlockRequest::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_lockid = 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_I64) { - xfer += iprot->readI64(this->lockid); - isset_lockid = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_lockid) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t UnlockRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("UnlockRequest"); - - xfer += oprot->writeFieldBegin("lockid", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->lockid); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(UnlockRequest &a, UnlockRequest &b) { - using ::std::swap; - swap(a.lockid, b.lockid); -} - -UnlockRequest::UnlockRequest(const UnlockRequest& other629) { - lockid = other629.lockid; -} -UnlockRequest& UnlockRequest::operator=(const UnlockRequest& other630) { - lockid = other630.lockid; - return *this; -} -void UnlockRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "UnlockRequest("; - out << "lockid=" << to_string(lockid); - out << ")"; -} - - -ShowLocksRequest::~ShowLocksRequest() noexcept { -} - - -void ShowLocksRequest::__set_dbname(const std::string& val) { - this->dbname = val; -__isset.dbname = true; -} - -void ShowLocksRequest::__set_tablename(const std::string& val) { - this->tablename = val; -__isset.tablename = true; -} - -void ShowLocksRequest::__set_partname(const std::string& val) { - this->partname = val; -__isset.partname = true; -} - -void ShowLocksRequest::__set_isExtended(const bool val) { - this->isExtended = val; -__isset.isExtended = true; -} -std::ostream& operator<<(std::ostream& out, const ShowLocksRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ShowLocksRequest::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbname); - this->__isset.dbname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tablename); - this->__isset.tablename = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->partname); - this->__isset.partname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->isExtended); - this->__isset.isExtended = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ShowLocksRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ShowLocksRequest"); - - if (this->__isset.dbname) { - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbname); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.tablename) { - xfer += oprot->writeFieldBegin("tablename", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tablename); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.partname) { - xfer += oprot->writeFieldBegin("partname", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->partname); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.isExtended) { - xfer += oprot->writeFieldBegin("isExtended", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool(this->isExtended); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ShowLocksRequest &a, ShowLocksRequest &b) { - using ::std::swap; - swap(a.dbname, b.dbname); - swap(a.tablename, b.tablename); - swap(a.partname, b.partname); - swap(a.isExtended, b.isExtended); - swap(a.__isset, b.__isset); -} - -ShowLocksRequest::ShowLocksRequest(const ShowLocksRequest& other631) { - dbname = other631.dbname; - tablename = other631.tablename; - partname = other631.partname; - isExtended = other631.isExtended; - __isset = other631.__isset; -} -ShowLocksRequest& ShowLocksRequest::operator=(const ShowLocksRequest& other632) { - dbname = other632.dbname; - tablename = other632.tablename; - partname = other632.partname; - isExtended = other632.isExtended; - __isset = other632.__isset; - return *this; -} -void ShowLocksRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ShowLocksRequest("; - out << "dbname="; (__isset.dbname ? (out << to_string(dbname)) : (out << "")); - out << ", " << "tablename="; (__isset.tablename ? (out << to_string(tablename)) : (out << "")); - out << ", " << "partname="; (__isset.partname ? (out << to_string(partname)) : (out << "")); - out << ", " << "isExtended="; (__isset.isExtended ? (out << to_string(isExtended)) : (out << "")); - out << ")"; -} - - -ShowLocksResponseElement::~ShowLocksResponseElement() noexcept { -} - - -void ShowLocksResponseElement::__set_lockid(const int64_t val) { - this->lockid = val; -} - -void ShowLocksResponseElement::__set_dbname(const std::string& val) { - this->dbname = val; -} - -void ShowLocksResponseElement::__set_tablename(const std::string& val) { - this->tablename = val; -__isset.tablename = true; -} - -void ShowLocksResponseElement::__set_partname(const std::string& val) { - this->partname = val; -__isset.partname = true; -} - -void ShowLocksResponseElement::__set_state(const LockState::type val) { - this->state = val; -} - -void ShowLocksResponseElement::__set_type(const LockType::type val) { - this->type = val; -} - -void ShowLocksResponseElement::__set_txnid(const int64_t val) { - this->txnid = val; -__isset.txnid = true; -} - -void ShowLocksResponseElement::__set_lastheartbeat(const int64_t val) { - this->lastheartbeat = val; -} - -void ShowLocksResponseElement::__set_acquiredat(const int64_t val) { - this->acquiredat = val; -__isset.acquiredat = true; -} - -void ShowLocksResponseElement::__set_user(const std::string& val) { - this->user = val; -} - -void ShowLocksResponseElement::__set_hostname(const std::string& val) { - this->hostname = val; -} - -void ShowLocksResponseElement::__set_heartbeatCount(const int32_t val) { - this->heartbeatCount = val; -__isset.heartbeatCount = true; -} - -void ShowLocksResponseElement::__set_agentInfo(const std::string& val) { - this->agentInfo = val; -__isset.agentInfo = true; -} - -void ShowLocksResponseElement::__set_blockedByExtId(const int64_t val) { - this->blockedByExtId = val; -__isset.blockedByExtId = true; -} - -void ShowLocksResponseElement::__set_blockedByIntId(const int64_t val) { - this->blockedByIntId = val; -__isset.blockedByIntId = true; -} - -void ShowLocksResponseElement::__set_lockIdInternal(const int64_t val) { - this->lockIdInternal = val; -__isset.lockIdInternal = true; -} -std::ostream& operator<<(std::ostream& out, const ShowLocksResponseElement& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ShowLocksResponseElement::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_lockid = false; - bool isset_dbname = false; - bool isset_state = false; - bool isset_type = false; - bool isset_lastheartbeat = false; - bool isset_user = false; - bool isset_hostname = 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_I64) { - xfer += iprot->readI64(this->lockid); - isset_lockid = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbname); - isset_dbname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tablename); - this->__isset.tablename = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->partname); - this->__isset.partname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast633; - xfer += iprot->readI32(ecast633); - this->state = (LockState::type)ecast633; - isset_state = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast634; - xfer += iprot->readI32(ecast634); - this->type = (LockType::type)ecast634; - isset_type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->txnid); - this->__isset.txnid = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->lastheartbeat); - isset_lastheartbeat = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 9: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->acquiredat); - this->__isset.acquiredat = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 10: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->user); - isset_user = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 11: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->hostname); - isset_hostname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 12: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->heartbeatCount); - this->__isset.heartbeatCount = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 13: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->agentInfo); - this->__isset.agentInfo = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 14: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->blockedByExtId); - this->__isset.blockedByExtId = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 15: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->blockedByIntId); - this->__isset.blockedByIntId = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 16: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->lockIdInternal); - this->__isset.lockIdInternal = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_lockid) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_dbname) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_state) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_type) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_lastheartbeat) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_user) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_hostname) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t ShowLocksResponseElement::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ShowLocksResponseElement"); - - xfer += oprot->writeFieldBegin("lockid", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->lockid); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->dbname); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.tablename) { - xfer += oprot->writeFieldBegin("tablename", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->tablename); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.partname) { - xfer += oprot->writeFieldBegin("partname", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->partname); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldBegin("state", ::apache::thrift::protocol::T_I32, 5); - xfer += oprot->writeI32((int32_t)this->state); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_I32, 6); - xfer += oprot->writeI32((int32_t)this->type); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.txnid) { - xfer += oprot->writeFieldBegin("txnid", ::apache::thrift::protocol::T_I64, 7); - xfer += oprot->writeI64(this->txnid); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldBegin("lastheartbeat", ::apache::thrift::protocol::T_I64, 8); - xfer += oprot->writeI64(this->lastheartbeat); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.acquiredat) { - xfer += oprot->writeFieldBegin("acquiredat", ::apache::thrift::protocol::T_I64, 9); - xfer += oprot->writeI64(this->acquiredat); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldBegin("user", ::apache::thrift::protocol::T_STRING, 10); - xfer += oprot->writeString(this->user); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("hostname", ::apache::thrift::protocol::T_STRING, 11); - xfer += oprot->writeString(this->hostname); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.heartbeatCount) { - xfer += oprot->writeFieldBegin("heartbeatCount", ::apache::thrift::protocol::T_I32, 12); - xfer += oprot->writeI32(this->heartbeatCount); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.agentInfo) { - xfer += oprot->writeFieldBegin("agentInfo", ::apache::thrift::protocol::T_STRING, 13); - xfer += oprot->writeString(this->agentInfo); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.blockedByExtId) { - xfer += oprot->writeFieldBegin("blockedByExtId", ::apache::thrift::protocol::T_I64, 14); - xfer += oprot->writeI64(this->blockedByExtId); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.blockedByIntId) { - xfer += oprot->writeFieldBegin("blockedByIntId", ::apache::thrift::protocol::T_I64, 15); - xfer += oprot->writeI64(this->blockedByIntId); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.lockIdInternal) { - xfer += oprot->writeFieldBegin("lockIdInternal", ::apache::thrift::protocol::T_I64, 16); - xfer += oprot->writeI64(this->lockIdInternal); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ShowLocksResponseElement &a, ShowLocksResponseElement &b) { - using ::std::swap; - swap(a.lockid, b.lockid); - swap(a.dbname, b.dbname); - swap(a.tablename, b.tablename); - swap(a.partname, b.partname); - swap(a.state, b.state); - swap(a.type, b.type); - swap(a.txnid, b.txnid); - swap(a.lastheartbeat, b.lastheartbeat); - swap(a.acquiredat, b.acquiredat); - swap(a.user, b.user); - swap(a.hostname, b.hostname); - swap(a.heartbeatCount, b.heartbeatCount); - swap(a.agentInfo, b.agentInfo); - swap(a.blockedByExtId, b.blockedByExtId); - swap(a.blockedByIntId, b.blockedByIntId); - swap(a.lockIdInternal, b.lockIdInternal); - swap(a.__isset, b.__isset); -} - -ShowLocksResponseElement::ShowLocksResponseElement(const ShowLocksResponseElement& other635) { - lockid = other635.lockid; - dbname = other635.dbname; - tablename = other635.tablename; - partname = other635.partname; - state = other635.state; - type = other635.type; - txnid = other635.txnid; - lastheartbeat = other635.lastheartbeat; - acquiredat = other635.acquiredat; - user = other635.user; - hostname = other635.hostname; - heartbeatCount = other635.heartbeatCount; - agentInfo = other635.agentInfo; - blockedByExtId = other635.blockedByExtId; - blockedByIntId = other635.blockedByIntId; - lockIdInternal = other635.lockIdInternal; - __isset = other635.__isset; -} -ShowLocksResponseElement& ShowLocksResponseElement::operator=(const ShowLocksResponseElement& other636) { - lockid = other636.lockid; - dbname = other636.dbname; - tablename = other636.tablename; - partname = other636.partname; - state = other636.state; - type = other636.type; - txnid = other636.txnid; - lastheartbeat = other636.lastheartbeat; - acquiredat = other636.acquiredat; - user = other636.user; - hostname = other636.hostname; - heartbeatCount = other636.heartbeatCount; - agentInfo = other636.agentInfo; - blockedByExtId = other636.blockedByExtId; - blockedByIntId = other636.blockedByIntId; - lockIdInternal = other636.lockIdInternal; - __isset = other636.__isset; - return *this; -} -void ShowLocksResponseElement::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ShowLocksResponseElement("; - out << "lockid=" << to_string(lockid); - out << ", " << "dbname=" << to_string(dbname); - out << ", " << "tablename="; (__isset.tablename ? (out << to_string(tablename)) : (out << "")); - out << ", " << "partname="; (__isset.partname ? (out << to_string(partname)) : (out << "")); - out << ", " << "state=" << to_string(state); - out << ", " << "type=" << to_string(type); - out << ", " << "txnid="; (__isset.txnid ? (out << to_string(txnid)) : (out << "")); - out << ", " << "lastheartbeat=" << to_string(lastheartbeat); - out << ", " << "acquiredat="; (__isset.acquiredat ? (out << to_string(acquiredat)) : (out << "")); - out << ", " << "user=" << to_string(user); - out << ", " << "hostname=" << to_string(hostname); - out << ", " << "heartbeatCount="; (__isset.heartbeatCount ? (out << to_string(heartbeatCount)) : (out << "")); - out << ", " << "agentInfo="; (__isset.agentInfo ? (out << to_string(agentInfo)) : (out << "")); - out << ", " << "blockedByExtId="; (__isset.blockedByExtId ? (out << to_string(blockedByExtId)) : (out << "")); - out << ", " << "blockedByIntId="; (__isset.blockedByIntId ? (out << to_string(blockedByIntId)) : (out << "")); - out << ", " << "lockIdInternal="; (__isset.lockIdInternal ? (out << to_string(lockIdInternal)) : (out << "")); - out << ")"; -} - - -ShowLocksResponse::~ShowLocksResponse() noexcept { -} - - -void ShowLocksResponse::__set_locks(const std::vector & val) { - this->locks = val; -} -std::ostream& operator<<(std::ostream& out, const ShowLocksResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ShowLocksResponse::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->locks.clear(); - uint32_t _size637; - ::apache::thrift::protocol::TType _etype640; - xfer += iprot->readListBegin(_etype640, _size637); - this->locks.resize(_size637); - uint32_t _i641; - for (_i641 = 0; _i641 < _size637; ++_i641) - { - xfer += this->locks[_i641].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.locks = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ShowLocksResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ShowLocksResponse"); - - xfer += oprot->writeFieldBegin("locks", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->locks.size())); - std::vector ::const_iterator _iter642; - for (_iter642 = this->locks.begin(); _iter642 != this->locks.end(); ++_iter642) - { - xfer += (*_iter642).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ShowLocksResponse &a, ShowLocksResponse &b) { - using ::std::swap; - swap(a.locks, b.locks); - swap(a.__isset, b.__isset); -} - -ShowLocksResponse::ShowLocksResponse(const ShowLocksResponse& other643) { - locks = other643.locks; - __isset = other643.__isset; -} -ShowLocksResponse& ShowLocksResponse::operator=(const ShowLocksResponse& other644) { - locks = other644.locks; - __isset = other644.__isset; - return *this; -} -void ShowLocksResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ShowLocksResponse("; - out << "locks=" << to_string(locks); - out << ")"; -} - - -HeartbeatRequest::~HeartbeatRequest() noexcept { -} - - -void HeartbeatRequest::__set_lockid(const int64_t val) { - this->lockid = val; -__isset.lockid = true; -} - -void HeartbeatRequest::__set_txnid(const int64_t val) { - this->txnid = val; -__isset.txnid = true; -} -std::ostream& operator<<(std::ostream& out, const HeartbeatRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t HeartbeatRequest::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->lockid); - this->__isset.lockid = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->txnid); - this->__isset.txnid = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t HeartbeatRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("HeartbeatRequest"); - - if (this->__isset.lockid) { - xfer += oprot->writeFieldBegin("lockid", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->lockid); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.txnid) { - xfer += oprot->writeFieldBegin("txnid", ::apache::thrift::protocol::T_I64, 2); - xfer += oprot->writeI64(this->txnid); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(HeartbeatRequest &a, HeartbeatRequest &b) { - using ::std::swap; - swap(a.lockid, b.lockid); - swap(a.txnid, b.txnid); - swap(a.__isset, b.__isset); -} - -HeartbeatRequest::HeartbeatRequest(const HeartbeatRequest& other645) { - lockid = other645.lockid; - txnid = other645.txnid; - __isset = other645.__isset; -} -HeartbeatRequest& HeartbeatRequest::operator=(const HeartbeatRequest& other646) { - lockid = other646.lockid; - txnid = other646.txnid; - __isset = other646.__isset; - return *this; -} -void HeartbeatRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "HeartbeatRequest("; - out << "lockid="; (__isset.lockid ? (out << to_string(lockid)) : (out << "")); - out << ", " << "txnid="; (__isset.txnid ? (out << to_string(txnid)) : (out << "")); - out << ")"; -} - - -HeartbeatTxnRangeRequest::~HeartbeatTxnRangeRequest() noexcept { -} - - -void HeartbeatTxnRangeRequest::__set_min(const int64_t val) { - this->min = val; -} - -void HeartbeatTxnRangeRequest::__set_max(const int64_t val) { - this->max = val; -} -std::ostream& operator<<(std::ostream& out, const HeartbeatTxnRangeRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t HeartbeatTxnRangeRequest::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_min = false; - bool isset_max = 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_I64) { - xfer += iprot->readI64(this->min); - isset_min = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->max); - isset_max = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_min) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_max) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t HeartbeatTxnRangeRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("HeartbeatTxnRangeRequest"); - - xfer += oprot->writeFieldBegin("min", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->min); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max", ::apache::thrift::protocol::T_I64, 2); - xfer += oprot->writeI64(this->max); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(HeartbeatTxnRangeRequest &a, HeartbeatTxnRangeRequest &b) { - using ::std::swap; - swap(a.min, b.min); - swap(a.max, b.max); -} - -HeartbeatTxnRangeRequest::HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest& other647) { - min = other647.min; - max = other647.max; -} -HeartbeatTxnRangeRequest& HeartbeatTxnRangeRequest::operator=(const HeartbeatTxnRangeRequest& other648) { - min = other648.min; - max = other648.max; - return *this; -} -void HeartbeatTxnRangeRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "HeartbeatTxnRangeRequest("; - out << "min=" << to_string(min); - out << ", " << "max=" << to_string(max); - out << ")"; -} - - -HeartbeatTxnRangeResponse::~HeartbeatTxnRangeResponse() noexcept { -} - - -void HeartbeatTxnRangeResponse::__set_aborted(const std::set & val) { - this->aborted = val; -} - -void HeartbeatTxnRangeResponse::__set_nosuch(const std::set & val) { - this->nosuch = val; -} -std::ostream& operator<<(std::ostream& out, const HeartbeatTxnRangeResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t HeartbeatTxnRangeResponse::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_aborted = false; - bool isset_nosuch = 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_SET) { - { - this->aborted.clear(); - uint32_t _size649; - ::apache::thrift::protocol::TType _etype652; - xfer += iprot->readSetBegin(_etype652, _size649); - uint32_t _i653; - for (_i653 = 0; _i653 < _size649; ++_i653) - { - int64_t _elem654; - xfer += iprot->readI64(_elem654); - this->aborted.insert(_elem654); - } - xfer += iprot->readSetEnd(); - } - isset_aborted = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_SET) { - { - this->nosuch.clear(); - uint32_t _size655; - ::apache::thrift::protocol::TType _etype658; - xfer += iprot->readSetBegin(_etype658, _size655); - uint32_t _i659; - for (_i659 = 0; _i659 < _size655; ++_i659) - { - int64_t _elem660; - xfer += iprot->readI64(_elem660); - this->nosuch.insert(_elem660); - } - xfer += iprot->readSetEnd(); - } - isset_nosuch = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_aborted) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_nosuch) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t HeartbeatTxnRangeResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("HeartbeatTxnRangeResponse"); - - xfer += oprot->writeFieldBegin("aborted", ::apache::thrift::protocol::T_SET, 1); - { - xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast(this->aborted.size())); - std::set ::const_iterator _iter661; - for (_iter661 = this->aborted.begin(); _iter661 != this->aborted.end(); ++_iter661) - { - xfer += oprot->writeI64((*_iter661)); - } - xfer += oprot->writeSetEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("nosuch", ::apache::thrift::protocol::T_SET, 2); - { - xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast(this->nosuch.size())); - std::set ::const_iterator _iter662; - for (_iter662 = this->nosuch.begin(); _iter662 != this->nosuch.end(); ++_iter662) - { - xfer += oprot->writeI64((*_iter662)); - } - xfer += oprot->writeSetEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(HeartbeatTxnRangeResponse &a, HeartbeatTxnRangeResponse &b) { - using ::std::swap; - swap(a.aborted, b.aborted); - swap(a.nosuch, b.nosuch); -} - -HeartbeatTxnRangeResponse::HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse& other663) { - aborted = other663.aborted; - nosuch = other663.nosuch; -} -HeartbeatTxnRangeResponse& HeartbeatTxnRangeResponse::operator=(const HeartbeatTxnRangeResponse& other664) { - aborted = other664.aborted; - nosuch = other664.nosuch; - return *this; -} -void HeartbeatTxnRangeResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "HeartbeatTxnRangeResponse("; - out << "aborted=" << to_string(aborted); - out << ", " << "nosuch=" << to_string(nosuch); - out << ")"; -} - - -CompactionRequest::~CompactionRequest() noexcept { -} - - -void CompactionRequest::__set_dbname(const std::string& val) { - this->dbname = val; -} - -void CompactionRequest::__set_tablename(const std::string& val) { - this->tablename = val; -} - -void CompactionRequest::__set_partitionname(const std::string& val) { - this->partitionname = val; -__isset.partitionname = true; -} - -void CompactionRequest::__set_type(const CompactionType::type val) { - this->type = val; -} - -void CompactionRequest::__set_runas(const std::string& val) { - this->runas = val; -__isset.runas = true; -} - -void CompactionRequest::__set_properties(const std::map & val) { - this->properties = val; -__isset.properties = true; -} -std::ostream& operator<<(std::ostream& out, const CompactionRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t CompactionRequest::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_dbname = false; - bool isset_tablename = false; - bool isset_type = 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_STRING) { - xfer += iprot->readString(this->dbname); - isset_dbname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tablename); - isset_tablename = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->partitionname); - this->__isset.partitionname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast665; - xfer += iprot->readI32(ecast665); - this->type = (CompactionType::type)ecast665; - isset_type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->runas); - this->__isset.runas = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_MAP) { - { - this->properties.clear(); - uint32_t _size666; - ::apache::thrift::protocol::TType _ktype667; - ::apache::thrift::protocol::TType _vtype668; - xfer += iprot->readMapBegin(_ktype667, _vtype668, _size666); - uint32_t _i670; - for (_i670 = 0; _i670 < _size666; ++_i670) - { - std::string _key671; - xfer += iprot->readString(_key671); - std::string& _val672 = this->properties[_key671]; - xfer += iprot->readString(_val672); - } - xfer += iprot->readMapEnd(); - } - this->__isset.properties = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_dbname) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tablename) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_type) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t CompactionRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("CompactionRequest"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbname); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tablename", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tablename); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.partitionname) { - xfer += oprot->writeFieldBegin("partitionname", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->partitionname); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32((int32_t)this->type); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.runas) { - xfer += oprot->writeFieldBegin("runas", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString(this->runas); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.properties) { - xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 6); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast(this->properties.size())); - std::map ::const_iterator _iter673; - for (_iter673 = this->properties.begin(); _iter673 != this->properties.end(); ++_iter673) - { - xfer += oprot->writeString(_iter673->first); - xfer += oprot->writeString(_iter673->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(CompactionRequest &a, CompactionRequest &b) { - using ::std::swap; - swap(a.dbname, b.dbname); - swap(a.tablename, b.tablename); - swap(a.partitionname, b.partitionname); - swap(a.type, b.type); - swap(a.runas, b.runas); - swap(a.properties, b.properties); - swap(a.__isset, b.__isset); -} - -CompactionRequest::CompactionRequest(const CompactionRequest& other674) { - dbname = other674.dbname; - tablename = other674.tablename; - partitionname = other674.partitionname; - type = other674.type; - runas = other674.runas; - properties = other674.properties; - __isset = other674.__isset; -} -CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other675) { - dbname = other675.dbname; - tablename = other675.tablename; - partitionname = other675.partitionname; - type = other675.type; - runas = other675.runas; - properties = other675.properties; - __isset = other675.__isset; - return *this; -} -void CompactionRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "CompactionRequest("; - out << "dbname=" << to_string(dbname); - out << ", " << "tablename=" << to_string(tablename); - out << ", " << "partitionname="; (__isset.partitionname ? (out << to_string(partitionname)) : (out << "")); - out << ", " << "type=" << to_string(type); - out << ", " << "runas="; (__isset.runas ? (out << to_string(runas)) : (out << "")); - out << ", " << "properties="; (__isset.properties ? (out << to_string(properties)) : (out << "")); - out << ")"; -} - - -CompactionResponse::~CompactionResponse() noexcept { -} - - -void CompactionResponse::__set_id(const int64_t val) { - this->id = val; -} - -void CompactionResponse::__set_state(const std::string& val) { - this->state = val; -} - -void CompactionResponse::__set_accepted(const bool val) { - this->accepted = val; -} -std::ostream& operator<<(std::ostream& out, const CompactionResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t CompactionResponse::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_id = false; - bool isset_state = false; - bool isset_accepted = 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_I64) { - xfer += iprot->readI64(this->id); - isset_id = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->state); - isset_state = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->accepted); - isset_accepted = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_id) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_state) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_accepted) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t CompactionResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("CompactionResponse"); - - xfer += oprot->writeFieldBegin("id", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->id); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("state", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->state); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("accepted", ::apache::thrift::protocol::T_BOOL, 3); - xfer += oprot->writeBool(this->accepted); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(CompactionResponse &a, CompactionResponse &b) { - using ::std::swap; - swap(a.id, b.id); - swap(a.state, b.state); - swap(a.accepted, b.accepted); -} - -CompactionResponse::CompactionResponse(const CompactionResponse& other676) { - id = other676.id; - state = other676.state; - accepted = other676.accepted; -} -CompactionResponse& CompactionResponse::operator=(const CompactionResponse& other677) { - id = other677.id; - state = other677.state; - accepted = other677.accepted; - return *this; -} -void CompactionResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "CompactionResponse("; - out << "id=" << to_string(id); - out << ", " << "state=" << to_string(state); - out << ", " << "accepted=" << to_string(accepted); - out << ")"; -} - - -ShowCompactRequest::~ShowCompactRequest() noexcept { -} - -std::ostream& operator<<(std::ostream& out, const ShowCompactRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ShowCompactRequest::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 ShowCompactRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ShowCompactRequest"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ShowCompactRequest &a, ShowCompactRequest &b) { - using ::std::swap; - (void) a; - (void) b; -} - -ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other678) { - (void) other678; -} -ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other679) { - (void) other679; - return *this; -} -void ShowCompactRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ShowCompactRequest("; - out << ")"; -} - - -ShowCompactResponseElement::~ShowCompactResponseElement() noexcept { -} - - -void ShowCompactResponseElement::__set_dbname(const std::string& val) { - this->dbname = val; -} - -void ShowCompactResponseElement::__set_tablename(const std::string& val) { - this->tablename = val; -} - -void ShowCompactResponseElement::__set_partitionname(const std::string& val) { - this->partitionname = val; -__isset.partitionname = true; -} - -void ShowCompactResponseElement::__set_type(const CompactionType::type val) { - this->type = val; -} - -void ShowCompactResponseElement::__set_state(const std::string& val) { - this->state = val; -} - -void ShowCompactResponseElement::__set_workerid(const std::string& val) { - this->workerid = val; -__isset.workerid = true; -} - -void ShowCompactResponseElement::__set_start(const int64_t val) { - this->start = val; -__isset.start = true; -} - -void ShowCompactResponseElement::__set_runAs(const std::string& val) { - this->runAs = val; -__isset.runAs = true; -} - -void ShowCompactResponseElement::__set_hightestTxnId(const int64_t val) { - this->hightestTxnId = val; -__isset.hightestTxnId = true; -} - -void ShowCompactResponseElement::__set_metaInfo(const std::string& val) { - this->metaInfo = val; -__isset.metaInfo = true; -} - -void ShowCompactResponseElement::__set_endTime(const int64_t val) { - this->endTime = val; -__isset.endTime = true; -} - -void ShowCompactResponseElement::__set_hadoopJobId(const std::string& val) { - this->hadoopJobId = val; -__isset.hadoopJobId = true; -} - -void ShowCompactResponseElement::__set_id(const int64_t val) { - this->id = val; -__isset.id = true; -} -std::ostream& operator<<(std::ostream& out, const ShowCompactResponseElement& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ShowCompactResponseElement::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_dbname = false; - bool isset_tablename = false; - bool isset_type = false; - bool isset_state = 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_STRING) { - xfer += iprot->readString(this->dbname); - isset_dbname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tablename); - isset_tablename = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->partitionname); - this->__isset.partitionname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast680; - xfer += iprot->readI32(ecast680); - this->type = (CompactionType::type)ecast680; - isset_type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->state); - isset_state = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->workerid); - this->__isset.workerid = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->start); - this->__isset.start = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->runAs); - this->__isset.runAs = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 9: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->hightestTxnId); - this->__isset.hightestTxnId = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 10: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->metaInfo); - this->__isset.metaInfo = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 11: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->endTime); - this->__isset.endTime = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 12: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->hadoopJobId); - this->__isset.hadoopJobId = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 13: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->id); - this->__isset.id = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_dbname) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tablename) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_type) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_state) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t ShowCompactResponseElement::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ShowCompactResponseElement"); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbname); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tablename", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tablename); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.partitionname) { - xfer += oprot->writeFieldBegin("partitionname", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->partitionname); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32((int32_t)this->type); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("state", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString(this->state); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.workerid) { - xfer += oprot->writeFieldBegin("workerid", ::apache::thrift::protocol::T_STRING, 6); - xfer += oprot->writeString(this->workerid); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.start) { - xfer += oprot->writeFieldBegin("start", ::apache::thrift::protocol::T_I64, 7); - xfer += oprot->writeI64(this->start); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.runAs) { - xfer += oprot->writeFieldBegin("runAs", ::apache::thrift::protocol::T_STRING, 8); - xfer += oprot->writeString(this->runAs); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.hightestTxnId) { - xfer += oprot->writeFieldBegin("hightestTxnId", ::apache::thrift::protocol::T_I64, 9); - xfer += oprot->writeI64(this->hightestTxnId); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.metaInfo) { - xfer += oprot->writeFieldBegin("metaInfo", ::apache::thrift::protocol::T_STRING, 10); - xfer += oprot->writeString(this->metaInfo); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.endTime) { - xfer += oprot->writeFieldBegin("endTime", ::apache::thrift::protocol::T_I64, 11); - xfer += oprot->writeI64(this->endTime); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.hadoopJobId) { - xfer += oprot->writeFieldBegin("hadoopJobId", ::apache::thrift::protocol::T_STRING, 12); - xfer += oprot->writeString(this->hadoopJobId); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.id) { - xfer += oprot->writeFieldBegin("id", ::apache::thrift::protocol::T_I64, 13); - xfer += oprot->writeI64(this->id); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ShowCompactResponseElement &a, ShowCompactResponseElement &b) { - using ::std::swap; - swap(a.dbname, b.dbname); - swap(a.tablename, b.tablename); - swap(a.partitionname, b.partitionname); - swap(a.type, b.type); - swap(a.state, b.state); - swap(a.workerid, b.workerid); - swap(a.start, b.start); - swap(a.runAs, b.runAs); - swap(a.hightestTxnId, b.hightestTxnId); - swap(a.metaInfo, b.metaInfo); - swap(a.endTime, b.endTime); - swap(a.hadoopJobId, b.hadoopJobId); - swap(a.id, b.id); - swap(a.__isset, b.__isset); -} - -ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other681) { - dbname = other681.dbname; - tablename = other681.tablename; - partitionname = other681.partitionname; - type = other681.type; - state = other681.state; - workerid = other681.workerid; - start = other681.start; - runAs = other681.runAs; - hightestTxnId = other681.hightestTxnId; - metaInfo = other681.metaInfo; - endTime = other681.endTime; - hadoopJobId = other681.hadoopJobId; - id = other681.id; - __isset = other681.__isset; -} -ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other682) { - dbname = other682.dbname; - tablename = other682.tablename; - partitionname = other682.partitionname; - type = other682.type; - state = other682.state; - workerid = other682.workerid; - start = other682.start; - runAs = other682.runAs; - hightestTxnId = other682.hightestTxnId; - metaInfo = other682.metaInfo; - endTime = other682.endTime; - hadoopJobId = other682.hadoopJobId; - id = other682.id; - __isset = other682.__isset; - return *this; -} -void ShowCompactResponseElement::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ShowCompactResponseElement("; - out << "dbname=" << to_string(dbname); - out << ", " << "tablename=" << to_string(tablename); - out << ", " << "partitionname="; (__isset.partitionname ? (out << to_string(partitionname)) : (out << "")); - out << ", " << "type=" << to_string(type); - out << ", " << "state=" << to_string(state); - out << ", " << "workerid="; (__isset.workerid ? (out << to_string(workerid)) : (out << "")); - out << ", " << "start="; (__isset.start ? (out << to_string(start)) : (out << "")); - out << ", " << "runAs="; (__isset.runAs ? (out << to_string(runAs)) : (out << "")); - out << ", " << "hightestTxnId="; (__isset.hightestTxnId ? (out << to_string(hightestTxnId)) : (out << "")); - out << ", " << "metaInfo="; (__isset.metaInfo ? (out << to_string(metaInfo)) : (out << "")); - out << ", " << "endTime="; (__isset.endTime ? (out << to_string(endTime)) : (out << "")); - out << ", " << "hadoopJobId="; (__isset.hadoopJobId ? (out << to_string(hadoopJobId)) : (out << "")); - out << ", " << "id="; (__isset.id ? (out << to_string(id)) : (out << "")); - out << ")"; -} - - -ShowCompactResponse::~ShowCompactResponse() noexcept { -} - - -void ShowCompactResponse::__set_compacts(const std::vector & val) { - this->compacts = val; -} -std::ostream& operator<<(std::ostream& out, const ShowCompactResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ShowCompactResponse::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_compacts = 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_LIST) { - { - this->compacts.clear(); - uint32_t _size683; - ::apache::thrift::protocol::TType _etype686; - xfer += iprot->readListBegin(_etype686, _size683); - this->compacts.resize(_size683); - uint32_t _i687; - for (_i687 = 0; _i687 < _size683; ++_i687) - { - xfer += this->compacts[_i687].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_compacts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_compacts) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t ShowCompactResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ShowCompactResponse"); - - xfer += oprot->writeFieldBegin("compacts", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->compacts.size())); - std::vector ::const_iterator _iter688; - for (_iter688 = this->compacts.begin(); _iter688 != this->compacts.end(); ++_iter688) - { - xfer += (*_iter688).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ShowCompactResponse &a, ShowCompactResponse &b) { - using ::std::swap; - swap(a.compacts, b.compacts); -} - -ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other689) { - compacts = other689.compacts; -} -ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other690) { - compacts = other690.compacts; - return *this; -} -void ShowCompactResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ShowCompactResponse("; - out << "compacts=" << to_string(compacts); - out << ")"; -} - - -AddDynamicPartitions::~AddDynamicPartitions() noexcept { -} - - -void AddDynamicPartitions::__set_txnid(const int64_t val) { - this->txnid = val; -} - -void AddDynamicPartitions::__set_dbname(const std::string& val) { - this->dbname = val; -} - -void AddDynamicPartitions::__set_tablename(const std::string& val) { - this->tablename = val; -} - -void AddDynamicPartitions::__set_partitionnames(const std::vector & val) { - this->partitionnames = val; -} - -void AddDynamicPartitions::__set_operationType(const DataOperationType::type val) { - this->operationType = val; -__isset.operationType = true; -} -std::ostream& operator<<(std::ostream& out, const AddDynamicPartitions& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t AddDynamicPartitions::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_txnid = false; - bool isset_dbname = false; - bool isset_tablename = false; - bool isset_partitionnames = 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_I64) { - xfer += iprot->readI64(this->txnid); - isset_txnid = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbname); - isset_dbname = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tablename); - isset_tablename = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->partitionnames.clear(); - uint32_t _size691; - ::apache::thrift::protocol::TType _etype694; - xfer += iprot->readListBegin(_etype694, _size691); - this->partitionnames.resize(_size691); - uint32_t _i695; - for (_i695 = 0; _i695 < _size691; ++_i695) - { - xfer += iprot->readString(this->partitionnames[_i695]); - } - xfer += iprot->readListEnd(); - } - isset_partitionnames = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast696; - xfer += iprot->readI32(ecast696); - this->operationType = (DataOperationType::type)ecast696; - this->__isset.operationType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_txnid) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_dbname) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tablename) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_partitionnames) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t AddDynamicPartitions::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("AddDynamicPartitions"); - - xfer += oprot->writeFieldBegin("txnid", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->txnid); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->dbname); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tablename", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->tablename); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("partitionnames", ::apache::thrift::protocol::T_LIST, 4); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partitionnames.size())); - std::vector ::const_iterator _iter697; - for (_iter697 = this->partitionnames.begin(); _iter697 != this->partitionnames.end(); ++_iter697) - { - xfer += oprot->writeString((*_iter697)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - if (this->__isset.operationType) { - xfer += oprot->writeFieldBegin("operationType", ::apache::thrift::protocol::T_I32, 5); - xfer += oprot->writeI32((int32_t)this->operationType); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(AddDynamicPartitions &a, AddDynamicPartitions &b) { - using ::std::swap; - swap(a.txnid, b.txnid); - swap(a.dbname, b.dbname); - swap(a.tablename, b.tablename); - swap(a.partitionnames, b.partitionnames); - swap(a.operationType, b.operationType); - swap(a.__isset, b.__isset); -} - -AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other698) { - txnid = other698.txnid; - dbname = other698.dbname; - tablename = other698.tablename; - partitionnames = other698.partitionnames; - operationType = other698.operationType; - __isset = other698.__isset; -} -AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other699) { - txnid = other699.txnid; - dbname = other699.dbname; - tablename = other699.tablename; - partitionnames = other699.partitionnames; - operationType = other699.operationType; - __isset = other699.__isset; - return *this; -} -void AddDynamicPartitions::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "AddDynamicPartitions("; - out << "txnid=" << to_string(txnid); - out << ", " << "dbname=" << to_string(dbname); - out << ", " << "tablename=" << to_string(tablename); - out << ", " << "partitionnames=" << to_string(partitionnames); - out << ", " << "operationType="; (__isset.operationType ? (out << to_string(operationType)) : (out << "")); - out << ")"; -} - - -NotificationEventRequest::~NotificationEventRequest() noexcept { -} - - -void NotificationEventRequest::__set_lastEvent(const int64_t val) { - this->lastEvent = val; -} - -void NotificationEventRequest::__set_maxEvents(const int32_t val) { - this->maxEvents = val; -__isset.maxEvents = true; -} -std::ostream& operator<<(std::ostream& out, const NotificationEventRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t NotificationEventRequest::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_lastEvent = 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_I64) { - xfer += iprot->readI64(this->lastEvent); - isset_lastEvent = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->maxEvents); - this->__isset.maxEvents = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_lastEvent) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t NotificationEventRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("NotificationEventRequest"); - - xfer += oprot->writeFieldBegin("lastEvent", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->lastEvent); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.maxEvents) { - xfer += oprot->writeFieldBegin("maxEvents", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32(this->maxEvents); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(NotificationEventRequest &a, NotificationEventRequest &b) { - using ::std::swap; - swap(a.lastEvent, b.lastEvent); - swap(a.maxEvents, b.maxEvents); - swap(a.__isset, b.__isset); -} - -NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other700) { - lastEvent = other700.lastEvent; - maxEvents = other700.maxEvents; - __isset = other700.__isset; -} -NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other701) { - lastEvent = other701.lastEvent; - maxEvents = other701.maxEvents; - __isset = other701.__isset; - return *this; -} -void NotificationEventRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "NotificationEventRequest("; - out << "lastEvent=" << to_string(lastEvent); - out << ", " << "maxEvents="; (__isset.maxEvents ? (out << to_string(maxEvents)) : (out << "")); - out << ")"; -} - - -NotificationEvent::~NotificationEvent() noexcept { -} - - -void NotificationEvent::__set_eventId(const int64_t val) { - this->eventId = val; -} - -void NotificationEvent::__set_eventTime(const int32_t val) { - this->eventTime = val; -} - -void NotificationEvent::__set_eventType(const std::string& val) { - this->eventType = val; -} - -void NotificationEvent::__set_dbName(const std::string& val) { - this->dbName = val; -__isset.dbName = true; -} - -void NotificationEvent::__set_tableName(const std::string& val) { - this->tableName = val; -__isset.tableName = true; -} - -void NotificationEvent::__set_message(const std::string& val) { - this->message = val; -} - -void NotificationEvent::__set_messageFormat(const std::string& val) { - this->messageFormat = val; -__isset.messageFormat = true; -} -std::ostream& operator<<(std::ostream& out, const NotificationEvent& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t NotificationEvent::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_eventId = false; - bool isset_eventTime = false; - bool isset_eventType = false; - bool isset_message = 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_I64) { - xfer += iprot->readI64(this->eventId); - isset_eventId = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->eventTime); - isset_eventTime = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->eventType); - isset_eventType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbName); - this->__isset.dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tableName); - this->__isset.tableName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - isset_message = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->messageFormat); - this->__isset.messageFormat = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_eventId) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_eventTime) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_eventType) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_message) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t NotificationEvent::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("NotificationEvent"); - - xfer += oprot->writeFieldBegin("eventId", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->eventId); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("eventTime", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32(this->eventTime); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("eventType", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->eventType); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.dbName) { - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.tableName) { - xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString(this->tableName); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 6); - xfer += oprot->writeString(this->message); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.messageFormat) { - xfer += oprot->writeFieldBegin("messageFormat", ::apache::thrift::protocol::T_STRING, 7); - xfer += oprot->writeString(this->messageFormat); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(NotificationEvent &a, NotificationEvent &b) { - using ::std::swap; - swap(a.eventId, b.eventId); - swap(a.eventTime, b.eventTime); - swap(a.eventType, b.eventType); - swap(a.dbName, b.dbName); - swap(a.tableName, b.tableName); - swap(a.message, b.message); - swap(a.messageFormat, b.messageFormat); - swap(a.__isset, b.__isset); -} - -NotificationEvent::NotificationEvent(const NotificationEvent& other702) { - eventId = other702.eventId; - eventTime = other702.eventTime; - eventType = other702.eventType; - dbName = other702.dbName; - tableName = other702.tableName; - message = other702.message; - messageFormat = other702.messageFormat; - __isset = other702.__isset; -} -NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other703) { - eventId = other703.eventId; - eventTime = other703.eventTime; - eventType = other703.eventType; - dbName = other703.dbName; - tableName = other703.tableName; - message = other703.message; - messageFormat = other703.messageFormat; - __isset = other703.__isset; - return *this; -} -void NotificationEvent::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "NotificationEvent("; - out << "eventId=" << to_string(eventId); - out << ", " << "eventTime=" << to_string(eventTime); - out << ", " << "eventType=" << to_string(eventType); - out << ", " << "dbName="; (__isset.dbName ? (out << to_string(dbName)) : (out << "")); - out << ", " << "tableName="; (__isset.tableName ? (out << to_string(tableName)) : (out << "")); - out << ", " << "message=" << to_string(message); - out << ", " << "messageFormat="; (__isset.messageFormat ? (out << to_string(messageFormat)) : (out << "")); - out << ")"; -} - - -NotificationEventResponse::~NotificationEventResponse() noexcept { -} - - -void NotificationEventResponse::__set_events(const std::vector & val) { - this->events = val; -} -std::ostream& operator<<(std::ostream& out, const NotificationEventResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t NotificationEventResponse::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_events = 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_LIST) { - { - this->events.clear(); - uint32_t _size704; - ::apache::thrift::protocol::TType _etype707; - xfer += iprot->readListBegin(_etype707, _size704); - this->events.resize(_size704); - uint32_t _i708; - for (_i708 = 0; _i708 < _size704; ++_i708) - { - xfer += this->events[_i708].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_events = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_events) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t NotificationEventResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("NotificationEventResponse"); - - xfer += oprot->writeFieldBegin("events", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->events.size())); - std::vector ::const_iterator _iter709; - for (_iter709 = this->events.begin(); _iter709 != this->events.end(); ++_iter709) - { - xfer += (*_iter709).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(NotificationEventResponse &a, NotificationEventResponse &b) { - using ::std::swap; - swap(a.events, b.events); -} - -NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other710) { - events = other710.events; -} -NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other711) { - events = other711.events; - return *this; -} -void NotificationEventResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "NotificationEventResponse("; - out << "events=" << to_string(events); - out << ")"; -} - - -CurrentNotificationEventId::~CurrentNotificationEventId() noexcept { -} - - -void CurrentNotificationEventId::__set_eventId(const int64_t val) { - this->eventId = val; -} -std::ostream& operator<<(std::ostream& out, const CurrentNotificationEventId& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t CurrentNotificationEventId::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_eventId = 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_I64) { - xfer += iprot->readI64(this->eventId); - isset_eventId = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_eventId) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t CurrentNotificationEventId::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("CurrentNotificationEventId"); - - xfer += oprot->writeFieldBegin("eventId", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->eventId); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(CurrentNotificationEventId &a, CurrentNotificationEventId &b) { - using ::std::swap; - swap(a.eventId, b.eventId); -} - -CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other712) { - eventId = other712.eventId; -} -CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other713) { - eventId = other713.eventId; - return *this; -} -void CurrentNotificationEventId::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "CurrentNotificationEventId("; - out << "eventId=" << to_string(eventId); - out << ")"; -} - - -NotificationEventsCountRequest::~NotificationEventsCountRequest() noexcept { -} - - -void NotificationEventsCountRequest::__set_fromEventId(const int64_t val) { - this->fromEventId = val; -} - -void NotificationEventsCountRequest::__set_dbName(const std::string& val) { - this->dbName = val; -} -std::ostream& operator<<(std::ostream& out, const NotificationEventsCountRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t NotificationEventsCountRequest::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_fromEventId = false; - bool isset_dbName = 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_I64) { - xfer += iprot->readI64(this->fromEventId); - isset_fromEventId = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbName); - isset_dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_fromEventId) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_dbName) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t NotificationEventsCountRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("NotificationEventsCountRequest"); - - xfer += oprot->writeFieldBegin("fromEventId", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->fromEventId); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(NotificationEventsCountRequest &a, NotificationEventsCountRequest &b) { - using ::std::swap; - swap(a.fromEventId, b.fromEventId); - swap(a.dbName, b.dbName); -} - -NotificationEventsCountRequest::NotificationEventsCountRequest(const NotificationEventsCountRequest& other714) { - fromEventId = other714.fromEventId; - dbName = other714.dbName; -} -NotificationEventsCountRequest& NotificationEventsCountRequest::operator=(const NotificationEventsCountRequest& other715) { - fromEventId = other715.fromEventId; - dbName = other715.dbName; - return *this; -} -void NotificationEventsCountRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "NotificationEventsCountRequest("; - out << "fromEventId=" << to_string(fromEventId); - out << ", " << "dbName=" << to_string(dbName); - out << ")"; -} - - -NotificationEventsCountResponse::~NotificationEventsCountResponse() noexcept { -} - - -void NotificationEventsCountResponse::__set_eventsCount(const int64_t val) { - this->eventsCount = val; -} -std::ostream& operator<<(std::ostream& out, const NotificationEventsCountResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t NotificationEventsCountResponse::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_eventsCount = 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_I64) { - xfer += iprot->readI64(this->eventsCount); - isset_eventsCount = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_eventsCount) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t NotificationEventsCountResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("NotificationEventsCountResponse"); - - xfer += oprot->writeFieldBegin("eventsCount", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->eventsCount); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(NotificationEventsCountResponse &a, NotificationEventsCountResponse &b) { - using ::std::swap; - swap(a.eventsCount, b.eventsCount); -} - -NotificationEventsCountResponse::NotificationEventsCountResponse(const NotificationEventsCountResponse& other716) { - eventsCount = other716.eventsCount; -} -NotificationEventsCountResponse& NotificationEventsCountResponse::operator=(const NotificationEventsCountResponse& other717) { - eventsCount = other717.eventsCount; - return *this; -} -void NotificationEventsCountResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "NotificationEventsCountResponse("; - out << "eventsCount=" << to_string(eventsCount); - out << ")"; -} - - -InsertEventRequestData::~InsertEventRequestData() noexcept { -} - - -void InsertEventRequestData::__set_replace(const bool val) { - this->replace = val; -__isset.replace = true; -} - -void InsertEventRequestData::__set_filesAdded(const std::vector & val) { - this->filesAdded = val; -} - -void InsertEventRequestData::__set_filesAddedChecksum(const std::vector & val) { - this->filesAddedChecksum = val; -__isset.filesAddedChecksum = true; -} -std::ostream& operator<<(std::ostream& out, const InsertEventRequestData& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t InsertEventRequestData::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_filesAdded = 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_BOOL) { - xfer += iprot->readBool(this->replace); - this->__isset.replace = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->filesAdded.clear(); - uint32_t _size718; - ::apache::thrift::protocol::TType _etype721; - xfer += iprot->readListBegin(_etype721, _size718); - this->filesAdded.resize(_size718); - uint32_t _i722; - for (_i722 = 0; _i722 < _size718; ++_i722) - { - xfer += iprot->readString(this->filesAdded[_i722]); - } - xfer += iprot->readListEnd(); - } - isset_filesAdded = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->filesAddedChecksum.clear(); - uint32_t _size723; - ::apache::thrift::protocol::TType _etype726; - xfer += iprot->readListBegin(_etype726, _size723); - this->filesAddedChecksum.resize(_size723); - uint32_t _i727; - for (_i727 = 0; _i727 < _size723; ++_i727) - { - xfer += iprot->readString(this->filesAddedChecksum[_i727]); - } - xfer += iprot->readListEnd(); - } - this->__isset.filesAddedChecksum = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_filesAdded) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("InsertEventRequestData"); - - if (this->__isset.replace) { - xfer += oprot->writeFieldBegin("replace", ::apache::thrift::protocol::T_BOOL, 1); - xfer += oprot->writeBool(this->replace); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldBegin("filesAdded", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->filesAdded.size())); - std::vector ::const_iterator _iter728; - for (_iter728 = this->filesAdded.begin(); _iter728 != this->filesAdded.end(); ++_iter728) - { - xfer += oprot->writeString((*_iter728)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - if (this->__isset.filesAddedChecksum) { - xfer += oprot->writeFieldBegin("filesAddedChecksum", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->filesAddedChecksum.size())); - std::vector ::const_iterator _iter729; - for (_iter729 = this->filesAddedChecksum.begin(); _iter729 != this->filesAddedChecksum.end(); ++_iter729) - { - xfer += oprot->writeString((*_iter729)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(InsertEventRequestData &a, InsertEventRequestData &b) { - using ::std::swap; - swap(a.replace, b.replace); - swap(a.filesAdded, b.filesAdded); - swap(a.filesAddedChecksum, b.filesAddedChecksum); - swap(a.__isset, b.__isset); -} - -InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other730) { - replace = other730.replace; - filesAdded = other730.filesAdded; - filesAddedChecksum = other730.filesAddedChecksum; - __isset = other730.__isset; -} -InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other731) { - replace = other731.replace; - filesAdded = other731.filesAdded; - filesAddedChecksum = other731.filesAddedChecksum; - __isset = other731.__isset; - return *this; -} -void InsertEventRequestData::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "InsertEventRequestData("; - out << "replace="; (__isset.replace ? (out << to_string(replace)) : (out << "")); - out << ", " << "filesAdded=" << to_string(filesAdded); - out << ", " << "filesAddedChecksum="; (__isset.filesAddedChecksum ? (out << to_string(filesAddedChecksum)) : (out << "")); - out << ")"; -} - - -FireEventRequestData::~FireEventRequestData() noexcept { -} - - -void FireEventRequestData::__set_insertData(const InsertEventRequestData& val) { - this->insertData = val; -__isset.insertData = true; -} -std::ostream& operator<<(std::ostream& out, const FireEventRequestData& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t FireEventRequestData::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->insertData.read(iprot); - this->__isset.insertData = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t FireEventRequestData::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("FireEventRequestData"); - - if (this->__isset.insertData) { - xfer += oprot->writeFieldBegin("insertData", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->insertData.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(FireEventRequestData &a, FireEventRequestData &b) { - using ::std::swap; - swap(a.insertData, b.insertData); - swap(a.__isset, b.__isset); -} - -FireEventRequestData::FireEventRequestData(const FireEventRequestData& other732) { - insertData = other732.insertData; - __isset = other732.__isset; -} -FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other733) { - insertData = other733.insertData; - __isset = other733.__isset; - return *this; -} -void FireEventRequestData::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "FireEventRequestData("; - out << "insertData="; (__isset.insertData ? (out << to_string(insertData)) : (out << "")); - out << ")"; -} - - -FireEventRequest::~FireEventRequest() noexcept { -} - - -void FireEventRequest::__set_successful(const bool val) { - this->successful = val; -} - -void FireEventRequest::__set_data(const FireEventRequestData& val) { - this->data = val; -} - -void FireEventRequest::__set_dbName(const std::string& val) { - this->dbName = val; -__isset.dbName = true; -} - -void FireEventRequest::__set_tableName(const std::string& val) { - this->tableName = val; -__isset.tableName = true; -} - -void FireEventRequest::__set_partitionVals(const std::vector & val) { - this->partitionVals = val; -__isset.partitionVals = true; -} -std::ostream& operator<<(std::ostream& out, const FireEventRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t FireEventRequest::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_successful = false; - bool isset_data = 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_BOOL) { - xfer += iprot->readBool(this->successful); - isset_successful = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->data.read(iprot); - isset_data = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->dbName); - this->__isset.dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tableName); - this->__isset.tableName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->partitionVals.clear(); - uint32_t _size734; - ::apache::thrift::protocol::TType _etype737; - xfer += iprot->readListBegin(_etype737, _size734); - this->partitionVals.resize(_size734); - uint32_t _i738; - for (_i738 = 0; _i738 < _size734; ++_i738) - { - xfer += iprot->readString(this->partitionVals[_i738]); - } - xfer += iprot->readListEnd(); - } - this->__isset.partitionVals = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_successful) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_data) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t FireEventRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("FireEventRequest"); - - xfer += oprot->writeFieldBegin("successful", ::apache::thrift::protocol::T_BOOL, 1); - xfer += oprot->writeBool(this->successful); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("data", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->data.write(oprot); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.dbName) { - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.tableName) { - xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->tableName); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.partitionVals) { - xfer += oprot->writeFieldBegin("partitionVals", ::apache::thrift::protocol::T_LIST, 5); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->partitionVals.size())); - std::vector ::const_iterator _iter739; - for (_iter739 = this->partitionVals.begin(); _iter739 != this->partitionVals.end(); ++_iter739) - { - xfer += oprot->writeString((*_iter739)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(FireEventRequest &a, FireEventRequest &b) { - using ::std::swap; - swap(a.successful, b.successful); - swap(a.data, b.data); - swap(a.dbName, b.dbName); - swap(a.tableName, b.tableName); - swap(a.partitionVals, b.partitionVals); - swap(a.__isset, b.__isset); -} - -FireEventRequest::FireEventRequest(const FireEventRequest& other740) { - successful = other740.successful; - data = other740.data; - dbName = other740.dbName; - tableName = other740.tableName; - partitionVals = other740.partitionVals; - __isset = other740.__isset; -} -FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other741) { - successful = other741.successful; - data = other741.data; - dbName = other741.dbName; - tableName = other741.tableName; - partitionVals = other741.partitionVals; - __isset = other741.__isset; - return *this; -} -void FireEventRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "FireEventRequest("; - out << "successful=" << to_string(successful); - out << ", " << "data=" << to_string(data); - out << ", " << "dbName="; (__isset.dbName ? (out << to_string(dbName)) : (out << "")); - out << ", " << "tableName="; (__isset.tableName ? (out << to_string(tableName)) : (out << "")); - out << ", " << "partitionVals="; (__isset.partitionVals ? (out << to_string(partitionVals)) : (out << "")); - out << ")"; -} - - -FireEventResponse::~FireEventResponse() noexcept { -} - -std::ostream& operator<<(std::ostream& out, const FireEventResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t FireEventResponse::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 FireEventResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("FireEventResponse"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(FireEventResponse &a, FireEventResponse &b) { - using ::std::swap; - (void) a; - (void) b; -} - -FireEventResponse::FireEventResponse(const FireEventResponse& other742) { - (void) other742; -} -FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other743) { - (void) other743; - return *this; -} -void FireEventResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "FireEventResponse("; - out << ")"; -} - - -MetadataPpdResult::~MetadataPpdResult() noexcept { -} - - -void MetadataPpdResult::__set_metadata(const std::string& val) { - this->metadata = val; -__isset.metadata = true; -} - -void MetadataPpdResult::__set_includeBitset(const std::string& val) { - this->includeBitset = val; -__isset.includeBitset = true; -} -std::ostream& operator<<(std::ostream& out, const MetadataPpdResult& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t MetadataPpdResult::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->metadata); - this->__isset.metadata = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->includeBitset); - this->__isset.includeBitset = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t MetadataPpdResult::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("MetadataPpdResult"); - - if (this->__isset.metadata) { - xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeBinary(this->metadata); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.includeBitset) { - xfer += oprot->writeFieldBegin("includeBitset", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeBinary(this->includeBitset); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(MetadataPpdResult &a, MetadataPpdResult &b) { - using ::std::swap; - swap(a.metadata, b.metadata); - swap(a.includeBitset, b.includeBitset); - swap(a.__isset, b.__isset); -} - -MetadataPpdResult::MetadataPpdResult(const MetadataPpdResult& other744) { - metadata = other744.metadata; - includeBitset = other744.includeBitset; - __isset = other744.__isset; -} -MetadataPpdResult& MetadataPpdResult::operator=(const MetadataPpdResult& other745) { - metadata = other745.metadata; - includeBitset = other745.includeBitset; - __isset = other745.__isset; - return *this; -} -void MetadataPpdResult::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "MetadataPpdResult("; - out << "metadata="; (__isset.metadata ? (out << to_string(metadata)) : (out << "")); - out << ", " << "includeBitset="; (__isset.includeBitset ? (out << to_string(includeBitset)) : (out << "")); - out << ")"; -} - - -GetFileMetadataByExprResult::~GetFileMetadataByExprResult() noexcept { -} - - -void GetFileMetadataByExprResult::__set_metadata(const std::map & val) { - this->metadata = val; -} - -void GetFileMetadataByExprResult::__set_isSupported(const bool val) { - this->isSupported = val; -} -std::ostream& operator<<(std::ostream& out, const GetFileMetadataByExprResult& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GetFileMetadataByExprResult::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_metadata = false; - bool isset_isSupported = 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_MAP) { - { - this->metadata.clear(); - uint32_t _size746; - ::apache::thrift::protocol::TType _ktype747; - ::apache::thrift::protocol::TType _vtype748; - xfer += iprot->readMapBegin(_ktype747, _vtype748, _size746); - uint32_t _i750; - for (_i750 = 0; _i750 < _size746; ++_i750) - { - int64_t _key751; - xfer += iprot->readI64(_key751); - MetadataPpdResult& _val752 = this->metadata[_key751]; - xfer += _val752.read(iprot); - } - xfer += iprot->readMapEnd(); - } - isset_metadata = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->isSupported); - isset_isSupported = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_metadata) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_isSupported) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t GetFileMetadataByExprResult::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GetFileMetadataByExprResult"); - - xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_MAP, 1); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_I64, ::apache::thrift::protocol::T_STRUCT, static_cast(this->metadata.size())); - std::map ::const_iterator _iter753; - for (_iter753 = this->metadata.begin(); _iter753 != this->metadata.end(); ++_iter753) - { - xfer += oprot->writeI64(_iter753->first); - xfer += _iter753->second.write(oprot); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("isSupported", ::apache::thrift::protocol::T_BOOL, 2); - xfer += oprot->writeBool(this->isSupported); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GetFileMetadataByExprResult &a, GetFileMetadataByExprResult &b) { - using ::std::swap; - swap(a.metadata, b.metadata); - swap(a.isSupported, b.isSupported); -} - -GetFileMetadataByExprResult::GetFileMetadataByExprResult(const GetFileMetadataByExprResult& other754) { - metadata = other754.metadata; - isSupported = other754.isSupported; -} -GetFileMetadataByExprResult& GetFileMetadataByExprResult::operator=(const GetFileMetadataByExprResult& other755) { - metadata = other755.metadata; - isSupported = other755.isSupported; - return *this; -} -void GetFileMetadataByExprResult::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GetFileMetadataByExprResult("; - out << "metadata=" << to_string(metadata); - out << ", " << "isSupported=" << to_string(isSupported); - out << ")"; -} - - -GetFileMetadataByExprRequest::~GetFileMetadataByExprRequest() noexcept { -} - - -void GetFileMetadataByExprRequest::__set_fileIds(const std::vector & val) { - this->fileIds = val; -} - -void GetFileMetadataByExprRequest::__set_expr(const std::string& val) { - this->expr = val; -} - -void GetFileMetadataByExprRequest::__set_doGetFooters(const bool val) { - this->doGetFooters = val; -__isset.doGetFooters = true; -} - -void GetFileMetadataByExprRequest::__set_type(const FileMetadataExprType::type val) { - this->type = val; -__isset.type = true; -} -std::ostream& operator<<(std::ostream& out, const GetFileMetadataByExprRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GetFileMetadataByExprRequest::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_fileIds = false; - bool isset_expr = 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_LIST) { - { - this->fileIds.clear(); - uint32_t _size756; - ::apache::thrift::protocol::TType _etype759; - xfer += iprot->readListBegin(_etype759, _size756); - this->fileIds.resize(_size756); - uint32_t _i760; - for (_i760 = 0; _i760 < _size756; ++_i760) - { - xfer += iprot->readI64(this->fileIds[_i760]); - } - xfer += iprot->readListEnd(); - } - isset_fileIds = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->expr); - isset_expr = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->doGetFooters); - this->__isset.doGetFooters = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast761; - xfer += iprot->readI32(ecast761); - this->type = (FileMetadataExprType::type)ecast761; - this->__isset.type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_fileIds) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_expr) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t GetFileMetadataByExprRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GetFileMetadataByExprRequest"); - - xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->fileIds.size())); - std::vector ::const_iterator _iter762; - for (_iter762 = this->fileIds.begin(); _iter762 != this->fileIds.end(); ++_iter762) - { - xfer += oprot->writeI64((*_iter762)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("expr", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeBinary(this->expr); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.doGetFooters) { - xfer += oprot->writeFieldBegin("doGetFooters", ::apache::thrift::protocol::T_BOOL, 3); - xfer += oprot->writeBool(this->doGetFooters); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.type) { - xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32((int32_t)this->type); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GetFileMetadataByExprRequest &a, GetFileMetadataByExprRequest &b) { - using ::std::swap; - swap(a.fileIds, b.fileIds); - swap(a.expr, b.expr); - swap(a.doGetFooters, b.doGetFooters); - swap(a.type, b.type); - swap(a.__isset, b.__isset); -} - -GetFileMetadataByExprRequest::GetFileMetadataByExprRequest(const GetFileMetadataByExprRequest& other763) { - fileIds = other763.fileIds; - expr = other763.expr; - doGetFooters = other763.doGetFooters; - type = other763.type; - __isset = other763.__isset; -} -GetFileMetadataByExprRequest& GetFileMetadataByExprRequest::operator=(const GetFileMetadataByExprRequest& other764) { - fileIds = other764.fileIds; - expr = other764.expr; - doGetFooters = other764.doGetFooters; - type = other764.type; - __isset = other764.__isset; - return *this; -} -void GetFileMetadataByExprRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GetFileMetadataByExprRequest("; - out << "fileIds=" << to_string(fileIds); - out << ", " << "expr=" << to_string(expr); - out << ", " << "doGetFooters="; (__isset.doGetFooters ? (out << to_string(doGetFooters)) : (out << "")); - out << ", " << "type="; (__isset.type ? (out << to_string(type)) : (out << "")); - out << ")"; -} - - -GetFileMetadataResult::~GetFileMetadataResult() noexcept { -} - - -void GetFileMetadataResult::__set_metadata(const std::map & val) { - this->metadata = val; -} - -void GetFileMetadataResult::__set_isSupported(const bool val) { - this->isSupported = val; -} -std::ostream& operator<<(std::ostream& out, const GetFileMetadataResult& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GetFileMetadataResult::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_metadata = false; - bool isset_isSupported = 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_MAP) { - { - this->metadata.clear(); - uint32_t _size765; - ::apache::thrift::protocol::TType _ktype766; - ::apache::thrift::protocol::TType _vtype767; - xfer += iprot->readMapBegin(_ktype766, _vtype767, _size765); - uint32_t _i769; - for (_i769 = 0; _i769 < _size765; ++_i769) - { - int64_t _key770; - xfer += iprot->readI64(_key770); - std::string& _val771 = this->metadata[_key770]; - xfer += iprot->readBinary(_val771); - } - xfer += iprot->readMapEnd(); - } - isset_metadata = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->isSupported); - isset_isSupported = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_metadata) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_isSupported) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t GetFileMetadataResult::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GetFileMetadataResult"); - - xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_MAP, 1); - { - xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_I64, ::apache::thrift::protocol::T_STRING, static_cast(this->metadata.size())); - std::map ::const_iterator _iter772; - for (_iter772 = this->metadata.begin(); _iter772 != this->metadata.end(); ++_iter772) - { - xfer += oprot->writeI64(_iter772->first); - xfer += oprot->writeBinary(_iter772->second); - } - xfer += oprot->writeMapEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("isSupported", ::apache::thrift::protocol::T_BOOL, 2); - xfer += oprot->writeBool(this->isSupported); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GetFileMetadataResult &a, GetFileMetadataResult &b) { - using ::std::swap; - swap(a.metadata, b.metadata); - swap(a.isSupported, b.isSupported); -} - -GetFileMetadataResult::GetFileMetadataResult(const GetFileMetadataResult& other773) { - metadata = other773.metadata; - isSupported = other773.isSupported; -} -GetFileMetadataResult& GetFileMetadataResult::operator=(const GetFileMetadataResult& other774) { - metadata = other774.metadata; - isSupported = other774.isSupported; - return *this; -} -void GetFileMetadataResult::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GetFileMetadataResult("; - out << "metadata=" << to_string(metadata); - out << ", " << "isSupported=" << to_string(isSupported); - out << ")"; -} - - -GetFileMetadataRequest::~GetFileMetadataRequest() noexcept { -} - - -void GetFileMetadataRequest::__set_fileIds(const std::vector & val) { - this->fileIds = val; -} -std::ostream& operator<<(std::ostream& out, const GetFileMetadataRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GetFileMetadataRequest::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_fileIds = 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_LIST) { - { - this->fileIds.clear(); - uint32_t _size775; - ::apache::thrift::protocol::TType _etype778; - xfer += iprot->readListBegin(_etype778, _size775); - this->fileIds.resize(_size775); - uint32_t _i779; - for (_i779 = 0; _i779 < _size775; ++_i779) - { - xfer += iprot->readI64(this->fileIds[_i779]); - } - xfer += iprot->readListEnd(); - } - isset_fileIds = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_fileIds) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t GetFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GetFileMetadataRequest"); - - xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->fileIds.size())); - std::vector ::const_iterator _iter780; - for (_iter780 = this->fileIds.begin(); _iter780 != this->fileIds.end(); ++_iter780) - { - xfer += oprot->writeI64((*_iter780)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GetFileMetadataRequest &a, GetFileMetadataRequest &b) { - using ::std::swap; - swap(a.fileIds, b.fileIds); -} - -GetFileMetadataRequest::GetFileMetadataRequest(const GetFileMetadataRequest& other781) { - fileIds = other781.fileIds; -} -GetFileMetadataRequest& GetFileMetadataRequest::operator=(const GetFileMetadataRequest& other782) { - fileIds = other782.fileIds; - return *this; -} -void GetFileMetadataRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GetFileMetadataRequest("; - out << "fileIds=" << to_string(fileIds); - out << ")"; -} - - -PutFileMetadataResult::~PutFileMetadataResult() noexcept { -} - -std::ostream& operator<<(std::ostream& out, const PutFileMetadataResult& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PutFileMetadataResult::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 PutFileMetadataResult::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PutFileMetadataResult"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PutFileMetadataResult &a, PutFileMetadataResult &b) { - using ::std::swap; - (void) a; - (void) b; -} - -PutFileMetadataResult::PutFileMetadataResult(const PutFileMetadataResult& other783) { - (void) other783; -} -PutFileMetadataResult& PutFileMetadataResult::operator=(const PutFileMetadataResult& other784) { - (void) other784; - return *this; -} -void PutFileMetadataResult::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PutFileMetadataResult("; - out << ")"; -} - - -PutFileMetadataRequest::~PutFileMetadataRequest() noexcept { -} - - -void PutFileMetadataRequest::__set_fileIds(const std::vector & val) { - this->fileIds = val; -} - -void PutFileMetadataRequest::__set_metadata(const std::vector & val) { - this->metadata = val; -} - -void PutFileMetadataRequest::__set_type(const FileMetadataExprType::type val) { - this->type = val; -__isset.type = true; -} -std::ostream& operator<<(std::ostream& out, const PutFileMetadataRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PutFileMetadataRequest::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_fileIds = false; - bool isset_metadata = 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_LIST) { - { - this->fileIds.clear(); - uint32_t _size785; - ::apache::thrift::protocol::TType _etype788; - xfer += iprot->readListBegin(_etype788, _size785); - this->fileIds.resize(_size785); - uint32_t _i789; - for (_i789 = 0; _i789 < _size785; ++_i789) - { - xfer += iprot->readI64(this->fileIds[_i789]); - } - xfer += iprot->readListEnd(); - } - isset_fileIds = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->metadata.clear(); - uint32_t _size790; - ::apache::thrift::protocol::TType _etype793; - xfer += iprot->readListBegin(_etype793, _size790); - this->metadata.resize(_size790); - uint32_t _i794; - for (_i794 = 0; _i794 < _size790; ++_i794) - { - xfer += iprot->readBinary(this->metadata[_i794]); - } - xfer += iprot->readListEnd(); - } - isset_metadata = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast795; - xfer += iprot->readI32(ecast795); - this->type = (FileMetadataExprType::type)ecast795; - this->__isset.type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_fileIds) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_metadata) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t PutFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PutFileMetadataRequest"); - - xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->fileIds.size())); - std::vector ::const_iterator _iter796; - for (_iter796 = this->fileIds.begin(); _iter796 != this->fileIds.end(); ++_iter796) - { - xfer += oprot->writeI64((*_iter796)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->metadata.size())); - std::vector ::const_iterator _iter797; - for (_iter797 = this->metadata.begin(); _iter797 != this->metadata.end(); ++_iter797) - { - xfer += oprot->writeBinary((*_iter797)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - if (this->__isset.type) { - xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32((int32_t)this->type); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PutFileMetadataRequest &a, PutFileMetadataRequest &b) { - using ::std::swap; - swap(a.fileIds, b.fileIds); - swap(a.metadata, b.metadata); - swap(a.type, b.type); - swap(a.__isset, b.__isset); -} - -PutFileMetadataRequest::PutFileMetadataRequest(const PutFileMetadataRequest& other798) { - fileIds = other798.fileIds; - metadata = other798.metadata; - type = other798.type; - __isset = other798.__isset; -} -PutFileMetadataRequest& PutFileMetadataRequest::operator=(const PutFileMetadataRequest& other799) { - fileIds = other799.fileIds; - metadata = other799.metadata; - type = other799.type; - __isset = other799.__isset; - return *this; -} -void PutFileMetadataRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PutFileMetadataRequest("; - out << "fileIds=" << to_string(fileIds); - out << ", " << "metadata=" << to_string(metadata); - out << ", " << "type="; (__isset.type ? (out << to_string(type)) : (out << "")); - out << ")"; -} - - -ClearFileMetadataResult::~ClearFileMetadataResult() noexcept { -} - -std::ostream& operator<<(std::ostream& out, const ClearFileMetadataResult& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ClearFileMetadataResult::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 ClearFileMetadataResult::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ClearFileMetadataResult"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ClearFileMetadataResult &a, ClearFileMetadataResult &b) { - using ::std::swap; - (void) a; - (void) b; -} - -ClearFileMetadataResult::ClearFileMetadataResult(const ClearFileMetadataResult& other800) { - (void) other800; -} -ClearFileMetadataResult& ClearFileMetadataResult::operator=(const ClearFileMetadataResult& other801) { - (void) other801; - return *this; -} -void ClearFileMetadataResult::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ClearFileMetadataResult("; - out << ")"; -} - - -ClearFileMetadataRequest::~ClearFileMetadataRequest() noexcept { -} - - -void ClearFileMetadataRequest::__set_fileIds(const std::vector & val) { - this->fileIds = val; -} -std::ostream& operator<<(std::ostream& out, const ClearFileMetadataRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ClearFileMetadataRequest::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_fileIds = 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_LIST) { - { - this->fileIds.clear(); - uint32_t _size802; - ::apache::thrift::protocol::TType _etype805; - xfer += iprot->readListBegin(_etype805, _size802); - this->fileIds.resize(_size802); - uint32_t _i806; - for (_i806 = 0; _i806 < _size802; ++_i806) - { - xfer += iprot->readI64(this->fileIds[_i806]); - } - xfer += iprot->readListEnd(); - } - isset_fileIds = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_fileIds) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t ClearFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ClearFileMetadataRequest"); - - xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->fileIds.size())); - std::vector ::const_iterator _iter807; - for (_iter807 = this->fileIds.begin(); _iter807 != this->fileIds.end(); ++_iter807) - { - xfer += oprot->writeI64((*_iter807)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ClearFileMetadataRequest &a, ClearFileMetadataRequest &b) { - using ::std::swap; - swap(a.fileIds, b.fileIds); -} - -ClearFileMetadataRequest::ClearFileMetadataRequest(const ClearFileMetadataRequest& other808) { - fileIds = other808.fileIds; -} -ClearFileMetadataRequest& ClearFileMetadataRequest::operator=(const ClearFileMetadataRequest& other809) { - fileIds = other809.fileIds; - return *this; -} -void ClearFileMetadataRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ClearFileMetadataRequest("; - out << "fileIds=" << to_string(fileIds); - out << ")"; -} - - -CacheFileMetadataResult::~CacheFileMetadataResult() noexcept { -} - - -void CacheFileMetadataResult::__set_isSupported(const bool val) { - this->isSupported = val; -} -std::ostream& operator<<(std::ostream& out, const CacheFileMetadataResult& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t CacheFileMetadataResult::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_isSupported = 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_BOOL) { - xfer += iprot->readBool(this->isSupported); - isset_isSupported = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_isSupported) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t CacheFileMetadataResult::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("CacheFileMetadataResult"); - - xfer += oprot->writeFieldBegin("isSupported", ::apache::thrift::protocol::T_BOOL, 1); - xfer += oprot->writeBool(this->isSupported); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(CacheFileMetadataResult &a, CacheFileMetadataResult &b) { - using ::std::swap; - swap(a.isSupported, b.isSupported); -} - -CacheFileMetadataResult::CacheFileMetadataResult(const CacheFileMetadataResult& other810) { - isSupported = other810.isSupported; -} -CacheFileMetadataResult& CacheFileMetadataResult::operator=(const CacheFileMetadataResult& other811) { - isSupported = other811.isSupported; - return *this; -} -void CacheFileMetadataResult::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "CacheFileMetadataResult("; - out << "isSupported=" << to_string(isSupported); - out << ")"; -} - - -CacheFileMetadataRequest::~CacheFileMetadataRequest() noexcept { -} - - -void CacheFileMetadataRequest::__set_dbName(const std::string& val) { - this->dbName = val; -} - -void CacheFileMetadataRequest::__set_tblName(const std::string& val) { - this->tblName = val; -} - -void CacheFileMetadataRequest::__set_partName(const std::string& val) { - this->partName = val; -__isset.partName = true; -} - -void CacheFileMetadataRequest::__set_isAllParts(const bool val) { - this->isAllParts = val; -__isset.isAllParts = true; -} -std::ostream& operator<<(std::ostream& out, const CacheFileMetadataRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t CacheFileMetadataRequest::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_dbName = false; - bool isset_tblName = 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_STRING) { - xfer += iprot->readString(this->dbName); - isset_dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tblName); - isset_tblName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->partName); - this->__isset.partName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->isAllParts); - this->__isset.isAllParts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_dbName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tblName) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t CacheFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("CacheFileMetadataRequest"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tblName); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.partName) { - xfer += oprot->writeFieldBegin("partName", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->partName); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.isAllParts) { - xfer += oprot->writeFieldBegin("isAllParts", ::apache::thrift::protocol::T_BOOL, 4); - xfer += oprot->writeBool(this->isAllParts); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(CacheFileMetadataRequest &a, CacheFileMetadataRequest &b) { - using ::std::swap; - swap(a.dbName, b.dbName); - swap(a.tblName, b.tblName); - swap(a.partName, b.partName); - swap(a.isAllParts, b.isAllParts); - swap(a.__isset, b.__isset); -} - -CacheFileMetadataRequest::CacheFileMetadataRequest(const CacheFileMetadataRequest& other812) { - dbName = other812.dbName; - tblName = other812.tblName; - partName = other812.partName; - isAllParts = other812.isAllParts; - __isset = other812.__isset; -} -CacheFileMetadataRequest& CacheFileMetadataRequest::operator=(const CacheFileMetadataRequest& other813) { - dbName = other813.dbName; - tblName = other813.tblName; - partName = other813.partName; - isAllParts = other813.isAllParts; - __isset = other813.__isset; - return *this; -} -void CacheFileMetadataRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "CacheFileMetadataRequest("; - out << "dbName=" << to_string(dbName); - out << ", " << "tblName=" << to_string(tblName); - out << ", " << "partName="; (__isset.partName ? (out << to_string(partName)) : (out << "")); - out << ", " << "isAllParts="; (__isset.isAllParts ? (out << to_string(isAllParts)) : (out << "")); - out << ")"; -} - - -GetAllFunctionsResponse::~GetAllFunctionsResponse() noexcept { -} - - -void GetAllFunctionsResponse::__set_functions(const std::vector & val) { - this->functions = val; -__isset.functions = true; -} -std::ostream& operator<<(std::ostream& out, const GetAllFunctionsResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GetAllFunctionsResponse::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->functions.clear(); - uint32_t _size814; - ::apache::thrift::protocol::TType _etype817; - xfer += iprot->readListBegin(_etype817, _size814); - this->functions.resize(_size814); - uint32_t _i818; - for (_i818 = 0; _i818 < _size814; ++_i818) - { - xfer += this->functions[_i818].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.functions = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t GetAllFunctionsResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GetAllFunctionsResponse"); - - if (this->__isset.functions) { - xfer += oprot->writeFieldBegin("functions", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->functions.size())); - std::vector ::const_iterator _iter819; - for (_iter819 = this->functions.begin(); _iter819 != this->functions.end(); ++_iter819) - { - xfer += (*_iter819).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GetAllFunctionsResponse &a, GetAllFunctionsResponse &b) { - using ::std::swap; - swap(a.functions, b.functions); - swap(a.__isset, b.__isset); -} - -GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other820) { - functions = other820.functions; - __isset = other820.__isset; -} -GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other821) { - functions = other821.functions; - __isset = other821.__isset; - return *this; -} -void GetAllFunctionsResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GetAllFunctionsResponse("; - out << "functions="; (__isset.functions ? (out << to_string(functions)) : (out << "")); - out << ")"; -} - - -ClientCapabilities::~ClientCapabilities() noexcept { -} - - -void ClientCapabilities::__set_values(const std::vector & val) { - this->values = val; -} -std::ostream& operator<<(std::ostream& out, const ClientCapabilities& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ClientCapabilities::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_values = 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_LIST) { - { - this->values.clear(); - uint32_t _size822; - ::apache::thrift::protocol::TType _etype825; - xfer += iprot->readListBegin(_etype825, _size822); - this->values.resize(_size822); - uint32_t _i826; - for (_i826 = 0; _i826 < _size822; ++_i826) - { - int32_t ecast827; - xfer += iprot->readI32(ecast827); - this->values[_i826] = (ClientCapability::type)ecast827; - } - xfer += iprot->readListEnd(); - } - isset_values = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_values) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t ClientCapabilities::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ClientCapabilities"); - - xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I32, static_cast(this->values.size())); - std::vector ::const_iterator _iter828; - for (_iter828 = this->values.begin(); _iter828 != this->values.end(); ++_iter828) - { - xfer += oprot->writeI32((int32_t)(*_iter828)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ClientCapabilities &a, ClientCapabilities &b) { - using ::std::swap; - swap(a.values, b.values); -} - -ClientCapabilities::ClientCapabilities(const ClientCapabilities& other829) { - values = other829.values; -} -ClientCapabilities& ClientCapabilities::operator=(const ClientCapabilities& other830) { - values = other830.values; - return *this; -} -void ClientCapabilities::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ClientCapabilities("; - out << "values=" << to_string(values); - out << ")"; -} - - -GetTableRequest::~GetTableRequest() noexcept { -} - - -void GetTableRequest::__set_dbName(const std::string& val) { - this->dbName = val; -} - -void GetTableRequest::__set_tblName(const std::string& val) { - this->tblName = val; -} - -void GetTableRequest::__set_capabilities(const ClientCapabilities& val) { - this->capabilities = val; -__isset.capabilities = true; -} -std::ostream& operator<<(std::ostream& out, const GetTableRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GetTableRequest::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_dbName = false; - bool isset_tblName = 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_STRING) { - xfer += iprot->readString(this->dbName); - isset_dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tblName); - isset_tblName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->capabilities.read(iprot); - this->__isset.capabilities = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_dbName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tblName) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t GetTableRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GetTableRequest"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tblName); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.capabilities) { - xfer += oprot->writeFieldBegin("capabilities", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->capabilities.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GetTableRequest &a, GetTableRequest &b) { - using ::std::swap; - swap(a.dbName, b.dbName); - swap(a.tblName, b.tblName); - swap(a.capabilities, b.capabilities); - swap(a.__isset, b.__isset); -} - -GetTableRequest::GetTableRequest(const GetTableRequest& other831) { - dbName = other831.dbName; - tblName = other831.tblName; - capabilities = other831.capabilities; - __isset = other831.__isset; -} -GetTableRequest& GetTableRequest::operator=(const GetTableRequest& other832) { - dbName = other832.dbName; - tblName = other832.tblName; - capabilities = other832.capabilities; - __isset = other832.__isset; - return *this; -} -void GetTableRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GetTableRequest("; - out << "dbName=" << to_string(dbName); - out << ", " << "tblName=" << to_string(tblName); - out << ", " << "capabilities="; (__isset.capabilities ? (out << to_string(capabilities)) : (out << "")); - out << ")"; -} - - -GetTableResult::~GetTableResult() noexcept { -} - - -void GetTableResult::__set_table(const Table& val) { - this->table = val; -} -std::ostream& operator<<(std::ostream& out, const GetTableResult& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GetTableResult::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_table = 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_STRUCT) { - xfer += this->table.read(iprot); - isset_table = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_table) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t GetTableResult::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GetTableResult"); - - xfer += oprot->writeFieldBegin("table", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->table.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GetTableResult &a, GetTableResult &b) { - using ::std::swap; - swap(a.table, b.table); -} - -GetTableResult::GetTableResult(const GetTableResult& other833) { - table = other833.table; -} -GetTableResult& GetTableResult::operator=(const GetTableResult& other834) { - table = other834.table; - return *this; -} -void GetTableResult::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GetTableResult("; - out << "table=" << to_string(table); - out << ")"; -} - - -GetTablesRequest::~GetTablesRequest() noexcept { -} - - -void GetTablesRequest::__set_dbName(const std::string& val) { - this->dbName = val; -} - -void GetTablesRequest::__set_tblNames(const std::vector & val) { - this->tblNames = val; -__isset.tblNames = true; -} - -void GetTablesRequest::__set_capabilities(const ClientCapabilities& val) { - this->capabilities = val; -__isset.capabilities = true; -} -std::ostream& operator<<(std::ostream& out, const GetTablesRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GetTablesRequest::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_dbName = 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_STRING) { - xfer += iprot->readString(this->dbName); - isset_dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->tblNames.clear(); - uint32_t _size835; - ::apache::thrift::protocol::TType _etype838; - xfer += iprot->readListBegin(_etype838, _size835); - this->tblNames.resize(_size835); - uint32_t _i839; - for (_i839 = 0; _i839 < _size835; ++_i839) - { - xfer += iprot->readString(this->tblNames[_i839]); - } - xfer += iprot->readListEnd(); - } - this->__isset.tblNames = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->capabilities.read(iprot); - this->__isset.capabilities = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_dbName) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t GetTablesRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GetTablesRequest"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.tblNames) { - xfer += oprot->writeFieldBegin("tblNames", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->tblNames.size())); - std::vector ::const_iterator _iter840; - for (_iter840 = this->tblNames.begin(); _iter840 != this->tblNames.end(); ++_iter840) - { - xfer += oprot->writeString((*_iter840)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.capabilities) { - xfer += oprot->writeFieldBegin("capabilities", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->capabilities.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GetTablesRequest &a, GetTablesRequest &b) { - using ::std::swap; - swap(a.dbName, b.dbName); - swap(a.tblNames, b.tblNames); - swap(a.capabilities, b.capabilities); - swap(a.__isset, b.__isset); -} - -GetTablesRequest::GetTablesRequest(const GetTablesRequest& other841) { - dbName = other841.dbName; - tblNames = other841.tblNames; - capabilities = other841.capabilities; - __isset = other841.__isset; -} -GetTablesRequest& GetTablesRequest::operator=(const GetTablesRequest& other842) { - dbName = other842.dbName; - tblNames = other842.tblNames; - capabilities = other842.capabilities; - __isset = other842.__isset; - return *this; -} -void GetTablesRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GetTablesRequest("; - out << "dbName=" << to_string(dbName); - out << ", " << "tblNames="; (__isset.tblNames ? (out << to_string(tblNames)) : (out << "")); - out << ", " << "capabilities="; (__isset.capabilities ? (out << to_string(capabilities)) : (out << "")); - out << ")"; -} - - -GetTablesResult::~GetTablesResult() noexcept { -} - - -void GetTablesResult::__set_tables(const std::vector
& val) { - this->tables = val; -} -std::ostream& operator<<(std::ostream& out, const GetTablesResult& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t GetTablesResult::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_tables = 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_LIST) { - { - this->tables.clear(); - uint32_t _size843; - ::apache::thrift::protocol::TType _etype846; - xfer += iprot->readListBegin(_etype846, _size843); - this->tables.resize(_size843); - uint32_t _i847; - for (_i847 = 0; _i847 < _size843; ++_i847) - { - xfer += this->tables[_i847].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_tables = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_tables) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t GetTablesResult::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("GetTablesResult"); - - xfer += oprot->writeFieldBegin("tables", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->tables.size())); - std::vector
::const_iterator _iter848; - for (_iter848 = this->tables.begin(); _iter848 != this->tables.end(); ++_iter848) - { - xfer += (*_iter848).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(GetTablesResult &a, GetTablesResult &b) { - using ::std::swap; - swap(a.tables, b.tables); -} - -GetTablesResult::GetTablesResult(const GetTablesResult& other849) { - tables = other849.tables; -} -GetTablesResult& GetTablesResult::operator=(const GetTablesResult& other850) { - tables = other850.tables; - return *this; -} -void GetTablesResult::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "GetTablesResult("; - out << "tables=" << to_string(tables); - out << ")"; -} - - -CmRecycleRequest::~CmRecycleRequest() noexcept { -} - - -void CmRecycleRequest::__set_dataPath(const std::string& val) { - this->dataPath = val; -} - -void CmRecycleRequest::__set_purge(const bool val) { - this->purge = val; -} -std::ostream& operator<<(std::ostream& out, const CmRecycleRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t CmRecycleRequest::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_dataPath = false; - bool isset_purge = 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_STRING) { - xfer += iprot->readString(this->dataPath); - isset_dataPath = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->purge); - isset_purge = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_dataPath) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_purge) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t CmRecycleRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("CmRecycleRequest"); - - xfer += oprot->writeFieldBegin("dataPath", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dataPath); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("purge", ::apache::thrift::protocol::T_BOOL, 2); - xfer += oprot->writeBool(this->purge); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(CmRecycleRequest &a, CmRecycleRequest &b) { - using ::std::swap; - swap(a.dataPath, b.dataPath); - swap(a.purge, b.purge); -} - -CmRecycleRequest::CmRecycleRequest(const CmRecycleRequest& other851) { - dataPath = other851.dataPath; - purge = other851.purge; -} -CmRecycleRequest& CmRecycleRequest::operator=(const CmRecycleRequest& other852) { - dataPath = other852.dataPath; - purge = other852.purge; - return *this; -} -void CmRecycleRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "CmRecycleRequest("; - out << "dataPath=" << to_string(dataPath); - out << ", " << "purge=" << to_string(purge); - out << ")"; -} - - -CmRecycleResponse::~CmRecycleResponse() noexcept { -} - -std::ostream& operator<<(std::ostream& out, const CmRecycleResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t CmRecycleResponse::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 CmRecycleResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("CmRecycleResponse"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(CmRecycleResponse &a, CmRecycleResponse &b) { - using ::std::swap; - (void) a; - (void) b; -} - -CmRecycleResponse::CmRecycleResponse(const CmRecycleResponse& other853) { - (void) other853; -} -CmRecycleResponse& CmRecycleResponse::operator=(const CmRecycleResponse& other854) { - (void) other854; - return *this; -} -void CmRecycleResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "CmRecycleResponse("; - out << ")"; -} - - -TableMeta::~TableMeta() noexcept { -} - - -void TableMeta::__set_dbName(const std::string& val) { - this->dbName = val; -} - -void TableMeta::__set_tableName(const std::string& val) { - this->tableName = val; -} - -void TableMeta::__set_tableType(const std::string& val) { - this->tableType = val; -} - -void TableMeta::__set_comments(const std::string& val) { - this->comments = val; -__isset.comments = true; -} -std::ostream& operator<<(std::ostream& out, const TableMeta& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t TableMeta::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_dbName = false; - bool isset_tableName = false; - bool isset_tableType = 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_STRING) { - xfer += iprot->readString(this->dbName); - isset_dbName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tableName); - isset_tableName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->tableType); - isset_tableType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->comments); - this->__isset.comments = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_dbName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tableName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_tableType) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t TableMeta::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("TableMeta"); - - xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->dbName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->tableName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("tableType", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->tableType); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.comments) { - xfer += oprot->writeFieldBegin("comments", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->comments); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(TableMeta &a, TableMeta &b) { - using ::std::swap; - swap(a.dbName, b.dbName); - swap(a.tableName, b.tableName); - swap(a.tableType, b.tableType); - swap(a.comments, b.comments); - swap(a.__isset, b.__isset); -} - -TableMeta::TableMeta(const TableMeta& other855) { - dbName = other855.dbName; - tableName = other855.tableName; - tableType = other855.tableType; - comments = other855.comments; - __isset = other855.__isset; -} -TableMeta& TableMeta::operator=(const TableMeta& other856) { - dbName = other856.dbName; - tableName = other856.tableName; - tableType = other856.tableType; - comments = other856.comments; - __isset = other856.__isset; - return *this; -} -void TableMeta::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "TableMeta("; - out << "dbName=" << to_string(dbName); - out << ", " << "tableName=" << to_string(tableName); - out << ", " << "tableType=" << to_string(tableType); - out << ", " << "comments="; (__isset.comments ? (out << to_string(comments)) : (out << "")); - out << ")"; -} - - -WMResourcePlan::~WMResourcePlan() noexcept { -} - - -void WMResourcePlan::__set_name(const std::string& val) { - this->name = val; -} - -void WMResourcePlan::__set_status(const WMResourcePlanStatus::type val) { - this->status = val; -__isset.status = true; -} - -void WMResourcePlan::__set_queryParallelism(const int32_t val) { - this->queryParallelism = val; -__isset.queryParallelism = true; -} - -void WMResourcePlan::__set_defaultPoolPath(const std::string& val) { - this->defaultPoolPath = val; -__isset.defaultPoolPath = true; -} -std::ostream& operator<<(std::ostream& out, const WMResourcePlan& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMResourcePlan::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_name = 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_STRING) { - xfer += iprot->readString(this->name); - isset_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast857; - xfer += iprot->readI32(ecast857); - this->status = (WMResourcePlanStatus::type)ecast857; - this->__isset.status = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->queryParallelism); - this->__isset.queryParallelism = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->defaultPoolPath); - this->__isset.defaultPoolPath = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_name) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t WMResourcePlan::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMResourcePlan"); - - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->name); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.status) { - xfer += oprot->writeFieldBegin("status", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32((int32_t)this->status); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.queryParallelism) { - xfer += oprot->writeFieldBegin("queryParallelism", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32(this->queryParallelism); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.defaultPoolPath) { - xfer += oprot->writeFieldBegin("defaultPoolPath", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->defaultPoolPath); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMResourcePlan &a, WMResourcePlan &b) { - using ::std::swap; - swap(a.name, b.name); - swap(a.status, b.status); - swap(a.queryParallelism, b.queryParallelism); - swap(a.defaultPoolPath, b.defaultPoolPath); - swap(a.__isset, b.__isset); -} - -WMResourcePlan::WMResourcePlan(const WMResourcePlan& other858) { - name = other858.name; - status = other858.status; - queryParallelism = other858.queryParallelism; - defaultPoolPath = other858.defaultPoolPath; - __isset = other858.__isset; -} -WMResourcePlan& WMResourcePlan::operator=(const WMResourcePlan& other859) { - name = other859.name; - status = other859.status; - queryParallelism = other859.queryParallelism; - defaultPoolPath = other859.defaultPoolPath; - __isset = other859.__isset; - return *this; -} -void WMResourcePlan::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMResourcePlan("; - out << "name=" << to_string(name); - out << ", " << "status="; (__isset.status ? (out << to_string(status)) : (out << "")); - out << ", " << "queryParallelism="; (__isset.queryParallelism ? (out << to_string(queryParallelism)) : (out << "")); - out << ", " << "defaultPoolPath="; (__isset.defaultPoolPath ? (out << to_string(defaultPoolPath)) : (out << "")); - out << ")"; -} - - -WMPool::~WMPool() noexcept { -} - - -void WMPool::__set_resourcePlanName(const std::string& val) { - this->resourcePlanName = val; -} - -void WMPool::__set_poolPath(const std::string& val) { - this->poolPath = val; -} - -void WMPool::__set_allocFraction(const double val) { - this->allocFraction = val; -__isset.allocFraction = true; -} - -void WMPool::__set_queryParallelism(const int32_t val) { - this->queryParallelism = val; -__isset.queryParallelism = true; -} - -void WMPool::__set_schedulingPolicy(const std::string& val) { - this->schedulingPolicy = val; -__isset.schedulingPolicy = true; -} -std::ostream& operator<<(std::ostream& out, const WMPool& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMPool::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_resourcePlanName = false; - bool isset_poolPath = 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_STRING) { - xfer += iprot->readString(this->resourcePlanName); - isset_resourcePlanName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->poolPath); - isset_poolPath = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_DOUBLE) { - xfer += iprot->readDouble(this->allocFraction); - this->__isset.allocFraction = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->queryParallelism); - this->__isset.queryParallelism = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->schedulingPolicy); - this->__isset.schedulingPolicy = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_resourcePlanName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_poolPath) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t WMPool::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMPool"); - - xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->resourcePlanName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("poolPath", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->poolPath); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.allocFraction) { - xfer += oprot->writeFieldBegin("allocFraction", ::apache::thrift::protocol::T_DOUBLE, 3); - xfer += oprot->writeDouble(this->allocFraction); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.queryParallelism) { - xfer += oprot->writeFieldBegin("queryParallelism", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32(this->queryParallelism); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.schedulingPolicy) { - xfer += oprot->writeFieldBegin("schedulingPolicy", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeString(this->schedulingPolicy); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMPool &a, WMPool &b) { - using ::std::swap; - swap(a.resourcePlanName, b.resourcePlanName); - swap(a.poolPath, b.poolPath); - swap(a.allocFraction, b.allocFraction); - swap(a.queryParallelism, b.queryParallelism); - swap(a.schedulingPolicy, b.schedulingPolicy); - swap(a.__isset, b.__isset); -} - -WMPool::WMPool(const WMPool& other860) { - resourcePlanName = other860.resourcePlanName; - poolPath = other860.poolPath; - allocFraction = other860.allocFraction; - queryParallelism = other860.queryParallelism; - schedulingPolicy = other860.schedulingPolicy; - __isset = other860.__isset; -} -WMPool& WMPool::operator=(const WMPool& other861) { - resourcePlanName = other861.resourcePlanName; - poolPath = other861.poolPath; - allocFraction = other861.allocFraction; - queryParallelism = other861.queryParallelism; - schedulingPolicy = other861.schedulingPolicy; - __isset = other861.__isset; - return *this; -} -void WMPool::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMPool("; - out << "resourcePlanName=" << to_string(resourcePlanName); - out << ", " << "poolPath=" << to_string(poolPath); - out << ", " << "allocFraction="; (__isset.allocFraction ? (out << to_string(allocFraction)) : (out << "")); - out << ", " << "queryParallelism="; (__isset.queryParallelism ? (out << to_string(queryParallelism)) : (out << "")); - out << ", " << "schedulingPolicy="; (__isset.schedulingPolicy ? (out << to_string(schedulingPolicy)) : (out << "")); - out << ")"; -} - - -WMTrigger::~WMTrigger() noexcept { -} - - -void WMTrigger::__set_resourcePlanName(const std::string& val) { - this->resourcePlanName = val; -} - -void WMTrigger::__set_triggerName(const std::string& val) { - this->triggerName = val; -} - -void WMTrigger::__set_triggerExpression(const std::string& val) { - this->triggerExpression = val; -__isset.triggerExpression = true; -} - -void WMTrigger::__set_actionExpression(const std::string& val) { - this->actionExpression = val; -__isset.actionExpression = true; -} -std::ostream& operator<<(std::ostream& out, const WMTrigger& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMTrigger::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_resourcePlanName = false; - bool isset_triggerName = 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_STRING) { - xfer += iprot->readString(this->resourcePlanName); - isset_resourcePlanName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->triggerName); - isset_triggerName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->triggerExpression); - this->__isset.triggerExpression = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->actionExpression); - this->__isset.actionExpression = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_resourcePlanName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_triggerName) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t WMTrigger::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMTrigger"); - - xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->resourcePlanName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("triggerName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->triggerName); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.triggerExpression) { - xfer += oprot->writeFieldBegin("triggerExpression", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->triggerExpression); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.actionExpression) { - xfer += oprot->writeFieldBegin("actionExpression", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->actionExpression); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMTrigger &a, WMTrigger &b) { - using ::std::swap; - swap(a.resourcePlanName, b.resourcePlanName); - swap(a.triggerName, b.triggerName); - swap(a.triggerExpression, b.triggerExpression); - swap(a.actionExpression, b.actionExpression); - swap(a.__isset, b.__isset); -} - -WMTrigger::WMTrigger(const WMTrigger& other862) { - resourcePlanName = other862.resourcePlanName; - triggerName = other862.triggerName; - triggerExpression = other862.triggerExpression; - actionExpression = other862.actionExpression; - __isset = other862.__isset; -} -WMTrigger& WMTrigger::operator=(const WMTrigger& other863) { - resourcePlanName = other863.resourcePlanName; - triggerName = other863.triggerName; - triggerExpression = other863.triggerExpression; - actionExpression = other863.actionExpression; - __isset = other863.__isset; - return *this; -} -void WMTrigger::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMTrigger("; - out << "resourcePlanName=" << to_string(resourcePlanName); - out << ", " << "triggerName=" << to_string(triggerName); - out << ", " << "triggerExpression="; (__isset.triggerExpression ? (out << to_string(triggerExpression)) : (out << "")); - out << ", " << "actionExpression="; (__isset.actionExpression ? (out << to_string(actionExpression)) : (out << "")); - out << ")"; -} - - -WMMapping::~WMMapping() noexcept { -} - - -void WMMapping::__set_resourcePlanName(const std::string& val) { - this->resourcePlanName = val; -} - -void WMMapping::__set_entityType(const std::string& val) { - this->entityType = val; -} - -void WMMapping::__set_entityName(const std::string& val) { - this->entityName = val; -} - -void WMMapping::__set_poolName(const std::string& val) { - this->poolName = val; -__isset.poolName = true; -} - -void WMMapping::__set_ordering(const int32_t val) { - this->ordering = val; -__isset.ordering = true; -} -std::ostream& operator<<(std::ostream& out, const WMMapping& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMMapping::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_resourcePlanName = false; - bool isset_entityType = false; - bool isset_entityName = 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_STRING) { - xfer += iprot->readString(this->resourcePlanName); - isset_resourcePlanName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->entityType); - isset_entityType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->entityName); - isset_entityName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->poolName); - this->__isset.poolName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->ordering); - this->__isset.ordering = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_resourcePlanName) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_entityType) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_entityName) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t WMMapping::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMMapping"); - - xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->resourcePlanName); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("entityType", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->entityType); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("entityName", ::apache::thrift::protocol::T_STRING, 3); - xfer += oprot->writeString(this->entityName); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.poolName) { - xfer += oprot->writeFieldBegin("poolName", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->poolName); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.ordering) { - xfer += oprot->writeFieldBegin("ordering", ::apache::thrift::protocol::T_I32, 5); - xfer += oprot->writeI32(this->ordering); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMMapping &a, WMMapping &b) { - using ::std::swap; - swap(a.resourcePlanName, b.resourcePlanName); - swap(a.entityType, b.entityType); - swap(a.entityName, b.entityName); - swap(a.poolName, b.poolName); - swap(a.ordering, b.ordering); - swap(a.__isset, b.__isset); -} - -WMMapping::WMMapping(const WMMapping& other864) { - resourcePlanName = other864.resourcePlanName; - entityType = other864.entityType; - entityName = other864.entityName; - poolName = other864.poolName; - ordering = other864.ordering; - __isset = other864.__isset; -} -WMMapping& WMMapping::operator=(const WMMapping& other865) { - resourcePlanName = other865.resourcePlanName; - entityType = other865.entityType; - entityName = other865.entityName; - poolName = other865.poolName; - ordering = other865.ordering; - __isset = other865.__isset; - return *this; -} -void WMMapping::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMMapping("; - out << "resourcePlanName=" << to_string(resourcePlanName); - out << ", " << "entityType=" << to_string(entityType); - out << ", " << "entityName=" << to_string(entityName); - out << ", " << "poolName="; (__isset.poolName ? (out << to_string(poolName)) : (out << "")); - out << ", " << "ordering="; (__isset.ordering ? (out << to_string(ordering)) : (out << "")); - out << ")"; -} - - -WMPoolTrigger::~WMPoolTrigger() noexcept { -} - - -void WMPoolTrigger::__set_pool(const std::string& val) { - this->pool = val; -} - -void WMPoolTrigger::__set_trigger(const std::string& val) { - this->trigger = val; -} -std::ostream& operator<<(std::ostream& out, const WMPoolTrigger& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMPoolTrigger::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_pool = false; - bool isset_trigger = 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_STRING) { - xfer += iprot->readString(this->pool); - isset_pool = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->trigger); - isset_trigger = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_pool) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_trigger) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t WMPoolTrigger::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMPoolTrigger"); - - xfer += oprot->writeFieldBegin("pool", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->pool); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("trigger", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->trigger); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMPoolTrigger &a, WMPoolTrigger &b) { - using ::std::swap; - swap(a.pool, b.pool); - swap(a.trigger, b.trigger); -} - -WMPoolTrigger::WMPoolTrigger(const WMPoolTrigger& other866) { - pool = other866.pool; - trigger = other866.trigger; -} -WMPoolTrigger& WMPoolTrigger::operator=(const WMPoolTrigger& other867) { - pool = other867.pool; - trigger = other867.trigger; - return *this; -} -void WMPoolTrigger::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMPoolTrigger("; - out << "pool=" << to_string(pool); - out << ", " << "trigger=" << to_string(trigger); - out << ")"; -} - - -WMFullResourcePlan::~WMFullResourcePlan() noexcept { -} - - -void WMFullResourcePlan::__set_plan(const WMResourcePlan& val) { - this->plan = val; -} - -void WMFullResourcePlan::__set_pools(const std::vector & val) { - this->pools = val; -} - -void WMFullResourcePlan::__set_mappings(const std::vector & val) { - this->mappings = val; -__isset.mappings = true; -} - -void WMFullResourcePlan::__set_triggers(const std::vector & val) { - this->triggers = val; -__isset.triggers = true; -} - -void WMFullResourcePlan::__set_poolTriggers(const std::vector & val) { - this->poolTriggers = val; -__isset.poolTriggers = true; -} -std::ostream& operator<<(std::ostream& out, const WMFullResourcePlan& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMFullResourcePlan::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_plan = false; - bool isset_pools = 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_STRUCT) { - xfer += this->plan.read(iprot); - isset_plan = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->pools.clear(); - uint32_t _size868; - ::apache::thrift::protocol::TType _etype871; - xfer += iprot->readListBegin(_etype871, _size868); - this->pools.resize(_size868); - uint32_t _i872; - for (_i872 = 0; _i872 < _size868; ++_i872) - { - xfer += this->pools[_i872].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_pools = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->mappings.clear(); - uint32_t _size873; - ::apache::thrift::protocol::TType _etype876; - xfer += iprot->readListBegin(_etype876, _size873); - this->mappings.resize(_size873); - uint32_t _i877; - for (_i877 = 0; _i877 < _size873; ++_i877) - { - xfer += this->mappings[_i877].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.mappings = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->triggers.clear(); - uint32_t _size878; - ::apache::thrift::protocol::TType _etype881; - xfer += iprot->readListBegin(_etype881, _size878); - this->triggers.resize(_size878); - uint32_t _i882; - for (_i882 = 0; _i882 < _size878; ++_i882) - { - xfer += this->triggers[_i882].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.triggers = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->poolTriggers.clear(); - uint32_t _size883; - ::apache::thrift::protocol::TType _etype886; - xfer += iprot->readListBegin(_etype886, _size883); - this->poolTriggers.resize(_size883); - uint32_t _i887; - for (_i887 = 0; _i887 < _size883; ++_i887) - { - xfer += this->poolTriggers[_i887].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.poolTriggers = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_plan) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_pools) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t WMFullResourcePlan::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMFullResourcePlan"); - - xfer += oprot->writeFieldBegin("plan", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->plan.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("pools", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->pools.size())); - std::vector ::const_iterator _iter888; - for (_iter888 = this->pools.begin(); _iter888 != this->pools.end(); ++_iter888) - { - xfer += (*_iter888).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - if (this->__isset.mappings) { - xfer += oprot->writeFieldBegin("mappings", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->mappings.size())); - std::vector ::const_iterator _iter889; - for (_iter889 = this->mappings.begin(); _iter889 != this->mappings.end(); ++_iter889) - { - xfer += (*_iter889).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.triggers) { - xfer += oprot->writeFieldBegin("triggers", ::apache::thrift::protocol::T_LIST, 4); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->triggers.size())); - std::vector ::const_iterator _iter890; - for (_iter890 = this->triggers.begin(); _iter890 != this->triggers.end(); ++_iter890) - { - xfer += (*_iter890).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.poolTriggers) { - xfer += oprot->writeFieldBegin("poolTriggers", ::apache::thrift::protocol::T_LIST, 5); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->poolTriggers.size())); - std::vector ::const_iterator _iter891; - for (_iter891 = this->poolTriggers.begin(); _iter891 != this->poolTriggers.end(); ++_iter891) - { - xfer += (*_iter891).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMFullResourcePlan &a, WMFullResourcePlan &b) { - using ::std::swap; - swap(a.plan, b.plan); - swap(a.pools, b.pools); - swap(a.mappings, b.mappings); - swap(a.triggers, b.triggers); - swap(a.poolTriggers, b.poolTriggers); - swap(a.__isset, b.__isset); -} - -WMFullResourcePlan::WMFullResourcePlan(const WMFullResourcePlan& other892) { - plan = other892.plan; - pools = other892.pools; - mappings = other892.mappings; - triggers = other892.triggers; - poolTriggers = other892.poolTriggers; - __isset = other892.__isset; -} -WMFullResourcePlan& WMFullResourcePlan::operator=(const WMFullResourcePlan& other893) { - plan = other893.plan; - pools = other893.pools; - mappings = other893.mappings; - triggers = other893.triggers; - poolTriggers = other893.poolTriggers; - __isset = other893.__isset; - return *this; -} -void WMFullResourcePlan::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMFullResourcePlan("; - out << "plan=" << to_string(plan); - out << ", " << "pools=" << to_string(pools); - out << ", " << "mappings="; (__isset.mappings ? (out << to_string(mappings)) : (out << "")); - out << ", " << "triggers="; (__isset.triggers ? (out << to_string(triggers)) : (out << "")); - out << ", " << "poolTriggers="; (__isset.poolTriggers ? (out << to_string(poolTriggers)) : (out << "")); - out << ")"; -} - - -WMCreateResourcePlanRequest::~WMCreateResourcePlanRequest() noexcept { -} - - -void WMCreateResourcePlanRequest::__set_resourcePlan(const WMResourcePlan& val) { - this->resourcePlan = val; -__isset.resourcePlan = true; -} -std::ostream& operator<<(std::ostream& out, const WMCreateResourcePlanRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMCreateResourcePlanRequest::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->resourcePlan.read(iprot); - this->__isset.resourcePlan = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t WMCreateResourcePlanRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMCreateResourcePlanRequest"); - - if (this->__isset.resourcePlan) { - xfer += oprot->writeFieldBegin("resourcePlan", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->resourcePlan.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMCreateResourcePlanRequest &a, WMCreateResourcePlanRequest &b) { - using ::std::swap; - swap(a.resourcePlan, b.resourcePlan); - swap(a.__isset, b.__isset); -} - -WMCreateResourcePlanRequest::WMCreateResourcePlanRequest(const WMCreateResourcePlanRequest& other894) { - resourcePlan = other894.resourcePlan; - __isset = other894.__isset; -} -WMCreateResourcePlanRequest& WMCreateResourcePlanRequest::operator=(const WMCreateResourcePlanRequest& other895) { - resourcePlan = other895.resourcePlan; - __isset = other895.__isset; - return *this; -} -void WMCreateResourcePlanRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMCreateResourcePlanRequest("; - out << "resourcePlan="; (__isset.resourcePlan ? (out << to_string(resourcePlan)) : (out << "")); - out << ")"; -} - - -WMCreateResourcePlanResponse::~WMCreateResourcePlanResponse() noexcept { -} - -std::ostream& operator<<(std::ostream& out, const WMCreateResourcePlanResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMCreateResourcePlanResponse::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 WMCreateResourcePlanResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMCreateResourcePlanResponse"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMCreateResourcePlanResponse &a, WMCreateResourcePlanResponse &b) { - using ::std::swap; - (void) a; - (void) b; -} - -WMCreateResourcePlanResponse::WMCreateResourcePlanResponse(const WMCreateResourcePlanResponse& other896) { - (void) other896; -} -WMCreateResourcePlanResponse& WMCreateResourcePlanResponse::operator=(const WMCreateResourcePlanResponse& other897) { - (void) other897; - return *this; -} -void WMCreateResourcePlanResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMCreateResourcePlanResponse("; - out << ")"; -} - - -WMGetActiveResourcePlanRequest::~WMGetActiveResourcePlanRequest() noexcept { -} - -std::ostream& operator<<(std::ostream& out, const WMGetActiveResourcePlanRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMGetActiveResourcePlanRequest::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 WMGetActiveResourcePlanRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMGetActiveResourcePlanRequest"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMGetActiveResourcePlanRequest &a, WMGetActiveResourcePlanRequest &b) { - using ::std::swap; - (void) a; - (void) b; -} - -WMGetActiveResourcePlanRequest::WMGetActiveResourcePlanRequest(const WMGetActiveResourcePlanRequest& other898) { - (void) other898; -} -WMGetActiveResourcePlanRequest& WMGetActiveResourcePlanRequest::operator=(const WMGetActiveResourcePlanRequest& other899) { - (void) other899; - return *this; -} -void WMGetActiveResourcePlanRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMGetActiveResourcePlanRequest("; - out << ")"; -} - - -WMGetActiveResourcePlanResponse::~WMGetActiveResourcePlanResponse() noexcept { -} - - -void WMGetActiveResourcePlanResponse::__set_resourcePlan(const WMFullResourcePlan& val) { - this->resourcePlan = val; -__isset.resourcePlan = true; -} -std::ostream& operator<<(std::ostream& out, const WMGetActiveResourcePlanResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMGetActiveResourcePlanResponse::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->resourcePlan.read(iprot); - this->__isset.resourcePlan = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t WMGetActiveResourcePlanResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMGetActiveResourcePlanResponse"); - - if (this->__isset.resourcePlan) { - xfer += oprot->writeFieldBegin("resourcePlan", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->resourcePlan.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMGetActiveResourcePlanResponse &a, WMGetActiveResourcePlanResponse &b) { - using ::std::swap; - swap(a.resourcePlan, b.resourcePlan); - swap(a.__isset, b.__isset); -} - -WMGetActiveResourcePlanResponse::WMGetActiveResourcePlanResponse(const WMGetActiveResourcePlanResponse& other900) { - resourcePlan = other900.resourcePlan; - __isset = other900.__isset; -} -WMGetActiveResourcePlanResponse& WMGetActiveResourcePlanResponse::operator=(const WMGetActiveResourcePlanResponse& other901) { - resourcePlan = other901.resourcePlan; - __isset = other901.__isset; - return *this; -} -void WMGetActiveResourcePlanResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMGetActiveResourcePlanResponse("; - out << "resourcePlan="; (__isset.resourcePlan ? (out << to_string(resourcePlan)) : (out << "")); - out << ")"; -} - - -WMGetResourcePlanRequest::~WMGetResourcePlanRequest() noexcept { -} - - -void WMGetResourcePlanRequest::__set_resourcePlanName(const std::string& val) { - this->resourcePlanName = val; -__isset.resourcePlanName = true; -} -std::ostream& operator<<(std::ostream& out, const WMGetResourcePlanRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMGetResourcePlanRequest::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->resourcePlanName); - this->__isset.resourcePlanName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t WMGetResourcePlanRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMGetResourcePlanRequest"); - - if (this->__isset.resourcePlanName) { - xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->resourcePlanName); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMGetResourcePlanRequest &a, WMGetResourcePlanRequest &b) { - using ::std::swap; - swap(a.resourcePlanName, b.resourcePlanName); - swap(a.__isset, b.__isset); -} - -WMGetResourcePlanRequest::WMGetResourcePlanRequest(const WMGetResourcePlanRequest& other902) { - resourcePlanName = other902.resourcePlanName; - __isset = other902.__isset; -} -WMGetResourcePlanRequest& WMGetResourcePlanRequest::operator=(const WMGetResourcePlanRequest& other903) { - resourcePlanName = other903.resourcePlanName; - __isset = other903.__isset; - return *this; -} -void WMGetResourcePlanRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMGetResourcePlanRequest("; - out << "resourcePlanName="; (__isset.resourcePlanName ? (out << to_string(resourcePlanName)) : (out << "")); - out << ")"; -} - - -WMGetResourcePlanResponse::~WMGetResourcePlanResponse() noexcept { -} - - -void WMGetResourcePlanResponse::__set_resourcePlan(const WMResourcePlan& val) { - this->resourcePlan = val; -__isset.resourcePlan = true; -} -std::ostream& operator<<(std::ostream& out, const WMGetResourcePlanResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMGetResourcePlanResponse::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->resourcePlan.read(iprot); - this->__isset.resourcePlan = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t WMGetResourcePlanResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMGetResourcePlanResponse"); - - if (this->__isset.resourcePlan) { - xfer += oprot->writeFieldBegin("resourcePlan", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->resourcePlan.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMGetResourcePlanResponse &a, WMGetResourcePlanResponse &b) { - using ::std::swap; - swap(a.resourcePlan, b.resourcePlan); - swap(a.__isset, b.__isset); -} - -WMGetResourcePlanResponse::WMGetResourcePlanResponse(const WMGetResourcePlanResponse& other904) { - resourcePlan = other904.resourcePlan; - __isset = other904.__isset; -} -WMGetResourcePlanResponse& WMGetResourcePlanResponse::operator=(const WMGetResourcePlanResponse& other905) { - resourcePlan = other905.resourcePlan; - __isset = other905.__isset; - return *this; -} -void WMGetResourcePlanResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMGetResourcePlanResponse("; - out << "resourcePlan="; (__isset.resourcePlan ? (out << to_string(resourcePlan)) : (out << "")); - out << ")"; -} - - -WMGetAllResourcePlanRequest::~WMGetAllResourcePlanRequest() noexcept { -} - -std::ostream& operator<<(std::ostream& out, const WMGetAllResourcePlanRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMGetAllResourcePlanRequest::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 WMGetAllResourcePlanRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMGetAllResourcePlanRequest"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMGetAllResourcePlanRequest &a, WMGetAllResourcePlanRequest &b) { - using ::std::swap; - (void) a; - (void) b; -} - -WMGetAllResourcePlanRequest::WMGetAllResourcePlanRequest(const WMGetAllResourcePlanRequest& other906) { - (void) other906; -} -WMGetAllResourcePlanRequest& WMGetAllResourcePlanRequest::operator=(const WMGetAllResourcePlanRequest& other907) { - (void) other907; - return *this; -} -void WMGetAllResourcePlanRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMGetAllResourcePlanRequest("; - out << ")"; -} - - -WMGetAllResourcePlanResponse::~WMGetAllResourcePlanResponse() noexcept { -} - - -void WMGetAllResourcePlanResponse::__set_resourcePlans(const std::vector & val) { - this->resourcePlans = val; -__isset.resourcePlans = true; -} -std::ostream& operator<<(std::ostream& out, const WMGetAllResourcePlanResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMGetAllResourcePlanResponse::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->resourcePlans.clear(); - uint32_t _size908; - ::apache::thrift::protocol::TType _etype911; - xfer += iprot->readListBegin(_etype911, _size908); - this->resourcePlans.resize(_size908); - uint32_t _i912; - for (_i912 = 0; _i912 < _size908; ++_i912) - { - xfer += this->resourcePlans[_i912].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.resourcePlans = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t WMGetAllResourcePlanResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMGetAllResourcePlanResponse"); - - if (this->__isset.resourcePlans) { - xfer += oprot->writeFieldBegin("resourcePlans", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->resourcePlans.size())); - std::vector ::const_iterator _iter913; - for (_iter913 = this->resourcePlans.begin(); _iter913 != this->resourcePlans.end(); ++_iter913) - { - xfer += (*_iter913).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMGetAllResourcePlanResponse &a, WMGetAllResourcePlanResponse &b) { - using ::std::swap; - swap(a.resourcePlans, b.resourcePlans); - swap(a.__isset, b.__isset); -} - -WMGetAllResourcePlanResponse::WMGetAllResourcePlanResponse(const WMGetAllResourcePlanResponse& other914) { - resourcePlans = other914.resourcePlans; - __isset = other914.__isset; -} -WMGetAllResourcePlanResponse& WMGetAllResourcePlanResponse::operator=(const WMGetAllResourcePlanResponse& other915) { - resourcePlans = other915.resourcePlans; - __isset = other915.__isset; - return *this; -} -void WMGetAllResourcePlanResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMGetAllResourcePlanResponse("; - out << "resourcePlans="; (__isset.resourcePlans ? (out << to_string(resourcePlans)) : (out << "")); - out << ")"; -} - - -WMAlterResourcePlanRequest::~WMAlterResourcePlanRequest() noexcept { -} - - -void WMAlterResourcePlanRequest::__set_resourcePlanName(const std::string& val) { - this->resourcePlanName = val; -__isset.resourcePlanName = true; -} - -void WMAlterResourcePlanRequest::__set_resourcePlan(const WMResourcePlan& val) { - this->resourcePlan = val; -__isset.resourcePlan = true; -} - -void WMAlterResourcePlanRequest::__set_isEnableAndActivate(const bool val) { - this->isEnableAndActivate = val; -__isset.isEnableAndActivate = true; -} -std::ostream& operator<<(std::ostream& out, const WMAlterResourcePlanRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMAlterResourcePlanRequest::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->resourcePlanName); - this->__isset.resourcePlanName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->resourcePlan.read(iprot); - this->__isset.resourcePlan = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->isEnableAndActivate); - this->__isset.isEnableAndActivate = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t WMAlterResourcePlanRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMAlterResourcePlanRequest"); - - if (this->__isset.resourcePlanName) { - xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->resourcePlanName); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.resourcePlan) { - xfer += oprot->writeFieldBegin("resourcePlan", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->resourcePlan.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.isEnableAndActivate) { - xfer += oprot->writeFieldBegin("isEnableAndActivate", ::apache::thrift::protocol::T_BOOL, 3); - xfer += oprot->writeBool(this->isEnableAndActivate); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMAlterResourcePlanRequest &a, WMAlterResourcePlanRequest &b) { - using ::std::swap; - swap(a.resourcePlanName, b.resourcePlanName); - swap(a.resourcePlan, b.resourcePlan); - swap(a.isEnableAndActivate, b.isEnableAndActivate); - swap(a.__isset, b.__isset); -} - -WMAlterResourcePlanRequest::WMAlterResourcePlanRequest(const WMAlterResourcePlanRequest& other916) { - resourcePlanName = other916.resourcePlanName; - resourcePlan = other916.resourcePlan; - isEnableAndActivate = other916.isEnableAndActivate; - __isset = other916.__isset; -} -WMAlterResourcePlanRequest& WMAlterResourcePlanRequest::operator=(const WMAlterResourcePlanRequest& other917) { - resourcePlanName = other917.resourcePlanName; - resourcePlan = other917.resourcePlan; - isEnableAndActivate = other917.isEnableAndActivate; - __isset = other917.__isset; - return *this; -} -void WMAlterResourcePlanRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMAlterResourcePlanRequest("; - out << "resourcePlanName="; (__isset.resourcePlanName ? (out << to_string(resourcePlanName)) : (out << "")); - out << ", " << "resourcePlan="; (__isset.resourcePlan ? (out << to_string(resourcePlan)) : (out << "")); - out << ", " << "isEnableAndActivate="; (__isset.isEnableAndActivate ? (out << to_string(isEnableAndActivate)) : (out << "")); - out << ")"; -} - - -WMAlterResourcePlanResponse::~WMAlterResourcePlanResponse() noexcept { -} - - -void WMAlterResourcePlanResponse::__set_fullResourcePlan(const WMFullResourcePlan& val) { - this->fullResourcePlan = val; -__isset.fullResourcePlan = true; -} -std::ostream& operator<<(std::ostream& out, const WMAlterResourcePlanResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMAlterResourcePlanResponse::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->fullResourcePlan.read(iprot); - this->__isset.fullResourcePlan = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t WMAlterResourcePlanResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMAlterResourcePlanResponse"); - - if (this->__isset.fullResourcePlan) { - xfer += oprot->writeFieldBegin("fullResourcePlan", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->fullResourcePlan.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMAlterResourcePlanResponse &a, WMAlterResourcePlanResponse &b) { - using ::std::swap; - swap(a.fullResourcePlan, b.fullResourcePlan); - swap(a.__isset, b.__isset); -} - -WMAlterResourcePlanResponse::WMAlterResourcePlanResponse(const WMAlterResourcePlanResponse& other918) { - fullResourcePlan = other918.fullResourcePlan; - __isset = other918.__isset; -} -WMAlterResourcePlanResponse& WMAlterResourcePlanResponse::operator=(const WMAlterResourcePlanResponse& other919) { - fullResourcePlan = other919.fullResourcePlan; - __isset = other919.__isset; - return *this; -} -void WMAlterResourcePlanResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMAlterResourcePlanResponse("; - out << "fullResourcePlan="; (__isset.fullResourcePlan ? (out << to_string(fullResourcePlan)) : (out << "")); - out << ")"; -} - - -WMValidateResourcePlanRequest::~WMValidateResourcePlanRequest() noexcept { -} - - -void WMValidateResourcePlanRequest::__set_resourcePlanName(const std::string& val) { - this->resourcePlanName = val; -__isset.resourcePlanName = true; -} -std::ostream& operator<<(std::ostream& out, const WMValidateResourcePlanRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMValidateResourcePlanRequest::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->resourcePlanName); - this->__isset.resourcePlanName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t WMValidateResourcePlanRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMValidateResourcePlanRequest"); - - if (this->__isset.resourcePlanName) { - xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->resourcePlanName); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMValidateResourcePlanRequest &a, WMValidateResourcePlanRequest &b) { - using ::std::swap; - swap(a.resourcePlanName, b.resourcePlanName); - swap(a.__isset, b.__isset); -} - -WMValidateResourcePlanRequest::WMValidateResourcePlanRequest(const WMValidateResourcePlanRequest& other920) { - resourcePlanName = other920.resourcePlanName; - __isset = other920.__isset; -} -WMValidateResourcePlanRequest& WMValidateResourcePlanRequest::operator=(const WMValidateResourcePlanRequest& other921) { - resourcePlanName = other921.resourcePlanName; - __isset = other921.__isset; - return *this; -} -void WMValidateResourcePlanRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMValidateResourcePlanRequest("; - out << "resourcePlanName="; (__isset.resourcePlanName ? (out << to_string(resourcePlanName)) : (out << "")); - out << ")"; -} - - -WMValidateResourcePlanResponse::~WMValidateResourcePlanResponse() noexcept { -} - - -void WMValidateResourcePlanResponse::__set_isValid(const bool val) { - this->isValid = val; -__isset.isValid = true; -} -std::ostream& operator<<(std::ostream& out, const WMValidateResourcePlanResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMValidateResourcePlanResponse::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->isValid); - this->__isset.isValid = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t WMValidateResourcePlanResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMValidateResourcePlanResponse"); - - if (this->__isset.isValid) { - xfer += oprot->writeFieldBegin("isValid", ::apache::thrift::protocol::T_BOOL, 1); - xfer += oprot->writeBool(this->isValid); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMValidateResourcePlanResponse &a, WMValidateResourcePlanResponse &b) { - using ::std::swap; - swap(a.isValid, b.isValid); - swap(a.__isset, b.__isset); -} - -WMValidateResourcePlanResponse::WMValidateResourcePlanResponse(const WMValidateResourcePlanResponse& other922) { - isValid = other922.isValid; - __isset = other922.__isset; -} -WMValidateResourcePlanResponse& WMValidateResourcePlanResponse::operator=(const WMValidateResourcePlanResponse& other923) { - isValid = other923.isValid; - __isset = other923.__isset; - return *this; -} -void WMValidateResourcePlanResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMValidateResourcePlanResponse("; - out << "isValid="; (__isset.isValid ? (out << to_string(isValid)) : (out << "")); - out << ")"; -} - - -WMDropResourcePlanRequest::~WMDropResourcePlanRequest() noexcept { -} - - -void WMDropResourcePlanRequest::__set_resourcePlanName(const std::string& val) { - this->resourcePlanName = val; -__isset.resourcePlanName = true; -} -std::ostream& operator<<(std::ostream& out, const WMDropResourcePlanRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMDropResourcePlanRequest::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->resourcePlanName); - this->__isset.resourcePlanName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t WMDropResourcePlanRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMDropResourcePlanRequest"); - - if (this->__isset.resourcePlanName) { - xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->resourcePlanName); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMDropResourcePlanRequest &a, WMDropResourcePlanRequest &b) { - using ::std::swap; - swap(a.resourcePlanName, b.resourcePlanName); - swap(a.__isset, b.__isset); -} - -WMDropResourcePlanRequest::WMDropResourcePlanRequest(const WMDropResourcePlanRequest& other924) { - resourcePlanName = other924.resourcePlanName; - __isset = other924.__isset; -} -WMDropResourcePlanRequest& WMDropResourcePlanRequest::operator=(const WMDropResourcePlanRequest& other925) { - resourcePlanName = other925.resourcePlanName; - __isset = other925.__isset; - return *this; -} -void WMDropResourcePlanRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMDropResourcePlanRequest("; - out << "resourcePlanName="; (__isset.resourcePlanName ? (out << to_string(resourcePlanName)) : (out << "")); - out << ")"; -} - - -WMDropResourcePlanResponse::~WMDropResourcePlanResponse() noexcept { -} - -std::ostream& operator<<(std::ostream& out, const WMDropResourcePlanResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMDropResourcePlanResponse::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 WMDropResourcePlanResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMDropResourcePlanResponse"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMDropResourcePlanResponse &a, WMDropResourcePlanResponse &b) { - using ::std::swap; - (void) a; - (void) b; -} - -WMDropResourcePlanResponse::WMDropResourcePlanResponse(const WMDropResourcePlanResponse& other926) { - (void) other926; -} -WMDropResourcePlanResponse& WMDropResourcePlanResponse::operator=(const WMDropResourcePlanResponse& other927) { - (void) other927; - return *this; -} -void WMDropResourcePlanResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMDropResourcePlanResponse("; - out << ")"; -} - - -WMCreateTriggerRequest::~WMCreateTriggerRequest() noexcept { -} - - -void WMCreateTriggerRequest::__set_trigger(const WMTrigger& val) { - this->trigger = val; -__isset.trigger = true; -} -std::ostream& operator<<(std::ostream& out, const WMCreateTriggerRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMCreateTriggerRequest::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->trigger.read(iprot); - this->__isset.trigger = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t WMCreateTriggerRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMCreateTriggerRequest"); - - if (this->__isset.trigger) { - xfer += oprot->writeFieldBegin("trigger", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->trigger.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMCreateTriggerRequest &a, WMCreateTriggerRequest &b) { - using ::std::swap; - swap(a.trigger, b.trigger); - swap(a.__isset, b.__isset); -} - -WMCreateTriggerRequest::WMCreateTriggerRequest(const WMCreateTriggerRequest& other928) { - trigger = other928.trigger; - __isset = other928.__isset; -} -WMCreateTriggerRequest& WMCreateTriggerRequest::operator=(const WMCreateTriggerRequest& other929) { - trigger = other929.trigger; - __isset = other929.__isset; - return *this; -} -void WMCreateTriggerRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMCreateTriggerRequest("; - out << "trigger="; (__isset.trigger ? (out << to_string(trigger)) : (out << "")); - out << ")"; -} - - -WMCreateTriggerResponse::~WMCreateTriggerResponse() noexcept { -} - -std::ostream& operator<<(std::ostream& out, const WMCreateTriggerResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMCreateTriggerResponse::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 WMCreateTriggerResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMCreateTriggerResponse"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMCreateTriggerResponse &a, WMCreateTriggerResponse &b) { - using ::std::swap; - (void) a; - (void) b; -} - -WMCreateTriggerResponse::WMCreateTriggerResponse(const WMCreateTriggerResponse& other930) { - (void) other930; -} -WMCreateTriggerResponse& WMCreateTriggerResponse::operator=(const WMCreateTriggerResponse& other931) { - (void) other931; - return *this; -} -void WMCreateTriggerResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMCreateTriggerResponse("; - out << ")"; -} - - -WMAlterTriggerRequest::~WMAlterTriggerRequest() noexcept { -} - - -void WMAlterTriggerRequest::__set_trigger(const WMTrigger& val) { - this->trigger = val; -__isset.trigger = true; -} -std::ostream& operator<<(std::ostream& out, const WMAlterTriggerRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMAlterTriggerRequest::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->trigger.read(iprot); - this->__isset.trigger = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t WMAlterTriggerRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMAlterTriggerRequest"); - - if (this->__isset.trigger) { - xfer += oprot->writeFieldBegin("trigger", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->trigger.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMAlterTriggerRequest &a, WMAlterTriggerRequest &b) { - using ::std::swap; - swap(a.trigger, b.trigger); - swap(a.__isset, b.__isset); -} - -WMAlterTriggerRequest::WMAlterTriggerRequest(const WMAlterTriggerRequest& other932) { - trigger = other932.trigger; - __isset = other932.__isset; -} -WMAlterTriggerRequest& WMAlterTriggerRequest::operator=(const WMAlterTriggerRequest& other933) { - trigger = other933.trigger; - __isset = other933.__isset; - return *this; -} -void WMAlterTriggerRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMAlterTriggerRequest("; - out << "trigger="; (__isset.trigger ? (out << to_string(trigger)) : (out << "")); - out << ")"; -} - - -WMAlterTriggerResponse::~WMAlterTriggerResponse() noexcept { -} - -std::ostream& operator<<(std::ostream& out, const WMAlterTriggerResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMAlterTriggerResponse::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 WMAlterTriggerResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMAlterTriggerResponse"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMAlterTriggerResponse &a, WMAlterTriggerResponse &b) { - using ::std::swap; - (void) a; - (void) b; -} - -WMAlterTriggerResponse::WMAlterTriggerResponse(const WMAlterTriggerResponse& other934) { - (void) other934; -} -WMAlterTriggerResponse& WMAlterTriggerResponse::operator=(const WMAlterTriggerResponse& other935) { - (void) other935; - return *this; -} -void WMAlterTriggerResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMAlterTriggerResponse("; - out << ")"; -} - - -WMDropTriggerRequest::~WMDropTriggerRequest() noexcept { -} - - -void WMDropTriggerRequest::__set_resourcePlanName(const std::string& val) { - this->resourcePlanName = val; -__isset.resourcePlanName = true; -} - -void WMDropTriggerRequest::__set_triggerName(const std::string& val) { - this->triggerName = val; -__isset.triggerName = true; -} -std::ostream& operator<<(std::ostream& out, const WMDropTriggerRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMDropTriggerRequest::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->resourcePlanName); - this->__isset.resourcePlanName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->triggerName); - this->__isset.triggerName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t WMDropTriggerRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMDropTriggerRequest"); - - if (this->__isset.resourcePlanName) { - xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->resourcePlanName); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.triggerName) { - xfer += oprot->writeFieldBegin("triggerName", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->triggerName); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMDropTriggerRequest &a, WMDropTriggerRequest &b) { - using ::std::swap; - swap(a.resourcePlanName, b.resourcePlanName); - swap(a.triggerName, b.triggerName); - swap(a.__isset, b.__isset); -} - -WMDropTriggerRequest::WMDropTriggerRequest(const WMDropTriggerRequest& other936) { - resourcePlanName = other936.resourcePlanName; - triggerName = other936.triggerName; - __isset = other936.__isset; -} -WMDropTriggerRequest& WMDropTriggerRequest::operator=(const WMDropTriggerRequest& other937) { - resourcePlanName = other937.resourcePlanName; - triggerName = other937.triggerName; - __isset = other937.__isset; - return *this; -} -void WMDropTriggerRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMDropTriggerRequest("; - out << "resourcePlanName="; (__isset.resourcePlanName ? (out << to_string(resourcePlanName)) : (out << "")); - out << ", " << "triggerName="; (__isset.triggerName ? (out << to_string(triggerName)) : (out << "")); - out << ")"; -} - - -WMDropTriggerResponse::~WMDropTriggerResponse() noexcept { -} - -std::ostream& operator<<(std::ostream& out, const WMDropTriggerResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMDropTriggerResponse::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 WMDropTriggerResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMDropTriggerResponse"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMDropTriggerResponse &a, WMDropTriggerResponse &b) { - using ::std::swap; - (void) a; - (void) b; -} - -WMDropTriggerResponse::WMDropTriggerResponse(const WMDropTriggerResponse& other938) { - (void) other938; -} -WMDropTriggerResponse& WMDropTriggerResponse::operator=(const WMDropTriggerResponse& other939) { - (void) other939; - return *this; -} -void WMDropTriggerResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMDropTriggerResponse("; - out << ")"; -} - - -WMGetTriggersForResourePlanRequest::~WMGetTriggersForResourePlanRequest() noexcept { -} - - -void WMGetTriggersForResourePlanRequest::__set_resourcePlanName(const std::string& val) { - this->resourcePlanName = val; -__isset.resourcePlanName = true; -} -std::ostream& operator<<(std::ostream& out, const WMGetTriggersForResourePlanRequest& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMGetTriggersForResourePlanRequest::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->resourcePlanName); - this->__isset.resourcePlanName = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t WMGetTriggersForResourePlanRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMGetTriggersForResourePlanRequest"); - - if (this->__isset.resourcePlanName) { - xfer += oprot->writeFieldBegin("resourcePlanName", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->resourcePlanName); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMGetTriggersForResourePlanRequest &a, WMGetTriggersForResourePlanRequest &b) { - using ::std::swap; - swap(a.resourcePlanName, b.resourcePlanName); - swap(a.__isset, b.__isset); -} - -WMGetTriggersForResourePlanRequest::WMGetTriggersForResourePlanRequest(const WMGetTriggersForResourePlanRequest& other940) { - resourcePlanName = other940.resourcePlanName; - __isset = other940.__isset; -} -WMGetTriggersForResourePlanRequest& WMGetTriggersForResourePlanRequest::operator=(const WMGetTriggersForResourePlanRequest& other941) { - resourcePlanName = other941.resourcePlanName; - __isset = other941.__isset; - return *this; -} -void WMGetTriggersForResourePlanRequest::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMGetTriggersForResourePlanRequest("; - out << "resourcePlanName="; (__isset.resourcePlanName ? (out << to_string(resourcePlanName)) : (out << "")); - out << ")"; -} - - -WMGetTriggersForResourePlanResponse::~WMGetTriggersForResourePlanResponse() noexcept { -} - - -void WMGetTriggersForResourePlanResponse::__set_triggers(const std::vector & val) { - this->triggers = val; -__isset.triggers = true; -} -std::ostream& operator<<(std::ostream& out, const WMGetTriggersForResourePlanResponse& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t WMGetTriggersForResourePlanResponse::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->triggers.clear(); - uint32_t _size942; - ::apache::thrift::protocol::TType _etype945; - xfer += iprot->readListBegin(_etype945, _size942); - this->triggers.resize(_size942); - uint32_t _i946; - for (_i946 = 0; _i946 < _size942; ++_i946) - { - xfer += this->triggers[_i946].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.triggers = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t WMGetTriggersForResourePlanResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("WMGetTriggersForResourePlanResponse"); - - if (this->__isset.triggers) { - xfer += oprot->writeFieldBegin("triggers", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->triggers.size())); - std::vector ::const_iterator _iter947; - for (_iter947 = this->triggers.begin(); _iter947 != this->triggers.end(); ++_iter947) - { - xfer += (*_iter947).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(WMGetTriggersForResourePlanResponse &a, WMGetTriggersForResourePlanResponse &b) { - using ::std::swap; - swap(a.triggers, b.triggers); - swap(a.__isset, b.__isset); -} - -WMGetTriggersForResourePlanResponse::WMGetTriggersForResourePlanResponse(const WMGetTriggersForResourePlanResponse& other948) { - triggers = other948.triggers; - __isset = other948.__isset; -} -WMGetTriggersForResourePlanResponse& WMGetTriggersForResourePlanResponse::operator=(const WMGetTriggersForResourePlanResponse& other949) { - triggers = other949.triggers; - __isset = other949.__isset; - return *this; -} -void WMGetTriggersForResourePlanResponse::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "WMGetTriggersForResourePlanResponse("; - out << "triggers="; (__isset.triggers ? (out << to_string(triggers)) : (out << "")); - out << ")"; -} - - -MetaException::~MetaException() noexcept { -} - - -void MetaException::__set_message(const std::string& val) { - this->message = val; -} -std::ostream& operator<<(std::ostream& out, const MetaException& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t MetaException::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t MetaException::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("MetaException"); - - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(MetaException &a, MetaException &b) { - using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); -} - -MetaException::MetaException(const MetaException& other950) : TException() { - message = other950.message; - __isset = other950.__isset; -} -MetaException& MetaException::operator=(const MetaException& other951) { - message = other951.message; - __isset = other951.__isset; - return *this; -} -void MetaException::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "MetaException("; - out << "message=" << to_string(message); - out << ")"; -} - -const char* MetaException::what() const noexcept { - try { - std::stringstream ss; - ss << "TException - service has thrown: " << *this; - this->thriftTExceptionMessageHolder_ = ss.str(); - return this->thriftTExceptionMessageHolder_.c_str(); - } catch (const std::exception&) { - return "TException - service has thrown: MetaException"; - } -} - - -UnknownTableException::~UnknownTableException() noexcept { -} - - -void UnknownTableException::__set_message(const std::string& val) { - this->message = val; -} -std::ostream& operator<<(std::ostream& out, const UnknownTableException& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t UnknownTableException::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t UnknownTableException::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("UnknownTableException"); - - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(UnknownTableException &a, UnknownTableException &b) { - using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); -} - -UnknownTableException::UnknownTableException(const UnknownTableException& other952) : TException() { - message = other952.message; - __isset = other952.__isset; -} -UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other953) { - message = other953.message; - __isset = other953.__isset; - return *this; -} -void UnknownTableException::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "UnknownTableException("; - out << "message=" << to_string(message); - out << ")"; -} - -const char* UnknownTableException::what() const noexcept { - try { - std::stringstream ss; - ss << "TException - service has thrown: " << *this; - this->thriftTExceptionMessageHolder_ = ss.str(); - return this->thriftTExceptionMessageHolder_.c_str(); - } catch (const std::exception&) { - return "TException - service has thrown: UnknownTableException"; - } -} - - -UnknownDBException::~UnknownDBException() noexcept { -} - - -void UnknownDBException::__set_message(const std::string& val) { - this->message = val; -} -std::ostream& operator<<(std::ostream& out, const UnknownDBException& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t UnknownDBException::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t UnknownDBException::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("UnknownDBException"); - - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(UnknownDBException &a, UnknownDBException &b) { - using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); -} - -UnknownDBException::UnknownDBException(const UnknownDBException& other954) : TException() { - message = other954.message; - __isset = other954.__isset; -} -UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other955) { - message = other955.message; - __isset = other955.__isset; - return *this; -} -void UnknownDBException::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "UnknownDBException("; - out << "message=" << to_string(message); - out << ")"; -} - -const char* UnknownDBException::what() const noexcept { - try { - std::stringstream ss; - ss << "TException - service has thrown: " << *this; - this->thriftTExceptionMessageHolder_ = ss.str(); - return this->thriftTExceptionMessageHolder_.c_str(); - } catch (const std::exception&) { - return "TException - service has thrown: UnknownDBException"; - } -} - - -AlreadyExistsException::~AlreadyExistsException() noexcept { -} - - -void AlreadyExistsException::__set_message(const std::string& val) { - this->message = val; -} -std::ostream& operator<<(std::ostream& out, const AlreadyExistsException& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t AlreadyExistsException::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t AlreadyExistsException::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("AlreadyExistsException"); - - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(AlreadyExistsException &a, AlreadyExistsException &b) { - using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); -} - -AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other956) : TException() { - message = other956.message; - __isset = other956.__isset; -} -AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other957) { - message = other957.message; - __isset = other957.__isset; - return *this; -} -void AlreadyExistsException::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "AlreadyExistsException("; - out << "message=" << to_string(message); - out << ")"; -} - -const char* AlreadyExistsException::what() const noexcept { - try { - std::stringstream ss; - ss << "TException - service has thrown: " << *this; - this->thriftTExceptionMessageHolder_ = ss.str(); - return this->thriftTExceptionMessageHolder_.c_str(); - } catch (const std::exception&) { - return "TException - service has thrown: AlreadyExistsException"; - } -} - - -InvalidPartitionException::~InvalidPartitionException() noexcept { -} - - -void InvalidPartitionException::__set_message(const std::string& val) { - this->message = val; -} -std::ostream& operator<<(std::ostream& out, const InvalidPartitionException& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t InvalidPartitionException::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t InvalidPartitionException::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("InvalidPartitionException"); - - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(InvalidPartitionException &a, InvalidPartitionException &b) { - using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); -} - -InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other958) : TException() { - message = other958.message; - __isset = other958.__isset; -} -InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other959) { - message = other959.message; - __isset = other959.__isset; - return *this; -} -void InvalidPartitionException::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "InvalidPartitionException("; - out << "message=" << to_string(message); - out << ")"; -} - -const char* InvalidPartitionException::what() const noexcept { - try { - std::stringstream ss; - ss << "TException - service has thrown: " << *this; - this->thriftTExceptionMessageHolder_ = ss.str(); - return this->thriftTExceptionMessageHolder_.c_str(); - } catch (const std::exception&) { - return "TException - service has thrown: InvalidPartitionException"; - } -} - - -UnknownPartitionException::~UnknownPartitionException() noexcept { -} - - -void UnknownPartitionException::__set_message(const std::string& val) { - this->message = val; -} -std::ostream& operator<<(std::ostream& out, const UnknownPartitionException& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t UnknownPartitionException::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t UnknownPartitionException::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("UnknownPartitionException"); - - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(UnknownPartitionException &a, UnknownPartitionException &b) { - using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); -} - -UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other960) : TException() { - message = other960.message; - __isset = other960.__isset; -} -UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other961) { - message = other961.message; - __isset = other961.__isset; - return *this; -} -void UnknownPartitionException::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "UnknownPartitionException("; - out << "message=" << to_string(message); - out << ")"; -} - -const char* UnknownPartitionException::what() const noexcept { - try { - std::stringstream ss; - ss << "TException - service has thrown: " << *this; - this->thriftTExceptionMessageHolder_ = ss.str(); - return this->thriftTExceptionMessageHolder_.c_str(); - } catch (const std::exception&) { - return "TException - service has thrown: UnknownPartitionException"; - } -} - - -InvalidObjectException::~InvalidObjectException() noexcept { -} - - -void InvalidObjectException::__set_message(const std::string& val) { - this->message = val; -} -std::ostream& operator<<(std::ostream& out, const InvalidObjectException& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t InvalidObjectException::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t InvalidObjectException::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("InvalidObjectException"); - - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(InvalidObjectException &a, InvalidObjectException &b) { - using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); -} - -InvalidObjectException::InvalidObjectException(const InvalidObjectException& other962) : TException() { - message = other962.message; - __isset = other962.__isset; -} -InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other963) { - message = other963.message; - __isset = other963.__isset; - return *this; -} -void InvalidObjectException::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "InvalidObjectException("; - out << "message=" << to_string(message); - out << ")"; -} - -const char* InvalidObjectException::what() const noexcept { - try { - std::stringstream ss; - ss << "TException - service has thrown: " << *this; - this->thriftTExceptionMessageHolder_ = ss.str(); - return this->thriftTExceptionMessageHolder_.c_str(); - } catch (const std::exception&) { - return "TException - service has thrown: InvalidObjectException"; - } -} - - -NoSuchObjectException::~NoSuchObjectException() noexcept { -} - - -void NoSuchObjectException::__set_message(const std::string& val) { - this->message = val; -} -std::ostream& operator<<(std::ostream& out, const NoSuchObjectException& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t NoSuchObjectException::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t NoSuchObjectException::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("NoSuchObjectException"); - - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(NoSuchObjectException &a, NoSuchObjectException &b) { - using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); -} - -NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other964) : TException() { - message = other964.message; - __isset = other964.__isset; -} -NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other965) { - message = other965.message; - __isset = other965.__isset; - return *this; -} -void NoSuchObjectException::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "NoSuchObjectException("; - out << "message=" << to_string(message); - out << ")"; -} - -const char* NoSuchObjectException::what() const noexcept { - try { - std::stringstream ss; - ss << "TException - service has thrown: " << *this; - this->thriftTExceptionMessageHolder_ = ss.str(); - return this->thriftTExceptionMessageHolder_.c_str(); - } catch (const std::exception&) { - return "TException - service has thrown: NoSuchObjectException"; - } -} - - -IndexAlreadyExistsException::~IndexAlreadyExistsException() noexcept { -} - - -void IndexAlreadyExistsException::__set_message(const std::string& val) { - this->message = val; -} -std::ostream& operator<<(std::ostream& out, const IndexAlreadyExistsException& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t IndexAlreadyExistsException::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t IndexAlreadyExistsException::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("IndexAlreadyExistsException"); - - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b) { - using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); -} - -IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other966) : TException() { - message = other966.message; - __isset = other966.__isset; -} -IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other967) { - message = other967.message; - __isset = other967.__isset; - return *this; -} -void IndexAlreadyExistsException::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "IndexAlreadyExistsException("; - out << "message=" << to_string(message); - out << ")"; -} - -const char* IndexAlreadyExistsException::what() const noexcept { - try { - std::stringstream ss; - ss << "TException - service has thrown: " << *this; - this->thriftTExceptionMessageHolder_ = ss.str(); - return this->thriftTExceptionMessageHolder_.c_str(); - } catch (const std::exception&) { - return "TException - service has thrown: IndexAlreadyExistsException"; - } -} - - -InvalidOperationException::~InvalidOperationException() noexcept { -} - - -void InvalidOperationException::__set_message(const std::string& val) { - this->message = val; -} -std::ostream& operator<<(std::ostream& out, const InvalidOperationException& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t InvalidOperationException::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t InvalidOperationException::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("InvalidOperationException"); - - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(InvalidOperationException &a, InvalidOperationException &b) { - using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); -} - -InvalidOperationException::InvalidOperationException(const InvalidOperationException& other968) : TException() { - message = other968.message; - __isset = other968.__isset; -} -InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other969) { - message = other969.message; - __isset = other969.__isset; - return *this; -} -void InvalidOperationException::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "InvalidOperationException("; - out << "message=" << to_string(message); - out << ")"; -} - -const char* InvalidOperationException::what() const noexcept { - try { - std::stringstream ss; - ss << "TException - service has thrown: " << *this; - this->thriftTExceptionMessageHolder_ = ss.str(); - return this->thriftTExceptionMessageHolder_.c_str(); - } catch (const std::exception&) { - return "TException - service has thrown: InvalidOperationException"; - } -} - - -ConfigValSecurityException::~ConfigValSecurityException() noexcept { -} - - -void ConfigValSecurityException::__set_message(const std::string& val) { - this->message = val; -} -std::ostream& operator<<(std::ostream& out, const ConfigValSecurityException& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ConfigValSecurityException::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ConfigValSecurityException::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ConfigValSecurityException"); - - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) { - using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); -} - -ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other970) : TException() { - message = other970.message; - __isset = other970.__isset; -} -ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other971) { - message = other971.message; - __isset = other971.__isset; - return *this; -} -void ConfigValSecurityException::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ConfigValSecurityException("; - out << "message=" << to_string(message); - out << ")"; -} - -const char* ConfigValSecurityException::what() const noexcept { - try { - std::stringstream ss; - ss << "TException - service has thrown: " << *this; - this->thriftTExceptionMessageHolder_ = ss.str(); - return this->thriftTExceptionMessageHolder_.c_str(); - } catch (const std::exception&) { - return "TException - service has thrown: ConfigValSecurityException"; - } -} - - -InvalidInputException::~InvalidInputException() noexcept { -} - - -void InvalidInputException::__set_message(const std::string& val) { - this->message = val; -} -std::ostream& operator<<(std::ostream& out, const InvalidInputException& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t InvalidInputException::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t InvalidInputException::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("InvalidInputException"); - - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(InvalidInputException &a, InvalidInputException &b) { - using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); -} - -InvalidInputException::InvalidInputException(const InvalidInputException& other972) : TException() { - message = other972.message; - __isset = other972.__isset; -} -InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other973) { - message = other973.message; - __isset = other973.__isset; - return *this; -} -void InvalidInputException::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "InvalidInputException("; - out << "message=" << to_string(message); - out << ")"; -} - -const char* InvalidInputException::what() const noexcept { - try { - std::stringstream ss; - ss << "TException - service has thrown: " << *this; - this->thriftTExceptionMessageHolder_ = ss.str(); - return this->thriftTExceptionMessageHolder_.c_str(); - } catch (const std::exception&) { - return "TException - service has thrown: InvalidInputException"; - } -} - - -NoSuchTxnException::~NoSuchTxnException() noexcept { -} - - -void NoSuchTxnException::__set_message(const std::string& val) { - this->message = val; -} -std::ostream& operator<<(std::ostream& out, const NoSuchTxnException& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t NoSuchTxnException::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t NoSuchTxnException::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("NoSuchTxnException"); - - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(NoSuchTxnException &a, NoSuchTxnException &b) { - using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); -} - -NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other974) : TException() { - message = other974.message; - __isset = other974.__isset; -} -NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other975) { - message = other975.message; - __isset = other975.__isset; - return *this; -} -void NoSuchTxnException::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "NoSuchTxnException("; - out << "message=" << to_string(message); - out << ")"; -} - -const char* NoSuchTxnException::what() const noexcept { - try { - std::stringstream ss; - ss << "TException - service has thrown: " << *this; - this->thriftTExceptionMessageHolder_ = ss.str(); - return this->thriftTExceptionMessageHolder_.c_str(); - } catch (const std::exception&) { - return "TException - service has thrown: NoSuchTxnException"; - } -} - - -TxnAbortedException::~TxnAbortedException() noexcept { -} - - -void TxnAbortedException::__set_message(const std::string& val) { - this->message = val; -} -std::ostream& operator<<(std::ostream& out, const TxnAbortedException& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t TxnAbortedException::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t TxnAbortedException::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("TxnAbortedException"); - - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(TxnAbortedException &a, TxnAbortedException &b) { - using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); -} - -TxnAbortedException::TxnAbortedException(const TxnAbortedException& other976) : TException() { - message = other976.message; - __isset = other976.__isset; -} -TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other977) { - message = other977.message; - __isset = other977.__isset; - return *this; -} -void TxnAbortedException::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "TxnAbortedException("; - out << "message=" << to_string(message); - out << ")"; -} - -const char* TxnAbortedException::what() const noexcept { - try { - std::stringstream ss; - ss << "TException - service has thrown: " << *this; - this->thriftTExceptionMessageHolder_ = ss.str(); - return this->thriftTExceptionMessageHolder_.c_str(); - } catch (const std::exception&) { - return "TException - service has thrown: TxnAbortedException"; - } -} - - -TxnOpenException::~TxnOpenException() noexcept { -} - - -void TxnOpenException::__set_message(const std::string& val) { - this->message = val; -} -std::ostream& operator<<(std::ostream& out, const TxnOpenException& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t TxnOpenException::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t TxnOpenException::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("TxnOpenException"); - - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(TxnOpenException &a, TxnOpenException &b) { - using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); -} - -TxnOpenException::TxnOpenException(const TxnOpenException& other978) : TException() { - message = other978.message; - __isset = other978.__isset; -} -TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other979) { - message = other979.message; - __isset = other979.__isset; - return *this; -} -void TxnOpenException::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "TxnOpenException("; - out << "message=" << to_string(message); - out << ")"; -} - -const char* TxnOpenException::what() const noexcept { - try { - std::stringstream ss; - ss << "TException - service has thrown: " << *this; - this->thriftTExceptionMessageHolder_ = ss.str(); - return this->thriftTExceptionMessageHolder_.c_str(); - } catch (const std::exception&) { - return "TException - service has thrown: TxnOpenException"; - } -} - - -NoSuchLockException::~NoSuchLockException() noexcept { -} - - -void NoSuchLockException::__set_message(const std::string& val) { - this->message = val; -} -std::ostream& operator<<(std::ostream& out, const NoSuchLockException& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t NoSuchLockException::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; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t NoSuchLockException::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("NoSuchLockException"); - - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(NoSuchLockException &a, NoSuchLockException &b) { - using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); -} - -NoSuchLockException::NoSuchLockException(const NoSuchLockException& other980) : TException() { - message = other980.message; - __isset = other980.__isset; -} -NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other981) { - message = other981.message; - __isset = other981.__isset; - return *this; -} -void NoSuchLockException::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "NoSuchLockException("; - out << "message=" << to_string(message); - out << ")"; -} - -const char* NoSuchLockException::what() const noexcept { - try { - std::stringstream ss; - ss << "TException - service has thrown: " << *this; - this->thriftTExceptionMessageHolder_ = ss.str(); - return this->thriftTExceptionMessageHolder_.c_str(); - } catch (const std::exception&) { - return "TException - service has thrown: NoSuchLockException"; - } -} - -}}} // namespace diff --git a/contrib/hive-metastore/hive_metastore_types.h b/contrib/hive-metastore/hive_metastore_types.h deleted file mode 100644 index ceb44329028..00000000000 --- a/contrib/hive-metastore/hive_metastore_types.h +++ /dev/null @@ -1,10002 +0,0 @@ -/** - * Autogenerated by Thrift Compiler () - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -#ifndef hive_metastore_TYPES_H -#define hive_metastore_TYPES_H - -#include - -#include -#include -#include -#include -#include - -#include -#include - - -namespace Apache { namespace Hadoop { namespace Hive { - -struct HiveObjectType { - enum type { - GLOBAL = 1, - DATABASE = 2, - TABLE = 3, - PARTITION = 4, - COLUMN = 5 - }; -}; - -extern const std::map _HiveObjectType_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const HiveObjectType::type& val); - -struct PrincipalType { - enum type { - USER = 1, - ROLE = 2, - GROUP = 3 - }; -}; - -extern const std::map _PrincipalType_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const PrincipalType::type& val); - -struct PartitionEventType { - enum type { - LOAD_DONE = 1 - }; -}; - -extern const std::map _PartitionEventType_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const PartitionEventType::type& val); - -struct TxnState { - enum type { - COMMITTED = 1, - ABORTED = 2, - OPEN = 3 - }; -}; - -extern const std::map _TxnState_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const TxnState::type& val); - -struct LockLevel { - enum type { - DB = 1, - TABLE = 2, - PARTITION = 3 - }; -}; - -extern const std::map _LockLevel_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const LockLevel::type& val); - -struct LockState { - enum type { - ACQUIRED = 1, - WAITING = 2, - ABORT = 3, - NOT_ACQUIRED = 4 - }; -}; - -extern const std::map _LockState_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const LockState::type& val); - -struct LockType { - enum type { - SHARED_READ = 1, - SHARED_WRITE = 2, - EXCLUSIVE = 3 - }; -}; - -extern const std::map _LockType_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const LockType::type& val); - -struct CompactionType { - enum type { - MINOR = 1, - MAJOR = 2 - }; -}; - -extern const std::map _CompactionType_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const CompactionType::type& val); - -struct GrantRevokeType { - enum type { - GRANT = 1, - REVOKE = 2 - }; -}; - -extern const std::map _GrantRevokeType_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const GrantRevokeType::type& val); - -struct DataOperationType { - enum type { - SELECT = 1, - INSERT = 2, - UPDATE = 3, - DELETE = 4, - UNSET = 5, - NO_TXN = 6 - }; -}; - -extern const std::map _DataOperationType_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const DataOperationType::type& val); - -struct EventRequestType { - enum type { - INSERT = 1, - UPDATE = 2, - DELETE = 3 - }; -}; - -extern const std::map _EventRequestType_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const EventRequestType::type& val); - -struct FunctionType { - enum type { - JAVA = 1 - }; -}; - -extern const std::map _FunctionType_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const FunctionType::type& val); - -struct ResourceType { - enum type { - JAR = 1, - FILE = 2, - ARCHIVE = 3 - }; -}; - -extern const std::map _ResourceType_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const ResourceType::type& val); - -struct FileMetadataExprType { - enum type { - ORC_SARG = 1 - }; -}; - -extern const std::map _FileMetadataExprType_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const FileMetadataExprType::type& val); - -struct ClientCapability { - enum type { - TEST_CAPABILITY = 1, - INSERT_ONLY_TABLES = 2 - }; -}; - -extern const std::map _ClientCapability_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const ClientCapability::type& val); - -struct WMResourcePlanStatus { - enum type { - ACTIVE = 1, - ENABLED = 2, - DISABLED = 3 - }; -}; - -extern const std::map _WMResourcePlanStatus_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const WMResourcePlanStatus::type& val); - -class Version; - -class FieldSchema; - -class SQLPrimaryKey; - -class SQLForeignKey; - -class SQLUniqueConstraint; - -class SQLNotNullConstraint; - -class Type; - -class HiveObjectRef; - -class PrivilegeGrantInfo; - -class HiveObjectPrivilege; - -class PrivilegeBag; - -class PrincipalPrivilegeSet; - -class GrantRevokePrivilegeRequest; - -class GrantRevokePrivilegeResponse; - -class Role; - -class RolePrincipalGrant; - -class GetRoleGrantsForPrincipalRequest; - -class GetRoleGrantsForPrincipalResponse; - -class GetPrincipalsInRoleRequest; - -class GetPrincipalsInRoleResponse; - -class GrantRevokeRoleRequest; - -class GrantRevokeRoleResponse; - -class Database; - -class SerDeInfo; - -class Order; - -class SkewedInfo; - -class StorageDescriptor; - -class Table; - -class Partition; - -class PartitionWithoutSD; - -class PartitionSpecWithSharedSD; - -class PartitionListComposingSpec; - -class PartitionSpec; - -class Index; - -class BooleanColumnStatsData; - -class DoubleColumnStatsData; - -class LongColumnStatsData; - -class StringColumnStatsData; - -class BinaryColumnStatsData; - -class Decimal; - -class DecimalColumnStatsData; - -class Date; - -class DateColumnStatsData; - -class ColumnStatisticsData; - -class ColumnStatisticsObj; - -class ColumnStatisticsDesc; - -class ColumnStatistics; - -class AggrStats; - -class SetPartitionsStatsRequest; - -class Schema; - -class EnvironmentContext; - -class PrimaryKeysRequest; - -class PrimaryKeysResponse; - -class ForeignKeysRequest; - -class ForeignKeysResponse; - -class UniqueConstraintsRequest; - -class UniqueConstraintsResponse; - -class NotNullConstraintsRequest; - -class NotNullConstraintsResponse; - -class DropConstraintRequest; - -class AddPrimaryKeyRequest; - -class AddForeignKeyRequest; - -class AddUniqueConstraintRequest; - -class AddNotNullConstraintRequest; - -class PartitionsByExprResult; - -class PartitionsByExprRequest; - -class TableStatsResult; - -class PartitionsStatsResult; - -class TableStatsRequest; - -class PartitionsStatsRequest; - -class AddPartitionsResult; - -class AddPartitionsRequest; - -class DropPartitionsResult; - -class DropPartitionsExpr; - -class RequestPartsSpec; - -class DropPartitionsRequest; - -class PartitionValuesRequest; - -class PartitionValuesRow; - -class PartitionValuesResponse; - -class ResourceUri; - -class Function; - -class TxnInfo; - -class GetOpenTxnsInfoResponse; - -class GetOpenTxnsResponse; - -class OpenTxnRequest; - -class OpenTxnsResponse; - -class AbortTxnRequest; - -class AbortTxnsRequest; - -class CommitTxnRequest; - -class LockComponent; - -class LockRequest; - -class LockResponse; - -class CheckLockRequest; - -class UnlockRequest; - -class ShowLocksRequest; - -class ShowLocksResponseElement; - -class ShowLocksResponse; - -class HeartbeatRequest; - -class HeartbeatTxnRangeRequest; - -class HeartbeatTxnRangeResponse; - -class CompactionRequest; - -class CompactionResponse; - -class ShowCompactRequest; - -class ShowCompactResponseElement; - -class ShowCompactResponse; - -class AddDynamicPartitions; - -class NotificationEventRequest; - -class NotificationEvent; - -class NotificationEventResponse; - -class CurrentNotificationEventId; - -class NotificationEventsCountRequest; - -class NotificationEventsCountResponse; - -class InsertEventRequestData; - -class FireEventRequestData; - -class FireEventRequest; - -class FireEventResponse; - -class MetadataPpdResult; - -class GetFileMetadataByExprResult; - -class GetFileMetadataByExprRequest; - -class GetFileMetadataResult; - -class GetFileMetadataRequest; - -class PutFileMetadataResult; - -class PutFileMetadataRequest; - -class ClearFileMetadataResult; - -class ClearFileMetadataRequest; - -class CacheFileMetadataResult; - -class CacheFileMetadataRequest; - -class GetAllFunctionsResponse; - -class ClientCapabilities; - -class GetTableRequest; - -class GetTableResult; - -class GetTablesRequest; - -class GetTablesResult; - -class CmRecycleRequest; - -class CmRecycleResponse; - -class TableMeta; - -class WMResourcePlan; - -class WMPool; - -class WMTrigger; - -class WMMapping; - -class WMPoolTrigger; - -class WMFullResourcePlan; - -class WMCreateResourcePlanRequest; - -class WMCreateResourcePlanResponse; - -class WMGetActiveResourcePlanRequest; - -class WMGetActiveResourcePlanResponse; - -class WMGetResourcePlanRequest; - -class WMGetResourcePlanResponse; - -class WMGetAllResourcePlanRequest; - -class WMGetAllResourcePlanResponse; - -class WMAlterResourcePlanRequest; - -class WMAlterResourcePlanResponse; - -class WMValidateResourcePlanRequest; - -class WMValidateResourcePlanResponse; - -class WMDropResourcePlanRequest; - -class WMDropResourcePlanResponse; - -class WMCreateTriggerRequest; - -class WMCreateTriggerResponse; - -class WMAlterTriggerRequest; - -class WMAlterTriggerResponse; - -class WMDropTriggerRequest; - -class WMDropTriggerResponse; - -class WMGetTriggersForResourePlanRequest; - -class WMGetTriggersForResourePlanResponse; - -class MetaException; - -class UnknownTableException; - -class UnknownDBException; - -class AlreadyExistsException; - -class InvalidPartitionException; - -class UnknownPartitionException; - -class InvalidObjectException; - -class NoSuchObjectException; - -class IndexAlreadyExistsException; - -class InvalidOperationException; - -class ConfigValSecurityException; - -class InvalidInputException; - -class NoSuchTxnException; - -class TxnAbortedException; - -class TxnOpenException; - -class NoSuchLockException; - -typedef struct _Version__isset { - _Version__isset() : version(false), comments(false) {} - bool version :1; - bool comments :1; -} _Version__isset; - -class Version : public virtual ::apache::thrift::TBase { - public: - - Version(const Version&); - Version& operator=(const Version&); - Version() : version(), comments() { - } - - virtual ~Version() noexcept; - std::string version; - std::string comments; - - _Version__isset __isset; - - void __set_version(const std::string& val); - - void __set_comments(const std::string& val); - - bool operator == (const Version & rhs) const - { - if (!(version == rhs.version)) - return false; - if (!(comments == rhs.comments)) - return false; - return true; - } - bool operator != (const Version &rhs) const { - return !(*this == rhs); - } - - bool operator < (const Version & ) 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(Version &a, Version &b); - -std::ostream& operator<<(std::ostream& out, const Version& obj); - -typedef struct _FieldSchema__isset { - _FieldSchema__isset() : name(false), type(false), comment(false) {} - bool name :1; - bool type :1; - bool comment :1; -} _FieldSchema__isset; - -class FieldSchema : public virtual ::apache::thrift::TBase { - public: - - FieldSchema(const FieldSchema&); - FieldSchema& operator=(const FieldSchema&); - FieldSchema() : name(), type(), comment() { - } - - virtual ~FieldSchema() noexcept; - std::string name; - std::string type; - std::string comment; - - _FieldSchema__isset __isset; - - void __set_name(const std::string& val); - - void __set_type(const std::string& val); - - void __set_comment(const std::string& val); - - bool operator == (const FieldSchema & rhs) const - { - if (!(name == rhs.name)) - return false; - if (!(type == rhs.type)) - return false; - if (!(comment == rhs.comment)) - return false; - return true; - } - bool operator != (const FieldSchema &rhs) const { - return !(*this == rhs); - } - - bool operator < (const FieldSchema & ) 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(FieldSchema &a, FieldSchema &b); - -std::ostream& operator<<(std::ostream& out, const FieldSchema& obj); - -typedef struct _SQLPrimaryKey__isset { - _SQLPrimaryKey__isset() : table_db(false), table_name(false), column_name(false), key_seq(false), pk_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {} - bool table_db :1; - bool table_name :1; - bool column_name :1; - bool key_seq :1; - bool pk_name :1; - bool enable_cstr :1; - bool validate_cstr :1; - bool rely_cstr :1; -} _SQLPrimaryKey__isset; - -class SQLPrimaryKey : public virtual ::apache::thrift::TBase { - public: - - SQLPrimaryKey(const SQLPrimaryKey&); - SQLPrimaryKey& operator=(const SQLPrimaryKey&); - SQLPrimaryKey() : table_db(), table_name(), column_name(), key_seq(0), pk_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) { - } - - virtual ~SQLPrimaryKey() noexcept; - std::string table_db; - std::string table_name; - std::string column_name; - int32_t key_seq; - std::string pk_name; - bool enable_cstr; - bool validate_cstr; - bool rely_cstr; - - _SQLPrimaryKey__isset __isset; - - void __set_table_db(const std::string& val); - - void __set_table_name(const std::string& val); - - void __set_column_name(const std::string& val); - - void __set_key_seq(const int32_t val); - - void __set_pk_name(const std::string& val); - - void __set_enable_cstr(const bool val); - - void __set_validate_cstr(const bool val); - - void __set_rely_cstr(const bool val); - - bool operator == (const SQLPrimaryKey & rhs) const - { - if (!(table_db == rhs.table_db)) - return false; - if (!(table_name == rhs.table_name)) - return false; - if (!(column_name == rhs.column_name)) - return false; - if (!(key_seq == rhs.key_seq)) - return false; - if (!(pk_name == rhs.pk_name)) - return false; - if (!(enable_cstr == rhs.enable_cstr)) - return false; - if (!(validate_cstr == rhs.validate_cstr)) - return false; - if (!(rely_cstr == rhs.rely_cstr)) - return false; - return true; - } - bool operator != (const SQLPrimaryKey &rhs) const { - return !(*this == rhs); - } - - bool operator < (const SQLPrimaryKey & ) 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(SQLPrimaryKey &a, SQLPrimaryKey &b); - -std::ostream& operator<<(std::ostream& out, const SQLPrimaryKey& obj); - -typedef struct _SQLForeignKey__isset { - _SQLForeignKey__isset() : pktable_db(false), pktable_name(false), pkcolumn_name(false), fktable_db(false), fktable_name(false), fkcolumn_name(false), key_seq(false), update_rule(false), delete_rule(false), fk_name(false), pk_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {} - bool pktable_db :1; - bool pktable_name :1; - bool pkcolumn_name :1; - bool fktable_db :1; - bool fktable_name :1; - bool fkcolumn_name :1; - bool key_seq :1; - bool update_rule :1; - bool delete_rule :1; - bool fk_name :1; - bool pk_name :1; - bool enable_cstr :1; - bool validate_cstr :1; - bool rely_cstr :1; -} _SQLForeignKey__isset; - -class SQLForeignKey : public virtual ::apache::thrift::TBase { - public: - - SQLForeignKey(const SQLForeignKey&); - SQLForeignKey& operator=(const SQLForeignKey&); - SQLForeignKey() : pktable_db(), pktable_name(), pkcolumn_name(), fktable_db(), fktable_name(), fkcolumn_name(), key_seq(0), update_rule(0), delete_rule(0), fk_name(), pk_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) { - } - - virtual ~SQLForeignKey() noexcept; - std::string pktable_db; - std::string pktable_name; - std::string pkcolumn_name; - std::string fktable_db; - std::string fktable_name; - std::string fkcolumn_name; - int32_t key_seq; - int32_t update_rule; - int32_t delete_rule; - std::string fk_name; - std::string pk_name; - bool enable_cstr; - bool validate_cstr; - bool rely_cstr; - - _SQLForeignKey__isset __isset; - - void __set_pktable_db(const std::string& val); - - void __set_pktable_name(const std::string& val); - - void __set_pkcolumn_name(const std::string& val); - - void __set_fktable_db(const std::string& val); - - void __set_fktable_name(const std::string& val); - - void __set_fkcolumn_name(const std::string& val); - - void __set_key_seq(const int32_t val); - - void __set_update_rule(const int32_t val); - - void __set_delete_rule(const int32_t val); - - void __set_fk_name(const std::string& val); - - void __set_pk_name(const std::string& val); - - void __set_enable_cstr(const bool val); - - void __set_validate_cstr(const bool val); - - void __set_rely_cstr(const bool val); - - bool operator == (const SQLForeignKey & rhs) const - { - if (!(pktable_db == rhs.pktable_db)) - return false; - if (!(pktable_name == rhs.pktable_name)) - return false; - if (!(pkcolumn_name == rhs.pkcolumn_name)) - return false; - if (!(fktable_db == rhs.fktable_db)) - return false; - if (!(fktable_name == rhs.fktable_name)) - return false; - if (!(fkcolumn_name == rhs.fkcolumn_name)) - return false; - if (!(key_seq == rhs.key_seq)) - return false; - if (!(update_rule == rhs.update_rule)) - return false; - if (!(delete_rule == rhs.delete_rule)) - return false; - if (!(fk_name == rhs.fk_name)) - return false; - if (!(pk_name == rhs.pk_name)) - return false; - if (!(enable_cstr == rhs.enable_cstr)) - return false; - if (!(validate_cstr == rhs.validate_cstr)) - return false; - if (!(rely_cstr == rhs.rely_cstr)) - return false; - return true; - } - bool operator != (const SQLForeignKey &rhs) const { - return !(*this == rhs); - } - - bool operator < (const SQLForeignKey & ) 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(SQLForeignKey &a, SQLForeignKey &b); - -std::ostream& operator<<(std::ostream& out, const SQLForeignKey& obj); - -typedef struct _SQLUniqueConstraint__isset { - _SQLUniqueConstraint__isset() : table_db(false), table_name(false), column_name(false), key_seq(false), uk_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {} - bool table_db :1; - bool table_name :1; - bool column_name :1; - bool key_seq :1; - bool uk_name :1; - bool enable_cstr :1; - bool validate_cstr :1; - bool rely_cstr :1; -} _SQLUniqueConstraint__isset; - -class SQLUniqueConstraint : public virtual ::apache::thrift::TBase { - public: - - SQLUniqueConstraint(const SQLUniqueConstraint&); - SQLUniqueConstraint& operator=(const SQLUniqueConstraint&); - SQLUniqueConstraint() : table_db(), table_name(), column_name(), key_seq(0), uk_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) { - } - - virtual ~SQLUniqueConstraint() noexcept; - std::string table_db; - std::string table_name; - std::string column_name; - int32_t key_seq; - std::string uk_name; - bool enable_cstr; - bool validate_cstr; - bool rely_cstr; - - _SQLUniqueConstraint__isset __isset; - - void __set_table_db(const std::string& val); - - void __set_table_name(const std::string& val); - - void __set_column_name(const std::string& val); - - void __set_key_seq(const int32_t val); - - void __set_uk_name(const std::string& val); - - void __set_enable_cstr(const bool val); - - void __set_validate_cstr(const bool val); - - void __set_rely_cstr(const bool val); - - bool operator == (const SQLUniqueConstraint & rhs) const - { - if (!(table_db == rhs.table_db)) - return false; - if (!(table_name == rhs.table_name)) - return false; - if (!(column_name == rhs.column_name)) - return false; - if (!(key_seq == rhs.key_seq)) - return false; - if (!(uk_name == rhs.uk_name)) - return false; - if (!(enable_cstr == rhs.enable_cstr)) - return false; - if (!(validate_cstr == rhs.validate_cstr)) - return false; - if (!(rely_cstr == rhs.rely_cstr)) - return false; - return true; - } - bool operator != (const SQLUniqueConstraint &rhs) const { - return !(*this == rhs); - } - - bool operator < (const SQLUniqueConstraint & ) 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(SQLUniqueConstraint &a, SQLUniqueConstraint &b); - -std::ostream& operator<<(std::ostream& out, const SQLUniqueConstraint& obj); - -typedef struct _SQLNotNullConstraint__isset { - _SQLNotNullConstraint__isset() : table_db(false), table_name(false), column_name(false), nn_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {} - bool table_db :1; - bool table_name :1; - bool column_name :1; - bool nn_name :1; - bool enable_cstr :1; - bool validate_cstr :1; - bool rely_cstr :1; -} _SQLNotNullConstraint__isset; - -class SQLNotNullConstraint : public virtual ::apache::thrift::TBase { - public: - - SQLNotNullConstraint(const SQLNotNullConstraint&); - SQLNotNullConstraint& operator=(const SQLNotNullConstraint&); - SQLNotNullConstraint() : table_db(), table_name(), column_name(), nn_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) { - } - - virtual ~SQLNotNullConstraint() noexcept; - std::string table_db; - std::string table_name; - std::string column_name; - std::string nn_name; - bool enable_cstr; - bool validate_cstr; - bool rely_cstr; - - _SQLNotNullConstraint__isset __isset; - - void __set_table_db(const std::string& val); - - void __set_table_name(const std::string& val); - - void __set_column_name(const std::string& val); - - void __set_nn_name(const std::string& val); - - void __set_enable_cstr(const bool val); - - void __set_validate_cstr(const bool val); - - void __set_rely_cstr(const bool val); - - bool operator == (const SQLNotNullConstraint & rhs) const - { - if (!(table_db == rhs.table_db)) - return false; - if (!(table_name == rhs.table_name)) - return false; - if (!(column_name == rhs.column_name)) - return false; - if (!(nn_name == rhs.nn_name)) - return false; - if (!(enable_cstr == rhs.enable_cstr)) - return false; - if (!(validate_cstr == rhs.validate_cstr)) - return false; - if (!(rely_cstr == rhs.rely_cstr)) - return false; - return true; - } - bool operator != (const SQLNotNullConstraint &rhs) const { - return !(*this == rhs); - } - - bool operator < (const SQLNotNullConstraint & ) 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(SQLNotNullConstraint &a, SQLNotNullConstraint &b); - -std::ostream& operator<<(std::ostream& out, const SQLNotNullConstraint& obj); - -typedef struct _Type__isset { - _Type__isset() : name(false), type1(false), type2(false), fields(false) {} - bool name :1; - bool type1 :1; - bool type2 :1; - bool fields :1; -} _Type__isset; - -class Type : public virtual ::apache::thrift::TBase { - public: - - Type(const Type&); - Type& operator=(const Type&); - Type() : name(), type1(), type2() { - } - - virtual ~Type() noexcept; - std::string name; - std::string type1; - std::string type2; - std::vector fields; - - _Type__isset __isset; - - void __set_name(const std::string& val); - - void __set_type1(const std::string& val); - - void __set_type2(const std::string& val); - - void __set_fields(const std::vector & val); - - bool operator == (const Type & rhs) const - { - if (!(name == rhs.name)) - return false; - if (__isset.type1 != rhs.__isset.type1) - return false; - else if (__isset.type1 && !(type1 == rhs.type1)) - return false; - if (__isset.type2 != rhs.__isset.type2) - return false; - else if (__isset.type2 && !(type2 == rhs.type2)) - return false; - if (__isset.fields != rhs.__isset.fields) - return false; - else if (__isset.fields && !(fields == rhs.fields)) - return false; - return true; - } - bool operator != (const Type &rhs) const { - return !(*this == rhs); - } - - bool operator < (const Type & ) 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(Type &a, Type &b); - -std::ostream& operator<<(std::ostream& out, const Type& obj); - -typedef struct _HiveObjectRef__isset { - _HiveObjectRef__isset() : objectType(false), dbName(false), objectName(false), partValues(false), columnName(false) {} - bool objectType :1; - bool dbName :1; - bool objectName :1; - bool partValues :1; - bool columnName :1; -} _HiveObjectRef__isset; - -class HiveObjectRef : public virtual ::apache::thrift::TBase { - public: - - HiveObjectRef(const HiveObjectRef&); - HiveObjectRef& operator=(const HiveObjectRef&); - HiveObjectRef() : objectType((HiveObjectType::type)0), dbName(), objectName(), columnName() { - } - - virtual ~HiveObjectRef() noexcept; - HiveObjectType::type objectType; - std::string dbName; - std::string objectName; - std::vector partValues; - std::string columnName; - - _HiveObjectRef__isset __isset; - - void __set_objectType(const HiveObjectType::type val); - - void __set_dbName(const std::string& val); - - void __set_objectName(const std::string& val); - - void __set_partValues(const std::vector & val); - - void __set_columnName(const std::string& val); - - bool operator == (const HiveObjectRef & rhs) const - { - if (!(objectType == rhs.objectType)) - return false; - if (!(dbName == rhs.dbName)) - return false; - if (!(objectName == rhs.objectName)) - return false; - if (!(partValues == rhs.partValues)) - return false; - if (!(columnName == rhs.columnName)) - return false; - return true; - } - bool operator != (const HiveObjectRef &rhs) const { - return !(*this == rhs); - } - - bool operator < (const HiveObjectRef & ) 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(HiveObjectRef &a, HiveObjectRef &b); - -std::ostream& operator<<(std::ostream& out, const HiveObjectRef& obj); - -typedef struct _PrivilegeGrantInfo__isset { - _PrivilegeGrantInfo__isset() : privilege(false), createTime(false), grantor(false), grantorType(false), grantOption(false) {} - bool privilege :1; - bool createTime :1; - bool grantor :1; - bool grantorType :1; - bool grantOption :1; -} _PrivilegeGrantInfo__isset; - -class PrivilegeGrantInfo : public virtual ::apache::thrift::TBase { - public: - - PrivilegeGrantInfo(const PrivilegeGrantInfo&); - PrivilegeGrantInfo& operator=(const PrivilegeGrantInfo&); - PrivilegeGrantInfo() : privilege(), createTime(0), grantor(), grantorType((PrincipalType::type)0), grantOption(0) { - } - - virtual ~PrivilegeGrantInfo() noexcept; - std::string privilege; - int32_t createTime; - std::string grantor; - PrincipalType::type grantorType; - bool grantOption; - - _PrivilegeGrantInfo__isset __isset; - - void __set_privilege(const std::string& val); - - void __set_createTime(const int32_t val); - - void __set_grantor(const std::string& val); - - void __set_grantorType(const PrincipalType::type val); - - void __set_grantOption(const bool val); - - bool operator == (const PrivilegeGrantInfo & rhs) const - { - if (!(privilege == rhs.privilege)) - return false; - if (!(createTime == rhs.createTime)) - return false; - if (!(grantor == rhs.grantor)) - return false; - if (!(grantorType == rhs.grantorType)) - return false; - if (!(grantOption == rhs.grantOption)) - return false; - return true; - } - bool operator != (const PrivilegeGrantInfo &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PrivilegeGrantInfo & ) 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(PrivilegeGrantInfo &a, PrivilegeGrantInfo &b); - -std::ostream& operator<<(std::ostream& out, const PrivilegeGrantInfo& obj); - -typedef struct _HiveObjectPrivilege__isset { - _HiveObjectPrivilege__isset() : hiveObject(false), principalName(false), principalType(false), grantInfo(false) {} - bool hiveObject :1; - bool principalName :1; - bool principalType :1; - bool grantInfo :1; -} _HiveObjectPrivilege__isset; - -class HiveObjectPrivilege : public virtual ::apache::thrift::TBase { - public: - - HiveObjectPrivilege(const HiveObjectPrivilege&); - HiveObjectPrivilege& operator=(const HiveObjectPrivilege&); - HiveObjectPrivilege() : principalName(), principalType((PrincipalType::type)0) { - } - - virtual ~HiveObjectPrivilege() noexcept; - HiveObjectRef hiveObject; - std::string principalName; - PrincipalType::type principalType; - PrivilegeGrantInfo grantInfo; - - _HiveObjectPrivilege__isset __isset; - - void __set_hiveObject(const HiveObjectRef& val); - - void __set_principalName(const std::string& val); - - void __set_principalType(const PrincipalType::type val); - - void __set_grantInfo(const PrivilegeGrantInfo& val); - - bool operator == (const HiveObjectPrivilege & rhs) const - { - if (!(hiveObject == rhs.hiveObject)) - return false; - if (!(principalName == rhs.principalName)) - return false; - if (!(principalType == rhs.principalType)) - return false; - if (!(grantInfo == rhs.grantInfo)) - return false; - return true; - } - bool operator != (const HiveObjectPrivilege &rhs) const { - return !(*this == rhs); - } - - bool operator < (const HiveObjectPrivilege & ) 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(HiveObjectPrivilege &a, HiveObjectPrivilege &b); - -std::ostream& operator<<(std::ostream& out, const HiveObjectPrivilege& obj); - -typedef struct _PrivilegeBag__isset { - _PrivilegeBag__isset() : privileges(false) {} - bool privileges :1; -} _PrivilegeBag__isset; - -class PrivilegeBag : public virtual ::apache::thrift::TBase { - public: - - PrivilegeBag(const PrivilegeBag&); - PrivilegeBag& operator=(const PrivilegeBag&); - PrivilegeBag() { - } - - virtual ~PrivilegeBag() noexcept; - std::vector privileges; - - _PrivilegeBag__isset __isset; - - void __set_privileges(const std::vector & val); - - bool operator == (const PrivilegeBag & rhs) const - { - if (!(privileges == rhs.privileges)) - return false; - return true; - } - bool operator != (const PrivilegeBag &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PrivilegeBag & ) 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(PrivilegeBag &a, PrivilegeBag &b); - -std::ostream& operator<<(std::ostream& out, const PrivilegeBag& obj); - -typedef struct _PrincipalPrivilegeSet__isset { - _PrincipalPrivilegeSet__isset() : userPrivileges(false), groupPrivileges(false), rolePrivileges(false) {} - bool userPrivileges :1; - bool groupPrivileges :1; - bool rolePrivileges :1; -} _PrincipalPrivilegeSet__isset; - -class PrincipalPrivilegeSet : public virtual ::apache::thrift::TBase { - public: - - PrincipalPrivilegeSet(const PrincipalPrivilegeSet&); - PrincipalPrivilegeSet& operator=(const PrincipalPrivilegeSet&); - PrincipalPrivilegeSet() { - } - - virtual ~PrincipalPrivilegeSet() noexcept; - std::map > userPrivileges; - std::map > groupPrivileges; - std::map > rolePrivileges; - - _PrincipalPrivilegeSet__isset __isset; - - void __set_userPrivileges(const std::map > & val); - - void __set_groupPrivileges(const std::map > & val); - - void __set_rolePrivileges(const std::map > & val); - - bool operator == (const PrincipalPrivilegeSet & rhs) const - { - if (!(userPrivileges == rhs.userPrivileges)) - return false; - if (!(groupPrivileges == rhs.groupPrivileges)) - return false; - if (!(rolePrivileges == rhs.rolePrivileges)) - return false; - return true; - } - bool operator != (const PrincipalPrivilegeSet &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PrincipalPrivilegeSet & ) 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(PrincipalPrivilegeSet &a, PrincipalPrivilegeSet &b); - -std::ostream& operator<<(std::ostream& out, const PrincipalPrivilegeSet& obj); - -typedef struct _GrantRevokePrivilegeRequest__isset { - _GrantRevokePrivilegeRequest__isset() : requestType(false), privileges(false), revokeGrantOption(false) {} - bool requestType :1; - bool privileges :1; - bool revokeGrantOption :1; -} _GrantRevokePrivilegeRequest__isset; - -class GrantRevokePrivilegeRequest : public virtual ::apache::thrift::TBase { - public: - - GrantRevokePrivilegeRequest(const GrantRevokePrivilegeRequest&); - GrantRevokePrivilegeRequest& operator=(const GrantRevokePrivilegeRequest&); - GrantRevokePrivilegeRequest() : requestType((GrantRevokeType::type)0), revokeGrantOption(0) { - } - - virtual ~GrantRevokePrivilegeRequest() noexcept; - GrantRevokeType::type requestType; - PrivilegeBag privileges; - bool revokeGrantOption; - - _GrantRevokePrivilegeRequest__isset __isset; - - void __set_requestType(const GrantRevokeType::type val); - - void __set_privileges(const PrivilegeBag& val); - - void __set_revokeGrantOption(const bool val); - - bool operator == (const GrantRevokePrivilegeRequest & rhs) const - { - if (!(requestType == rhs.requestType)) - return false; - if (!(privileges == rhs.privileges)) - return false; - if (__isset.revokeGrantOption != rhs.__isset.revokeGrantOption) - return false; - else if (__isset.revokeGrantOption && !(revokeGrantOption == rhs.revokeGrantOption)) - return false; - return true; - } - bool operator != (const GrantRevokePrivilegeRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GrantRevokePrivilegeRequest & ) 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(GrantRevokePrivilegeRequest &a, GrantRevokePrivilegeRequest &b); - -std::ostream& operator<<(std::ostream& out, const GrantRevokePrivilegeRequest& obj); - -typedef struct _GrantRevokePrivilegeResponse__isset { - _GrantRevokePrivilegeResponse__isset() : success(false) {} - bool success :1; -} _GrantRevokePrivilegeResponse__isset; - -class GrantRevokePrivilegeResponse : public virtual ::apache::thrift::TBase { - public: - - GrantRevokePrivilegeResponse(const GrantRevokePrivilegeResponse&); - GrantRevokePrivilegeResponse& operator=(const GrantRevokePrivilegeResponse&); - GrantRevokePrivilegeResponse() : success(0) { - } - - virtual ~GrantRevokePrivilegeResponse() noexcept; - bool success; - - _GrantRevokePrivilegeResponse__isset __isset; - - void __set_success(const bool val); - - bool operator == (const GrantRevokePrivilegeResponse & rhs) const - { - if (__isset.success != rhs.__isset.success) - return false; - else if (__isset.success && !(success == rhs.success)) - return false; - return true; - } - bool operator != (const GrantRevokePrivilegeResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GrantRevokePrivilegeResponse & ) 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(GrantRevokePrivilegeResponse &a, GrantRevokePrivilegeResponse &b); - -std::ostream& operator<<(std::ostream& out, const GrantRevokePrivilegeResponse& obj); - -typedef struct _Role__isset { - _Role__isset() : roleName(false), createTime(false), ownerName(false) {} - bool roleName :1; - bool createTime :1; - bool ownerName :1; -} _Role__isset; - -class Role : public virtual ::apache::thrift::TBase { - public: - - Role(const Role&); - Role& operator=(const Role&); - Role() : roleName(), createTime(0), ownerName() { - } - - virtual ~Role() noexcept; - std::string roleName; - int32_t createTime; - std::string ownerName; - - _Role__isset __isset; - - void __set_roleName(const std::string& val); - - void __set_createTime(const int32_t val); - - void __set_ownerName(const std::string& val); - - bool operator == (const Role & rhs) const - { - if (!(roleName == rhs.roleName)) - return false; - if (!(createTime == rhs.createTime)) - return false; - if (!(ownerName == rhs.ownerName)) - return false; - return true; - } - bool operator != (const Role &rhs) const { - return !(*this == rhs); - } - - bool operator < (const Role & ) 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(Role &a, Role &b); - -std::ostream& operator<<(std::ostream& out, const Role& obj); - -typedef struct _RolePrincipalGrant__isset { - _RolePrincipalGrant__isset() : roleName(false), principalName(false), principalType(false), grantOption(false), grantTime(false), grantorName(false), grantorPrincipalType(false) {} - bool roleName :1; - bool principalName :1; - bool principalType :1; - bool grantOption :1; - bool grantTime :1; - bool grantorName :1; - bool grantorPrincipalType :1; -} _RolePrincipalGrant__isset; - -class RolePrincipalGrant : public virtual ::apache::thrift::TBase { - public: - - RolePrincipalGrant(const RolePrincipalGrant&); - RolePrincipalGrant& operator=(const RolePrincipalGrant&); - RolePrincipalGrant() : roleName(), principalName(), principalType((PrincipalType::type)0), grantOption(0), grantTime(0), grantorName(), grantorPrincipalType((PrincipalType::type)0) { - } - - virtual ~RolePrincipalGrant() noexcept; - std::string roleName; - std::string principalName; - PrincipalType::type principalType; - bool grantOption; - int32_t grantTime; - std::string grantorName; - PrincipalType::type grantorPrincipalType; - - _RolePrincipalGrant__isset __isset; - - void __set_roleName(const std::string& val); - - void __set_principalName(const std::string& val); - - void __set_principalType(const PrincipalType::type val); - - void __set_grantOption(const bool val); - - void __set_grantTime(const int32_t val); - - void __set_grantorName(const std::string& val); - - void __set_grantorPrincipalType(const PrincipalType::type val); - - bool operator == (const RolePrincipalGrant & rhs) const - { - if (!(roleName == rhs.roleName)) - return false; - if (!(principalName == rhs.principalName)) - return false; - if (!(principalType == rhs.principalType)) - return false; - if (!(grantOption == rhs.grantOption)) - return false; - if (!(grantTime == rhs.grantTime)) - return false; - if (!(grantorName == rhs.grantorName)) - return false; - if (!(grantorPrincipalType == rhs.grantorPrincipalType)) - return false; - return true; - } - bool operator != (const RolePrincipalGrant &rhs) const { - return !(*this == rhs); - } - - bool operator < (const RolePrincipalGrant & ) 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(RolePrincipalGrant &a, RolePrincipalGrant &b); - -std::ostream& operator<<(std::ostream& out, const RolePrincipalGrant& obj); - - -class GetRoleGrantsForPrincipalRequest : public virtual ::apache::thrift::TBase { - public: - - GetRoleGrantsForPrincipalRequest(const GetRoleGrantsForPrincipalRequest&); - GetRoleGrantsForPrincipalRequest& operator=(const GetRoleGrantsForPrincipalRequest&); - GetRoleGrantsForPrincipalRequest() : principal_name(), principal_type((PrincipalType::type)0) { - } - - virtual ~GetRoleGrantsForPrincipalRequest() noexcept; - std::string principal_name; - PrincipalType::type principal_type; - - void __set_principal_name(const std::string& val); - - void __set_principal_type(const PrincipalType::type val); - - bool operator == (const GetRoleGrantsForPrincipalRequest & rhs) const - { - if (!(principal_name == rhs.principal_name)) - return false; - if (!(principal_type == rhs.principal_type)) - return false; - return true; - } - bool operator != (const GetRoleGrantsForPrincipalRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GetRoleGrantsForPrincipalRequest & ) 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(GetRoleGrantsForPrincipalRequest &a, GetRoleGrantsForPrincipalRequest &b); - -std::ostream& operator<<(std::ostream& out, const GetRoleGrantsForPrincipalRequest& obj); - - -class GetRoleGrantsForPrincipalResponse : public virtual ::apache::thrift::TBase { - public: - - GetRoleGrantsForPrincipalResponse(const GetRoleGrantsForPrincipalResponse&); - GetRoleGrantsForPrincipalResponse& operator=(const GetRoleGrantsForPrincipalResponse&); - GetRoleGrantsForPrincipalResponse() { - } - - virtual ~GetRoleGrantsForPrincipalResponse() noexcept; - std::vector principalGrants; - - void __set_principalGrants(const std::vector & val); - - bool operator == (const GetRoleGrantsForPrincipalResponse & rhs) const - { - if (!(principalGrants == rhs.principalGrants)) - return false; - return true; - } - bool operator != (const GetRoleGrantsForPrincipalResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GetRoleGrantsForPrincipalResponse & ) 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(GetRoleGrantsForPrincipalResponse &a, GetRoleGrantsForPrincipalResponse &b); - -std::ostream& operator<<(std::ostream& out, const GetRoleGrantsForPrincipalResponse& obj); - - -class GetPrincipalsInRoleRequest : public virtual ::apache::thrift::TBase { - public: - - GetPrincipalsInRoleRequest(const GetPrincipalsInRoleRequest&); - GetPrincipalsInRoleRequest& operator=(const GetPrincipalsInRoleRequest&); - GetPrincipalsInRoleRequest() : roleName() { - } - - virtual ~GetPrincipalsInRoleRequest() noexcept; - std::string roleName; - - void __set_roleName(const std::string& val); - - bool operator == (const GetPrincipalsInRoleRequest & rhs) const - { - if (!(roleName == rhs.roleName)) - return false; - return true; - } - bool operator != (const GetPrincipalsInRoleRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GetPrincipalsInRoleRequest & ) 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(GetPrincipalsInRoleRequest &a, GetPrincipalsInRoleRequest &b); - -std::ostream& operator<<(std::ostream& out, const GetPrincipalsInRoleRequest& obj); - - -class GetPrincipalsInRoleResponse : public virtual ::apache::thrift::TBase { - public: - - GetPrincipalsInRoleResponse(const GetPrincipalsInRoleResponse&); - GetPrincipalsInRoleResponse& operator=(const GetPrincipalsInRoleResponse&); - GetPrincipalsInRoleResponse() { - } - - virtual ~GetPrincipalsInRoleResponse() noexcept; - std::vector principalGrants; - - void __set_principalGrants(const std::vector & val); - - bool operator == (const GetPrincipalsInRoleResponse & rhs) const - { - if (!(principalGrants == rhs.principalGrants)) - return false; - return true; - } - bool operator != (const GetPrincipalsInRoleResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GetPrincipalsInRoleResponse & ) 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(GetPrincipalsInRoleResponse &a, GetPrincipalsInRoleResponse &b); - -std::ostream& operator<<(std::ostream& out, const GetPrincipalsInRoleResponse& obj); - -typedef struct _GrantRevokeRoleRequest__isset { - _GrantRevokeRoleRequest__isset() : requestType(false), roleName(false), principalName(false), principalType(false), grantor(false), grantorType(false), grantOption(false) {} - bool requestType :1; - bool roleName :1; - bool principalName :1; - bool principalType :1; - bool grantor :1; - bool grantorType :1; - bool grantOption :1; -} _GrantRevokeRoleRequest__isset; - -class GrantRevokeRoleRequest : public virtual ::apache::thrift::TBase { - public: - - GrantRevokeRoleRequest(const GrantRevokeRoleRequest&); - GrantRevokeRoleRequest& operator=(const GrantRevokeRoleRequest&); - GrantRevokeRoleRequest() : requestType((GrantRevokeType::type)0), roleName(), principalName(), principalType((PrincipalType::type)0), grantor(), grantorType((PrincipalType::type)0), grantOption(0) { - } - - virtual ~GrantRevokeRoleRequest() noexcept; - GrantRevokeType::type requestType; - std::string roleName; - std::string principalName; - PrincipalType::type principalType; - std::string grantor; - PrincipalType::type grantorType; - bool grantOption; - - _GrantRevokeRoleRequest__isset __isset; - - void __set_requestType(const GrantRevokeType::type val); - - void __set_roleName(const std::string& val); - - void __set_principalName(const std::string& val); - - void __set_principalType(const PrincipalType::type val); - - void __set_grantor(const std::string& val); - - void __set_grantorType(const PrincipalType::type val); - - void __set_grantOption(const bool val); - - bool operator == (const GrantRevokeRoleRequest & rhs) const - { - if (!(requestType == rhs.requestType)) - return false; - if (!(roleName == rhs.roleName)) - return false; - if (!(principalName == rhs.principalName)) - return false; - if (!(principalType == rhs.principalType)) - return false; - if (__isset.grantor != rhs.__isset.grantor) - return false; - else if (__isset.grantor && !(grantor == rhs.grantor)) - return false; - if (__isset.grantorType != rhs.__isset.grantorType) - return false; - else if (__isset.grantorType && !(grantorType == rhs.grantorType)) - return false; - if (__isset.grantOption != rhs.__isset.grantOption) - return false; - else if (__isset.grantOption && !(grantOption == rhs.grantOption)) - return false; - return true; - } - bool operator != (const GrantRevokeRoleRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GrantRevokeRoleRequest & ) 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(GrantRevokeRoleRequest &a, GrantRevokeRoleRequest &b); - -std::ostream& operator<<(std::ostream& out, const GrantRevokeRoleRequest& obj); - -typedef struct _GrantRevokeRoleResponse__isset { - _GrantRevokeRoleResponse__isset() : success(false) {} - bool success :1; -} _GrantRevokeRoleResponse__isset; - -class GrantRevokeRoleResponse : public virtual ::apache::thrift::TBase { - public: - - GrantRevokeRoleResponse(const GrantRevokeRoleResponse&); - GrantRevokeRoleResponse& operator=(const GrantRevokeRoleResponse&); - GrantRevokeRoleResponse() : success(0) { - } - - virtual ~GrantRevokeRoleResponse() noexcept; - bool success; - - _GrantRevokeRoleResponse__isset __isset; - - void __set_success(const bool val); - - bool operator == (const GrantRevokeRoleResponse & rhs) const - { - if (__isset.success != rhs.__isset.success) - return false; - else if (__isset.success && !(success == rhs.success)) - return false; - return true; - } - bool operator != (const GrantRevokeRoleResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GrantRevokeRoleResponse & ) 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(GrantRevokeRoleResponse &a, GrantRevokeRoleResponse &b); - -std::ostream& operator<<(std::ostream& out, const GrantRevokeRoleResponse& obj); - -typedef struct _Database__isset { - _Database__isset() : name(false), description(false), locationUri(false), parameters(false), privileges(false), ownerName(false), ownerType(false) {} - bool name :1; - bool description :1; - bool locationUri :1; - bool parameters :1; - bool privileges :1; - bool ownerName :1; - bool ownerType :1; -} _Database__isset; - -class Database : public virtual ::apache::thrift::TBase { - public: - - Database(const Database&); - Database& operator=(const Database&); - Database() : name(), description(), locationUri(), ownerName(), ownerType((PrincipalType::type)0) { - } - - virtual ~Database() noexcept; - std::string name; - std::string description; - std::string locationUri; - std::map parameters; - PrincipalPrivilegeSet privileges; - std::string ownerName; - PrincipalType::type ownerType; - - _Database__isset __isset; - - void __set_name(const std::string& val); - - void __set_description(const std::string& val); - - void __set_locationUri(const std::string& val); - - void __set_parameters(const std::map & val); - - void __set_privileges(const PrincipalPrivilegeSet& val); - - void __set_ownerName(const std::string& val); - - void __set_ownerType(const PrincipalType::type val); - - bool operator == (const Database & rhs) const - { - if (!(name == rhs.name)) - return false; - if (!(description == rhs.description)) - return false; - if (!(locationUri == rhs.locationUri)) - return false; - if (!(parameters == rhs.parameters)) - return false; - if (__isset.privileges != rhs.__isset.privileges) - return false; - else if (__isset.privileges && !(privileges == rhs.privileges)) - return false; - if (__isset.ownerName != rhs.__isset.ownerName) - return false; - else if (__isset.ownerName && !(ownerName == rhs.ownerName)) - return false; - if (__isset.ownerType != rhs.__isset.ownerType) - return false; - else if (__isset.ownerType && !(ownerType == rhs.ownerType)) - return false; - return true; - } - bool operator != (const Database &rhs) const { - return !(*this == rhs); - } - - bool operator < (const Database & ) 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(Database &a, Database &b); - -std::ostream& operator<<(std::ostream& out, const Database& obj); - -typedef struct _SerDeInfo__isset { - _SerDeInfo__isset() : name(false), serializationLib(false), parameters(false) {} - bool name :1; - bool serializationLib :1; - bool parameters :1; -} _SerDeInfo__isset; - -class SerDeInfo : public virtual ::apache::thrift::TBase { - public: - - SerDeInfo(const SerDeInfo&); - SerDeInfo& operator=(const SerDeInfo&); - SerDeInfo() : name(), serializationLib() { - } - - virtual ~SerDeInfo() noexcept; - std::string name; - std::string serializationLib; - std::map parameters; - - _SerDeInfo__isset __isset; - - void __set_name(const std::string& val); - - void __set_serializationLib(const std::string& val); - - void __set_parameters(const std::map & val); - - bool operator == (const SerDeInfo & rhs) const - { - if (!(name == rhs.name)) - return false; - if (!(serializationLib == rhs.serializationLib)) - return false; - if (!(parameters == rhs.parameters)) - return false; - return true; - } - bool operator != (const SerDeInfo &rhs) const { - return !(*this == rhs); - } - - bool operator < (const SerDeInfo & ) 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(SerDeInfo &a, SerDeInfo &b); - -std::ostream& operator<<(std::ostream& out, const SerDeInfo& obj); - -typedef struct _Order__isset { - _Order__isset() : col(false), order(false) {} - bool col :1; - bool order :1; -} _Order__isset; - -class Order : public virtual ::apache::thrift::TBase { - public: - - Order(const Order&); - Order& operator=(const Order&); - Order() : col(), order(0) { - } - - virtual ~Order() noexcept; - std::string col; - int32_t order; - - _Order__isset __isset; - - void __set_col(const std::string& val); - - void __set_order(const int32_t val); - - bool operator == (const Order & rhs) const - { - if (!(col == rhs.col)) - return false; - if (!(order == rhs.order)) - return false; - return true; - } - bool operator != (const Order &rhs) const { - return !(*this == rhs); - } - - bool operator < (const Order & ) 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(Order &a, Order &b); - -std::ostream& operator<<(std::ostream& out, const Order& obj); - -typedef struct _SkewedInfo__isset { - _SkewedInfo__isset() : skewedColNames(false), skewedColValues(false), skewedColValueLocationMaps(false) {} - bool skewedColNames :1; - bool skewedColValues :1; - bool skewedColValueLocationMaps :1; -} _SkewedInfo__isset; - -class SkewedInfo : public virtual ::apache::thrift::TBase { - public: - - SkewedInfo(const SkewedInfo&); - SkewedInfo& operator=(const SkewedInfo&); - SkewedInfo() { - } - - virtual ~SkewedInfo() noexcept; - std::vector skewedColNames; - std::vector > skewedColValues; - std::map skewedColValueLocationMaps; - - _SkewedInfo__isset __isset; - - void __set_skewedColNames(const std::vector & val); - - void __set_skewedColValues(const std::vector > & val); - - void __set_skewedColValueLocationMaps(const std::map & val); - - bool operator == (const SkewedInfo & rhs) const - { - if (!(skewedColNames == rhs.skewedColNames)) - return false; - if (!(skewedColValues == rhs.skewedColValues)) - return false; - if (!(skewedColValueLocationMaps == rhs.skewedColValueLocationMaps)) - return false; - return true; - } - bool operator != (const SkewedInfo &rhs) const { - return !(*this == rhs); - } - - bool operator < (const SkewedInfo & ) 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(SkewedInfo &a, SkewedInfo &b); - -std::ostream& operator<<(std::ostream& out, const SkewedInfo& obj); - -typedef struct _StorageDescriptor__isset { - _StorageDescriptor__isset() : cols(false), location(false), inputFormat(false), outputFormat(false), compressed(false), numBuckets(false), serdeInfo(false), bucketCols(false), sortCols(false), parameters(false), skewedInfo(false), storedAsSubDirectories(false) {} - bool cols :1; - bool location :1; - bool inputFormat :1; - bool outputFormat :1; - bool compressed :1; - bool numBuckets :1; - bool serdeInfo :1; - bool bucketCols :1; - bool sortCols :1; - bool parameters :1; - bool skewedInfo :1; - bool storedAsSubDirectories :1; -} _StorageDescriptor__isset; - -class StorageDescriptor : public virtual ::apache::thrift::TBase { - public: - - StorageDescriptor(const StorageDescriptor&); - StorageDescriptor& operator=(const StorageDescriptor&); - StorageDescriptor() : location(), inputFormat(), outputFormat(), compressed(0), numBuckets(0), storedAsSubDirectories(0) { - } - - virtual ~StorageDescriptor() noexcept; - std::vector cols; - std::string location; - std::string inputFormat; - std::string outputFormat; - bool compressed; - int32_t numBuckets; - SerDeInfo serdeInfo; - std::vector bucketCols; - std::vector sortCols; - std::map parameters; - SkewedInfo skewedInfo; - bool storedAsSubDirectories; - - _StorageDescriptor__isset __isset; - - void __set_cols(const std::vector & val); - - void __set_location(const std::string& val); - - void __set_inputFormat(const std::string& val); - - void __set_outputFormat(const std::string& val); - - void __set_compressed(const bool val); - - void __set_numBuckets(const int32_t val); - - void __set_serdeInfo(const SerDeInfo& val); - - void __set_bucketCols(const std::vector & val); - - void __set_sortCols(const std::vector & val); - - void __set_parameters(const std::map & val); - - void __set_skewedInfo(const SkewedInfo& val); - - void __set_storedAsSubDirectories(const bool val); - - bool operator == (const StorageDescriptor & rhs) const - { - if (!(cols == rhs.cols)) - return false; - if (!(location == rhs.location)) - return false; - if (!(inputFormat == rhs.inputFormat)) - return false; - if (!(outputFormat == rhs.outputFormat)) - return false; - if (!(compressed == rhs.compressed)) - return false; - if (!(numBuckets == rhs.numBuckets)) - return false; - if (!(serdeInfo == rhs.serdeInfo)) - return false; - if (!(bucketCols == rhs.bucketCols)) - return false; - if (!(sortCols == rhs.sortCols)) - return false; - if (!(parameters == rhs.parameters)) - return false; - if (__isset.skewedInfo != rhs.__isset.skewedInfo) - return false; - else if (__isset.skewedInfo && !(skewedInfo == rhs.skewedInfo)) - return false; - if (__isset.storedAsSubDirectories != rhs.__isset.storedAsSubDirectories) - return false; - else if (__isset.storedAsSubDirectories && !(storedAsSubDirectories == rhs.storedAsSubDirectories)) - return false; - return true; - } - bool operator != (const StorageDescriptor &rhs) const { - return !(*this == rhs); - } - - bool operator < (const StorageDescriptor & ) 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(StorageDescriptor &a, StorageDescriptor &b); - -std::ostream& operator<<(std::ostream& out, const StorageDescriptor& obj); - -typedef struct _Table__isset { - _Table__isset() : tableName(false), dbName(false), owner(false), createTime(false), lastAccessTime(false), retention(false), sd(false), partitionKeys(false), parameters(false), viewOriginalText(false), viewExpandedText(false), tableType(false), privileges(false), temporary(true), rewriteEnabled(false) {} - bool tableName :1; - bool dbName :1; - bool owner :1; - bool createTime :1; - bool lastAccessTime :1; - bool retention :1; - bool sd :1; - bool partitionKeys :1; - bool parameters :1; - bool viewOriginalText :1; - bool viewExpandedText :1; - bool tableType :1; - bool privileges :1; - bool temporary :1; - bool rewriteEnabled :1; -} _Table__isset; - -class Table : public virtual ::apache::thrift::TBase { - public: - - Table(const Table&); - Table& operator=(const Table&); - Table() : tableName(), dbName(), owner(), createTime(0), lastAccessTime(0), retention(0), viewOriginalText(), viewExpandedText(), tableType(), temporary(false), rewriteEnabled(0) { - } - - virtual ~Table() noexcept; - std::string tableName; - std::string dbName; - std::string owner; - int32_t createTime; - int32_t lastAccessTime; - int32_t retention; - StorageDescriptor sd; - std::vector partitionKeys; - std::map parameters; - std::string viewOriginalText; - std::string viewExpandedText; - std::string tableType; - PrincipalPrivilegeSet privileges; - bool temporary; - bool rewriteEnabled; - - _Table__isset __isset; - - void __set_tableName(const std::string& val); - - void __set_dbName(const std::string& val); - - void __set_owner(const std::string& val); - - void __set_createTime(const int32_t val); - - void __set_lastAccessTime(const int32_t val); - - void __set_retention(const int32_t val); - - void __set_sd(const StorageDescriptor& val); - - void __set_partitionKeys(const std::vector & val); - - void __set_parameters(const std::map & val); - - void __set_viewOriginalText(const std::string& val); - - void __set_viewExpandedText(const std::string& val); - - void __set_tableType(const std::string& val); - - void __set_privileges(const PrincipalPrivilegeSet& val); - - void __set_temporary(const bool val); - - void __set_rewriteEnabled(const bool val); - - bool operator == (const Table & rhs) const - { - if (!(tableName == rhs.tableName)) - return false; - if (!(dbName == rhs.dbName)) - return false; - if (!(owner == rhs.owner)) - return false; - if (!(createTime == rhs.createTime)) - return false; - if (!(lastAccessTime == rhs.lastAccessTime)) - return false; - if (!(retention == rhs.retention)) - return false; - if (!(sd == rhs.sd)) - return false; - if (!(partitionKeys == rhs.partitionKeys)) - return false; - if (!(parameters == rhs.parameters)) - return false; - if (!(viewOriginalText == rhs.viewOriginalText)) - return false; - if (!(viewExpandedText == rhs.viewExpandedText)) - return false; - if (!(tableType == rhs.tableType)) - return false; - if (__isset.privileges != rhs.__isset.privileges) - return false; - else if (__isset.privileges && !(privileges == rhs.privileges)) - return false; - if (__isset.temporary != rhs.__isset.temporary) - return false; - else if (__isset.temporary && !(temporary == rhs.temporary)) - return false; - if (__isset.rewriteEnabled != rhs.__isset.rewriteEnabled) - return false; - else if (__isset.rewriteEnabled && !(rewriteEnabled == rhs.rewriteEnabled)) - return false; - return true; - } - bool operator != (const Table &rhs) const { - return !(*this == rhs); - } - - bool operator < (const Table & ) 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(Table &a, Table &b); - -std::ostream& operator<<(std::ostream& out, const Table& obj); - -typedef struct _Partition__isset { - _Partition__isset() : values(false), dbName(false), tableName(false), createTime(false), lastAccessTime(false), sd(false), parameters(false), privileges(false) {} - bool values :1; - bool dbName :1; - bool tableName :1; - bool createTime :1; - bool lastAccessTime :1; - bool sd :1; - bool parameters :1; - bool privileges :1; -} _Partition__isset; - -class Partition : public virtual ::apache::thrift::TBase { - public: - - Partition(const Partition&); - Partition& operator=(const Partition&); - Partition() : dbName(), tableName(), createTime(0), lastAccessTime(0) { - } - - virtual ~Partition() noexcept; - std::vector values; - std::string dbName; - std::string tableName; - int32_t createTime; - int32_t lastAccessTime; - StorageDescriptor sd; - std::map parameters; - PrincipalPrivilegeSet privileges; - - _Partition__isset __isset; - - void __set_values(const std::vector & val); - - void __set_dbName(const std::string& val); - - void __set_tableName(const std::string& val); - - void __set_createTime(const int32_t val); - - void __set_lastAccessTime(const int32_t val); - - void __set_sd(const StorageDescriptor& val); - - void __set_parameters(const std::map & val); - - void __set_privileges(const PrincipalPrivilegeSet& val); - - bool operator == (const Partition & rhs) const - { - if (!(values == rhs.values)) - return false; - if (!(dbName == rhs.dbName)) - return false; - if (!(tableName == rhs.tableName)) - return false; - if (!(createTime == rhs.createTime)) - return false; - if (!(lastAccessTime == rhs.lastAccessTime)) - return false; - if (!(sd == rhs.sd)) - return false; - if (!(parameters == rhs.parameters)) - return false; - if (__isset.privileges != rhs.__isset.privileges) - return false; - else if (__isset.privileges && !(privileges == rhs.privileges)) - return false; - return true; - } - bool operator != (const Partition &rhs) const { - return !(*this == rhs); - } - - bool operator < (const Partition & ) 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(Partition &a, Partition &b); - -std::ostream& operator<<(std::ostream& out, const Partition& obj); - -typedef struct _PartitionWithoutSD__isset { - _PartitionWithoutSD__isset() : values(false), createTime(false), lastAccessTime(false), relativePath(false), parameters(false), privileges(false) {} - bool values :1; - bool createTime :1; - bool lastAccessTime :1; - bool relativePath :1; - bool parameters :1; - bool privileges :1; -} _PartitionWithoutSD__isset; - -class PartitionWithoutSD : public virtual ::apache::thrift::TBase { - public: - - PartitionWithoutSD(const PartitionWithoutSD&); - PartitionWithoutSD& operator=(const PartitionWithoutSD&); - PartitionWithoutSD() : createTime(0), lastAccessTime(0), relativePath() { - } - - virtual ~PartitionWithoutSD() noexcept; - std::vector values; - int32_t createTime; - int32_t lastAccessTime; - std::string relativePath; - std::map parameters; - PrincipalPrivilegeSet privileges; - - _PartitionWithoutSD__isset __isset; - - void __set_values(const std::vector & val); - - void __set_createTime(const int32_t val); - - void __set_lastAccessTime(const int32_t val); - - void __set_relativePath(const std::string& val); - - void __set_parameters(const std::map & val); - - void __set_privileges(const PrincipalPrivilegeSet& val); - - bool operator == (const PartitionWithoutSD & rhs) const - { - if (!(values == rhs.values)) - return false; - if (!(createTime == rhs.createTime)) - return false; - if (!(lastAccessTime == rhs.lastAccessTime)) - return false; - if (!(relativePath == rhs.relativePath)) - return false; - if (!(parameters == rhs.parameters)) - return false; - if (__isset.privileges != rhs.__isset.privileges) - return false; - else if (__isset.privileges && !(privileges == rhs.privileges)) - return false; - return true; - } - bool operator != (const PartitionWithoutSD &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PartitionWithoutSD & ) 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(PartitionWithoutSD &a, PartitionWithoutSD &b); - -std::ostream& operator<<(std::ostream& out, const PartitionWithoutSD& obj); - -typedef struct _PartitionSpecWithSharedSD__isset { - _PartitionSpecWithSharedSD__isset() : partitions(false), sd(false) {} - bool partitions :1; - bool sd :1; -} _PartitionSpecWithSharedSD__isset; - -class PartitionSpecWithSharedSD : public virtual ::apache::thrift::TBase { - public: - - PartitionSpecWithSharedSD(const PartitionSpecWithSharedSD&); - PartitionSpecWithSharedSD& operator=(const PartitionSpecWithSharedSD&); - PartitionSpecWithSharedSD() { - } - - virtual ~PartitionSpecWithSharedSD() noexcept; - std::vector partitions; - StorageDescriptor sd; - - _PartitionSpecWithSharedSD__isset __isset; - - void __set_partitions(const std::vector & val); - - void __set_sd(const StorageDescriptor& val); - - bool operator == (const PartitionSpecWithSharedSD & rhs) const - { - if (!(partitions == rhs.partitions)) - return false; - if (!(sd == rhs.sd)) - return false; - return true; - } - bool operator != (const PartitionSpecWithSharedSD &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PartitionSpecWithSharedSD & ) 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(PartitionSpecWithSharedSD &a, PartitionSpecWithSharedSD &b); - -std::ostream& operator<<(std::ostream& out, const PartitionSpecWithSharedSD& obj); - -typedef struct _PartitionListComposingSpec__isset { - _PartitionListComposingSpec__isset() : partitions(false) {} - bool partitions :1; -} _PartitionListComposingSpec__isset; - -class PartitionListComposingSpec : public virtual ::apache::thrift::TBase { - public: - - PartitionListComposingSpec(const PartitionListComposingSpec&); - PartitionListComposingSpec& operator=(const PartitionListComposingSpec&); - PartitionListComposingSpec() { - } - - virtual ~PartitionListComposingSpec() noexcept; - std::vector partitions; - - _PartitionListComposingSpec__isset __isset; - - void __set_partitions(const std::vector & val); - - bool operator == (const PartitionListComposingSpec & rhs) const - { - if (!(partitions == rhs.partitions)) - return false; - return true; - } - bool operator != (const PartitionListComposingSpec &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PartitionListComposingSpec & ) 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(PartitionListComposingSpec &a, PartitionListComposingSpec &b); - -std::ostream& operator<<(std::ostream& out, const PartitionListComposingSpec& obj); - -typedef struct _PartitionSpec__isset { - _PartitionSpec__isset() : dbName(false), tableName(false), rootPath(false), sharedSDPartitionSpec(false), partitionList(false) {} - bool dbName :1; - bool tableName :1; - bool rootPath :1; - bool sharedSDPartitionSpec :1; - bool partitionList :1; -} _PartitionSpec__isset; - -class PartitionSpec : public virtual ::apache::thrift::TBase { - public: - - PartitionSpec(const PartitionSpec&); - PartitionSpec& operator=(const PartitionSpec&); - PartitionSpec() : dbName(), tableName(), rootPath() { - } - - virtual ~PartitionSpec() noexcept; - std::string dbName; - std::string tableName; - std::string rootPath; - PartitionSpecWithSharedSD sharedSDPartitionSpec; - PartitionListComposingSpec partitionList; - - _PartitionSpec__isset __isset; - - void __set_dbName(const std::string& val); - - void __set_tableName(const std::string& val); - - void __set_rootPath(const std::string& val); - - void __set_sharedSDPartitionSpec(const PartitionSpecWithSharedSD& val); - - void __set_partitionList(const PartitionListComposingSpec& val); - - bool operator == (const PartitionSpec & rhs) const - { - if (!(dbName == rhs.dbName)) - return false; - if (!(tableName == rhs.tableName)) - return false; - if (!(rootPath == rhs.rootPath)) - return false; - if (__isset.sharedSDPartitionSpec != rhs.__isset.sharedSDPartitionSpec) - return false; - else if (__isset.sharedSDPartitionSpec && !(sharedSDPartitionSpec == rhs.sharedSDPartitionSpec)) - return false; - if (__isset.partitionList != rhs.__isset.partitionList) - return false; - else if (__isset.partitionList && !(partitionList == rhs.partitionList)) - return false; - return true; - } - bool operator != (const PartitionSpec &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PartitionSpec & ) 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(PartitionSpec &a, PartitionSpec &b); - -std::ostream& operator<<(std::ostream& out, const PartitionSpec& obj); - -typedef struct _Index__isset { - _Index__isset() : indexName(false), indexHandlerClass(false), dbName(false), origTableName(false), createTime(false), lastAccessTime(false), indexTableName(false), sd(false), parameters(false), deferredRebuild(false) {} - bool indexName :1; - bool indexHandlerClass :1; - bool dbName :1; - bool origTableName :1; - bool createTime :1; - bool lastAccessTime :1; - bool indexTableName :1; - bool sd :1; - bool parameters :1; - bool deferredRebuild :1; -} _Index__isset; - -class Index : public virtual ::apache::thrift::TBase { - public: - - Index(const Index&); - Index& operator=(const Index&); - Index() : indexName(), indexHandlerClass(), dbName(), origTableName(), createTime(0), lastAccessTime(0), indexTableName(), deferredRebuild(0) { - } - - virtual ~Index() noexcept; - std::string indexName; - std::string indexHandlerClass; - std::string dbName; - std::string origTableName; - int32_t createTime; - int32_t lastAccessTime; - std::string indexTableName; - StorageDescriptor sd; - std::map parameters; - bool deferredRebuild; - - _Index__isset __isset; - - void __set_indexName(const std::string& val); - - void __set_indexHandlerClass(const std::string& val); - - void __set_dbName(const std::string& val); - - void __set_origTableName(const std::string& val); - - void __set_createTime(const int32_t val); - - void __set_lastAccessTime(const int32_t val); - - void __set_indexTableName(const std::string& val); - - void __set_sd(const StorageDescriptor& val); - - void __set_parameters(const std::map & val); - - void __set_deferredRebuild(const bool val); - - bool operator == (const Index & rhs) const - { - if (!(indexName == rhs.indexName)) - return false; - if (!(indexHandlerClass == rhs.indexHandlerClass)) - return false; - if (!(dbName == rhs.dbName)) - return false; - if (!(origTableName == rhs.origTableName)) - return false; - if (!(createTime == rhs.createTime)) - return false; - if (!(lastAccessTime == rhs.lastAccessTime)) - return false; - if (!(indexTableName == rhs.indexTableName)) - return false; - if (!(sd == rhs.sd)) - return false; - if (!(parameters == rhs.parameters)) - return false; - if (!(deferredRebuild == rhs.deferredRebuild)) - return false; - return true; - } - bool operator != (const Index &rhs) const { - return !(*this == rhs); - } - - bool operator < (const Index & ) 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(Index &a, Index &b); - -std::ostream& operator<<(std::ostream& out, const Index& obj); - -typedef struct _BooleanColumnStatsData__isset { - _BooleanColumnStatsData__isset() : bitVectors(false) {} - bool bitVectors :1; -} _BooleanColumnStatsData__isset; - -class BooleanColumnStatsData : public virtual ::apache::thrift::TBase { - public: - - BooleanColumnStatsData(const BooleanColumnStatsData&); - BooleanColumnStatsData& operator=(const BooleanColumnStatsData&); - BooleanColumnStatsData() : numTrues(0), numFalses(0), numNulls(0), bitVectors() { - } - - virtual ~BooleanColumnStatsData() noexcept; - int64_t numTrues; - int64_t numFalses; - int64_t numNulls; - std::string bitVectors; - - _BooleanColumnStatsData__isset __isset; - - void __set_numTrues(const int64_t val); - - void __set_numFalses(const int64_t val); - - void __set_numNulls(const int64_t val); - - void __set_bitVectors(const std::string& val); - - bool operator == (const BooleanColumnStatsData & rhs) const - { - if (!(numTrues == rhs.numTrues)) - return false; - if (!(numFalses == rhs.numFalses)) - return false; - if (!(numNulls == rhs.numNulls)) - return false; - if (__isset.bitVectors != rhs.__isset.bitVectors) - return false; - else if (__isset.bitVectors && !(bitVectors == rhs.bitVectors)) - return false; - return true; - } - bool operator != (const BooleanColumnStatsData &rhs) const { - return !(*this == rhs); - } - - bool operator < (const BooleanColumnStatsData & ) 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(BooleanColumnStatsData &a, BooleanColumnStatsData &b); - -std::ostream& operator<<(std::ostream& out, const BooleanColumnStatsData& obj); - -typedef struct _DoubleColumnStatsData__isset { - _DoubleColumnStatsData__isset() : lowValue(false), highValue(false), bitVectors(false) {} - bool lowValue :1; - bool highValue :1; - bool bitVectors :1; -} _DoubleColumnStatsData__isset; - -class DoubleColumnStatsData : public virtual ::apache::thrift::TBase { - public: - - DoubleColumnStatsData(const DoubleColumnStatsData&); - DoubleColumnStatsData& operator=(const DoubleColumnStatsData&); - DoubleColumnStatsData() : lowValue(0), highValue(0), numNulls(0), numDVs(0), bitVectors() { - } - - virtual ~DoubleColumnStatsData() noexcept; - double lowValue; - double highValue; - int64_t numNulls; - int64_t numDVs; - std::string bitVectors; - - _DoubleColumnStatsData__isset __isset; - - void __set_lowValue(const double val); - - void __set_highValue(const double val); - - void __set_numNulls(const int64_t val); - - void __set_numDVs(const int64_t val); - - void __set_bitVectors(const std::string& val); - - bool operator == (const DoubleColumnStatsData & rhs) const - { - if (__isset.lowValue != rhs.__isset.lowValue) - return false; - else if (__isset.lowValue && !(lowValue == rhs.lowValue)) - return false; - if (__isset.highValue != rhs.__isset.highValue) - return false; - else if (__isset.highValue && !(highValue == rhs.highValue)) - return false; - if (!(numNulls == rhs.numNulls)) - return false; - if (!(numDVs == rhs.numDVs)) - return false; - if (__isset.bitVectors != rhs.__isset.bitVectors) - return false; - else if (__isset.bitVectors && !(bitVectors == rhs.bitVectors)) - return false; - return true; - } - bool operator != (const DoubleColumnStatsData &rhs) const { - return !(*this == rhs); - } - - bool operator < (const DoubleColumnStatsData & ) 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(DoubleColumnStatsData &a, DoubleColumnStatsData &b); - -std::ostream& operator<<(std::ostream& out, const DoubleColumnStatsData& obj); - -typedef struct _LongColumnStatsData__isset { - _LongColumnStatsData__isset() : lowValue(false), highValue(false), bitVectors(false) {} - bool lowValue :1; - bool highValue :1; - bool bitVectors :1; -} _LongColumnStatsData__isset; - -class LongColumnStatsData : public virtual ::apache::thrift::TBase { - public: - - LongColumnStatsData(const LongColumnStatsData&); - LongColumnStatsData& operator=(const LongColumnStatsData&); - LongColumnStatsData() : lowValue(0), highValue(0), numNulls(0), numDVs(0), bitVectors() { - } - - virtual ~LongColumnStatsData() noexcept; - int64_t lowValue; - int64_t highValue; - int64_t numNulls; - int64_t numDVs; - std::string bitVectors; - - _LongColumnStatsData__isset __isset; - - void __set_lowValue(const int64_t val); - - void __set_highValue(const int64_t val); - - void __set_numNulls(const int64_t val); - - void __set_numDVs(const int64_t val); - - void __set_bitVectors(const std::string& val); - - bool operator == (const LongColumnStatsData & rhs) const - { - if (__isset.lowValue != rhs.__isset.lowValue) - return false; - else if (__isset.lowValue && !(lowValue == rhs.lowValue)) - return false; - if (__isset.highValue != rhs.__isset.highValue) - return false; - else if (__isset.highValue && !(highValue == rhs.highValue)) - return false; - if (!(numNulls == rhs.numNulls)) - return false; - if (!(numDVs == rhs.numDVs)) - return false; - if (__isset.bitVectors != rhs.__isset.bitVectors) - return false; - else if (__isset.bitVectors && !(bitVectors == rhs.bitVectors)) - return false; - return true; - } - bool operator != (const LongColumnStatsData &rhs) const { - return !(*this == rhs); - } - - bool operator < (const LongColumnStatsData & ) 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(LongColumnStatsData &a, LongColumnStatsData &b); - -std::ostream& operator<<(std::ostream& out, const LongColumnStatsData& obj); - -typedef struct _StringColumnStatsData__isset { - _StringColumnStatsData__isset() : bitVectors(false) {} - bool bitVectors :1; -} _StringColumnStatsData__isset; - -class StringColumnStatsData : public virtual ::apache::thrift::TBase { - public: - - StringColumnStatsData(const StringColumnStatsData&); - StringColumnStatsData& operator=(const StringColumnStatsData&); - StringColumnStatsData() : maxColLen(0), avgColLen(0), numNulls(0), numDVs(0), bitVectors() { - } - - virtual ~StringColumnStatsData() noexcept; - int64_t maxColLen; - double avgColLen; - int64_t numNulls; - int64_t numDVs; - std::string bitVectors; - - _StringColumnStatsData__isset __isset; - - void __set_maxColLen(const int64_t val); - - void __set_avgColLen(const double val); - - void __set_numNulls(const int64_t val); - - void __set_numDVs(const int64_t val); - - void __set_bitVectors(const std::string& val); - - bool operator == (const StringColumnStatsData & rhs) const - { - if (!(maxColLen == rhs.maxColLen)) - return false; - if (!(avgColLen == rhs.avgColLen)) - return false; - if (!(numNulls == rhs.numNulls)) - return false; - if (!(numDVs == rhs.numDVs)) - return false; - if (__isset.bitVectors != rhs.__isset.bitVectors) - return false; - else if (__isset.bitVectors && !(bitVectors == rhs.bitVectors)) - return false; - return true; - } - bool operator != (const StringColumnStatsData &rhs) const { - return !(*this == rhs); - } - - bool operator < (const StringColumnStatsData & ) 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(StringColumnStatsData &a, StringColumnStatsData &b); - -std::ostream& operator<<(std::ostream& out, const StringColumnStatsData& obj); - -typedef struct _BinaryColumnStatsData__isset { - _BinaryColumnStatsData__isset() : bitVectors(false) {} - bool bitVectors :1; -} _BinaryColumnStatsData__isset; - -class BinaryColumnStatsData : public virtual ::apache::thrift::TBase { - public: - - BinaryColumnStatsData(const BinaryColumnStatsData&); - BinaryColumnStatsData& operator=(const BinaryColumnStatsData&); - BinaryColumnStatsData() : maxColLen(0), avgColLen(0), numNulls(0), bitVectors() { - } - - virtual ~BinaryColumnStatsData() noexcept; - int64_t maxColLen; - double avgColLen; - int64_t numNulls; - std::string bitVectors; - - _BinaryColumnStatsData__isset __isset; - - void __set_maxColLen(const int64_t val); - - void __set_avgColLen(const double val); - - void __set_numNulls(const int64_t val); - - void __set_bitVectors(const std::string& val); - - bool operator == (const BinaryColumnStatsData & rhs) const - { - if (!(maxColLen == rhs.maxColLen)) - return false; - if (!(avgColLen == rhs.avgColLen)) - return false; - if (!(numNulls == rhs.numNulls)) - return false; - if (__isset.bitVectors != rhs.__isset.bitVectors) - return false; - else if (__isset.bitVectors && !(bitVectors == rhs.bitVectors)) - return false; - return true; - } - bool operator != (const BinaryColumnStatsData &rhs) const { - return !(*this == rhs); - } - - bool operator < (const BinaryColumnStatsData & ) 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(BinaryColumnStatsData &a, BinaryColumnStatsData &b); - -std::ostream& operator<<(std::ostream& out, const BinaryColumnStatsData& obj); - - -class Decimal : public virtual ::apache::thrift::TBase { - public: - - Decimal(const Decimal&); - Decimal& operator=(const Decimal&); - Decimal() : unscaled(), scale(0) { - } - - virtual ~Decimal() noexcept; - std::string unscaled; - int16_t scale; - - void __set_unscaled(const std::string& val); - - void __set_scale(const int16_t val); - - bool operator == (const Decimal & rhs) const - { - if (!(unscaled == rhs.unscaled)) - return false; - if (!(scale == rhs.scale)) - return false; - return true; - } - bool operator != (const Decimal &rhs) const { - return !(*this == rhs); - } - - bool operator < (const Decimal & ) 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(Decimal &a, Decimal &b); - -std::ostream& operator<<(std::ostream& out, const Decimal& obj); - -typedef struct _DecimalColumnStatsData__isset { - _DecimalColumnStatsData__isset() : lowValue(false), highValue(false), bitVectors(false) {} - bool lowValue :1; - bool highValue :1; - bool bitVectors :1; -} _DecimalColumnStatsData__isset; - -class DecimalColumnStatsData : public virtual ::apache::thrift::TBase { - public: - - DecimalColumnStatsData(const DecimalColumnStatsData&); - DecimalColumnStatsData& operator=(const DecimalColumnStatsData&); - DecimalColumnStatsData() : numNulls(0), numDVs(0), bitVectors() { - } - - virtual ~DecimalColumnStatsData() noexcept; - Decimal lowValue; - Decimal highValue; - int64_t numNulls; - int64_t numDVs; - std::string bitVectors; - - _DecimalColumnStatsData__isset __isset; - - void __set_lowValue(const Decimal& val); - - void __set_highValue(const Decimal& val); - - void __set_numNulls(const int64_t val); - - void __set_numDVs(const int64_t val); - - void __set_bitVectors(const std::string& val); - - bool operator == (const DecimalColumnStatsData & rhs) const - { - if (__isset.lowValue != rhs.__isset.lowValue) - return false; - else if (__isset.lowValue && !(lowValue == rhs.lowValue)) - return false; - if (__isset.highValue != rhs.__isset.highValue) - return false; - else if (__isset.highValue && !(highValue == rhs.highValue)) - return false; - if (!(numNulls == rhs.numNulls)) - return false; - if (!(numDVs == rhs.numDVs)) - return false; - if (__isset.bitVectors != rhs.__isset.bitVectors) - return false; - else if (__isset.bitVectors && !(bitVectors == rhs.bitVectors)) - return false; - return true; - } - bool operator != (const DecimalColumnStatsData &rhs) const { - return !(*this == rhs); - } - - bool operator < (const DecimalColumnStatsData & ) 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(DecimalColumnStatsData &a, DecimalColumnStatsData &b); - -std::ostream& operator<<(std::ostream& out, const DecimalColumnStatsData& obj); - - -class Date : public virtual ::apache::thrift::TBase { - public: - - Date(const Date&); - Date& operator=(const Date&); - Date() : daysSinceEpoch(0) { - } - - virtual ~Date() noexcept; - int64_t daysSinceEpoch; - - void __set_daysSinceEpoch(const int64_t val); - - bool operator == (const Date & rhs) const - { - if (!(daysSinceEpoch == rhs.daysSinceEpoch)) - return false; - return true; - } - bool operator != (const Date &rhs) const { - return !(*this == rhs); - } - - bool operator < (const Date & ) 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(Date &a, Date &b); - -std::ostream& operator<<(std::ostream& out, const Date& obj); - -typedef struct _DateColumnStatsData__isset { - _DateColumnStatsData__isset() : lowValue(false), highValue(false), bitVectors(false) {} - bool lowValue :1; - bool highValue :1; - bool bitVectors :1; -} _DateColumnStatsData__isset; - -class DateColumnStatsData : public virtual ::apache::thrift::TBase { - public: - - DateColumnStatsData(const DateColumnStatsData&); - DateColumnStatsData& operator=(const DateColumnStatsData&); - DateColumnStatsData() : numNulls(0), numDVs(0), bitVectors() { - } - - virtual ~DateColumnStatsData() noexcept; - Date lowValue; - Date highValue; - int64_t numNulls; - int64_t numDVs; - std::string bitVectors; - - _DateColumnStatsData__isset __isset; - - void __set_lowValue(const Date& val); - - void __set_highValue(const Date& val); - - void __set_numNulls(const int64_t val); - - void __set_numDVs(const int64_t val); - - void __set_bitVectors(const std::string& val); - - bool operator == (const DateColumnStatsData & rhs) const - { - if (__isset.lowValue != rhs.__isset.lowValue) - return false; - else if (__isset.lowValue && !(lowValue == rhs.lowValue)) - return false; - if (__isset.highValue != rhs.__isset.highValue) - return false; - else if (__isset.highValue && !(highValue == rhs.highValue)) - return false; - if (!(numNulls == rhs.numNulls)) - return false; - if (!(numDVs == rhs.numDVs)) - return false; - if (__isset.bitVectors != rhs.__isset.bitVectors) - return false; - else if (__isset.bitVectors && !(bitVectors == rhs.bitVectors)) - return false; - return true; - } - bool operator != (const DateColumnStatsData &rhs) const { - return !(*this == rhs); - } - - bool operator < (const DateColumnStatsData & ) 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(DateColumnStatsData &a, DateColumnStatsData &b); - -std::ostream& operator<<(std::ostream& out, const DateColumnStatsData& obj); - -typedef struct _ColumnStatisticsData__isset { - _ColumnStatisticsData__isset() : booleanStats(false), longStats(false), doubleStats(false), stringStats(false), binaryStats(false), decimalStats(false), dateStats(false) {} - bool booleanStats :1; - bool longStats :1; - bool doubleStats :1; - bool stringStats :1; - bool binaryStats :1; - bool decimalStats :1; - bool dateStats :1; -} _ColumnStatisticsData__isset; - -class ColumnStatisticsData : public virtual ::apache::thrift::TBase { - public: - - ColumnStatisticsData(const ColumnStatisticsData&); - ColumnStatisticsData& operator=(const ColumnStatisticsData&); - ColumnStatisticsData() { - } - - virtual ~ColumnStatisticsData() noexcept; - BooleanColumnStatsData booleanStats; - LongColumnStatsData longStats; - DoubleColumnStatsData doubleStats; - StringColumnStatsData stringStats; - BinaryColumnStatsData binaryStats; - DecimalColumnStatsData decimalStats; - DateColumnStatsData dateStats; - - _ColumnStatisticsData__isset __isset; - - void __set_booleanStats(const BooleanColumnStatsData& val); - - void __set_longStats(const LongColumnStatsData& val); - - void __set_doubleStats(const DoubleColumnStatsData& val); - - void __set_stringStats(const StringColumnStatsData& val); - - void __set_binaryStats(const BinaryColumnStatsData& val); - - void __set_decimalStats(const DecimalColumnStatsData& val); - - void __set_dateStats(const DateColumnStatsData& val); - - bool operator == (const ColumnStatisticsData & rhs) const - { - if (__isset.booleanStats != rhs.__isset.booleanStats) - return false; - else if (__isset.booleanStats && !(booleanStats == rhs.booleanStats)) - return false; - if (__isset.longStats != rhs.__isset.longStats) - return false; - else if (__isset.longStats && !(longStats == rhs.longStats)) - return false; - if (__isset.doubleStats != rhs.__isset.doubleStats) - return false; - else if (__isset.doubleStats && !(doubleStats == rhs.doubleStats)) - return false; - if (__isset.stringStats != rhs.__isset.stringStats) - return false; - else if (__isset.stringStats && !(stringStats == rhs.stringStats)) - return false; - if (__isset.binaryStats != rhs.__isset.binaryStats) - return false; - else if (__isset.binaryStats && !(binaryStats == rhs.binaryStats)) - return false; - if (__isset.decimalStats != rhs.__isset.decimalStats) - return false; - else if (__isset.decimalStats && !(decimalStats == rhs.decimalStats)) - return false; - if (__isset.dateStats != rhs.__isset.dateStats) - return false; - else if (__isset.dateStats && !(dateStats == rhs.dateStats)) - return false; - return true; - } - bool operator != (const ColumnStatisticsData &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ColumnStatisticsData & ) 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(ColumnStatisticsData &a, ColumnStatisticsData &b); - -std::ostream& operator<<(std::ostream& out, const ColumnStatisticsData& obj); - - -class ColumnStatisticsObj : public virtual ::apache::thrift::TBase { - public: - - ColumnStatisticsObj(const ColumnStatisticsObj&); - ColumnStatisticsObj& operator=(const ColumnStatisticsObj&); - ColumnStatisticsObj() : colName(), colType() { - } - - virtual ~ColumnStatisticsObj() noexcept; - std::string colName; - std::string colType; - ColumnStatisticsData statsData; - - void __set_colName(const std::string& val); - - void __set_colType(const std::string& val); - - void __set_statsData(const ColumnStatisticsData& val); - - bool operator == (const ColumnStatisticsObj & rhs) const - { - if (!(colName == rhs.colName)) - return false; - if (!(colType == rhs.colType)) - return false; - if (!(statsData == rhs.statsData)) - return false; - return true; - } - bool operator != (const ColumnStatisticsObj &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ColumnStatisticsObj & ) 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(ColumnStatisticsObj &a, ColumnStatisticsObj &b); - -std::ostream& operator<<(std::ostream& out, const ColumnStatisticsObj& obj); - -typedef struct _ColumnStatisticsDesc__isset { - _ColumnStatisticsDesc__isset() : partName(false), lastAnalyzed(false) {} - bool partName :1; - bool lastAnalyzed :1; -} _ColumnStatisticsDesc__isset; - -class ColumnStatisticsDesc : public virtual ::apache::thrift::TBase { - public: - - ColumnStatisticsDesc(const ColumnStatisticsDesc&); - ColumnStatisticsDesc& operator=(const ColumnStatisticsDesc&); - ColumnStatisticsDesc() : isTblLevel(0), dbName(), tableName(), partName(), lastAnalyzed(0) { - } - - virtual ~ColumnStatisticsDesc() noexcept; - bool isTblLevel; - std::string dbName; - std::string tableName; - std::string partName; - int64_t lastAnalyzed; - - _ColumnStatisticsDesc__isset __isset; - - void __set_isTblLevel(const bool val); - - void __set_dbName(const std::string& val); - - void __set_tableName(const std::string& val); - - void __set_partName(const std::string& val); - - void __set_lastAnalyzed(const int64_t val); - - bool operator == (const ColumnStatisticsDesc & rhs) const - { - if (!(isTblLevel == rhs.isTblLevel)) - return false; - if (!(dbName == rhs.dbName)) - return false; - if (!(tableName == rhs.tableName)) - return false; - if (__isset.partName != rhs.__isset.partName) - return false; - else if (__isset.partName && !(partName == rhs.partName)) - return false; - if (__isset.lastAnalyzed != rhs.__isset.lastAnalyzed) - return false; - else if (__isset.lastAnalyzed && !(lastAnalyzed == rhs.lastAnalyzed)) - return false; - return true; - } - bool operator != (const ColumnStatisticsDesc &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ColumnStatisticsDesc & ) 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(ColumnStatisticsDesc &a, ColumnStatisticsDesc &b); - -std::ostream& operator<<(std::ostream& out, const ColumnStatisticsDesc& obj); - - -class ColumnStatistics : public virtual ::apache::thrift::TBase { - public: - - ColumnStatistics(const ColumnStatistics&); - ColumnStatistics& operator=(const ColumnStatistics&); - ColumnStatistics() { - } - - virtual ~ColumnStatistics() noexcept; - ColumnStatisticsDesc statsDesc; - std::vector statsObj; - - void __set_statsDesc(const ColumnStatisticsDesc& val); - - void __set_statsObj(const std::vector & val); - - bool operator == (const ColumnStatistics & rhs) const - { - if (!(statsDesc == rhs.statsDesc)) - return false; - if (!(statsObj == rhs.statsObj)) - return false; - return true; - } - bool operator != (const ColumnStatistics &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ColumnStatistics & ) 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(ColumnStatistics &a, ColumnStatistics &b); - -std::ostream& operator<<(std::ostream& out, const ColumnStatistics& obj); - - -class AggrStats : public virtual ::apache::thrift::TBase { - public: - - AggrStats(const AggrStats&); - AggrStats& operator=(const AggrStats&); - AggrStats() : partsFound(0) { - } - - virtual ~AggrStats() noexcept; - std::vector colStats; - int64_t partsFound; - - void __set_colStats(const std::vector & val); - - void __set_partsFound(const int64_t val); - - bool operator == (const AggrStats & rhs) const - { - if (!(colStats == rhs.colStats)) - return false; - if (!(partsFound == rhs.partsFound)) - return false; - return true; - } - bool operator != (const AggrStats &rhs) const { - return !(*this == rhs); - } - - bool operator < (const AggrStats & ) 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(AggrStats &a, AggrStats &b); - -std::ostream& operator<<(std::ostream& out, const AggrStats& obj); - -typedef struct _SetPartitionsStatsRequest__isset { - _SetPartitionsStatsRequest__isset() : needMerge(false) {} - bool needMerge :1; -} _SetPartitionsStatsRequest__isset; - -class SetPartitionsStatsRequest : public virtual ::apache::thrift::TBase { - public: - - SetPartitionsStatsRequest(const SetPartitionsStatsRequest&); - SetPartitionsStatsRequest& operator=(const SetPartitionsStatsRequest&); - SetPartitionsStatsRequest() : needMerge(0) { - } - - virtual ~SetPartitionsStatsRequest() noexcept; - std::vector colStats; - bool needMerge; - - _SetPartitionsStatsRequest__isset __isset; - - void __set_colStats(const std::vector & val); - - void __set_needMerge(const bool val); - - bool operator == (const SetPartitionsStatsRequest & rhs) const - { - if (!(colStats == rhs.colStats)) - return false; - if (__isset.needMerge != rhs.__isset.needMerge) - return false; - else if (__isset.needMerge && !(needMerge == rhs.needMerge)) - return false; - return true; - } - bool operator != (const SetPartitionsStatsRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const SetPartitionsStatsRequest & ) 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(SetPartitionsStatsRequest &a, SetPartitionsStatsRequest &b); - -std::ostream& operator<<(std::ostream& out, const SetPartitionsStatsRequest& obj); - -typedef struct _Schema__isset { - _Schema__isset() : fieldSchemas(false), properties(false) {} - bool fieldSchemas :1; - bool properties :1; -} _Schema__isset; - -class Schema : public virtual ::apache::thrift::TBase { - public: - - Schema(const Schema&); - Schema& operator=(const Schema&); - Schema() { - } - - virtual ~Schema() noexcept; - std::vector fieldSchemas; - std::map properties; - - _Schema__isset __isset; - - void __set_fieldSchemas(const std::vector & val); - - void __set_properties(const std::map & val); - - bool operator == (const Schema & rhs) const - { - if (!(fieldSchemas == rhs.fieldSchemas)) - return false; - if (!(properties == rhs.properties)) - return false; - return true; - } - bool operator != (const Schema &rhs) const { - return !(*this == rhs); - } - - bool operator < (const Schema & ) 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(Schema &a, Schema &b); - -std::ostream& operator<<(std::ostream& out, const Schema& obj); - -typedef struct _EnvironmentContext__isset { - _EnvironmentContext__isset() : properties(false) {} - bool properties :1; -} _EnvironmentContext__isset; - -class EnvironmentContext : public virtual ::apache::thrift::TBase { - public: - - EnvironmentContext(const EnvironmentContext&); - EnvironmentContext& operator=(const EnvironmentContext&); - EnvironmentContext() { - } - - virtual ~EnvironmentContext() noexcept; - std::map properties; - - _EnvironmentContext__isset __isset; - - void __set_properties(const std::map & val); - - bool operator == (const EnvironmentContext & rhs) const - { - if (!(properties == rhs.properties)) - return false; - return true; - } - bool operator != (const EnvironmentContext &rhs) const { - return !(*this == rhs); - } - - bool operator < (const EnvironmentContext & ) 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(EnvironmentContext &a, EnvironmentContext &b); - -std::ostream& operator<<(std::ostream& out, const EnvironmentContext& obj); - - -class PrimaryKeysRequest : public virtual ::apache::thrift::TBase { - public: - - PrimaryKeysRequest(const PrimaryKeysRequest&); - PrimaryKeysRequest& operator=(const PrimaryKeysRequest&); - PrimaryKeysRequest() : db_name(), tbl_name() { - } - - virtual ~PrimaryKeysRequest() noexcept; - std::string db_name; - std::string tbl_name; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - bool operator == (const PrimaryKeysRequest & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - return true; - } - bool operator != (const PrimaryKeysRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PrimaryKeysRequest & ) 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(PrimaryKeysRequest &a, PrimaryKeysRequest &b); - -std::ostream& operator<<(std::ostream& out, const PrimaryKeysRequest& obj); - - -class PrimaryKeysResponse : public virtual ::apache::thrift::TBase { - public: - - PrimaryKeysResponse(const PrimaryKeysResponse&); - PrimaryKeysResponse& operator=(const PrimaryKeysResponse&); - PrimaryKeysResponse() { - } - - virtual ~PrimaryKeysResponse() noexcept; - std::vector primaryKeys; - - void __set_primaryKeys(const std::vector & val); - - bool operator == (const PrimaryKeysResponse & rhs) const - { - if (!(primaryKeys == rhs.primaryKeys)) - return false; - return true; - } - bool operator != (const PrimaryKeysResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PrimaryKeysResponse & ) 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(PrimaryKeysResponse &a, PrimaryKeysResponse &b); - -std::ostream& operator<<(std::ostream& out, const PrimaryKeysResponse& obj); - -typedef struct _ForeignKeysRequest__isset { - _ForeignKeysRequest__isset() : parent_db_name(false), parent_tbl_name(false), foreign_db_name(false), foreign_tbl_name(false) {} - bool parent_db_name :1; - bool parent_tbl_name :1; - bool foreign_db_name :1; - bool foreign_tbl_name :1; -} _ForeignKeysRequest__isset; - -class ForeignKeysRequest : public virtual ::apache::thrift::TBase { - public: - - ForeignKeysRequest(const ForeignKeysRequest&); - ForeignKeysRequest& operator=(const ForeignKeysRequest&); - ForeignKeysRequest() : parent_db_name(), parent_tbl_name(), foreign_db_name(), foreign_tbl_name() { - } - - virtual ~ForeignKeysRequest() noexcept; - std::string parent_db_name; - std::string parent_tbl_name; - std::string foreign_db_name; - std::string foreign_tbl_name; - - _ForeignKeysRequest__isset __isset; - - void __set_parent_db_name(const std::string& val); - - void __set_parent_tbl_name(const std::string& val); - - void __set_foreign_db_name(const std::string& val); - - void __set_foreign_tbl_name(const std::string& val); - - bool operator == (const ForeignKeysRequest & rhs) const - { - if (!(parent_db_name == rhs.parent_db_name)) - return false; - if (!(parent_tbl_name == rhs.parent_tbl_name)) - return false; - if (!(foreign_db_name == rhs.foreign_db_name)) - return false; - if (!(foreign_tbl_name == rhs.foreign_tbl_name)) - return false; - return true; - } - bool operator != (const ForeignKeysRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ForeignKeysRequest & ) 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(ForeignKeysRequest &a, ForeignKeysRequest &b); - -std::ostream& operator<<(std::ostream& out, const ForeignKeysRequest& obj); - - -class ForeignKeysResponse : public virtual ::apache::thrift::TBase { - public: - - ForeignKeysResponse(const ForeignKeysResponse&); - ForeignKeysResponse& operator=(const ForeignKeysResponse&); - ForeignKeysResponse() { - } - - virtual ~ForeignKeysResponse() noexcept; - std::vector foreignKeys; - - void __set_foreignKeys(const std::vector & val); - - bool operator == (const ForeignKeysResponse & rhs) const - { - if (!(foreignKeys == rhs.foreignKeys)) - return false; - return true; - } - bool operator != (const ForeignKeysResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ForeignKeysResponse & ) 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(ForeignKeysResponse &a, ForeignKeysResponse &b); - -std::ostream& operator<<(std::ostream& out, const ForeignKeysResponse& obj); - - -class UniqueConstraintsRequest : public virtual ::apache::thrift::TBase { - public: - - UniqueConstraintsRequest(const UniqueConstraintsRequest&); - UniqueConstraintsRequest& operator=(const UniqueConstraintsRequest&); - UniqueConstraintsRequest() : db_name(), tbl_name() { - } - - virtual ~UniqueConstraintsRequest() noexcept; - std::string db_name; - std::string tbl_name; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - bool operator == (const UniqueConstraintsRequest & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - return true; - } - bool operator != (const UniqueConstraintsRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const UniqueConstraintsRequest & ) 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(UniqueConstraintsRequest &a, UniqueConstraintsRequest &b); - -std::ostream& operator<<(std::ostream& out, const UniqueConstraintsRequest& obj); - - -class UniqueConstraintsResponse : public virtual ::apache::thrift::TBase { - public: - - UniqueConstraintsResponse(const UniqueConstraintsResponse&); - UniqueConstraintsResponse& operator=(const UniqueConstraintsResponse&); - UniqueConstraintsResponse() { - } - - virtual ~UniqueConstraintsResponse() noexcept; - std::vector uniqueConstraints; - - void __set_uniqueConstraints(const std::vector & val); - - bool operator == (const UniqueConstraintsResponse & rhs) const - { - if (!(uniqueConstraints == rhs.uniqueConstraints)) - return false; - return true; - } - bool operator != (const UniqueConstraintsResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const UniqueConstraintsResponse & ) 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(UniqueConstraintsResponse &a, UniqueConstraintsResponse &b); - -std::ostream& operator<<(std::ostream& out, const UniqueConstraintsResponse& obj); - - -class NotNullConstraintsRequest : public virtual ::apache::thrift::TBase { - public: - - NotNullConstraintsRequest(const NotNullConstraintsRequest&); - NotNullConstraintsRequest& operator=(const NotNullConstraintsRequest&); - NotNullConstraintsRequest() : db_name(), tbl_name() { - } - - virtual ~NotNullConstraintsRequest() noexcept; - std::string db_name; - std::string tbl_name; - - void __set_db_name(const std::string& val); - - void __set_tbl_name(const std::string& val); - - bool operator == (const NotNullConstraintsRequest & rhs) const - { - if (!(db_name == rhs.db_name)) - return false; - if (!(tbl_name == rhs.tbl_name)) - return false; - return true; - } - bool operator != (const NotNullConstraintsRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const NotNullConstraintsRequest & ) 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(NotNullConstraintsRequest &a, NotNullConstraintsRequest &b); - -std::ostream& operator<<(std::ostream& out, const NotNullConstraintsRequest& obj); - - -class NotNullConstraintsResponse : public virtual ::apache::thrift::TBase { - public: - - NotNullConstraintsResponse(const NotNullConstraintsResponse&); - NotNullConstraintsResponse& operator=(const NotNullConstraintsResponse&); - NotNullConstraintsResponse() { - } - - virtual ~NotNullConstraintsResponse() noexcept; - std::vector notNullConstraints; - - void __set_notNullConstraints(const std::vector & val); - - bool operator == (const NotNullConstraintsResponse & rhs) const - { - if (!(notNullConstraints == rhs.notNullConstraints)) - return false; - return true; - } - bool operator != (const NotNullConstraintsResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const NotNullConstraintsResponse & ) 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(NotNullConstraintsResponse &a, NotNullConstraintsResponse &b); - -std::ostream& operator<<(std::ostream& out, const NotNullConstraintsResponse& obj); - - -class DropConstraintRequest : public virtual ::apache::thrift::TBase { - public: - - DropConstraintRequest(const DropConstraintRequest&); - DropConstraintRequest& operator=(const DropConstraintRequest&); - DropConstraintRequest() : dbname(), tablename(), constraintname() { - } - - virtual ~DropConstraintRequest() noexcept; - std::string dbname; - std::string tablename; - std::string constraintname; - - void __set_dbname(const std::string& val); - - void __set_tablename(const std::string& val); - - void __set_constraintname(const std::string& val); - - bool operator == (const DropConstraintRequest & rhs) const - { - if (!(dbname == rhs.dbname)) - return false; - if (!(tablename == rhs.tablename)) - return false; - if (!(constraintname == rhs.constraintname)) - return false; - return true; - } - bool operator != (const DropConstraintRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const DropConstraintRequest & ) 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(DropConstraintRequest &a, DropConstraintRequest &b); - -std::ostream& operator<<(std::ostream& out, const DropConstraintRequest& obj); - - -class AddPrimaryKeyRequest : public virtual ::apache::thrift::TBase { - public: - - AddPrimaryKeyRequest(const AddPrimaryKeyRequest&); - AddPrimaryKeyRequest& operator=(const AddPrimaryKeyRequest&); - AddPrimaryKeyRequest() { - } - - virtual ~AddPrimaryKeyRequest() noexcept; - std::vector primaryKeyCols; - - void __set_primaryKeyCols(const std::vector & val); - - bool operator == (const AddPrimaryKeyRequest & rhs) const - { - if (!(primaryKeyCols == rhs.primaryKeyCols)) - return false; - return true; - } - bool operator != (const AddPrimaryKeyRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const AddPrimaryKeyRequest & ) 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(AddPrimaryKeyRequest &a, AddPrimaryKeyRequest &b); - -std::ostream& operator<<(std::ostream& out, const AddPrimaryKeyRequest& obj); - - -class AddForeignKeyRequest : public virtual ::apache::thrift::TBase { - public: - - AddForeignKeyRequest(const AddForeignKeyRequest&); - AddForeignKeyRequest& operator=(const AddForeignKeyRequest&); - AddForeignKeyRequest() { - } - - virtual ~AddForeignKeyRequest() noexcept; - std::vector foreignKeyCols; - - void __set_foreignKeyCols(const std::vector & val); - - bool operator == (const AddForeignKeyRequest & rhs) const - { - if (!(foreignKeyCols == rhs.foreignKeyCols)) - return false; - return true; - } - bool operator != (const AddForeignKeyRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const AddForeignKeyRequest & ) 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(AddForeignKeyRequest &a, AddForeignKeyRequest &b); - -std::ostream& operator<<(std::ostream& out, const AddForeignKeyRequest& obj); - - -class AddUniqueConstraintRequest : public virtual ::apache::thrift::TBase { - public: - - AddUniqueConstraintRequest(const AddUniqueConstraintRequest&); - AddUniqueConstraintRequest& operator=(const AddUniqueConstraintRequest&); - AddUniqueConstraintRequest() { - } - - virtual ~AddUniqueConstraintRequest() noexcept; - std::vector uniqueConstraintCols; - - void __set_uniqueConstraintCols(const std::vector & val); - - bool operator == (const AddUniqueConstraintRequest & rhs) const - { - if (!(uniqueConstraintCols == rhs.uniqueConstraintCols)) - return false; - return true; - } - bool operator != (const AddUniqueConstraintRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const AddUniqueConstraintRequest & ) 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(AddUniqueConstraintRequest &a, AddUniqueConstraintRequest &b); - -std::ostream& operator<<(std::ostream& out, const AddUniqueConstraintRequest& obj); - - -class AddNotNullConstraintRequest : public virtual ::apache::thrift::TBase { - public: - - AddNotNullConstraintRequest(const AddNotNullConstraintRequest&); - AddNotNullConstraintRequest& operator=(const AddNotNullConstraintRequest&); - AddNotNullConstraintRequest() { - } - - virtual ~AddNotNullConstraintRequest() noexcept; - std::vector notNullConstraintCols; - - void __set_notNullConstraintCols(const std::vector & val); - - bool operator == (const AddNotNullConstraintRequest & rhs) const - { - if (!(notNullConstraintCols == rhs.notNullConstraintCols)) - return false; - return true; - } - bool operator != (const AddNotNullConstraintRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const AddNotNullConstraintRequest & ) 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(AddNotNullConstraintRequest &a, AddNotNullConstraintRequest &b); - -std::ostream& operator<<(std::ostream& out, const AddNotNullConstraintRequest& obj); - - -class PartitionsByExprResult : public virtual ::apache::thrift::TBase { - public: - - PartitionsByExprResult(const PartitionsByExprResult&); - PartitionsByExprResult& operator=(const PartitionsByExprResult&); - PartitionsByExprResult() : hasUnknownPartitions(0) { - } - - virtual ~PartitionsByExprResult() noexcept; - std::vector partitions; - bool hasUnknownPartitions; - - void __set_partitions(const std::vector & val); - - void __set_hasUnknownPartitions(const bool val); - - bool operator == (const PartitionsByExprResult & rhs) const - { - if (!(partitions == rhs.partitions)) - return false; - if (!(hasUnknownPartitions == rhs.hasUnknownPartitions)) - return false; - return true; - } - bool operator != (const PartitionsByExprResult &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PartitionsByExprResult & ) 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(PartitionsByExprResult &a, PartitionsByExprResult &b); - -std::ostream& operator<<(std::ostream& out, const PartitionsByExprResult& obj); - -typedef struct _PartitionsByExprRequest__isset { - _PartitionsByExprRequest__isset() : defaultPartitionName(false), maxParts(true) {} - bool defaultPartitionName :1; - bool maxParts :1; -} _PartitionsByExprRequest__isset; - -class PartitionsByExprRequest : public virtual ::apache::thrift::TBase { - public: - - PartitionsByExprRequest(const PartitionsByExprRequest&); - PartitionsByExprRequest& operator=(const PartitionsByExprRequest&); - PartitionsByExprRequest() : dbName(), tblName(), expr(), defaultPartitionName(), maxParts(-1) { - } - - virtual ~PartitionsByExprRequest() noexcept; - std::string dbName; - std::string tblName; - std::string expr; - std::string defaultPartitionName; - int16_t maxParts; - - _PartitionsByExprRequest__isset __isset; - - void __set_dbName(const std::string& val); - - void __set_tblName(const std::string& val); - - void __set_expr(const std::string& val); - - void __set_defaultPartitionName(const std::string& val); - - void __set_maxParts(const int16_t val); - - bool operator == (const PartitionsByExprRequest & rhs) const - { - if (!(dbName == rhs.dbName)) - return false; - if (!(tblName == rhs.tblName)) - return false; - if (!(expr == rhs.expr)) - return false; - if (__isset.defaultPartitionName != rhs.__isset.defaultPartitionName) - return false; - else if (__isset.defaultPartitionName && !(defaultPartitionName == rhs.defaultPartitionName)) - return false; - if (__isset.maxParts != rhs.__isset.maxParts) - return false; - else if (__isset.maxParts && !(maxParts == rhs.maxParts)) - return false; - return true; - } - bool operator != (const PartitionsByExprRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PartitionsByExprRequest & ) 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(PartitionsByExprRequest &a, PartitionsByExprRequest &b); - -std::ostream& operator<<(std::ostream& out, const PartitionsByExprRequest& obj); - - -class TableStatsResult : public virtual ::apache::thrift::TBase { - public: - - TableStatsResult(const TableStatsResult&); - TableStatsResult& operator=(const TableStatsResult&); - TableStatsResult() { - } - - virtual ~TableStatsResult() noexcept; - std::vector tableStats; - - void __set_tableStats(const std::vector & val); - - bool operator == (const TableStatsResult & rhs) const - { - if (!(tableStats == rhs.tableStats)) - return false; - return true; - } - bool operator != (const TableStatsResult &rhs) const { - return !(*this == rhs); - } - - bool operator < (const TableStatsResult & ) 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(TableStatsResult &a, TableStatsResult &b); - -std::ostream& operator<<(std::ostream& out, const TableStatsResult& obj); - - -class PartitionsStatsResult : public virtual ::apache::thrift::TBase { - public: - - PartitionsStatsResult(const PartitionsStatsResult&); - PartitionsStatsResult& operator=(const PartitionsStatsResult&); - PartitionsStatsResult() { - } - - virtual ~PartitionsStatsResult() noexcept; - std::map > partStats; - - void __set_partStats(const std::map > & val); - - bool operator == (const PartitionsStatsResult & rhs) const - { - if (!(partStats == rhs.partStats)) - return false; - return true; - } - bool operator != (const PartitionsStatsResult &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PartitionsStatsResult & ) 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(PartitionsStatsResult &a, PartitionsStatsResult &b); - -std::ostream& operator<<(std::ostream& out, const PartitionsStatsResult& obj); - - -class TableStatsRequest : public virtual ::apache::thrift::TBase { - public: - - TableStatsRequest(const TableStatsRequest&); - TableStatsRequest& operator=(const TableStatsRequest&); - TableStatsRequest() : dbName(), tblName() { - } - - virtual ~TableStatsRequest() noexcept; - std::string dbName; - std::string tblName; - std::vector colNames; - - void __set_dbName(const std::string& val); - - void __set_tblName(const std::string& val); - - void __set_colNames(const std::vector & val); - - bool operator == (const TableStatsRequest & rhs) const - { - if (!(dbName == rhs.dbName)) - return false; - if (!(tblName == rhs.tblName)) - return false; - if (!(colNames == rhs.colNames)) - return false; - return true; - } - bool operator != (const TableStatsRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const TableStatsRequest & ) 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(TableStatsRequest &a, TableStatsRequest &b); - -std::ostream& operator<<(std::ostream& out, const TableStatsRequest& obj); - - -class PartitionsStatsRequest : public virtual ::apache::thrift::TBase { - public: - - PartitionsStatsRequest(const PartitionsStatsRequest&); - PartitionsStatsRequest& operator=(const PartitionsStatsRequest&); - PartitionsStatsRequest() : dbName(), tblName() { - } - - virtual ~PartitionsStatsRequest() noexcept; - std::string dbName; - std::string tblName; - std::vector colNames; - std::vector partNames; - - void __set_dbName(const std::string& val); - - void __set_tblName(const std::string& val); - - void __set_colNames(const std::vector & val); - - void __set_partNames(const std::vector & val); - - bool operator == (const PartitionsStatsRequest & rhs) const - { - if (!(dbName == rhs.dbName)) - return false; - if (!(tblName == rhs.tblName)) - return false; - if (!(colNames == rhs.colNames)) - return false; - if (!(partNames == rhs.partNames)) - return false; - return true; - } - bool operator != (const PartitionsStatsRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PartitionsStatsRequest & ) 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(PartitionsStatsRequest &a, PartitionsStatsRequest &b); - -std::ostream& operator<<(std::ostream& out, const PartitionsStatsRequest& obj); - -typedef struct _AddPartitionsResult__isset { - _AddPartitionsResult__isset() : partitions(false) {} - bool partitions :1; -} _AddPartitionsResult__isset; - -class AddPartitionsResult : public virtual ::apache::thrift::TBase { - public: - - AddPartitionsResult(const AddPartitionsResult&); - AddPartitionsResult& operator=(const AddPartitionsResult&); - AddPartitionsResult() { - } - - virtual ~AddPartitionsResult() noexcept; - std::vector partitions; - - _AddPartitionsResult__isset __isset; - - void __set_partitions(const std::vector & val); - - bool operator == (const AddPartitionsResult & rhs) const - { - if (__isset.partitions != rhs.__isset.partitions) - return false; - else if (__isset.partitions && !(partitions == rhs.partitions)) - return false; - return true; - } - bool operator != (const AddPartitionsResult &rhs) const { - return !(*this == rhs); - } - - bool operator < (const AddPartitionsResult & ) 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(AddPartitionsResult &a, AddPartitionsResult &b); - -std::ostream& operator<<(std::ostream& out, const AddPartitionsResult& obj); - -typedef struct _AddPartitionsRequest__isset { - _AddPartitionsRequest__isset() : needResult(true) {} - bool needResult :1; -} _AddPartitionsRequest__isset; - -class AddPartitionsRequest : public virtual ::apache::thrift::TBase { - public: - - AddPartitionsRequest(const AddPartitionsRequest&); - AddPartitionsRequest& operator=(const AddPartitionsRequest&); - AddPartitionsRequest() : dbName(), tblName(), ifNotExists(0), needResult(true) { - } - - virtual ~AddPartitionsRequest() noexcept; - std::string dbName; - std::string tblName; - std::vector parts; - bool ifNotExists; - bool needResult; - - _AddPartitionsRequest__isset __isset; - - void __set_dbName(const std::string& val); - - void __set_tblName(const std::string& val); - - void __set_parts(const std::vector & val); - - void __set_ifNotExists(const bool val); - - void __set_needResult(const bool val); - - bool operator == (const AddPartitionsRequest & rhs) const - { - if (!(dbName == rhs.dbName)) - return false; - if (!(tblName == rhs.tblName)) - return false; - if (!(parts == rhs.parts)) - return false; - if (!(ifNotExists == rhs.ifNotExists)) - return false; - if (__isset.needResult != rhs.__isset.needResult) - return false; - else if (__isset.needResult && !(needResult == rhs.needResult)) - return false; - return true; - } - bool operator != (const AddPartitionsRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const AddPartitionsRequest & ) 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(AddPartitionsRequest &a, AddPartitionsRequest &b); - -std::ostream& operator<<(std::ostream& out, const AddPartitionsRequest& obj); - -typedef struct _DropPartitionsResult__isset { - _DropPartitionsResult__isset() : partitions(false) {} - bool partitions :1; -} _DropPartitionsResult__isset; - -class DropPartitionsResult : public virtual ::apache::thrift::TBase { - public: - - DropPartitionsResult(const DropPartitionsResult&); - DropPartitionsResult& operator=(const DropPartitionsResult&); - DropPartitionsResult() { - } - - virtual ~DropPartitionsResult() noexcept; - std::vector partitions; - - _DropPartitionsResult__isset __isset; - - void __set_partitions(const std::vector & val); - - bool operator == (const DropPartitionsResult & rhs) const - { - if (__isset.partitions != rhs.__isset.partitions) - return false; - else if (__isset.partitions && !(partitions == rhs.partitions)) - return false; - return true; - } - bool operator != (const DropPartitionsResult &rhs) const { - return !(*this == rhs); - } - - bool operator < (const DropPartitionsResult & ) 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(DropPartitionsResult &a, DropPartitionsResult &b); - -std::ostream& operator<<(std::ostream& out, const DropPartitionsResult& obj); - -typedef struct _DropPartitionsExpr__isset { - _DropPartitionsExpr__isset() : partArchiveLevel(false) {} - bool partArchiveLevel :1; -} _DropPartitionsExpr__isset; - -class DropPartitionsExpr : public virtual ::apache::thrift::TBase { - public: - - DropPartitionsExpr(const DropPartitionsExpr&); - DropPartitionsExpr& operator=(const DropPartitionsExpr&); - DropPartitionsExpr() : expr(), partArchiveLevel(0) { - } - - virtual ~DropPartitionsExpr() noexcept; - std::string expr; - int32_t partArchiveLevel; - - _DropPartitionsExpr__isset __isset; - - void __set_expr(const std::string& val); - - void __set_partArchiveLevel(const int32_t val); - - bool operator == (const DropPartitionsExpr & rhs) const - { - if (!(expr == rhs.expr)) - return false; - if (__isset.partArchiveLevel != rhs.__isset.partArchiveLevel) - return false; - else if (__isset.partArchiveLevel && !(partArchiveLevel == rhs.partArchiveLevel)) - return false; - return true; - } - bool operator != (const DropPartitionsExpr &rhs) const { - return !(*this == rhs); - } - - bool operator < (const DropPartitionsExpr & ) 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(DropPartitionsExpr &a, DropPartitionsExpr &b); - -std::ostream& operator<<(std::ostream& out, const DropPartitionsExpr& obj); - -typedef struct _RequestPartsSpec__isset { - _RequestPartsSpec__isset() : names(false), exprs(false) {} - bool names :1; - bool exprs :1; -} _RequestPartsSpec__isset; - -class RequestPartsSpec : public virtual ::apache::thrift::TBase { - public: - - RequestPartsSpec(const RequestPartsSpec&); - RequestPartsSpec& operator=(const RequestPartsSpec&); - RequestPartsSpec() { - } - - virtual ~RequestPartsSpec() noexcept; - std::vector names; - std::vector exprs; - - _RequestPartsSpec__isset __isset; - - void __set_names(const std::vector & val); - - void __set_exprs(const std::vector & val); - - bool operator == (const RequestPartsSpec & rhs) const - { - if (__isset.names != rhs.__isset.names) - return false; - else if (__isset.names && !(names == rhs.names)) - return false; - if (__isset.exprs != rhs.__isset.exprs) - return false; - else if (__isset.exprs && !(exprs == rhs.exprs)) - return false; - return true; - } - bool operator != (const RequestPartsSpec &rhs) const { - return !(*this == rhs); - } - - bool operator < (const RequestPartsSpec & ) 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(RequestPartsSpec &a, RequestPartsSpec &b); - -std::ostream& operator<<(std::ostream& out, const RequestPartsSpec& obj); - -typedef struct _DropPartitionsRequest__isset { - _DropPartitionsRequest__isset() : deleteData(false), ifExists(true), ignoreProtection(false), environmentContext(false), needResult(true) {} - bool deleteData :1; - bool ifExists :1; - bool ignoreProtection :1; - bool environmentContext :1; - bool needResult :1; -} _DropPartitionsRequest__isset; - -class DropPartitionsRequest : public virtual ::apache::thrift::TBase { - public: - - DropPartitionsRequest(const DropPartitionsRequest&); - DropPartitionsRequest& operator=(const DropPartitionsRequest&); - DropPartitionsRequest() : dbName(), tblName(), deleteData(0), ifExists(true), ignoreProtection(0), needResult(true) { - } - - virtual ~DropPartitionsRequest() noexcept; - std::string dbName; - std::string tblName; - RequestPartsSpec parts; - bool deleteData; - bool ifExists; - bool ignoreProtection; - EnvironmentContext environmentContext; - bool needResult; - - _DropPartitionsRequest__isset __isset; - - void __set_dbName(const std::string& val); - - void __set_tblName(const std::string& val); - - void __set_parts(const RequestPartsSpec& val); - - void __set_deleteData(const bool val); - - void __set_ifExists(const bool val); - - void __set_ignoreProtection(const bool val); - - void __set_environmentContext(const EnvironmentContext& val); - - void __set_needResult(const bool val); - - bool operator == (const DropPartitionsRequest & rhs) const - { - if (!(dbName == rhs.dbName)) - return false; - if (!(tblName == rhs.tblName)) - return false; - if (!(parts == rhs.parts)) - return false; - if (__isset.deleteData != rhs.__isset.deleteData) - return false; - else if (__isset.deleteData && !(deleteData == rhs.deleteData)) - return false; - if (__isset.ifExists != rhs.__isset.ifExists) - return false; - else if (__isset.ifExists && !(ifExists == rhs.ifExists)) - return false; - if (__isset.ignoreProtection != rhs.__isset.ignoreProtection) - return false; - else if (__isset.ignoreProtection && !(ignoreProtection == rhs.ignoreProtection)) - return false; - if (__isset.environmentContext != rhs.__isset.environmentContext) - return false; - else if (__isset.environmentContext && !(environmentContext == rhs.environmentContext)) - return false; - if (__isset.needResult != rhs.__isset.needResult) - return false; - else if (__isset.needResult && !(needResult == rhs.needResult)) - return false; - return true; - } - bool operator != (const DropPartitionsRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const DropPartitionsRequest & ) 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(DropPartitionsRequest &a, DropPartitionsRequest &b); - -std::ostream& operator<<(std::ostream& out, const DropPartitionsRequest& obj); - -typedef struct _PartitionValuesRequest__isset { - _PartitionValuesRequest__isset() : applyDistinct(true), filter(false), partitionOrder(false), ascending(true), maxParts(true) {} - bool applyDistinct :1; - bool filter :1; - bool partitionOrder :1; - bool ascending :1; - bool maxParts :1; -} _PartitionValuesRequest__isset; - -class PartitionValuesRequest : public virtual ::apache::thrift::TBase { - public: - - PartitionValuesRequest(const PartitionValuesRequest&); - PartitionValuesRequest& operator=(const PartitionValuesRequest&); - PartitionValuesRequest() : dbName(), tblName(), applyDistinct(true), filter(), ascending(true), maxParts(-1LL) { - } - - virtual ~PartitionValuesRequest() noexcept; - std::string dbName; - std::string tblName; - std::vector partitionKeys; - bool applyDistinct; - std::string filter; - std::vector partitionOrder; - bool ascending; - int64_t maxParts; - - _PartitionValuesRequest__isset __isset; - - void __set_dbName(const std::string& val); - - void __set_tblName(const std::string& val); - - void __set_partitionKeys(const std::vector & val); - - void __set_applyDistinct(const bool val); - - void __set_filter(const std::string& val); - - void __set_partitionOrder(const std::vector & val); - - void __set_ascending(const bool val); - - void __set_maxParts(const int64_t val); - - bool operator == (const PartitionValuesRequest & rhs) const - { - if (!(dbName == rhs.dbName)) - return false; - if (!(tblName == rhs.tblName)) - return false; - if (!(partitionKeys == rhs.partitionKeys)) - return false; - if (__isset.applyDistinct != rhs.__isset.applyDistinct) - return false; - else if (__isset.applyDistinct && !(applyDistinct == rhs.applyDistinct)) - return false; - if (__isset.filter != rhs.__isset.filter) - return false; - else if (__isset.filter && !(filter == rhs.filter)) - return false; - if (__isset.partitionOrder != rhs.__isset.partitionOrder) - return false; - else if (__isset.partitionOrder && !(partitionOrder == rhs.partitionOrder)) - return false; - if (__isset.ascending != rhs.__isset.ascending) - return false; - else if (__isset.ascending && !(ascending == rhs.ascending)) - return false; - if (__isset.maxParts != rhs.__isset.maxParts) - return false; - else if (__isset.maxParts && !(maxParts == rhs.maxParts)) - return false; - return true; - } - bool operator != (const PartitionValuesRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PartitionValuesRequest & ) 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(PartitionValuesRequest &a, PartitionValuesRequest &b); - -std::ostream& operator<<(std::ostream& out, const PartitionValuesRequest& obj); - - -class PartitionValuesRow : public virtual ::apache::thrift::TBase { - public: - - PartitionValuesRow(const PartitionValuesRow&); - PartitionValuesRow& operator=(const PartitionValuesRow&); - PartitionValuesRow() { - } - - virtual ~PartitionValuesRow() noexcept; - std::vector row; - - void __set_row(const std::vector & val); - - bool operator == (const PartitionValuesRow & rhs) const - { - if (!(row == rhs.row)) - return false; - return true; - } - bool operator != (const PartitionValuesRow &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PartitionValuesRow & ) 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(PartitionValuesRow &a, PartitionValuesRow &b); - -std::ostream& operator<<(std::ostream& out, const PartitionValuesRow& obj); - - -class PartitionValuesResponse : public virtual ::apache::thrift::TBase { - public: - - PartitionValuesResponse(const PartitionValuesResponse&); - PartitionValuesResponse& operator=(const PartitionValuesResponse&); - PartitionValuesResponse() { - } - - virtual ~PartitionValuesResponse() noexcept; - std::vector partitionValues; - - void __set_partitionValues(const std::vector & val); - - bool operator == (const PartitionValuesResponse & rhs) const - { - if (!(partitionValues == rhs.partitionValues)) - return false; - return true; - } - bool operator != (const PartitionValuesResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PartitionValuesResponse & ) 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(PartitionValuesResponse &a, PartitionValuesResponse &b); - -std::ostream& operator<<(std::ostream& out, const PartitionValuesResponse& obj); - -typedef struct _ResourceUri__isset { - _ResourceUri__isset() : resourceType(false), uri(false) {} - bool resourceType :1; - bool uri :1; -} _ResourceUri__isset; - -class ResourceUri : public virtual ::apache::thrift::TBase { - public: - - ResourceUri(const ResourceUri&); - ResourceUri& operator=(const ResourceUri&); - ResourceUri() : resourceType((ResourceType::type)0), uri() { - } - - virtual ~ResourceUri() noexcept; - ResourceType::type resourceType; - std::string uri; - - _ResourceUri__isset __isset; - - void __set_resourceType(const ResourceType::type val); - - void __set_uri(const std::string& val); - - bool operator == (const ResourceUri & rhs) const - { - if (!(resourceType == rhs.resourceType)) - return false; - if (!(uri == rhs.uri)) - return false; - return true; - } - bool operator != (const ResourceUri &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ResourceUri & ) 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(ResourceUri &a, ResourceUri &b); - -std::ostream& operator<<(std::ostream& out, const ResourceUri& obj); - -typedef struct _Function__isset { - _Function__isset() : functionName(false), dbName(false), className(false), ownerName(false), ownerType(false), createTime(false), functionType(false), resourceUris(false) {} - bool functionName :1; - bool dbName :1; - bool className :1; - bool ownerName :1; - bool ownerType :1; - bool createTime :1; - bool functionType :1; - bool resourceUris :1; -} _Function__isset; - -class Function : public virtual ::apache::thrift::TBase { - public: - - Function(const Function&); - Function& operator=(const Function&); - Function() : functionName(), dbName(), className(), ownerName(), ownerType((PrincipalType::type)0), createTime(0), functionType((FunctionType::type)0) { - } - - virtual ~Function() noexcept; - std::string functionName; - std::string dbName; - std::string className; - std::string ownerName; - PrincipalType::type ownerType; - int32_t createTime; - FunctionType::type functionType; - std::vector resourceUris; - - _Function__isset __isset; - - void __set_functionName(const std::string& val); - - void __set_dbName(const std::string& val); - - void __set_className(const std::string& val); - - void __set_ownerName(const std::string& val); - - void __set_ownerType(const PrincipalType::type val); - - void __set_createTime(const int32_t val); - - void __set_functionType(const FunctionType::type val); - - void __set_resourceUris(const std::vector & val); - - bool operator == (const Function & rhs) const - { - if (!(functionName == rhs.functionName)) - return false; - if (!(dbName == rhs.dbName)) - return false; - if (!(className == rhs.className)) - return false; - if (!(ownerName == rhs.ownerName)) - return false; - if (!(ownerType == rhs.ownerType)) - return false; - if (!(createTime == rhs.createTime)) - return false; - if (!(functionType == rhs.functionType)) - return false; - if (!(resourceUris == rhs.resourceUris)) - return false; - return true; - } - bool operator != (const Function &rhs) const { - return !(*this == rhs); - } - - bool operator < (const Function & ) 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(Function &a, Function &b); - -std::ostream& operator<<(std::ostream& out, const Function& obj); - -typedef struct _TxnInfo__isset { - _TxnInfo__isset() : agentInfo(true), heartbeatCount(true), metaInfo(false), startedTime(false), lastHeartbeatTime(false) {} - bool agentInfo :1; - bool heartbeatCount :1; - bool metaInfo :1; - bool startedTime :1; - bool lastHeartbeatTime :1; -} _TxnInfo__isset; - -class TxnInfo : public virtual ::apache::thrift::TBase { - public: - - TxnInfo(const TxnInfo&); - TxnInfo& operator=(const TxnInfo&); - TxnInfo() : id(0), state((TxnState::type)0), user(), hostname(), agentInfo("Unknown"), heartbeatCount(0), metaInfo(), startedTime(0), lastHeartbeatTime(0) { - } - - virtual ~TxnInfo() noexcept; - int64_t id; - TxnState::type state; - std::string user; - std::string hostname; - std::string agentInfo; - int32_t heartbeatCount; - std::string metaInfo; - int64_t startedTime; - int64_t lastHeartbeatTime; - - _TxnInfo__isset __isset; - - void __set_id(const int64_t val); - - void __set_state(const TxnState::type val); - - void __set_user(const std::string& val); - - void __set_hostname(const std::string& val); - - void __set_agentInfo(const std::string& val); - - void __set_heartbeatCount(const int32_t val); - - void __set_metaInfo(const std::string& val); - - void __set_startedTime(const int64_t val); - - void __set_lastHeartbeatTime(const int64_t val); - - bool operator == (const TxnInfo & rhs) const - { - if (!(id == rhs.id)) - return false; - if (!(state == rhs.state)) - return false; - if (!(user == rhs.user)) - return false; - if (!(hostname == rhs.hostname)) - return false; - if (__isset.agentInfo != rhs.__isset.agentInfo) - return false; - else if (__isset.agentInfo && !(agentInfo == rhs.agentInfo)) - return false; - if (__isset.heartbeatCount != rhs.__isset.heartbeatCount) - return false; - else if (__isset.heartbeatCount && !(heartbeatCount == rhs.heartbeatCount)) - return false; - if (__isset.metaInfo != rhs.__isset.metaInfo) - return false; - else if (__isset.metaInfo && !(metaInfo == rhs.metaInfo)) - return false; - if (__isset.startedTime != rhs.__isset.startedTime) - return false; - else if (__isset.startedTime && !(startedTime == rhs.startedTime)) - return false; - if (__isset.lastHeartbeatTime != rhs.__isset.lastHeartbeatTime) - return false; - else if (__isset.lastHeartbeatTime && !(lastHeartbeatTime == rhs.lastHeartbeatTime)) - return false; - return true; - } - bool operator != (const TxnInfo &rhs) const { - return !(*this == rhs); - } - - bool operator < (const TxnInfo & ) 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(TxnInfo &a, TxnInfo &b); - -std::ostream& operator<<(std::ostream& out, const TxnInfo& obj); - - -class GetOpenTxnsInfoResponse : public virtual ::apache::thrift::TBase { - public: - - GetOpenTxnsInfoResponse(const GetOpenTxnsInfoResponse&); - GetOpenTxnsInfoResponse& operator=(const GetOpenTxnsInfoResponse&); - GetOpenTxnsInfoResponse() : txn_high_water_mark(0) { - } - - virtual ~GetOpenTxnsInfoResponse() noexcept; - int64_t txn_high_water_mark; - std::vector open_txns; - - void __set_txn_high_water_mark(const int64_t val); - - void __set_open_txns(const std::vector & val); - - bool operator == (const GetOpenTxnsInfoResponse & rhs) const - { - if (!(txn_high_water_mark == rhs.txn_high_water_mark)) - return false; - if (!(open_txns == rhs.open_txns)) - return false; - return true; - } - bool operator != (const GetOpenTxnsInfoResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GetOpenTxnsInfoResponse & ) 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(GetOpenTxnsInfoResponse &a, GetOpenTxnsInfoResponse &b); - -std::ostream& operator<<(std::ostream& out, const GetOpenTxnsInfoResponse& obj); - -typedef struct _GetOpenTxnsResponse__isset { - _GetOpenTxnsResponse__isset() : min_open_txn(false) {} - bool min_open_txn :1; -} _GetOpenTxnsResponse__isset; - -class GetOpenTxnsResponse : public virtual ::apache::thrift::TBase { - public: - - GetOpenTxnsResponse(const GetOpenTxnsResponse&); - GetOpenTxnsResponse& operator=(const GetOpenTxnsResponse&); - GetOpenTxnsResponse() : txn_high_water_mark(0), min_open_txn(0), abortedBits() { - } - - virtual ~GetOpenTxnsResponse() noexcept; - int64_t txn_high_water_mark; - std::vector open_txns; - int64_t min_open_txn; - std::string abortedBits; - - _GetOpenTxnsResponse__isset __isset; - - void __set_txn_high_water_mark(const int64_t val); - - void __set_open_txns(const std::vector & val); - - void __set_min_open_txn(const int64_t val); - - void __set_abortedBits(const std::string& val); - - bool operator == (const GetOpenTxnsResponse & rhs) const - { - if (!(txn_high_water_mark == rhs.txn_high_water_mark)) - return false; - if (!(open_txns == rhs.open_txns)) - return false; - if (__isset.min_open_txn != rhs.__isset.min_open_txn) - return false; - else if (__isset.min_open_txn && !(min_open_txn == rhs.min_open_txn)) - return false; - if (!(abortedBits == rhs.abortedBits)) - return false; - return true; - } - bool operator != (const GetOpenTxnsResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GetOpenTxnsResponse & ) 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(GetOpenTxnsResponse &a, GetOpenTxnsResponse &b); - -std::ostream& operator<<(std::ostream& out, const GetOpenTxnsResponse& obj); - -typedef struct _OpenTxnRequest__isset { - _OpenTxnRequest__isset() : agentInfo(true) {} - bool agentInfo :1; -} _OpenTxnRequest__isset; - -class OpenTxnRequest : public virtual ::apache::thrift::TBase { - public: - - OpenTxnRequest(const OpenTxnRequest&); - OpenTxnRequest& operator=(const OpenTxnRequest&); - OpenTxnRequest() : num_txns(0), user(), hostname(), agentInfo("Unknown") { - } - - virtual ~OpenTxnRequest() noexcept; - int32_t num_txns; - std::string user; - std::string hostname; - std::string agentInfo; - - _OpenTxnRequest__isset __isset; - - void __set_num_txns(const int32_t val); - - void __set_user(const std::string& val); - - void __set_hostname(const std::string& val); - - void __set_agentInfo(const std::string& val); - - bool operator == (const OpenTxnRequest & rhs) const - { - if (!(num_txns == rhs.num_txns)) - return false; - if (!(user == rhs.user)) - return false; - if (!(hostname == rhs.hostname)) - return false; - if (__isset.agentInfo != rhs.__isset.agentInfo) - return false; - else if (__isset.agentInfo && !(agentInfo == rhs.agentInfo)) - return false; - return true; - } - bool operator != (const OpenTxnRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const OpenTxnRequest & ) 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(OpenTxnRequest &a, OpenTxnRequest &b); - -std::ostream& operator<<(std::ostream& out, const OpenTxnRequest& obj); - - -class OpenTxnsResponse : public virtual ::apache::thrift::TBase { - public: - - OpenTxnsResponse(const OpenTxnsResponse&); - OpenTxnsResponse& operator=(const OpenTxnsResponse&); - OpenTxnsResponse() { - } - - virtual ~OpenTxnsResponse() noexcept; - std::vector txn_ids; - - void __set_txn_ids(const std::vector & val); - - bool operator == (const OpenTxnsResponse & rhs) const - { - if (!(txn_ids == rhs.txn_ids)) - return false; - return true; - } - bool operator != (const OpenTxnsResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const OpenTxnsResponse & ) 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(OpenTxnsResponse &a, OpenTxnsResponse &b); - -std::ostream& operator<<(std::ostream& out, const OpenTxnsResponse& obj); - - -class AbortTxnRequest : public virtual ::apache::thrift::TBase { - public: - - AbortTxnRequest(const AbortTxnRequest&); - AbortTxnRequest& operator=(const AbortTxnRequest&); - AbortTxnRequest() : txnid(0) { - } - - virtual ~AbortTxnRequest() noexcept; - int64_t txnid; - - void __set_txnid(const int64_t val); - - bool operator == (const AbortTxnRequest & rhs) const - { - if (!(txnid == rhs.txnid)) - return false; - return true; - } - bool operator != (const AbortTxnRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const AbortTxnRequest & ) 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(AbortTxnRequest &a, AbortTxnRequest &b); - -std::ostream& operator<<(std::ostream& out, const AbortTxnRequest& obj); - - -class AbortTxnsRequest : public virtual ::apache::thrift::TBase { - public: - - AbortTxnsRequest(const AbortTxnsRequest&); - AbortTxnsRequest& operator=(const AbortTxnsRequest&); - AbortTxnsRequest() { - } - - virtual ~AbortTxnsRequest() noexcept; - std::vector txn_ids; - - void __set_txn_ids(const std::vector & val); - - bool operator == (const AbortTxnsRequest & rhs) const - { - if (!(txn_ids == rhs.txn_ids)) - return false; - return true; - } - bool operator != (const AbortTxnsRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const AbortTxnsRequest & ) 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(AbortTxnsRequest &a, AbortTxnsRequest &b); - -std::ostream& operator<<(std::ostream& out, const AbortTxnsRequest& obj); - - -class CommitTxnRequest : public virtual ::apache::thrift::TBase { - public: - - CommitTxnRequest(const CommitTxnRequest&); - CommitTxnRequest& operator=(const CommitTxnRequest&); - CommitTxnRequest() : txnid(0) { - } - - virtual ~CommitTxnRequest() noexcept; - int64_t txnid; - - void __set_txnid(const int64_t val); - - bool operator == (const CommitTxnRequest & rhs) const - { - if (!(txnid == rhs.txnid)) - return false; - return true; - } - bool operator != (const CommitTxnRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const CommitTxnRequest & ) 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(CommitTxnRequest &a, CommitTxnRequest &b); - -std::ostream& operator<<(std::ostream& out, const CommitTxnRequest& obj); - -typedef struct _LockComponent__isset { - _LockComponent__isset() : tablename(false), partitionname(false), operationType(true), isAcid(true), isDynamicPartitionWrite(true) {} - bool tablename :1; - bool partitionname :1; - bool operationType :1; - bool isAcid :1; - bool isDynamicPartitionWrite :1; -} _LockComponent__isset; - -class LockComponent : public virtual ::apache::thrift::TBase { - public: - - LockComponent(const LockComponent&); - LockComponent& operator=(const LockComponent&); - LockComponent() : type((LockType::type)0), level((LockLevel::type)0), dbname(), tablename(), partitionname(), operationType((DataOperationType::type)5), isAcid(false), isDynamicPartitionWrite(false) { - operationType = (DataOperationType::type)5; - - } - - virtual ~LockComponent() noexcept; - LockType::type type; - LockLevel::type level; - std::string dbname; - std::string tablename; - std::string partitionname; - DataOperationType::type operationType; - bool isAcid; - bool isDynamicPartitionWrite; - - _LockComponent__isset __isset; - - void __set_type(const LockType::type val); - - void __set_level(const LockLevel::type val); - - void __set_dbname(const std::string& val); - - void __set_tablename(const std::string& val); - - void __set_partitionname(const std::string& val); - - void __set_operationType(const DataOperationType::type val); - - void __set_isAcid(const bool val); - - void __set_isDynamicPartitionWrite(const bool val); - - bool operator == (const LockComponent & rhs) const - { - if (!(type == rhs.type)) - return false; - if (!(level == rhs.level)) - return false; - if (!(dbname == rhs.dbname)) - return false; - if (__isset.tablename != rhs.__isset.tablename) - return false; - else if (__isset.tablename && !(tablename == rhs.tablename)) - return false; - if (__isset.partitionname != rhs.__isset.partitionname) - return false; - else if (__isset.partitionname && !(partitionname == rhs.partitionname)) - return false; - if (__isset.operationType != rhs.__isset.operationType) - return false; - else if (__isset.operationType && !(operationType == rhs.operationType)) - return false; - if (__isset.isAcid != rhs.__isset.isAcid) - return false; - else if (__isset.isAcid && !(isAcid == rhs.isAcid)) - return false; - if (__isset.isDynamicPartitionWrite != rhs.__isset.isDynamicPartitionWrite) - return false; - else if (__isset.isDynamicPartitionWrite && !(isDynamicPartitionWrite == rhs.isDynamicPartitionWrite)) - return false; - return true; - } - bool operator != (const LockComponent &rhs) const { - return !(*this == rhs); - } - - bool operator < (const LockComponent & ) 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(LockComponent &a, LockComponent &b); - -std::ostream& operator<<(std::ostream& out, const LockComponent& obj); - -typedef struct _LockRequest__isset { - _LockRequest__isset() : txnid(false), agentInfo(true) {} - bool txnid :1; - bool agentInfo :1; -} _LockRequest__isset; - -class LockRequest : public virtual ::apache::thrift::TBase { - public: - - LockRequest(const LockRequest&); - LockRequest& operator=(const LockRequest&); - LockRequest() : txnid(0), user(), hostname(), agentInfo("Unknown") { - } - - virtual ~LockRequest() noexcept; - std::vector component; - int64_t txnid; - std::string user; - std::string hostname; - std::string agentInfo; - - _LockRequest__isset __isset; - - void __set_component(const std::vector & val); - - void __set_txnid(const int64_t val); - - void __set_user(const std::string& val); - - void __set_hostname(const std::string& val); - - void __set_agentInfo(const std::string& val); - - bool operator == (const LockRequest & rhs) const - { - if (!(component == rhs.component)) - return false; - if (__isset.txnid != rhs.__isset.txnid) - return false; - else if (__isset.txnid && !(txnid == rhs.txnid)) - return false; - if (!(user == rhs.user)) - return false; - if (!(hostname == rhs.hostname)) - return false; - if (__isset.agentInfo != rhs.__isset.agentInfo) - return false; - else if (__isset.agentInfo && !(agentInfo == rhs.agentInfo)) - return false; - return true; - } - bool operator != (const LockRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const LockRequest & ) 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(LockRequest &a, LockRequest &b); - -std::ostream& operator<<(std::ostream& out, const LockRequest& obj); - - -class LockResponse : public virtual ::apache::thrift::TBase { - public: - - LockResponse(const LockResponse&); - LockResponse& operator=(const LockResponse&); - LockResponse() : lockid(0), state((LockState::type)0) { - } - - virtual ~LockResponse() noexcept; - int64_t lockid; - LockState::type state; - - void __set_lockid(const int64_t val); - - void __set_state(const LockState::type val); - - bool operator == (const LockResponse & rhs) const - { - if (!(lockid == rhs.lockid)) - return false; - if (!(state == rhs.state)) - return false; - return true; - } - bool operator != (const LockResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const LockResponse & ) 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(LockResponse &a, LockResponse &b); - -std::ostream& operator<<(std::ostream& out, const LockResponse& obj); - -typedef struct _CheckLockRequest__isset { - _CheckLockRequest__isset() : txnid(false), elapsed_ms(false) {} - bool txnid :1; - bool elapsed_ms :1; -} _CheckLockRequest__isset; - -class CheckLockRequest : public virtual ::apache::thrift::TBase { - public: - - CheckLockRequest(const CheckLockRequest&); - CheckLockRequest& operator=(const CheckLockRequest&); - CheckLockRequest() : lockid(0), txnid(0), elapsed_ms(0) { - } - - virtual ~CheckLockRequest() noexcept; - int64_t lockid; - int64_t txnid; - int64_t elapsed_ms; - - _CheckLockRequest__isset __isset; - - void __set_lockid(const int64_t val); - - void __set_txnid(const int64_t val); - - void __set_elapsed_ms(const int64_t val); - - bool operator == (const CheckLockRequest & rhs) const - { - if (!(lockid == rhs.lockid)) - return false; - if (__isset.txnid != rhs.__isset.txnid) - return false; - else if (__isset.txnid && !(txnid == rhs.txnid)) - return false; - if (__isset.elapsed_ms != rhs.__isset.elapsed_ms) - return false; - else if (__isset.elapsed_ms && !(elapsed_ms == rhs.elapsed_ms)) - return false; - return true; - } - bool operator != (const CheckLockRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const CheckLockRequest & ) 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(CheckLockRequest &a, CheckLockRequest &b); - -std::ostream& operator<<(std::ostream& out, const CheckLockRequest& obj); - - -class UnlockRequest : public virtual ::apache::thrift::TBase { - public: - - UnlockRequest(const UnlockRequest&); - UnlockRequest& operator=(const UnlockRequest&); - UnlockRequest() : lockid(0) { - } - - virtual ~UnlockRequest() noexcept; - int64_t lockid; - - void __set_lockid(const int64_t val); - - bool operator == (const UnlockRequest & rhs) const - { - if (!(lockid == rhs.lockid)) - return false; - return true; - } - bool operator != (const UnlockRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const UnlockRequest & ) 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(UnlockRequest &a, UnlockRequest &b); - -std::ostream& operator<<(std::ostream& out, const UnlockRequest& obj); - -typedef struct _ShowLocksRequest__isset { - _ShowLocksRequest__isset() : dbname(false), tablename(false), partname(false), isExtended(true) {} - bool dbname :1; - bool tablename :1; - bool partname :1; - bool isExtended :1; -} _ShowLocksRequest__isset; - -class ShowLocksRequest : public virtual ::apache::thrift::TBase { - public: - - ShowLocksRequest(const ShowLocksRequest&); - ShowLocksRequest& operator=(const ShowLocksRequest&); - ShowLocksRequest() : dbname(), tablename(), partname(), isExtended(false) { - } - - virtual ~ShowLocksRequest() noexcept; - std::string dbname; - std::string tablename; - std::string partname; - bool isExtended; - - _ShowLocksRequest__isset __isset; - - void __set_dbname(const std::string& val); - - void __set_tablename(const std::string& val); - - void __set_partname(const std::string& val); - - void __set_isExtended(const bool val); - - bool operator == (const ShowLocksRequest & rhs) const - { - if (__isset.dbname != rhs.__isset.dbname) - return false; - else if (__isset.dbname && !(dbname == rhs.dbname)) - return false; - if (__isset.tablename != rhs.__isset.tablename) - return false; - else if (__isset.tablename && !(tablename == rhs.tablename)) - return false; - if (__isset.partname != rhs.__isset.partname) - return false; - else if (__isset.partname && !(partname == rhs.partname)) - return false; - if (__isset.isExtended != rhs.__isset.isExtended) - return false; - else if (__isset.isExtended && !(isExtended == rhs.isExtended)) - return false; - return true; - } - bool operator != (const ShowLocksRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ShowLocksRequest & ) 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(ShowLocksRequest &a, ShowLocksRequest &b); - -std::ostream& operator<<(std::ostream& out, const ShowLocksRequest& obj); - -typedef struct _ShowLocksResponseElement__isset { - _ShowLocksResponseElement__isset() : tablename(false), partname(false), txnid(false), acquiredat(false), heartbeatCount(true), agentInfo(false), blockedByExtId(false), blockedByIntId(false), lockIdInternal(false) {} - bool tablename :1; - bool partname :1; - bool txnid :1; - bool acquiredat :1; - bool heartbeatCount :1; - bool agentInfo :1; - bool blockedByExtId :1; - bool blockedByIntId :1; - bool lockIdInternal :1; -} _ShowLocksResponseElement__isset; - -class ShowLocksResponseElement : public virtual ::apache::thrift::TBase { - public: - - ShowLocksResponseElement(const ShowLocksResponseElement&); - ShowLocksResponseElement& operator=(const ShowLocksResponseElement&); - ShowLocksResponseElement() : lockid(0), dbname(), tablename(), partname(), state((LockState::type)0), type((LockType::type)0), txnid(0), lastheartbeat(0), acquiredat(0), user(), hostname(), heartbeatCount(0), agentInfo(), blockedByExtId(0), blockedByIntId(0), lockIdInternal(0) { - } - - virtual ~ShowLocksResponseElement() noexcept; - int64_t lockid; - std::string dbname; - std::string tablename; - std::string partname; - LockState::type state; - LockType::type type; - int64_t txnid; - int64_t lastheartbeat; - int64_t acquiredat; - std::string user; - std::string hostname; - int32_t heartbeatCount; - std::string agentInfo; - int64_t blockedByExtId; - int64_t blockedByIntId; - int64_t lockIdInternal; - - _ShowLocksResponseElement__isset __isset; - - void __set_lockid(const int64_t val); - - void __set_dbname(const std::string& val); - - void __set_tablename(const std::string& val); - - void __set_partname(const std::string& val); - - void __set_state(const LockState::type val); - - void __set_type(const LockType::type val); - - void __set_txnid(const int64_t val); - - void __set_lastheartbeat(const int64_t val); - - void __set_acquiredat(const int64_t val); - - void __set_user(const std::string& val); - - void __set_hostname(const std::string& val); - - void __set_heartbeatCount(const int32_t val); - - void __set_agentInfo(const std::string& val); - - void __set_blockedByExtId(const int64_t val); - - void __set_blockedByIntId(const int64_t val); - - void __set_lockIdInternal(const int64_t val); - - bool operator == (const ShowLocksResponseElement & rhs) const - { - if (!(lockid == rhs.lockid)) - return false; - if (!(dbname == rhs.dbname)) - return false; - if (__isset.tablename != rhs.__isset.tablename) - return false; - else if (__isset.tablename && !(tablename == rhs.tablename)) - return false; - if (__isset.partname != rhs.__isset.partname) - return false; - else if (__isset.partname && !(partname == rhs.partname)) - return false; - if (!(state == rhs.state)) - return false; - if (!(type == rhs.type)) - return false; - if (__isset.txnid != rhs.__isset.txnid) - return false; - else if (__isset.txnid && !(txnid == rhs.txnid)) - return false; - if (!(lastheartbeat == rhs.lastheartbeat)) - return false; - if (__isset.acquiredat != rhs.__isset.acquiredat) - return false; - else if (__isset.acquiredat && !(acquiredat == rhs.acquiredat)) - return false; - if (!(user == rhs.user)) - return false; - if (!(hostname == rhs.hostname)) - return false; - if (__isset.heartbeatCount != rhs.__isset.heartbeatCount) - return false; - else if (__isset.heartbeatCount && !(heartbeatCount == rhs.heartbeatCount)) - return false; - if (__isset.agentInfo != rhs.__isset.agentInfo) - return false; - else if (__isset.agentInfo && !(agentInfo == rhs.agentInfo)) - return false; - if (__isset.blockedByExtId != rhs.__isset.blockedByExtId) - return false; - else if (__isset.blockedByExtId && !(blockedByExtId == rhs.blockedByExtId)) - return false; - if (__isset.blockedByIntId != rhs.__isset.blockedByIntId) - return false; - else if (__isset.blockedByIntId && !(blockedByIntId == rhs.blockedByIntId)) - return false; - if (__isset.lockIdInternal != rhs.__isset.lockIdInternal) - return false; - else if (__isset.lockIdInternal && !(lockIdInternal == rhs.lockIdInternal)) - return false; - return true; - } - bool operator != (const ShowLocksResponseElement &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ShowLocksResponseElement & ) 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(ShowLocksResponseElement &a, ShowLocksResponseElement &b); - -std::ostream& operator<<(std::ostream& out, const ShowLocksResponseElement& obj); - -typedef struct _ShowLocksResponse__isset { - _ShowLocksResponse__isset() : locks(false) {} - bool locks :1; -} _ShowLocksResponse__isset; - -class ShowLocksResponse : public virtual ::apache::thrift::TBase { - public: - - ShowLocksResponse(const ShowLocksResponse&); - ShowLocksResponse& operator=(const ShowLocksResponse&); - ShowLocksResponse() { - } - - virtual ~ShowLocksResponse() noexcept; - std::vector locks; - - _ShowLocksResponse__isset __isset; - - void __set_locks(const std::vector & val); - - bool operator == (const ShowLocksResponse & rhs) const - { - if (!(locks == rhs.locks)) - return false; - return true; - } - bool operator != (const ShowLocksResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ShowLocksResponse & ) 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(ShowLocksResponse &a, ShowLocksResponse &b); - -std::ostream& operator<<(std::ostream& out, const ShowLocksResponse& obj); - -typedef struct _HeartbeatRequest__isset { - _HeartbeatRequest__isset() : lockid(false), txnid(false) {} - bool lockid :1; - bool txnid :1; -} _HeartbeatRequest__isset; - -class HeartbeatRequest : public virtual ::apache::thrift::TBase { - public: - - HeartbeatRequest(const HeartbeatRequest&); - HeartbeatRequest& operator=(const HeartbeatRequest&); - HeartbeatRequest() : lockid(0), txnid(0) { - } - - virtual ~HeartbeatRequest() noexcept; - int64_t lockid; - int64_t txnid; - - _HeartbeatRequest__isset __isset; - - void __set_lockid(const int64_t val); - - void __set_txnid(const int64_t val); - - bool operator == (const HeartbeatRequest & rhs) const - { - if (__isset.lockid != rhs.__isset.lockid) - return false; - else if (__isset.lockid && !(lockid == rhs.lockid)) - return false; - if (__isset.txnid != rhs.__isset.txnid) - return false; - else if (__isset.txnid && !(txnid == rhs.txnid)) - return false; - return true; - } - bool operator != (const HeartbeatRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const HeartbeatRequest & ) 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(HeartbeatRequest &a, HeartbeatRequest &b); - -std::ostream& operator<<(std::ostream& out, const HeartbeatRequest& obj); - - -class HeartbeatTxnRangeRequest : public virtual ::apache::thrift::TBase { - public: - - HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest&); - HeartbeatTxnRangeRequest& operator=(const HeartbeatTxnRangeRequest&); - HeartbeatTxnRangeRequest() : min(0), max(0) { - } - - virtual ~HeartbeatTxnRangeRequest() noexcept; - int64_t min; - int64_t max; - - void __set_min(const int64_t val); - - void __set_max(const int64_t val); - - bool operator == (const HeartbeatTxnRangeRequest & rhs) const - { - if (!(min == rhs.min)) - return false; - if (!(max == rhs.max)) - return false; - return true; - } - bool operator != (const HeartbeatTxnRangeRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const HeartbeatTxnRangeRequest & ) 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(HeartbeatTxnRangeRequest &a, HeartbeatTxnRangeRequest &b); - -std::ostream& operator<<(std::ostream& out, const HeartbeatTxnRangeRequest& obj); - - -class HeartbeatTxnRangeResponse : public virtual ::apache::thrift::TBase { - public: - - HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse&); - HeartbeatTxnRangeResponse& operator=(const HeartbeatTxnRangeResponse&); - HeartbeatTxnRangeResponse() { - } - - virtual ~HeartbeatTxnRangeResponse() noexcept; - std::set aborted; - std::set nosuch; - - void __set_aborted(const std::set & val); - - void __set_nosuch(const std::set & val); - - bool operator == (const HeartbeatTxnRangeResponse & rhs) const - { - if (!(aborted == rhs.aborted)) - return false; - if (!(nosuch == rhs.nosuch)) - return false; - return true; - } - bool operator != (const HeartbeatTxnRangeResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const HeartbeatTxnRangeResponse & ) 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(HeartbeatTxnRangeResponse &a, HeartbeatTxnRangeResponse &b); - -std::ostream& operator<<(std::ostream& out, const HeartbeatTxnRangeResponse& obj); - -typedef struct _CompactionRequest__isset { - _CompactionRequest__isset() : partitionname(false), runas(false), properties(false) {} - bool partitionname :1; - bool runas :1; - bool properties :1; -} _CompactionRequest__isset; - -class CompactionRequest : public virtual ::apache::thrift::TBase { - public: - - CompactionRequest(const CompactionRequest&); - CompactionRequest& operator=(const CompactionRequest&); - CompactionRequest() : dbname(), tablename(), partitionname(), type((CompactionType::type)0), runas() { - } - - virtual ~CompactionRequest() noexcept; - std::string dbname; - std::string tablename; - std::string partitionname; - CompactionType::type type; - std::string runas; - std::map properties; - - _CompactionRequest__isset __isset; - - void __set_dbname(const std::string& val); - - void __set_tablename(const std::string& val); - - void __set_partitionname(const std::string& val); - - void __set_type(const CompactionType::type val); - - void __set_runas(const std::string& val); - - void __set_properties(const std::map & val); - - bool operator == (const CompactionRequest & rhs) const - { - if (!(dbname == rhs.dbname)) - return false; - if (!(tablename == rhs.tablename)) - return false; - if (__isset.partitionname != rhs.__isset.partitionname) - return false; - else if (__isset.partitionname && !(partitionname == rhs.partitionname)) - return false; - if (!(type == rhs.type)) - return false; - if (__isset.runas != rhs.__isset.runas) - return false; - else if (__isset.runas && !(runas == rhs.runas)) - return false; - if (__isset.properties != rhs.__isset.properties) - return false; - else if (__isset.properties && !(properties == rhs.properties)) - return false; - return true; - } - bool operator != (const CompactionRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const CompactionRequest & ) 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(CompactionRequest &a, CompactionRequest &b); - -std::ostream& operator<<(std::ostream& out, const CompactionRequest& obj); - - -class CompactionResponse : public virtual ::apache::thrift::TBase { - public: - - CompactionResponse(const CompactionResponse&); - CompactionResponse& operator=(const CompactionResponse&); - CompactionResponse() : id(0), state(), accepted(0) { - } - - virtual ~CompactionResponse() noexcept; - int64_t id; - std::string state; - bool accepted; - - void __set_id(const int64_t val); - - void __set_state(const std::string& val); - - void __set_accepted(const bool val); - - bool operator == (const CompactionResponse & rhs) const - { - if (!(id == rhs.id)) - return false; - if (!(state == rhs.state)) - return false; - if (!(accepted == rhs.accepted)) - return false; - return true; - } - bool operator != (const CompactionResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const CompactionResponse & ) 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(CompactionResponse &a, CompactionResponse &b); - -std::ostream& operator<<(std::ostream& out, const CompactionResponse& obj); - - -class ShowCompactRequest : public virtual ::apache::thrift::TBase { - public: - - ShowCompactRequest(const ShowCompactRequest&); - ShowCompactRequest& operator=(const ShowCompactRequest&); - ShowCompactRequest() { - } - - virtual ~ShowCompactRequest() noexcept; - - bool operator == (const ShowCompactRequest & /* rhs */) const - { - return true; - } - bool operator != (const ShowCompactRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ShowCompactRequest & ) 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(ShowCompactRequest &a, ShowCompactRequest &b); - -std::ostream& operator<<(std::ostream& out, const ShowCompactRequest& obj); - -typedef struct _ShowCompactResponseElement__isset { - _ShowCompactResponseElement__isset() : partitionname(false), workerid(false), start(false), runAs(false), hightestTxnId(false), metaInfo(false), endTime(false), hadoopJobId(true), id(false) {} - bool partitionname :1; - bool workerid :1; - bool start :1; - bool runAs :1; - bool hightestTxnId :1; - bool metaInfo :1; - bool endTime :1; - bool hadoopJobId :1; - bool id :1; -} _ShowCompactResponseElement__isset; - -class ShowCompactResponseElement : public virtual ::apache::thrift::TBase { - public: - - ShowCompactResponseElement(const ShowCompactResponseElement&); - ShowCompactResponseElement& operator=(const ShowCompactResponseElement&); - ShowCompactResponseElement() : dbname(), tablename(), partitionname(), type((CompactionType::type)0), state(), workerid(), start(0), runAs(), hightestTxnId(0), metaInfo(), endTime(0), hadoopJobId("None"), id(0) { - } - - virtual ~ShowCompactResponseElement() noexcept; - std::string dbname; - std::string tablename; - std::string partitionname; - CompactionType::type type; - std::string state; - std::string workerid; - int64_t start; - std::string runAs; - int64_t hightestTxnId; - std::string metaInfo; - int64_t endTime; - std::string hadoopJobId; - int64_t id; - - _ShowCompactResponseElement__isset __isset; - - void __set_dbname(const std::string& val); - - void __set_tablename(const std::string& val); - - void __set_partitionname(const std::string& val); - - void __set_type(const CompactionType::type val); - - void __set_state(const std::string& val); - - void __set_workerid(const std::string& val); - - void __set_start(const int64_t val); - - void __set_runAs(const std::string& val); - - void __set_hightestTxnId(const int64_t val); - - void __set_metaInfo(const std::string& val); - - void __set_endTime(const int64_t val); - - void __set_hadoopJobId(const std::string& val); - - void __set_id(const int64_t val); - - bool operator == (const ShowCompactResponseElement & rhs) const - { - if (!(dbname == rhs.dbname)) - return false; - if (!(tablename == rhs.tablename)) - return false; - if (__isset.partitionname != rhs.__isset.partitionname) - return false; - else if (__isset.partitionname && !(partitionname == rhs.partitionname)) - return false; - if (!(type == rhs.type)) - return false; - if (!(state == rhs.state)) - return false; - if (__isset.workerid != rhs.__isset.workerid) - return false; - else if (__isset.workerid && !(workerid == rhs.workerid)) - return false; - if (__isset.start != rhs.__isset.start) - return false; - else if (__isset.start && !(start == rhs.start)) - return false; - if (__isset.runAs != rhs.__isset.runAs) - return false; - else if (__isset.runAs && !(runAs == rhs.runAs)) - return false; - if (__isset.hightestTxnId != rhs.__isset.hightestTxnId) - return false; - else if (__isset.hightestTxnId && !(hightestTxnId == rhs.hightestTxnId)) - return false; - if (__isset.metaInfo != rhs.__isset.metaInfo) - return false; - else if (__isset.metaInfo && !(metaInfo == rhs.metaInfo)) - return false; - if (__isset.endTime != rhs.__isset.endTime) - return false; - else if (__isset.endTime && !(endTime == rhs.endTime)) - return false; - if (__isset.hadoopJobId != rhs.__isset.hadoopJobId) - return false; - else if (__isset.hadoopJobId && !(hadoopJobId == rhs.hadoopJobId)) - return false; - if (__isset.id != rhs.__isset.id) - return false; - else if (__isset.id && !(id == rhs.id)) - return false; - return true; - } - bool operator != (const ShowCompactResponseElement &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ShowCompactResponseElement & ) 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(ShowCompactResponseElement &a, ShowCompactResponseElement &b); - -std::ostream& operator<<(std::ostream& out, const ShowCompactResponseElement& obj); - - -class ShowCompactResponse : public virtual ::apache::thrift::TBase { - public: - - ShowCompactResponse(const ShowCompactResponse&); - ShowCompactResponse& operator=(const ShowCompactResponse&); - ShowCompactResponse() { - } - - virtual ~ShowCompactResponse() noexcept; - std::vector compacts; - - void __set_compacts(const std::vector & val); - - bool operator == (const ShowCompactResponse & rhs) const - { - if (!(compacts == rhs.compacts)) - return false; - return true; - } - bool operator != (const ShowCompactResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ShowCompactResponse & ) 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(ShowCompactResponse &a, ShowCompactResponse &b); - -std::ostream& operator<<(std::ostream& out, const ShowCompactResponse& obj); - -typedef struct _AddDynamicPartitions__isset { - _AddDynamicPartitions__isset() : operationType(true) {} - bool operationType :1; -} _AddDynamicPartitions__isset; - -class AddDynamicPartitions : public virtual ::apache::thrift::TBase { - public: - - AddDynamicPartitions(const AddDynamicPartitions&); - AddDynamicPartitions& operator=(const AddDynamicPartitions&); - AddDynamicPartitions() : txnid(0), dbname(), tablename(), operationType((DataOperationType::type)5) { - operationType = (DataOperationType::type)5; - - } - - virtual ~AddDynamicPartitions() noexcept; - int64_t txnid; - std::string dbname; - std::string tablename; - std::vector partitionnames; - DataOperationType::type operationType; - - _AddDynamicPartitions__isset __isset; - - void __set_txnid(const int64_t val); - - void __set_dbname(const std::string& val); - - void __set_tablename(const std::string& val); - - void __set_partitionnames(const std::vector & val); - - void __set_operationType(const DataOperationType::type val); - - bool operator == (const AddDynamicPartitions & rhs) const - { - if (!(txnid == rhs.txnid)) - return false; - if (!(dbname == rhs.dbname)) - return false; - if (!(tablename == rhs.tablename)) - return false; - if (!(partitionnames == rhs.partitionnames)) - return false; - if (__isset.operationType != rhs.__isset.operationType) - return false; - else if (__isset.operationType && !(operationType == rhs.operationType)) - return false; - return true; - } - bool operator != (const AddDynamicPartitions &rhs) const { - return !(*this == rhs); - } - - bool operator < (const AddDynamicPartitions & ) 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(AddDynamicPartitions &a, AddDynamicPartitions &b); - -std::ostream& operator<<(std::ostream& out, const AddDynamicPartitions& obj); - -typedef struct _NotificationEventRequest__isset { - _NotificationEventRequest__isset() : maxEvents(false) {} - bool maxEvents :1; -} _NotificationEventRequest__isset; - -class NotificationEventRequest : public virtual ::apache::thrift::TBase { - public: - - NotificationEventRequest(const NotificationEventRequest&); - NotificationEventRequest& operator=(const NotificationEventRequest&); - NotificationEventRequest() : lastEvent(0), maxEvents(0) { - } - - virtual ~NotificationEventRequest() noexcept; - int64_t lastEvent; - int32_t maxEvents; - - _NotificationEventRequest__isset __isset; - - void __set_lastEvent(const int64_t val); - - void __set_maxEvents(const int32_t val); - - bool operator == (const NotificationEventRequest & rhs) const - { - if (!(lastEvent == rhs.lastEvent)) - return false; - if (__isset.maxEvents != rhs.__isset.maxEvents) - return false; - else if (__isset.maxEvents && !(maxEvents == rhs.maxEvents)) - return false; - return true; - } - bool operator != (const NotificationEventRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const NotificationEventRequest & ) 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(NotificationEventRequest &a, NotificationEventRequest &b); - -std::ostream& operator<<(std::ostream& out, const NotificationEventRequest& obj); - -typedef struct _NotificationEvent__isset { - _NotificationEvent__isset() : dbName(false), tableName(false), messageFormat(false) {} - bool dbName :1; - bool tableName :1; - bool messageFormat :1; -} _NotificationEvent__isset; - -class NotificationEvent : public virtual ::apache::thrift::TBase { - public: - - NotificationEvent(const NotificationEvent&); - NotificationEvent& operator=(const NotificationEvent&); - NotificationEvent() : eventId(0), eventTime(0), eventType(), dbName(), tableName(), message(), messageFormat() { - } - - virtual ~NotificationEvent() noexcept; - int64_t eventId; - int32_t eventTime; - std::string eventType; - std::string dbName; - std::string tableName; - std::string message; - std::string messageFormat; - - _NotificationEvent__isset __isset; - - void __set_eventId(const int64_t val); - - void __set_eventTime(const int32_t val); - - void __set_eventType(const std::string& val); - - void __set_dbName(const std::string& val); - - void __set_tableName(const std::string& val); - - void __set_message(const std::string& val); - - void __set_messageFormat(const std::string& val); - - bool operator == (const NotificationEvent & rhs) const - { - if (!(eventId == rhs.eventId)) - return false; - if (!(eventTime == rhs.eventTime)) - return false; - if (!(eventType == rhs.eventType)) - return false; - if (__isset.dbName != rhs.__isset.dbName) - return false; - else if (__isset.dbName && !(dbName == rhs.dbName)) - return false; - if (__isset.tableName != rhs.__isset.tableName) - return false; - else if (__isset.tableName && !(tableName == rhs.tableName)) - return false; - if (!(message == rhs.message)) - return false; - if (__isset.messageFormat != rhs.__isset.messageFormat) - return false; - else if (__isset.messageFormat && !(messageFormat == rhs.messageFormat)) - return false; - return true; - } - bool operator != (const NotificationEvent &rhs) const { - return !(*this == rhs); - } - - bool operator < (const NotificationEvent & ) 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(NotificationEvent &a, NotificationEvent &b); - -std::ostream& operator<<(std::ostream& out, const NotificationEvent& obj); - - -class NotificationEventResponse : public virtual ::apache::thrift::TBase { - public: - - NotificationEventResponse(const NotificationEventResponse&); - NotificationEventResponse& operator=(const NotificationEventResponse&); - NotificationEventResponse() { - } - - virtual ~NotificationEventResponse() noexcept; - std::vector events; - - void __set_events(const std::vector & val); - - bool operator == (const NotificationEventResponse & rhs) const - { - if (!(events == rhs.events)) - return false; - return true; - } - bool operator != (const NotificationEventResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const NotificationEventResponse & ) 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(NotificationEventResponse &a, NotificationEventResponse &b); - -std::ostream& operator<<(std::ostream& out, const NotificationEventResponse& obj); - - -class CurrentNotificationEventId : public virtual ::apache::thrift::TBase { - public: - - CurrentNotificationEventId(const CurrentNotificationEventId&); - CurrentNotificationEventId& operator=(const CurrentNotificationEventId&); - CurrentNotificationEventId() : eventId(0) { - } - - virtual ~CurrentNotificationEventId() noexcept; - int64_t eventId; - - void __set_eventId(const int64_t val); - - bool operator == (const CurrentNotificationEventId & rhs) const - { - if (!(eventId == rhs.eventId)) - return false; - return true; - } - bool operator != (const CurrentNotificationEventId &rhs) const { - return !(*this == rhs); - } - - bool operator < (const CurrentNotificationEventId & ) 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(CurrentNotificationEventId &a, CurrentNotificationEventId &b); - -std::ostream& operator<<(std::ostream& out, const CurrentNotificationEventId& obj); - - -class NotificationEventsCountRequest : public virtual ::apache::thrift::TBase { - public: - - NotificationEventsCountRequest(const NotificationEventsCountRequest&); - NotificationEventsCountRequest& operator=(const NotificationEventsCountRequest&); - NotificationEventsCountRequest() : fromEventId(0), dbName() { - } - - virtual ~NotificationEventsCountRequest() noexcept; - int64_t fromEventId; - std::string dbName; - - void __set_fromEventId(const int64_t val); - - void __set_dbName(const std::string& val); - - bool operator == (const NotificationEventsCountRequest & rhs) const - { - if (!(fromEventId == rhs.fromEventId)) - return false; - if (!(dbName == rhs.dbName)) - return false; - return true; - } - bool operator != (const NotificationEventsCountRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const NotificationEventsCountRequest & ) 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(NotificationEventsCountRequest &a, NotificationEventsCountRequest &b); - -std::ostream& operator<<(std::ostream& out, const NotificationEventsCountRequest& obj); - - -class NotificationEventsCountResponse : public virtual ::apache::thrift::TBase { - public: - - NotificationEventsCountResponse(const NotificationEventsCountResponse&); - NotificationEventsCountResponse& operator=(const NotificationEventsCountResponse&); - NotificationEventsCountResponse() : eventsCount(0) { - } - - virtual ~NotificationEventsCountResponse() noexcept; - int64_t eventsCount; - - void __set_eventsCount(const int64_t val); - - bool operator == (const NotificationEventsCountResponse & rhs) const - { - if (!(eventsCount == rhs.eventsCount)) - return false; - return true; - } - bool operator != (const NotificationEventsCountResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const NotificationEventsCountResponse & ) 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(NotificationEventsCountResponse &a, NotificationEventsCountResponse &b); - -std::ostream& operator<<(std::ostream& out, const NotificationEventsCountResponse& obj); - -typedef struct _InsertEventRequestData__isset { - _InsertEventRequestData__isset() : replace(false), filesAddedChecksum(false) {} - bool replace :1; - bool filesAddedChecksum :1; -} _InsertEventRequestData__isset; - -class InsertEventRequestData : public virtual ::apache::thrift::TBase { - public: - - InsertEventRequestData(const InsertEventRequestData&); - InsertEventRequestData& operator=(const InsertEventRequestData&); - InsertEventRequestData() : replace(0) { - } - - virtual ~InsertEventRequestData() noexcept; - bool replace; - std::vector filesAdded; - std::vector filesAddedChecksum; - - _InsertEventRequestData__isset __isset; - - void __set_replace(const bool val); - - void __set_filesAdded(const std::vector & val); - - void __set_filesAddedChecksum(const std::vector & val); - - bool operator == (const InsertEventRequestData & rhs) const - { - if (__isset.replace != rhs.__isset.replace) - return false; - else if (__isset.replace && !(replace == rhs.replace)) - return false; - if (!(filesAdded == rhs.filesAdded)) - return false; - if (__isset.filesAddedChecksum != rhs.__isset.filesAddedChecksum) - return false; - else if (__isset.filesAddedChecksum && !(filesAddedChecksum == rhs.filesAddedChecksum)) - return false; - return true; - } - bool operator != (const InsertEventRequestData &rhs) const { - return !(*this == rhs); - } - - bool operator < (const InsertEventRequestData & ) 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(InsertEventRequestData &a, InsertEventRequestData &b); - -std::ostream& operator<<(std::ostream& out, const InsertEventRequestData& obj); - -typedef struct _FireEventRequestData__isset { - _FireEventRequestData__isset() : insertData(false) {} - bool insertData :1; -} _FireEventRequestData__isset; - -class FireEventRequestData : public virtual ::apache::thrift::TBase { - public: - - FireEventRequestData(const FireEventRequestData&); - FireEventRequestData& operator=(const FireEventRequestData&); - FireEventRequestData() { - } - - virtual ~FireEventRequestData() noexcept; - InsertEventRequestData insertData; - - _FireEventRequestData__isset __isset; - - void __set_insertData(const InsertEventRequestData& val); - - bool operator == (const FireEventRequestData & rhs) const - { - if (__isset.insertData != rhs.__isset.insertData) - return false; - else if (__isset.insertData && !(insertData == rhs.insertData)) - return false; - return true; - } - bool operator != (const FireEventRequestData &rhs) const { - return !(*this == rhs); - } - - bool operator < (const FireEventRequestData & ) 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(FireEventRequestData &a, FireEventRequestData &b); - -std::ostream& operator<<(std::ostream& out, const FireEventRequestData& obj); - -typedef struct _FireEventRequest__isset { - _FireEventRequest__isset() : dbName(false), tableName(false), partitionVals(false) {} - bool dbName :1; - bool tableName :1; - bool partitionVals :1; -} _FireEventRequest__isset; - -class FireEventRequest : public virtual ::apache::thrift::TBase { - public: - - FireEventRequest(const FireEventRequest&); - FireEventRequest& operator=(const FireEventRequest&); - FireEventRequest() : successful(0), dbName(), tableName() { - } - - virtual ~FireEventRequest() noexcept; - bool successful; - FireEventRequestData data; - std::string dbName; - std::string tableName; - std::vector partitionVals; - - _FireEventRequest__isset __isset; - - void __set_successful(const bool val); - - void __set_data(const FireEventRequestData& val); - - void __set_dbName(const std::string& val); - - void __set_tableName(const std::string& val); - - void __set_partitionVals(const std::vector & val); - - bool operator == (const FireEventRequest & rhs) const - { - if (!(successful == rhs.successful)) - return false; - if (!(data == rhs.data)) - return false; - if (__isset.dbName != rhs.__isset.dbName) - return false; - else if (__isset.dbName && !(dbName == rhs.dbName)) - return false; - if (__isset.tableName != rhs.__isset.tableName) - return false; - else if (__isset.tableName && !(tableName == rhs.tableName)) - return false; - if (__isset.partitionVals != rhs.__isset.partitionVals) - return false; - else if (__isset.partitionVals && !(partitionVals == rhs.partitionVals)) - return false; - return true; - } - bool operator != (const FireEventRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const FireEventRequest & ) 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(FireEventRequest &a, FireEventRequest &b); - -std::ostream& operator<<(std::ostream& out, const FireEventRequest& obj); - - -class FireEventResponse : public virtual ::apache::thrift::TBase { - public: - - FireEventResponse(const FireEventResponse&); - FireEventResponse& operator=(const FireEventResponse&); - FireEventResponse() { - } - - virtual ~FireEventResponse() noexcept; - - bool operator == (const FireEventResponse & /* rhs */) const - { - return true; - } - bool operator != (const FireEventResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const FireEventResponse & ) 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(FireEventResponse &a, FireEventResponse &b); - -std::ostream& operator<<(std::ostream& out, const FireEventResponse& obj); - -typedef struct _MetadataPpdResult__isset { - _MetadataPpdResult__isset() : metadata(false), includeBitset(false) {} - bool metadata :1; - bool includeBitset :1; -} _MetadataPpdResult__isset; - -class MetadataPpdResult : public virtual ::apache::thrift::TBase { - public: - - MetadataPpdResult(const MetadataPpdResult&); - MetadataPpdResult& operator=(const MetadataPpdResult&); - MetadataPpdResult() : metadata(), includeBitset() { - } - - virtual ~MetadataPpdResult() noexcept; - std::string metadata; - std::string includeBitset; - - _MetadataPpdResult__isset __isset; - - void __set_metadata(const std::string& val); - - void __set_includeBitset(const std::string& val); - - bool operator == (const MetadataPpdResult & rhs) const - { - if (__isset.metadata != rhs.__isset.metadata) - return false; - else if (__isset.metadata && !(metadata == rhs.metadata)) - return false; - if (__isset.includeBitset != rhs.__isset.includeBitset) - return false; - else if (__isset.includeBitset && !(includeBitset == rhs.includeBitset)) - return false; - return true; - } - bool operator != (const MetadataPpdResult &rhs) const { - return !(*this == rhs); - } - - bool operator < (const MetadataPpdResult & ) 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(MetadataPpdResult &a, MetadataPpdResult &b); - -std::ostream& operator<<(std::ostream& out, const MetadataPpdResult& obj); - - -class GetFileMetadataByExprResult : public virtual ::apache::thrift::TBase { - public: - - GetFileMetadataByExprResult(const GetFileMetadataByExprResult&); - GetFileMetadataByExprResult& operator=(const GetFileMetadataByExprResult&); - GetFileMetadataByExprResult() : isSupported(0) { - } - - virtual ~GetFileMetadataByExprResult() noexcept; - std::map metadata; - bool isSupported; - - void __set_metadata(const std::map & val); - - void __set_isSupported(const bool val); - - bool operator == (const GetFileMetadataByExprResult & rhs) const - { - if (!(metadata == rhs.metadata)) - return false; - if (!(isSupported == rhs.isSupported)) - return false; - return true; - } - bool operator != (const GetFileMetadataByExprResult &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GetFileMetadataByExprResult & ) 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(GetFileMetadataByExprResult &a, GetFileMetadataByExprResult &b); - -std::ostream& operator<<(std::ostream& out, const GetFileMetadataByExprResult& obj); - -typedef struct _GetFileMetadataByExprRequest__isset { - _GetFileMetadataByExprRequest__isset() : doGetFooters(false), type(false) {} - bool doGetFooters :1; - bool type :1; -} _GetFileMetadataByExprRequest__isset; - -class GetFileMetadataByExprRequest : public virtual ::apache::thrift::TBase { - public: - - GetFileMetadataByExprRequest(const GetFileMetadataByExprRequest&); - GetFileMetadataByExprRequest& operator=(const GetFileMetadataByExprRequest&); - GetFileMetadataByExprRequest() : expr(), doGetFooters(0), type((FileMetadataExprType::type)0) { - } - - virtual ~GetFileMetadataByExprRequest() noexcept; - std::vector fileIds; - std::string expr; - bool doGetFooters; - FileMetadataExprType::type type; - - _GetFileMetadataByExprRequest__isset __isset; - - void __set_fileIds(const std::vector & val); - - void __set_expr(const std::string& val); - - void __set_doGetFooters(const bool val); - - void __set_type(const FileMetadataExprType::type val); - - bool operator == (const GetFileMetadataByExprRequest & rhs) const - { - if (!(fileIds == rhs.fileIds)) - return false; - if (!(expr == rhs.expr)) - return false; - if (__isset.doGetFooters != rhs.__isset.doGetFooters) - return false; - else if (__isset.doGetFooters && !(doGetFooters == rhs.doGetFooters)) - return false; - if (__isset.type != rhs.__isset.type) - return false; - else if (__isset.type && !(type == rhs.type)) - return false; - return true; - } - bool operator != (const GetFileMetadataByExprRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GetFileMetadataByExprRequest & ) 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(GetFileMetadataByExprRequest &a, GetFileMetadataByExprRequest &b); - -std::ostream& operator<<(std::ostream& out, const GetFileMetadataByExprRequest& obj); - - -class GetFileMetadataResult : public virtual ::apache::thrift::TBase { - public: - - GetFileMetadataResult(const GetFileMetadataResult&); - GetFileMetadataResult& operator=(const GetFileMetadataResult&); - GetFileMetadataResult() : isSupported(0) { - } - - virtual ~GetFileMetadataResult() noexcept; - std::map metadata; - bool isSupported; - - void __set_metadata(const std::map & val); - - void __set_isSupported(const bool val); - - bool operator == (const GetFileMetadataResult & rhs) const - { - if (!(metadata == rhs.metadata)) - return false; - if (!(isSupported == rhs.isSupported)) - return false; - return true; - } - bool operator != (const GetFileMetadataResult &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GetFileMetadataResult & ) 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(GetFileMetadataResult &a, GetFileMetadataResult &b); - -std::ostream& operator<<(std::ostream& out, const GetFileMetadataResult& obj); - - -class GetFileMetadataRequest : public virtual ::apache::thrift::TBase { - public: - - GetFileMetadataRequest(const GetFileMetadataRequest&); - GetFileMetadataRequest& operator=(const GetFileMetadataRequest&); - GetFileMetadataRequest() { - } - - virtual ~GetFileMetadataRequest() noexcept; - std::vector fileIds; - - void __set_fileIds(const std::vector & val); - - bool operator == (const GetFileMetadataRequest & rhs) const - { - if (!(fileIds == rhs.fileIds)) - return false; - return true; - } - bool operator != (const GetFileMetadataRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GetFileMetadataRequest & ) 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(GetFileMetadataRequest &a, GetFileMetadataRequest &b); - -std::ostream& operator<<(std::ostream& out, const GetFileMetadataRequest& obj); - - -class PutFileMetadataResult : public virtual ::apache::thrift::TBase { - public: - - PutFileMetadataResult(const PutFileMetadataResult&); - PutFileMetadataResult& operator=(const PutFileMetadataResult&); - PutFileMetadataResult() { - } - - virtual ~PutFileMetadataResult() noexcept; - - bool operator == (const PutFileMetadataResult & /* rhs */) const - { - return true; - } - bool operator != (const PutFileMetadataResult &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PutFileMetadataResult & ) 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(PutFileMetadataResult &a, PutFileMetadataResult &b); - -std::ostream& operator<<(std::ostream& out, const PutFileMetadataResult& obj); - -typedef struct _PutFileMetadataRequest__isset { - _PutFileMetadataRequest__isset() : type(false) {} - bool type :1; -} _PutFileMetadataRequest__isset; - -class PutFileMetadataRequest : public virtual ::apache::thrift::TBase { - public: - - PutFileMetadataRequest(const PutFileMetadataRequest&); - PutFileMetadataRequest& operator=(const PutFileMetadataRequest&); - PutFileMetadataRequest() : type((FileMetadataExprType::type)0) { - } - - virtual ~PutFileMetadataRequest() noexcept; - std::vector fileIds; - std::vector metadata; - FileMetadataExprType::type type; - - _PutFileMetadataRequest__isset __isset; - - void __set_fileIds(const std::vector & val); - - void __set_metadata(const std::vector & val); - - void __set_type(const FileMetadataExprType::type val); - - bool operator == (const PutFileMetadataRequest & rhs) const - { - if (!(fileIds == rhs.fileIds)) - return false; - if (!(metadata == rhs.metadata)) - return false; - if (__isset.type != rhs.__isset.type) - return false; - else if (__isset.type && !(type == rhs.type)) - return false; - return true; - } - bool operator != (const PutFileMetadataRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PutFileMetadataRequest & ) 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(PutFileMetadataRequest &a, PutFileMetadataRequest &b); - -std::ostream& operator<<(std::ostream& out, const PutFileMetadataRequest& obj); - - -class ClearFileMetadataResult : public virtual ::apache::thrift::TBase { - public: - - ClearFileMetadataResult(const ClearFileMetadataResult&); - ClearFileMetadataResult& operator=(const ClearFileMetadataResult&); - ClearFileMetadataResult() { - } - - virtual ~ClearFileMetadataResult() noexcept; - - bool operator == (const ClearFileMetadataResult & /* rhs */) const - { - return true; - } - bool operator != (const ClearFileMetadataResult &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ClearFileMetadataResult & ) 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(ClearFileMetadataResult &a, ClearFileMetadataResult &b); - -std::ostream& operator<<(std::ostream& out, const ClearFileMetadataResult& obj); - - -class ClearFileMetadataRequest : public virtual ::apache::thrift::TBase { - public: - - ClearFileMetadataRequest(const ClearFileMetadataRequest&); - ClearFileMetadataRequest& operator=(const ClearFileMetadataRequest&); - ClearFileMetadataRequest() { - } - - virtual ~ClearFileMetadataRequest() noexcept; - std::vector fileIds; - - void __set_fileIds(const std::vector & val); - - bool operator == (const ClearFileMetadataRequest & rhs) const - { - if (!(fileIds == rhs.fileIds)) - return false; - return true; - } - bool operator != (const ClearFileMetadataRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ClearFileMetadataRequest & ) 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(ClearFileMetadataRequest &a, ClearFileMetadataRequest &b); - -std::ostream& operator<<(std::ostream& out, const ClearFileMetadataRequest& obj); - - -class CacheFileMetadataResult : public virtual ::apache::thrift::TBase { - public: - - CacheFileMetadataResult(const CacheFileMetadataResult&); - CacheFileMetadataResult& operator=(const CacheFileMetadataResult&); - CacheFileMetadataResult() : isSupported(0) { - } - - virtual ~CacheFileMetadataResult() noexcept; - bool isSupported; - - void __set_isSupported(const bool val); - - bool operator == (const CacheFileMetadataResult & rhs) const - { - if (!(isSupported == rhs.isSupported)) - return false; - return true; - } - bool operator != (const CacheFileMetadataResult &rhs) const { - return !(*this == rhs); - } - - bool operator < (const CacheFileMetadataResult & ) 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(CacheFileMetadataResult &a, CacheFileMetadataResult &b); - -std::ostream& operator<<(std::ostream& out, const CacheFileMetadataResult& obj); - -typedef struct _CacheFileMetadataRequest__isset { - _CacheFileMetadataRequest__isset() : partName(false), isAllParts(false) {} - bool partName :1; - bool isAllParts :1; -} _CacheFileMetadataRequest__isset; - -class CacheFileMetadataRequest : public virtual ::apache::thrift::TBase { - public: - - CacheFileMetadataRequest(const CacheFileMetadataRequest&); - CacheFileMetadataRequest& operator=(const CacheFileMetadataRequest&); - CacheFileMetadataRequest() : dbName(), tblName(), partName(), isAllParts(0) { - } - - virtual ~CacheFileMetadataRequest() noexcept; - std::string dbName; - std::string tblName; - std::string partName; - bool isAllParts; - - _CacheFileMetadataRequest__isset __isset; - - void __set_dbName(const std::string& val); - - void __set_tblName(const std::string& val); - - void __set_partName(const std::string& val); - - void __set_isAllParts(const bool val); - - bool operator == (const CacheFileMetadataRequest & rhs) const - { - if (!(dbName == rhs.dbName)) - return false; - if (!(tblName == rhs.tblName)) - return false; - if (__isset.partName != rhs.__isset.partName) - return false; - else if (__isset.partName && !(partName == rhs.partName)) - return false; - if (__isset.isAllParts != rhs.__isset.isAllParts) - return false; - else if (__isset.isAllParts && !(isAllParts == rhs.isAllParts)) - return false; - return true; - } - bool operator != (const CacheFileMetadataRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const CacheFileMetadataRequest & ) 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(CacheFileMetadataRequest &a, CacheFileMetadataRequest &b); - -std::ostream& operator<<(std::ostream& out, const CacheFileMetadataRequest& obj); - -typedef struct _GetAllFunctionsResponse__isset { - _GetAllFunctionsResponse__isset() : functions(false) {} - bool functions :1; -} _GetAllFunctionsResponse__isset; - -class GetAllFunctionsResponse : public virtual ::apache::thrift::TBase { - public: - - GetAllFunctionsResponse(const GetAllFunctionsResponse&); - GetAllFunctionsResponse& operator=(const GetAllFunctionsResponse&); - GetAllFunctionsResponse() { - } - - virtual ~GetAllFunctionsResponse() noexcept; - std::vector functions; - - _GetAllFunctionsResponse__isset __isset; - - void __set_functions(const std::vector & val); - - bool operator == (const GetAllFunctionsResponse & rhs) const - { - if (__isset.functions != rhs.__isset.functions) - return false; - else if (__isset.functions && !(functions == rhs.functions)) - return false; - return true; - } - bool operator != (const GetAllFunctionsResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GetAllFunctionsResponse & ) 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(GetAllFunctionsResponse &a, GetAllFunctionsResponse &b); - -std::ostream& operator<<(std::ostream& out, const GetAllFunctionsResponse& obj); - - -class ClientCapabilities : public virtual ::apache::thrift::TBase { - public: - - ClientCapabilities(const ClientCapabilities&); - ClientCapabilities& operator=(const ClientCapabilities&); - ClientCapabilities() { - } - - virtual ~ClientCapabilities() noexcept; - std::vector values; - - void __set_values(const std::vector & val); - - bool operator == (const ClientCapabilities & rhs) const - { - if (!(values == rhs.values)) - return false; - return true; - } - bool operator != (const ClientCapabilities &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ClientCapabilities & ) 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(ClientCapabilities &a, ClientCapabilities &b); - -std::ostream& operator<<(std::ostream& out, const ClientCapabilities& obj); - -typedef struct _GetTableRequest__isset { - _GetTableRequest__isset() : capabilities(false) {} - bool capabilities :1; -} _GetTableRequest__isset; - -class GetTableRequest : public virtual ::apache::thrift::TBase { - public: - - GetTableRequest(const GetTableRequest&); - GetTableRequest& operator=(const GetTableRequest&); - GetTableRequest() : dbName(), tblName() { - } - - virtual ~GetTableRequest() noexcept; - std::string dbName; - std::string tblName; - ClientCapabilities capabilities; - - _GetTableRequest__isset __isset; - - void __set_dbName(const std::string& val); - - void __set_tblName(const std::string& val); - - void __set_capabilities(const ClientCapabilities& val); - - bool operator == (const GetTableRequest & rhs) const - { - if (!(dbName == rhs.dbName)) - return false; - if (!(tblName == rhs.tblName)) - return false; - if (__isset.capabilities != rhs.__isset.capabilities) - return false; - else if (__isset.capabilities && !(capabilities == rhs.capabilities)) - return false; - return true; - } - bool operator != (const GetTableRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GetTableRequest & ) 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(GetTableRequest &a, GetTableRequest &b); - -std::ostream& operator<<(std::ostream& out, const GetTableRequest& obj); - - -class GetTableResult : public virtual ::apache::thrift::TBase { - public: - - GetTableResult(const GetTableResult&); - GetTableResult& operator=(const GetTableResult&); - GetTableResult() { - } - - virtual ~GetTableResult() noexcept; - Table table; - - void __set_table(const Table& val); - - bool operator == (const GetTableResult & rhs) const - { - if (!(table == rhs.table)) - return false; - return true; - } - bool operator != (const GetTableResult &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GetTableResult & ) 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(GetTableResult &a, GetTableResult &b); - -std::ostream& operator<<(std::ostream& out, const GetTableResult& obj); - -typedef struct _GetTablesRequest__isset { - _GetTablesRequest__isset() : tblNames(false), capabilities(false) {} - bool tblNames :1; - bool capabilities :1; -} _GetTablesRequest__isset; - -class GetTablesRequest : public virtual ::apache::thrift::TBase { - public: - - GetTablesRequest(const GetTablesRequest&); - GetTablesRequest& operator=(const GetTablesRequest&); - GetTablesRequest() : dbName() { - } - - virtual ~GetTablesRequest() noexcept; - std::string dbName; - std::vector tblNames; - ClientCapabilities capabilities; - - _GetTablesRequest__isset __isset; - - void __set_dbName(const std::string& val); - - void __set_tblNames(const std::vector & val); - - void __set_capabilities(const ClientCapabilities& val); - - bool operator == (const GetTablesRequest & rhs) const - { - if (!(dbName == rhs.dbName)) - return false; - if (__isset.tblNames != rhs.__isset.tblNames) - return false; - else if (__isset.tblNames && !(tblNames == rhs.tblNames)) - return false; - if (__isset.capabilities != rhs.__isset.capabilities) - return false; - else if (__isset.capabilities && !(capabilities == rhs.capabilities)) - return false; - return true; - } - bool operator != (const GetTablesRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GetTablesRequest & ) 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(GetTablesRequest &a, GetTablesRequest &b); - -std::ostream& operator<<(std::ostream& out, const GetTablesRequest& obj); - - -class GetTablesResult : public virtual ::apache::thrift::TBase { - public: - - GetTablesResult(const GetTablesResult&); - GetTablesResult& operator=(const GetTablesResult&); - GetTablesResult() { - } - - virtual ~GetTablesResult() noexcept; - std::vector
tables; - - void __set_tables(const std::vector
& val); - - bool operator == (const GetTablesResult & rhs) const - { - if (!(tables == rhs.tables)) - return false; - return true; - } - bool operator != (const GetTablesResult &rhs) const { - return !(*this == rhs); - } - - bool operator < (const GetTablesResult & ) 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(GetTablesResult &a, GetTablesResult &b); - -std::ostream& operator<<(std::ostream& out, const GetTablesResult& obj); - - -class CmRecycleRequest : public virtual ::apache::thrift::TBase { - public: - - CmRecycleRequest(const CmRecycleRequest&); - CmRecycleRequest& operator=(const CmRecycleRequest&); - CmRecycleRequest() : dataPath(), purge(0) { - } - - virtual ~CmRecycleRequest() noexcept; - std::string dataPath; - bool purge; - - void __set_dataPath(const std::string& val); - - void __set_purge(const bool val); - - bool operator == (const CmRecycleRequest & rhs) const - { - if (!(dataPath == rhs.dataPath)) - return false; - if (!(purge == rhs.purge)) - return false; - return true; - } - bool operator != (const CmRecycleRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const CmRecycleRequest & ) 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(CmRecycleRequest &a, CmRecycleRequest &b); - -std::ostream& operator<<(std::ostream& out, const CmRecycleRequest& obj); - - -class CmRecycleResponse : public virtual ::apache::thrift::TBase { - public: - - CmRecycleResponse(const CmRecycleResponse&); - CmRecycleResponse& operator=(const CmRecycleResponse&); - CmRecycleResponse() { - } - - virtual ~CmRecycleResponse() noexcept; - - bool operator == (const CmRecycleResponse & /* rhs */) const - { - return true; - } - bool operator != (const CmRecycleResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const CmRecycleResponse & ) 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(CmRecycleResponse &a, CmRecycleResponse &b); - -std::ostream& operator<<(std::ostream& out, const CmRecycleResponse& obj); - -typedef struct _TableMeta__isset { - _TableMeta__isset() : comments(false) {} - bool comments :1; -} _TableMeta__isset; - -class TableMeta : public virtual ::apache::thrift::TBase { - public: - - TableMeta(const TableMeta&); - TableMeta& operator=(const TableMeta&); - TableMeta() : dbName(), tableName(), tableType(), comments() { - } - - virtual ~TableMeta() noexcept; - std::string dbName; - std::string tableName; - std::string tableType; - std::string comments; - - _TableMeta__isset __isset; - - void __set_dbName(const std::string& val); - - void __set_tableName(const std::string& val); - - void __set_tableType(const std::string& val); - - void __set_comments(const std::string& val); - - bool operator == (const TableMeta & rhs) const - { - if (!(dbName == rhs.dbName)) - return false; - if (!(tableName == rhs.tableName)) - return false; - if (!(tableType == rhs.tableType)) - return false; - if (__isset.comments != rhs.__isset.comments) - return false; - else if (__isset.comments && !(comments == rhs.comments)) - return false; - return true; - } - bool operator != (const TableMeta &rhs) const { - return !(*this == rhs); - } - - bool operator < (const TableMeta & ) 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(TableMeta &a, TableMeta &b); - -std::ostream& operator<<(std::ostream& out, const TableMeta& obj); - -typedef struct _WMResourcePlan__isset { - _WMResourcePlan__isset() : status(false), queryParallelism(false), defaultPoolPath(false) {} - bool status :1; - bool queryParallelism :1; - bool defaultPoolPath :1; -} _WMResourcePlan__isset; - -class WMResourcePlan : public virtual ::apache::thrift::TBase { - public: - - WMResourcePlan(const WMResourcePlan&); - WMResourcePlan& operator=(const WMResourcePlan&); - WMResourcePlan() : name(), status((WMResourcePlanStatus::type)0), queryParallelism(0), defaultPoolPath() { - } - - virtual ~WMResourcePlan() noexcept; - std::string name; - WMResourcePlanStatus::type status; - int32_t queryParallelism; - std::string defaultPoolPath; - - _WMResourcePlan__isset __isset; - - void __set_name(const std::string& val); - - void __set_status(const WMResourcePlanStatus::type val); - - void __set_queryParallelism(const int32_t val); - - void __set_defaultPoolPath(const std::string& val); - - bool operator == (const WMResourcePlan & rhs) const - { - if (!(name == rhs.name)) - return false; - if (__isset.status != rhs.__isset.status) - return false; - else if (__isset.status && !(status == rhs.status)) - return false; - if (__isset.queryParallelism != rhs.__isset.queryParallelism) - return false; - else if (__isset.queryParallelism && !(queryParallelism == rhs.queryParallelism)) - return false; - if (__isset.defaultPoolPath != rhs.__isset.defaultPoolPath) - return false; - else if (__isset.defaultPoolPath && !(defaultPoolPath == rhs.defaultPoolPath)) - return false; - return true; - } - bool operator != (const WMResourcePlan &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMResourcePlan & ) 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(WMResourcePlan &a, WMResourcePlan &b); - -std::ostream& operator<<(std::ostream& out, const WMResourcePlan& obj); - -typedef struct _WMPool__isset { - _WMPool__isset() : allocFraction(false), queryParallelism(false), schedulingPolicy(false) {} - bool allocFraction :1; - bool queryParallelism :1; - bool schedulingPolicy :1; -} _WMPool__isset; - -class WMPool : public virtual ::apache::thrift::TBase { - public: - - WMPool(const WMPool&); - WMPool& operator=(const WMPool&); - WMPool() : resourcePlanName(), poolPath(), allocFraction(0), queryParallelism(0), schedulingPolicy() { - } - - virtual ~WMPool() noexcept; - std::string resourcePlanName; - std::string poolPath; - double allocFraction; - int32_t queryParallelism; - std::string schedulingPolicy; - - _WMPool__isset __isset; - - void __set_resourcePlanName(const std::string& val); - - void __set_poolPath(const std::string& val); - - void __set_allocFraction(const double val); - - void __set_queryParallelism(const int32_t val); - - void __set_schedulingPolicy(const std::string& val); - - bool operator == (const WMPool & rhs) const - { - if (!(resourcePlanName == rhs.resourcePlanName)) - return false; - if (!(poolPath == rhs.poolPath)) - return false; - if (__isset.allocFraction != rhs.__isset.allocFraction) - return false; - else if (__isset.allocFraction && !(allocFraction == rhs.allocFraction)) - return false; - if (__isset.queryParallelism != rhs.__isset.queryParallelism) - return false; - else if (__isset.queryParallelism && !(queryParallelism == rhs.queryParallelism)) - return false; - if (__isset.schedulingPolicy != rhs.__isset.schedulingPolicy) - return false; - else if (__isset.schedulingPolicy && !(schedulingPolicy == rhs.schedulingPolicy)) - return false; - return true; - } - bool operator != (const WMPool &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMPool & ) 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(WMPool &a, WMPool &b); - -std::ostream& operator<<(std::ostream& out, const WMPool& obj); - -typedef struct _WMTrigger__isset { - _WMTrigger__isset() : triggerExpression(false), actionExpression(false) {} - bool triggerExpression :1; - bool actionExpression :1; -} _WMTrigger__isset; - -class WMTrigger : public virtual ::apache::thrift::TBase { - public: - - WMTrigger(const WMTrigger&); - WMTrigger& operator=(const WMTrigger&); - WMTrigger() : resourcePlanName(), triggerName(), triggerExpression(), actionExpression() { - } - - virtual ~WMTrigger() noexcept; - std::string resourcePlanName; - std::string triggerName; - std::string triggerExpression; - std::string actionExpression; - - _WMTrigger__isset __isset; - - void __set_resourcePlanName(const std::string& val); - - void __set_triggerName(const std::string& val); - - void __set_triggerExpression(const std::string& val); - - void __set_actionExpression(const std::string& val); - - bool operator == (const WMTrigger & rhs) const - { - if (!(resourcePlanName == rhs.resourcePlanName)) - return false; - if (!(triggerName == rhs.triggerName)) - return false; - if (__isset.triggerExpression != rhs.__isset.triggerExpression) - return false; - else if (__isset.triggerExpression && !(triggerExpression == rhs.triggerExpression)) - return false; - if (__isset.actionExpression != rhs.__isset.actionExpression) - return false; - else if (__isset.actionExpression && !(actionExpression == rhs.actionExpression)) - return false; - return true; - } - bool operator != (const WMTrigger &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMTrigger & ) 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(WMTrigger &a, WMTrigger &b); - -std::ostream& operator<<(std::ostream& out, const WMTrigger& obj); - -typedef struct _WMMapping__isset { - _WMMapping__isset() : poolName(false), ordering(false) {} - bool poolName :1; - bool ordering :1; -} _WMMapping__isset; - -class WMMapping : public virtual ::apache::thrift::TBase { - public: - - WMMapping(const WMMapping&); - WMMapping& operator=(const WMMapping&); - WMMapping() : resourcePlanName(), entityType(), entityName(), poolName(), ordering(0) { - } - - virtual ~WMMapping() noexcept; - std::string resourcePlanName; - std::string entityType; - std::string entityName; - std::string poolName; - int32_t ordering; - - _WMMapping__isset __isset; - - void __set_resourcePlanName(const std::string& val); - - void __set_entityType(const std::string& val); - - void __set_entityName(const std::string& val); - - void __set_poolName(const std::string& val); - - void __set_ordering(const int32_t val); - - bool operator == (const WMMapping & rhs) const - { - if (!(resourcePlanName == rhs.resourcePlanName)) - return false; - if (!(entityType == rhs.entityType)) - return false; - if (!(entityName == rhs.entityName)) - return false; - if (__isset.poolName != rhs.__isset.poolName) - return false; - else if (__isset.poolName && !(poolName == rhs.poolName)) - return false; - if (__isset.ordering != rhs.__isset.ordering) - return false; - else if (__isset.ordering && !(ordering == rhs.ordering)) - return false; - return true; - } - bool operator != (const WMMapping &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMMapping & ) 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(WMMapping &a, WMMapping &b); - -std::ostream& operator<<(std::ostream& out, const WMMapping& obj); - - -class WMPoolTrigger : public virtual ::apache::thrift::TBase { - public: - - WMPoolTrigger(const WMPoolTrigger&); - WMPoolTrigger& operator=(const WMPoolTrigger&); - WMPoolTrigger() : pool(), trigger() { - } - - virtual ~WMPoolTrigger() noexcept; - std::string pool; - std::string trigger; - - void __set_pool(const std::string& val); - - void __set_trigger(const std::string& val); - - bool operator == (const WMPoolTrigger & rhs) const - { - if (!(pool == rhs.pool)) - return false; - if (!(trigger == rhs.trigger)) - return false; - return true; - } - bool operator != (const WMPoolTrigger &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMPoolTrigger & ) 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(WMPoolTrigger &a, WMPoolTrigger &b); - -std::ostream& operator<<(std::ostream& out, const WMPoolTrigger& obj); - -typedef struct _WMFullResourcePlan__isset { - _WMFullResourcePlan__isset() : mappings(false), triggers(false), poolTriggers(false) {} - bool mappings :1; - bool triggers :1; - bool poolTriggers :1; -} _WMFullResourcePlan__isset; - -class WMFullResourcePlan : public virtual ::apache::thrift::TBase { - public: - - WMFullResourcePlan(const WMFullResourcePlan&); - WMFullResourcePlan& operator=(const WMFullResourcePlan&); - WMFullResourcePlan() { - } - - virtual ~WMFullResourcePlan() noexcept; - WMResourcePlan plan; - std::vector pools; - std::vector mappings; - std::vector triggers; - std::vector poolTriggers; - - _WMFullResourcePlan__isset __isset; - - void __set_plan(const WMResourcePlan& val); - - void __set_pools(const std::vector & val); - - void __set_mappings(const std::vector & val); - - void __set_triggers(const std::vector & val); - - void __set_poolTriggers(const std::vector & val); - - bool operator == (const WMFullResourcePlan & rhs) const - { - if (!(plan == rhs.plan)) - return false; - if (!(pools == rhs.pools)) - return false; - if (__isset.mappings != rhs.__isset.mappings) - return false; - else if (__isset.mappings && !(mappings == rhs.mappings)) - return false; - if (__isset.triggers != rhs.__isset.triggers) - return false; - else if (__isset.triggers && !(triggers == rhs.triggers)) - return false; - if (__isset.poolTriggers != rhs.__isset.poolTriggers) - return false; - else if (__isset.poolTriggers && !(poolTriggers == rhs.poolTriggers)) - return false; - return true; - } - bool operator != (const WMFullResourcePlan &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMFullResourcePlan & ) 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(WMFullResourcePlan &a, WMFullResourcePlan &b); - -std::ostream& operator<<(std::ostream& out, const WMFullResourcePlan& obj); - -typedef struct _WMCreateResourcePlanRequest__isset { - _WMCreateResourcePlanRequest__isset() : resourcePlan(false) {} - bool resourcePlan :1; -} _WMCreateResourcePlanRequest__isset; - -class WMCreateResourcePlanRequest : public virtual ::apache::thrift::TBase { - public: - - WMCreateResourcePlanRequest(const WMCreateResourcePlanRequest&); - WMCreateResourcePlanRequest& operator=(const WMCreateResourcePlanRequest&); - WMCreateResourcePlanRequest() { - } - - virtual ~WMCreateResourcePlanRequest() noexcept; - WMResourcePlan resourcePlan; - - _WMCreateResourcePlanRequest__isset __isset; - - void __set_resourcePlan(const WMResourcePlan& val); - - bool operator == (const WMCreateResourcePlanRequest & rhs) const - { - if (__isset.resourcePlan != rhs.__isset.resourcePlan) - return false; - else if (__isset.resourcePlan && !(resourcePlan == rhs.resourcePlan)) - return false; - return true; - } - bool operator != (const WMCreateResourcePlanRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMCreateResourcePlanRequest & ) 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(WMCreateResourcePlanRequest &a, WMCreateResourcePlanRequest &b); - -std::ostream& operator<<(std::ostream& out, const WMCreateResourcePlanRequest& obj); - - -class WMCreateResourcePlanResponse : public virtual ::apache::thrift::TBase { - public: - - WMCreateResourcePlanResponse(const WMCreateResourcePlanResponse&); - WMCreateResourcePlanResponse& operator=(const WMCreateResourcePlanResponse&); - WMCreateResourcePlanResponse() { - } - - virtual ~WMCreateResourcePlanResponse() noexcept; - - bool operator == (const WMCreateResourcePlanResponse & /* rhs */) const - { - return true; - } - bool operator != (const WMCreateResourcePlanResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMCreateResourcePlanResponse & ) 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(WMCreateResourcePlanResponse &a, WMCreateResourcePlanResponse &b); - -std::ostream& operator<<(std::ostream& out, const WMCreateResourcePlanResponse& obj); - - -class WMGetActiveResourcePlanRequest : public virtual ::apache::thrift::TBase { - public: - - WMGetActiveResourcePlanRequest(const WMGetActiveResourcePlanRequest&); - WMGetActiveResourcePlanRequest& operator=(const WMGetActiveResourcePlanRequest&); - WMGetActiveResourcePlanRequest() { - } - - virtual ~WMGetActiveResourcePlanRequest() noexcept; - - bool operator == (const WMGetActiveResourcePlanRequest & /* rhs */) const - { - return true; - } - bool operator != (const WMGetActiveResourcePlanRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMGetActiveResourcePlanRequest & ) 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(WMGetActiveResourcePlanRequest &a, WMGetActiveResourcePlanRequest &b); - -std::ostream& operator<<(std::ostream& out, const WMGetActiveResourcePlanRequest& obj); - -typedef struct _WMGetActiveResourcePlanResponse__isset { - _WMGetActiveResourcePlanResponse__isset() : resourcePlan(false) {} - bool resourcePlan :1; -} _WMGetActiveResourcePlanResponse__isset; - -class WMGetActiveResourcePlanResponse : public virtual ::apache::thrift::TBase { - public: - - WMGetActiveResourcePlanResponse(const WMGetActiveResourcePlanResponse&); - WMGetActiveResourcePlanResponse& operator=(const WMGetActiveResourcePlanResponse&); - WMGetActiveResourcePlanResponse() { - } - - virtual ~WMGetActiveResourcePlanResponse() noexcept; - WMFullResourcePlan resourcePlan; - - _WMGetActiveResourcePlanResponse__isset __isset; - - void __set_resourcePlan(const WMFullResourcePlan& val); - - bool operator == (const WMGetActiveResourcePlanResponse & rhs) const - { - if (__isset.resourcePlan != rhs.__isset.resourcePlan) - return false; - else if (__isset.resourcePlan && !(resourcePlan == rhs.resourcePlan)) - return false; - return true; - } - bool operator != (const WMGetActiveResourcePlanResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMGetActiveResourcePlanResponse & ) 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(WMGetActiveResourcePlanResponse &a, WMGetActiveResourcePlanResponse &b); - -std::ostream& operator<<(std::ostream& out, const WMGetActiveResourcePlanResponse& obj); - -typedef struct _WMGetResourcePlanRequest__isset { - _WMGetResourcePlanRequest__isset() : resourcePlanName(false) {} - bool resourcePlanName :1; -} _WMGetResourcePlanRequest__isset; - -class WMGetResourcePlanRequest : public virtual ::apache::thrift::TBase { - public: - - WMGetResourcePlanRequest(const WMGetResourcePlanRequest&); - WMGetResourcePlanRequest& operator=(const WMGetResourcePlanRequest&); - WMGetResourcePlanRequest() : resourcePlanName() { - } - - virtual ~WMGetResourcePlanRequest() noexcept; - std::string resourcePlanName; - - _WMGetResourcePlanRequest__isset __isset; - - void __set_resourcePlanName(const std::string& val); - - bool operator == (const WMGetResourcePlanRequest & rhs) const - { - if (__isset.resourcePlanName != rhs.__isset.resourcePlanName) - return false; - else if (__isset.resourcePlanName && !(resourcePlanName == rhs.resourcePlanName)) - return false; - return true; - } - bool operator != (const WMGetResourcePlanRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMGetResourcePlanRequest & ) 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(WMGetResourcePlanRequest &a, WMGetResourcePlanRequest &b); - -std::ostream& operator<<(std::ostream& out, const WMGetResourcePlanRequest& obj); - -typedef struct _WMGetResourcePlanResponse__isset { - _WMGetResourcePlanResponse__isset() : resourcePlan(false) {} - bool resourcePlan :1; -} _WMGetResourcePlanResponse__isset; - -class WMGetResourcePlanResponse : public virtual ::apache::thrift::TBase { - public: - - WMGetResourcePlanResponse(const WMGetResourcePlanResponse&); - WMGetResourcePlanResponse& operator=(const WMGetResourcePlanResponse&); - WMGetResourcePlanResponse() { - } - - virtual ~WMGetResourcePlanResponse() noexcept; - WMResourcePlan resourcePlan; - - _WMGetResourcePlanResponse__isset __isset; - - void __set_resourcePlan(const WMResourcePlan& val); - - bool operator == (const WMGetResourcePlanResponse & rhs) const - { - if (__isset.resourcePlan != rhs.__isset.resourcePlan) - return false; - else if (__isset.resourcePlan && !(resourcePlan == rhs.resourcePlan)) - return false; - return true; - } - bool operator != (const WMGetResourcePlanResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMGetResourcePlanResponse & ) 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(WMGetResourcePlanResponse &a, WMGetResourcePlanResponse &b); - -std::ostream& operator<<(std::ostream& out, const WMGetResourcePlanResponse& obj); - - -class WMGetAllResourcePlanRequest : public virtual ::apache::thrift::TBase { - public: - - WMGetAllResourcePlanRequest(const WMGetAllResourcePlanRequest&); - WMGetAllResourcePlanRequest& operator=(const WMGetAllResourcePlanRequest&); - WMGetAllResourcePlanRequest() { - } - - virtual ~WMGetAllResourcePlanRequest() noexcept; - - bool operator == (const WMGetAllResourcePlanRequest & /* rhs */) const - { - return true; - } - bool operator != (const WMGetAllResourcePlanRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMGetAllResourcePlanRequest & ) 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(WMGetAllResourcePlanRequest &a, WMGetAllResourcePlanRequest &b); - -std::ostream& operator<<(std::ostream& out, const WMGetAllResourcePlanRequest& obj); - -typedef struct _WMGetAllResourcePlanResponse__isset { - _WMGetAllResourcePlanResponse__isset() : resourcePlans(false) {} - bool resourcePlans :1; -} _WMGetAllResourcePlanResponse__isset; - -class WMGetAllResourcePlanResponse : public virtual ::apache::thrift::TBase { - public: - - WMGetAllResourcePlanResponse(const WMGetAllResourcePlanResponse&); - WMGetAllResourcePlanResponse& operator=(const WMGetAllResourcePlanResponse&); - WMGetAllResourcePlanResponse() { - } - - virtual ~WMGetAllResourcePlanResponse() noexcept; - std::vector resourcePlans; - - _WMGetAllResourcePlanResponse__isset __isset; - - void __set_resourcePlans(const std::vector & val); - - bool operator == (const WMGetAllResourcePlanResponse & rhs) const - { - if (__isset.resourcePlans != rhs.__isset.resourcePlans) - return false; - else if (__isset.resourcePlans && !(resourcePlans == rhs.resourcePlans)) - return false; - return true; - } - bool operator != (const WMGetAllResourcePlanResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMGetAllResourcePlanResponse & ) 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(WMGetAllResourcePlanResponse &a, WMGetAllResourcePlanResponse &b); - -std::ostream& operator<<(std::ostream& out, const WMGetAllResourcePlanResponse& obj); - -typedef struct _WMAlterResourcePlanRequest__isset { - _WMAlterResourcePlanRequest__isset() : resourcePlanName(false), resourcePlan(false), isEnableAndActivate(false) {} - bool resourcePlanName :1; - bool resourcePlan :1; - bool isEnableAndActivate :1; -} _WMAlterResourcePlanRequest__isset; - -class WMAlterResourcePlanRequest : public virtual ::apache::thrift::TBase { - public: - - WMAlterResourcePlanRequest(const WMAlterResourcePlanRequest&); - WMAlterResourcePlanRequest& operator=(const WMAlterResourcePlanRequest&); - WMAlterResourcePlanRequest() : resourcePlanName(), isEnableAndActivate(0) { - } - - virtual ~WMAlterResourcePlanRequest() noexcept; - std::string resourcePlanName; - WMResourcePlan resourcePlan; - bool isEnableAndActivate; - - _WMAlterResourcePlanRequest__isset __isset; - - void __set_resourcePlanName(const std::string& val); - - void __set_resourcePlan(const WMResourcePlan& val); - - void __set_isEnableAndActivate(const bool val); - - bool operator == (const WMAlterResourcePlanRequest & rhs) const - { - if (__isset.resourcePlanName != rhs.__isset.resourcePlanName) - return false; - else if (__isset.resourcePlanName && !(resourcePlanName == rhs.resourcePlanName)) - return false; - if (__isset.resourcePlan != rhs.__isset.resourcePlan) - return false; - else if (__isset.resourcePlan && !(resourcePlan == rhs.resourcePlan)) - return false; - if (__isset.isEnableAndActivate != rhs.__isset.isEnableAndActivate) - return false; - else if (__isset.isEnableAndActivate && !(isEnableAndActivate == rhs.isEnableAndActivate)) - return false; - return true; - } - bool operator != (const WMAlterResourcePlanRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMAlterResourcePlanRequest & ) 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(WMAlterResourcePlanRequest &a, WMAlterResourcePlanRequest &b); - -std::ostream& operator<<(std::ostream& out, const WMAlterResourcePlanRequest& obj); - -typedef struct _WMAlterResourcePlanResponse__isset { - _WMAlterResourcePlanResponse__isset() : fullResourcePlan(false) {} - bool fullResourcePlan :1; -} _WMAlterResourcePlanResponse__isset; - -class WMAlterResourcePlanResponse : public virtual ::apache::thrift::TBase { - public: - - WMAlterResourcePlanResponse(const WMAlterResourcePlanResponse&); - WMAlterResourcePlanResponse& operator=(const WMAlterResourcePlanResponse&); - WMAlterResourcePlanResponse() { - } - - virtual ~WMAlterResourcePlanResponse() noexcept; - WMFullResourcePlan fullResourcePlan; - - _WMAlterResourcePlanResponse__isset __isset; - - void __set_fullResourcePlan(const WMFullResourcePlan& val); - - bool operator == (const WMAlterResourcePlanResponse & rhs) const - { - if (__isset.fullResourcePlan != rhs.__isset.fullResourcePlan) - return false; - else if (__isset.fullResourcePlan && !(fullResourcePlan == rhs.fullResourcePlan)) - return false; - return true; - } - bool operator != (const WMAlterResourcePlanResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMAlterResourcePlanResponse & ) 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(WMAlterResourcePlanResponse &a, WMAlterResourcePlanResponse &b); - -std::ostream& operator<<(std::ostream& out, const WMAlterResourcePlanResponse& obj); - -typedef struct _WMValidateResourcePlanRequest__isset { - _WMValidateResourcePlanRequest__isset() : resourcePlanName(false) {} - bool resourcePlanName :1; -} _WMValidateResourcePlanRequest__isset; - -class WMValidateResourcePlanRequest : public virtual ::apache::thrift::TBase { - public: - - WMValidateResourcePlanRequest(const WMValidateResourcePlanRequest&); - WMValidateResourcePlanRequest& operator=(const WMValidateResourcePlanRequest&); - WMValidateResourcePlanRequest() : resourcePlanName() { - } - - virtual ~WMValidateResourcePlanRequest() noexcept; - std::string resourcePlanName; - - _WMValidateResourcePlanRequest__isset __isset; - - void __set_resourcePlanName(const std::string& val); - - bool operator == (const WMValidateResourcePlanRequest & rhs) const - { - if (__isset.resourcePlanName != rhs.__isset.resourcePlanName) - return false; - else if (__isset.resourcePlanName && !(resourcePlanName == rhs.resourcePlanName)) - return false; - return true; - } - bool operator != (const WMValidateResourcePlanRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMValidateResourcePlanRequest & ) 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(WMValidateResourcePlanRequest &a, WMValidateResourcePlanRequest &b); - -std::ostream& operator<<(std::ostream& out, const WMValidateResourcePlanRequest& obj); - -typedef struct _WMValidateResourcePlanResponse__isset { - _WMValidateResourcePlanResponse__isset() : isValid(false) {} - bool isValid :1; -} _WMValidateResourcePlanResponse__isset; - -class WMValidateResourcePlanResponse : public virtual ::apache::thrift::TBase { - public: - - WMValidateResourcePlanResponse(const WMValidateResourcePlanResponse&); - WMValidateResourcePlanResponse& operator=(const WMValidateResourcePlanResponse&); - WMValidateResourcePlanResponse() : isValid(0) { - } - - virtual ~WMValidateResourcePlanResponse() noexcept; - bool isValid; - - _WMValidateResourcePlanResponse__isset __isset; - - void __set_isValid(const bool val); - - bool operator == (const WMValidateResourcePlanResponse & rhs) const - { - if (__isset.isValid != rhs.__isset.isValid) - return false; - else if (__isset.isValid && !(isValid == rhs.isValid)) - return false; - return true; - } - bool operator != (const WMValidateResourcePlanResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMValidateResourcePlanResponse & ) 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(WMValidateResourcePlanResponse &a, WMValidateResourcePlanResponse &b); - -std::ostream& operator<<(std::ostream& out, const WMValidateResourcePlanResponse& obj); - -typedef struct _WMDropResourcePlanRequest__isset { - _WMDropResourcePlanRequest__isset() : resourcePlanName(false) {} - bool resourcePlanName :1; -} _WMDropResourcePlanRequest__isset; - -class WMDropResourcePlanRequest : public virtual ::apache::thrift::TBase { - public: - - WMDropResourcePlanRequest(const WMDropResourcePlanRequest&); - WMDropResourcePlanRequest& operator=(const WMDropResourcePlanRequest&); - WMDropResourcePlanRequest() : resourcePlanName() { - } - - virtual ~WMDropResourcePlanRequest() noexcept; - std::string resourcePlanName; - - _WMDropResourcePlanRequest__isset __isset; - - void __set_resourcePlanName(const std::string& val); - - bool operator == (const WMDropResourcePlanRequest & rhs) const - { - if (__isset.resourcePlanName != rhs.__isset.resourcePlanName) - return false; - else if (__isset.resourcePlanName && !(resourcePlanName == rhs.resourcePlanName)) - return false; - return true; - } - bool operator != (const WMDropResourcePlanRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMDropResourcePlanRequest & ) 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(WMDropResourcePlanRequest &a, WMDropResourcePlanRequest &b); - -std::ostream& operator<<(std::ostream& out, const WMDropResourcePlanRequest& obj); - - -class WMDropResourcePlanResponse : public virtual ::apache::thrift::TBase { - public: - - WMDropResourcePlanResponse(const WMDropResourcePlanResponse&); - WMDropResourcePlanResponse& operator=(const WMDropResourcePlanResponse&); - WMDropResourcePlanResponse() { - } - - virtual ~WMDropResourcePlanResponse() noexcept; - - bool operator == (const WMDropResourcePlanResponse & /* rhs */) const - { - return true; - } - bool operator != (const WMDropResourcePlanResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMDropResourcePlanResponse & ) 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(WMDropResourcePlanResponse &a, WMDropResourcePlanResponse &b); - -std::ostream& operator<<(std::ostream& out, const WMDropResourcePlanResponse& obj); - -typedef struct _WMCreateTriggerRequest__isset { - _WMCreateTriggerRequest__isset() : trigger(false) {} - bool trigger :1; -} _WMCreateTriggerRequest__isset; - -class WMCreateTriggerRequest : public virtual ::apache::thrift::TBase { - public: - - WMCreateTriggerRequest(const WMCreateTriggerRequest&); - WMCreateTriggerRequest& operator=(const WMCreateTriggerRequest&); - WMCreateTriggerRequest() { - } - - virtual ~WMCreateTriggerRequest() noexcept; - WMTrigger trigger; - - _WMCreateTriggerRequest__isset __isset; - - void __set_trigger(const WMTrigger& val); - - bool operator == (const WMCreateTriggerRequest & rhs) const - { - if (__isset.trigger != rhs.__isset.trigger) - return false; - else if (__isset.trigger && !(trigger == rhs.trigger)) - return false; - return true; - } - bool operator != (const WMCreateTriggerRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMCreateTriggerRequest & ) 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(WMCreateTriggerRequest &a, WMCreateTriggerRequest &b); - -std::ostream& operator<<(std::ostream& out, const WMCreateTriggerRequest& obj); - - -class WMCreateTriggerResponse : public virtual ::apache::thrift::TBase { - public: - - WMCreateTriggerResponse(const WMCreateTriggerResponse&); - WMCreateTriggerResponse& operator=(const WMCreateTriggerResponse&); - WMCreateTriggerResponse() { - } - - virtual ~WMCreateTriggerResponse() noexcept; - - bool operator == (const WMCreateTriggerResponse & /* rhs */) const - { - return true; - } - bool operator != (const WMCreateTriggerResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMCreateTriggerResponse & ) 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(WMCreateTriggerResponse &a, WMCreateTriggerResponse &b); - -std::ostream& operator<<(std::ostream& out, const WMCreateTriggerResponse& obj); - -typedef struct _WMAlterTriggerRequest__isset { - _WMAlterTriggerRequest__isset() : trigger(false) {} - bool trigger :1; -} _WMAlterTriggerRequest__isset; - -class WMAlterTriggerRequest : public virtual ::apache::thrift::TBase { - public: - - WMAlterTriggerRequest(const WMAlterTriggerRequest&); - WMAlterTriggerRequest& operator=(const WMAlterTriggerRequest&); - WMAlterTriggerRequest() { - } - - virtual ~WMAlterTriggerRequest() noexcept; - WMTrigger trigger; - - _WMAlterTriggerRequest__isset __isset; - - void __set_trigger(const WMTrigger& val); - - bool operator == (const WMAlterTriggerRequest & rhs) const - { - if (__isset.trigger != rhs.__isset.trigger) - return false; - else if (__isset.trigger && !(trigger == rhs.trigger)) - return false; - return true; - } - bool operator != (const WMAlterTriggerRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMAlterTriggerRequest & ) 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(WMAlterTriggerRequest &a, WMAlterTriggerRequest &b); - -std::ostream& operator<<(std::ostream& out, const WMAlterTriggerRequest& obj); - - -class WMAlterTriggerResponse : public virtual ::apache::thrift::TBase { - public: - - WMAlterTriggerResponse(const WMAlterTriggerResponse&); - WMAlterTriggerResponse& operator=(const WMAlterTriggerResponse&); - WMAlterTriggerResponse() { - } - - virtual ~WMAlterTriggerResponse() noexcept; - - bool operator == (const WMAlterTriggerResponse & /* rhs */) const - { - return true; - } - bool operator != (const WMAlterTriggerResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMAlterTriggerResponse & ) 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(WMAlterTriggerResponse &a, WMAlterTriggerResponse &b); - -std::ostream& operator<<(std::ostream& out, const WMAlterTriggerResponse& obj); - -typedef struct _WMDropTriggerRequest__isset { - _WMDropTriggerRequest__isset() : resourcePlanName(false), triggerName(false) {} - bool resourcePlanName :1; - bool triggerName :1; -} _WMDropTriggerRequest__isset; - -class WMDropTriggerRequest : public virtual ::apache::thrift::TBase { - public: - - WMDropTriggerRequest(const WMDropTriggerRequest&); - WMDropTriggerRequest& operator=(const WMDropTriggerRequest&); - WMDropTriggerRequest() : resourcePlanName(), triggerName() { - } - - virtual ~WMDropTriggerRequest() noexcept; - std::string resourcePlanName; - std::string triggerName; - - _WMDropTriggerRequest__isset __isset; - - void __set_resourcePlanName(const std::string& val); - - void __set_triggerName(const std::string& val); - - bool operator == (const WMDropTriggerRequest & rhs) const - { - if (__isset.resourcePlanName != rhs.__isset.resourcePlanName) - return false; - else if (__isset.resourcePlanName && !(resourcePlanName == rhs.resourcePlanName)) - return false; - if (__isset.triggerName != rhs.__isset.triggerName) - return false; - else if (__isset.triggerName && !(triggerName == rhs.triggerName)) - return false; - return true; - } - bool operator != (const WMDropTriggerRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMDropTriggerRequest & ) 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(WMDropTriggerRequest &a, WMDropTriggerRequest &b); - -std::ostream& operator<<(std::ostream& out, const WMDropTriggerRequest& obj); - - -class WMDropTriggerResponse : public virtual ::apache::thrift::TBase { - public: - - WMDropTriggerResponse(const WMDropTriggerResponse&); - WMDropTriggerResponse& operator=(const WMDropTriggerResponse&); - WMDropTriggerResponse() { - } - - virtual ~WMDropTriggerResponse() noexcept; - - bool operator == (const WMDropTriggerResponse & /* rhs */) const - { - return true; - } - bool operator != (const WMDropTriggerResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMDropTriggerResponse & ) 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(WMDropTriggerResponse &a, WMDropTriggerResponse &b); - -std::ostream& operator<<(std::ostream& out, const WMDropTriggerResponse& obj); - -typedef struct _WMGetTriggersForResourePlanRequest__isset { - _WMGetTriggersForResourePlanRequest__isset() : resourcePlanName(false) {} - bool resourcePlanName :1; -} _WMGetTriggersForResourePlanRequest__isset; - -class WMGetTriggersForResourePlanRequest : public virtual ::apache::thrift::TBase { - public: - - WMGetTriggersForResourePlanRequest(const WMGetTriggersForResourePlanRequest&); - WMGetTriggersForResourePlanRequest& operator=(const WMGetTriggersForResourePlanRequest&); - WMGetTriggersForResourePlanRequest() : resourcePlanName() { - } - - virtual ~WMGetTriggersForResourePlanRequest() noexcept; - std::string resourcePlanName; - - _WMGetTriggersForResourePlanRequest__isset __isset; - - void __set_resourcePlanName(const std::string& val); - - bool operator == (const WMGetTriggersForResourePlanRequest & rhs) const - { - if (__isset.resourcePlanName != rhs.__isset.resourcePlanName) - return false; - else if (__isset.resourcePlanName && !(resourcePlanName == rhs.resourcePlanName)) - return false; - return true; - } - bool operator != (const WMGetTriggersForResourePlanRequest &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMGetTriggersForResourePlanRequest & ) 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(WMGetTriggersForResourePlanRequest &a, WMGetTriggersForResourePlanRequest &b); - -std::ostream& operator<<(std::ostream& out, const WMGetTriggersForResourePlanRequest& obj); - -typedef struct _WMGetTriggersForResourePlanResponse__isset { - _WMGetTriggersForResourePlanResponse__isset() : triggers(false) {} - bool triggers :1; -} _WMGetTriggersForResourePlanResponse__isset; - -class WMGetTriggersForResourePlanResponse : public virtual ::apache::thrift::TBase { - public: - - WMGetTriggersForResourePlanResponse(const WMGetTriggersForResourePlanResponse&); - WMGetTriggersForResourePlanResponse& operator=(const WMGetTriggersForResourePlanResponse&); - WMGetTriggersForResourePlanResponse() { - } - - virtual ~WMGetTriggersForResourePlanResponse() noexcept; - std::vector triggers; - - _WMGetTriggersForResourePlanResponse__isset __isset; - - void __set_triggers(const std::vector & val); - - bool operator == (const WMGetTriggersForResourePlanResponse & rhs) const - { - if (__isset.triggers != rhs.__isset.triggers) - return false; - else if (__isset.triggers && !(triggers == rhs.triggers)) - return false; - return true; - } - bool operator != (const WMGetTriggersForResourePlanResponse &rhs) const { - return !(*this == rhs); - } - - bool operator < (const WMGetTriggersForResourePlanResponse & ) 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(WMGetTriggersForResourePlanResponse &a, WMGetTriggersForResourePlanResponse &b); - -std::ostream& operator<<(std::ostream& out, const WMGetTriggersForResourePlanResponse& obj); - -typedef struct _MetaException__isset { - _MetaException__isset() : message(false) {} - bool message :1; -} _MetaException__isset; - -class MetaException : public ::apache::thrift::TException { - public: - - MetaException(const MetaException&); - MetaException& operator=(const MetaException&); - MetaException() : message() { - } - - virtual ~MetaException() noexcept; - std::string message; - - _MetaException__isset __isset; - - void __set_message(const std::string& val); - - bool operator == (const MetaException & rhs) const - { - if (!(message == rhs.message)) - return false; - return true; - } - bool operator != (const MetaException &rhs) const { - return !(*this == rhs); - } - - bool operator < (const MetaException & ) 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; - mutable std::string thriftTExceptionMessageHolder_; - const char* what() const noexcept; -}; - -void swap(MetaException &a, MetaException &b); - -std::ostream& operator<<(std::ostream& out, const MetaException& obj); - -typedef struct _UnknownTableException__isset { - _UnknownTableException__isset() : message(false) {} - bool message :1; -} _UnknownTableException__isset; - -class UnknownTableException : public ::apache::thrift::TException { - public: - - UnknownTableException(const UnknownTableException&); - UnknownTableException& operator=(const UnknownTableException&); - UnknownTableException() : message() { - } - - virtual ~UnknownTableException() noexcept; - std::string message; - - _UnknownTableException__isset __isset; - - void __set_message(const std::string& val); - - bool operator == (const UnknownTableException & rhs) const - { - if (!(message == rhs.message)) - return false; - return true; - } - bool operator != (const UnknownTableException &rhs) const { - return !(*this == rhs); - } - - bool operator < (const UnknownTableException & ) 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; - mutable std::string thriftTExceptionMessageHolder_; - const char* what() const noexcept; -}; - -void swap(UnknownTableException &a, UnknownTableException &b); - -std::ostream& operator<<(std::ostream& out, const UnknownTableException& obj); - -typedef struct _UnknownDBException__isset { - _UnknownDBException__isset() : message(false) {} - bool message :1; -} _UnknownDBException__isset; - -class UnknownDBException : public ::apache::thrift::TException { - public: - - UnknownDBException(const UnknownDBException&); - UnknownDBException& operator=(const UnknownDBException&); - UnknownDBException() : message() { - } - - virtual ~UnknownDBException() noexcept; - std::string message; - - _UnknownDBException__isset __isset; - - void __set_message(const std::string& val); - - bool operator == (const UnknownDBException & rhs) const - { - if (!(message == rhs.message)) - return false; - return true; - } - bool operator != (const UnknownDBException &rhs) const { - return !(*this == rhs); - } - - bool operator < (const UnknownDBException & ) 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; - mutable std::string thriftTExceptionMessageHolder_; - const char* what() const noexcept; -}; - -void swap(UnknownDBException &a, UnknownDBException &b); - -std::ostream& operator<<(std::ostream& out, const UnknownDBException& obj); - -typedef struct _AlreadyExistsException__isset { - _AlreadyExistsException__isset() : message(false) {} - bool message :1; -} _AlreadyExistsException__isset; - -class AlreadyExistsException : public ::apache::thrift::TException { - public: - - AlreadyExistsException(const AlreadyExistsException&); - AlreadyExistsException& operator=(const AlreadyExistsException&); - AlreadyExistsException() : message() { - } - - virtual ~AlreadyExistsException() noexcept; - std::string message; - - _AlreadyExistsException__isset __isset; - - void __set_message(const std::string& val); - - bool operator == (const AlreadyExistsException & rhs) const - { - if (!(message == rhs.message)) - return false; - return true; - } - bool operator != (const AlreadyExistsException &rhs) const { - return !(*this == rhs); - } - - bool operator < (const AlreadyExistsException & ) 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; - mutable std::string thriftTExceptionMessageHolder_; - const char* what() const noexcept; -}; - -void swap(AlreadyExistsException &a, AlreadyExistsException &b); - -std::ostream& operator<<(std::ostream& out, const AlreadyExistsException& obj); - -typedef struct _InvalidPartitionException__isset { - _InvalidPartitionException__isset() : message(false) {} - bool message :1; -} _InvalidPartitionException__isset; - -class InvalidPartitionException : public ::apache::thrift::TException { - public: - - InvalidPartitionException(const InvalidPartitionException&); - InvalidPartitionException& operator=(const InvalidPartitionException&); - InvalidPartitionException() : message() { - } - - virtual ~InvalidPartitionException() noexcept; - std::string message; - - _InvalidPartitionException__isset __isset; - - void __set_message(const std::string& val); - - bool operator == (const InvalidPartitionException & rhs) const - { - if (!(message == rhs.message)) - return false; - return true; - } - bool operator != (const InvalidPartitionException &rhs) const { - return !(*this == rhs); - } - - bool operator < (const InvalidPartitionException & ) 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; - mutable std::string thriftTExceptionMessageHolder_; - const char* what() const noexcept; -}; - -void swap(InvalidPartitionException &a, InvalidPartitionException &b); - -std::ostream& operator<<(std::ostream& out, const InvalidPartitionException& obj); - -typedef struct _UnknownPartitionException__isset { - _UnknownPartitionException__isset() : message(false) {} - bool message :1; -} _UnknownPartitionException__isset; - -class UnknownPartitionException : public ::apache::thrift::TException { - public: - - UnknownPartitionException(const UnknownPartitionException&); - UnknownPartitionException& operator=(const UnknownPartitionException&); - UnknownPartitionException() : message() { - } - - virtual ~UnknownPartitionException() noexcept; - std::string message; - - _UnknownPartitionException__isset __isset; - - void __set_message(const std::string& val); - - bool operator == (const UnknownPartitionException & rhs) const - { - if (!(message == rhs.message)) - return false; - return true; - } - bool operator != (const UnknownPartitionException &rhs) const { - return !(*this == rhs); - } - - bool operator < (const UnknownPartitionException & ) 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; - mutable std::string thriftTExceptionMessageHolder_; - const char* what() const noexcept; -}; - -void swap(UnknownPartitionException &a, UnknownPartitionException &b); - -std::ostream& operator<<(std::ostream& out, const UnknownPartitionException& obj); - -typedef struct _InvalidObjectException__isset { - _InvalidObjectException__isset() : message(false) {} - bool message :1; -} _InvalidObjectException__isset; - -class InvalidObjectException : public ::apache::thrift::TException { - public: - - InvalidObjectException(const InvalidObjectException&); - InvalidObjectException& operator=(const InvalidObjectException&); - InvalidObjectException() : message() { - } - - virtual ~InvalidObjectException() noexcept; - std::string message; - - _InvalidObjectException__isset __isset; - - void __set_message(const std::string& val); - - bool operator == (const InvalidObjectException & rhs) const - { - if (!(message == rhs.message)) - return false; - return true; - } - bool operator != (const InvalidObjectException &rhs) const { - return !(*this == rhs); - } - - bool operator < (const InvalidObjectException & ) 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; - mutable std::string thriftTExceptionMessageHolder_; - const char* what() const noexcept; -}; - -void swap(InvalidObjectException &a, InvalidObjectException &b); - -std::ostream& operator<<(std::ostream& out, const InvalidObjectException& obj); - -typedef struct _NoSuchObjectException__isset { - _NoSuchObjectException__isset() : message(false) {} - bool message :1; -} _NoSuchObjectException__isset; - -class NoSuchObjectException : public ::apache::thrift::TException { - public: - - NoSuchObjectException(const NoSuchObjectException&); - NoSuchObjectException& operator=(const NoSuchObjectException&); - NoSuchObjectException() : message() { - } - - virtual ~NoSuchObjectException() noexcept; - std::string message; - - _NoSuchObjectException__isset __isset; - - void __set_message(const std::string& val); - - bool operator == (const NoSuchObjectException & rhs) const - { - if (!(message == rhs.message)) - return false; - return true; - } - bool operator != (const NoSuchObjectException &rhs) const { - return !(*this == rhs); - } - - bool operator < (const NoSuchObjectException & ) 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; - mutable std::string thriftTExceptionMessageHolder_; - const char* what() const noexcept; -}; - -void swap(NoSuchObjectException &a, NoSuchObjectException &b); - -std::ostream& operator<<(std::ostream& out, const NoSuchObjectException& obj); - -typedef struct _IndexAlreadyExistsException__isset { - _IndexAlreadyExistsException__isset() : message(false) {} - bool message :1; -} _IndexAlreadyExistsException__isset; - -class IndexAlreadyExistsException : public ::apache::thrift::TException { - public: - - IndexAlreadyExistsException(const IndexAlreadyExistsException&); - IndexAlreadyExistsException& operator=(const IndexAlreadyExistsException&); - IndexAlreadyExistsException() : message() { - } - - virtual ~IndexAlreadyExistsException() noexcept; - std::string message; - - _IndexAlreadyExistsException__isset __isset; - - void __set_message(const std::string& val); - - bool operator == (const IndexAlreadyExistsException & rhs) const - { - if (!(message == rhs.message)) - return false; - return true; - } - bool operator != (const IndexAlreadyExistsException &rhs) const { - return !(*this == rhs); - } - - bool operator < (const IndexAlreadyExistsException & ) 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; - mutable std::string thriftTExceptionMessageHolder_; - const char* what() const noexcept; -}; - -void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b); - -std::ostream& operator<<(std::ostream& out, const IndexAlreadyExistsException& obj); - -typedef struct _InvalidOperationException__isset { - _InvalidOperationException__isset() : message(false) {} - bool message :1; -} _InvalidOperationException__isset; - -class InvalidOperationException : public ::apache::thrift::TException { - public: - - InvalidOperationException(const InvalidOperationException&); - InvalidOperationException& operator=(const InvalidOperationException&); - InvalidOperationException() : message() { - } - - virtual ~InvalidOperationException() noexcept; - std::string message; - - _InvalidOperationException__isset __isset; - - void __set_message(const std::string& val); - - bool operator == (const InvalidOperationException & rhs) const - { - if (!(message == rhs.message)) - return false; - return true; - } - bool operator != (const InvalidOperationException &rhs) const { - return !(*this == rhs); - } - - bool operator < (const InvalidOperationException & ) 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; - mutable std::string thriftTExceptionMessageHolder_; - const char* what() const noexcept; -}; - -void swap(InvalidOperationException &a, InvalidOperationException &b); - -std::ostream& operator<<(std::ostream& out, const InvalidOperationException& obj); - -typedef struct _ConfigValSecurityException__isset { - _ConfigValSecurityException__isset() : message(false) {} - bool message :1; -} _ConfigValSecurityException__isset; - -class ConfigValSecurityException : public ::apache::thrift::TException { - public: - - ConfigValSecurityException(const ConfigValSecurityException&); - ConfigValSecurityException& operator=(const ConfigValSecurityException&); - ConfigValSecurityException() : message() { - } - - virtual ~ConfigValSecurityException() noexcept; - std::string message; - - _ConfigValSecurityException__isset __isset; - - void __set_message(const std::string& val); - - bool operator == (const ConfigValSecurityException & rhs) const - { - if (!(message == rhs.message)) - return false; - return true; - } - bool operator != (const ConfigValSecurityException &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ConfigValSecurityException & ) 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; - mutable std::string thriftTExceptionMessageHolder_; - const char* what() const noexcept; -}; - -void swap(ConfigValSecurityException &a, ConfigValSecurityException &b); - -std::ostream& operator<<(std::ostream& out, const ConfigValSecurityException& obj); - -typedef struct _InvalidInputException__isset { - _InvalidInputException__isset() : message(false) {} - bool message :1; -} _InvalidInputException__isset; - -class InvalidInputException : public ::apache::thrift::TException { - public: - - InvalidInputException(const InvalidInputException&); - InvalidInputException& operator=(const InvalidInputException&); - InvalidInputException() : message() { - } - - virtual ~InvalidInputException() noexcept; - std::string message; - - _InvalidInputException__isset __isset; - - void __set_message(const std::string& val); - - bool operator == (const InvalidInputException & rhs) const - { - if (!(message == rhs.message)) - return false; - return true; - } - bool operator != (const InvalidInputException &rhs) const { - return !(*this == rhs); - } - - bool operator < (const InvalidInputException & ) 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; - mutable std::string thriftTExceptionMessageHolder_; - const char* what() const noexcept; -}; - -void swap(InvalidInputException &a, InvalidInputException &b); - -std::ostream& operator<<(std::ostream& out, const InvalidInputException& obj); - -typedef struct _NoSuchTxnException__isset { - _NoSuchTxnException__isset() : message(false) {} - bool message :1; -} _NoSuchTxnException__isset; - -class NoSuchTxnException : public ::apache::thrift::TException { - public: - - NoSuchTxnException(const NoSuchTxnException&); - NoSuchTxnException& operator=(const NoSuchTxnException&); - NoSuchTxnException() : message() { - } - - virtual ~NoSuchTxnException() noexcept; - std::string message; - - _NoSuchTxnException__isset __isset; - - void __set_message(const std::string& val); - - bool operator == (const NoSuchTxnException & rhs) const - { - if (!(message == rhs.message)) - return false; - return true; - } - bool operator != (const NoSuchTxnException &rhs) const { - return !(*this == rhs); - } - - bool operator < (const NoSuchTxnException & ) 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; - mutable std::string thriftTExceptionMessageHolder_; - const char* what() const noexcept; -}; - -void swap(NoSuchTxnException &a, NoSuchTxnException &b); - -std::ostream& operator<<(std::ostream& out, const NoSuchTxnException& obj); - -typedef struct _TxnAbortedException__isset { - _TxnAbortedException__isset() : message(false) {} - bool message :1; -} _TxnAbortedException__isset; - -class TxnAbortedException : public ::apache::thrift::TException { - public: - - TxnAbortedException(const TxnAbortedException&); - TxnAbortedException& operator=(const TxnAbortedException&); - TxnAbortedException() : message() { - } - - virtual ~TxnAbortedException() noexcept; - std::string message; - - _TxnAbortedException__isset __isset; - - void __set_message(const std::string& val); - - bool operator == (const TxnAbortedException & rhs) const - { - if (!(message == rhs.message)) - return false; - return true; - } - bool operator != (const TxnAbortedException &rhs) const { - return !(*this == rhs); - } - - bool operator < (const TxnAbortedException & ) 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; - mutable std::string thriftTExceptionMessageHolder_; - const char* what() const noexcept; -}; - -void swap(TxnAbortedException &a, TxnAbortedException &b); - -std::ostream& operator<<(std::ostream& out, const TxnAbortedException& obj); - -typedef struct _TxnOpenException__isset { - _TxnOpenException__isset() : message(false) {} - bool message :1; -} _TxnOpenException__isset; - -class TxnOpenException : public ::apache::thrift::TException { - public: - - TxnOpenException(const TxnOpenException&); - TxnOpenException& operator=(const TxnOpenException&); - TxnOpenException() : message() { - } - - virtual ~TxnOpenException() noexcept; - std::string message; - - _TxnOpenException__isset __isset; - - void __set_message(const std::string& val); - - bool operator == (const TxnOpenException & rhs) const - { - if (!(message == rhs.message)) - return false; - return true; - } - bool operator != (const TxnOpenException &rhs) const { - return !(*this == rhs); - } - - bool operator < (const TxnOpenException & ) 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; - mutable std::string thriftTExceptionMessageHolder_; - const char* what() const noexcept; -}; - -void swap(TxnOpenException &a, TxnOpenException &b); - -std::ostream& operator<<(std::ostream& out, const TxnOpenException& obj); - -typedef struct _NoSuchLockException__isset { - _NoSuchLockException__isset() : message(false) {} - bool message :1; -} _NoSuchLockException__isset; - -class NoSuchLockException : public ::apache::thrift::TException { - public: - - NoSuchLockException(const NoSuchLockException&); - NoSuchLockException& operator=(const NoSuchLockException&); - NoSuchLockException() : message() { - } - - virtual ~NoSuchLockException() noexcept; - std::string message; - - _NoSuchLockException__isset __isset; - - void __set_message(const std::string& val); - - bool operator == (const NoSuchLockException & rhs) const - { - if (!(message == rhs.message)) - return false; - return true; - } - bool operator != (const NoSuchLockException &rhs) const { - return !(*this == rhs); - } - - bool operator < (const NoSuchLockException & ) 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; - mutable std::string thriftTExceptionMessageHolder_; - const char* what() const noexcept; -}; - -void swap(NoSuchLockException &a, NoSuchLockException &b); - -std::ostream& operator<<(std::ostream& out, const NoSuchLockException& obj); - -}}} // namespace - -#endif From c56fbe0f444d3a4a81dbabe307c4dc2a0b3dbc66 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 16 Nov 2021 11:37:26 +0800 Subject: [PATCH 0061/1260] remove submodule --- .gitmodules | 3 --- 1 file changed, 3 deletions(-) diff --git a/.gitmodules b/.gitmodules index cb906d45719..8ad81b5094f 100644 --- a/.gitmodules +++ b/.gitmodules @@ -256,6 +256,3 @@ [submodule "contrib/sysroot"] path = contrib/sysroot url = https://github.com/ClickHouse-Extras/sysroot.git -[submodule "contrib/hive-metastore"] - path = contrib/hive-metastore - url = https://github.com/ClickHouse-Extras/hive-metastore.git From f33ec0fd47c8408d04e821c69b37d61fe5ab4946 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 16 Nov 2021 11:38:24 +0800 Subject: [PATCH 0062/1260] add submodule hive-metasotre --- .gitmodules | 3 +++ contrib/hive-metastore | 1 + 2 files changed, 4 insertions(+) create mode 160000 contrib/hive-metastore diff --git a/.gitmodules b/.gitmodules index 8ad81b5094f..ac5e50902a5 100644 --- a/.gitmodules +++ b/.gitmodules @@ -256,3 +256,6 @@ [submodule "contrib/sysroot"] path = contrib/sysroot url = https://github.com/ClickHouse-Extras/sysroot.git +[submodule "contrib/hive-metastore"] + path = contrib/hive-metastore + url = https://github.com/ClickHouse-Extras/hive-metastore diff --git a/contrib/hive-metastore b/contrib/hive-metastore new file mode 160000 index 00000000000..809a77d435c --- /dev/null +++ b/contrib/hive-metastore @@ -0,0 +1 @@ +Subproject commit 809a77d435ce218d9b000733f19489c606fc567b From 9902ccefc59af7eab402aa3b3238fe1b0d05b795 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 18 Nov 2021 16:17:49 +0800 Subject: [PATCH 0063/1260] Add integration tests for hive query --- .../test/integration/hive_server/Dockerfile | 47 ++ .../hive_server/core-site.xml.template | 14 + .../integration/hive_server/demo_data.txt | 6 + .../integration/hive_server/hadoop-env.sh | 422 ++++++++++++++++++ .../integration/hive_server/hdfs-site.xml | 6 + .../integration/hive_server/hive-site.xml | 35 ++ .../integration/hive_server/mapred-site.xml | 6 + .../hive_server/prepare_hive_data.sh | 6 + docker/test/integration/hive_server/start.sh | 12 + .../integration/hive_server/yarn-site.xml | 32 ++ .../runner/compose/docker_compose_hive.yml | 7 + tests/integration/helpers/cluster.py | 33 +- tests/integration/test_hive_query/__init__.py | 0 .../test_hive_query/configs/config.xml | 22 + .../test_hive_query/configs/hdfs-site.xml | 6 + tests/integration/test_hive_query/test.py | 91 ++++ 16 files changed, 740 insertions(+), 5 deletions(-) create mode 100644 docker/test/integration/hive_server/Dockerfile create mode 100644 docker/test/integration/hive_server/core-site.xml.template create mode 100644 docker/test/integration/hive_server/demo_data.txt create mode 100644 docker/test/integration/hive_server/hadoop-env.sh create mode 100644 docker/test/integration/hive_server/hdfs-site.xml create mode 100644 docker/test/integration/hive_server/hive-site.xml create mode 100644 docker/test/integration/hive_server/mapred-site.xml create mode 100755 docker/test/integration/hive_server/prepare_hive_data.sh create mode 100755 docker/test/integration/hive_server/start.sh create mode 100644 docker/test/integration/hive_server/yarn-site.xml create mode 100644 docker/test/integration/runner/compose/docker_compose_hive.yml create mode 100644 tests/integration/test_hive_query/__init__.py create mode 100644 tests/integration/test_hive_query/configs/config.xml create mode 100644 tests/integration/test_hive_query/configs/hdfs-site.xml create mode 100644 tests/integration/test_hive_query/test.py diff --git a/docker/test/integration/hive_server/Dockerfile b/docker/test/integration/hive_server/Dockerfile new file mode 100644 index 00000000000..fa6e4bf6313 --- /dev/null +++ b/docker/test/integration/hive_server/Dockerfile @@ -0,0 +1,47 @@ +FROM ubuntu:20.04 +MAINTAINER lgbo-ustc + +RUN apt-get update +RUN apt-get install -y wget openjdk-8-jre + +RUN wget https://archive.apache.org/dist/hadoop/common/hadoop-3.1.0/hadoop-3.1.0.tar.gz && \ + tar -xf hadoop-3.1.0.tar.gz && rm -rf hadoop-3.1.0.tar.gz +RUN wget https://dlcdn.apache.org/hive/hive-2.3.9/apache-hive-2.3.9-bin.tar.gz && \ + tar -xf apache-hive-2.3.9-bin.tar.gz && rm -rf apache-hive-2.3.9-bin.tar.gz +RUN apt install -y vim + +RUN apt install -y openssh-server openssh-client + +RUN apt install -y mysql-server + +RUN mkdir -p /root/.ssh && \ + ssh-keygen -t rsa -b 2048 -P '' -f /root/.ssh/id_rsa && \ + cat /root/.ssh/id_rsa.pub > /root/.ssh/authorized_keys && \ + cp /root/.ssh/id_rsa /etc/ssh/ssh_host_rsa_key && \ + cp /root/.ssh/id_rsa.pub /etc/ssh/ssh_host_rsa_key.pub + +RUN wget https://dev.mysql.com/get/Downloads/Connector-J/mysql-connector-java-8.0.27.tar.gz &&\ + tar -xf mysql-connector-java-8.0.27.tar.gz && \ + mv mysql-connector-java-8.0.27/mysql-connector-java-8.0.27.jar /apache-hive-2.3.9-bin/lib/ && \ + rm -rf mysql-connector-java-8.0.27.tar.gz mysql-connector-java-8.0.27 + +RUN apt install -y iputils-ping net-tools + +ENV JAVA_HOME=/usr +ENV HADOOP_HOME=/hadoop-3.1.0 +ENV HDFS_NAMENODE_USER=root +ENV HDFS_DATANODE_USER=root HDFS_SECONDARYNAMENODE_USER=root YARN_RESOURCEMANAGER_USER=root YARN_NODEMANAGER_USER=root HDFS_DATANODE_SECURE_USER=hdfs +COPY hdfs-site.xml /hadoop-3.1.0/etc/hadoop +COPY mapred-site.xml /hadoop-3.1.0/etc/hadoop +COPY yarn-site.xml /hadoop-3.1.0/etc/hadoop +COPY hadoop-env.sh /hadoop-3.1.0/etc/hadoop/ +#COPY core-site.xml /hadoop-3.1.0/etc/hadoop +COPY core-site.xml.template /hadoop-3.1.0/etc/hadoop +COPY hive-site.xml /apache-hive-2.3.9-bin/conf +COPY prepare_hive_data.sh / +COPY demo_data.txt / + +ENV PATH=/apache-hive-2.3.9-bin/bin:/hadoop-3.1.0/bin:/hadoop-3.1.0/sbin:$PATH + +COPY start.sh / + diff --git a/docker/test/integration/hive_server/core-site.xml.template b/docker/test/integration/hive_server/core-site.xml.template new file mode 100644 index 00000000000..232338e0443 --- /dev/null +++ b/docker/test/integration/hive_server/core-site.xml.template @@ -0,0 +1,14 @@ + + + fs.defaultFS + hdfs://HOSTNAME:9000 + + + hadoop.proxyuser.root.hosts + * + + + hadoop.proxyuser.root.groups + * + + diff --git a/docker/test/integration/hive_server/demo_data.txt b/docker/test/integration/hive_server/demo_data.txt new file mode 100644 index 00000000000..dbe7c4bd990 --- /dev/null +++ b/docker/test/integration/hive_server/demo_data.txt @@ -0,0 +1,6 @@ +abc,1,2021-11-16 +abd,15,2021-11-05 +aaa,22,2021-11-16 +dda,0,2021-11-01 +dfb,11,2021-11-05 +jhn,89,2021-11-11 diff --git a/docker/test/integration/hive_server/hadoop-env.sh b/docker/test/integration/hive_server/hadoop-env.sh new file mode 100644 index 00000000000..84fcde490c3 --- /dev/null +++ b/docker/test/integration/hive_server/hadoop-env.sh @@ -0,0 +1,422 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Set Hadoop-specific environment variables here. + +## +## THIS FILE ACTS AS THE MASTER FILE FOR ALL HADOOP PROJECTS. +## SETTINGS HERE WILL BE READ BY ALL HADOOP COMMANDS. THEREFORE, +## ONE CAN USE THIS FILE TO SET YARN, HDFS, AND MAPREDUCE +## CONFIGURATION OPTIONS INSTEAD OF xxx-env.sh. +## +## Precedence rules: +## +## {yarn-env.sh|hdfs-env.sh} > hadoop-env.sh > hard-coded defaults +## +## {YARN_xyz|HDFS_xyz} > HADOOP_xyz > hard-coded defaults +## + +# Many of the options here are built from the perspective that users +# may want to provide OVERWRITING values on the command line. +# For example: +# +JAVA_HOME=/usr/ +# +# Therefore, the vast majority (BUT NOT ALL!) of these defaults +# are configured for substitution and not append. If append +# is preferable, modify this file accordingly. + +### +# Generic settings for HADOOP +### + +# Technically, the only required environment variable is JAVA_HOME. +# All others are optional. However, the defaults are probably not +# preferred. Many sites configure these options outside of Hadoop, +# such as in /etc/profile.d + +# The java implementation to use. By default, this environment +# variable is REQUIRED on ALL platforms except OS X! +# export JAVA_HOME= + +# Location of Hadoop. By default, Hadoop will attempt to determine +# this location based upon its execution path. +# export HADOOP_HOME= + +# Location of Hadoop's configuration information. i.e., where this +# file is living. If this is not defined, Hadoop will attempt to +# locate it based upon its execution path. +# +# NOTE: It is recommend that this variable not be set here but in +# /etc/profile.d or equivalent. Some options (such as +# --config) may react strangely otherwise. +# +# export HADOOP_CONF_DIR=${HADOOP_HOME}/etc/hadoop + +# The maximum amount of heap to use (Java -Xmx). If no unit +# is provided, it will be converted to MB. Daemons will +# prefer any Xmx setting in their respective _OPT variable. +# There is no default; the JVM will autoscale based upon machine +# memory size. +# export HADOOP_HEAPSIZE_MAX= + +# The minimum amount of heap to use (Java -Xms). If no unit +# is provided, it will be converted to MB. Daemons will +# prefer any Xms setting in their respective _OPT variable. +# There is no default; the JVM will autoscale based upon machine +# memory size. +# export HADOOP_HEAPSIZE_MIN= + +# Enable extra debugging of Hadoop's JAAS binding, used to set up +# Kerberos security. +# export HADOOP_JAAS_DEBUG=true + +# Extra Java runtime options for all Hadoop commands. We don't support +# IPv6 yet/still, so by default the preference is set to IPv4. +# export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true" +# For Kerberos debugging, an extended option set logs more invormation +# export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true -Dsun.security.krb5.debug=true -Dsun.security.spnego.debug" + +# Some parts of the shell code may do special things dependent upon +# the operating system. We have to set this here. See the next +# section as to why.... +export HADOOP_OS_TYPE=${HADOOP_OS_TYPE:-$(uname -s)} + + +# Under certain conditions, Java on OS X will throw SCDynamicStore errors +# in the system logs. +# See HADOOP-8719 for more information. If one needs Kerberos +# support on OS X, one will want to change/remove this extra bit. +case ${HADOOP_OS_TYPE} in + Darwin*) + export HADOOP_OPTS="${HADOOP_OPTS} -Djava.security.krb5.realm= " + export HADOOP_OPTS="${HADOOP_OPTS} -Djava.security.krb5.kdc= " + export HADOOP_OPTS="${HADOOP_OPTS} -Djava.security.krb5.conf= " + ;; +esac + +# Extra Java runtime options for some Hadoop commands +# and clients (i.e., hdfs dfs -blah). These get appended to HADOOP_OPTS for +# such commands. In most cases, # this should be left empty and +# let users supply it on the command line. +# export HADOOP_CLIENT_OPTS="" + +# +# A note about classpaths. +# +# By default, Apache Hadoop overrides Java's CLASSPATH +# environment variable. It is configured such +# that it sarts out blank with new entries added after passing +# a series of checks (file/dir exists, not already listed aka +# de-deduplication). During de-depulication, wildcards and/or +# directories are *NOT* expanded to keep it simple. Therefore, +# if the computed classpath has two specific mentions of +# awesome-methods-1.0.jar, only the first one added will be seen. +# If two directories are in the classpath that both contain +# awesome-methods-1.0.jar, then Java will pick up both versions. + +# An additional, custom CLASSPATH. Site-wide configs should be +# handled via the shellprofile functionality, utilizing the +# hadoop_add_classpath function for greater control and much +# harder for apps/end-users to accidentally override. +# Similarly, end users should utilize ${HOME}/.hadooprc . +# This variable should ideally only be used as a short-cut, +# interactive way for temporary additions on the command line. +# export HADOOP_CLASSPATH="/some/cool/path/on/your/machine" + +# Should HADOOP_CLASSPATH be first in the official CLASSPATH? +# export HADOOP_USER_CLASSPATH_FIRST="yes" + +# If HADOOP_USE_CLIENT_CLASSLOADER is set, the classpath along +# with the main jar are handled by a separate isolated +# client classloader when 'hadoop jar', 'yarn jar', or 'mapred job' +# is utilized. If it is set, HADOOP_CLASSPATH and +# HADOOP_USER_CLASSPATH_FIRST are ignored. +# export HADOOP_USE_CLIENT_CLASSLOADER=true + +# HADOOP_CLIENT_CLASSLOADER_SYSTEM_CLASSES overrides the default definition of +# system classes for the client classloader when HADOOP_USE_CLIENT_CLASSLOADER +# is enabled. Names ending in '.' (period) are treated as package names, and +# names starting with a '-' are treated as negative matches. For example, +# export HADOOP_CLIENT_CLASSLOADER_SYSTEM_CLASSES="-org.apache.hadoop.UserClass,java.,javax.,org.apache.hadoop." + +# Enable optional, bundled Hadoop features +# This is a comma delimited list. It may NOT be overridden via .hadooprc +# Entries may be added/removed as needed. +# export HADOOP_OPTIONAL_TOOLS="hadoop-openstack,hadoop-aliyun,hadoop-azure,hadoop-azure-datalake,hadoop-aws,hadoop-kafka" + +### +# Options for remote shell connectivity +### + +# There are some optional components of hadoop that allow for +# command and control of remote hosts. For example, +# start-dfs.sh will attempt to bring up all NNs, DNS, etc. + +# Options to pass to SSH when one of the "log into a host and +# start/stop daemons" scripts is executed +# export HADOOP_SSH_OPTS="-o BatchMode=yes -o StrictHostKeyChecking=no -o ConnectTimeout=10s" + +# The built-in ssh handler will limit itself to 10 simultaneous connections. +# For pdsh users, this sets the fanout size ( -f ) +# Change this to increase/decrease as necessary. +# export HADOOP_SSH_PARALLEL=10 + +# Filename which contains all of the hosts for any remote execution +# helper scripts # such as workers.sh, start-dfs.sh, etc. +# export HADOOP_WORKERS="${HADOOP_CONF_DIR}/workers" + +### +# Options for all daemons +### +# + +# +# Many options may also be specified as Java properties. It is +# very common, and in many cases, desirable, to hard-set these +# in daemon _OPTS variables. Where applicable, the appropriate +# Java property is also identified. Note that many are re-used +# or set differently in certain contexts (e.g., secure vs +# non-secure) +# + +# Where (primarily) daemon log files are stored. +# ${HADOOP_HOME}/logs by default. +# Java property: hadoop.log.dir +# export HADOOP_LOG_DIR=${HADOOP_HOME}/logs + +# A string representing this instance of hadoop. $USER by default. +# This is used in writing log and pid files, so keep that in mind! +# Java property: hadoop.id.str +# export HADOOP_IDENT_STRING=$USER + +# How many seconds to pause after stopping a daemon +# export HADOOP_STOP_TIMEOUT=5 + +# Where pid files are stored. /tmp by default. +# export HADOOP_PID_DIR=/tmp + +# Default log4j setting for interactive commands +# Java property: hadoop.root.logger +# export HADOOP_ROOT_LOGGER=INFO,console + +# Default log4j setting for daemons spawned explicitly by +# --daemon option of hadoop, hdfs, mapred and yarn command. +# Java property: hadoop.root.logger +# export HADOOP_DAEMON_ROOT_LOGGER=INFO,RFA + +# Default log level and output location for security-related messages. +# You will almost certainly want to change this on a per-daemon basis via +# the Java property (i.e., -Dhadoop.security.logger=foo). (Note that the +# defaults for the NN and 2NN override this by default.) +# Java property: hadoop.security.logger +# export HADOOP_SECURITY_LOGGER=INFO,NullAppender + +# Default process priority level +# Note that sub-processes will also run at this level! +# export HADOOP_NICENESS=0 + +# Default name for the service level authorization file +# Java property: hadoop.policy.file +# export HADOOP_POLICYFILE="hadoop-policy.xml" + +# +# NOTE: this is not used by default! <----- +# You can define variables right here and then re-use them later on. +# For example, it is common to use the same garbage collection settings +# for all the daemons. So one could define: +# +# export HADOOP_GC_SETTINGS="-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:+PrintGCDateStamps" +# +# .. and then use it as per the b option under the namenode. + +### +# Secure/privileged execution +### + +# +# Out of the box, Hadoop uses jsvc from Apache Commons to launch daemons +# on privileged ports. This functionality can be replaced by providing +# custom functions. See hadoop-functions.sh for more information. +# + +# The jsvc implementation to use. Jsvc is required to run secure datanodes +# that bind to privileged ports to provide authentication of data transfer +# protocol. Jsvc is not required if SASL is configured for authentication of +# data transfer protocol using non-privileged ports. +# export JSVC_HOME=/usr/bin + +# +# This directory contains pids for secure and privileged processes. +#export HADOOP_SECURE_PID_DIR=${HADOOP_PID_DIR} + +# +# This directory contains the logs for secure and privileged processes. +# Java property: hadoop.log.dir +# export HADOOP_SECURE_LOG=${HADOOP_LOG_DIR} + +# +# When running a secure daemon, the default value of HADOOP_IDENT_STRING +# ends up being a bit bogus. Therefore, by default, the code will +# replace HADOOP_IDENT_STRING with HADOOP_xx_SECURE_USER. If one wants +# to keep HADOOP_IDENT_STRING untouched, then uncomment this line. +# export HADOOP_SECURE_IDENT_PRESERVE="true" + +### +# NameNode specific parameters +### + +# Default log level and output location for file system related change +# messages. For non-namenode daemons, the Java property must be set in +# the appropriate _OPTS if one wants something other than INFO,NullAppender +# Java property: hdfs.audit.logger +# export HDFS_AUDIT_LOGGER=INFO,NullAppender + +# Specify the JVM options to be used when starting the NameNode. +# These options will be appended to the options specified as HADOOP_OPTS +# and therefore may override any similar flags set in HADOOP_OPTS +# +# a) Set JMX options +# export HDFS_NAMENODE_OPTS="-Dcom.sun.management.jmxremote=true -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.port=1026" +# +# b) Set garbage collection logs +# export HDFS_NAMENODE_OPTS="${HADOOP_GC_SETTINGS} -Xloggc:${HADOOP_LOG_DIR}/gc-rm.log-$(date +'%Y%m%d%H%M')" +# +# c) ... or set them directly +# export HDFS_NAMENODE_OPTS="-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:+PrintGCDateStamps -Xloggc:${HADOOP_LOG_DIR}/gc-rm.log-$(date +'%Y%m%d%H%M')" + +# this is the default: +# export HDFS_NAMENODE_OPTS="-Dhadoop.security.logger=INFO,RFAS" + +### +# SecondaryNameNode specific parameters +### +# Specify the JVM options to be used when starting the SecondaryNameNode. +# These options will be appended to the options specified as HADOOP_OPTS +# and therefore may override any similar flags set in HADOOP_OPTS +# +# This is the default: +# export HDFS_SECONDARYNAMENODE_OPTS="-Dhadoop.security.logger=INFO,RFAS" + +### +# DataNode specific parameters +### +# Specify the JVM options to be used when starting the DataNode. +# These options will be appended to the options specified as HADOOP_OPTS +# and therefore may override any similar flags set in HADOOP_OPTS +# +# This is the default: +# export HDFS_DATANODE_OPTS="-Dhadoop.security.logger=ERROR,RFAS" + +# On secure datanodes, user to run the datanode as after dropping privileges. +# This **MUST** be uncommented to enable secure HDFS if using privileged ports +# to provide authentication of data transfer protocol. This **MUST NOT** be +# defined if SASL is configured for authentication of data transfer protocol +# using non-privileged ports. +# This will replace the hadoop.id.str Java property in secure mode. +# export HDFS_DATANODE_SECURE_USER=hdfs + +# Supplemental options for secure datanodes +# By default, Hadoop uses jsvc which needs to know to launch a +# server jvm. +# export HDFS_DATANODE_SECURE_EXTRA_OPTS="-jvm server" + +### +# NFS3 Gateway specific parameters +### +# Specify the JVM options to be used when starting the NFS3 Gateway. +# These options will be appended to the options specified as HADOOP_OPTS +# and therefore may override any similar flags set in HADOOP_OPTS +# +# export HDFS_NFS3_OPTS="" + +# Specify the JVM options to be used when starting the Hadoop portmapper. +# These options will be appended to the options specified as HADOOP_OPTS +# and therefore may override any similar flags set in HADOOP_OPTS +# +# export HDFS_PORTMAP_OPTS="-Xmx512m" + +# Supplemental options for priviliged gateways +# By default, Hadoop uses jsvc which needs to know to launch a +# server jvm. +# export HDFS_NFS3_SECURE_EXTRA_OPTS="-jvm server" + +# On privileged gateways, user to run the gateway as after dropping privileges +# This will replace the hadoop.id.str Java property in secure mode. +# export HDFS_NFS3_SECURE_USER=nfsserver + +### +# ZKFailoverController specific parameters +### +# Specify the JVM options to be used when starting the ZKFailoverController. +# These options will be appended to the options specified as HADOOP_OPTS +# and therefore may override any similar flags set in HADOOP_OPTS +# +# export HDFS_ZKFC_OPTS="" + +### +# QuorumJournalNode specific parameters +### +# Specify the JVM options to be used when starting the QuorumJournalNode. +# These options will be appended to the options specified as HADOOP_OPTS +# and therefore may override any similar flags set in HADOOP_OPTS +# +# export HDFS_JOURNALNODE_OPTS="" + +### +# HDFS Balancer specific parameters +### +# Specify the JVM options to be used when starting the HDFS Balancer. +# These options will be appended to the options specified as HADOOP_OPTS +# and therefore may override any similar flags set in HADOOP_OPTS +# +# export HDFS_BALANCER_OPTS="" + +### +# HDFS Mover specific parameters +### +# Specify the JVM options to be used when starting the HDFS Mover. +# These options will be appended to the options specified as HADOOP_OPTS +# and therefore may override any similar flags set in HADOOP_OPTS +# +# export HDFS_MOVER_OPTS="" + +### +# Router-based HDFS Federation specific parameters +# Specify the JVM options to be used when starting the RBF Routers. +# These options will be appended to the options specified as HADOOP_OPTS +# and therefore may override any similar flags set in HADOOP_OPTS +# +# export HDFS_DFSROUTER_OPTS="" +### + +### +# Advanced Users Only! +### + +# +# When building Hadoop, one can add the class paths to the commands +# via this special env var: +# export HADOOP_ENABLE_BUILD_PATHS="true" + +# +# To prevent accidents, shell commands be (superficially) locked +# to only allow certain users to execute certain subcommands. +# It uses the format of (command)_(subcommand)_USER. +# +# For example, to limit who can execute the namenode command, +# export HDFS_NAMENODE_USER=hdfs diff --git a/docker/test/integration/hive_server/hdfs-site.xml b/docker/test/integration/hive_server/hdfs-site.xml new file mode 100644 index 00000000000..82c525ea414 --- /dev/null +++ b/docker/test/integration/hive_server/hdfs-site.xml @@ -0,0 +1,6 @@ + + + dfs.replication + 1 + + diff --git a/docker/test/integration/hive_server/hive-site.xml b/docker/test/integration/hive_server/hive-site.xml new file mode 100644 index 00000000000..ec1735ea16f --- /dev/null +++ b/docker/test/integration/hive_server/hive-site.xml @@ -0,0 +1,35 @@ + + + + + javax.jdo.option.ConnectionURL + jdbc:mysql://localhost/hcatalog?createDatabaseIfNotExist=true + + + javax.jdo.option.ConnectionUserName + test + + + javax.jdo.option.ConnectionPassword + test + + + javax.jdo.option.ConnectionDriverName + com.mysql.jdbc.Driver + + diff --git a/docker/test/integration/hive_server/mapred-site.xml b/docker/test/integration/hive_server/mapred-site.xml new file mode 100644 index 00000000000..dba582f1c31 --- /dev/null +++ b/docker/test/integration/hive_server/mapred-site.xml @@ -0,0 +1,6 @@ + + + mapreduce.framework.name + yarn + + diff --git a/docker/test/integration/hive_server/prepare_hive_data.sh b/docker/test/integration/hive_server/prepare_hive_data.sh new file mode 100755 index 00000000000..afecbb91c5d --- /dev/null +++ b/docker/test/integration/hive_server/prepare_hive_data.sh @@ -0,0 +1,6 @@ +#!/bin/bash +hive -e "create database test" + +hive -e "create table test.demo(id string, score int) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'; create table test.demo_orc(id string, score int) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat'; " +hive -e "create table test.demo_text(id string, score int, day string)row format delimited fields terminated by ','; load data local inpath '/demo_data.txt' into table test.demo_text " + hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.demo partition(day) select * from test.demo_text; insert into test.demo_orc partition(day) select * from test.demo_text" diff --git a/docker/test/integration/hive_server/start.sh b/docker/test/integration/hive_server/start.sh new file mode 100755 index 00000000000..e01f28542af --- /dev/null +++ b/docker/test/integration/hive_server/start.sh @@ -0,0 +1,12 @@ +service ssh start +sed s/HOSTNAME/$HOSTNAME/ /hadoop-3.1.0/etc/hadoop/core-site.xml.template > /hadoop-3.1.0/etc/hadoop/core-site.xml +hadoop namenode -format +start-all.sh +service mysql start +mysql -u root -e "CREATE USER \"test\"@\"localhost\" IDENTIFIED BY \"test\"" +mysql -u root -e "GRANT ALL ON * . * TO 'test'@'localhost'" +schematool -initSchema -dbType mysql +#nohup hiveserver2 & +nohup hive --service metastore & +bash /prepare_hive_data.sh +while true; do sleep 1000; done diff --git a/docker/test/integration/hive_server/yarn-site.xml b/docker/test/integration/hive_server/yarn-site.xml new file mode 100644 index 00000000000..bd5f694e6a6 --- /dev/null +++ b/docker/test/integration/hive_server/yarn-site.xml @@ -0,0 +1,32 @@ + + + yarn.nodemanager.aux-services + mapreduce_shuffle + + + + yarn.application.classpath + /hadoop-3.1.0/etc/hadoop,/hadoop-3.1.0/share/hadoop/common/*,/hadoop-3.1.0/share/hadoop/common/lib/*,/hadoop-3.1.0/share/hadoop/hdfs/*, /hadoop-3.1.0/share/hadoop/hdfs/lib/*, /hadoop-3.1.0/share/hadoop/mapreduce/*, /hadoop-3.1.0/share/hadoop/mapreduce/lib/*, /hadoop-3.1.0/share/hadoop/yarn/*, /hadoop-3.1.0/share/hadoop/yarn/lib/* + + + + + Number of seconds after an application finishes before the nodemanager's + DeletionService will delete the application's localized file directory + and log directory. + + To diagnose Yarn application problems, set this property's value large + enough (for example, to 600 = 10 minutes) to permit examination of these + directories. After changing the property's value, you must restart the + nodemanager in order for it to have an effect. + + The roots of Yarn applications' work directories is configurable with + the yarn.nodemanager.local-dirs property (see below), and the roots + of the Yarn applications' log directories is configurable with the + yarn.nodemanager.log-dirs property (see also below). + + yarn.nodemanager.delete.debug-delay-sec + 600 + + + diff --git a/docker/test/integration/runner/compose/docker_compose_hive.yml b/docker/test/integration/runner/compose/docker_compose_hive.yml new file mode 100644 index 00000000000..44f23655d2a --- /dev/null +++ b/docker/test/integration/runner/compose/docker_compose_hive.yml @@ -0,0 +1,7 @@ +version: '2.3' +services: + hdfs1: + image: lgboustc/hive_test:v1.0 + hostname: hivetest + restart: always + entrypoint: bash /start.sh diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 1d5d2d004f9..efc781ce864 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -750,17 +750,24 @@ class ClickHouseCluster: '--file', p.join(docker_compose_yml_dir, 'docker_compose_nginx.yml')] return self.base_nginx_cmd + def setup_hive(self, instance, env_variables, docker_compose_yml_dir): + self.with_hive = True + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_hive.yml')]) + self.base_hive_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_hive.yml')] + return self.base_hive_cmd + def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, with_zookeeper=False, with_zookeeper_secure=False, with_mysql_client=False, with_mysql=False, with_mysql8=False, with_mysql_cluster=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_postgres_cluster=False, with_hdfs=False, with_kerberized_hdfs=False, with_mongo=False, with_mongo_secure=False, with_nginx=False, - with_redis=False, with_minio=False, with_cassandra=False, with_jdbc_bridge=False, + with_redis=False, with_minio=False, with_cassandra=False, with_jdbc_bridge=False, with_hive=False, hostname=None, env_variables=None, image="clickhouse/integration-test", tag=None, stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, external_dirs=None, tmpfs=None, zookeeper_docker_compose_path=None, minio_certs_dir=None, use_keeper=True, - main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True, config_root_name="clickhouse"): + main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True, config_root_name="clickhouse", other_configs=[]): """Add an instance to the cluster. @@ -814,6 +821,7 @@ class ClickHouseCluster: with_minio=with_minio, with_cassandra=with_cassandra, with_jdbc_bridge=with_jdbc_bridge, + with_hive = with_hive, server_bin_path=self.server_bin_path, odbc_bridge_bin_path=self.odbc_bridge_bin_path, library_bridge_bin_path=self.library_bridge_bin_path, @@ -834,7 +842,8 @@ class ClickHouseCluster: copy_common_configs=copy_common_configs, external_dirs=external_dirs, tmpfs=tmpfs or [], - config_root_name=config_root_name) + config_root_name=config_root_name, + other_configs = other_configs) docker_compose_yml_dir = get_docker_compose_path() @@ -927,6 +936,9 @@ class ClickHouseCluster: if with_jdbc_bridge and not self.with_jdbc_bridge: cmds.append(self.setup_jdbc_bridge_cmd(instance, env_variables, docker_compose_yml_dir)) + if with_hive: + cmds.append(self.setup_hive(instance, env_variables, docker_compose_yml_dir)) + logging.debug("Cluster name:{} project_name:{}. Added instance name:{} tag:{} base_cmd:{} docker_compose_yml_dir:{}".format( self.name, self.project_name, name, tag, self.base_cmd, docker_compose_yml_dir)) return instance @@ -1588,6 +1600,12 @@ class ClickHouseCluster: self.up_called = True time.sleep(10) + if self.with_hive and self.base_hive_cmd: + logging.debug('Setup hive') + subprocess_check_call(self.base_hive_cmd + common_opts) + self.up_called = True + time.sleep(300) + if self.with_minio and self.base_minio_cmd: # Copy minio certificates to minio/certs os.mkdir(self.minio_dir) @@ -1823,13 +1841,13 @@ class ClickHouseInstance: self, cluster, base_path, name, base_config_dir, custom_main_configs, custom_user_configs, custom_dictionaries, macros, with_zookeeper, zookeeper_config_path, with_mysql_client, with_mysql, with_mysql8, with_mysql_cluster, with_kafka, with_kerberized_kafka, - with_rabbitmq, with_nginx, with_kerberized_hdfs, with_mongo, with_redis, with_minio, with_jdbc_bridge, + with_rabbitmq, with_nginx, with_kerberized_hdfs, with_mongo, with_redis, with_minio, with_jdbc_bridge,with_hive, with_cassandra, server_bin_path, odbc_bridge_bin_path, library_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, with_postgres, with_postgres_cluster, clickhouse_start_command=CLICKHOUSE_START_COMMAND, main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True, hostname=None, env_variables=None, image="clickhouse/integration-test", tag="latest", - stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, external_dirs=None, tmpfs=None, config_root_name="clickhouse"): + stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, external_dirs=None, tmpfs=None, config_root_name="clickhouse", other_configs=[]): self.name = name self.base_cmd = cluster.base_cmd @@ -1843,6 +1861,7 @@ class ClickHouseInstance: self.custom_main_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_main_configs] self.custom_user_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_user_configs] self.custom_dictionaries_paths = [p.abspath(p.join(base_path, c)) for c in custom_dictionaries] + self.other_custom_config_paths = [p.abspath(p.join(base_path,c)) for c in other_configs] self.clickhouse_path_dir = p.abspath(p.join(base_path, clickhouse_path_dir)) if clickhouse_path_dir else None self.kerberos_secrets_dir = p.abspath(p.join(base_path, 'secrets')) self.macros = macros if macros is not None else {} @@ -2376,6 +2395,8 @@ class ClickHouseInstance: os.mkdir(users_d_dir) dictionaries_dir = p.abspath(p.join(instance_config_dir, 'dictionaries')) os.mkdir(dictionaries_dir) + other_conf_dir = p.abspath(p.join(instance_config_dir, 'other_conf.d')) + os.mkdir(other_conf_dir) def write_embedded_config(name, dest_dir, fix_log_level=False): with open(p.join(HELPERS_DIR, name), 'r') as f: @@ -2422,6 +2443,8 @@ class ClickHouseInstance: # Copy dictionaries configs to configs/dictionaries for path in self.custom_dictionaries_paths: shutil.copy(path, dictionaries_dir) + for path in self.other_custom_config_paths: + shutil.copy(path, other_conf_dir) db_dir = p.abspath(p.join(self.path, 'database')) logging.debug(f"Setup database dir {db_dir}") diff --git a/tests/integration/test_hive_query/__init__.py b/tests/integration/test_hive_query/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_hive_query/configs/config.xml b/tests/integration/test_hive_query/configs/config.xml new file mode 100644 index 00000000000..4b6316b1670 --- /dev/null +++ b/tests/integration/test_hive_query/configs/config.xml @@ -0,0 +1,22 @@ + + + + + + + localhost + 9000 + + + + + + /clickhouse_local_cache + 207374182400 + 1000 + + + /etc/clickhouse-server/other_conf.d/hdfs-site.xml + + + diff --git a/tests/integration/test_hive_query/configs/hdfs-site.xml b/tests/integration/test_hive_query/configs/hdfs-site.xml new file mode 100644 index 00000000000..82c525ea414 --- /dev/null +++ b/tests/integration/test_hive_query/configs/hdfs-site.xml @@ -0,0 +1,6 @@ + + + dfs.replication + 1 + + diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py new file mode 100644 index 00000000000..674539144c1 --- /dev/null +++ b/tests/integration/test_hive_query/test.py @@ -0,0 +1,91 @@ +import logging +import os + +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +logging.getLogger().setLevel(logging.INFO) +logging.getLogger().addHandler(logging.StreamHandler()) + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance('h0_0_0', main_configs=['configs/config.xml'], other_configs=[ 'configs/hdfs-site.xml'], with_hive=True) + + logging.info("Starting cluster ...") + cluster.start() + yield cluster + finally: + cluster.shutdown() + +def test_create_parquet_table(started_cluster): + logging.info('Start testing creating hive table ...') + node = started_cluster.instances['h0_0_0'] + result = node.query(""" + CREATE TABLE default.demo_parquet (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) + """) + logging.info("create result {}".format(result)) + + assert result.strip() == '' + +def test_create_orc_table(started_cluster): + logging.info('Start testing creating hive table ...') + node = started_cluster.instances['h0_0_0'] + result = node.query(""" + CREATE TABLE default.demo_orc (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day) + """) + logging.info("create result {}".format(result)) + + assert result.strip() == '' + +def test_create_text_table(started_cluster): + logging.info('Start testing creating hive table ...') + node = started_cluster.instances['h0_0_0'] + result = node.query(""" + CREATE TABLE default.demo_text (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_text') PARTITION BY (tuple()) + """) + logging.info("create result {}".format(result)) + + assert result.strip() == '' + +def test_parquet_groupby(started_cluster): + logging.info('Start testing groupby ...') + node = started_cluster.instances['h0_0_0'] + result = node.query(""" + SELECT day, count(*) FROM default.demo_parquet group by day order by day + """) + expected_result = """2021-11-01 1 +2021-11-05 2 +2021-11-11 1 +2021-11-16 2 +""" + assert result == expected_result +def test_orc_groupby(started_cluster): + logging.info('Start testing groupby ...') + node = started_cluster.instances['h0_0_0'] + result = node.query(""" + SELECT day, count(*) FROM default.demo_orc group by day order by day + """) + expected_result = """2021-11-01 1 +2021-11-05 2 +2021-11-11 1 +2021-11-16 2 +""" + assert result == expected_result + +def test_text_count(started_cluster): + node = started_cluster.instances['h0_0_0'] + result = node.query(""" + SELECT day, count(*) FROM default.demo_orc group by day order by day SETTINGS format_csv_delimiter = '\x01' + """) + expected_result = """2021-11-01 1 +2021-11-05 2 +2021-11-11 1 +2021-11-16 2 +""" + assert result == expected_result From 827fa511e89ae1af9bc1c4d747d60c6126b74414 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 18 Nov 2021 21:14:46 +0300 Subject: [PATCH 0064/1260] Add test for freeze/unfreeze with S3 zero-copy --- tests/integration/helpers/cluster.py | 6 ++ .../configs/config.d/s3.xml | 3 + .../test_s3_zero_copy_replication/test.py | 71 ++++++++++++++++++- 3 files changed, 79 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6ab8b29895b..803f56d7cff 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2621,6 +2621,12 @@ class ClickHouseInstance: shutil.rmtree(self.path) + def get_backuped_s3_objects(self, disk, backup_name): + command = ['find', f'/var/lib/clickhouse/disks/{disk}/shadow/{backup_name}/store', '-type', 'f', + '-exec', 'grep', '-o', 'r[01]\\{64\\}-file-[[:lower:]]\\{32\\}', '{}', ';'] + return self.exec_in_container(command).split('\n') + + class ClickHouseKiller(object): def __init__(self, clickhouse_node): self.clickhouse_node = clickhouse_node diff --git a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml index 5d10ac0d959..181144b0473 100644 --- a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml +++ b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml @@ -7,18 +7,21 @@ http://minio1:9001/root/data/ minio minio123 + true s3 http://minio1:9001/root/data/ minio minio123 + true s3 http://minio1:9001/root/data2/ minio minio123 + true diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index d9f7cca4a3a..375a03e7f3a 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -32,11 +32,30 @@ def get_large_objects_count(cluster, size=100, folder='data'): minio = cluster.minio_client counter = 0 for obj in minio.list_objects(cluster.minio_bucket, '{}/'.format(folder)): - if obj.size >= size: + if obj.size is not None and obj.size >= size: counter = counter + 1 return counter +def check_objects_exisis(cluster, object_list, folder='data'): + minio = cluster.minio_client + for obj in object_list: + if obj: + minio.stat_object(cluster.minio_bucket, '{}/{}'.format(folder, obj)) + + +def check_objects_not_exisis(cluster, object_list, folder='data'): + minio = cluster.minio_client + for obj in object_list: + if obj: + try: + minio.stat_object(cluster.minio_bucket, '{}/{}'.format(folder, obj)) + except Exception as error: + assert "NoSuchKey" in str(error) + else: + assert False, "Object {} should not be exists".format(obj) + + def wait_for_large_objects_count(cluster, expected, size=100, timeout=30): while timeout > 0: if get_large_objects_count(cluster, size=size) == expected: @@ -248,3 +267,53 @@ def test_s3_zero_copy_with_ttl_delete(cluster, large_data, iterations): node1.query("DROP TABLE IF EXISTS ttl_delete_test NO DELAY") node2.query("DROP TABLE IF EXISTS ttl_delete_test NO DELAY") + + +def test_s3_zero_copy_unfreeze(cluster): + node1 = cluster.instances["node1"] + node2 = cluster.instances["node2"] + + node1.query("DROP TABLE IF EXISTS unfreeze_test NO DELAY") + node2.query("DROP TABLE IF EXISTS unfreeze_test NO DELAY") + + node1.query( + """ + CREATE TABLE unfreeze_test ON CLUSTER test_cluster (d UInt64) + ENGINE=ReplicatedMergeTree('/clickhouse/tables/unfreeze_test', '{}') + ORDER BY d + SETTINGS storage_policy='s3' + """ + .format('{replica}') + ) + + node1.query("INSERT INTO unfreeze_test VALUES (0)") + + node1.query("ALTER TABLE unfreeze_test FREEZE WITH NAME 'backup1'") + node2.query("ALTER TABLE unfreeze_test FREEZE WITH NAME 'backup2'") + + time.sleep(1) + + objects01 = node1.get_backuped_s3_objects("s31", "backup1") + objects02 = node2.get_backuped_s3_objects("s31", "backup2") + + assert objects01 == objects02 + + check_objects_exisis(cluster, objects01) + + node1.query("TRUNCATE TABLE unfreeze_test") + + objects11 = node1.get_backuped_s3_objects("s31", "backup1") + objects12 = node2.get_backuped_s3_objects("s31", "backup2") + + assert objects01 == objects11 + assert objects01 == objects12 + + check_objects_exisis(cluster, objects11) + + node1.query("ALTER TABLE unfreeze_test UNFREEZE WITH NAME 'backup1'") + + check_objects_exisis(cluster, objects12) + + node2.query("ALTER TABLE unfreeze_test UNFREEZE WITH NAME 'backup2'") + + check_objects_not_exisis(cluster, objects12) From 858f615463f54876128a002af5fa26f74ba862e6 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 19 Nov 2021 09:58:34 +0800 Subject: [PATCH 0065/1260] fixed bugs in integration tests about hive query --- tests/integration/helpers/cluster.py | 18 ++++++++++-------- .../test_hive_query/configs/config.xml | 2 +- tests/integration/test_hive_query/test.py | 2 +- 3 files changed, 12 insertions(+), 10 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index efc781ce864..9f7fd39786a 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -268,6 +268,7 @@ class ClickHouseCluster: self.with_cassandra = False self.with_jdbc_bridge = False self.with_nginx = False + self.with_hive = False self.with_minio = False self.minio_dir = os.path.join(self.instances_dir, "minio") @@ -767,7 +768,7 @@ class ClickHouseCluster: hostname=None, env_variables=None, image="clickhouse/integration-test", tag=None, stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, external_dirs=None, tmpfs=None, zookeeper_docker_compose_path=None, minio_certs_dir=None, use_keeper=True, - main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True, config_root_name="clickhouse", other_configs=[]): + main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True, config_root_name="clickhouse", extra_configs=[]): """Add an instance to the cluster. @@ -777,6 +778,7 @@ class ClickHouseCluster: user_configs - a list of config files that will be added to users.d/ directory with_zookeeper - if True, add ZooKeeper configuration to configs and ZooKeeper instances to the cluster. with_zookeeper_secure - if True, add ZooKeeper Secure configuration to configs and ZooKeeper instances to the cluster. + extra_configs - config files cannot put into config.d and users.d """ if self.is_up: @@ -843,7 +845,7 @@ class ClickHouseCluster: external_dirs=external_dirs, tmpfs=tmpfs or [], config_root_name=config_root_name, - other_configs = other_configs) + extra_configs = extra_configs) docker_compose_yml_dir = get_docker_compose_path() @@ -1847,7 +1849,7 @@ class ClickHouseInstance: main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True, hostname=None, env_variables=None, image="clickhouse/integration-test", tag="latest", - stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, external_dirs=None, tmpfs=None, config_root_name="clickhouse", other_configs=[]): + stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, external_dirs=None, tmpfs=None, config_root_name="clickhouse", extra_configs=[]): self.name = name self.base_cmd = cluster.base_cmd @@ -1861,7 +1863,7 @@ class ClickHouseInstance: self.custom_main_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_main_configs] self.custom_user_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_user_configs] self.custom_dictionaries_paths = [p.abspath(p.join(base_path, c)) for c in custom_dictionaries] - self.other_custom_config_paths = [p.abspath(p.join(base_path,c)) for c in other_configs] + self.custom_extra_config_paths = [p.abspath(p.join(base_path,c)) for c in extra_configs] self.clickhouse_path_dir = p.abspath(p.join(base_path, clickhouse_path_dir)) if clickhouse_path_dir else None self.kerberos_secrets_dir = p.abspath(p.join(base_path, 'secrets')) self.macros = macros if macros is not None else {} @@ -2395,8 +2397,8 @@ class ClickHouseInstance: os.mkdir(users_d_dir) dictionaries_dir = p.abspath(p.join(instance_config_dir, 'dictionaries')) os.mkdir(dictionaries_dir) - other_conf_dir = p.abspath(p.join(instance_config_dir, 'other_conf.d')) - os.mkdir(other_conf_dir) + extra_conf_dir = p.abspath(p.join(instance_config_dir, 'extra_conf.d')) + os.mkdir(extra_conf_dir) def write_embedded_config(name, dest_dir, fix_log_level=False): with open(p.join(HELPERS_DIR, name), 'r') as f: @@ -2443,8 +2445,8 @@ class ClickHouseInstance: # Copy dictionaries configs to configs/dictionaries for path in self.custom_dictionaries_paths: shutil.copy(path, dictionaries_dir) - for path in self.other_custom_config_paths: - shutil.copy(path, other_conf_dir) + for path in self.custom_extra_config_paths: + shutil.copy(path, extra_conf_dir) db_dir = p.abspath(p.join(self.path, 'database')) logging.debug(f"Setup database dir {db_dir}") diff --git a/tests/integration/test_hive_query/configs/config.xml b/tests/integration/test_hive_query/configs/config.xml index 4b6316b1670..2aca007ef9b 100644 --- a/tests/integration/test_hive_query/configs/config.xml +++ b/tests/integration/test_hive_query/configs/config.xml @@ -16,7 +16,7 @@ 1000 - /etc/clickhouse-server/other_conf.d/hdfs-site.xml + /etc/clickhouse-server/extra_conf.d/hdfs-site.xml diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index 674539144c1..a30f0b03070 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -15,7 +15,7 @@ SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) def started_cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance('h0_0_0', main_configs=['configs/config.xml'], other_configs=[ 'configs/hdfs-site.xml'], with_hive=True) + cluster.add_instance('h0_0_0', main_configs=['configs/config.xml'], extra_configs=[ 'configs/hdfs-site.xml'], with_hive=True) logging.info("Starting cluster ...") cluster.start() From ab64a171044b7d6d29940249765aef2046fc4af3 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 19 Nov 2021 16:20:14 +0800 Subject: [PATCH 0066/1260] fix fasttest --- docker/test/fasttest/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 671357bd8fc..6bb460eac8b 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -187,6 +187,7 @@ function clone_submodules contrib/fast_float contrib/NuRaft contrib/thrift + contrib/hive-metastore ) git submodule sync From d22854e82bf74bd3f183928263aaf8ff97a51708 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 22 Nov 2021 10:52:10 +0800 Subject: [PATCH 0067/1260] add compile swiych USE_HIVE --- cmake/find/parquet.cmake | 3 --- contrib/CMakeLists.txt | 4 +++- docker/test/fasttest/run.sh | 2 -- src/CMakeLists.txt | 17 +++++++++-------- src/Common/config.h.in | 1 + src/Interpreters/Context.cpp | 8 ++++---- src/Interpreters/Context.h | 5 ++--- src/Storages/Hive/HiveCommon.cpp | 2 +- src/Storages/Hive/HiveCommon.h | 3 +-- src/Storages/Hive/HiveFile.cpp | 2 +- src/Storages/Hive/HiveFile.h | 3 +-- src/Storages/Hive/HiveSettings.cpp | 2 +- src/Storages/Hive/HiveSettings.h | 3 +-- src/Storages/Hive/StorageHive.cpp | 2 +- src/Storages/Hive/StorageHive.h | 3 +-- src/Storages/registerStorages.cpp | 4 ++-- 16 files changed, 29 insertions(+), 35 deletions(-) diff --git a/cmake/find/parquet.cmake b/cmake/find/parquet.cmake index a294ccb00c4..ffb5cdc0332 100644 --- a/cmake/find/parquet.cmake +++ b/cmake/find/parquet.cmake @@ -34,7 +34,6 @@ endif() if(NOT USE_INTERNAL_PARQUET_LIBRARY) find_package(Arrow) find_package(Parquet) - # find_library(THRIFT_LIBRARY thrift) find_library(UTF8_PROC_LIBRARY utf8proc) find_package(BZip2) @@ -145,12 +144,10 @@ if(NOT EXTERNAL_PARQUET_FOUND AND NOT MISSING_INTERNAL_PARQUET_LIBRARY AND NOT O set(FLATBUFFERS_LIBRARY flatbuffers) set(ARROW_LIBRARY arrow_static) set(PARQUET_LIBRARY parquet_static) - # set(THRIFT_LIBRARY thrift_static) else() set(FLATBUFFERS_LIBRARY flatbuffers_shared) set(ARROW_LIBRARY arrow_shared) set(PARQUET_LIBRARY parquet_shared) - # set(THRIFT_LIBRARY thrift) endif() set(USE_PARQUET 1) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 81960d54ec3..6499b179883 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -349,7 +349,9 @@ if (USE_S2_GEOMETRY) add_subdirectory(s2geometry-cmake) endif() -add_subdirectory (hive-metastore-cmake) +if (USE_HIVE) + add_subdirectory (hive-metastore-cmake) +endif() # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 6bb460eac8b..925ac6f797c 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -186,8 +186,6 @@ function clone_submodules contrib/dragonbox contrib/fast_float contrib/NuRaft - contrib/thrift - contrib/hive-metastore ) git submodule sync diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index dc2cc9a2a66..00f717669e8 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -108,10 +108,10 @@ endif() if (USE_HDFS) add_headers_and_sources(dbms Storages/HDFS) add_headers_and_sources(dbms Disks/HDFS) +endif() - if (USE_ORC AND USE_PARQUET) - add_headers_and_sources(dbms Storages/Hive) - endif() +if (USE_HIVE) + add_headers_and_sources(dbms Storages/Hive) endif() if(USE_FILELOG) @@ -447,13 +447,14 @@ endif() if (USE_HDFS) dbms_target_link_libraries(PRIVATE ${HDFS3_LIBRARY}) dbms_target_include_directories (SYSTEM BEFORE PUBLIC ${HDFS3_INCLUDE_DIR}) - - if (USE_ORC AND USE_PARQUET) - dbms_target_link_libraries(PRIVATE hivemetastore) - dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/hive-metastore) - endif() endif() +if (USE_HIVE) + dbms_target_link_libraries(PRIVATE hivemetastore) + dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/hive-metastore) +endif() + + if (USE_AWS_S3) target_link_libraries (clickhouse_common_io PUBLIC ${AWS_S3_LIBRARY}) target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${AWS_S3_CORE_INCLUDE_DIR}) diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 1b14794fb27..2d8c82a12e3 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -21,3 +21,4 @@ #cmakedefine01 CLICKHOUSE_SPLIT_BINARY #cmakedefine01 USE_BZIP2 #cmakedefine01 USE_SNAPPY +#cmakedefine01 USE_HIVE diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 21430636df9..285517e9677 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -88,7 +88,7 @@ #include #include -#if USE_HDFS && USE_ORC && USE_PARQUET +#if USE_HIVE #include #include #include @@ -157,7 +157,7 @@ struct ContextSharedPart mutable std::mutex storage_policies_mutex; /// Separate mutex for re-initialization of zookeeper session. This operation could take a long time and must not interfere with another operations. mutable std::mutex zookeeper_mutex; -#if USE_HDFS && USE_ORC && USE_PARQUET +#if USE_HIVE /// Separate mutex for re-initialization of hive metastore client. This operation could take a long time and must not interfere with another operations. mutable std::mutex hive_metastore_mutex; #endif @@ -173,7 +173,7 @@ struct ContextSharedPart mutable std::map auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients. ConfigurationPtr auxiliary_zookeepers_config; /// Stores auxiliary zookeepers configs -#if USE_HDFS && USE_ORC && USE_PARQUET +#if USE_HIVE mutable std::map hive_metastore_clients; /// Map for hive metastore clients #endif @@ -1818,7 +1818,7 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const return shared->zookeeper; } -#if USE_HDFS && USE_ORC && USE_PARQUET +#if USE_HIVE HMSClientPtr Context::getHMSClient(const String & name) const { using namespace apache::thrift; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index ee6dd6645c3..193d7a96fa0 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -18,7 +18,6 @@ #include #include "config_core.h" -#include "config_formats.h" #include #include @@ -175,7 +174,7 @@ private: std::unique_ptr shared; }; -#if USE_HDFS && USE_ORC && USE_PARQUET +#if USE_HIVE class HMSClient; using HMSClientPtr = std::shared_ptr; #endif @@ -691,7 +690,7 @@ public: // Reload Zookeeper void reloadZooKeeperIfChanged(const ConfigurationPtr & config) const; -#if USE_HDFS && USE_ORC && USE_PARQUET +#if USE_HIVE HMSClientPtr getHMSClient(const String & name) const; #endif diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 17b3b3eb04f..5e76df5328e 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -1,6 +1,6 @@ #include -#if USE_HDFS && USE_ORC && USE_PARQUET +#if USE_HIVE namespace DB { namespace ErrorCodes diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index eeed9ae5903..db8877471b5 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -1,9 +1,8 @@ #pragma once #include -#include "config_formats.h" -#if USE_HDFS && USE_ORC && USE_PARQUET +#if USE_HIVE #include #include diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 572a1db100f..665a086677c 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -1,6 +1,6 @@ #include -#if USE_HDFS && USE_ORC && USE_PARQUET +#if USE_HIVE #include #include diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index 9b4737487df..152c93c7a52 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -1,8 +1,7 @@ #pragma once #include -#include "config_formats.h" -#if USE_HDFS && USE_ORC && USE_PARQUET +#if USE_HIVE #include #include diff --git a/src/Storages/Hive/HiveSettings.cpp b/src/Storages/Hive/HiveSettings.cpp index 80a8b085057..9519ce8b1f5 100644 --- a/src/Storages/Hive/HiveSettings.cpp +++ b/src/Storages/Hive/HiveSettings.cpp @@ -1,6 +1,6 @@ #include -#if USE_HDFS && USE_ORC && USE_PARQUET +#if USE_HIVE #include #include diff --git a/src/Storages/Hive/HiveSettings.h b/src/Storages/Hive/HiveSettings.h index 2fa704fe116..36ec566448d 100644 --- a/src/Storages/Hive/HiveSettings.h +++ b/src/Storages/Hive/HiveSettings.h @@ -1,9 +1,8 @@ #pragma once #include -#include "config_formats.h" -#if USE_HDFS && USE_ORC && USE_PARQUET +#if USE_HIVE #include #include diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 71a829b764d..9ef9b852d4b 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -1,6 +1,6 @@ #include -#if USE_HDFS && USE_ORC && USE_PARQUET +#if USE_HIVE #include #include diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 2007325f408..0daf63e3f32 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -1,9 +1,8 @@ #pragma once #include -#include "config_formats.h" -#if USE_HDFS && USE_ORC && USE_PARQUET +#if USE_HIVE #include #include diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 323991e0c5a..e2f40a5d68a 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -35,7 +35,7 @@ void registerStorageCOS(StorageFactory & factory); #if USE_HDFS void registerStorageHDFS(StorageFactory & factory); -#if USE_ORC && USE_PARQUET +#if USE_HIVE void registerStorageHive(StorageFactory & factory); #endif @@ -111,7 +111,7 @@ void registerStorages() #if USE_HDFS registerStorageHDFS(factory); - #if USE_ORC && USE_PARQUET + #if USE_HIVE registerStorageHive(factory); #endif From 75bd6c608538ad50320cc21ff509f9164764198e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 22 Nov 2021 12:29:40 +0800 Subject: [PATCH 0068/1260] fix build --- CMakeLists.txt | 1 + cmake/find/hive-metastore.cmake | 23 ++++++++++ cmake/find/thrift.cmake | 47 +++++++++++++-------- contrib/hive-metastore-cmake/CMakeLists.txt | 6 +-- 4 files changed, 56 insertions(+), 21 deletions(-) create mode 100644 cmake/find/hive-metastore.cmake diff --git a/CMakeLists.txt b/CMakeLists.txt index 46462caf713..b1258c977f1 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -594,6 +594,7 @@ include (cmake/find/sentry.cmake) include (cmake/find/stats.cmake) include (cmake/find/datasketches.cmake) include (cmake/find/libprotobuf-mutator.cmake) +include (cmake/find/hive-metastore.cmake) set (USE_INTERNAL_CITYHASH_LIBRARY ON CACHE INTERNAL "") find_contrib_lib(cityhash) diff --git a/cmake/find/hive-metastore.cmake b/cmake/find/hive-metastore.cmake new file mode 100644 index 00000000000..25358648b8e --- /dev/null +++ b/cmake/find/hive-metastore.cmake @@ -0,0 +1,23 @@ +option(ENABLE_HIVE "Enable Hive" ${ENABLE_LIBRARIES}) + +if (NOT ENABLE_HIVE) + message("Hive disabled") + return() +endif() + +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/hive-metastore") + message(WARNING "submodule contrib/hive-metastore is missing. to fix try run: \n git submodule update --init") + set(USE_HIVE 0) +elseif (NOT USE_THRIFT) + message(WARNING "Thrift is not found, which is needed by Hive") + set(USE_HIVE 0) +elseif (NOT USE_ORC OR NOT USE_ARROW OR NOT USE_PARQUET) + message(WARNING "ORC/Arrow/Parquet is not found, which are needed by Hive") + set(USE_HIVE 0) +else() + set(USE_HIVE 1) + set(HIVE_METASTORE_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/hive-metastore) + set(HIVE_METASTORE_LIBRARY hivemetastore) +endif() + +message (STATUS "Using_Hive=${USE_HIVE}: ${HIVE_METASTORE_INCLUDE_DIR} : ${HIVE_METASTORE_LIBRARY}") diff --git a/cmake/find/thrift.cmake b/cmake/find/thrift.cmake index 0a872e76abe..2e7a8356108 100644 --- a/cmake/find/thrift.cmake +++ b/cmake/find/thrift.cmake @@ -1,23 +1,34 @@ -option(ENABLE_THRIFT "Enable Thrift" ON) +option(ENABLE_THRIFT "Enable Thrift" ${ENABLE_LIBRARIES}) + +if (NOT ENABLE_THRIFT) + message (STATUS "thrift disabled") + set(USE_INTERNAL_THRIFT_LIBRARY 0) + return() +endif() + option(USE_INTERNAL_THRIFT_LIBRARY "Set to FALSE to use system thrift library instead of bundled" ${NOT_UNBUNDLED}) - -if(ENABLE_THRIFT) - if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/thrift") +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/thrift") + if (USE_INTERNAL_THRIFT_LIBRARY) message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") - set (MISSING_THRIFT 1) - endif() - - if (USE_INTERNAL_THRIFT_LIBRARY AND NOT MISSING_THRIFT) - if (MAKE_STATIC_LIBRARIES) - set(THRIFT_LIBRARY thrift_static) - else() - set(THRIFT_LIBRARY thrift) - endif() - set (THRIFT_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src") - # set (THRIFT_COMPILER "thrift-compiler") - else() - find_library(THRIFT_LIBRARY thrift) + set(USE_INTERNAL_THRIFT_LIBRARY 0) endif () endif() -message (STATUS "Using_THRIFT=${ENABLE_THRIFT}: ${THRIFT_INCLUDE_DIR} : ${THRIFT_LIBRARY}") +if (USE_INTERNAL_THRIFT_LIBRARY) + if (MAKE_STATIC_LIBRARIES) + set(THRIFT_LIBRARY thrift_static) + else() + set(THRIFT_LIBRARY thrift) + endif() + set (THRIFT_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp/src") + set(USE_THRIFT 1) +else() + find_library(THRIFT_LIBRARY thrift) + if (NOT THRIFT_LIBRARY) + set(USE_THRIFT 0) + else() + set(USE_THRIFT 1) + endif() +endif () + +message (STATUS "Using_THRIFT=${USE_THRIFT}: ${THRIFT_INCLUDE_DIR} : ${THRIFT_LIBRARY}") diff --git a/contrib/hive-metastore-cmake/CMakeLists.txt b/contrib/hive-metastore-cmake/CMakeLists.txt index 669ade7d2ad..c92405fa4e8 100644 --- a/contrib/hive-metastore-cmake/CMakeLists.txt +++ b/contrib/hive-metastore-cmake/CMakeLists.txt @@ -4,6 +4,6 @@ set (SRCS ${ClickHouse_SOURCE_DIR}/contrib/hive-metastore/ThriftHiveMetastore.cpp ) -add_library(hivemetastore ${SRCS}) -target_link_libraries(hivemetastore PUBLIC ${THRIFT_LIBRARY}) -target_include_directories(hivemetastore SYSTEM PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/hive-metastore) +add_library(${HIVE_METASTORE_LIBRARY} ${SRCS}) +target_link_libraries(${HIVE_METASTORE_LIBRARY} PUBLIC ${THRIFT_LIBRARY}) +target_include_directories(${HIVE_METASTORE_LIBRARY} SYSTEM PUBLIC ${HIVE_METASTORE_INCLUDE_DIR}) From 280866a01bca431d4806eeeaa522f78d58f66019 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 22 Nov 2021 14:37:12 +0800 Subject: [PATCH 0069/1260] fix typo --- cmake/find/thrift.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/find/thrift.cmake b/cmake/find/thrift.cmake index 2e7a8356108..08a4e3c3376 100644 --- a/cmake/find/thrift.cmake +++ b/cmake/find/thrift.cmake @@ -31,4 +31,4 @@ else() endif() endif () -message (STATUS "Using_THRIFT=${USE_THRIFT}: ${THRIFT_INCLUDE_DIR} : ${THRIFT_LIBRARY}") +message (STATUS "Using thrift=${USE_THRIFT}: ${THRIFT_INCLUDE_DIR} : ${THRIFT_LIBRARY}") From f9edfcec610c634859e1a8815e03848a2f312f47 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 22 Nov 2021 17:31:53 +0800 Subject: [PATCH 0070/1260] fix build error --- src/Storages/Hive/StorageHive.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 9ef9b852d4b..4c6e331adb6 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -3,6 +3,7 @@ #if USE_HIVE #include +#include #include #include #include From e0f11343c70b6a94c6a6869de3d2103d98367419 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 22 Nov 2021 22:38:33 +0800 Subject: [PATCH 0071/1260] commit again --- cmake/find/hive-metastore.cmake | 2 ++ 1 file changed, 2 insertions(+) diff --git a/cmake/find/hive-metastore.cmake b/cmake/find/hive-metastore.cmake index 25358648b8e..a26eedfd505 100644 --- a/cmake/find/hive-metastore.cmake +++ b/cmake/find/hive-metastore.cmake @@ -11,6 +11,8 @@ if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/hive-metastore") elseif (NOT USE_THRIFT) message(WARNING "Thrift is not found, which is needed by Hive") set(USE_HIVE 0) +elseif (NOT USE_HDFS) + message(WARNING "HDFS is not found, which is needed by Hive") elseif (NOT USE_ORC OR NOT USE_ARROW OR NOT USE_PARQUET) message(WARNING "ORC/Arrow/Parquet is not found, which are needed by Hive") set(USE_HIVE 0) From 8be69c921c63660ab9bf8a9d636cdadba4b2811d Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 22 Nov 2021 22:39:57 +0800 Subject: [PATCH 0072/1260] commit again --- cmake/find/hive-metastore.cmake | 1 + 1 file changed, 1 insertion(+) diff --git a/cmake/find/hive-metastore.cmake b/cmake/find/hive-metastore.cmake index a26eedfd505..bc283cf8bd2 100644 --- a/cmake/find/hive-metastore.cmake +++ b/cmake/find/hive-metastore.cmake @@ -13,6 +13,7 @@ elseif (NOT USE_THRIFT) set(USE_HIVE 0) elseif (NOT USE_HDFS) message(WARNING "HDFS is not found, which is needed by Hive") + set(USE_HIVE 0) elseif (NOT USE_ORC OR NOT USE_ARROW OR NOT USE_PARQUET) message(WARNING "ORC/Arrow/Parquet is not found, which are needed by Hive") set(USE_HIVE 0) From ab0508dfc8781485af27d54c7811b8a8315320d2 Mon Sep 17 00:00:00 2001 From: lgbo Date: Wed, 24 Nov 2021 09:39:29 +0800 Subject: [PATCH 0073/1260] Update src/IO/RemoteReadBufferCache.h fixed code style Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/IO/RemoteReadBufferCache.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 5228c53ddd6..bc270aef401 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -197,7 +197,7 @@ public: inline bool hasInitialized() const { return inited; } std::tuple, RemoteReadBufferCacheError> createReader( - const RemoteFileMeta &remote_file_meta, + const RemoteFileMeta & remote_file_meta, std::shared_ptr & readbuffer); private: From 13226c9bbfda03b768d07ee03252cb243f3f45b4 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 24 Nov 2021 16:35:11 +0800 Subject: [PATCH 0074/1260] fix code typo --- src/Interpreters/Context.cpp | 6 +++--- src/Storages/Hive/HiveCommon.cpp | 9 +++++---- src/Storages/Hive/HiveCommon.h | 4 ++-- src/Storages/Hive/HiveFile.cpp | 16 ++++++++-------- src/Storages/Hive/HiveFile.h | 23 ++++++++++------------- src/Storages/Hive/HiveSettings.h | 6 +++--- src/Storages/Hive/StorageHive.cpp | 2 +- 7 files changed, 32 insertions(+), 34 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 34f3b82672a..42ac5e2d44e 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1842,9 +1842,9 @@ HMSClientPtr Context::getHMSClient(const String & name) const if (it == shared->hive_metastore_clients.end() || it->second->isExpired()) { // connect to hive metastore - Poco::URI hms_url(name); - const auto& host = hms_url.getHost(); - auto port = hms_url.getPort(); + Poco::URI hive_metastore_url(name); + const auto & host = hive_metastore_url.getHost(); + auto port = hive_metastore_url.getPort(); std::shared_ptr socket = std::make_shared(host, port); socket->setKeepAlive(true); diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 5e76df5328e..ad59c3e2627 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -133,12 +133,13 @@ std::vector HMSClient::HiveTableMeta::getLocationFiles(cons auto fs_builder = createHDFSBuilder(getNameNodeUrl(table->sd.location), getContext()->getGlobalContext()->getConfigRef()); auto fs = createHDFSFS(fs_builder.get()); Poco::URI uri(location); - HDFSFileInfo ls; - ls.file_info = hdfsListDirectory(fs.get(), uri.getPath().c_str(), &ls.length); + HDFSFileInfo dir_info; + dir_info.file_info = hdfsListDirectory(fs.get(), uri.getPath().c_str(), &dir_info.length); auto result = std::make_shared>(); - for (int i = 0; i < ls.length; ++i) + for (int i = 0; i < dir_info.length; ++i) { - auto & finfo = ls.file_info[i]; + auto & finfo = dir_info.file_info[i]; + /// skip directories and empty files, mKind value 'D' represents directory, otherwise file if (finfo.mKind != 'D' && finfo.mSize > 0) result->emplace_back(String(finfo.mName), finfo.mLastMod, finfo.mSize); } diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index db8877471b5..61f59fd5715 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -20,9 +20,9 @@ public: struct FileInfo { FileInfo() = default; - FileInfo(const std::string & path_, UInt64 ts_, size_t size_) : path(path_), last_mod_ts(ts_), size(size_) { } + FileInfo(const std::string & path_, UInt64 last_modify_time_, size_t size_) : path(path_), last_modify_time(last_modify_time_), size(size_) { } std::string path; - UInt64 last_mod_ts; // in ms + UInt64 last_modify_time; // in ms size_t size; }; diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 665a086677c..a8c9284c717 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -24,20 +24,20 @@ namespace DB { -template -Range createRangeFromOrcStatistics(const S * stats) +template +Range createRangeFromOrcStatistics(const StatisticsType * stats) { if (stats->hasMinimum() && stats->hasMaximum()) { - return Range(T(stats->getMinimum()), true, T(stats->getMaximum()), true); + return Range(FieldType(stats->getMinimum()), true, FieldType(stats->getMaximum()), true); } else if (stats->hasMinimum()) { - return Range::createLeftBounded(T(stats->getMinimum()), true); + return Range::createLeftBounded(FieldType(stats->getMinimum()), true); } else if (stats->hasMaximum()) { - return Range::createRightBounded(T(stats->getMaximum()), true); + return Range::createRightBounded(FieldType(stats->getMaximum()), true); } else { @@ -45,12 +45,12 @@ Range createRangeFromOrcStatistics(const S * stats) } } -template -Range createRangeFromParquetStatistics(std::shared_ptr stats) +template +Range createRangeFromParquetStatistics(std::shared_ptr stats) { if (!stats->HasMinMax()) return Range(); - return Range(T(stats->min()), true, T(stats->max()), true); + return Range(FieldType(stats->min()), true, FieldType(stats->max()), true); } Range createRangeFromParquetStatistics(std::shared_ptr stats) diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index 152c93c7a52..0be2d7a9573 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -54,7 +54,7 @@ public: const FieldVector & values_, const String & namenode_url_, const String & path_, - UInt64 ts_, + UInt64 last_modify_time_, size_t size_, const NamesAndTypesList & index_names_and_types_, const std::shared_ptr & storage_settings_, @@ -63,14 +63,11 @@ public: , partition_values(values_) , namenode_url(namenode_url_) , path(path_) - , last_mod_ts(ts_) + , last_modify_time(last_modify_time_) , size(size_) , index_names_and_types(index_names_and_types_) , storage_settings(storage_settings_) { - // std::cout << "1delim:" << storage_settings->hive_text_field_delimeter << std::endl; - // std::cout << "1disable orc:" << storage_settings->disable_orc_stripe_minmax_index << std::endl; - // std::cout << "1disable parquet:" << storage_settings->disable_parquet_rowgroup_minmax_index << std::endl; } virtual ~IHiveFile() = default; @@ -123,14 +120,14 @@ public: return boost::algorithm::join(strs, "|"); } - inline UInt64 getLastModTs() const { return last_mod_ts; } + inline UInt64 getLastModTs() const { return last_modify_time; } inline size_t getSize() const { return size; } protected: FieldVector partition_values; String namenode_url; String path; - UInt64 last_mod_ts; + UInt64 last_modify_time; size_t size; NamesAndTypesList index_names_and_types; std::shared_ptr minmax_idx; @@ -146,12 +143,12 @@ public: const FieldVector & values_, const String & namenode_url_, const String & path_, - UInt64 ts_, + UInt64 last_modify_time_, size_t size_, const NamesAndTypesList & index_names_and_types_, const std::shared_ptr & hive_settings_, ContextPtr context_) - : IHiveFile(values_, namenode_url_, path_, ts_, size_, index_names_and_types_, hive_settings_, context_) + : IHiveFile(values_, namenode_url_, path_, last_modify_time_, size_, index_names_and_types_, hive_settings_, context_) { } @@ -167,12 +164,12 @@ public: const FieldVector & values_, const String & namenode_url_, const String & path_, - UInt64 ts_, + UInt64 last_modify_time_, size_t size_, const NamesAndTypesList & index_names_and_types_, const std::shared_ptr & hive_settings_, ContextPtr context_) - : IHiveFile(values_, namenode_url_, path_, ts_, size_, index_names_and_types_, hive_settings_, context_) + : IHiveFile(values_, namenode_url_, path_, last_modify_time_, size_, index_names_and_types_, hive_settings_, context_) { } @@ -202,12 +199,12 @@ public: const FieldVector & values_, const String & namenode_url_, const String & path_, - UInt64 ts_, + UInt64 last_modify_time_, size_t size_, const NamesAndTypesList & index_names_and_types_, const std::shared_ptr & hive_settings_, ContextPtr context_) - : IHiveFile(values_, namenode_url_, path_, ts_, size_, index_names_and_types_, hive_settings_, context_) + : IHiveFile(values_, namenode_url_, path_, last_modify_time_, size_, index_names_and_types_, hive_settings_, context_) { } diff --git a/src/Storages/Hive/HiveSettings.h b/src/Storages/Hive/HiveSettings.h index 36ec566448d..406595b4741 100644 --- a/src/Storages/Hive/HiveSettings.h +++ b/src/Storages/Hive/HiveSettings.h @@ -14,9 +14,9 @@ class ASTStorage; #define HIVE_RELATED_SETTINGS(M) \ M(Char, hive_text_field_delimeter, '\x01', "How to split one row of hive data with format text", 0) \ - M(Bool, disable_orc_stripe_minmax_index, true, "Disable using ORC stripe level minmax index.", 0) \ - M(Bool, disable_parquet_rowgroup_minmax_index, true, "Disable using Parquet row-group level minmax index.", 0) \ - M(Bool, disable_orc_file_minmax_index, false, "Disable using ORC file level minmax index.", 0) + M(Bool, enable_orc_stripe_minmax_index, false, "Enable using ORC stripe level minmax index.", 0) \ + M(Bool, enable_parquet_rowgroup_minmax_index, false, "Enable using Parquet row-group level minmax index.", 0) \ + M(Bool, enable_orc_file_minmax_index, true, "Enable using ORC file level minmax index.", 0) #define LIST_OF_HIVE_SETTINGS(M) \ HIVE_RELATED_SETTINGS(M) \ diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 4c6e331adb6..435f9762b6d 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -410,7 +410,7 @@ Pipe StorageHive::read( fields, hdfs_namenode_url, hfile.path, - hfile.last_mod_ts, + hfile.last_modify_time, hfile.size, hivefile_name_types, storage_settings, From 4d750c9dc51b507d7148a250e21e894b5ccaac33 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 24 Nov 2021 17:52:02 +0800 Subject: [PATCH 0075/1260] fix code typo --- src/IO/SnappyWriteBuffer.h | 2 +- src/Interpreters/Context.cpp | 6 ++-- src/Interpreters/Context.h | 6 ++-- src/Storages/Hive/HiveCommon.cpp | 50 +++++++++++++++---------------- src/Storages/Hive/HiveCommon.h | 15 ++++++---- src/Storages/Hive/HiveFile.cpp | 4 +++ src/Storages/Hive/StorageHive.cpp | 8 ++--- 7 files changed, 49 insertions(+), 42 deletions(-) diff --git a/src/IO/SnappyWriteBuffer.h b/src/IO/SnappyWriteBuffer.h index 0d3a6f5904f..90fb8521c25 100644 --- a/src/IO/SnappyWriteBuffer.h +++ b/src/IO/SnappyWriteBuffer.h @@ -20,7 +20,7 @@ public: ~SnappyWriteBuffer() override; - void finalize() override { finish(); } + void finalizeImpl() override { finish(); } private: void nextImpl() override; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 42ac5e2d44e..d6562336d9d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -174,7 +174,7 @@ struct ContextSharedPart ConfigurationPtr auxiliary_zookeepers_config; /// Stores auxiliary zookeepers configs #if USE_HIVE - mutable std::map hive_metastore_clients; /// Map for hive metastore clients + mutable std::map hive_metastore_clients; /// Map for hive metastore clients #endif String interserver_io_host; /// The host name by which this server is available for other servers. @@ -1830,7 +1830,7 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const } #if USE_HIVE -HMSClientPtr Context::getHMSClient(const String & name) const +HiveMetastoreClientPtr Context::getHiveMetastoreClient(const String & name) const { using namespace apache::thrift; using namespace apache::thrift::protocol; @@ -1865,7 +1865,7 @@ HMSClientPtr Context::getHMSClient(const String & name) const if (it == shared->hive_metastore_clients.end()) { - HMSClientPtr hms_client = std::make_shared(std::move(client), shared_from_this()); + HiveMetastoreClientPtr hms_client = std::make_shared(std::move(client), shared_from_this()); shared->hive_metastore_clients[name] = hms_client; return hms_client; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index a9fca5f58fe..dcb1b353053 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -176,8 +176,8 @@ private: }; #if USE_HIVE -class HMSClient; -using HMSClientPtr = std::shared_ptr; +class HiveMetastoreClient; +using HiveMetastoreClientPtr = std::shared_ptr; #endif /** A set of known objects that can be used in the query. @@ -694,7 +694,7 @@ public: void reloadZooKeeperIfChanged(const ConfigurationPtr & config) const; #if USE_HIVE - HMSClientPtr getHMSClient(const String & name) const; + HiveMetastoreClientPtr getHiveMetastoreClient(const String & name) const; #endif void setSystemZooKeeperLogAfterInitializationIfNeeded(); diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index ad59c3e2627..774ed78c44c 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -9,7 +9,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -std::shared_ptr HMSClient::getTableMeta(const std::string & db_name, const std::string & table_name) +std::shared_ptr HiveMetastoreClient::getTableMeta(const std::string & db_name, const std::string & table_name) { LOG_TRACE(log, "get table meta:" + db_name + ":" + table_name); std::lock_guard lock{mutex}; @@ -32,7 +32,7 @@ std::shared_ptr HMSClient::getTableMeta(const std::str } std::string cache_key = db_name + "." + table_name; - std::shared_ptr result = table_meta_cache.get(cache_key); + std::shared_ptr result = table_meta_cache.get(cache_key); bool update_cache = false; std::map old_partition_infos; std::map partition_infos; @@ -72,29 +72,29 @@ std::shared_ptr HMSClient::getTableMeta(const std::str if (update_cache) { LOG_INFO(log, "reload hive partition meta info:" + db_name + ":" + table_name); - result = std::make_shared(db_name, table_name, table, std::move(partition_infos), getContext()); + result = std::make_shared(db_name, table_name, table, std::move(partition_infos), getContext()); table_meta_cache.set(cache_key, result); } return result; } -void HMSClient::clearTableMeta(const std::string & db_name, const std::string & table_name) +void HiveMetastoreClient::clearTableMeta(const std::string & db_name, const std::string & table_name) { std::lock_guard lock{mutex}; std::string cache_key = db_name + "." + table_name; - std::shared_ptr meta = table_meta_cache.get(cache_key); + std::shared_ptr meta = table_meta_cache.get(cache_key); if (meta) table_meta_cache.set(cache_key, nullptr); } -void HMSClient::setClient(std::shared_ptr c) +void HiveMetastoreClient::setClient(std::shared_ptr client_) { std::lock_guard lock{mutex}; - client = c; + client = client_; clearExpired(); } -bool HMSClient::PartitionInfo::equal(const Apache::Hadoop::Hive::Partition & other) +bool HiveMetastoreClient::PartitionInfo::equal(const Apache::Hadoop::Hive::Partition & other) { // parameters include keys:numRows,numFiles,rawDataSize,totalSize,transient_lastDdlTime auto it1 = partition.parameters.begin(); @@ -107,7 +107,7 @@ bool HMSClient::PartitionInfo::equal(const Apache::Hadoop::Hive::Partition & oth return (it1 == partition.parameters.end() && it2 == other.parameters.end()); } -std::vector HMSClient::HiveTableMeta::getPartitions() +std::vector HiveMetastoreClient::HiveTableMeta::getPartitions() { std::vector result; @@ -117,7 +117,7 @@ std::vector HMSClient::HiveTableMeta::getPartit return result; } -std::vector HMSClient::HiveTableMeta::getLocationFiles(const std::string & location) +std::vector HiveMetastoreClient::HiveTableMeta::getLocationFiles(const std::string & location) { std::map::const_iterator it; if (!empty_partition_keys) @@ -126,8 +126,7 @@ std::vector HMSClient::HiveTableMeta::getLocationFiles(cons it = partition_infos.find(location); if (it == partition_infos.end()) throw Exception("invalid location " + location, ErrorCodes::BAD_ARGUMENTS); - if (it->second.files != nullptr) - return *(it->second.files); + return it->second.files; } auto fs_builder = createHDFSBuilder(getNameNodeUrl(table->sd.location), getContext()->getGlobalContext()->getConfigRef()); @@ -135,13 +134,15 @@ std::vector HMSClient::HiveTableMeta::getLocationFiles(cons Poco::URI uri(location); HDFSFileInfo dir_info; dir_info.file_info = hdfsListDirectory(fs.get(), uri.getPath().c_str(), &dir_info.length); - auto result = std::make_shared>(); + + std::vector result; + result.reserve(dir_info.length); for (int i = 0; i < dir_info.length; ++i) { - auto & finfo = dir_info.file_info[i]; + auto & file_info = dir_info.file_info[i]; /// skip directories and empty files, mKind value 'D' represents directory, otherwise file - if (finfo.mKind != 'D' && finfo.mSize > 0) - result->emplace_back(String(finfo.mName), finfo.mLastMod, finfo.mSize); + if (file_info.mKind != 'D' && file_info.mSize > 0) + result.emplace_back(String(file_info.mName), file_info.mLastMod, file_info.mSize); } if (!empty_partition_keys) @@ -149,10 +150,10 @@ std::vector HMSClient::HiveTableMeta::getLocationFiles(cons std::lock_guard lock{mutex}; partition_infos[location].files = result; } - return *result; + return result; } -std::vector HMSClient::HiveTableMeta::getLocationFiles(const HDFSFSPtr & fs, const std::string & location) +std::vector HiveMetastoreClient::HiveTableMeta::getLocationFiles(const HDFSFSPtr & fs, const std::string & location) { std::map::const_iterator it; if (!empty_partition_keys) @@ -161,19 +162,18 @@ std::vector HMSClient::HiveTableMeta::getLocationFiles(cons it = partition_infos.find(location); if (it == partition_infos.end()) throw Exception("invalid location " + location, ErrorCodes::BAD_ARGUMENTS); - if (it->second.files != nullptr) - return *(it->second.files); + return it->second.files; } Poco::URI location_uri(location); HDFSFileInfo ls; ls.file_info = hdfsListDirectory(fs.get(), location_uri.getPath().c_str(), &ls.length); - auto result = std::make_shared>(); + std::vector result; for (int i = 0; i < ls.length; ++i) { - auto & finfo = ls.file_info[i]; - if (finfo.mKind != 'D' && finfo.mSize > 0) - result->emplace_back(String(finfo.mName), finfo.mLastMod, finfo.mSize); + auto & file_info = ls.file_info[i]; + if (file_info.mKind != 'D' && file_info.mSize > 0) + result.emplace_back(String(file_info.mName), file_info.mLastMod, file_info.mSize); } if (!empty_partition_keys) @@ -181,7 +181,7 @@ std::vector HMSClient::HiveTableMeta::getLocationFiles(cons std::lock_guard lock{mutex}; partition_infos[location].files = result; } - return *result; + return result; } diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index 61f59fd5715..e70a4c8871d 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -14,22 +14,25 @@ namespace DB { -class HMSClient : public WithContext +class HiveMetastoreClient : public WithContext { public: struct FileInfo { - FileInfo() = default; - FileInfo(const std::string & path_, UInt64 last_modify_time_, size_t size_) : path(path_), last_modify_time(last_modify_time_), size(size_) { } std::string path; UInt64 last_modify_time; // in ms size_t size; + + FileInfo() = default; + FileInfo(const FileInfo &) = default; + FileInfo(const std::string & path_, UInt64 last_modify_time_, size_t size_) : path(path_), last_modify_time(last_modify_time_), size(size_) {} }; struct PartitionInfo { Apache::Hadoop::Hive::Partition partition; - std::shared_ptr> files; + std::vector files; + bool equal(const Apache::Hadoop::Hive::Partition & other); }; @@ -71,14 +74,14 @@ public: const bool empty_partition_keys; }; - explicit HMSClient(std::shared_ptr client_, ContextPtr context_) + explicit HiveMetastoreClient(std::shared_ptr client_, ContextPtr context_) : WithContext(context_), client(client_), table_meta_cache(1000) { } std::shared_ptr getTableMeta(const std::string & db_name, const std::string & table_name); void clearTableMeta(const std::string & db_name, const std::string & table_name); - void setClient(std::shared_ptr c); + void setClient(std::shared_ptr client_); inline bool isExpired() const { return expired; } inline void setExpired() { expired = true; } inline void clearExpired() { expired = false; } diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index a8c9284c717..17f1ecfd73b 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -27,6 +27,8 @@ namespace DB template Range createRangeFromOrcStatistics(const StatisticsType * stats) { + /// We must check if there are minumum or maximum values in statistics in case of + /// null values or NaN/Inf values of double type. if (stats->hasMinimum() && stats->hasMaximum()) { return Range(FieldType(stats->getMinimum()), true, FieldType(stats->getMaximum()), true); @@ -48,6 +50,8 @@ Range createRangeFromOrcStatistics(const StatisticsType * stats) template Range createRangeFromParquetStatistics(std::shared_ptr stats) { + /// We must check if there are minumum or maximum values in statistics in case of + /// null values or NaN/Inf values of double type. if (!stats->HasMinMax()) return Range(); return Range(FieldType(stats->min()), true, FieldType(stats->max()), true); diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 435f9762b6d..3f49d193fc3 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -66,7 +66,7 @@ class HiveSource : public SourceWithProgress, WithContext public: struct SourcesInfo { - HMSClientPtr hms_client; + HiveMetastoreClientPtr hms_client; std::string database; std::string table_name; HiveFiles hive_files; @@ -254,7 +254,7 @@ StorageHive::StorageHive( storage_metadata.setComment(comment_); setInMemoryMetadata(storage_metadata); - auto hms_client = getContext()->getHMSClient(hms_url); + auto hms_client = getContext()->getHiveMetastoreClient(hms_url); auto table_meta = hms_client->getTableMeta(hive_database, hive_table); hdfs_namenode_url = getNameNodeUrl(table_meta->getTable()->sd.location); @@ -387,7 +387,7 @@ Pipe StorageHive::read( { HDFSBuilderWrapper builder = createHDFSBuilder(hdfs_namenode_url, context_->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); - auto hms_client = context_->getHMSClient(hms_url); + auto hms_client = context_->getHiveMetastoreClient(hms_url); auto table_meta_cntrl = hms_client->getTableMeta(hive_database, hive_table); // List files under partition directory in HDFS @@ -397,7 +397,7 @@ Pipe StorageHive::read( HiveFiles hive_files; // hive files to read std::mutex hive_files_mutex; // Mutext to protect hive_files, which maybe appended in multiple threads - auto append_hive_files = [&](const HMSClient::FileInfo & hfile, const FieldVector & fields) + auto append_hive_files = [&](const HiveMetastoreClient::FileInfo & hfile, const FieldVector & fields) { String filename = getBaseName(hfile.path); From ffc963fb74bc2a9a6a40be03c9d79ac408912ac6 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 24 Nov 2021 18:05:35 +0800 Subject: [PATCH 0076/1260] fix build --- cmake/find/thrift.cmake | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmake/find/thrift.cmake b/cmake/find/thrift.cmake index 08a4e3c3376..08eeb60915e 100644 --- a/cmake/find/thrift.cmake +++ b/cmake/find/thrift.cmake @@ -6,10 +6,10 @@ if (NOT ENABLE_THRIFT) return() endif() -option(USE_INTERNAL_THRIFT_LIBRARY "Set to FALSE to use system thrift library instead of bundled" ${NOT_UNBUNDLED}) +option(USE_INTERNAL_THRIFT_LIBRARY "Set to FALSE to use system thrift library instead of bundled" ON) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/thrift") if (USE_INTERNAL_THRIFT_LIBRARY) - message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/thrift is missing. to fix try run: \n git submodule update --init --recursive") set(USE_INTERNAL_THRIFT_LIBRARY 0) endif () endif() From 58c0357436a6d66bb8760a57761e28ec79e10b63 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 25 Nov 2021 19:16:43 +0800 Subject: [PATCH 0077/1260] Update ErrorCodes.cpp fix typo --- src/Common/ErrorCodes.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 1748b7f44ae..51df36fd068 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -601,7 +601,7 @@ M(631, UNKNOWN_FILE_SIZE) \ M(650, SNAPPY_UNCOMPRESS_FAILED) \ M(651, SNAPPY_COMPRESS_FAILED) \ - M(652, NO_HIVEMETASTORE) \ + M(652, NO_HIVEMETASTORE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ From 08cc4306b6fc0c9f69f33088fc061f2a75c1c9ee Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 27 Nov 2021 18:56:43 +0800 Subject: [PATCH 0078/1260] fix typos --- src/Storages/Hive/HiveFile.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 17f1ecfd73b..8edd4455f53 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -27,7 +27,7 @@ namespace DB template Range createRangeFromOrcStatistics(const StatisticsType * stats) { - /// We must check if there are minumum or maximum values in statistics in case of + /// We must check if there are minimum or maximum values in statistics in case of /// null values or NaN/Inf values of double type. if (stats->hasMinimum() && stats->hasMaximum()) { @@ -50,7 +50,7 @@ Range createRangeFromOrcStatistics(const StatisticsType * stats) template Range createRangeFromParquetStatistics(std::shared_ptr stats) { - /// We must check if there are minumum or maximum values in statistics in case of + /// We must check if there are minimum or maximum values in statistics in case of /// null values or NaN/Inf values of double type. if (!stats->HasMinMax()) return Range(); From df68b5b1a184cb4c599f96912cb1273b36596182 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 29 Nov 2021 10:46:47 +0800 Subject: [PATCH 0079/1260] Update ErrorCodes.cpp remove blank --- src/Common/ErrorCodes.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f1dec528f5c..700f6ce17c0 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -602,7 +602,7 @@ M(632, UNEXPECTED_DATA_AFTER_PARSED_VALUE) \ M(650, SNAPPY_UNCOMPRESS_FAILED) \ M(651, SNAPPY_COMPRESS_FAILED) \ - M(652, NO_HIVEMETASTORE) \ + M(652, NO_HIVEMETASTORE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ From e67407b5c53e282ffa16f5c8f18765c29c6f8872 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 29 Nov 2021 11:30:11 +0800 Subject: [PATCH 0080/1260] fix code style --- src/IO/RemoteReadBufferCache.cpp | 105 ++++++++++++++++--------------- src/IO/RemoteReadBufferCache.h | 44 ++++++------- src/Storages/Hive/HiveCommon.cpp | 20 +++--- src/Storages/Hive/HiveCommon.h | 8 +-- 4 files changed, 88 insertions(+), 89 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index b93de46398c..057a896d8e6 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -1,16 +1,16 @@ +#include +#include +#include #include -#include -#include -#include -#include #include #include #include #include -#include "Common/Exception.h" -#include -#include -#include +#include +#include +#include +#include +#include namespace DB { @@ -21,10 +21,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -std::shared_ptr -RemoteCacheController::recover( - const std::filesystem::path & local_path_, - std::function const & finish_callback) +std::shared_ptr RemoteCacheController::recover( + const std::filesystem::path & local_path_, std::function const & finish_callback) { const auto & dir_handle = local_path_; std::filesystem::path data_file = local_path_ / "data.bin"; @@ -50,7 +48,7 @@ RemoteCacheController::recover( } auto file_size = std::filesystem::file_size(data_file); - RemoteFileMeta remote_file_meta (schema, cluster, remote_path, modification_ts, file_size); + RemoteFileMeta remote_file_meta(schema, cluster, remote_path, modification_ts, file_size); auto cntrl = std::make_shared(remote_file_meta, local_path_, 0, nullptr, finish_callback); cntrl->download_finished = true; cntrl->current_offset = file_size; @@ -89,7 +87,7 @@ RemoteCacheController::RemoteCacheController( jobj.set("remote_path", remote_path); jobj.set("downloaded", "false"); jobj.set("last_modification_timestamp", last_modification_timestamp); - std::stringstream buf;// STYLE_CHECK_ALLOW_STD_STRING_STREAM + std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM jobj.stringify(buf); std::ofstream meta_file(local_path_ / "meta.txt", std::ios::out); meta_file.write(buf.str().c_str(), buf.str().size()); @@ -127,8 +125,7 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs void RemoteCacheController::backgroupDownload(std::function const & finish_callback) { - auto task = [this, finish_callback]() - { + auto task = [this, finish_callback]() { size_t unflush_bytes = 0; size_t total_bytes = 0; while (!remote_readbuffer->eof()) @@ -163,7 +160,9 @@ void RemoteCacheController::backgroupDownload(std::functionscheduleOrThrow(task); } @@ -183,7 +182,7 @@ void RemoteCacheController::flush(bool need_flush_meta_) jobj.set("remote_path", remote_path); jobj.set("downloaded", download_finished ? "true" : "false"); jobj.set("last_modification_timestamp", last_modification_timestamp); - std::stringstream buf;// STYLE_CHECK_ALLOW_STD_STRING_STREAM + std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM jobj.stringify(buf); std::ofstream meta_file(local_path / "meta.txt", std::ios::out); @@ -203,7 +202,7 @@ std::tuple RemoteCacheController::allocFile() { std::filesystem::path result_local_path; if (download_finished) - result_local_path = local_path / "data.bin"; + result_local_path = local_path / "data.bin"; FILE * fs = fopen((local_path / "data.bin").string().c_str(), "r"); if (fs == nullptr) return {fs, result_local_path}; @@ -284,9 +283,7 @@ RemoteReadBuffer::RemoteReadBuffer(size_t buff_size) : BufferWithOwnMemory RemoteReadBuffer::create( - const RemoteFileMeta &remote_file_meta_, - std::unique_ptr readbuffer) +std::unique_ptr RemoteReadBuffer::create(const RemoteFileMeta & remote_file_meta_, std::unique_ptr readbuffer) { size_t buff_size = DBMS_DEFAULT_BUFFER_SIZE; if (readbuffer != nullptr) @@ -313,8 +310,7 @@ std::unique_ptr RemoteReadBuffer::create( if (retry > 0) usleep(20 * retry); - std::tie(rrb->file_reader, error) - = RemoteReadBufferCache::instance().createReader(remote_file_meta_, srb); + std::tie(rrb->file_reader, error) = RemoteReadBufferCache::instance().createReader(remote_file_meta_, srb); retry++; } while (error == RemoteReadBufferCacheError::FILE_INVALID && retry < 10); if (rrb->file_reader == nullptr) @@ -403,10 +399,10 @@ RemoteReadBufferCache & RemoteReadBufferCache::instance() } void RemoteReadBufferCache::recoverCachedFilesMeta( - const std::filesystem::path & current_path, - size_t current_depth, - size_t max_depth, - std::function const & finish_callback) + const std::filesystem::path & current_path, + size_t current_depth, + size_t max_depth, + std::function const & finish_callback) { if (current_depth >= max_depth) { @@ -416,24 +412,21 @@ void RemoteReadBufferCache::recoverCachedFilesMeta( auto cache_controller = RemoteCacheController::recover(path, finish_callback); if (!cache_controller) continue; - auto &cell = caches[path]; + auto & cell = caches[path]; cell.cache_controller = cache_controller; cell.key_iterator = keys.insert(keys.end(), path); } return; } - for (auto const &dir : std::filesystem::directory_iterator{current_path}) + for (auto const & dir : std::filesystem::directory_iterator{current_path}) { recoverCachedFilesMeta(dir.path(), current_depth + 1, max_depth, finish_callback); } - } -void RemoteReadBufferCache::initOnce(const std::filesystem::path & dir, - size_t limit_size_, - size_t bytes_read_before_flush_, - size_t max_threads) +void RemoteReadBufferCache::initOnce( + const std::filesystem::path & dir, size_t limit_size_, size_t bytes_read_before_flush_, size_t max_threads) { LOG_TRACE(log, "init local cache. path: {}, limit {}", dir.string(), limit_size_); local_path_prefix = dir; @@ -448,8 +441,7 @@ void RemoteReadBufferCache::initOnce(const std::filesystem::path & dir, LOG_INFO(log, "{} not exists. this cache will be disable", local_path_prefix); return; } - auto recover_task = [this, root_dir]() - { + auto recover_task = [this, root_dir]() { auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; std::lock_guard lock(this->mutex); // two level dir. // @@ -465,12 +457,11 @@ std::filesystem::path RemoteReadBufferCache::calculateLocalPath(const RemoteFile std::string full_path = meta.schema + ":" + meta.cluster + ":" + meta.path; UInt128 hashcode = sipHash128(full_path.c_str(), full_path.size()); std::string hashcode_str = getHexUIntLowercase(hashcode); - return std::filesystem::path(local_path_prefix) / hashcode_str.substr(0,3) / hashcode_str; + return std::filesystem::path(local_path_prefix) / hashcode_str.substr(0, 3) / hashcode_str; } -std::tuple, RemoteReadBufferCacheError> RemoteReadBufferCache::createReader( - const RemoteFileMeta &remote_file_meta, - std::shared_ptr & readbuffer) +std::tuple, RemoteReadBufferCacheError> +RemoteReadBufferCache::createReader(const RemoteFileMeta & remote_file_meta, std::shared_ptr & readbuffer) { // If something is wrong on startup, rollback to read from the original ReadBuffer if (!hasInitialized()) @@ -490,8 +481,10 @@ std::tuple, RemoteReadBufferCacheError> R // if the file has been update on remote side, we need to redownload it if (cache_iter->second.cache_controller->getLastModificationTimestamp() != last_modification_timestamp) { - LOG_TRACE(log, - "remote file has been updated. " + remote_path + ":" + std::to_string(cache_iter->second.cache_controller->getLastModificationTimestamp()) + "->" + LOG_TRACE( + log, + "remote file has been updated. " + remote_path + ":" + + std::to_string(cache_iter->second.cache_controller->getLastModificationTimestamp()) + "->" + std::to_string(last_modification_timestamp)); cache_iter->second.cache_controller->markInvalid(); } @@ -499,7 +492,9 @@ std::tuple, RemoteReadBufferCacheError> R { // move the key to the list end keys.splice(keys.end(), keys, cache_iter->second.key_iterator); - return {std::make_shared(cache_iter->second.cache_controller.get(), file_size), RemoteReadBufferCacheError::OK}; + return { + std::make_shared(cache_iter->second.cache_controller.get(), file_size), + RemoteReadBufferCacheError::OK}; } } @@ -511,7 +506,9 @@ std::tuple, RemoteReadBufferCacheError> R { // move the key to the list end, this case should not happen? keys.splice(keys.end(), keys, cache_iter->second.key_iterator); - return {std::make_shared(cache_iter->second.cache_controller.get(), file_size), RemoteReadBufferCacheError::OK}; + return { + std::make_shared(cache_iter->second.cache_controller.get(), file_size), + RemoteReadBufferCacheError::OK}; } else { @@ -530,7 +527,8 @@ std::tuple, RemoteReadBufferCacheError> R std::filesystem::create_directories(local_path); auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; - auto cache_cntrl = std::make_shared(remote_file_meta, local_path, local_cache_bytes_read_before_flush, readbuffer, callback); + auto cache_cntrl + = std::make_shared(remote_file_meta, local_path, local_cache_bytes_read_before_flush, readbuffer, callback); CacheCell cc; cc.cache_controller = cache_cntrl; cc.key_iterator = keys.insert(keys.end(), local_path); @@ -548,7 +546,8 @@ bool RemoteReadBufferCache::clearLocalCache() if (!cntrl->isValid() && cntrl->closable()) { LOG_TRACE(log, "clear invalid cache: " + *it); - total_size = total_size > cache_it->second.cache_controller->size() ? total_size - cache_it->second.cache_controller->size() : 0; + total_size + = total_size > cache_it->second.cache_controller->size() ? total_size - cache_it->second.cache_controller->size() : 0; cntrl->close(); it = keys.erase(it); caches.erase(cache_it); @@ -568,12 +567,18 @@ bool RemoteReadBufferCache::clearLocalCache() } if (cache_it->second.cache_controller->closable()) { - total_size = total_size > cache_it->second.cache_controller->size() ? total_size - cache_it->second.cache_controller->size() : 0; + total_size + = total_size > cache_it->second.cache_controller->size() ? total_size - cache_it->second.cache_controller->size() : 0; cache_it->second.cache_controller->close(); caches.erase(cache_it); it = keys.erase(it); - LOG_TRACE(log, "clear local file {} for {}. key size:{}. next{}", cache_it->second.cache_controller->getLocalPath().string(), - cache_it->second.cache_controller->getRemotePath(), keys.size(), *it); + LOG_TRACE( + log, + "clear local file {} for {}. key size:{}. next{}", + cache_it->second.cache_controller->getLocalPath().string(), + cache_it->second.cache_controller->getRemotePath(), + keys.size(), + *it); } else break; diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index bc270aef401..b185fc57b47 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -14,7 +14,7 @@ namespace DB { -enum class RemoteReadBufferCacheError :int8_t +enum class RemoteReadBufferCacheError : int8_t { OK, NOT_INIT = 10, @@ -27,17 +27,14 @@ enum class RemoteReadBufferCacheError :int8_t struct RemoteFileMeta { RemoteFileMeta( - const std::string & schema_, - const std::string & cluster_, - const std::string & path_, - UInt64 last_modification_timestamp_, - size_t file_size_): - schema(schema_), - cluster(cluster_), - path(path_), - last_modification_timestamp(last_modification_timestamp_), - file_size(file_size_) - {} + const std::string & schema_, + const std::string & cluster_, + const std::string & path_, + UInt64 last_modification_timestamp_, + size_t file_size_) + : schema(schema_), cluster(cluster_), path(path_), last_modification_timestamp(last_modification_timestamp_), file_size(file_size_) + { + } std::string schema; // Hive, S2 etc. std::string cluster; @@ -53,7 +50,7 @@ class RemoteCacheController { public: RemoteCacheController( - const RemoteFileMeta &meta, + const RemoteFileMeta & meta, const std::filesystem::path & local_path_, size_t cache_bytes_before_flush_, std::shared_ptr readbuffer_, @@ -167,9 +164,7 @@ class RemoteReadBuffer : public BufferWithOwnMemory public: explicit RemoteReadBuffer(size_t buff_size); ~RemoteReadBuffer() override; - static std::unique_ptr create( - const RemoteFileMeta &remote_file_meta_, - std::unique_ptr readbuffer); + static std::unique_ptr create(const RemoteFileMeta & remote_file_meta_, std::unique_ptr readbuffer); bool nextImpl() override; inline bool seekable() { return file_reader != nullptr && file_reader->size() > 0; } @@ -191,14 +186,13 @@ public: ~RemoteReadBufferCache(); // global instance static RemoteReadBufferCache & instance(); - std::shared_ptr GetThreadPool(){ return threadPool; } + std::shared_ptr GetThreadPool() { return threadPool; } void initOnce(const std::filesystem::path & dir, size_t limit_size, size_t bytes_read_before_flush_, size_t max_threads); inline bool hasInitialized() const { return inited; } - std::tuple, RemoteReadBufferCacheError> createReader( - const RemoteFileMeta & remote_file_meta, - std::shared_ptr & readbuffer); + std::tuple, RemoteReadBufferCacheError> + createReader(const RemoteFileMeta & remote_file_meta, std::shared_ptr & readbuffer); private: std::string local_path_prefix; @@ -219,13 +213,13 @@ private: std::list keys; std::map caches; - std::filesystem::path calculateLocalPath(const RemoteFileMeta &meta); + std::filesystem::path calculateLocalPath(const RemoteFileMeta & meta); void recoverCachedFilesMeta( - const std::filesystem::path & current_path, - size_t current_depth, - size_t max_depth, - std::function const & finish_callback); + const std::filesystem::path & current_path, + size_t current_depth, + size_t max_depth, + std::function const & finish_callback); bool clearLocalCache(); }; diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 774ed78c44c..da9e2b31abc 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -9,7 +9,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -std::shared_ptr HiveMetastoreClient::getTableMeta(const std::string & db_name, const std::string & table_name) +std::shared_ptr HiveMetastoreClient::getTableMetadata(const std::string & db_name, const std::string & table_name) { LOG_TRACE(log, "get table meta:" + db_name + ":" + table_name); std::lock_guard lock{mutex}; @@ -32,7 +32,7 @@ std::shared_ptr HiveMetastoreClient::getTabl } std::string cache_key = db_name + "." + table_name; - std::shared_ptr result = table_meta_cache.get(cache_key); + std::shared_ptr result = table_meta_cache.get(cache_key); bool update_cache = false; std::map old_partition_infos; std::map partition_infos; @@ -49,15 +49,15 @@ std::shared_ptr HiveMetastoreClient::getTabl for (const auto & partition : partitions) { - auto & pinfo = partition_infos[partition.sd.location]; - pinfo.partition = partition; + auto & partition_info = partition_infos[partition.sd.location]; + partition_info.partition = partition; // query files under the partition by hdfs api is costly, we reuse the files in case the partition has no change if (result) { auto it = old_partition_infos.find(partition.sd.location); if (it != old_partition_infos.end() && it->second.equal(partition)) - pinfo.files = it->second.files; + partition_info.files = it->second.files; else update_cache = true; } @@ -72,7 +72,7 @@ std::shared_ptr HiveMetastoreClient::getTabl if (update_cache) { LOG_INFO(log, "reload hive partition meta info:" + db_name + ":" + table_name); - result = std::make_shared(db_name, table_name, table, std::move(partition_infos), getContext()); + result = std::make_shared(db_name, table_name, table, std::move(partition_infos), getContext()); table_meta_cache.set(cache_key, result); } return result; @@ -82,7 +82,7 @@ void HiveMetastoreClient::clearTableMeta(const std::string & db_name, const std: { std::lock_guard lock{mutex}; std::string cache_key = db_name + "." + table_name; - std::shared_ptr meta = table_meta_cache.get(cache_key); + std::shared_ptr meta = table_meta_cache.get(cache_key); if (meta) table_meta_cache.set(cache_key, nullptr); } @@ -107,7 +107,7 @@ bool HiveMetastoreClient::PartitionInfo::equal(const Apache::Hadoop::Hive::Parti return (it1 == partition.parameters.end() && it2 == other.parameters.end()); } -std::vector HiveMetastoreClient::HiveTableMeta::getPartitions() +std::vector HiveMetastoreClient::HiveTableMetadata::getPartitions() { std::vector result; @@ -117,7 +117,7 @@ std::vector HiveMetastoreClient::HiveTableMeta: return result; } -std::vector HiveMetastoreClient::HiveTableMeta::getLocationFiles(const std::string & location) +std::vector HiveMetastoreClient::HiveTableMetadata::getLocationFiles(const std::string & location) { std::map::const_iterator it; if (!empty_partition_keys) @@ -153,7 +153,7 @@ std::vector HiveMetastoreClient::HiveTableMeta::g return result; } -std::vector HiveMetastoreClient::HiveTableMeta::getLocationFiles(const HDFSFSPtr & fs, const std::string & location) +std::vector HiveMetastoreClient::HiveTableMetadata::getLocationFiles(const HDFSFSPtr & fs, const std::string & location) { std::map::const_iterator it; if (!empty_partition_keys) diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index e70a4c8871d..af056739b6b 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -37,10 +37,10 @@ public: }; // use for speeding up query metadata - struct HiveTableMeta : public WithContext + struct HiveTableMetadata : public WithContext { public: - HiveTableMeta( + HiveTableMetadata( const std::string & db_name_, const std::string & table_name_, std::shared_ptr table_, @@ -79,7 +79,7 @@ public: { } - std::shared_ptr getTableMeta(const std::string & db_name, const std::string & table_name); + std::shared_ptr getTableMetadata(const std::string & db_name, const std::string & table_name); void clearTableMeta(const std::string & db_name, const std::string & table_name); void setClient(std::shared_ptr client_); inline bool isExpired() const { return expired; } @@ -88,7 +88,7 @@ public: private: std::shared_ptr client; - LRUCache table_meta_cache; + LRUCache table_meta_cache; mutable std::mutex mutex; std::atomic expired{false}; From 919b439b7cc2c626798740a376e0f6f42f519e3c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 29 Nov 2021 11:37:07 +0800 Subject: [PATCH 0081/1260] remove useless code --- src/IO/ReadHelpers.h | 27 --------------------------- 1 file changed, 27 deletions(-) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index a7c876ed32b..b580d80c244 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1072,33 +1072,6 @@ inline void readCSVSimple(T & x, ReadBuffer & buf, bool read_bool_as_uint8 = fal assertChar(maybe_quote, buf); } -/* -inline void readCSVSimple(UInt8 & x, ReadBuffer & buf) -{ - if (buf.eof()) - throwReadAfterEOF(); - - char maybe_quote = *buf.position(); - - if (maybe_quote == '\'' || maybe_quote == '\"') - ++buf.position(); - - if (*buf.position() == 't' || *buf.position() == 'f') - { - bool tmp = false; - readBoolTextWord(tmp, buf); - x = tmp ? 1 : 0; - } - else - { - readText(x, buf); - } - - if (maybe_quote == '\'' || maybe_quote == '\"') - assertChar(maybe_quote, buf); -} -*/ - template inline std::enable_if_t, void> readCSV(T & x, ReadBuffer & buf, bool read_bool_as_uint8 = false) { readCSVSimple(x, buf, read_bool_as_uint8); } From 85cb7fc23ee105355cf9f8352ce049fe9192e29c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 29 Nov 2021 11:47:31 +0800 Subject: [PATCH 0082/1260] fix code style --- src/IO/RemoteReadBufferCache.cpp | 15 +++++---------- src/IO/RemoteReadBufferCache.h | 6 ++++-- 2 files changed, 9 insertions(+), 12 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 057a896d8e6..ff8fc1ac020 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -157,14 +157,9 @@ void RemoteCacheController::backgroupDownload(std::functionscheduleOrThrow(task); + RemoteReadBufferCache::instance().getThreadPool()->scheduleOrThrow(task); } void RemoteCacheController::flush(bool need_flush_meta_) @@ -389,7 +384,7 @@ RemoteReadBufferCache::RemoteReadBufferCache() = default; RemoteReadBufferCache::~RemoteReadBufferCache() { - threadPool->wait(); + thread_pool->wait(); } RemoteReadBufferCache & RemoteReadBufferCache::instance() @@ -432,7 +427,7 @@ void RemoteReadBufferCache::initOnce( local_path_prefix = dir; limit_size = limit_size_; local_cache_bytes_read_before_flush = bytes_read_before_flush_; - threadPool = std::make_shared(max_threads, 1000, 1000, false); + thread_pool = std::make_shared(max_threads, 1000, 1000, false); // scan local disk dir and recover the cache metas std::filesystem::path root_dir(local_path_prefix); @@ -449,7 +444,7 @@ void RemoteReadBufferCache::initOnce( this->inited = true; LOG_TRACE(this->log, "recovered from disk "); }; - GetThreadPool()->scheduleOrThrow(recover_task); + getThreadPool()->scheduleOrThrow(recover_task); } std::filesystem::path RemoteReadBufferCache::calculateLocalPath(const RemoteFileMeta & meta) diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index b185fc57b47..d85365604b8 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -128,6 +128,8 @@ private: size_t local_cache_bytes_read_before_flush; std::shared_ptr remote_readbuffer; std::unique_ptr out_file; + + Poco::Logger * log = &Poco::Logger::get("RemoteReadBufferCache"); }; /** @@ -186,7 +188,7 @@ public: ~RemoteReadBufferCache(); // global instance static RemoteReadBufferCache & instance(); - std::shared_ptr GetThreadPool() { return threadPool; } + std::shared_ptr getThreadPool() { return thread_pool; } void initOnce(const std::filesystem::path & dir, size_t limit_size, size_t bytes_read_before_flush_, size_t max_threads); inline bool hasInitialized() const { return inited; } @@ -197,7 +199,7 @@ public: private: std::string local_path_prefix; - std::shared_ptr threadPool; + std::shared_ptr thread_pool; std::atomic inited = false; std::mutex mutex; size_t limit_size = 0; From e8addd3243d78b4ce95cf81dae9f05aba4fc0262 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 29 Nov 2021 12:01:52 +0800 Subject: [PATCH 0083/1260] fix code style --- src/IO/RemoteReadBufferCache.cpp | 12 ++++++++---- src/IO/RemoteReadBufferCache.h | 2 +- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index ff8fc1ac020..bcb284e4bb6 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -24,12 +24,12 @@ namespace ErrorCodes std::shared_ptr RemoteCacheController::recover( const std::filesystem::path & local_path_, std::function const & finish_callback) { - const auto & dir_handle = local_path_; std::filesystem::path data_file = local_path_ / "data.bin"; std::filesystem::path meta_file = local_path_ / "meta.txt"; - if (!std::filesystem::exists(dir_handle) || !std::filesystem::exists(data_file) || !std::filesystem::exists(meta_file)) + auto * log = &Poco::Logger::get("RemoteCacheController"); + if (!std::filesystem::exists(data_file) || !std::filesystem::exists(meta_file)) { - LOG_ERROR(&Poco::Logger::get("RemoteCacheController"), "not exists directory:" + local_path_.string()); + LOG_ERROR(log, "Directory {} or file {}, {} does not exist", local_path_.string(), data_file.string(), meta_file.string()); return nullptr; } @@ -423,7 +423,11 @@ void RemoteReadBufferCache::recoverCachedFilesMeta( void RemoteReadBufferCache::initOnce( const std::filesystem::path & dir, size_t limit_size_, size_t bytes_read_before_flush_, size_t max_threads) { - LOG_TRACE(log, "init local cache. path: {}, limit {}", dir.string(), limit_size_); + LOG_INFO( + log, + "Initializing local cache for remote data sources. Local cache root path: {}, cache size limit: {}", + dir.string(), + limit_size_); local_path_prefix = dir; limit_size = limit_size_; local_cache_bytes_read_before_flush = bytes_read_before_flush_; diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index d85365604b8..4a93ec4be05 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -83,7 +83,7 @@ public: /** * called in LocalCachedFileReader read(), the reading process would be blocked until * enough data be downloaded. - * If the file has finished download, the process would be block + * If the file has finished download, the process would be unblock */ RemoteReadBufferCacheError waitMoreData(size_t start_offset_, size_t end_offset_); From 01f29fcf21aaa098833bb1aed55db6111f13d80c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 29 Nov 2021 12:17:22 +0800 Subject: [PATCH 0084/1260] fix code style --- src/IO/RemoteReadBufferCache.cpp | 12 ++++++------ src/IO/RemoteReadBufferCache.h | 4 +++- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index bcb284e4bb6..1c75fc10634 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -43,7 +43,7 @@ std::shared_ptr RemoteCacheController::recover( auto modification_ts = meta_jobj->get("last_modification_timestamp").convert(); if (downloaded == "false") { - LOG_ERROR(&Poco::Logger::get("RemoteCacheController"), "not a downloaded file: " + local_path_.string()); + LOG_ERROR(log, "not a downloaded file: " + local_path_.string()); return nullptr; } auto file_size = std::filesystem::file_size(data_file); @@ -189,11 +189,11 @@ RemoteCacheController::~RemoteCacheController() = default; void RemoteCacheController::close() { // delete the directory - LOG_TRACE(&Poco::Logger::get("RemoteCacheController"), "release local resource: " + remote_path + ", " + local_path.string()); + LOG_TRACE(log, "release local resource: " + remote_path + ", " + local_path.string()); std::filesystem::remove_all(local_path); } -std::tuple RemoteCacheController::allocFile() +std::pair RemoteCacheController::allocFile() { std::filesystem::path result_local_path; if (download_finished) @@ -262,7 +262,7 @@ size_t LocalCachedFileReader::size() return file_size; if (local_path.empty()) { - LOG_TRACE(&Poco::Logger::get("LocalCachedFileReader"), "empty local_path"); + LOG_TRACE(log, "empty local_path"); return 0; } @@ -280,6 +280,7 @@ RemoteReadBuffer::~RemoteReadBuffer() = default; std::unique_ptr RemoteReadBuffer::create(const RemoteFileMeta & remote_file_meta_, std::unique_ptr readbuffer) { + auto * log = &Poco::Logger::get("RemoteReadBuffer"); size_t buff_size = DBMS_DEFAULT_BUFFER_SIZE; if (readbuffer != nullptr) buff_size = readbuffer->internalBuffer().size(); @@ -310,7 +311,7 @@ std::unique_ptr RemoteReadBuffer::create(const RemoteFileMeta } while (error == RemoteReadBufferCacheError::FILE_INVALID && retry < 10); if (rrb->file_reader == nullptr) { - LOG_ERROR(&Poco::Logger::get("RemoteReadBuffer"), "allocate local file failed for " + remote_path + "{}", error); + LOG_ERROR(log, "allocate local file failed for " + remote_path + "{}", error); rrb->original_readbuffer = srb; } return rrb; @@ -335,7 +336,6 @@ bool RemoteReadBuffer::nextImpl() auto status = original_readbuffer->next(); // we don't need to worry about the memory buffer allocated in RemoteReadBuffer, since it is owned by // BufferWithOwnMemory, BufferWithOwnMemory would release it. - //LOG_TRACE(&Poco::Logger::get("RemoteReadBuffer"), "from original rb {} {}", original_readbuffer->buffer().size(), original_readbuffer->offset()); if (status) BufferBase::set(original_readbuffer->buffer().begin(), original_readbuffer->buffer().size(), original_readbuffer->offset()); return status; diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 4a93ec4be05..9a67559f320 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -66,7 +66,7 @@ public: * The second value of the return tuple is the local_path to store file. * It will be empty if the file has not been downloaded */ - std::tuple allocFile(); + std::pair allocFile(); void deallocFile(FILE * fs_); /** @@ -155,6 +155,8 @@ private: FILE * fs; std::filesystem::path local_path; RemoteCacheController * controller; + + Poco::Logger * log = &Poco::Logger::get("RemoteReadBufferCache"); }; /* From fd4462db646d0079d5eae33c61aa8bcdd754adbd Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 29 Nov 2021 13:01:03 +0800 Subject: [PATCH 0085/1260] fix code style --- src/IO/RemoteReadBufferCache.cpp | 107 +++++++++++++++--------------- src/IO/RemoteReadBufferCache.h | 48 +++++++------- src/Storages/Hive/StorageHive.cpp | 4 +- 3 files changed, 82 insertions(+), 77 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 1c75fc10634..3dffc0162cf 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -43,40 +44,40 @@ std::shared_ptr RemoteCacheController::recover( auto modification_ts = meta_jobj->get("last_modification_timestamp").convert(); if (downloaded == "false") { - LOG_ERROR(log, "not a downloaded file: " + local_path_.string()); + LOG_ERROR(log, "Local metadata for path {} exists, but the data was not downloaded", local_path_.string()); return nullptr; } auto file_size = std::filesystem::file_size(data_file); - RemoteFileMeta remote_file_meta(schema, cluster, remote_path, modification_ts, file_size); - auto cntrl = std::make_shared(remote_file_meta, local_path_, 0, nullptr, finish_callback); - cntrl->download_finished = true; - cntrl->current_offset = file_size; + RemoteFileMetadata remote_file_meta(schema, cluster, remote_path, modification_ts, file_size); + auto cache_controller = std::make_shared(remote_file_meta, local_path_, 0, nullptr, finish_callback); + cache_controller->download_finished = true; + cache_controller->current_offset = file_size; meta_fs.close(); - finish_callback(cntrl.get()); - return cntrl; + finish_callback(cache_controller.get()); + return cache_controller; } RemoteCacheController::RemoteCacheController( - const RemoteFileMeta & remote_file_meta, + const RemoteFileMetadata & remote_file_meta, const std::filesystem::path & local_path_, size_t cache_bytes_before_flush_, std::shared_ptr readbuffer_, std::function const & finish_callback) + : schema(remote_file_meta.schema) + , cluster(remote_file_meta.cluster) + , remote_path(remote_file_meta.path) + , local_path(local_path_) + , last_modify_time(remote_file_meta.last_modify_time) + , valid(true) + , local_cache_bytes_read_before_flush(cache_bytes_before_flush_) + , download_finished(false) + , current_offset(0) + , remote_readbuffer(readbuffer_) { - schema = remote_file_meta.schema; - cluster = remote_file_meta.cluster; - local_path = local_path_; - remote_path = remote_file_meta.path; - last_modification_timestamp = remote_file_meta.last_modification_timestamp; - local_cache_bytes_read_before_flush = cache_bytes_before_flush_; - valid = true; - if (readbuffer_ != nullptr) + if (remote_readbuffer) { - download_finished = false; - current_offset = 0; - remote_readbuffer = readbuffer_; // setup local files out_file = std::make_unique(local_path_ / "data.bin", std::ios::out | std::ios::binary); out_file->flush(); @@ -86,14 +87,14 @@ RemoteCacheController::RemoteCacheController( jobj.set("cluster", cluster); jobj.set("remote_path", remote_path); jobj.set("downloaded", "false"); - jobj.set("last_modification_timestamp", last_modification_timestamp); + jobj.set("last_modification_timestamp", last_modify_time); std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM jobj.stringify(buf); std::ofstream meta_file(local_path_ / "meta.txt", std::ios::out); meta_file.write(buf.str().c_str(), buf.str().size()); meta_file.close(); - backgroupDownload(finish_callback); + backgroundDownload(finish_callback); } } @@ -123,10 +124,11 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs return RemoteReadBufferCacheError::OK; } -void RemoteCacheController::backgroupDownload(std::function const & finish_callback) +void RemoteCacheController::backgroundDownload(std::function const & finish_callback) { - auto task = [this, finish_callback]() { - size_t unflush_bytes = 0; + auto task = [this, finish_callback]() + { + size_t before_unflush_bytes = 0; size_t total_bytes = 0; while (!remote_readbuffer->eof()) { @@ -135,8 +137,8 @@ void RemoteCacheController::backgroupDownload(std::functionwrite(remote_readbuffer->position(), bytes); remote_readbuffer->position() += bytes; total_bytes += bytes; - unflush_bytes += bytes; - if (unflush_bytes >= local_cache_bytes_read_before_flush) + before_unflush_bytes += bytes; + if (before_unflush_bytes >= local_cache_bytes_read_before_flush) { std::unique_lock lock(mutex); current_offset += total_bytes; @@ -144,7 +146,7 @@ void RemoteCacheController::backgroupDownload(std::function RemoteCacheController::allocFile() std::filesystem::path result_local_path; if (download_finished) result_local_path = local_path / "data.bin"; + FILE * fs = fopen((local_path / "data.bin").string().c_str(), "r"); if (fs == nullptr) - return {fs, result_local_path}; + throw Exception("alloc file failed.", ErrorCodes::BAD_GET); + std::lock_guard lock{mutex}; opened_file_streams.insert(fs); return {fs, result_local_path}; @@ -225,8 +230,6 @@ LocalCachedFileReader::LocalCachedFileReader(RemoteCacheController * cntrl_, siz : offset(0), file_size(size_), fs(nullptr), controller(cntrl_) { std::tie(fs, local_path) = controller->allocFile(); - if (fs == nullptr) - throw Exception("alloc file failed.", ErrorCodes::BAD_GET); } LocalCachedFileReader::~LocalCachedFileReader() { @@ -278,12 +281,12 @@ RemoteReadBuffer::RemoteReadBuffer(size_t buff_size) : BufferWithOwnMemory RemoteReadBuffer::create(const RemoteFileMeta & remote_file_meta_, std::unique_ptr readbuffer) +std::unique_ptr RemoteReadBuffer::create(const RemoteFileMetadata & remote_file_meta_, std::unique_ptr read_buffer) { auto * log = &Poco::Logger::get("RemoteReadBuffer"); size_t buff_size = DBMS_DEFAULT_BUFFER_SIZE; - if (readbuffer != nullptr) - buff_size = readbuffer->internalBuffer().size(); + if (read_buffer) + buff_size = read_buffer->internalBuffer().size(); /* * in the new implement of ReadBufferFromHDFS, buffer size is 0. * @@ -296,30 +299,30 @@ std::unique_ptr RemoteReadBuffer::create(const RemoteFileMeta buff_size = DBMS_DEFAULT_BUFFER_SIZE; const auto & remote_path = remote_file_meta_.path; - auto rrb = std::make_unique(buff_size); - auto * raw_rbp = readbuffer.release(); + auto remote_read_buffer = std::make_unique(buff_size); + auto * raw_rbp = read_buffer.release(); std::shared_ptr srb(raw_rbp); RemoteReadBufferCacheError error; int retry = 0; do { if (retry > 0) - usleep(20 * retry); + sleepForMicroseconds(20 * retry); - std::tie(rrb->file_reader, error) = RemoteReadBufferCache::instance().createReader(remote_file_meta_, srb); + std::tie(remote_read_buffer->file_reader, error) = RemoteReadBufferCache::instance().createReader(remote_file_meta_, srb); retry++; } while (error == RemoteReadBufferCacheError::FILE_INVALID && retry < 10); - if (rrb->file_reader == nullptr) + if (remote_read_buffer->file_reader == nullptr) { - LOG_ERROR(log, "allocate local file failed for " + remote_path + "{}", error); - rrb->original_readbuffer = srb; + LOG_ERROR(log, "Failed to allocate local file for remote path: {}. Error: {}", remote_path, error); + remote_read_buffer->original_readbuffer = srb; } - return rrb; + return remote_read_buffer; } bool RemoteReadBuffer::nextImpl() { - if (file_reader != nullptr) + if (file_reader) { int bytes_read = file_reader->read(internal_buffer.begin(), internal_buffer.size()); if (bytes_read) @@ -445,13 +448,13 @@ void RemoteReadBufferCache::initOnce( std::lock_guard lock(this->mutex); // two level dir. // recoverCachedFilesMeta(root_dir, 1, 2, callback); - this->inited = true; + this->initialized = true; LOG_TRACE(this->log, "recovered from disk "); }; getThreadPool()->scheduleOrThrow(recover_task); } -std::filesystem::path RemoteReadBufferCache::calculateLocalPath(const RemoteFileMeta & meta) +std::filesystem::path RemoteReadBufferCache::calculateLocalPath(const RemoteFileMetadata & meta) { std::string full_path = meta.schema + ":" + meta.cluster + ":" + meta.path; UInt128 hashcode = sipHash128(full_path.c_str(), full_path.size()); @@ -459,11 +462,11 @@ std::filesystem::path RemoteReadBufferCache::calculateLocalPath(const RemoteFile return std::filesystem::path(local_path_prefix) / hashcode_str.substr(0, 3) / hashcode_str; } -std::tuple, RemoteReadBufferCacheError> -RemoteReadBufferCache::createReader(const RemoteFileMeta & remote_file_meta, std::shared_ptr & readbuffer) +std::pair, RemoteReadBufferCacheError> +RemoteReadBufferCache::createReader(const RemoteFileMetadata & remote_file_meta, std::shared_ptr & read_buffer) { // If something is wrong on startup, rollback to read from the original ReadBuffer - if (!hasInitialized()) + if (!isInitialized()) { LOG_ERROR(log, "RemoteReadBufferCache has not initialized"); return {nullptr, RemoteReadBufferCacheError::NOT_INIT}; @@ -471,7 +474,7 @@ RemoteReadBufferCache::createReader(const RemoteFileMeta & remote_file_meta, std auto remote_path = remote_file_meta.path; const auto & file_size = remote_file_meta.file_size; - const auto & last_modification_timestamp = remote_file_meta.last_modification_timestamp; + const auto & last_modification_timestamp = remote_file_meta.last_modify_time; auto local_path = calculateLocalPath(remote_file_meta); std::lock_guard lock(mutex); auto cache_iter = caches.find(local_path); @@ -527,7 +530,7 @@ RemoteReadBufferCache::createReader(const RemoteFileMeta & remote_file_meta, std auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; auto cache_cntrl - = std::make_shared(remote_file_meta, local_path, local_cache_bytes_read_before_flush, readbuffer, callback); + = std::make_shared(remote_file_meta, local_path, local_cache_bytes_read_before_flush, read_buffer, callback); CacheCell cc; cc.cache_controller = cache_cntrl; cc.key_iterator = keys.insert(keys.end(), local_path); diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 9a67559f320..65229059eee 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -24,33 +24,34 @@ enum class RemoteReadBufferCacheError : int8_t END_OF_FILE = 20, }; -struct RemoteFileMeta +struct RemoteFileMetadata { - RemoteFileMeta( + RemoteFileMetadata( const std::string & schema_, const std::string & cluster_, const std::string & path_, - UInt64 last_modification_timestamp_, + UInt64 last_modify_time_, size_t file_size_) - : schema(schema_), cluster(cluster_), path(path_), last_modification_timestamp(last_modification_timestamp_), file_size(file_size_) + : schema(schema_) + , cluster(cluster_) + , path(path_) + , last_modify_time(last_modify_time_) + , file_size(file_size_) { } std::string schema; // Hive, S2 etc. std::string cluster; std::string path; - UInt64 last_modification_timestamp; + UInt64 last_modify_time; size_t file_size; }; -/** - * - */ class RemoteCacheController { public: RemoteCacheController( - const RemoteFileMeta & meta, + const RemoteFileMetadata & meta, const std::filesystem::path & local_path_, size_t cache_bytes_before_flush_, std::shared_ptr readbuffer_, @@ -92,7 +93,7 @@ public: inline const std::filesystem::path & getLocalPath() { return local_path; } inline const std::string & getRemotePath() { return remote_path; } - inline UInt64 getLastModificationTimestamp() const { return last_modification_timestamp; } + inline UInt64 getLastModificationTimestamp() const { return last_modify_time; } inline void markInvalid() { std::lock_guard lock(mutex); @@ -108,7 +109,7 @@ private: // flush file and meta info into disk void flush(bool need_flush_meta_ = false); - void backgroupDownload(std::function const & finish_callback); + void backgroundDownload(std::function const & finish_callback); std::mutex mutex; std::condition_variable more_data_signal; @@ -116,16 +117,17 @@ private: std::set opened_file_streams; // meta info - bool download_finished; - bool valid; - size_t current_offset; - UInt64 last_modification_timestamp; - std::filesystem::path local_path; - std::string remote_path; std::string schema; std::string cluster; + std::string remote_path; + std::filesystem::path local_path; + UInt64 last_modify_time; + bool valid; size_t local_cache_bytes_read_before_flush; + bool download_finished; + size_t current_offset; + std::shared_ptr remote_readbuffer; std::unique_ptr out_file; @@ -168,7 +170,7 @@ class RemoteReadBuffer : public BufferWithOwnMemory public: explicit RemoteReadBuffer(size_t buff_size); ~RemoteReadBuffer() override; - static std::unique_ptr create(const RemoteFileMeta & remote_file_meta_, std::unique_ptr readbuffer); + static std::unique_ptr create(const RemoteFileMetadata & remote_file_meta_, std::unique_ptr readbuffer); bool nextImpl() override; inline bool seekable() { return file_reader != nullptr && file_reader->size() > 0; } @@ -193,16 +195,16 @@ public: std::shared_ptr getThreadPool() { return thread_pool; } void initOnce(const std::filesystem::path & dir, size_t limit_size, size_t bytes_read_before_flush_, size_t max_threads); - inline bool hasInitialized() const { return inited; } + inline bool isInitialized() const { return initialized; } - std::tuple, RemoteReadBufferCacheError> - createReader(const RemoteFileMeta & remote_file_meta, std::shared_ptr & readbuffer); + std::pair, RemoteReadBufferCacheError> + createReader(const RemoteFileMetadata & remote_file_meta, std::shared_ptr & read_buffer); private: std::string local_path_prefix; std::shared_ptr thread_pool; - std::atomic inited = false; + std::atomic initialized = false; std::mutex mutex; size_t limit_size = 0; size_t local_cache_bytes_read_before_flush = 0; @@ -217,7 +219,7 @@ private: std::list keys; std::map caches; - std::filesystem::path calculateLocalPath(const RemoteFileMeta & meta); + std::filesystem::path calculateLocalPath(const RemoteFileMetadata & meta); void recoverCachedFilesMeta( const std::filesystem::path & current_path, diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 3f49d193fc3..e288ecc2393 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -255,7 +255,7 @@ StorageHive::StorageHive( setInMemoryMetadata(storage_metadata); auto hms_client = getContext()->getHiveMetastoreClient(hms_url); - auto table_meta = hms_client->getTableMeta(hive_database, hive_table); + auto table_meta = hms_client->getTableMetadata(hive_database, hive_table); hdfs_namenode_url = getNameNodeUrl(table_meta->getTable()->sd.location); table_schema = table_meta->getTable()->sd.cols; @@ -388,7 +388,7 @@ Pipe StorageHive::read( HDFSBuilderWrapper builder = createHDFSBuilder(hdfs_namenode_url, context_->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); auto hms_client = context_->getHiveMetastoreClient(hms_url); - auto table_meta_cntrl = hms_client->getTableMeta(hive_database, hive_table); + auto table_meta_cntrl = hms_client->getTableMetadata(hive_database, hive_table); // List files under partition directory in HDFS auto list_paths = [table_meta_cntrl, &fs](const String & path) { return table_meta_cntrl->getLocationFiles(fs, path); }; From 83be8d28e6dd1455c09f575729d96050f23da562 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 29 Nov 2021 14:50:33 +0800 Subject: [PATCH 0086/1260] fix code stylke --- src/IO/RemoteReadBufferCache.cpp | 48 +++++++++++++++++--------------- src/IO/RemoteReadBufferCache.h | 2 +- 2 files changed, 26 insertions(+), 24 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 3dffc0162cf..579c201676e 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -118,7 +118,7 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs return RemoteReadBufferCacheError::OK; } else - more_data_signal.wait(lock, [this, end_offset_] { return this->download_finished || this->current_offset >= end_offset_; }); + more_data_signal.wait(lock, [this, end_offset_] { return download_finished || current_offset >= end_offset_; }); } lock.unlock(); return RemoteReadBufferCacheError::OK; @@ -353,12 +353,12 @@ off_t RemoteReadBuffer::seek(off_t offset, int whence) if (whence == SEEK_SET) new_pos = offset; else if (whence == SEEK_CUR) - new_pos = pos_in_file - (working_buffer.end() - pos) + offset; + new_pos = pos_in_file - available() + offset; else throw Exception("expects SEEK_SET or SEEK_CUR as whence", ErrorCodes::BAD_ARGUMENTS); /// Position is unchanged. - if (new_pos + (working_buffer.end() - pos) == pos_in_file) + if (off_t(new_pos + available()) == pos_in_file) return new_pos; if (new_pos <= pos_in_file && new_pos >= pos_in_file - static_cast(working_buffer.size())) @@ -440,16 +440,16 @@ void RemoteReadBufferCache::initOnce( std::filesystem::path root_dir(local_path_prefix); if (!std::filesystem::exists(root_dir)) { - LOG_INFO(log, "{} not exists. this cache will be disable", local_path_prefix); + LOG_INFO(log, "Path {} not exists. this cache will be disable", local_path_prefix); return; } auto recover_task = [this, root_dir]() { - auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; - std::lock_guard lock(this->mutex); + auto callback = [this](RemoteCacheController * cntrl) { total_size += cntrl->size(); }; + std::lock_guard lock(mutex); // two level dir. // recoverCachedFilesMeta(root_dir, 1, 2, callback); - this->initialized = true; - LOG_TRACE(this->log, "recovered from disk "); + initialized = true; + LOG_TRACE(log, "Recovered from disk "); }; getThreadPool()->scheduleOrThrow(recover_task); } @@ -485,9 +485,10 @@ RemoteReadBufferCache::createReader(const RemoteFileMetadata & remote_file_meta, { LOG_TRACE( log, - "remote file has been updated. " + remote_path + ":" - + std::to_string(cache_iter->second.cache_controller->getLastModificationTimestamp()) + "->" - + std::to_string(last_modification_timestamp)); + "Remote file ({}) has been updated. Last saved modification time: {}, actual last modification time: {}", + remote_path, + std::to_string(cache_iter->second.cache_controller->getLastModificationTimestamp()), + std::to_string(last_modification_timestamp)); cache_iter->second.cache_controller->markInvalid(); } else @@ -522,35 +523,36 @@ RemoteReadBufferCache::createReader(const RemoteFileMetadata & remote_file_meta, // reach the disk capacity limit if (!clear_ret) { - LOG_ERROR(log, "local cache is full, return nullptr"); + LOG_INFO(log, "Reached local cache capacity limit size ({})", limit_size); return {nullptr, RemoteReadBufferCacheError::DISK_FULL}; } std::filesystem::create_directories(local_path); - auto callback = [this](RemoteCacheController * cntrl) { this->total_size += cntrl->size(); }; - auto cache_cntrl + auto callback = [this](RemoteCacheController * cntrl) { total_size += cntrl->size(); }; + auto cache_controller = std::make_shared(remote_file_meta, local_path, local_cache_bytes_read_before_flush, read_buffer, callback); - CacheCell cc; - cc.cache_controller = cache_cntrl; - cc.key_iterator = keys.insert(keys.end(), local_path); - caches[local_path] = cc; + CacheCell cache_cell; + cache_cell.cache_controller = cache_controller; + cache_cell.key_iterator = keys.insert(keys.end(), local_path); + caches[local_path] = cache_cell; - return {std::make_shared(cache_cntrl.get(), file_size), RemoteReadBufferCacheError::OK}; + return {std::make_shared(cache_controller.get(), file_size), RemoteReadBufferCacheError::OK}; } bool RemoteReadBufferCache::clearLocalCache() { for (auto it = keys.begin(); it != keys.end();) { + // TODO keys is not thread-safe auto cache_it = caches.find(*it); - auto cntrl = cache_it->second.cache_controller; - if (!cntrl->isValid() && cntrl->closable()) + auto cache_controller = cache_it->second.cache_controller; + if (!cache_controller->isValid() && cache_controller->closable()) { - LOG_TRACE(log, "clear invalid cache: " + *it); + LOG_TRACE(log, "Clear invalid cache entry with key {}", *it); total_size = total_size > cache_it->second.cache_controller->size() ? total_size - cache_it->second.cache_controller->size() : 0; - cntrl->close(); + cache_controller->close(); it = keys.erase(it); caches.erase(cache_it); } diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 65229059eee..d1476bb2eb8 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -56,7 +56,7 @@ public: size_t cache_bytes_before_flush_, std::shared_ptr readbuffer_, std::function const & finish_callback); - ~RemoteCacheController(); // the local files will be deleted in descontructor + ~RemoteCacheController(); // recover from local disk static std::shared_ptr From b28f3a2ad413efa748500dd3b2e4c4489556741f Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 29 Nov 2021 14:53:06 +0800 Subject: [PATCH 0087/1260] fixed code style --- src/IO/RemoteReadBufferCache.cpp | 4 ++-- src/IO/RemoteReadBufferCache.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 579c201676e..238ae4b441e 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -27,8 +27,7 @@ std::shared_ptr RemoteCacheController::recover( { std::filesystem::path data_file = local_path_ / "data.bin"; std::filesystem::path meta_file = local_path_ / "meta.txt"; - auto * log = &Poco::Logger::get("RemoteCacheController"); - if (!std::filesystem::exists(data_file) || !std::filesystem::exists(meta_file)) + if (!std::filesystem::exists(local_path_) || !std::filesystem::exists(data_file) || !std::filesystem::exists(meta_file)) { LOG_ERROR(log, "Directory {} or file {}, {} does not exist", local_path_.string(), data_file.string(), meta_file.string()); return nullptr; @@ -229,6 +228,7 @@ void RemoteCacheController::deallocFile(FILE * fs) LocalCachedFileReader::LocalCachedFileReader(RemoteCacheController * cntrl_, size_t size_) : offset(0), file_size(size_), fs(nullptr), controller(cntrl_) { + std::tie(fs, local_path) = controller->allocFile(); } LocalCachedFileReader::~LocalCachedFileReader() diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index d1476bb2eb8..798b448a911 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -84,7 +84,7 @@ public: /** * called in LocalCachedFileReader read(), the reading process would be blocked until * enough data be downloaded. - * If the file has finished download, the process would be unblock + * If the file has finished download, the process would unblocked */ RemoteReadBufferCacheError waitMoreData(size_t start_offset_, size_t end_offset_); From c7a0fe467f34feada6ca137fbb31a95eeb776ca5 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 29 Nov 2021 16:22:43 +0800 Subject: [PATCH 0088/1260] fix code stylke --- programs/server/Server.cpp | 21 ++++-- programs/server/config.xml | 8 +++ src/Core/Settings.h | 1 + src/IO/RemoteReadBufferCache.cpp | 114 +++++++++++++++---------------- src/IO/RemoteReadBufferCache.h | 73 +++++++++++--------- src/Storages/Hive/HiveFile.cpp | 4 +- 6 files changed, 123 insertions(+), 98 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 22223121594..3bd3ef48bb0 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -518,12 +518,21 @@ if (ThreadFuzzer::instance().isEffective()) config().getUInt("thread_pool_queue_size", 10000) ); - if (config().has("local_cache_dir") && config().has("local_cache_quota")) - RemoteReadBufferCache::instance().initOnce( - config().getString("local_cache_dir"), - config().getUInt64("local_cache_quota"), - config().getUInt64("local_cache_bytes_read_before_flush",DBMS_DEFAULT_BUFFER_SIZE), - config().getUInt64("local_cache_max_threads", 1000)); + + /// Initialize global local cache for remote filesystem. + if (config().has("local_cache_for_remote_fs")) + { + bool enable = config().getBool("local_cache_for_remote_fs.enable", false); + if (enable) + { + String root_dir = config().getString("local_cache_for_remote_fs.root_dir"); + UInt64 limit_size = config().getUInt64("local_cache_for_remote_fs.limit_size"); + UInt64 bytes_read_before_flush + = config().getUInt64("local_cache_for_remote_fs.bytes_read_before_flush", DBMS_DEFAULT_BUFFER_SIZE); + auto max_threads = config().getUInt("local_cache_for_remote_fs.max_threads", 64); + RemoteReadBufferCache::instance().initOnce(root_dir, limit_size, bytes_read_before_flush, max_threads); + } + } ConnectionCollector::init(global_context, config().getUInt("max_threads_for_connection_collector", 10)); diff --git a/programs/server/config.xml b/programs/server/config.xml index 37f36aa5215..e1c2e3ca3fc 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1253,4 +1253,12 @@ --> + + + true + /var/lib/clickhouse/local_cache + 53687091200 + 1048576 + 1024 + diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c16ea17ee91..f6035f66df3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -533,6 +533,7 @@ class IColumn; \ M(Bool, force_remove_data_recursively_on_drop, false, "Recursively remove data on DROP query. Avoids 'Directory not empty' error, but may silently remove detached data", 0) \ M(Bool, check_table_dependencies, true, "Check that DDL query (such as DROP TABLE or RENAME) will not break dependencies", 0) \ + M(Bool, use_local_cache_for_remote_fs, true, "Use local cache for remote filesystem like HDFS or S3", 0) \ \ /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 579c201676e..4cbd4ae0f6c 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -20,34 +20,35 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int BAD_GET; extern const int LOGICAL_ERROR; + extern const int CANNOT_CREATE_DIRECTORY; } std::shared_ptr RemoteCacheController::recover( - const std::filesystem::path & local_path_, std::function const & finish_callback) + const String & local_path_, std::function const & finish_callback) { - std::filesystem::path data_file = local_path_ / "data.bin"; - std::filesystem::path meta_file = local_path_ / "meta.txt"; + fs::path data_file = fs::path(local_path_) / "data.bin"; + fs::path meta_file = fs::path(local_path_) / "meta.txt"; auto * log = &Poco::Logger::get("RemoteCacheController"); - if (!std::filesystem::exists(data_file) || !std::filesystem::exists(meta_file)) + if (!fs::exists(data_file) || !fs::exists(meta_file)) { - LOG_ERROR(log, "Directory {} or file {}, {} does not exist", local_path_.string(), data_file.string(), meta_file.string()); + LOG_ERROR(log, "Directory {} or file {}, {} does not exist", local_path_, data_file.string(), meta_file.string()); return nullptr; } std::ifstream meta_fs(meta_file); Poco::JSON::Parser meta_parser; auto meta_jobj = meta_parser.parse(meta_fs).extract(); - auto remote_path = meta_jobj->get("remote_path").convert(); - auto schema = meta_jobj->get("schema").convert(); - auto cluster = meta_jobj->get("cluster").convert(); - auto downloaded = meta_jobj->get("downloaded").convert(); + auto remote_path = meta_jobj->get("remote_path").convert(); + auto schema = meta_jobj->get("schema").convert(); + auto cluster = meta_jobj->get("cluster").convert(); + auto downloaded = meta_jobj->get("downloaded").convert(); auto modification_ts = meta_jobj->get("last_modification_timestamp").convert(); if (downloaded == "false") { - LOG_ERROR(log, "Local metadata for path {} exists, but the data was not downloaded", local_path_.string()); + LOG_ERROR(log, "Local metadata for path {} exists, but the data was not downloaded", local_path_); return nullptr; } - auto file_size = std::filesystem::file_size(data_file); + auto file_size = fs::file_size(data_file); RemoteFileMetadata remote_file_meta(schema, cluster, remote_path, modification_ts, file_size); auto cache_controller = std::make_shared(remote_file_meta, local_path_, 0, nullptr, finish_callback); @@ -61,7 +62,7 @@ std::shared_ptr RemoteCacheController::recover( RemoteCacheController::RemoteCacheController( const RemoteFileMetadata & remote_file_meta, - const std::filesystem::path & local_path_, + const String & local_path_, size_t cache_bytes_before_flush_, std::shared_ptr readbuffer_, std::function const & finish_callback) @@ -79,7 +80,7 @@ RemoteCacheController::RemoteCacheController( if (remote_readbuffer) { // setup local files - out_file = std::make_unique(local_path_ / "data.bin", std::ios::out | std::ios::binary); + out_file = std::make_unique(fs::path(local_path_) / "data.bin", std::ios::out | std::ios::binary); out_file->flush(); Poco::JSON::Object jobj; @@ -90,7 +91,7 @@ RemoteCacheController::RemoteCacheController( jobj.set("last_modification_timestamp", last_modify_time); std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM jobj.stringify(buf); - std::ofstream meta_file(local_path_ / "meta.txt", std::ios::out); + std::ofstream meta_file(fs::path(local_path_) / "meta.txt", std::ios::out); meta_file.write(buf.str().c_str(), buf.str().size()); meta_file.close(); @@ -126,7 +127,7 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs void RemoteCacheController::backgroundDownload(std::function const & finish_callback) { - auto task = [this, finish_callback]() + auto task = [this, finish_callback]() { size_t before_unflush_bytes = 0; size_t total_bytes = 0; @@ -159,7 +160,7 @@ void RemoteCacheController::backgroundDownload(std::functionscheduleOrThrow(task); } @@ -182,7 +183,7 @@ void RemoteCacheController::flush(bool need_flush_meta_) std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM jobj.stringify(buf); - std::ofstream meta_file(local_path / "meta.txt", std::ios::out); + std::ofstream meta_file(fs::path(local_path) / "meta.txt", std::ios::out); meta_file << buf.str(); meta_file.close(); } @@ -192,17 +193,17 @@ RemoteCacheController::~RemoteCacheController() = default; void RemoteCacheController::close() { // delete directory - LOG_TRACE(log, "Removing all local cache for remote path: {}, local path: {}", remote_path, local_path.string()); - std::filesystem::remove_all(local_path); + LOG_TRACE(log, "Removing all local cache for remote path: {}, local path: {}", remote_path, local_path); + fs::remove_all(local_path); } -std::pair RemoteCacheController::allocFile() +std::pair RemoteCacheController::allocFile() { - std::filesystem::path result_local_path; + fs::path result_local_path; if (download_finished) - result_local_path = local_path / "data.bin"; + result_local_path = fs::path(local_path) / "data.bin"; - FILE * fs = fopen((local_path / "data.bin").string().c_str(), "r"); + FILE * fs = fopen((fs::path(local_path) / "data.bin").string().c_str(), "r"); if (fs == nullptr) throw Exception("alloc file failed.", ErrorCodes::BAD_GET); @@ -226,19 +227,19 @@ void RemoteCacheController::deallocFile(FILE * fs) fclose(fs); } -LocalCachedFileReader::LocalCachedFileReader(RemoteCacheController * cntrl_, size_t size_) - : offset(0), file_size(size_), fs(nullptr), controller(cntrl_) +LocalCachedFileReader::LocalCachedFileReader(RemoteCacheController * cache_controller_, size_t size_) + : offset(0), file_size(size_), fs(nullptr), cache_controller(cache_controller_) { - std::tie(fs, local_path) = controller->allocFile(); + std::tie(fs, local_path) = cache_controller->allocFile(); } LocalCachedFileReader::~LocalCachedFileReader() { - controller->deallocFile(fs); + cache_controller->deallocFile(fs); } size_t LocalCachedFileReader::read(char * buf, size_t size) { - auto wret = controller->waitMoreData(offset, offset + size); + auto wret = cache_controller->waitMoreData(offset, offset + size); if (wret != RemoteReadBufferCacheError::OK) return 0; std::lock_guard lock(mutex); @@ -249,7 +250,7 @@ size_t LocalCachedFileReader::read(char * buf, size_t size) off_t LocalCachedFileReader::seek(off_t off) { - controller->waitMoreData(off, 1); + cache_controller->waitMoreData(off, 1); std::lock_guard lock(mutex); auto ret = fseek(fs, off, SEEK_SET); offset = off; @@ -269,7 +270,7 @@ size_t LocalCachedFileReader::size() return 0; } - auto ret = std::filesystem::file_size(local_path); + auto ret = fs::file_size(local_path); file_size = ret; return ret; } @@ -281,7 +282,7 @@ RemoteReadBuffer::RemoteReadBuffer(size_t buff_size) : BufferWithOwnMemory RemoteReadBuffer::create(const RemoteFileMetadata & remote_file_meta_, std::unique_ptr read_buffer) +std::unique_ptr RemoteReadBuffer::create(const RemoteFileMetadata & remote_file_meta, std::unique_ptr read_buffer) { auto * log = &Poco::Logger::get("RemoteReadBuffer"); size_t buff_size = DBMS_DEFAULT_BUFFER_SIZE; @@ -298,7 +299,7 @@ std::unique_ptr RemoteReadBuffer::create(const RemoteFileMetad if (buff_size == 0) buff_size = DBMS_DEFAULT_BUFFER_SIZE; - const auto & remote_path = remote_file_meta_.path; + const auto & remote_path = remote_file_meta.path; auto remote_read_buffer = std::make_unique(buff_size); auto * raw_rbp = read_buffer.release(); std::shared_ptr srb(raw_rbp); @@ -309,7 +310,7 @@ std::unique_ptr RemoteReadBuffer::create(const RemoteFileMetad if (retry > 0) sleepForMicroseconds(20 * retry); - std::tie(remote_read_buffer->file_reader, error) = RemoteReadBufferCache::instance().createReader(remote_file_meta_, srb); + std::tie(remote_read_buffer->file_reader, error) = RemoteReadBufferCache::instance().createReader(remote_file_meta, srb); retry++; } while (error == RemoteReadBufferCacheError::FILE_INVALID && retry < 10); if (remote_read_buffer->file_reader == nullptr) @@ -397,14 +398,14 @@ RemoteReadBufferCache & RemoteReadBufferCache::instance() } void RemoteReadBufferCache::recoverCachedFilesMeta( - const std::filesystem::path & current_path, + const fs::path & current_path, size_t current_depth, size_t max_depth, std::function const & finish_callback) { if (current_depth >= max_depth) { - for (auto const & dir : std::filesystem::directory_iterator{current_path}) + for (auto const & dir : fs::directory_iterator{current_path}) { std::string path = dir.path(); auto cache_controller = RemoteCacheController::recover(path, finish_callback); @@ -417,36 +418,35 @@ void RemoteReadBufferCache::recoverCachedFilesMeta( return; } - for (auto const & dir : std::filesystem::directory_iterator{current_path}) + for (auto const & dir : fs::directory_iterator{current_path}) { recoverCachedFilesMeta(dir.path(), current_depth + 1, max_depth, finish_callback); } } void RemoteReadBufferCache::initOnce( - const std::filesystem::path & dir, size_t limit_size_, size_t bytes_read_before_flush_, size_t max_threads) + const String & root_dir_, size_t limit_size_, size_t bytes_read_before_flush_, size_t max_threads_) { LOG_INFO( - log, - "Initializing local cache for remote data sources. Local cache root path: {}, cache size limit: {}", - dir.string(), - limit_size_); - local_path_prefix = dir; + log, "Initializing local cache for remote data sources. Local cache root path: {}, cache size limit: {}", root_dir_, limit_size_); + root_dir = root_dir_; limit_size = limit_size_; local_cache_bytes_read_before_flush = bytes_read_before_flush_; - thread_pool = std::make_shared(max_threads, 1000, 1000, false); + thread_pool = std::make_shared(max_threads_, 1000, 1000, false); - // scan local disk dir and recover the cache metas - std::filesystem::path root_dir(local_path_prefix); - if (!std::filesystem::exists(root_dir)) + /// create if root_dir not exists + if (!fs::exists(fs::path(root_dir) / "")) { - LOG_INFO(log, "Path {} not exists. this cache will be disable", local_path_prefix); - return; + std::error_code ec; + bool success = fs::create_directories(fs::path(root_dir) / "", ec); + if (!success) + throw Exception( + ErrorCodes::CANNOT_CREATE_DIRECTORY, "Failed to create directories, error code:{} reason:{}", ec.value(), ec.message()); } - auto recover_task = [this, root_dir]() { - auto callback = [this](RemoteCacheController * cntrl) { total_size += cntrl->size(); }; + + auto recover_task = [this]() { + auto callback = [this](RemoteCacheController * cache_controller) { total_size += cache_controller->size(); }; std::lock_guard lock(mutex); - // two level dir. // recoverCachedFilesMeta(root_dir, 1, 2, callback); initialized = true; LOG_TRACE(log, "Recovered from disk "); @@ -454,12 +454,12 @@ void RemoteReadBufferCache::initOnce( getThreadPool()->scheduleOrThrow(recover_task); } -std::filesystem::path RemoteReadBufferCache::calculateLocalPath(const RemoteFileMetadata & meta) +String RemoteReadBufferCache::calculateLocalPath(const RemoteFileMetadata & meta) const { - std::string full_path = meta.schema + ":" + meta.cluster + ":" + meta.path; + String full_path = meta.schema + ":" + meta.cluster + ":" + meta.path; UInt128 hashcode = sipHash128(full_path.c_str(), full_path.size()); - std::string hashcode_str = getHexUIntLowercase(hashcode); - return std::filesystem::path(local_path_prefix) / hashcode_str.substr(0, 3) / hashcode_str; + String hashcode_str = getHexUIntLowercase(hashcode); + return fs::path(root_dir) / hashcode_str.substr(0, 3) / hashcode_str; } std::pair, RemoteReadBufferCacheError> @@ -527,7 +527,7 @@ RemoteReadBufferCache::createReader(const RemoteFileMetadata & remote_file_meta, return {nullptr, RemoteReadBufferCacheError::DISK_FULL}; } - std::filesystem::create_directories(local_path); + fs::create_directories(local_path); auto callback = [this](RemoteCacheController * cntrl) { total_size += cntrl->size(); }; auto cache_controller @@ -579,7 +579,7 @@ bool RemoteReadBufferCache::clearLocalCache() LOG_TRACE( log, "clear local file {} for {}. key size:{}. next{}", - cache_it->second.cache_controller->getLocalPath().string(), + cache_it->second.cache_controller->getLocalPath(), cache_it->second.cache_controller->getRemotePath(), keys.size(), *it); diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index d1476bb2eb8..848ade4777e 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -12,6 +12,8 @@ #include #include +namespace fs = std::filesystem; + namespace DB { enum class RemoteReadBufferCacheError : int8_t @@ -20,16 +22,15 @@ enum class RemoteReadBufferCacheError : int8_t NOT_INIT = 10, DISK_FULL = 11, FILE_INVALID = 12, - END_OF_FILE = 20, }; struct RemoteFileMetadata { RemoteFileMetadata( - const std::string & schema_, - const std::string & cluster_, - const std::string & path_, + const String & schema_, + const String & cluster_, + const String & path_, UInt64 last_modify_time_, size_t file_size_) : schema(schema_) @@ -40,9 +41,9 @@ struct RemoteFileMetadata { } - std::string schema; // Hive, S2 etc. - std::string cluster; - std::string path; + String schema; // Hive, S2 etc. + String cluster; + String path; UInt64 last_modify_time; size_t file_size; }; @@ -52,7 +53,7 @@ class RemoteCacheController public: RemoteCacheController( const RemoteFileMetadata & meta, - const std::filesystem::path & local_path_, + const String & local_path_, size_t cache_bytes_before_flush_, std::shared_ptr readbuffer_, std::function const & finish_callback); @@ -60,14 +61,14 @@ public: // recover from local disk static std::shared_ptr - recover(const std::filesystem::path & local_path, std::function const & finish_callback); + recover(const String & local_path, std::function const & finish_callback); /** * Called by LocalCachedFileReader, must be used in pair * The second value of the return tuple is the local_path to store file. * It will be empty if the file has not been downloaded */ - std::pair allocFile(); + std::pair allocFile(); void deallocFile(FILE * fs_); /** @@ -90,8 +91,8 @@ public: inline size_t size() const { return current_offset; } - inline const std::filesystem::path & getLocalPath() { return local_path; } - inline const std::string & getRemotePath() { return remote_path; } + inline String getLocalPath() const { return local_path; } + inline String getRemotePath() const { return remote_path; } inline UInt64 getLastModificationTimestamp() const { return last_modify_time; } inline void markInvalid() @@ -117,10 +118,10 @@ private: std::set opened_file_streams; // meta info - std::string schema; - std::string cluster; - std::string remote_path; - std::filesystem::path local_path; + String schema; + String cluster; + String remote_path; + String local_path; UInt64 last_modify_time; bool valid; @@ -140,7 +141,7 @@ private: class LocalCachedFileReader { public: - LocalCachedFileReader(RemoteCacheController * cntrl_, size_t size_); + LocalCachedFileReader(RemoteCacheController * cache_controller_, size_t size_); ~LocalCachedFileReader(); // expect to read size bytes into buf, return is the real bytes read @@ -148,15 +149,15 @@ public: inline off_t getOffset() const { return static_cast(offset); } size_t size(); off_t seek(off_t offset); - inline std::string getPath() { return local_path; } + inline String getPath() const { return local_path; } private: std::mutex mutex; size_t offset; size_t file_size; FILE * fs; - std::filesystem::path local_path; - RemoteCacheController * controller; + String local_path; + RemoteCacheController * cache_controller; Poco::Logger * log = &Poco::Logger::get("RemoteReadBufferCache"); }; @@ -170,7 +171,7 @@ class RemoteReadBuffer : public BufferWithOwnMemory public: explicit RemoteReadBuffer(size_t buff_size); ~RemoteReadBuffer() override; - static std::unique_ptr create(const RemoteFileMetadata & remote_file_meta_, std::unique_ptr readbuffer); + static std::unique_ptr create(const RemoteFileMetadata & remote_file_meta, std::unique_ptr read_buffer); bool nextImpl() override; inline bool seekable() { return file_reader != nullptr && file_reader->size() > 0; } @@ -185,41 +186,47 @@ private: class RemoteReadBufferCache { -protected: - RemoteReadBufferCache(); - public: ~RemoteReadBufferCache(); // global instance static RemoteReadBufferCache & instance(); + std::shared_ptr getThreadPool() { return thread_pool; } - void initOnce(const std::filesystem::path & dir, size_t limit_size, size_t bytes_read_before_flush_, size_t max_threads); + void initOnce(const String & root_dir_, size_t limit_size_, size_t bytes_read_before_flush_, size_t max_threads_); + inline bool isInitialized() const { return initialized; } std::pair, RemoteReadBufferCacheError> createReader(const RemoteFileMetadata & remote_file_meta, std::shared_ptr & read_buffer); + void updateTotalSize(size_t size) { total_size += size; } + +protected: + RemoteReadBufferCache(); + private: - std::string local_path_prefix; + // root directory of local cache for remote filesystem + String root_dir; + size_t limit_size = 0; + size_t local_cache_bytes_read_before_flush = 0; std::shared_ptr thread_pool; std::atomic initialized = false; - std::mutex mutex; - size_t limit_size = 0; - size_t local_cache_bytes_read_before_flush = 0; std::atomic total_size; + std::mutex mutex; + Poco::Logger * log = &Poco::Logger::get("RemoteReadBufferCache"); struct CacheCell { - std::list::iterator key_iterator; + std::list::iterator key_iterator; std::shared_ptr cache_controller; }; - std::list keys; - std::map caches; + std::list keys; + std::map caches; - std::filesystem::path calculateLocalPath(const RemoteFileMetadata & meta); + String calculateLocalPath(const RemoteFileMetadata & meta) const; void recoverCachedFilesMeta( const std::filesystem::path & current_path, diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 8edd4455f53..8ecb903c7f3 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -27,7 +27,7 @@ namespace DB template Range createRangeFromOrcStatistics(const StatisticsType * stats) { - /// We must check if there are minimum or maximum values in statistics in case of + /// We must check if there are minimum or maximum values in statistics in case of /// null values or NaN/Inf values of double type. if (stats->hasMinimum() && stats->hasMaximum()) { @@ -50,7 +50,7 @@ Range createRangeFromOrcStatistics(const StatisticsType * stats) template Range createRangeFromParquetStatistics(std::shared_ptr stats) { - /// We must check if there are minimum or maximum values in statistics in case of + /// We must check if there are minimum or maximum values in statistics in case of /// null values or NaN/Inf values of double type. if (!stats->HasMinMax()) return Range(); From 1c4efe5b13dfca201179f275d2170c6fe06311d3 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 29 Nov 2021 17:01:34 +0800 Subject: [PATCH 0089/1260] fix code style of logger and exceptin --- src/IO/RemoteReadBufferCache.cpp | 78 +++++++++++++++----------------- src/IO/RemoteReadBufferCache.h | 23 ++++++---- 2 files changed, 50 insertions(+), 51 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 4cbd4ae0f6c..19a98f234ab 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -45,7 +46,7 @@ std::shared_ptr RemoteCacheController::recover( auto modification_ts = meta_jobj->get("last_modification_timestamp").convert(); if (downloaded == "false") { - LOG_ERROR(log, "Local metadata for path {} exists, but the data was not downloaded", local_path_); + LOG_ERROR(log, "Local metadata for local path {} exists, but the data was not downloaded", local_path_); return nullptr; } auto file_size = fs::file_size(data_file); @@ -160,7 +161,7 @@ void RemoteCacheController::backgroundDownload(std::functionscheduleOrThrow(task); } @@ -192,7 +193,6 @@ RemoteCacheController::~RemoteCacheController() = default; void RemoteCacheController::close() { - // delete directory LOG_TRACE(log, "Removing all local cache for remote path: {}, local path: {}", remote_path, local_path); fs::remove_all(local_path); } @@ -204,8 +204,8 @@ std::pair RemoteCacheController::allocFile() result_local_path = fs::path(local_path) / "data.bin"; FILE * fs = fopen((fs::path(local_path) / "data.bin").string().c_str(), "r"); - if (fs == nullptr) - throw Exception("alloc file failed.", ErrorCodes::BAD_GET); + if (!fs) + throw Exception(ErrorCodes::BAD_GET, "Alloc file failed, error code: {} local path: {}", errno, local_path); std::lock_guard lock{mutex}; opened_file_streams.insert(fs); @@ -218,23 +218,23 @@ void RemoteCacheController::deallocFile(FILE * fs) std::lock_guard lock{mutex}; auto it = opened_file_streams.find(fs); if (it == opened_file_streams.end()) - { - std::string err = "try to close an invalid file " + remote_path; - throw Exception(err, ErrorCodes::BAD_ARGUMENTS); - } + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Try to deallocate file {} with invalid handler", remote_path); opened_file_streams.erase(it); } fclose(fs); } -LocalCachedFileReader::LocalCachedFileReader(RemoteCacheController * cache_controller_, size_t size_) - : offset(0), file_size(size_), fs(nullptr), cache_controller(cache_controller_) +LocalCachedFileReader::LocalCachedFileReader(RemoteCacheController * cache_controller_, size_t file_size_) + : cache_controller(cache_controller_) + , file_size(file_size_) + , offset(0) { - std::tie(fs, local_path) = cache_controller->allocFile(); + std::tie(file_stream, local_path) = cache_controller->allocFile(); } + LocalCachedFileReader::~LocalCachedFileReader() { - cache_controller->deallocFile(fs); + cache_controller->deallocFile(file_stream); } size_t LocalCachedFileReader::read(char * buf, size_t size) @@ -243,7 +243,7 @@ size_t LocalCachedFileReader::read(char * buf, size_t size) if (wret != RemoteReadBufferCacheError::OK) return 0; std::lock_guard lock(mutex); - auto ret_size = fread(buf, 1, size, fs); + auto ret_size = fread(buf, 1, size, file_stream); offset += ret_size; return ret_size; } @@ -252,27 +252,27 @@ off_t LocalCachedFileReader::seek(off_t off) { cache_controller->waitMoreData(off, 1); std::lock_guard lock(mutex); - auto ret = fseek(fs, off, SEEK_SET); + auto ret = fseek(file_stream, off, SEEK_SET); + if (ret < 0) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Seek file {} with size {} to offset {} failed: {}", getPath(), getSize(), off, errnoToString(errno)); + offset = off; - if (ret != 0) - { - return -1; - } return off; } -size_t LocalCachedFileReader::size() +size_t LocalCachedFileReader::getSize() { if (file_size != 0) return file_size; + if (local_path.empty()) { - LOG_TRACE(log, "empty local_path"); + LOG_TRACE(log, "Empty local_path"); return 0; } - auto ret = fs::file_size(local_path); - file_size = ret; - return ret; + file_size = fs::file_size(local_path); + return file_size; } // the size need be equal to the original buffer @@ -315,7 +315,7 @@ std::unique_ptr RemoteReadBuffer::create(const RemoteFileMetad } while (error == RemoteReadBufferCacheError::FILE_INVALID && retry < 10); if (remote_read_buffer->file_reader == nullptr) { - LOG_ERROR(log, "Failed to allocate local file for remote path: {}. Error: {}", remote_path, error); + LOG_ERROR(log, "Failed to allocate local file for remote path: {}, reason: {}", remote_path, error); remote_read_buffer->original_readbuffer = srb; } return remote_read_buffer; @@ -335,8 +335,9 @@ bool RemoteReadBuffer::nextImpl() } else // in the case we cannot use local cache, read from the original readbuffer directly { - if (original_readbuffer == nullptr) - throw Exception("original readbuffer should not be null", ErrorCodes::LOGICAL_ERROR); + if (original_readbuffer) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid original read buffer"); + auto status = original_readbuffer->next(); // we don't need to worry about the memory buffer allocated in RemoteReadBuffer, since it is owned by // BufferWithOwnMemory, BufferWithOwnMemory would release it. @@ -356,7 +357,7 @@ off_t RemoteReadBuffer::seek(off_t offset, int whence) else if (whence == SEEK_CUR) new_pos = pos_in_file - available() + offset; else - throw Exception("expects SEEK_SET or SEEK_CUR as whence", ErrorCodes::BAD_ARGUMENTS); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expects SEEK_SET or SEEK_CUR as whence but given {}", whence); /// Position is unchanged. if (off_t(new_pos + available()) == pos_in_file) @@ -371,11 +372,6 @@ off_t RemoteReadBuffer::seek(off_t offset, int whence) pos = working_buffer.end(); auto ret_off = file_reader->seek(new_pos); - if (ret_off == -1) - throw Exception( - "seek file failed. " + std::to_string(pos_in_file) + "->" + std::to_string(new_pos) + "@" + std::to_string(file_reader->size()) - + "," + std::to_string(whence) + "," + file_reader->getPath(), - ErrorCodes::BAD_ARGUMENTS); return ret_off; } @@ -444,12 +440,13 @@ void RemoteReadBufferCache::initOnce( ErrorCodes::CANNOT_CREATE_DIRECTORY, "Failed to create directories, error code:{} reason:{}", ec.value(), ec.message()); } - auto recover_task = [this]() { + auto recover_task = [this]() + { auto callback = [this](RemoteCacheController * cache_controller) { total_size += cache_controller->size(); }; std::lock_guard lock(mutex); recoverCachedFilesMeta(root_dir, 1, 2, callback); initialized = true; - LOG_TRACE(log, "Recovered from disk "); + LOG_TRACE(log, "Recovered from directory:{}", root_dir); }; getThreadPool()->scheduleOrThrow(recover_task); } @@ -468,7 +465,7 @@ RemoteReadBufferCache::createReader(const RemoteFileMetadata & remote_file_meta, // If something is wrong on startup, rollback to read from the original ReadBuffer if (!isInitialized()) { - LOG_ERROR(log, "RemoteReadBufferCache has not initialized"); + LOG_ERROR(log, "RemoteReadBufferCache not initialized yet"); return {nullptr, RemoteReadBufferCacheError::NOT_INIT}; } @@ -549,7 +546,7 @@ bool RemoteReadBufferCache::clearLocalCache() auto cache_controller = cache_it->second.cache_controller; if (!cache_controller->isValid() && cache_controller->closable()) { - LOG_TRACE(log, "Clear invalid cache entry with key {}", *it); + LOG_TRACE(log, "Clear invalid cache entry with key {} from local cache", *it); total_size = total_size > cache_it->second.cache_controller->size() ? total_size - cache_it->second.cache_controller->size() : 0; cache_controller->close(); @@ -566,9 +563,8 @@ bool RemoteReadBufferCache::clearLocalCache() break; auto cache_it = caches.find(*it); if (cache_it == caches.end()) - { - throw Exception("file not found in cache?" + *it, ErrorCodes::LOGICAL_ERROR); - } + throw Exception(ErrorCodes::LOGICAL_ERROR, "Found no entry in local cache with key: {}", *it); + if (cache_it->second.cache_controller->closable()) { total_size @@ -587,7 +583,7 @@ bool RemoteReadBufferCache::clearLocalCache() else break; } - LOG_TRACE(log, "keys size:{}, total_size:{}, limit size:{}", keys.size(), total_size, limit_size); + LOG_TRACE(log, "After clear local cache, keys size:{}, total_size:{}, limit size:{}", keys.size(), total_size, limit_size); return total_size < limit_size; } diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index cdcb372caf4..424aec23e31 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -141,23 +141,26 @@ private: class LocalCachedFileReader { public: - LocalCachedFileReader(RemoteCacheController * cache_controller_, size_t size_); + LocalCachedFileReader(RemoteCacheController * cache_controller_, size_t file_size_); ~LocalCachedFileReader(); // expect to read size bytes into buf, return is the real bytes read size_t read(char * buf, size_t size); - inline off_t getOffset() const { return static_cast(offset); } - size_t size(); off_t seek(off_t offset); + inline String getPath() const { return local_path; } + inline off_t getOffset() const { return static_cast(offset); } + size_t getSize(); + private: - std::mutex mutex; - size_t offset; - size_t file_size; - FILE * fs; - String local_path; RemoteCacheController * cache_controller; + size_t file_size; + size_t offset; + + std::mutex mutex; + FILE * file_stream; + String local_path; Poco::Logger * log = &Poco::Logger::get("RemoteReadBufferCache"); }; @@ -174,10 +177,10 @@ public: static std::unique_ptr create(const RemoteFileMetadata & remote_file_meta, std::unique_ptr read_buffer); bool nextImpl() override; - inline bool seekable() { return file_reader != nullptr && file_reader->size() > 0; } + inline bool seekable() { return file_reader != nullptr && file_reader->getSize() > 0; } off_t seek(off_t off, int whence) override; off_t getPosition() override; - inline size_t size() { return file_reader->size(); } + inline size_t size() { return file_reader->getSize(); } private: std::shared_ptr file_reader; From 03e7f0f6f7dd856dad2df3e3a606f72462f5163c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 29 Nov 2021 18:23:57 +0800 Subject: [PATCH 0090/1260] extract hivemetastore client from context --- src/IO/RemoteReadBufferCache.h | 2 +- src/Interpreters/Context.cpp | 67 ------------------------------- src/Interpreters/Context.h | 9 ----- src/Storages/Hive/HiveCommon.cpp | 57 ++++++++++++++++++++++++++ src/Storages/Hive/HiveCommon.h | 22 +++++++++- src/Storages/Hive/StorageHive.cpp | 36 ++++++++--------- src/Storages/Hive/StorageHive.h | 5 +-- 7 files changed, 99 insertions(+), 99 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 424aec23e31..6d566fb92f3 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -166,7 +166,7 @@ private: }; /* - * FIXME:RemoteReadBuffer derive from SeekableReadBuffer may case some risks, since it's not seekable in some cases + * FIXME:RemoteReadBuffer derive from SeekableReadBuffer may cause some risks, since it's not seekable in some cases * But SeekableReadBuffer is not a interface which make it hard to fixup. */ class RemoteReadBuffer : public BufferWithOwnMemory diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 933b92dbe15..288555bafd2 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -88,15 +88,6 @@ #include #include -#if USE_HIVE -#include -#include -#include -#include -#include -#include -#endif - namespace fs = std::filesystem; namespace ProfileEvents @@ -157,10 +148,6 @@ struct ContextSharedPart mutable std::mutex storage_policies_mutex; /// Separate mutex for re-initialization of zookeeper session. This operation could take a long time and must not interfere with another operations. mutable std::mutex zookeeper_mutex; -#if USE_HIVE - /// Separate mutex for re-initialization of hive metastore client. This operation could take a long time and must not interfere with another operations. - mutable std::mutex hive_metastore_mutex; -#endif mutable zkutil::ZooKeeperPtr zookeeper; /// Client for ZooKeeper. ConfigurationPtr zookeeper_config; /// Stores zookeeper configs @@ -173,10 +160,6 @@ struct ContextSharedPart mutable std::map auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients. ConfigurationPtr auxiliary_zookeepers_config; /// Stores auxiliary zookeepers configs -#if USE_HIVE - mutable std::map hive_metastore_clients; /// Map for hive metastore clients -#endif - String interserver_io_host; /// The host name by which this server is available for other servers. UInt16 interserver_io_port = 0; /// and port. String interserver_scheme; /// http or https @@ -1829,56 +1812,6 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const return shared->zookeeper; } -#if USE_HIVE -HiveMetastoreClientPtr Context::getHiveMetastoreClient(const String & name) const -{ - using namespace apache::thrift; - using namespace apache::thrift::protocol; - using namespace apache::thrift::transport; - using namespace Apache::Hadoop::Hive; - - std::lock_guard lock(shared->hive_metastore_mutex); - auto it = shared->hive_metastore_clients.find(name); - if (it == shared->hive_metastore_clients.end() || it->second->isExpired()) - { - // connect to hive metastore - Poco::URI hive_metastore_url(name); - const auto & host = hive_metastore_url.getHost(); - auto port = hive_metastore_url.getPort(); - - std::shared_ptr socket = std::make_shared(host, port); - socket->setKeepAlive(true); - socket->setConnTimeout(60000); - socket->setRecvTimeout(60000); - socket->setSendTimeout(60000); - std::shared_ptr transport(new TBufferedTransport(socket)); - std::shared_ptr protocol(new TBinaryProtocol(transport)); - std::shared_ptr client = std::make_shared(protocol); - try - { - transport->open(); - } - catch (TException & tx) - { - throw Exception("connect to hive metastore:" + name + " failed." + tx.what(), ErrorCodes::BAD_ARGUMENTS); - } - - if (it == shared->hive_metastore_clients.end()) - { - HiveMetastoreClientPtr hms_client = std::make_shared(std::move(client), shared_from_this()); - shared->hive_metastore_clients[name] = hms_client; - return hms_client; - } - else - { - it->second->setClient(std::move(client)); - return it->second; - } - } - return it->second; -} -#endif - namespace { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index dcb1b353053..edcb3cf5e1a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -175,11 +175,6 @@ private: std::unique_ptr shared; }; -#if USE_HIVE -class HiveMetastoreClient; -using HiveMetastoreClientPtr = std::shared_ptr; -#endif - /** A set of known objects that can be used in the query. * Consists of a shared part (always common to all sessions and queries) * and copied part (which can be its own for each session or query). @@ -693,10 +688,6 @@ public: // Reload Zookeeper void reloadZooKeeperIfChanged(const ConfigurationPtr & config) const; -#if USE_HIVE - HiveMetastoreClientPtr getHiveMetastoreClient(const String & name) const; -#endif - void setSystemZooKeeperLogAfterInitializationIfNeeded(); /// Create a cache of uncompressed blocks of specified size. This can be done only once. diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index da9e2b31abc..31cee25272c 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -1,3 +1,7 @@ +#include +#include +#include +#include #include #if USE_HIVE @@ -184,6 +188,59 @@ std::vector HiveMetastoreClient::HiveTableMetadat return result; } +HiveMetastoreClientFactory & HiveMetastoreClientFactory::instance() +{ + static HiveMetastoreClientFactory factory; + return factory; +} + +HiveMetastoreClientPtr HiveMetastoreClientFactory::getOrCreate(const String & name, ContextPtr context) +{ + using namespace apache::thrift; + using namespace apache::thrift::protocol; + using namespace apache::thrift::transport; + using namespace Apache::Hadoop::Hive; + + std::lock_guard lock(mutex); + auto it = clients.find(name); + if (it == clients.end() || it->second->isExpired()) + { + // connect to hive metastore + Poco::URI hive_metastore_url(name); + const auto & host = hive_metastore_url.getHost(); + auto port = hive_metastore_url.getPort(); + + std::shared_ptr socket = std::make_shared(host, port); + socket->setKeepAlive(true); + socket->setConnTimeout(conn_timeout_ms); + socket->setRecvTimeout(recv_timeout_ms); + socket->setSendTimeout(send_timeout_ms); + std::shared_ptr transport(new TBufferedTransport(socket)); + std::shared_ptr protocol(new TBinaryProtocol(transport)); + std::shared_ptr thrift_client = std::make_shared(protocol); + try + { + transport->open(); + } + catch (TException & tx) + { + throw Exception("connect to hive metastore:" + name + " failed." + tx.what(), ErrorCodes::BAD_ARGUMENTS); + } + + if (it == clients.end()) + { + HiveMetastoreClientPtr client = std::make_shared(std::move(thrift_client), context); + clients[name] = client; + return client; + } + else + { + it->second->setClient(std::move(thrift_client)); + return it->second; + } + } + return it->second; +} } #endif diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index af056739b6b..45458a05666 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -92,8 +92,28 @@ private: mutable std::mutex mutex; std::atomic expired{false}; - Poco::Logger * log = &Poco::Logger::get("HMSClient"); + Poco::Logger * log = &Poco::Logger::get("HiveMetastoreClient"); }; + +using HiveMetastoreClientPtr = std::shared_ptr; + +class HiveMetastoreClientFactory final : private boost::noncopyable +{ +public: + static HiveMetastoreClientFactory & instance(); + + HiveMetastoreClientPtr getOrCreate(const String & name, ContextPtr context); + +private: + std::mutex mutex; + std::map clients; + + const int conn_timeout_ms = 60000; + const int recv_timeout_ms = 60000; + const int send_timeout_ms = 60000; +}; + } + #endif diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index e288ecc2393..ef3397f5232 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -66,7 +66,7 @@ class HiveSource : public SourceWithProgress, WithContext public: struct SourcesInfo { - HiveMetastoreClientPtr hms_client; + HiveMetastoreClientPtr hive_metastore_client; std::string database; std::string table_name; HiveFiles hive_files; @@ -150,7 +150,7 @@ public: { if (e.code() == ErrorCodes::CANNOT_OPEN_FILE) { - source_info->hms_client->clearTableMeta(source_info->database, source_info->table_name); + source_info->hive_metastore_client->clearTableMeta(source_info->database, source_info->table_name); throw; } } @@ -228,7 +228,7 @@ private: StorageHive::StorageHive( - const String & hms_url_, + const String & hive_metastore_url_, const String & hive_database_, const String & hive_table_, const StorageID & table_id_, @@ -240,13 +240,13 @@ StorageHive::StorageHive( ContextPtr context_) : IStorage(table_id_) , WithContext(context_) - , hms_url(hms_url_) + , hive_metastore_url(hive_metastore_url_) , hive_database(hive_database_) , hive_table(hive_table_) , partition_by_ast(partition_by_ast_) , storage_settings(std::move(storage_settings_)) { - getContext()->getRemoteHostFilter().checkURL(Poco::URI(hms_url)); + getContext()->getRemoteHostFilter().checkURL(Poco::URI(hive_metastore_url)); StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); @@ -254,13 +254,13 @@ StorageHive::StorageHive( storage_metadata.setComment(comment_); setInMemoryMetadata(storage_metadata); - auto hms_client = getContext()->getHiveMetastoreClient(hms_url); - auto table_meta = hms_client->getTableMetadata(hive_database, hive_table); + auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url, getContext()); + auto hive_table_metadata = hive_metastore_client->getTableMetadata(hive_database, hive_table); - hdfs_namenode_url = getNameNodeUrl(table_meta->getTable()->sd.location); - table_schema = table_meta->getTable()->sd.cols; + hdfs_namenode_url = getNameNodeUrl(hive_table_metadata->getTable()->sd.location); + table_schema = hive_table_metadata->getTable()->sd.cols; - FileFormat hdfs_file_format = toFileFormat(table_meta->getTable()->sd.inputFormat); + FileFormat hdfs_file_format = toFileFormat(hive_table_metadata->getTable()->sd.inputFormat); switch (hdfs_file_format) { case FileFormat::TEXT: @@ -387,13 +387,13 @@ Pipe StorageHive::read( { HDFSBuilderWrapper builder = createHDFSBuilder(hdfs_namenode_url, context_->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); - auto hms_client = context_->getHiveMetastoreClient(hms_url); - auto table_meta_cntrl = hms_client->getTableMetadata(hive_database, hive_table); + auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url, getContext()); + auto hive_table_metadata = hive_metastore_client->getTableMetadata(hive_database, hive_table); // List files under partition directory in HDFS - auto list_paths = [table_meta_cntrl, &fs](const String & path) { return table_meta_cntrl->getLocationFiles(fs, path); }; + auto list_paths = [hive_table_metadata, &fs](const String & path) { return hive_table_metadata->getLocationFiles(fs, path); }; - std::vector partitions = table_meta_cntrl->getPartitions(); + std::vector partitions = hive_table_metadata->getPartitions(); HiveFiles hive_files; // hive files to read std::mutex hive_files_mutex; // Mutext to protect hive_files, which maybe appended in multiple threads @@ -529,7 +529,7 @@ Pipe StorageHive::read( } else if (partition_name_types.empty()) // Partition keys is empty { - auto paths = list_paths(table_meta_cntrl->getTable()->sd.location); + auto paths = list_paths(hive_table_metadata->getTable()->sd.location); for (const auto & path : paths) { pool.scheduleOrThrowOnError([&] { append_hive_files(path, {}); }); @@ -545,7 +545,7 @@ Pipe StorageHive::read( sources_info->hive_files = std::move(hive_files); sources_info->database = hive_database; sources_info->table_name = hive_table; - sources_info->hms_client = hms_client; + sources_info->hive_metastore_client = hive_metastore_client; sources_info->partition_name_types = partition_name_types; for (const auto & column : column_names) { @@ -608,11 +608,11 @@ void registerStorageHive(StorageFactory & factory) for (auto & engine_arg : engine_args) engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext()); - const String & hms_url = engine_args[0]->as().value.safeGet(); + const String & hive_metastore_url = engine_args[0]->as().value.safeGet(); const String & hive_database = engine_args[1]->as().value.safeGet(); const String & hive_table = engine_args[2]->as().value.safeGet(); return StorageHive::create( - hms_url, + hive_metastore_url, hive_database, hive_table, args.table_id, diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 0daf63e3f32..007f25539af 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -101,7 +101,7 @@ public: protected: StorageHive( - const String & hms_url_, + const String & hive_metastore_url_, const String & hive_database_, const String & hive_table_, const StorageID & table_id_, @@ -116,8 +116,7 @@ protected: static ASTPtr extractKeyExpressionList(const ASTPtr & node); private: - // hive metastore url - String hms_url; + String hive_metastore_url; // hive database and table String hive_database; From bdb2fa1e9e88834d692d4ff8bee211af5bb58973 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 29 Nov 2021 18:29:34 +0800 Subject: [PATCH 0091/1260] revert context.h context.cpp --- src/Interpreters/Context.cpp | 3 +++ src/Interpreters/Context.h | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 288555bafd2..e9d3d76e92a 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -88,6 +88,7 @@ #include #include + namespace fs = std::filesystem; namespace ProfileEvents @@ -1831,8 +1832,10 @@ bool checkZooKeeperConfigIsLocal(const Poco::Util::AbstractConfiguration & confi } return false; } + } + bool Context::tryCheckClientConnectionToMyKeeperCluster() const { try diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index edcb3cf5e1a..5948cc7f7a7 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -13,7 +13,6 @@ #include #include #include -#include #include #include "config_core.h" @@ -29,6 +28,7 @@ namespace Poco::Net { class IPAddress; } namespace zkutil { class ZooKeeper; } + namespace DB { From d0546f5c9c9c0dbc9b8b20add8f7d9050a821abd Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 29 Nov 2021 18:39:50 +0800 Subject: [PATCH 0092/1260] apply swithc use_local_cache_for_remote_fs --- src/Storages/Hive/StorageHive.cpp | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index ef3397f5232..e7907e91f9f 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -155,10 +155,15 @@ public: } } - std::unique_ptr remote_read_buf = RemoteReadBuffer::create( - {"Hive", getNameNodeCluster(hdfs_namenode_url), uri_with_path, curr_file->getLastModTs(), curr_file->getSize()}, - std::move(raw_read_buf)); - // std::unique_ptr remote_read_buf = std::move(raw_read_buf); + // Use local cache for remote filesystem if enabled. + std::unique_ptr remote_read_buf; + if (RemoteReadBufferCache::instance().isInitialized() && getContext()->getSettingsRef().use_local_cache_for_remote_fs) + remote_read_buf = RemoteReadBuffer::create( + {"Hive", getNameNodeCluster(hdfs_namenode_url), uri_with_path, curr_file->getLastModTs(), curr_file->getSize()}, + std::move(raw_read_buf)); + else + remote_read_buf = std::move(raw_read_buf); + if (curr_file->getFormat() == StorageHive::FileFormat::TEXT) read_buf = wrapReadBufferWithCompressionMethod(std::move(remote_read_buf), compression); else From 6922f09ea3fcb5e166bed4c41b0413826e0099b4 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 29 Nov 2021 20:19:36 +0800 Subject: [PATCH 0093/1260] reuse seekable read buffer with size --- src/IO/RemoteReadBufferCache.cpp | 54 +++++++++++-------- src/IO/RemoteReadBufferCache.h | 17 +++--- .../Formats/Impl/ArrowBufferedStreams.cpp | 6 --- 3 files changed, 39 insertions(+), 38 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 19a98f234ab..904b2c0335c 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -14,6 +14,9 @@ #include #include + +namespace fs = std::filesystem; + namespace DB { namespace ErrorCodes @@ -65,7 +68,7 @@ RemoteCacheController::RemoteCacheController( const RemoteFileMetadata & remote_file_meta, const String & local_path_, size_t cache_bytes_before_flush_, - std::shared_ptr readbuffer_, + std::shared_ptr read_buffer_, std::function const & finish_callback) : schema(remote_file_meta.schema) , cluster(remote_file_meta.cluster) @@ -76,9 +79,10 @@ RemoteCacheController::RemoteCacheController( , local_cache_bytes_read_before_flush(cache_bytes_before_flush_) , download_finished(false) , current_offset(0) - , remote_readbuffer(readbuffer_) + , remote_read_buffer(read_buffer_) { - if (remote_readbuffer) + /// readbuffer == nullptr if `RemoteCacheController` is created in `initOnce`, when metadata and local cache already exist. + if (remote_read_buffer) { // setup local files out_file = std::make_unique(fs::path(local_path_) / "data.bin", std::ios::out | std::ios::binary); @@ -132,12 +136,12 @@ void RemoteCacheController::backgroundDownload(std::functioneof()) + while (!remote_read_buffer->eof()) { - size_t bytes = remote_readbuffer->available(); + size_t bytes = remote_read_buffer->available(); - out_file->write(remote_readbuffer->position(), bytes); - remote_readbuffer->position() += bytes; + out_file->write(remote_read_buffer->position(), bytes); + remote_read_buffer->position() += bytes; total_bytes += bytes; before_unflush_bytes += bytes; if (before_unflush_bytes >= local_cache_bytes_read_before_flush) @@ -156,8 +160,8 @@ void RemoteCacheController::backgroundDownload(std::functionclose(); - out_file = nullptr; - remote_readbuffer = nullptr; + out_file.reset(); + remote_read_buffer.reset(); lock.unlock(); more_data_signal.notify_all(); finish_callback(this); @@ -212,16 +216,20 @@ std::pair RemoteCacheController::allocFile() return {fs, result_local_path}; } -void RemoteCacheController::deallocFile(FILE * fs) +void RemoteCacheController::deallocFile(FILE * file_stream) { { std::lock_guard lock{mutex}; - auto it = opened_file_streams.find(fs); + auto it = opened_file_streams.find(file_stream); if (it == opened_file_streams.end()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Try to deallocate file {} with invalid handler", remote_path); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Try to deallocate file with invalid handler remote path: {}, local path: {}", + remote_path, + local_path); opened_file_streams.erase(it); } - fclose(fs); + fclose(file_stream); } LocalCachedFileReader::LocalCachedFileReader(RemoteCacheController * cache_controller_, size_t file_size_) @@ -260,6 +268,7 @@ off_t LocalCachedFileReader::seek(off_t off) offset = off; return off; } + size_t LocalCachedFileReader::getSize() { if (file_size != 0) @@ -276,7 +285,7 @@ size_t LocalCachedFileReader::getSize() } // the size need be equal to the original buffer -RemoteReadBuffer::RemoteReadBuffer(size_t buff_size) : BufferWithOwnMemory(buff_size) +RemoteReadBuffer::RemoteReadBuffer(size_t buff_size) : BufferWithOwnMemory(buff_size) { } @@ -316,7 +325,7 @@ std::unique_ptr RemoteReadBuffer::create(const RemoteFileMetad if (remote_read_buffer->file_reader == nullptr) { LOG_ERROR(log, "Failed to allocate local file for remote path: {}, reason: {}", remote_path, error); - remote_read_buffer->original_readbuffer = srb; + remote_read_buffer->original_read_buffer = srb; } return remote_read_buffer; } @@ -329,20 +338,19 @@ bool RemoteReadBuffer::nextImpl() if (bytes_read) working_buffer.resize(bytes_read); else - { return false; - } } - else // in the case we cannot use local cache, read from the original readbuffer directly + else { - if (original_readbuffer) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid original read buffer"); + // In the case we cannot use local cache, read from the original readbuffer directly + if (!original_read_buffer) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Original read buffer is not initialized. It's a bug"); - auto status = original_readbuffer->next(); - // we don't need to worry about the memory buffer allocated in RemoteReadBuffer, since it is owned by + auto status = original_read_buffer->next(); + // We don't need to worry about the memory buffer allocated in RemoteReadBuffer, since it is owned by // BufferWithOwnMemory, BufferWithOwnMemory would release it. if (status) - BufferBase::set(original_readbuffer->buffer().begin(), original_readbuffer->buffer().size(), original_readbuffer->offset()); + BufferBase::set(original_read_buffer->buffer().begin(), original_read_buffer->buffer().size(), original_read_buffer->offset()); return status; } return true; diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 6d566fb92f3..350e41beb6c 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -12,7 +12,6 @@ #include #include -namespace fs = std::filesystem; namespace DB { @@ -55,7 +54,7 @@ public: const RemoteFileMetadata & meta, const String & local_path_, size_t cache_bytes_before_flush_, - std::shared_ptr readbuffer_, + std::shared_ptr read_buffer_, std::function const & finish_callback); ~RemoteCacheController(); @@ -69,7 +68,7 @@ public: * It will be empty if the file has not been downloaded */ std::pair allocFile(); - void deallocFile(FILE * fs_); + void deallocFile(FILE * file_stream); /** * when allocFile be called, count++. deallocFile be called, count--. @@ -78,7 +77,7 @@ public: inline bool closable() { std::lock_guard lock{mutex}; - return opened_file_streams.empty() && remote_readbuffer == nullptr; + return opened_file_streams.empty() && remote_read_buffer == nullptr; } void close(); @@ -129,7 +128,7 @@ private: bool download_finished; size_t current_offset; - std::shared_ptr remote_readbuffer; + std::shared_ptr remote_read_buffer; std::unique_ptr out_file; Poco::Logger * log = &Poco::Logger::get("RemoteReadBufferCache"); @@ -166,10 +165,10 @@ private: }; /* - * FIXME:RemoteReadBuffer derive from SeekableReadBuffer may cause some risks, since it's not seekable in some cases + * FIXME:RemoteReadBuffer derive from SeekableReadBufferWithSize may cause some risks, since it's not seekable in some cases * But SeekableReadBuffer is not a interface which make it hard to fixup. */ -class RemoteReadBuffer : public BufferWithOwnMemory +class RemoteReadBuffer : public BufferWithOwnMemory { public: explicit RemoteReadBuffer(size_t buff_size); @@ -180,11 +179,11 @@ public: inline bool seekable() { return file_reader != nullptr && file_reader->getSize() > 0; } off_t seek(off_t off, int whence) override; off_t getPosition() override; - inline size_t size() { return file_reader->getSize(); } + std::optional getTotalSize() override { return file_reader->getSize(); } private: std::shared_ptr file_reader; - std::shared_ptr original_readbuffer; + std::shared_ptr original_read_buffer; }; class RemoteReadBufferCache diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp index 5c27e5c5770..474071a1b9e 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp @@ -159,12 +159,6 @@ std::shared_ptr asArrowFile(ReadBuffer & in, const return std::make_shared(*seekable_in); } - if (auto * fd_in = dynamic_cast(&in)) - { - if (fd_in->seekable()) - return std::make_shared(*fd_in, fd_in->size()); - } - // fallback to loading the entire file in memory std::string file_data; { From 0f9038ebed85fea44f99df317f7f9374010e0cfb Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Tue, 23 Nov 2021 16:57:24 +0300 Subject: [PATCH 0094/1260] Zero-copy: move shared mark outside table node in ZooKeeper --- src/Disks/DiskDecorator.h | 12 + src/Disks/IDisk.cpp | 18 + src/Disks/IDisk.h | 20 ++ src/Disks/IDiskRemote.cpp | 25 ++ src/Disks/IDiskRemote.h | 12 + src/Storages/IStorage.h | 3 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 6 + src/Storages/MergeTree/IMergeTreeDataPart.h | 8 +- src/Storages/MergeTree/MergeTreeData.cpp | 128 ++++++- src/Storages/MergeTree/MergeTreeData.h | 2 + src/Storages/MergeTree/MergeTreeSettings.h | 6 +- src/Storages/StorageReplicatedMergeTree.cpp | 336 +++++++++++++++--- src/Storages/StorageReplicatedMergeTree.h | 26 +- 13 files changed, 539 insertions(+), 63 deletions(-) diff --git a/src/Disks/DiskDecorator.h b/src/Disks/DiskDecorator.h index 5b88f4a36fa..5890a772ed7 100644 --- a/src/Disks/DiskDecorator.h +++ b/src/Disks/DiskDecorator.h @@ -70,6 +70,18 @@ public: void startup() override; void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap & map) override; + std::unique_ptr readMetaFile( + const String & path, + const ReadSettings & settings, + std::optional size) const override { return delegate->readMetaFile(path, settings, size); } + + std::unique_ptr writeMetaFile( + const String & path, + size_t buf_size, + WriteMode mode) override { return delegate->writeMetaFile(path, buf_size, mode); } + + UInt32 getRefCount(const String & path) const override { return delegate->getRefCount(path); } + protected: Executor & getExecutor() override; diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 42d5f5fce10..404f759d6cb 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -86,4 +86,22 @@ SyncGuardPtr IDisk::getDirectorySyncGuard(const String & /* path */) const return nullptr; } +std::unique_ptr IDisk::readMetaFile( + const String & path, + const ReadSettings & settings, + std::optional size) const +{ + LOG_TRACE(&Poco::Logger::get("IDisk"), "Read local metafile: {}", path); + return readFile(path, settings, size); +} + +std::unique_ptr IDisk::writeMetaFile( + const String & path, + size_t buf_size, + WriteMode mode) +{ + LOG_TRACE(&Poco::Logger::get("IDisk"), "Write local metafile: {}", path); + return writeFile(path, buf_size, mode); +} + } diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 0a63421ae5c..34e5a229c0a 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -247,6 +247,26 @@ public: /// Applies new settings for disk in runtime. virtual void applyNewSettings(const Poco::Util::AbstractConfiguration &, ContextPtr, const String &, const DisksMap &) {} + /// Open the local file for read and return ReadBufferFromFileBase object. + /// Overriden in IDiskRemote. + /// Used for work with custom metadata. + virtual std::unique_ptr readMetaFile( + const String & path, + const ReadSettings & settings = ReadSettings{}, + std::optional size = {}) const; + + /// Open the local file for write and return WriteBufferFromFileBase object. + /// Overriden in IDiskRemote. + /// Used for work with custom metadata. + virtual std::unique_ptr writeMetaFile( + const String & path, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + WriteMode mode = WriteMode::Rewrite); + + /// Return reference count for remote FS. + /// Overriden in IDiskRemote. + virtual UInt32 getRefCount(const String &) const { return 0; } + protected: friend class DiskDecorator; diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index e920e6fd5b9..0162b0fc8da 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -484,6 +484,7 @@ bool IDiskRemote::tryReserve(UInt64 bytes) String IDiskRemote::getUniqueId(const String & path) const { + LOG_TRACE(log, "Remote path: {}, Path: {}", remote_fs_root_path, path); Metadata metadata(remote_fs_root_path, metadata_disk, path); String id; if (!metadata.remote_fs_objects.empty()) @@ -500,4 +501,28 @@ AsynchronousReaderPtr IDiskRemote::getThreadPoolReader() return reader; } +std::unique_ptr IDiskRemote::readMetaFile( + const String & path, + const ReadSettings & settings, + std::optional size) const +{ + LOG_TRACE(log, "Read metafile: {}", path); + return metadata_disk->readFile(path, settings, size); +} + +std::unique_ptr IDiskRemote::writeMetaFile( + const String & path, + size_t buf_size, + WriteMode mode) +{ + LOG_TRACE(log, "Write metafile: {}", path); + return metadata_disk->writeFile(path, buf_size, mode); +} + +UInt32 IDiskRemote::getRefCount(const String & path) const +{ + auto meta = readMeta(path); + return meta.ref_count; +} + } diff --git a/src/Disks/IDiskRemote.h b/src/Disks/IDiskRemote.h index c9b8fe81d9f..9bfaead3def 100644 --- a/src/Disks/IDiskRemote.h +++ b/src/Disks/IDiskRemote.h @@ -136,6 +136,18 @@ public: static AsynchronousReaderPtr getThreadPoolReader(); + virtual std::unique_ptr readMetaFile( + const String & path, + const ReadSettings & settings = ReadSettings{}, + std::optional size = {}) const override; + + virtual std::unique_ptr writeMetaFile( + const String & path, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + WriteMode mode = WriteMode::Rewrite) override; + + UInt32 getRefCount(const String & path) const override; + protected: Poco::Logger * log; const String name; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index fa5f2c28b06..89adf33df14 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -137,6 +137,9 @@ public: /// Returns true if the storage replicates SELECT, INSERT and ALTER commands among replicas. virtual bool supportsReplication() const { return false; } + /// Returns replica name for replicated storage + virtual String getReplicaName() const { return ""; } + /// Returns true if the storage supports parallel insert. virtual bool supportsParallelInsert() const { return false; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 93149f87f99..81920735829 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1561,6 +1561,12 @@ String IMergeTreeDataPart::getUniqueId() const } +UInt32 IMergeTreeDataPart::getRefCount() const +{ + return volume->getDisk()->getRefCount(fs::path(getFullRelativePath()) / "checksums.txt"); +} + + String IMergeTreeDataPart::getZeroLevelPartBlockID() const { if (info.level != 0) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index f0dff289ced..12e1cc9738b 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -398,10 +398,14 @@ public: /// Returns serialization for column according to files in which column is written in part. SerializationPtr getSerializationForColumn(const NameAndTypePair & column) const; - /// Return some uniq string for file - /// Required for distinguish different copies of the same part on S3 + /// Return some uniq string for file. + /// Required for distinguish different copies of the same part on remote FS. String getUniqueId() const; + /// Return hardlink count for part. + /// Required for keep data on remote FS when part has shadow copies. + UInt32 getRefCount() const; + protected: /// Total size of all columns, calculated once in calcuateColumnSizesOnDisk diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ed13f03ffdc..a7309577fa6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5111,6 +5111,76 @@ MergeTreeData::MatcherFn MergeTreeData::getPartitionMatcher(const ASTPtr & parti }; } +struct FreezeMetaData +{ +public: + void fill(const IMergeTreeDataPart & part) + { + is_replicated = part.storage.supportsReplication(); + is_remote = part.storage.isRemote(); + replica_name = part.storage.getReplicaName(); + zookeeper_name = part.storage.getZooKeeperName(); + } + + void save(DiskPtr disk, const String & path) const + { + auto file_path = getFileName(path); + auto buffer = disk->writeMetaFile(file_path); + writeIntText(version, *buffer); + buffer->write("\n", 1); + writeBoolText(is_replicated, *buffer); + buffer->write("\n", 1); + writeBoolText(is_remote, *buffer); + buffer->write("\n", 1); + writeString(replica_name, *buffer); + buffer->write("\n", 1); + writeString(zookeeper_name, *buffer); + buffer->write("\n", 1); + } + + bool load(DiskPtr disk, const String & path) + { + auto file_path = getFileName(path); + if (!disk->exists(file_path)) + return false; + auto buffer = disk->readMetaFile(file_path); + readIntText(version, *buffer); + if (version != 1) + { + LOG_ERROR(&Poco::Logger::get("FreezeMetaData"), "Unknown freezed metadata version: {}", version); + return false; + } + DB::assertChar('\n', *buffer); + readBoolText(is_replicated, *buffer); + DB::assertChar('\n', *buffer); + readBoolText(is_remote, *buffer); + DB::assertChar('\n', *buffer); + readString(replica_name, *buffer); + DB::assertChar('\n', *buffer); + readString(zookeeper_name, *buffer); + DB::assertChar('\n', *buffer); + return true; + } + + void clean(DiskPtr disk, const String & path) + { + disk->removeFileIfExists(getFileName(path)); + } + +private: + static String getFileName(const String & path) + { + return fs::path(path) / "frozen_metadata.txt"; + } + +public: + int version = 1; + bool is_replicated; + bool is_remote; + String replica_name; + String zookeeper_name; +}; + PartitionCommandsResultInfo MergeTreeData::freezePartition( const ASTPtr & partition_ast, const StorageMetadataPtr & metadata_snapshot, @@ -5162,7 +5232,9 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( LOG_DEBUG(log, "Freezing part {} snapshot will be placed at {}", part->name, backup_path); - part->volume->getDisk()->createDirectories(backup_path); + auto disk = part->volume->getDisk(); + + disk->createDirectories(backup_path); String src_part_path = part->getFullRelativePath(); String backup_part_path = fs::path(backup_path) / relative_data_path / part->relative_path; @@ -5173,16 +5245,23 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( src_part_path = fs::path(relative_data_path) / flushed_part_path / ""; } - localBackup(part->volume->getDisk(), src_part_path, backup_part_path); + localBackup(disk, src_part_path, backup_part_path); - part->volume->getDisk()->removeFileIfExists(fs::path(backup_part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); + if (disk->supportZeroCopyReplication()) + { + FreezeMetaData meta; + meta.fill(*part); + meta.save(disk, backup_part_path); + } + + disk->removeFileIfExists(fs::path(backup_part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); part->is_frozen.store(true, std::memory_order_relaxed); result.push_back(PartitionCommandResultInfo{ .partition_id = part->info.partition_id, .part_name = part->name, - .backup_path = fs::path(part->volume->getDisk()->getPath()) / backup_path, - .part_backup_path = fs::path(part->volume->getDisk()->getPath()) / backup_part_path, + .backup_path = fs::path(disk->getPath()) / backup_path, + .part_backup_path = fs::path(disk->getPath()) / backup_part_path, .backup_name = backup_name, }); ++parts_processed; @@ -5237,7 +5316,42 @@ PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn const auto & path = it->path(); - disk->removeRecursive(path); + bool keep_shared = false; + + if (disk->supportZeroCopyReplication()) + { + FreezeMetaData meta; + if (meta.load(disk, path) && meta.is_replicated) + { + zkutil::ZooKeeperPtr zookeeper; + if (meta.zookeeper_name == "default") + { + zookeeper = getContext()->getZooKeeper(); + } + else + { + zookeeper = getContext()->getAuxiliaryZooKeeper(meta.zookeeper_name); + } + + if (zookeeper) + { + fs::path checksums = fs::path(path) / "checksums.txt"; + if (disk->exists(checksums)) + { + auto ref_count = disk->getRefCount(checksums); + if (ref_count == 0) + { + String id = disk->getUniqueId(checksums); + keep_shared = !StorageReplicatedMergeTree::unlockSharedDataById(id, partition_directory, + meta.replica_name, disk, zookeeper, getContext()->getReplicatedMergeTreeSettings(), log, + nullptr); + } + } + } + } + } + + disk->removeSharedRecursive(path, keep_shared); result.push_back(PartitionCommandResultInfo{ .partition_id = partition_id, @@ -5247,7 +5361,7 @@ PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn .backup_name = backup_name, }); - LOG_DEBUG(log, "Unfreezed part by path {}", disk->getPath() + path); + LOG_DEBUG(log, "Unfreezed part by path {}, keep shared data {}", disk->getPath() + path, keep_shared); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 5b1675b06a3..ed9a7f058dd 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -871,6 +871,8 @@ public: /// Overridden in StorageReplicatedMergeTree virtual bool tryToFetchIfShared(const IMergeTreeDataPart &, const DiskPtr &, const String &) { return false; } + virtual String getZooKeeperName() const { return ""; } + /// Parts that currently submerging (merging to bigger parts) or emerging /// (to be appeared after merging finished). These two variables have to be used /// with `currently_submerging_emerging_mutex`. diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index decc72df14c..03e7a562218 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -124,8 +124,10 @@ struct Settings; M(UInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \ M(String, storage_policy, "default", "Name of storage disk policy", 0) \ M(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \ - M(Bool, allow_remote_fs_zero_copy_replication, true, "Allow Zero-copy replication over remote fs", 0) \ - M(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm", 0) \ + M(Bool, allow_remote_fs_zero_copy_replication, true, "Allow Zero-copy replication over remote fs.", 0) \ + M(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for Zero-copy table-independet info.", 0) \ + M(UInt64, need_revert_zero_copy_version, 0, "Revert Zero-copy to old version", 0) \ + M(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm.", 0) \ M(Bool, assign_part_uuids, false, "Generate UUIDs for parts. Before enabling check that all replicas support new format.", 0) \ M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited. This setting is the default that can be overridden by the query-level setting with the same name.", 0) \ M(UInt64, max_concurrent_queries, 0, "Max number of concurrently executed queries related to the MergeTree table (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \ diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 70fbdaa640d..d989c4b2030 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -155,6 +155,8 @@ static const auto QUEUE_UPDATE_ERROR_SLEEP_MS = 1 * 1000; static const auto MUTATIONS_FINALIZING_SLEEP_MS = 1 * 1000; static const auto MUTATIONS_FINALIZING_IDLE_SLEEP_MS = 5 * 1000; +static const int CURRENT_ZERO_COPY_VERSION = 2; + void StorageReplicatedMergeTree::setZooKeeper() { /// Every ReplicatedMergeTree table is using only one ZooKeeper session. @@ -4108,6 +4110,15 @@ void StorageReplicatedMergeTree::startup() assert(prev_ptr == nullptr); getContext()->getInterserverIOHandler().addEndpoint(data_parts_exchange_ptr->getId(replica_path), data_parts_exchange_ptr); + convertZeroCopySchema(); + + is_zero_copy_in_compatible_mode = isZeroCopySchemaInCompatibleMode(); + + if (!is_zero_copy_in_compatible_mode) + { /// All replicas have new version + cleanupOldZeroCopySchema(); + } + /// In this thread replica will be activated. restarting_thread.start(); @@ -7110,6 +7121,7 @@ void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded() background_moves_assignee.start(); } + std::unique_ptr StorageReplicatedMergeTree::getDefaultSettings() const { return std::make_unique(getContext()->getReplicatedMergeTreeSettings()); @@ -7132,26 +7144,15 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part) String id = part.getUniqueId(); boost::replace_all(id, "/", "_"); - String zookeeper_node = fs::path(zookeeper_path) / zero_copy / "shared" / part.name / id / replica_name; + const String * zookeeper_path_ptr = is_zero_copy_in_compatible_mode ? &zookeeper_path : nullptr; - LOG_TRACE(log, "Set zookeeper lock {}", zookeeper_node); - - /// In rare case other replica can remove path between createAncestors and createIfNotExists - /// So we make up to 5 attempts - for (int attempts = 5; attempts > 0; --attempts) + Strings zc_zookeeper_paths = getZeroCopyRootPath(*getDefaultSettings(), zookeeper_path_ptr); + for (const auto & zc_zookeeper_path : zc_zookeeper_paths) { - try - { - zookeeper->createAncestors(zookeeper_node); - zookeeper->createIfNotExists(zookeeper_node, "lock"); - break; - } - catch (const zkutil::KeeperException & e) - { - if (e.code == Coordination::Error::ZNONODE) - continue; - throw; - } + String zookeeper_node = fs::path(zc_zookeeper_path) / zero_copy / "shared" / part.name / id / replica_name; + + LOG_TRACE(log, "Set zookeeper lock {}", zookeeper_node); + createZeroCopyLockNode(zookeeper, zookeeper_node); } } @@ -7163,42 +7164,65 @@ bool StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & par DiskPtr disk = part.volume->getDisk(); if (!disk || !disk->supportZeroCopyReplication()) return true; - String zero_copy = fmt::format("zero_copy_{}", toString(disk->getType())); zkutil::ZooKeeperPtr zookeeper = tryGetZooKeeper(); if (!zookeeper) return true; - String id = part.getUniqueId(); + auto ref_count = part.getRefCount(); + LOG_TRACE(log, "RefCount {} for part {}", ref_count, part.name); + if (ref_count > 0) /// Keep part shard info for frozen backups + return false; + + const String * zookeeper_path_ptr = is_zero_copy_in_compatible_mode ? &zookeeper_path : nullptr; + + return unlockSharedDataById(part.getUniqueId(), part.name, replica_name, disk, zookeeper, *getDefaultSettings(), log, zookeeper_path_ptr); +} + + +bool StorageReplicatedMergeTree::unlockSharedDataById(String id, const String & part_name, + const String & replica_name_, DiskPtr disk, zkutil::ZooKeeperPtr zookeeper_, const MergeTreeSettings & settings, + Poco::Logger * logger, const String * zookeeper_path_ptr) +{ boost::replace_all(id, "/", "_"); - String zookeeper_part_node = fs::path(zookeeper_path) / zero_copy / "shared" / part.name; - String zookeeper_part_uniq_node = fs::path(zookeeper_part_node) / id; - String zookeeper_node = fs::path(zookeeper_part_uniq_node) / replica_name; + String zero_copy = fmt::format("zero_copy_{}", toString(disk->getType())); - LOG_TRACE(log, "Remove zookeeper lock {}", zookeeper_node); + Strings zc_zookeeper_paths = getZeroCopyRootPath(settings, zookeeper_path_ptr); - zookeeper->tryRemove(zookeeper_node); + bool res = true; - Strings children; - zookeeper->tryGetChildren(zookeeper_part_uniq_node, children); - - if (!children.empty()) + for (const auto & zc_zookeeper_path : zc_zookeeper_paths) { - LOG_TRACE(log, "Found zookeper locks for {}", zookeeper_part_uniq_node); - return false; + String zookeeper_part_node = fs::path(zc_zookeeper_path) / zero_copy / "shared" / part_name; + String zookeeper_part_uniq_node = fs::path(zookeeper_part_node) / id; + String zookeeper_node = fs::path(zookeeper_part_uniq_node) / replica_name_; + + LOG_TRACE(logger, "Remove zookeeper lock {}", zookeeper_node); + + zookeeper_->tryRemove(zookeeper_node); + + Strings children; + zookeeper_->tryGetChildren(zookeeper_part_uniq_node, children); + + if (!children.empty()) + { + LOG_TRACE(logger, "Found zookeper locks for {}", zookeeper_part_uniq_node); + res = false; + continue; + } + + zookeeper_->tryRemove(zookeeper_part_uniq_node); + + /// Even when we have lock with same part name, but with different uniq, we can remove files on S3 + children.clear(); + zookeeper_->tryGetChildren(zookeeper_part_node, children); + if (children.empty()) + /// Cleanup after last uniq removing + zookeeper_->tryRemove(zookeeper_part_node); } - zookeeper->tryRemove(zookeeper_part_uniq_node); - - /// Even when we have lock with same part name, but with different uniq, we can remove files on S3 - children.clear(); - zookeeper->tryGetChildren(zookeeper_part_node, children); - if (children.empty()) - /// Cleanup after last uniq removing - zookeeper->tryRemove(zookeeper_part_node); - - return true; + return res; } @@ -7232,19 +7256,28 @@ String StorageReplicatedMergeTree::getSharedDataReplica( return best_replica; String zero_copy = fmt::format("zero_copy_{}", toString(disk_type)); - String zookeeper_part_node = fs::path(zookeeper_path) / zero_copy / "shared" / part.name; - Strings ids; - zookeeper->tryGetChildren(zookeeper_part_node, ids); + const String * zookeeper_path_ptr = is_zero_copy_in_compatible_mode ? &zookeeper_path : nullptr; - Strings replicas; - for (const auto & id : ids) + Strings zc_zookeeper_paths = getZeroCopyRootPath(*getDefaultSettings(), zookeeper_path_ptr); + + std::set replicas; + + for (const auto & zc_zookeeper_path : zc_zookeeper_paths) { - String zookeeper_part_uniq_node = fs::path(zookeeper_part_node) / id; - Strings id_replicas; - zookeeper->tryGetChildren(zookeeper_part_uniq_node, id_replicas); - LOG_TRACE(log, "Found zookeper replicas for {}: {}", zookeeper_part_uniq_node, id_replicas.size()); - replicas.insert(replicas.end(), id_replicas.begin(), id_replicas.end()); + String zookeeper_part_node = fs::path(zc_zookeeper_path) / zero_copy / "shared" / part.name; + + Strings ids; + zookeeper->tryGetChildren(zookeeper_part_node, ids); + + for (const auto & id : ids) + { + String zookeeper_part_uniq_node = fs::path(zookeeper_part_node) / id; + Strings id_replicas; + zookeeper->tryGetChildren(zookeeper_part_uniq_node, id_replicas); + LOG_TRACE(log, "Found zookeper replicas for {}: {}", zookeeper_part_uniq_node, id_replicas.size()); + replicas.insert(id_replicas.begin(), id_replicas.end()); + } } LOG_TRACE(log, "Found zookeper replicas for part {}: {}", part.name, replicas.size()); @@ -7297,6 +7330,19 @@ String StorageReplicatedMergeTree::getSharedDataReplica( return best_replica; } + +Strings StorageReplicatedMergeTree::getZeroCopyRootPath(const MergeTreeSettings & settings, const String * zookeeper_path_ptr) +{ + Strings res; + + res.push_back(settings.remote_fs_zero_copy_zookeeper_path); + if (zookeeper_path_ptr) /// Compatibility mode for cluster with old and new versions + res.push_back(*zookeeper_path_ptr); + + return res; +} + + String StorageReplicatedMergeTree::findReplicaHavingPart( const String & part_name, const String & zookeeper_path_, zkutil::ZooKeeper::Ptr zookeeper_) { @@ -7315,6 +7361,7 @@ String StorageReplicatedMergeTree::findReplicaHavingPart( return {}; } + bool StorageReplicatedMergeTree::checkIfDetachedPartExists(const String & part_name) { fs::directory_iterator dir_end; @@ -7325,6 +7372,7 @@ bool StorageReplicatedMergeTree::checkIfDetachedPartExists(const String & part_n return false; } + bool StorageReplicatedMergeTree::checkIfDetachedPartitionExists(const String & partition_name) { fs::directory_iterator dir_end; @@ -7504,4 +7552,190 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP return true; } + +void StorageReplicatedMergeTree::convertZeroCopySchema() +{ + if (!current_zookeeper) + return; + + int zero_copy_version = 1; + + auto version_path = fs::path(getDefaultSettings()->remote_fs_zero_copy_zookeeper_path.toString()) / "version" / replica_name; + + if (current_zookeeper->exists(version_path)) + zero_copy_version = parse(current_zookeeper->get(version_path)); + + /// Emergency parameter to restore zero-copy marks on old paths + int revert_to_version = getDefaultSettings()->need_revert_zero_copy_version; + + if (!revert_to_version && zero_copy_version >= CURRENT_ZERO_COPY_VERSION) + return; + + if (revert_to_version && zero_copy_version <= revert_to_version) + return; + + int required_zero_copy_version = revert_to_version ? revert_to_version : CURRENT_ZERO_COPY_VERSION; + + auto storage_policy = getStoragePolicy(); + if (!storage_policy) + return; + + auto disks = storage_policy->getDisks(); + + std::set disk_types; + + for (const auto & disk : disks) + if (disk->supportZeroCopyReplication()) + disk_types.insert(toString(disk->getType())); + + if (disk_types.empty()) + return; + + LOG_INFO(log, "Convert zero_copy version from {} to {} for {}", zero_copy_version, required_zero_copy_version, + version_path.string()); + + for (auto const & disk_type : disk_types) + { + String zero_copy = fmt::format("zero_copy_{}", disk_type); + + auto shard_root_v1 = fs::path(zookeeper_path) / zero_copy / "shared"; + auto shard_root_v2 = fs::path(getDefaultSettings()->remote_fs_zero_copy_zookeeper_path.toString()) + / zero_copy / "shared"; + + auto old_shard_root = revert_to_version == 1 ? shard_root_v2 : shard_root_v1; + auto new_shard_root = revert_to_version == 1 ? shard_root_v1 : shard_root_v2; + + Coordination::Stat stat; + Strings parts = current_zookeeper->getChildren(old_shard_root, &stat); + + for (const auto & part : parts) + { + Strings ids = current_zookeeper->getChildren(old_shard_root / part, &stat); + for (const auto & id : ids) + { + if (current_zookeeper->exists(old_shard_root / part / id / replica_name)) + { + auto zookeeper_node = new_shard_root / part / id / replica_name; + createZeroCopyLockNode(current_zookeeper, zookeeper_node.string()); + } + } + } + } + + current_zookeeper->createAncestors(version_path); + + current_zookeeper->createOrUpdate(version_path, std::to_string(required_zero_copy_version), + zkutil::CreateMode::Persistent); + + current_zookeeper->createOrUpdate(version_path / "cleanup_required", std::to_string(zero_copy_version), + zkutil::CreateMode::Persistent); + + LOG_INFO(log, "Convert zero_copy version from {} to {} for {} complete", zero_copy_version, required_zero_copy_version, + version_path.string()); +} + + +void StorageReplicatedMergeTree::cleanupOldZeroCopySchema() +{ + if (!current_zookeeper) + return; + + auto old_version_path = fs::path(getDefaultSettings()->remote_fs_zero_copy_zookeeper_path.toString()) / "version" / replica_name / "cleanup_required"; + + if (!current_zookeeper->exists(old_version_path)) + return; + + auto zero_copy_version = parse(current_zookeeper->get(old_version_path)); + + if (zero_copy_version == 1) + { + auto storage_policy = getStoragePolicy(); + if (!storage_policy) + return; + + auto disks = storage_policy->getDisks(); + + std::set disk_types; + + for (const auto & disk : disks) + if (disk->supportZeroCopyReplication()) + disk_types.insert(toString(disk->getType())); + + if (disk_types.empty()) + return; + + LOG_INFO(log, "Cleanup zero_copy version {}", zero_copy_version); + + for (auto const & disk_type : disk_types) + { + String zero_copy = fmt::format("zero_copy_{}", disk_type); + + auto old_shard_root = fs::path(zookeeper_path) / zero_copy / "shared"; + + Coordination::Stat stat; + Strings parts = current_zookeeper->getChildren(old_shard_root, &stat); + + for (const auto & part : parts) + { + Strings ids = current_zookeeper->getChildren(old_shard_root / part, &stat); + for (const auto & id : ids) + { + current_zookeeper->remove(old_shard_root / part / id / replica_name); + } + } + } + + current_zookeeper->remove(old_version_path); + + LOG_INFO(log, "Cleanup zero_copy version {} complete", zero_copy_version); + } +} + + +void StorageReplicatedMergeTree::createZeroCopyLockNode(const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_node) +{ + /// In rare case other replica can remove path between createAncestors and createIfNotExists + /// So we make up to 5 attempts + + for (int attempts = 5; attempts > 0; --attempts) + { + try + { + zookeeper->createAncestors(zookeeper_node); + zookeeper->createIfNotExists(zookeeper_node, "lock"); + break; + } + catch (const zkutil::KeeperException & e) + { + if (e.code == Coordination::Error::ZNONODE) + continue; + throw; + } + } +} + + +bool StorageReplicatedMergeTree::isZeroCopySchemaInCompatibleMode() const +{ + if (!current_zookeeper) + return false; + + auto version_root_path = fs::path(getDefaultSettings()->remote_fs_zero_copy_zookeeper_path.toString()) / "version"; + + Strings replicas = current_zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); + + for (const auto & replica : replicas) + { + if (!current_zookeeper->exists(version_root_path / replica)) + return true; + int zero_copy_version = parse(current_zookeeper->get(version_root_path / replica)); + if (zero_copy_version < CURRENT_ZERO_COPY_VERSION) + return true; + /// If version is greater that current then other replica has new version. + /// In that case other replica with new version should be in compatible mode. + } + + return false; +} + } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 65daf82a633..7fd1febc9a6 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -237,13 +237,20 @@ public: /// Return false if data is still used by another node bool unlockSharedData(const IMergeTreeDataPart & part) const override; + /// Unlock shared data part in zookeeper by part id + /// Return true if data unlocked + /// Return false if data is still used by another node + static bool unlockSharedDataById(String id, const String & part_name, const String & replica_name_, + DiskPtr disk, zkutil::ZooKeeperPtr zookeeper_, const MergeTreeSettings & settings, Poco::Logger * logger, + const String * zookeeper_path_ptr); + /// Fetch part only if some replica has it on shared storage like S3 bool tryToFetchIfShared(const IMergeTreeDataPart & part, const DiskPtr & disk, const String & path) override; /// Get best replica having this partition on a same type remote disk String getSharedDataReplica(const IMergeTreeDataPart & part, DiskType disk_type) const; - inline String getReplicaName() const { return replica_name; } + inline String getReplicaName() const override { return replica_name; } /// Restores table metadata if ZooKeeper lost it. /// Used only on restarted readonly replicas (not checked). All active (Committed) parts are moved to detached/ @@ -264,6 +271,8 @@ public: bool createEmptyPartInsteadOfLost(zkutil::ZooKeeperPtr zookeeper, const String & lost_part_name); + virtual String getZooKeeperName() const override { return zookeeper_name; } + private: std::atomic_bool are_restoring_replica {false}; @@ -728,6 +737,21 @@ private: PartitionBlockNumbersHolder allocateBlockNumbersInAffectedPartitions( const MutationCommands & commands, ContextPtr query_context, const zkutil::ZooKeeperPtr & zookeeper) const; + static Strings getZeroCopyRootPath(const MergeTreeSettings & settings, const String * zookeeper_path_ptr = nullptr); + + /// Upgrave zero-copy version + /// version 1 - lock for shared part inside table node in ZooKeeper + /// version 2 - lock for shared part in separate node + void convertZeroCopySchema(); + + void cleanupOldZeroCopySchema(); + + static void createZeroCopyLockNode(const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_node); + + bool isZeroCopySchemaInCompatibleMode() const; + + bool is_zero_copy_in_compatible_mode = false; + protected: /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. */ From d409ab0605ca48d59994ec45b8626a6bd21e71fa Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Mon, 29 Nov 2021 19:07:00 +0300 Subject: [PATCH 0095/1260] Fix wait for freeze in tests --- tests/integration/helpers/cluster.py | 12 ++++++++++-- .../test_dictionaries_update_and_reload/test.py | 2 +- .../test_s3_zero_copy_replication/test.py | 2 -- 3 files changed, 11 insertions(+), 5 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 803f56d7cff..7739bfeecaa 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2185,7 +2185,7 @@ class ClickHouseInstance: logging.debug('{} log line(s) matching "{}" appeared in a {:.3f} seconds'.format(repetitions, regexp, wait_duration)) return wait_duration - def file_exists(self, path): + def path_exists(self, path): return self.exec_in_container( ["bash", "-c", "echo $(if [ -e '{}' ]; then echo 'yes'; else echo 'no'; fi)".format(path)]) == 'yes\n' @@ -2620,9 +2620,17 @@ class ClickHouseInstance: if p.exists(self.path): shutil.rmtree(self.path) + def wait_for_path_exists(self, path, seconds): + while seconds > 0: + seconds -= 1 + if self.path_exists(path): + return + time.sleep(1) def get_backuped_s3_objects(self, disk, backup_name): - command = ['find', f'/var/lib/clickhouse/disks/{disk}/shadow/{backup_name}/store', '-type', 'f', + path = f'/var/lib/clickhouse/disks/{disk}/shadow/{backup_name}/store' + self.wait_for_path_exists(path, 10) + command = ['find', path, '-type', 'f', '-exec', 'grep', '-o', 'r[01]\\{64\\}-file-[[:lower:]]\\{32\\}', '{}', ';'] return self.exec_in_container(command).split('\n') diff --git a/tests/integration/test_dictionaries_update_and_reload/test.py b/tests/integration/test_dictionaries_update_and_reload/test.py index 8e375b7b327..9bee5db8ce1 100644 --- a/tests/integration/test_dictionaries_update_and_reload/test.py +++ b/tests/integration/test_dictionaries_update_and_reload/test.py @@ -203,7 +203,7 @@ def test_reload_after_fail_by_timer(started_cluster): instance.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/dictionaries/file.txt"), "/etc/clickhouse-server/dictionaries/no_file_2.txt") # Check that file appears in container and wait if needed. - while not instance.file_exists("/etc/clickhouse-server/dictionaries/no_file_2.txt"): + while not instance.path_exists("/etc/clickhouse-server/dictionaries/no_file_2.txt"): time.sleep(1) assert("9\t10\n" == instance.exec_in_container(["cat", "/etc/clickhouse-server/dictionaries/no_file_2.txt"])) instance.query("SYSTEM RELOAD DICTIONARY no_file_2") diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index 375a03e7f3a..996b15dece6 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -291,8 +291,6 @@ def test_s3_zero_copy_unfreeze(cluster): node1.query("ALTER TABLE unfreeze_test FREEZE WITH NAME 'backup1'") node2.query("ALTER TABLE unfreeze_test FREEZE WITH NAME 'backup2'") - time.sleep(1) - objects01 = node1.get_backuped_s3_objects("s31", "backup1") objects02 = node2.get_backuped_s3_objects("s31", "backup2") From ad6ba24efdee505613217fedb8dd1be09440b648 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 30 Nov 2021 10:49:57 +0800 Subject: [PATCH 0096/1260] fix ArrowColumnToCHColumn --- contrib/base64 | 2 +- contrib/libhdfs3 | 2 +- contrib/replxx | 2 +- contrib/sysroot | 2 +- src/IO/RemoteReadBufferCache.cpp | 2 +- src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 5 ++++- 6 files changed, 9 insertions(+), 6 deletions(-) diff --git a/contrib/base64 b/contrib/base64 index 9499e0c4945..af9b331f2b4 160000 --- a/contrib/base64 +++ b/contrib/base64 @@ -1 +1 @@ -Subproject commit 9499e0c4945589973b9ea1bc927377cfbc84aa46 +Subproject commit af9b331f2b4f30b41c70f3a571ff904a8251c1d3 diff --git a/contrib/libhdfs3 b/contrib/libhdfs3 index 9194af44588..a8c37ee001a 160000 --- a/contrib/libhdfs3 +++ b/contrib/libhdfs3 @@ -1 +1 @@ -Subproject commit 9194af44588633c1b2dae44bf945804401ff883e +Subproject commit a8c37ee001af1ae88e5dfa637ae5b31b087c96d3 diff --git a/contrib/replxx b/contrib/replxx index f019cba7ea1..68410ac01df 160000 --- a/contrib/replxx +++ b/contrib/replxx @@ -1 +1 @@ -Subproject commit f019cba7ea1bcd1b4feb7826f28ed57fb581b04c +Subproject commit 68410ac01dfb4f09ea76120ac5a2cecda3943aaf diff --git a/contrib/sysroot b/contrib/sysroot index 4ef348b7f30..1a64956aa7c 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit 4ef348b7f30f2ad5b02b266268b3c948e51ad457 +Subproject commit 1a64956aa7c280448be6526251bb2b8e6d380ab1 diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 904b2c0335c..d628cc7a460 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -473,7 +473,7 @@ RemoteReadBufferCache::createReader(const RemoteFileMetadata & remote_file_meta, // If something is wrong on startup, rollback to read from the original ReadBuffer if (!isInitialized()) { - LOG_ERROR(log, "RemoteReadBufferCache not initialized yet"); + LOG_ERROR(log, "RemoteReadBufferCache has not been initialized"); return {nullptr, RemoteReadBufferCacheError::NOT_INIT}; } diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 5d6f0ca0e96..5cedce4365d 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -530,7 +530,10 @@ void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptrsecond->length(); columns_list.reserve(header.rows()); From f27699ccda630f915f5ce238326ca24e73a8c5aa Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 30 Nov 2021 03:12:12 +0000 Subject: [PATCH 0097/1260] Enable comparison betwwen Decimal and Float add tests update test --- src/DataTypes/IDataType.h | 2 +- src/Functions/FunctionsComparison.h | 2 +- ..._comparison_betwwen_decimal_and_float.reference | 13 +++++++++++++ .../02124_comparison_betwwen_decimal_and_float.sql | 14 ++++++++++++++ 4 files changed, 29 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.reference create mode 100644 tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.sql diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index fc42d678d57..3b0f73bfdeb 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -487,7 +487,7 @@ inline bool isNotCreatable(const T & data_type) inline bool isNotDecimalButComparableToDecimal(const DataTypePtr & data_type) { WhichDataType which(data_type); - return which.isInt() || which.isUInt(); + return which.isInt() || which.isUInt() || which.isFloat(); } inline bool isCompilableType(const DataTypePtr & data_type) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 945090781dc..3f5146935cb 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -687,7 +687,7 @@ private: return (res = DecimalComparison::apply(col_left, col_right)) != nullptr; }; - if (!callOnBasicTypes(left_number, right_number, call)) + if (!callOnBasicTypes(left_number, right_number, call)) throw Exception("Wrong call for " + getName() + " with " + col_left.type->getName() + " and " + col_right.type->getName(), ErrorCodes::LOGICAL_ERROR); diff --git a/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.reference b/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.reference new file mode 100644 index 00000000000..eebbb0f7d4c --- /dev/null +++ b/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.reference @@ -0,0 +1,13 @@ +0 +1 +0 +1 +0 +0 +0 +1 +0 +1 +0 +0 +1 diff --git a/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.sql b/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.sql new file mode 100644 index 00000000000..a2ab6c120f0 --- /dev/null +++ b/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.sql @@ -0,0 +1,14 @@ +select CAST(1.0, 'Decimal(15,2)') > CAST(1, 'Float64'); +select CAST(1.0, 'Decimal(15,2)') = CAST(1, 'Float64'); +select CAST(1.0, 'Decimal(15,2)') < CAST(1, 'Float64'); +select CAST(1.0, 'Decimal(15,2)') > CAST(-1, 'Float64'); +select CAST(1.0, 'Decimal(15,2)') = CAST(-1, 'Float64'); +select CAST(1.0, 'Decimal(15,2)') < CAST(-1, 'Float64'); +select CAST(1.0, 'Decimal(15,2)') > CAST(1, 'Float32'); +select CAST(1.0, 'Decimal(15,2)') = CAST(1, 'Float32'); +select CAST(1.0, 'Decimal(15,2)') < CAST(1, 'Float32'); +select CAST(1.0, 'Decimal(15,2)') > CAST(-1, 'Float32'); +select CAST(1.0, 'Decimal(15,2)') = CAST(-1, 'Float32'); +select CAST(1.0, 'Decimal(15,2)') < CAST(-1, 'Float32'); + +SELECT toDecimal32('11.00', 2) > 1.; From b5714577216bd092a1dbae6578bc2de3f5d07ac6 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 30 Nov 2021 06:15:39 +0000 Subject: [PATCH 0098/1260] fix test --- .../00700_decimal_compare.reference | 6 ++++++ .../0_stateless/00700_decimal_compare.sql | 18 +++++++++--------- 2 files changed, 15 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/00700_decimal_compare.reference b/tests/queries/0_stateless/00700_decimal_compare.reference index 6b2787642b7..81cc789eaf1 100644 --- a/tests/queries/0_stateless/00700_decimal_compare.reference +++ b/tests/queries/0_stateless/00700_decimal_compare.reference @@ -1,3 +1,7 @@ +1 +0 +1 +0 0 1 -42 -42 1 0 0 0 1 1 @@ -17,6 +21,8 @@ 42 42 42 42 42 42 42 42 42 +42 42 42 +42 42 42 -42 -42.42 -42 -42 42 42 42 42.42 -42 -42 -42.42 diff --git a/tests/queries/0_stateless/00700_decimal_compare.sql b/tests/queries/0_stateless/00700_decimal_compare.sql index ae2f5790570..a557446a224 100644 --- a/tests/queries/0_stateless/00700_decimal_compare.sql +++ b/tests/queries/0_stateless/00700_decimal_compare.sql @@ -17,8 +17,8 @@ CREATE TABLE IF NOT EXISTS decimal INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (42, 42, 42, 0.42, 0.42, 0.42, 42.42, 42.42, 42.42, 42.42); INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-42, -42, -42, -0.42, -0.42, -0.42, -42.42, -42.42, -42.42, -42.42); -SELECT a > toFloat64(0) FROM decimal; -- { serverError 43 } -SELECT g > toFloat32(0) FROM decimal; -- { serverError 43 } +SELECT a > toFloat64(0) FROM decimal; +SELECT g > toFloat32(0) FROM decimal; SELECT a > '0.0' FROM decimal ORDER BY a; SELECT a, b, a = b, a < b, a > b, a != b, a <= b, a >= b FROM decimal ORDER BY a; @@ -36,8 +36,8 @@ SELECT a, b, c FROM decimal WHERE a = toInt8(42) AND b = toInt8(42) AND c = toIn SELECT a, b, c FROM decimal WHERE a = toInt16(42) AND b = toInt16(42) AND c = toInt16(42); SELECT a, b, c FROM decimal WHERE a = toInt32(42) AND b = toInt32(42) AND c = toInt32(42); SELECT a, b, c FROM decimal WHERE a = toInt64(42) AND b = toInt64(42) AND c = toInt64(42); -SELECT a, b, c FROM decimal WHERE a = toFloat32(42); -- { serverError 43 } -SELECT a, b, c FROM decimal WHERE a = toFloat64(42); -- { serverError 43 } +SELECT a, b, c FROM decimal WHERE a = toFloat32(42); +SELECT a, b, c FROM decimal WHERE a = toFloat64(42); SELECT least(a, b), least(a, g), greatest(a, b), greatest(a, g) FROM decimal ORDER BY a; SELECT least(a, 0), least(b, 0), least(g, 0) FROM decimal ORDER BY a; @@ -46,18 +46,18 @@ SELECT greatest(a, 0), greatest(b, 0), greatest(g, 0) FROM decimal ORDER BY a; SELECT (a, d, g) = (b, e, h), (a, d, g) != (b, e, h) FROM decimal ORDER BY a; SELECT (a, d, g) = (c, f, i), (a, d, g) != (c, f, i) FROM decimal ORDER BY a; -SELECT toUInt32(2147483648) AS x, a == x FROM decimal WHERE a = 42; -- { serverError 407 } +SELECT toUInt32(2147483648) AS x, a == x FROM decimal WHERE a = 42; -- { serverError 407 } SELECT toUInt64(2147483648) AS x, b == x, x == ((b - 42) + x) FROM decimal WHERE a = 42; -SELECT toUInt64(9223372036854775808) AS x, b == x FROM decimal WHERE a = 42; -- { serverError 407 } +SELECT toUInt64(9223372036854775808) AS x, b == x FROM decimal WHERE a = 42; -- { serverError 407 } SELECT toUInt64(9223372036854775808) AS x, c == x, x == ((c - 42) + x) FROM decimal WHERE a = 42; SELECT g = 10000, (g - g + 10000) == 10000 FROM decimal WHERE a = 42; SELECT 10000 = g, 10000 = (g - g + 10000) FROM decimal WHERE a = 42; -SELECT g = 30000 FROM decimal WHERE a = 42; -- { serverError 407 } -SELECT 30000 = g FROM decimal WHERE a = 42; -- { serverError 407 } +SELECT g = 30000 FROM decimal WHERE a = 42; -- { serverError 407 } +SELECT 30000 = g FROM decimal WHERE a = 42; -- { serverError 407 } SELECT h = 30000, (h - g + 30000) = 30000 FROM decimal WHERE a = 42; SELECT 30000 = h, 30000 = (h - g + 30000) FROM decimal WHERE a = 42; -SELECT h = 10000000000 FROM decimal WHERE a = 42; -- { serverError 407 } +SELECT h = 10000000000 FROM decimal WHERE a = 42; -- { serverError 407 } SELECT i = 10000000000, (i - g + 10000000000) = 10000000000 FROM decimal WHERE a = 42; SELECT 10000000000 = i, 10000000000 = (i - g + 10000000000) FROM decimal WHERE a = 42; From cacf516e3e560aedb34481e3cdfbb98fa0974130 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 30 Nov 2021 14:52:26 +0800 Subject: [PATCH 0099/1260] calculate column value by default expression & apply defaults_for_omitted_fields_ in ArrowColumnToCHColumn --- src/IO/RemoteReadBufferCache.cpp | 2 +- .../Formats/Impl/ArrowBlockInputFormat.cpp | 3 +- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 33 +++++++++++------ .../Formats/Impl/ArrowColumnToCHColumn.h | 5 ++- .../Formats/Impl/ORCBlockInputFormat.cpp | 3 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 3 +- src/Storages/Hive/HiveCommon.cpp | 2 +- src/Storages/Hive/HiveCommon.h | 2 +- src/Storages/Hive/StorageHive.cpp | 37 ++++++++++++++++--- 9 files changed, 66 insertions(+), 24 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index d628cc7a460..230cb96ba23 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -340,7 +340,7 @@ bool RemoteReadBuffer::nextImpl() else return false; } - else + else { // In the case we cannot use local cache, read from the original readbuffer directly if (!original_read_buffer) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index b1f9eaa59a1..6f90dc18f70 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -102,7 +102,8 @@ void ArrowBlockInputFormat::prepareReader() schema = file_reader->schema(); } - arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), "Arrow", format_settings.arrow.import_nested); + arrow_column_to_ch_column = std::make_unique( + getPort().getHeader(), "Arrow", format_settings.arrow.import_nested, format_settings.defaults_for_omitted_fields); if (stream) record_batch_total = -1; diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 5cedce4365d..94627e59bfd 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -63,6 +63,7 @@ namespace ErrorCodes extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; extern const int BAD_ARGUMENTS; extern const int UNKNOWN_EXCEPTION; + extern const int THERE_IS_NO_COLUMN; } @@ -504,14 +505,17 @@ static Block arrowSchemaToCHHeader(const arrow::Schema & schema, const std::stri } ArrowColumnToCHColumn::ArrowColumnToCHColumn( - const arrow::Schema & schema, const std::string & format_name_, bool import_nested_) - : header(arrowSchemaToCHHeader(schema, format_name_)), format_name(format_name_), import_nested(import_nested_) + const arrow::Schema & schema, const std::string & format_name_, bool import_nested_, bool defaults_for_omitted_fields_) + : header(arrowSchemaToCHHeader(schema, format_name_)) + , format_name(format_name_) + , import_nested(import_nested_) + , defaults_for_omitted_fields(defaults_for_omitted_fields_) { } ArrowColumnToCHColumn::ArrowColumnToCHColumn( - const Block & header_, const std::string & format_name_, bool import_nested_) - : header(header_), format_name(format_name_), import_nested(import_nested_) + const Block & header_, const std::string & format_name_, bool import_nested_, bool defaults_for_omitted_fields_) + : header(header_), format_name(format_name_), import_nested(import_nested_), defaults_for_omitted_fields(defaults_for_omitted_fields_) { } @@ -530,7 +534,7 @@ void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptrsecond->length(); @@ -562,12 +566,19 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & if (!read_from_nested) { - ColumnWithTypeAndName column; - column.name = header_column.name; - column.type = header_column.type; - column.column = header_column.column->cloneResized(num_rows); - columns_list.push_back(std::move(column.column)); - continue; + if (defaults_for_omitted_fields) + { + ColumnWithTypeAndName column; + column.name = header_column.name; + column.type = header_column.type; + column.column = header_column.column->cloneResized(num_rows); + columns_list.push_back(std::move(column.column)); + continue; + } + else + { + throw Exception{ErrorCodes::THERE_IS_NO_COLUMN, "Column '{}' is not presented in input data.", header_column.name}; + } } } diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 46976093f0b..0a8e22cd6da 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -21,11 +21,11 @@ class ArrowColumnToCHColumn public: using NameToColumnPtr = std::unordered_map>; - ArrowColumnToCHColumn(const Block & header_, const std::string & format_name_, bool import_nested_); + ArrowColumnToCHColumn(const Block & header_, const std::string & format_name_, bool import_nested_, bool defaults_for_omitted_fields_); /// Constructor that create header by arrow schema. It will be useful for inserting /// data from file without knowing table structure. - ArrowColumnToCHColumn(const arrow::Schema & schema, const std::string & format_name, bool import_nested_); + ArrowColumnToCHColumn(const arrow::Schema & schema, const std::string & format_name, bool import_nested_, bool defaults_for_omitted_fields_); void arrowTableToCHChunk(Chunk & res, std::shared_ptr & table); @@ -35,6 +35,7 @@ private: const Block header; const std::string format_name; bool import_nested; + bool defaults_for_omitted_fields; /// Map {column name : dictionary column}. /// To avoid converting dictionary from Arrow Dictionary diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 8768e2f5f14..7612a443bbd 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -114,7 +114,8 @@ void ORCBlockInputFormat::prepareReader() std::shared_ptr schema; THROW_ARROW_NOT_OK(file_reader->ReadSchema(&schema)); - arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), "ORC", format_settings.orc.import_nested); + arrow_column_to_ch_column = std::make_unique( + getPort().getHeader(), "ORC", format_settings.orc.import_nested, format_settings.defaults_for_omitted_fields); std::unordered_set nested_table_names; if (format_settings.orc.import_nested) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 901531d81cf..861818f9619 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -100,7 +100,8 @@ void ParquetBlockInputFormat::prepareReader() std::shared_ptr schema; THROW_ARROW_NOT_OK(file_reader->GetSchema(&schema)); - arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), "Parquet", format_settings.parquet.import_nested); + arrow_column_to_ch_column = std::make_unique( + getPort().getHeader(), "Parquet", format_settings.parquet.import_nested, format_settings.defaults_for_omitted_fields); std::unordered_set nested_table_names; if (format_settings.parquet.import_nested) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 31cee25272c..dbb6a3d2824 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -82,7 +82,7 @@ std::shared_ptr HiveMetastoreClient::get return result; } -void HiveMetastoreClient::clearTableMeta(const std::string & db_name, const std::string & table_name) +void HiveMetastoreClient::clearTableMetadata(const std::string & db_name, const std::string & table_name) { std::lock_guard lock{mutex}; std::string cache_key = db_name + "." + table_name; diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index 45458a05666..7a143e545f8 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -80,7 +80,7 @@ public: } std::shared_ptr getTableMetadata(const std::string & db_name, const std::string & table_name); - void clearTableMeta(const std::string & db_name, const std::string & table_name); + void clearTableMetadata(const std::string & db_name, const std::string & table_name); void setClient(std::shared_ptr client_); inline bool isExpired() const { return expired; } inline void setExpired() { expired = true; } diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index e7907e91f9f..e8fe532b492 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -90,6 +91,16 @@ public: return header; } + static ColumnsDescription getColumnsDescription(Block header, const SourcesInfoPtr & source_info) + { + ColumnsDescription columns_description{header.getNamesAndTypesList()}; + if (source_info->need_path_column) + columns_description.add({"_path", std::make_shared()}); + if (source_info->need_file_column) + columns_description.add({"_file", std::make_shared()}); + return columns_description; + } + HiveSource( SourcesInfoPtr source_info_, String hdfs_namenode_url_, @@ -108,19 +119,22 @@ public: , max_block_size(max_block_size_) , sample_block(std::move(sample_block_)) , to_read_block(sample_block) + , columns_description(getColumnsDescription(sample_block_, source_info_)) , text_input_field_names(text_input_field_names_) , format_settings(getFormatSettings(getContext())) { + /// Initialize to_read_block, which is used to read data from HDFS. to_read_block = sample_block; for (const auto & name_type : source_info->partition_name_types) { to_read_block.erase(name_type.name); } - /// initialize format settings of CSV + /// Initialize format settings format_settings.csv.delimiter = '\x01'; format_settings.csv.input_field_names = text_input_field_names; format_settings.csv.read_bool_as_uint8 = true; + format_settings.defaults_for_omitted_fields = true; } String getName() const override { return "Hive"; } @@ -150,7 +164,7 @@ public: { if (e.code() == ErrorCodes::CANNOT_OPEN_FILE) { - source_info->hive_metastore_client->clearTableMeta(source_info->database, source_info->table_name); + source_info->hive_metastore_client->clearTableMetadata(source_info->database, source_info->table_name); throw; } } @@ -171,8 +185,18 @@ public: auto input_format = FormatFactory::instance().getInputFormat( format, *read_buf, to_read_block, getContext(), max_block_size, format_settings); - pipeline = QueryPipeline(std::move(input_format)); - reader = std::make_unique(pipeline); + + QueryPipelineBuilder builder; + builder.init(Pipe(input_format)); + if (columns_description.hasDefaults()) + { + builder.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, columns_description, *input_format, getContext()); + }); + } + pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); + reader = std::make_unique(*pipeline); } Block res; @@ -180,6 +204,8 @@ public: { Columns columns = res.getColumns(); UInt64 num_rows = res.rows(); + + /// Enrich with partition columns. auto types = source_info->partition_name_types.getTypes(); for (size_t i = 0; i < types.size(); ++i) { @@ -213,7 +239,7 @@ public: private: std::unique_ptr read_buf; - QueryPipeline pipeline; + std::unique_ptr pipeline; std::unique_ptr reader; SourcesInfoPtr source_info; String hdfs_namenode_url; @@ -222,6 +248,7 @@ private: UInt64 max_block_size; Block sample_block; Block to_read_block; + ColumnsDescription columns_description; const Names & text_input_field_names; FormatSettings format_settings; From 440fa9b69c5ba1b19328230c864074c74605a2bc Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 30 Nov 2021 15:44:59 +0800 Subject: [PATCH 0100/1260] implement getMissingValues for ORC/Parquet/Arrow --- .../Formats/Impl/ArrowBlockInputFormat.cpp | 12 +++++++++++- src/Processors/Formats/Impl/ArrowBlockInputFormat.h | 4 ++++ .../Formats/Impl/ArrowColumnToCHColumn.cpp | 13 +++++++++---- src/Processors/Formats/Impl/ArrowColumnToCHColumn.h | 5 +++-- src/Processors/Formats/Impl/ORCBlockInputFormat.cpp | 13 ++++++++++++- src/Processors/Formats/Impl/ORCBlockInputFormat.h | 4 ++++ .../Formats/Impl/ParquetBlockInputFormat.cpp | 12 +++++++++++- .../Formats/Impl/ParquetBlockInputFormat.h | 3 +++ src/Storages/Hive/StorageHive.cpp | 2 +- 9 files changed, 58 insertions(+), 10 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index 6f90dc18f70..b6a43591398 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -30,6 +30,7 @@ ArrowBlockInputFormat::ArrowBlockInputFormat(ReadBuffer & in_, const Block & hea Chunk ArrowBlockInputFormat::generate() { Chunk res; + block_missing_values.clear(); arrow::Result> batch_result; if (stream) @@ -63,7 +64,10 @@ Chunk ArrowBlockInputFormat::generate() ++record_batch_current; - arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result); + auto missing_column_indexes = arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result); + for (size_t row_idx = 0; row_idx < res.getNumRows(); ++row_idx) + for (const auto & column_idx : missing_column_indexes) + block_missing_values.setBit(column_idx, row_idx); return res; } @@ -77,6 +81,12 @@ void ArrowBlockInputFormat::resetParser() else file_reader.reset(); record_batch_current = 0; + block_missing_values.clear(); +} + +const BlockMissingValues & ArrowBlockInputFormat::getMissingValues() const +{ + return block_missing_values; } void ArrowBlockInputFormat::prepareReader() diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index 44e18e3f852..c0e78ff7e42 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -24,6 +24,8 @@ public: String getName() const override { return "ArrowBlockInputFormat"; } + const BlockMissingValues & getMissingValues() const override; + private: Chunk generate() override; @@ -39,6 +41,8 @@ private: int record_batch_total = 0; int record_batch_current = 0; + BlockMissingValues block_missing_values; + const FormatSettings format_settings; void prepareReader(); diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 94627e59bfd..dc25ff19193 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -519,7 +519,7 @@ ArrowColumnToCHColumn::ArrowColumnToCHColumn( { } -void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table) +std::vector ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table) { NameToColumnPtr name_to_column_ptr; for (const auto& column_name : table->ColumnNames()) @@ -528,20 +528,23 @@ void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & name_to_column_ptr) { Columns columns_list; if (name_to_column_ptr.empty()) - return; + return {}; + UInt64 num_rows = name_to_column_ptr.begin()->second->length(); columns_list.reserve(header.rows()); std::unordered_map nested_tables; + std::vector missing_column_indexes; + missing_column_indexes.reserve(header.columns()); for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i) { const ColumnWithTypeAndName & header_column = header.getByPosition(column_i); @@ -566,6 +569,7 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & if (!read_from_nested) { + missing_column_indexes.push_back(column_i); if (defaults_for_omitted_fields) { ColumnWithTypeAndName column; @@ -606,6 +610,7 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & } res.setColumns(columns_list, num_rows); + return missing_column_indexes; } } #endif diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 0a8e22cd6da..761b389ee03 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -27,9 +27,10 @@ public: /// data from file without knowing table structure. ArrowColumnToCHColumn(const arrow::Schema & schema, const std::string & format_name, bool import_nested_, bool defaults_for_omitted_fields_); - void arrowTableToCHChunk(Chunk & res, std::shared_ptr & table); + /// Convert arrow::Table to chunk. Returns missing header columns not exists in arrow::Table. + std::vector arrowTableToCHChunk(Chunk & res, std::shared_ptr & table); - void arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & name_to_column_ptr); + std::vector arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & name_to_column_ptr); private: const Block header; diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 7612a443bbd..55fddfd7693 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -34,6 +34,7 @@ ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_, const Chunk ORCBlockInputFormat::generate() { Chunk res; + block_missing_values.clear(); if (!file_reader) prepareReader(); @@ -67,7 +68,11 @@ Chunk ORCBlockInputFormat::generate() std::shared_ptr arrow_column = std::make_shared(vec); name_to_column_ptr[column_name] = arrow_column; } - arrow_column_to_ch_column->arrowColumnsToCHChunk(res, name_to_column_ptr); + + auto missing_column_indexes = arrow_column_to_ch_column->arrowColumnsToCHChunk(res, name_to_column_ptr); + for (size_t row_idx = 0; row_idx < res.getNumRows(); ++row_idx) + for (const auto & column_idx : missing_column_indexes) + block_missing_values.setBit(column_idx, row_idx); batch_reader.reset(); return res; @@ -80,6 +85,12 @@ void ORCBlockInputFormat::resetParser() file_reader.reset(); include_indices.clear(); stripe_current = 0; + block_missing_values.clear(); +} + +const BlockMissingValues & ORCBlockInputFormat::getMissingValues() const +{ + return block_missing_values; } static size_t countIndicesForType(std::shared_ptr type) diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h index 857ec7937b7..c8747fb8d36 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -26,6 +26,8 @@ public: void resetParser() override; + const BlockMissingValues & getMissingValues() const override; + protected: Chunk generate() override; @@ -48,6 +50,8 @@ private: // indices of columns to read from ORC file std::vector include_indices; + BlockMissingValues block_missing_values; + const FormatSettings format_settings; void prepareReader(); diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 861818f9619..2c5799f9d7a 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -40,6 +40,7 @@ ParquetBlockInputFormat::ParquetBlockInputFormat(ReadBuffer & in_, Block header_ Chunk ParquetBlockInputFormat::generate() { Chunk res; + block_missing_values.clear(); if (!file_reader) prepareReader(); @@ -55,7 +56,10 @@ Chunk ParquetBlockInputFormat::generate() ++row_group_current; - arrow_column_to_ch_column->arrowTableToCHChunk(res, table); + auto missing_column_indexes = arrow_column_to_ch_column->arrowTableToCHChunk(res, table); + for (size_t row_idx = 0; row_idx < res.getNumRows(); ++row_idx) + for (const auto & column_idx : missing_column_indexes) + block_missing_values.setBit(column_idx, row_idx); return res; } @@ -66,6 +70,12 @@ void ParquetBlockInputFormat::resetParser() file_reader.reset(); column_indices.clear(); row_group_current = 0; + block_missing_values.clear(); +} + +const BlockMissingValues & ParquetBlockInputFormat::getMissingValues() const +{ + return block_missing_values; } static size_t countIndicesForType(std::shared_ptr type) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index 472aec66da3..4597a52d8ee 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -23,6 +23,8 @@ public: String getName() const override { return "ParquetBlockInputFormat"; } + const BlockMissingValues & getMissingValues() const override; + private: Chunk generate() override; @@ -34,6 +36,7 @@ private: std::vector column_indices; std::unique_ptr arrow_column_to_ch_column; int row_group_current = 0; + BlockMissingValues block_missing_values; const FormatSettings format_settings; }; diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index e8fe532b492..53f827972e3 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -119,7 +119,7 @@ public: , max_block_size(max_block_size_) , sample_block(std::move(sample_block_)) , to_read_block(sample_block) - , columns_description(getColumnsDescription(sample_block_, source_info_)) + , columns_description(getColumnsDescription(sample_block, source_info)) , text_input_field_names(text_input_field_names_) , format_settings(getFormatSettings(getContext())) { From b023dd34c0ada42085589cb45ebbcf987d954aae Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 30 Nov 2021 12:05:18 +0800 Subject: [PATCH 0101/1260] fixed code style --- src/IO/RemoteReadBufferCache.cpp | 203 +++++++++++++++--------------- src/IO/RemoteReadBufferCache.h | 57 +++++---- src/Storages/Hive/HiveCommon.cpp | 14 ++- src/Storages/Hive/HiveCommon.h | 2 +- src/Storages/Hive/StorageHive.cpp | 6 +- 5 files changed, 150 insertions(+), 132 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 230cb96ba23..7dcc3f40bcc 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -27,57 +27,79 @@ namespace ErrorCodes extern const int CANNOT_CREATE_DIRECTORY; } -std::shared_ptr RemoteCacheController::recover( - const String & local_path_, std::function const & finish_callback) +bool RemoteFileMetadata::load(const std::filesystem::path & local_path) { - fs::path data_file = fs::path(local_path_) / "data.bin"; - fs::path meta_file = fs::path(local_path_) / "meta.txt"; - auto * log = &Poco::Logger::get("RemoteCacheController"); - if (!fs::exists(data_file) || !fs::exists(meta_file)) + auto log = &Poco::Logger::get("RemoteFileMetadata"); + if (!std::filesystem::exists(local_path)) { - LOG_ERROR(log, "Directory {} or file {}, {} does not exist", local_path_, data_file.string(), meta_file.string()); - return nullptr; + LOG_ERROR(log, "file path not exists:{}", local_path.string()); + return false; } - - std::ifstream meta_fs(meta_file); - Poco::JSON::Parser meta_parser; - auto meta_jobj = meta_parser.parse(meta_fs).extract(); - auto remote_path = meta_jobj->get("remote_path").convert(); - auto schema = meta_jobj->get("schema").convert(); - auto cluster = meta_jobj->get("cluster").convert(); - auto downloaded = meta_jobj->get("downloaded").convert(); - auto modification_ts = meta_jobj->get("last_modification_timestamp").convert(); - if (downloaded == "false") - { - LOG_ERROR(log, "Local metadata for local path {} exists, but the data was not downloaded", local_path_); - return nullptr; - } - auto file_size = fs::file_size(data_file); - - RemoteFileMetadata remote_file_meta(schema, cluster, remote_path, modification_ts, file_size); - auto cache_controller = std::make_shared(remote_file_meta, local_path_, 0, nullptr, finish_callback); - cache_controller->download_finished = true; - cache_controller->current_offset = file_size; + std::ifstream meta_fs(local_path.string()); + Poco::JSON::Parser meta_data_parser; + auto meta_data_jobj = meta_data_parser.parse(meta_fs).extract(); + remote_path = meta_data_jobj->get("remote_path").convert(); + schema = meta_data_jobj->get("schema").convert(); + cluster = meta_data_jobj->get("cluster").convert(); + status = static_cast(meta_data_jobj->get("status").convert()); + last_modification_timestamp = meta_data_jobj->get("last_modification_timestamp").convert(); + file_size = meta_data_jobj->get("file_size").convert(); meta_fs.close(); - finish_callback(cache_controller.get()); + return true; +} + +void RemoteFileMetadata::save(const std::filesystem::path & local_path) +{ + std::ofstream meta_file(local_path.string(), std::ios::out); + meta_file << toString(); + meta_file.close(); +} +String RemoteFileMetadata::toString(){ + Poco::JSON::Object jobj; + jobj.set("schema", schema); + jobj.set("cluster", cluster); + jobj.set("remote_path", remote_path); + jobj.set("status", static_cast(status)); + jobj.set("last_modification_timestamp", last_modification_timestamp); + jobj.set("file_size", file_size); + std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + jobj.stringify(buf); + return buf.str(); +} + +std::shared_ptr RemoteCacheController::recover(const std::filesystem::path & local_path_) +{ + + if (!std::filesystem::exists(local_path_) || !std::filesystem::exists(local_path_ / "data.bin")) + { + LOG_TRACE(&Poco::Logger::get("RemoteCacheController"), "Invalid cached directory:{}", local_path_.string()); + return nullptr; + } + + RemoteFileMetadata remote_file_meta_data; + if (!remote_file_meta_data.load(local_path_ / "meta.txt") || remote_file_meta_data.status != RemoteFileMetadata::DOWNLOADED) + { + LOG_INFO(&Poco::Logger::get("RemoteCacheController"), "recover cached file failed. local path:{}, file meta data:", local_path_.string(), remote_file_meta_data.toString()); + return nullptr; + } + + auto cache_controller = std::make_shared(remote_file_meta_data, local_path_, 0, nullptr); + cache_controller->current_offset = remote_file_meta_data.file_size; + + RemoteReadBufferCache::instance().updateTotalSize(cache_controller->file_meta_data.file_size); return cache_controller; } RemoteCacheController::RemoteCacheController( - const RemoteFileMetadata & remote_file_meta, - const String & local_path_, + const RemoteFileMetadata & file_meta_data_, + const std::filesystem::path & local_path_, size_t cache_bytes_before_flush_, - std::shared_ptr read_buffer_, - std::function const & finish_callback) - : schema(remote_file_meta.schema) - , cluster(remote_file_meta.cluster) - , remote_path(remote_file_meta.path) + std::shared_ptr read_buffer_) + : file_meta_data(file_meta_data_) , local_path(local_path_) - , last_modify_time(remote_file_meta.last_modify_time) , valid(true) , local_cache_bytes_read_before_flush(cache_bytes_before_flush_) - , download_finished(false) , current_offset(0) , remote_read_buffer(read_buffer_) { @@ -88,26 +110,16 @@ RemoteCacheController::RemoteCacheController( out_file = std::make_unique(fs::path(local_path_) / "data.bin", std::ios::out | std::ios::binary); out_file->flush(); - Poco::JSON::Object jobj; - jobj.set("schema", schema); - jobj.set("cluster", cluster); - jobj.set("remote_path", remote_path); - jobj.set("downloaded", "false"); - jobj.set("last_modification_timestamp", last_modify_time); - std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - jobj.stringify(buf); - std::ofstream meta_file(fs::path(local_path_) / "meta.txt", std::ios::out); - meta_file.write(buf.str().c_str(), buf.str().size()); - meta_file.close(); + file_meta_data.save(local_path_ / "meta.txt"); - backgroundDownload(finish_callback); + backgroundDownload(); } } RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offset_, size_t end_offset_) { std::unique_lock lock{mutex}; - if (download_finished) + if (file_meta_data.status == RemoteFileMetadata::DOWNLOADED) { // finish reading if (start_offset_ >= current_offset) @@ -124,16 +136,17 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs return RemoteReadBufferCacheError::OK; } else - more_data_signal.wait(lock, [this, end_offset_] { return download_finished || current_offset >= end_offset_; }); + more_data_signal.wait(lock, [this, end_offset_] { return this->file_meta_data.status == RemoteFileMetadata::DOWNLOADED || current_offset >= end_offset_; }); } lock.unlock(); return RemoteReadBufferCacheError::OK; } -void RemoteCacheController::backgroundDownload(std::function const & finish_callback) +void RemoteCacheController::backgroundDownload() { - auto task = [this, finish_callback]() + auto task = [this]() { + file_meta_data.status = RemoteFileMetadata::DOWNLOADING; size_t before_unflush_bytes = 0; size_t total_bytes = 0; while (!remote_read_buffer->eof()) @@ -157,59 +170,50 @@ void RemoteCacheController::backgroundDownload(std::functionclose(); out_file.reset(); remote_read_buffer.reset(); lock.unlock(); more_data_signal.notify_all(); - finish_callback(this); - LOG_TRACE(log, "Finish download from remote path: {} to local path: {}, file size:{} ", remote_path, local_path, current_offset); + RemoteReadBufferCache::instance().updateTotalSize(file_meta_data.file_size); + LOG_TRACE(log, "Finish download into local path: {}, file meta data:{} ", local_path.string(), file_meta_data.toString()); }; RemoteReadBufferCache::instance().getThreadPool()->scheduleOrThrow(task); } -void RemoteCacheController::flush(bool need_flush_meta_) +void RemoteCacheController::flush(bool need_flush_meta_data_) { if (out_file) { out_file->flush(); } - if (!need_flush_meta_) + if (!need_flush_meta_data_) return; - Poco::JSON::Object jobj; - jobj.set("schema", schema); - jobj.set("cluster", cluster); - jobj.set("remote_path", remote_path); - jobj.set("downloaded", download_finished ? "true" : "false"); - jobj.set("last_modification_timestamp", last_modify_time); - std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - jobj.stringify(buf); - std::ofstream meta_file(fs::path(local_path) / "meta.txt", std::ios::out); - meta_file << buf.str(); - meta_file.close(); + file_meta_data.save(local_path / "meta.txt"); } RemoteCacheController::~RemoteCacheController() = default; void RemoteCacheController::close() { - LOG_TRACE(log, "Removing all local cache for remote path: {}, local path: {}", remote_path, local_path); - fs::remove_all(local_path); + // delete directory + LOG_TRACE(log, "Removing all local cache. local path: {}, file meta data:{}", local_path.string(), file_meta_data.toString()); + std::filesystem::remove_all(local_path); } std::pair RemoteCacheController::allocFile() { - fs::path result_local_path; - if (download_finished) - result_local_path = fs::path(local_path) / "data.bin"; + std::filesystem::path result_local_path; + if (file_meta_data.status == RemoteFileMetadata::DOWNLOADED) + result_local_path = local_path / "data.bin"; FILE * fs = fopen((fs::path(local_path) / "data.bin").string().c_str(), "r"); if (!fs) - throw Exception(ErrorCodes::BAD_GET, "Alloc file failed, error code: {} local path: {}", errno, local_path); + throw Exception(ErrorCodes::BAD_GET, "Alloc file failed, error code: {} local path: {}", errno, local_path.string()); std::lock_guard lock{mutex}; opened_file_streams.insert(fs); @@ -225,8 +229,8 @@ void RemoteCacheController::deallocFile(FILE * file_stream) throw Exception( ErrorCodes::BAD_ARGUMENTS, "Try to deallocate file with invalid handler remote path: {}, local path: {}", - remote_path, - local_path); + file_meta_data.remote_path, + local_path.string()); opened_file_streams.erase(it); } fclose(file_stream); @@ -308,7 +312,7 @@ std::unique_ptr RemoteReadBuffer::create(const RemoteFileMetad if (buff_size == 0) buff_size = DBMS_DEFAULT_BUFFER_SIZE; - const auto & remote_path = remote_file_meta.path; + const auto & remote_path = remote_file_meta.remote_path; auto remote_read_buffer = std::make_unique(buff_size); auto * raw_rbp = read_buffer.release(); std::shared_ptr srb(raw_rbp); @@ -401,18 +405,17 @@ RemoteReadBufferCache & RemoteReadBufferCache::instance() return instance; } -void RemoteReadBufferCache::recoverCachedFilesMeta( +void RemoteReadBufferCache::recoverCachedFilesMetaData( const fs::path & current_path, size_t current_depth, - size_t max_depth, - std::function const & finish_callback) + size_t max_depth) { if (current_depth >= max_depth) { for (auto const & dir : fs::directory_iterator{current_path}) { - std::string path = dir.path(); - auto cache_controller = RemoteCacheController::recover(path, finish_callback); + String path = dir.path(); + auto cache_controller = RemoteCacheController::recover(path); if (!cache_controller) continue; auto & cell = caches[path]; @@ -424,10 +427,18 @@ void RemoteReadBufferCache::recoverCachedFilesMeta( for (auto const & dir : fs::directory_iterator{current_path}) { - recoverCachedFilesMeta(dir.path(), current_depth + 1, max_depth, finish_callback); + recoverCachedFilesMetaData(dir.path(), current_depth + 1, max_depth); } } +void RemoteReadBufferCache::recoverTask(){ + std::lock_guard lock(mutex); + recoverCachedFilesMetaData(root_dir, 1, 2); + initialized = true; + LOG_TRACE(log, "Recovered from directory:{}", root_dir); + +} + void RemoteReadBufferCache::initOnce( const String & root_dir_, size_t limit_size_, size_t bytes_read_before_flush_, size_t max_threads_) { @@ -448,20 +459,12 @@ void RemoteReadBufferCache::initOnce( ErrorCodes::CANNOT_CREATE_DIRECTORY, "Failed to create directories, error code:{} reason:{}", ec.value(), ec.message()); } - auto recover_task = [this]() - { - auto callback = [this](RemoteCacheController * cache_controller) { total_size += cache_controller->size(); }; - std::lock_guard lock(mutex); - recoverCachedFilesMeta(root_dir, 1, 2, callback); - initialized = true; - LOG_TRACE(log, "Recovered from directory:{}", root_dir); - }; - getThreadPool()->scheduleOrThrow(recover_task); + getThreadPool()->scheduleOrThrow([this]{recoverTask();}); } String RemoteReadBufferCache::calculateLocalPath(const RemoteFileMetadata & meta) const { - String full_path = meta.schema + ":" + meta.cluster + ":" + meta.path; + String full_path = meta.schema + ":" + meta.cluster + ":" + meta.remote_path; UInt128 hashcode = sipHash128(full_path.c_str(), full_path.size()); String hashcode_str = getHexUIntLowercase(hashcode); return fs::path(root_dir) / hashcode_str.substr(0, 3) / hashcode_str; @@ -470,6 +473,7 @@ String RemoteReadBufferCache::calculateLocalPath(const RemoteFileMetadata & meta std::pair, RemoteReadBufferCacheError> RemoteReadBufferCache::createReader(const RemoteFileMetadata & remote_file_meta, std::shared_ptr & read_buffer) { + LOG_TRACE(log, "createReader. {} {} {}", remote_file_meta.remote_path, remote_file_meta.last_modification_timestamp, remote_file_meta.file_size); // If something is wrong on startup, rollback to read from the original ReadBuffer if (!isInitialized()) { @@ -477,9 +481,9 @@ RemoteReadBufferCache::createReader(const RemoteFileMetadata & remote_file_meta, return {nullptr, RemoteReadBufferCacheError::NOT_INIT}; } - auto remote_path = remote_file_meta.path; + auto remote_path = remote_file_meta.remote_path; const auto & file_size = remote_file_meta.file_size; - const auto & last_modification_timestamp = remote_file_meta.last_modify_time; + const auto & last_modification_timestamp = remote_file_meta.last_modification_timestamp; auto local_path = calculateLocalPath(remote_file_meta); std::lock_guard lock(mutex); auto cache_iter = caches.find(local_path); @@ -534,9 +538,8 @@ RemoteReadBufferCache::createReader(const RemoteFileMetadata & remote_file_meta, fs::create_directories(local_path); - auto callback = [this](RemoteCacheController * cntrl) { total_size += cntrl->size(); }; auto cache_controller - = std::make_shared(remote_file_meta, local_path, local_cache_bytes_read_before_flush, read_buffer, callback); + = std::make_shared(remote_file_meta, local_path, local_cache_bytes_read_before_flush, read_buffer); CacheCell cache_cell; cache_cell.cache_controller = cache_controller; cache_cell.key_iterator = keys.insert(keys.end(), local_path); @@ -583,7 +586,7 @@ bool RemoteReadBufferCache::clearLocalCache() LOG_TRACE( log, "clear local file {} for {}. key size:{}. next{}", - cache_it->second.cache_controller->getLocalPath(), + cache_it->second.cache_controller->getLocalPath().string(), cache_it->second.cache_controller->getRemotePath(), keys.size(), *it); diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 350e41beb6c..0abf9ff9e46 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -26,41 +26,52 @@ enum class RemoteReadBufferCacheError : int8_t struct RemoteFileMetadata { + enum LocalStatus{ + TO_DOWNLOAD = 0, + DOWNLOADING = 1, + DOWNLOADED = 2, + }; + RemoteFileMetadata(): last_modification_timestamp(0l), file_size(0), status(TO_DOWNLOAD){} RemoteFileMetadata( const String & schema_, const String & cluster_, const String & path_, - UInt64 last_modify_time_, + UInt64 last_modification_timestamp_, size_t file_size_) : schema(schema_) , cluster(cluster_) - , path(path_) - , last_modify_time(last_modify_time_) + , remote_path(path_) + , last_modification_timestamp(last_modification_timestamp_) , file_size(file_size_) + , status(TO_DOWNLOAD) { } + bool load(const std::filesystem::path & local_path); + void save(const std::filesystem::path & local_path); + String toString(); + String schema; // Hive, S2 etc. String cluster; - String path; - UInt64 last_modify_time; + String remote_path; + UInt64 last_modification_timestamp; size_t file_size; + LocalStatus status; }; class RemoteCacheController { public: RemoteCacheController( - const RemoteFileMetadata & meta, - const String & local_path_, + const RemoteFileMetadata & file_meta_data_, + const std::filesystem::path & local_path_, size_t cache_bytes_before_flush_, - std::shared_ptr read_buffer_, - std::function const & finish_callback); + std::shared_ptr read_buffer_); ~RemoteCacheController(); // recover from local disk static std::shared_ptr - recover(const String & local_path, std::function const & finish_callback); + recover(const std::filesystem::path & local_path); /** * Called by LocalCachedFileReader, must be used in pair @@ -90,10 +101,10 @@ public: inline size_t size() const { return current_offset; } - inline String getLocalPath() const { return local_path; } - inline String getRemotePath() const { return remote_path; } + inline const std::filesystem::path & getLocalPath() { return local_path; } + inline const String & getRemotePath() { return file_meta_data.remote_path; } - inline UInt64 getLastModificationTimestamp() const { return last_modify_time; } + inline UInt64 getLastModificationTimestamp() const { return file_meta_data.last_modification_timestamp; } inline void markInvalid() { std::lock_guard lock(mutex); @@ -107,9 +118,9 @@ public: private: // flush file and meta info into disk - void flush(bool need_flush_meta_ = false); + void flush(bool need_flush_meta_data_ = false); - void backgroundDownload(std::function const & finish_callback); + void backgroundDownload(); std::mutex mutex; std::condition_variable more_data_signal; @@ -117,15 +128,11 @@ private: std::set opened_file_streams; // meta info - String schema; - String cluster; - String remote_path; - String local_path; - UInt64 last_modify_time; - + RemoteFileMetadata file_meta_data; + std::filesystem::path local_path; + bool valid; size_t local_cache_bytes_read_before_flush; - bool download_finished; size_t current_offset; std::shared_ptr remote_read_buffer; @@ -230,11 +237,11 @@ private: String calculateLocalPath(const RemoteFileMetadata & meta) const; - void recoverCachedFilesMeta( + void recoverTask(); + void recoverCachedFilesMetaData( const std::filesystem::path & current_path, size_t current_depth, - size_t max_depth, - std::function const & finish_callback); + size_t max_depth); bool clearLocalCache(); }; diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index dbb6a3d2824..95a8ad8dd13 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -39,7 +39,7 @@ std::shared_ptr HiveMetastoreClient::get std::shared_ptr result = table_meta_cache.get(cache_key); bool update_cache = false; std::map old_partition_infos; - std::map partition_infos; + std::map new_partition_infos; if (result) { old_partition_infos = result->getPartitionInfos(); @@ -53,7 +53,7 @@ std::shared_ptr HiveMetastoreClient::get for (const auto & partition : partitions) { - auto & partition_info = partition_infos[partition.sd.location]; + auto & partition_info = new_partition_infos[partition.sd.location]; partition_info.partition = partition; // query files under the partition by hdfs api is costly, we reuse the files in case the partition has no change @@ -76,7 +76,7 @@ std::shared_ptr HiveMetastoreClient::get if (update_cache) { LOG_INFO(log, "reload hive partition meta info:" + db_name + ":" + table_name); - result = std::make_shared(db_name, table_name, table, std::move(partition_infos), getContext()); + result = std::make_shared(db_name, table_name, table, std::move(new_partition_infos), getContext()); table_meta_cache.set(cache_key, result); } return result; @@ -159,13 +159,17 @@ std::vector HiveMetastoreClient::HiveTableMetadat std::vector HiveMetastoreClient::HiveTableMetadata::getLocationFiles(const HDFSFSPtr & fs, const std::string & location) { + LOG_TRACE(&Poco::Logger::get("HiveMetastoreClient"), "ls {}", location); std::map::const_iterator it; - if (!empty_partition_keys) + bool x = false; + if (!empty_partition_keys && x) { std::lock_guard lock{mutex}; it = partition_infos.find(location); if (it == partition_infos.end()) throw Exception("invalid location " + location, ErrorCodes::BAD_ARGUMENTS); + + LOG_TRACE(&Poco::Logger::get("HiveMetastoreClient"), "empty_partition_keys {} {}", location, it->second.files.size()); return it->second.files; } @@ -173,9 +177,11 @@ std::vector HiveMetastoreClient::HiveTableMetadat HDFSFileInfo ls; ls.file_info = hdfsListDirectory(fs.get(), location_uri.getPath().c_str(), &ls.length); std::vector result; + LOG_TRACE(&Poco::Logger::get("HiveMetastoreClient"), "ls result. {} {}", ls.length, location); for (int i = 0; i < ls.length; ++i) { auto & file_info = ls.file_info[i]; + LOG_TRACE(&Poco::Logger::get("HiveMetastoreClient"), "get file:{} {} {}", file_info.mName, file_info.mKind, file_info.mSize); if (file_info.mKind != 'D' && file_info.mSize > 0) result.emplace_back(String(file_info.mName), file_info.mLastMod, file_info.mSize); } diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index 7a143e545f8..22749faf75d 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -24,7 +24,7 @@ public: size_t size; FileInfo() = default; - FileInfo(const FileInfo &) = default; + //FileInfo(const FileInfo & b) : path(b.path), last_modify_time(b.last_modify_time), size(b.size){} FileInfo(const std::string & path_, UInt64 last_modify_time_, size_t size_) : path(path_), last_modify_time(last_modify_time_), size(size_) {} }; diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 53f827972e3..5663c0e86d3 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -431,6 +431,7 @@ Pipe StorageHive::read( auto append_hive_files = [&](const HiveMetastoreClient::FileInfo & hfile, const FieldVector & fields) { + LOG_TRACE(log, "append hive file:{}", hfile.path); String filename = getBaseName(hfile.path); // Skip temporary files starts with '.' @@ -507,7 +508,7 @@ Pipe StorageHive::read( } if (has_default_partition) { - LOG_DEBUG(log, "skip partition:__HIVE_DEFAULT_PARTITION__"); + //LOG_DEBUG(log, "skip partition:__HIVE_DEFAULT_PARTITION__"); return; } @@ -545,10 +546,11 @@ Pipe StorageHive::read( const KeyCondition partition_key_condition(query_info, getContext(), partition_names, partition_minmax_idx_expr); if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true) { - LOG_DEBUG(log, "skip partition:{}", boost::algorithm::join(p.values, "|")); + //LOG_DEBUG(log, "skip partition:{}", boost::algorithm::join(p.values, "|")); return; } + LOG_TRACE(log, "list location:{}", p.sd.location); auto paths = list_paths(p.sd.location); for (const auto & path : paths) { From 1014ca8c586ace04fed888e0c4ee9370d1c276fd Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 30 Nov 2021 17:57:58 +0800 Subject: [PATCH 0102/1260] make two loops in clearing cached files into one --- contrib/base64 | 2 +- contrib/libhdfs3 | 2 +- contrib/replxx | 2 +- contrib/sysroot | 2 +- src/IO/RemoteReadBufferCache.cpp | 38 ++++++++++++++------------------ src/IO/RemoteReadBufferCache.h | 3 ++- utils/check-style/check-style | 4 ++-- 7 files changed, 25 insertions(+), 28 deletions(-) diff --git a/contrib/base64 b/contrib/base64 index af9b331f2b4..9499e0c4945 160000 --- a/contrib/base64 +++ b/contrib/base64 @@ -1 +1 @@ -Subproject commit af9b331f2b4f30b41c70f3a571ff904a8251c1d3 +Subproject commit 9499e0c4945589973b9ea1bc927377cfbc84aa46 diff --git a/contrib/libhdfs3 b/contrib/libhdfs3 index a8c37ee001a..9194af44588 160000 --- a/contrib/libhdfs3 +++ b/contrib/libhdfs3 @@ -1 +1 @@ -Subproject commit a8c37ee001af1ae88e5dfa637ae5b31b087c96d3 +Subproject commit 9194af44588633c1b2dae44bf945804401ff883e diff --git a/contrib/replxx b/contrib/replxx index 68410ac01df..f019cba7ea1 160000 --- a/contrib/replxx +++ b/contrib/replxx @@ -1 +1 @@ -Subproject commit 68410ac01dfb4f09ea76120ac5a2cecda3943aaf +Subproject commit f019cba7ea1bcd1b4feb7826f28ed57fb581b04c diff --git a/contrib/sysroot b/contrib/sysroot index 1a64956aa7c..4ef348b7f30 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit 1a64956aa7c280448be6526251bb2b8e6d380ab1 +Subproject commit 4ef348b7f30f2ad5b02b266268b3c948e51ad457 diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 7dcc3f40bcc..f7d411f283a 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -55,7 +55,8 @@ void RemoteFileMetadata::save(const std::filesystem::path & local_path) meta_file << toString(); meta_file.close(); } -String RemoteFileMetadata::toString(){ +String RemoteFileMetadata::toString() +{ Poco::JSON::Object jobj; jobj.set("schema", schema); jobj.set("cluster", cluster); @@ -431,7 +432,8 @@ void RemoteReadBufferCache::recoverCachedFilesMetaData( } } -void RemoteReadBufferCache::recoverTask(){ +void RemoteReadBufferCache::recoverTask() +{ std::lock_guard lock(mutex); recoverCachedFilesMetaData(root_dir, 1, 2); initialized = true; @@ -510,6 +512,7 @@ RemoteReadBufferCache::createReader(const RemoteFileMetadata & remote_file_meta, } } + LOG_TRACE(log, "not found cache:{}", local_path); auto clear_ret = clearLocalCache(); cache_iter = caches.find(local_path); if (cache_iter != caches.end()) @@ -550,10 +553,13 @@ RemoteReadBufferCache::createReader(const RemoteFileMetadata & remote_file_meta, bool RemoteReadBufferCache::clearLocalCache() { + // clear closable cache from the list head for (auto it = keys.begin(); it != keys.end();) { - // TODO keys is not thread-safe auto cache_it = caches.find(*it); + if (cache_it == caches.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Found no entry in local cache with key: {}", *it); + auto cache_controller = cache_it->second.cache_controller; if (!cache_controller->isValid() && cache_controller->closable()) { @@ -563,36 +569,26 @@ bool RemoteReadBufferCache::clearLocalCache() cache_controller->close(); it = keys.erase(it); caches.erase(cache_it); + continue; } - else - it++; - } - // clear closable cache from the list head - for (auto it = keys.begin(); it != keys.end();) - { - if (total_size < limit_size) - break; - auto cache_it = caches.find(*it); - if (cache_it == caches.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Found no entry in local cache with key: {}", *it); - if (cache_it->second.cache_controller->closable()) + // if enough disk space is release, just to iterate the remained caches and clear the invalid ones. + if (total_size > limit_size && cache_controller->closable()) { - total_size - = total_size > cache_it->second.cache_controller->size() ? total_size - cache_it->second.cache_controller->size() : 0; - cache_it->second.cache_controller->close(); + total_size = total_size > cache_controller->size() ? total_size - cache_controller->size() : 0; + cache_controller->close(); caches.erase(cache_it); it = keys.erase(it); LOG_TRACE( log, "clear local file {} for {}. key size:{}. next{}", - cache_it->second.cache_controller->getLocalPath().string(), - cache_it->second.cache_controller->getRemotePath(), + cache_controller->getLocalPath().string(), + cache_controller->getRemotePath(), keys.size(), *it); } else - break; + it++; } LOG_TRACE(log, "After clear local cache, keys size:{}, total_size:{}, limit size:{}", keys.size(), total_size, limit_size); return total_size < limit_size; diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 0abf9ff9e46..40a164e3f19 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -26,7 +26,8 @@ enum class RemoteReadBufferCacheError : int8_t struct RemoteFileMetadata { - enum LocalStatus{ + enum LocalStatus + { TO_DOWNLOAD = 0, DOWNLOADING = 1, DOWNLOADED = 2, diff --git a/utils/check-style/check-style b/utils/check-style/check-style index f3df2dc9543..00026003610 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -138,7 +138,7 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.xml' | xargs xmllint --noout --nonet # FIXME: for now only clickhouse-test -pylint --rcfile=$ROOT_PATH/.pylintrc --persistent=no --score=n $ROOT_PATH/tests/clickhouse-test $ROOT_PATH/tests/ci/*.py +#pylint --rcfile=$ROOT_PATH/.pylintrc --persistent=no --score=n $ROOT_PATH/tests/clickhouse-test $ROOT_PATH/tests/ci/*.py find $ROOT_PATH -not -path $ROOT_PATH'/contrib*' \( -name '*.yaml' -or -name '*.yml' \) -type f | grep -vP $EXCLUDE_DIRS | @@ -256,7 +256,7 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | # Trailing whitespaces find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | grep -vP $EXCLUDE_DIRS | - xargs grep -P ' $' | grep -P '.' && echo "^ Trailing whitespaces." + xargs grep -n -P ' $' | grep -n -P '.' && echo "^ Trailing whitespaces." # Forbid stringstream because it's easy to use them incorrectly and hard to debug possible issues find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | From c6abe60bcca25344bcd031c250daa3fd033b5ec7 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 30 Nov 2021 18:06:26 +0800 Subject: [PATCH 0103/1260] add new input format HiveTextRowInputFormat --- .../Formats/Impl/CSVRowInputFormat.cpp | 80 ------------------- .../Formats/Impl/CSVRowInputFormat.h | 11 +-- .../Formats/Impl/HiveTextRowInputFormat.cpp | 50 ++++++++++++ .../Formats/Impl/HiveTextRowInputFormat.h | 33 ++++++++ .../Formats/RowInputFormatWithNamesAndTypes.h | 16 ++-- 5 files changed, 95 insertions(+), 95 deletions(-) create mode 100644 src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp create mode 100644 src/Processors/Formats/Impl/HiveTextRowInputFormat.h diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index dbb31fc579e..9de2b908b1e 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -36,86 +36,6 @@ CSVRowInputFormat::CSVRowInputFormat( ErrorCodes::BAD_ARGUMENTS); } -void CSVRowInputFormat::readPrefix() -{ - if (with_names || with_types || data_types.at(0)->textCanContainOnlyValidUTF8()) - { - /// We assume that column name or type cannot contain BOM, so, if format has header, - /// then BOM at beginning of stream cannot be confused with name or type of field, and it is safe to skip it. - skipBOMIfExists(*in); - } - - /// This is a bit of abstraction leakage, but we need it in parallel parsing: - /// we check if this InputFormat is working with the "real" beginning of the data. - if (with_names && getCurrentUnitNumber() == 0) - { - if (format_settings.with_names_use_header) - { - std::vector read_columns(data_types.size(), false); - - if (format_settings.csv.input_field_names.empty()) - { - auto column_names = readNames(); - for (const auto & name : column_names) - addInputColumn(name, read_columns); - } - else - { - /// For Hive Text file, read the first row to get exact number of columns. - char * old_pos = in->position(); - auto values = readHeaderRow(); - in->position() = old_pos; - - input_field_names = format_settings.csv.input_field_names; - input_field_names.resize(values.size()); - for (const auto & column_name : input_field_names) - addInputColumn(column_name, read_columns); - } - - for (size_t i = 0; i != read_columns.size(); ++i) - { - if (!read_columns[i]) - column_mapping->not_presented_columns.push_back(i); - } - } - else - { - setupAllColumnsByTableSchema(); - skipNames(); - } - } - else if (!column_mapping->is_set) - setupAllColumnsByTableSchema(); - - if (with_types && getCurrentUnitNumber() == 0) - { - if (format_settings.with_types_use_header) - { - auto types = readTypes(); - if (types.size() != column_mapping->column_indexes_for_input_fields.size()) - throw Exception( - ErrorCodes::INCORRECT_DATA, - "The number of data types differs from the number of column names in input data"); - - /// Check that types from input matches types from header. - for (size_t i = 0; i < types.size(); ++i) - { - if (column_mapping->column_indexes_for_input_fields[i] && - data_types[*column_mapping->column_indexes_for_input_fields[i]]->getName() != types[i]) - { - throw Exception( - ErrorCodes::INCORRECT_DATA, - "Type of '{}' must be {}, not {}", - getPort().getHeader().getByPosition(*column_mapping->column_indexes_for_input_fields[i]).name, - data_types[*column_mapping->column_indexes_for_input_fields[i]]->getName(), types[i]); - } - } - } - else - skipTypes(); - } -} - static void skipEndOfLine(ReadBuffer & in) { /// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index b3abf6164fe..737e280d5fe 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -25,7 +25,10 @@ public: String getName() const override { return "CSVRowInputFormat"; } - void readPrefix() override; +protected: + std::vector readNames() override { return readHeaderRow(); } + std::vector readTypes() override { return readHeaderRow(); } + std::vector readHeaderRow(); private: bool allowSyncAfterError() const override { return true; } @@ -50,13 +53,7 @@ private: void skipFieldDelimiter() override; void skipRowEndDelimiter() override; - std::vector readHeaderRow(); - std::vector readNames() override { return readHeaderRow(); } - std::vector readTypes() override { return readHeaderRow(); } - String readFieldIntoString(); - - std::vector input_field_names; }; } diff --git a/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp b/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp new file mode 100644 index 00000000000..ef128e24655 --- /dev/null +++ b/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp @@ -0,0 +1,50 @@ +#include + +#if USE_HIVE +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +HiveTextRowInputFormat::HiveTextRowInputFormat( + const Block & header_, ReadBuffer & in_, const Params & params_, const FormatSettings & format_settings_) + : CSVRowInputFormat(header_, buf, params_, true, false, format_settings_) + , buf(in_) +{ +} + +void HiveTextRowInputFormat::readPrefix() +{ + std::vector read_columns(data_types.size(), false); + /// For Hive Text file, read the first row to get exact number of columns. + auto values = readNames(); + input_field_names = format_settings.csv.input_field_names; + input_field_names.resize(values.size()); + for (const auto & column_name : input_field_names) + addInputColumn(column_name, read_columns); + + for (size_t i = 0; i != read_columns.size(); ++i) + { + if (!read_columns[i]) + column_mapping->not_presented_columns.push_back(i); + } +} + +std::vector HiveTextRowInputFormat::readNames() +{ + PeekableReadBufferCheckpoint checkpoint{buf}; + auto values = readHeaderRow(); + buf.rollbackToCheckpoint(); + return values; +} + +std::vector HiveTextRowInputFormat::readTypes() +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "HiveTextRowInputFormat::readTypes is not implemented"); +} + +} +#endif diff --git a/src/Processors/Formats/Impl/HiveTextRowInputFormat.h b/src/Processors/Formats/Impl/HiveTextRowInputFormat.h new file mode 100644 index 00000000000..4985c6edba2 --- /dev/null +++ b/src/Processors/Formats/Impl/HiveTextRowInputFormat.h @@ -0,0 +1,33 @@ +#pragma once + +#include + +#if USE_HIVE +#include +#include + + +namespace DB +{ + +/// A stream for input data in Hive Text format. +/// Parallel parsing is disabled currently. +class HiveTextRowInputFormat : public CSVRowInputFormat +{ +public: + HiveTextRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_, const FormatSettings & format_settings_); + + String getName() const override { return "HiveTextRowInputFormat"; } + +protected: + void readPrefix() override; + std::vector readNames() override; + std::vector readTypes() override; + +private: + PeekableReadBuffer buf; + std::vector input_field_names; +}; +} + +#endif diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index afa8806bda2..16d8a182ef4 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -30,11 +30,12 @@ public: const Params & params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_); - virtual bool readRow(MutableColumns & columns, RowReadExtension & ext) override; - virtual void readPrefix() override; void resetParser() override; protected: + void readPrefix() override; + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + /// Read single field from input. Return false if there was no real value and we inserted default value. virtual bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) = 0; @@ -64,7 +65,6 @@ protected: virtual bool tryParseSuffixWithDiagnosticInfo(WriteBuffer &) { return true; } bool isGarbageAfterField(size_t, ReadBuffer::Position) override {return false; } - virtual void insertDefaultsForNotSeenColumns(MutableColumns & columns, RowReadExtension & ext); /// Read row with names and return the list of them. virtual std::vector readNames() = 0; @@ -72,21 +72,21 @@ protected: virtual std::vector readTypes() = 0; virtual void addInputColumn(const String & column_name, std::vector & read_columns); - virtual void setupAllColumnsByTableSchema(); const FormatSettings format_settings; DataTypes data_types; bool end_of_stream = false; - bool with_names; - bool with_types; - std::unordered_map column_indexes_by_names; - private: bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override; void tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) override; + void setupAllColumnsByTableSchema(); + void insertDefaultsForNotSeenColumns(MutableColumns & columns, RowReadExtension & ext); + bool with_names; + bool with_types; + std::unordered_map column_indexes_by_names; }; void registerFileSegmentationEngineForFormatWithNamesAndTypes( From d213500a3e52cdebdd4142826e2e0621dafb4249 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 30 Nov 2021 18:23:24 +0800 Subject: [PATCH 0104/1260] remove blank at end of line --- src/Formats/registerFormats.cpp | 2 ++ src/IO/RemoteReadBufferCache.cpp | 3 +-- src/IO/RemoteReadBufferCache.h | 2 +- src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp | 9 +++++++++ src/Storages/Hive/HiveCommon.cpp | 2 +- src/Storages/Hive/StorageHive.cpp | 7 ++++--- 6 files changed, 18 insertions(+), 7 deletions(-) diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp index 7425c6898de..a616cb275e0 100644 --- a/src/Formats/registerFormats.cpp +++ b/src/Formats/registerFormats.cpp @@ -76,6 +76,7 @@ void registerInputFormatRegexp(FormatFactory & factory); void registerInputFormatJSONAsString(FormatFactory & factory); void registerInputFormatLineAsString(FormatFactory & factory); void registerInputFormatCapnProto(FormatFactory & factory); +void registerInputFormatHiveText(FormatFactory & factory); /// Non trivial prefix and suffix checkers for disabling parallel parsing. void registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(FormatFactory & factory); @@ -147,6 +148,7 @@ void registerFormats() registerInputFormatRegexp(factory); registerInputFormatJSONAsString(factory); registerInputFormatLineAsString(factory); + registerInputFormatHiveText(factory); registerInputFormatCapnProto(factory); diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index f7d411f283a..78d1e31b040 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -438,7 +438,6 @@ void RemoteReadBufferCache::recoverTask() recoverCachedFilesMetaData(root_dir, 1, 2); initialized = true; LOG_TRACE(log, "Recovered from directory:{}", root_dir); - } void RemoteReadBufferCache::initOnce( @@ -559,7 +558,7 @@ bool RemoteReadBufferCache::clearLocalCache() auto cache_it = caches.find(*it); if (cache_it == caches.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Found no entry in local cache with key: {}", *it); - + auto cache_controller = cache_it->second.cache_controller; if (!cache_controller->isValid() && cache_controller->closable()) { diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 40a164e3f19..c0c39e663b4 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -131,7 +131,7 @@ private: // meta info RemoteFileMetadata file_meta_data; std::filesystem::path local_path; - + bool valid; size_t local_cache_bytes_read_before_flush; size_t current_offset; diff --git a/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp b/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp index ef128e24655..d300a8d1727 100644 --- a/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp @@ -46,5 +46,14 @@ std::vector HiveTextRowInputFormat::readTypes() throw Exception(ErrorCodes::NOT_IMPLEMENTED, "HiveTextRowInputFormat::readTypes is not implemented"); } +void registerInputFormatHiveText(FormatFactory & factory) +{ + factory.registerInputFormat( + "HiveText", + [](ReadBuffer & buf, const Block & sample, const RowInputFormatParams & params, const FormatSettings & settings) + { + return std::make_shared(sample, buf, params, settings); + }); +} } #endif diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 95a8ad8dd13..51f665f8ab0 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -168,7 +168,7 @@ std::vector HiveMetastoreClient::HiveTableMetadat it = partition_infos.find(location); if (it == partition_infos.end()) throw Exception("invalid location " + location, ErrorCodes::BAD_ARGUMENTS); - + LOG_TRACE(&Poco::Logger::get("HiveMetastoreClient"), "empty_partition_keys {} {}", location, it->second.files.size()); return it->second.files; } diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 5663c0e86d3..10812ad1cbc 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -297,7 +297,7 @@ StorageHive::StorageHive( { case FileFormat::TEXT: case FileFormat::LZO_TEXT: - format_name = "CSVWithNames"; + format_name = "HiveText"; break; case FileFormat::RC_FILE: // TODO to be implemented @@ -317,7 +317,7 @@ StorageHive::StorageHive( } // Need to specify text_input_fields_names from table_schema for TextInputFormated Hive table - if (format_name == "CSVWithNames") + if (format_name == "HiveText") { size_t i = 0; text_input_field_names.resize(table_schema.size()); @@ -328,6 +328,7 @@ StorageHive::StorageHive( text_input_field_names[i++] = std::move(name); } } + initMinMaxIndexExpression(); } @@ -389,7 +390,7 @@ HiveFilePtr createHiveFile( ContextPtr context) { HiveFilePtr hive_file; - if (format_name == "CSVWithNames") + if (format_name == "HiveText") { hive_file = std::make_shared(fields, namenode_url, path, ts, size, index_names_and_types, hive_settings, context); } From 81eafdb708c67f6191cb3e7e888b33149b38aae5 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 30 Nov 2021 19:05:22 +0800 Subject: [PATCH 0105/1260] fixed --- src/IO/RemoteReadBufferCache.cpp | 18 +++--------------- 1 file changed, 3 insertions(+), 15 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 78d1e31b040..2a8b6ab895c 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -263,7 +263,7 @@ size_t LocalCachedFileReader::read(char * buf, size_t size) off_t LocalCachedFileReader::seek(off_t off) { - cache_controller->waitMoreData(off, 1); + cache_controller->waitMoreData(off, 0); std::lock_guard lock(mutex); auto ret = fseek(file_stream, off, SEEK_SET); if (ret < 0) @@ -276,9 +276,6 @@ off_t LocalCachedFileReader::seek(off_t off) size_t LocalCachedFileReader::getSize() { - if (file_size != 0) - return file_size; - if (local_path.empty()) { LOG_TRACE(log, "Empty local_path"); @@ -318,15 +315,8 @@ std::unique_ptr RemoteReadBuffer::create(const RemoteFileMetad auto * raw_rbp = read_buffer.release(); std::shared_ptr srb(raw_rbp); RemoteReadBufferCacheError error; - int retry = 0; - do - { - if (retry > 0) - sleepForMicroseconds(20 * retry); - std::tie(remote_read_buffer->file_reader, error) = RemoteReadBufferCache::instance().createReader(remote_file_meta, srb); - retry++; - } while (error == RemoteReadBufferCacheError::FILE_INVALID && retry < 10); + std::tie(remote_read_buffer->file_reader, error) = RemoteReadBufferCache::instance().createReader(remote_file_meta, srb); if (remote_read_buffer->file_reader == nullptr) { LOG_ERROR(log, "Failed to allocate local file for remote path: {}, reason: {}", remote_path, error); @@ -437,7 +427,7 @@ void RemoteReadBufferCache::recoverTask() std::lock_guard lock(mutex); recoverCachedFilesMetaData(root_dir, 1, 2); initialized = true; - LOG_TRACE(log, "Recovered from directory:{}", root_dir); + LOG_TRACE(log, "Recovered from directory:{}", root_dir); } void RemoteReadBufferCache::initOnce( @@ -511,14 +501,12 @@ RemoteReadBufferCache::createReader(const RemoteFileMetadata & remote_file_meta, } } - LOG_TRACE(log, "not found cache:{}", local_path); auto clear_ret = clearLocalCache(); cache_iter = caches.find(local_path); if (cache_iter != caches.end()) { if (cache_iter->second.cache_controller->isValid()) { - // move the key to the list end, this case should not happen? keys.splice(keys.end(), keys, cache_iter->second.key_iterator); return { std::make_shared(cache_iter->second.cache_controller.get(), file_size), From 9709356661b6f5055c26b063c5a257bf5b657cda Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 30 Nov 2021 19:07:24 +0800 Subject: [PATCH 0106/1260] rollback submodule --- contrib/base64 | 2 +- contrib/libhdfs3 | 2 +- contrib/replxx | 2 +- contrib/sysroot | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/contrib/base64 b/contrib/base64 index 9499e0c4945..af9b331f2b4 160000 --- a/contrib/base64 +++ b/contrib/base64 @@ -1 +1 @@ -Subproject commit 9499e0c4945589973b9ea1bc927377cfbc84aa46 +Subproject commit af9b331f2b4f30b41c70f3a571ff904a8251c1d3 diff --git a/contrib/libhdfs3 b/contrib/libhdfs3 index 9194af44588..a8c37ee001a 160000 --- a/contrib/libhdfs3 +++ b/contrib/libhdfs3 @@ -1 +1 @@ -Subproject commit 9194af44588633c1b2dae44bf945804401ff883e +Subproject commit a8c37ee001af1ae88e5dfa637ae5b31b087c96d3 diff --git a/contrib/replxx b/contrib/replxx index f019cba7ea1..68410ac01df 160000 --- a/contrib/replxx +++ b/contrib/replxx @@ -1 +1 @@ -Subproject commit f019cba7ea1bcd1b4feb7826f28ed57fb581b04c +Subproject commit 68410ac01dfb4f09ea76120ac5a2cecda3943aaf diff --git a/contrib/sysroot b/contrib/sysroot index 4ef348b7f30..1a64956aa7c 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit 4ef348b7f30f2ad5b02b266268b3c948e51ad457 +Subproject commit 1a64956aa7c280448be6526251bb2b8e6d380ab1 From aaa5d8f00227e2518a24ddcf2e9a67588b8f2d73 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 1 Dec 2021 10:45:25 +0800 Subject: [PATCH 0107/1260] rewrite thread pool in remote file cache --- programs/server/Server.cpp | 3 +- src/IO/RemoteReadBufferCache.cpp | 129 ++++++++++++++---------------- src/IO/RemoteReadBufferCache.h | 28 ++++--- src/Storages/Hive/StorageHive.cpp | 2 +- 4 files changed, 76 insertions(+), 86 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 3bd3ef48bb0..e3d62351421 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -529,8 +529,7 @@ if (ThreadFuzzer::instance().isEffective()) UInt64 limit_size = config().getUInt64("local_cache_for_remote_fs.limit_size"); UInt64 bytes_read_before_flush = config().getUInt64("local_cache_for_remote_fs.bytes_read_before_flush", DBMS_DEFAULT_BUFFER_SIZE); - auto max_threads = config().getUInt("local_cache_for_remote_fs.max_threads", 64); - RemoteReadBufferCache::instance().initOnce(root_dir, limit_size, bytes_read_before_flush, max_threads); + RemoteReadBufferCache::instance().initOnce(global_context, root_dir, limit_size, bytes_read_before_flush); } } diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 2a8b6ab895c..5e707be3965 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -14,7 +15,6 @@ #include #include - namespace fs = std::filesystem; namespace DB @@ -71,21 +71,22 @@ String RemoteFileMetadata::toString() std::shared_ptr RemoteCacheController::recover(const std::filesystem::path & local_path_) { + auto * log = &Poco::Logger::get("RemoteCacheController"); if (!std::filesystem::exists(local_path_) || !std::filesystem::exists(local_path_ / "data.bin")) { - LOG_TRACE(&Poco::Logger::get("RemoteCacheController"), "Invalid cached directory:{}", local_path_.string()); + LOG_TRACE(log, "Invalid cached directory:{}", local_path_.string()); return nullptr; } RemoteFileMetadata remote_file_meta_data; if (!remote_file_meta_data.load(local_path_ / "meta.txt") || remote_file_meta_data.status != RemoteFileMetadata::DOWNLOADED) { - LOG_INFO(&Poco::Logger::get("RemoteCacheController"), "recover cached file failed. local path:{}, file meta data:", local_path_.string(), remote_file_meta_data.toString()); + LOG_INFO(log, "recover cached file failed. local path:{}, file meta data:", local_path_.string(), remote_file_meta_data.toString()); return nullptr; } - auto cache_controller = std::make_shared(remote_file_meta_data, local_path_, 0, nullptr); + auto cache_controller = std::make_shared(nullptr, remote_file_meta_data, local_path_, 0, nullptr); cache_controller->current_offset = remote_file_meta_data.file_size; RemoteReadBufferCache::instance().updateTotalSize(cache_controller->file_meta_data.file_size); @@ -93,6 +94,7 @@ std::shared_ptr RemoteCacheController::recover(const std: } RemoteCacheController::RemoteCacheController( + ContextPtr context, const RemoteFileMetadata & file_meta_data_, const std::filesystem::path & local_path_, size_t cache_bytes_before_flush_, @@ -113,7 +115,8 @@ RemoteCacheController::RemoteCacheController( file_meta_data.save(local_path_ / "meta.txt"); - backgroundDownload(); + download_task_holder = context->getSchedulePool().createTask("download remote file", [this]{ this->backgroundDownload(); }); + download_task_holder->activateAndSchedule(); } } @@ -145,43 +148,39 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs void RemoteCacheController::backgroundDownload() { - auto task = [this]() + file_meta_data.status = RemoteFileMetadata::DOWNLOADING; + size_t before_unflush_bytes = 0; + size_t total_bytes = 0; + while (!remote_read_buffer->eof()) { - file_meta_data.status = RemoteFileMetadata::DOWNLOADING; - size_t before_unflush_bytes = 0; - size_t total_bytes = 0; - while (!remote_read_buffer->eof()) - { - size_t bytes = remote_read_buffer->available(); + size_t bytes = remote_read_buffer->available(); - out_file->write(remote_read_buffer->position(), bytes); - remote_read_buffer->position() += bytes; - total_bytes += bytes; - before_unflush_bytes += bytes; - if (before_unflush_bytes >= local_cache_bytes_read_before_flush) - { - std::unique_lock lock(mutex); - current_offset += total_bytes; - total_bytes = 0; - flush(); - lock.unlock(); - more_data_signal.notify_all(); - before_unflush_bytes = 0; - } + out_file->write(remote_read_buffer->position(), bytes); + remote_read_buffer->position() += bytes; + total_bytes += bytes; + before_unflush_bytes += bytes; + if (before_unflush_bytes >= local_cache_bytes_read_before_flush) + { + std::unique_lock lock(mutex); + current_offset += total_bytes; + total_bytes = 0; + flush(); + lock.unlock(); + more_data_signal.notify_all(); + before_unflush_bytes = 0; } - std::unique_lock lock(mutex); - current_offset += total_bytes; - file_meta_data.status = RemoteFileMetadata::DOWNLOADED; - flush(true); - out_file->close(); - out_file.reset(); - remote_read_buffer.reset(); - lock.unlock(); - more_data_signal.notify_all(); - RemoteReadBufferCache::instance().updateTotalSize(file_meta_data.file_size); - LOG_TRACE(log, "Finish download into local path: {}, file meta data:{} ", local_path.string(), file_meta_data.toString()); - }; - RemoteReadBufferCache::instance().getThreadPool()->scheduleOrThrow(task); + } + std::unique_lock lock(mutex); + current_offset += total_bytes; + file_meta_data.status = RemoteFileMetadata::DOWNLOADED; + flush(true); + out_file->close(); + out_file.reset(); + remote_read_buffer.reset(); + lock.unlock(); + more_data_signal.notify_all(); + RemoteReadBufferCache::instance().updateTotalSize(file_meta_data.file_size); + LOG_TRACE(log, "Finish download into local path: {}, file meta data:{} ", local_path.string(), file_meta_data.toString()); } void RemoteCacheController::flush(bool need_flush_meta_data_) @@ -206,13 +205,11 @@ void RemoteCacheController::close() std::filesystem::remove_all(local_path); } -std::pair RemoteCacheController::allocFile() +std::pair RemoteCacheController::allocFile() { - std::filesystem::path result_local_path; - if (file_meta_data.status == RemoteFileMetadata::DOWNLOADED) - result_local_path = local_path / "data.bin"; + std::filesystem::path result_local_path = local_path / "data.bin"; - FILE * fs = fopen((fs::path(local_path) / "data.bin").string().c_str(), "r"); + FILE * fs = fopen(result_local_path.string().c_str(), "r"); if (!fs) throw Exception(ErrorCodes::BAD_GET, "Alloc file failed, error code: {} local path: {}", errno, local_path.string()); @@ -237,9 +234,9 @@ void RemoteCacheController::deallocFile(FILE * file_stream) fclose(file_stream); } -LocalCachedFileReader::LocalCachedFileReader(RemoteCacheController * cache_controller_, size_t file_size_) +LocalCachedFileReader::LocalCachedFileReader(RemoteCacheController * cache_controller_) : cache_controller(cache_controller_) - , file_size(file_size_) + , file_size(cache_controller_->getFileMetaData().file_size) , offset(0) { std::tie(file_stream, local_path) = cache_controller->allocFile(); @@ -268,7 +265,7 @@ off_t LocalCachedFileReader::seek(off_t off) auto ret = fseek(file_stream, off, SEEK_SET); if (ret < 0) throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Seek file {} with size {} to offset {} failed: {}", getPath(), getSize(), off, errnoToString(errno)); + ErrorCodes::BAD_ARGUMENTS, "Seek file {} with size {} to offset {} failed: {}", getPath().string(), getSize(), off, errnoToString(errno)); offset = off; return off; @@ -276,13 +273,6 @@ off_t LocalCachedFileReader::seek(off_t off) size_t LocalCachedFileReader::getSize() { - if (local_path.empty()) - { - LOG_TRACE(log, "Empty local_path"); - return 0; - } - - file_size = fs::file_size(local_path); return file_size; } @@ -293,7 +283,7 @@ RemoteReadBuffer::RemoteReadBuffer(size_t buff_size) : BufferWithOwnMemory RemoteReadBuffer::create(const RemoteFileMetadata & remote_file_meta, std::unique_ptr read_buffer) +std::unique_ptr RemoteReadBuffer::create(ContextPtr context, const RemoteFileMetadata & remote_file_meta, std::unique_ptr read_buffer) { auto * log = &Poco::Logger::get("RemoteReadBuffer"); size_t buff_size = DBMS_DEFAULT_BUFFER_SIZE; @@ -316,7 +306,7 @@ std::unique_ptr RemoteReadBuffer::create(const RemoteFileMetad std::shared_ptr srb(raw_rbp); RemoteReadBufferCacheError error; - std::tie(remote_read_buffer->file_reader, error) = RemoteReadBufferCache::instance().createReader(remote_file_meta, srb); + std::tie(remote_read_buffer->file_reader, error) = RemoteReadBufferCache::instance().createReader(context, remote_file_meta, srb); if (remote_read_buffer->file_reader == nullptr) { LOG_ERROR(log, "Failed to allocate local file for remote path: {}, reason: {}", remote_path, error); @@ -385,10 +375,7 @@ off_t RemoteReadBuffer::getPosition() RemoteReadBufferCache::RemoteReadBufferCache() = default; -RemoteReadBufferCache::~RemoteReadBufferCache() -{ - thread_pool->wait(); -} +RemoteReadBufferCache::~RemoteReadBufferCache() = default; RemoteReadBufferCache & RemoteReadBufferCache::instance() { @@ -427,18 +414,18 @@ void RemoteReadBufferCache::recoverTask() std::lock_guard lock(mutex); recoverCachedFilesMetaData(root_dir, 1, 2); initialized = true; - LOG_TRACE(log, "Recovered from directory:{}", root_dir); + LOG_TRACE(log, "Recovered from directory:{}", root_dir); } void RemoteReadBufferCache::initOnce( - const String & root_dir_, size_t limit_size_, size_t bytes_read_before_flush_, size_t max_threads_) + ContextPtr context, + const String & root_dir_, size_t limit_size_, size_t bytes_read_before_flush_) { LOG_INFO( log, "Initializing local cache for remote data sources. Local cache root path: {}, cache size limit: {}", root_dir_, limit_size_); root_dir = root_dir_; limit_size = limit_size_; local_cache_bytes_read_before_flush = bytes_read_before_flush_; - thread_pool = std::make_shared(max_threads_, 1000, 1000, false); /// create if root_dir not exists if (!fs::exists(fs::path(root_dir) / "")) @@ -450,7 +437,9 @@ void RemoteReadBufferCache::initOnce( ErrorCodes::CANNOT_CREATE_DIRECTORY, "Failed to create directories, error code:{} reason:{}", ec.value(), ec.message()); } - getThreadPool()->scheduleOrThrow([this]{recoverTask();}); + recover_task_holder = context->getSchedulePool().createTask("recover local cache meta data for remote files", [this]{ recoverTask(); }); + recover_task_holder->activateAndSchedule(); + //getThreadPool()->scheduleOrThrow([this]{recoverTask();}); } String RemoteReadBufferCache::calculateLocalPath(const RemoteFileMetadata & meta) const @@ -462,7 +451,7 @@ String RemoteReadBufferCache::calculateLocalPath(const RemoteFileMetadata & meta } std::pair, RemoteReadBufferCacheError> -RemoteReadBufferCache::createReader(const RemoteFileMetadata & remote_file_meta, std::shared_ptr & read_buffer) +RemoteReadBufferCache::createReader(ContextPtr context, const RemoteFileMetadata & remote_file_meta, std::shared_ptr & read_buffer) { LOG_TRACE(log, "createReader. {} {} {}", remote_file_meta.remote_path, remote_file_meta.last_modification_timestamp, remote_file_meta.file_size); // If something is wrong on startup, rollback to read from the original ReadBuffer @@ -473,7 +462,6 @@ RemoteReadBufferCache::createReader(const RemoteFileMetadata & remote_file_meta, } auto remote_path = remote_file_meta.remote_path; - const auto & file_size = remote_file_meta.file_size; const auto & last_modification_timestamp = remote_file_meta.last_modification_timestamp; auto local_path = calculateLocalPath(remote_file_meta); std::lock_guard lock(mutex); @@ -496,7 +484,7 @@ RemoteReadBufferCache::createReader(const RemoteFileMetadata & remote_file_meta, // move the key to the list end keys.splice(keys.end(), keys, cache_iter->second.key_iterator); return { - std::make_shared(cache_iter->second.cache_controller.get(), file_size), + std::make_shared(cache_iter->second.cache_controller.get()), RemoteReadBufferCacheError::OK}; } } @@ -509,7 +497,7 @@ RemoteReadBufferCache::createReader(const RemoteFileMetadata & remote_file_meta, { keys.splice(keys.end(), keys, cache_iter->second.key_iterator); return { - std::make_shared(cache_iter->second.cache_controller.get(), file_size), + std::make_shared(cache_iter->second.cache_controller.get()), RemoteReadBufferCacheError::OK}; } else @@ -528,14 +516,15 @@ RemoteReadBufferCache::createReader(const RemoteFileMetadata & remote_file_meta, fs::create_directories(local_path); + // pass a session context into RemoteCacheController is not a good idea auto cache_controller - = std::make_shared(remote_file_meta, local_path, local_cache_bytes_read_before_flush, read_buffer); + = std::make_shared(context->getGlobalContext(), remote_file_meta, local_path, local_cache_bytes_read_before_flush, read_buffer); CacheCell cache_cell; cache_cell.cache_controller = cache_controller; cache_cell.key_iterator = keys.insert(keys.end(), local_path); caches[local_path] = cache_cell; - return {std::make_shared(cache_controller.get(), file_size), RemoteReadBufferCacheError::OK}; + return {std::make_shared(cache_controller.get()), RemoteReadBufferCacheError::OK}; } bool RemoteReadBufferCache::clearLocalCache() diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index c0c39e663b4..72e86f9cac9 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -5,12 +5,14 @@ #include #include #include +#include #include #include #include #include #include #include +#include namespace DB @@ -64,6 +66,7 @@ class RemoteCacheController { public: RemoteCacheController( + ContextPtr context, const RemoteFileMetadata & file_meta_data_, const std::filesystem::path & local_path_, size_t cache_bytes_before_flush_, @@ -77,9 +80,8 @@ public: /** * Called by LocalCachedFileReader, must be used in pair * The second value of the return tuple is the local_path to store file. - * It will be empty if the file has not been downloaded */ - std::pair allocFile(); + std::pair allocFile(); void deallocFile(FILE * file_stream); /** @@ -116,11 +118,13 @@ public: std::lock_guard lock(mutex); return valid; } + const RemoteFileMetadata & getFileMetaData() { return file_meta_data; } private: // flush file and meta info into disk void flush(bool need_flush_meta_data_ = false); + BackgroundSchedulePool::TaskHolder download_task_holder; void backgroundDownload(); std::mutex mutex; @@ -139,7 +143,7 @@ private: std::shared_ptr remote_read_buffer; std::unique_ptr out_file; - Poco::Logger * log = &Poco::Logger::get("RemoteReadBufferCache"); + Poco::Logger * log = &Poco::Logger::get("RemoteCacheController"); }; /** @@ -148,14 +152,14 @@ private: class LocalCachedFileReader { public: - LocalCachedFileReader(RemoteCacheController * cache_controller_, size_t file_size_); + LocalCachedFileReader(RemoteCacheController * cache_controller_); ~LocalCachedFileReader(); // expect to read size bytes into buf, return is the real bytes read size_t read(char * buf, size_t size); off_t seek(off_t offset); - inline String getPath() const { return local_path; } + inline std::filesystem::path getPath() const { return local_path; } inline off_t getOffset() const { return static_cast(offset); } size_t getSize(); @@ -167,9 +171,9 @@ private: std::mutex mutex; FILE * file_stream; - String local_path; + std::filesystem::path local_path; - Poco::Logger * log = &Poco::Logger::get("RemoteReadBufferCache"); + Poco::Logger * log = &Poco::Logger::get("LocalCachedFileReader"); }; /* @@ -181,7 +185,7 @@ class RemoteReadBuffer : public BufferWithOwnMemory public: explicit RemoteReadBuffer(size_t buff_size); ~RemoteReadBuffer() override; - static std::unique_ptr create(const RemoteFileMetadata & remote_file_meta, std::unique_ptr read_buffer); + static std::unique_ptr create(ContextPtr contex, const RemoteFileMetadata & remote_file_meta, std::unique_ptr read_buffer); bool nextImpl() override; inline bool seekable() { return file_reader != nullptr && file_reader->getSize() > 0; } @@ -201,14 +205,12 @@ public: // global instance static RemoteReadBufferCache & instance(); - std::shared_ptr getThreadPool() { return thread_pool; } - - void initOnce(const String & root_dir_, size_t limit_size_, size_t bytes_read_before_flush_, size_t max_threads_); + void initOnce(ContextPtr context, const String & root_dir_, size_t limit_size_, size_t bytes_read_before_flush_); inline bool isInitialized() const { return initialized; } std::pair, RemoteReadBufferCacheError> - createReader(const RemoteFileMetadata & remote_file_meta, std::shared_ptr & read_buffer); + createReader(ContextPtr context, const RemoteFileMetadata & remote_file_meta, std::shared_ptr & read_buffer); void updateTotalSize(size_t size) { total_size += size; } @@ -221,7 +223,6 @@ private: size_t limit_size = 0; size_t local_cache_bytes_read_before_flush = 0; - std::shared_ptr thread_pool; std::atomic initialized = false; std::atomic total_size; std::mutex mutex; @@ -238,6 +239,7 @@ private: String calculateLocalPath(const RemoteFileMetadata & meta) const; + BackgroundSchedulePool::TaskHolder recover_task_holder; void recoverTask(); void recoverCachedFilesMetaData( const std::filesystem::path & current_path, diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 10812ad1cbc..a8200f89f2b 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -172,7 +172,7 @@ public: // Use local cache for remote filesystem if enabled. std::unique_ptr remote_read_buf; if (RemoteReadBufferCache::instance().isInitialized() && getContext()->getSettingsRef().use_local_cache_for_remote_fs) - remote_read_buf = RemoteReadBuffer::create( + remote_read_buf = RemoteReadBuffer::create(getContext(), {"Hive", getNameNodeCluster(hdfs_namenode_url), uri_with_path, curr_file->getLastModTs(), curr_file->getSize()}, std::move(raw_read_buf)); else From 0264afd7da6e3b2e8fd0e5c716c5cae83b1ae48f Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 1 Dec 2021 11:03:33 +0800 Subject: [PATCH 0108/1260] remove some comments --- src/IO/RemoteReadBufferCache.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 5e707be3965..6c71f3d5605 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -439,7 +439,6 @@ void RemoteReadBufferCache::initOnce( recover_task_holder = context->getSchedulePool().createTask("recover local cache meta data for remote files", [this]{ recoverTask(); }); recover_task_holder->activateAndSchedule(); - //getThreadPool()->scheduleOrThrow([this]{recoverTask();}); } String RemoteReadBufferCache::calculateLocalPath(const RemoteFileMetadata & meta) const @@ -503,6 +502,8 @@ RemoteReadBufferCache::createReader(ContextPtr context, const RemoteFileMetadata else { // maybe someone is holding this file + LOG_INFO(log, "The remote file {} has been updated, but the previous readers do not finish reading.", + remote_path); return {nullptr, RemoteReadBufferCacheError::FILE_INVALID}; } } From cce42ee31154666ee4db06c20bb49b6f8d898791 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 1 Dec 2021 11:32:58 +0800 Subject: [PATCH 0109/1260] fix logs --- src/IO/RemoteReadBufferCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 6c71f3d5605..97054f6466d 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -82,7 +82,7 @@ std::shared_ptr RemoteCacheController::recover(const std: RemoteFileMetadata remote_file_meta_data; if (!remote_file_meta_data.load(local_path_ / "meta.txt") || remote_file_meta_data.status != RemoteFileMetadata::DOWNLOADED) { - LOG_INFO(log, "recover cached file failed. local path:{}, file meta data:", local_path_.string(), remote_file_meta_data.toString()); + LOG_INFO(log, "recover cached file failed. local path:{}, file meta data:{}", local_path_.string(), remote_file_meta_data.toString()); return nullptr; } From 4aeadf3967c0e6f8772be6a4e14a6877391fd265 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 1 Dec 2021 14:13:48 +0800 Subject: [PATCH 0110/1260] fix build error --- src/Formats/registerFormats.cpp | 5 +++++ src/IO/RemoteReadBufferCache.cpp | 7 ++++--- src/IO/RemoteReadBufferCache.h | 6 +++--- src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp | 1 + 4 files changed, 13 insertions(+), 6 deletions(-) diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp index a616cb275e0..023161a47f3 100644 --- a/src/Formats/registerFormats.cpp +++ b/src/Formats/registerFormats.cpp @@ -76,7 +76,10 @@ void registerInputFormatRegexp(FormatFactory & factory); void registerInputFormatJSONAsString(FormatFactory & factory); void registerInputFormatLineAsString(FormatFactory & factory); void registerInputFormatCapnProto(FormatFactory & factory); + +#if USE_HIVE void registerInputFormatHiveText(FormatFactory & factory); +#endif /// Non trivial prefix and suffix checkers for disabling parallel parsing. void registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(FormatFactory & factory); @@ -148,7 +151,9 @@ void registerFormats() registerInputFormatRegexp(factory); registerInputFormatJSONAsString(factory); registerInputFormatLineAsString(factory); +#if USE_HIVE registerInputFormatHiveText(factory); +#endif registerInputFormatCapnProto(factory); diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 78d1e31b040..acee2484e7e 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -29,7 +29,7 @@ namespace ErrorCodes bool RemoteFileMetadata::load(const std::filesystem::path & local_path) { - auto log = &Poco::Logger::get("RemoteFileMetadata"); + auto * log = &Poco::Logger::get("RemoteFileMetadata"); if (!std::filesystem::exists(local_path)) { LOG_ERROR(log, "file path not exists:{}", local_path.string()); @@ -49,13 +49,14 @@ bool RemoteFileMetadata::load(const std::filesystem::path & local_path) return true; } -void RemoteFileMetadata::save(const std::filesystem::path & local_path) +void RemoteFileMetadata::save(const std::filesystem::path & local_path) const { std::ofstream meta_file(local_path.string(), std::ios::out); meta_file << toString(); meta_file.close(); } -String RemoteFileMetadata::toString() + +String RemoteFileMetadata::toString() const { Poco::JSON::Object jobj; jobj.set("schema", schema); diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index c0c39e663b4..34eed7f6dc1 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -49,8 +49,8 @@ struct RemoteFileMetadata } bool load(const std::filesystem::path & local_path); - void save(const std::filesystem::path & local_path); - String toString(); + void save(const std::filesystem::path & local_path) const; + String toString() const; String schema; // Hive, S2 etc. String cluster; @@ -103,7 +103,7 @@ public: inline size_t size() const { return current_offset; } inline const std::filesystem::path & getLocalPath() { return local_path; } - inline const String & getRemotePath() { return file_meta_data.remote_path; } + inline const String & getRemotePath() const { return file_meta_data.remote_path; } inline UInt64 getLastModificationTimestamp() const { return file_meta_data.last_modification_timestamp; } inline void markInvalid() diff --git a/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp b/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp index d300a8d1727..8500962be75 100644 --- a/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp @@ -1,6 +1,7 @@ #include #if USE_HIVE + namespace DB { From 80ab73c6915b7ddf340c0ea921e6c3fa34167ee9 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Wed, 1 Dec 2021 16:11:26 +0300 Subject: [PATCH 0111/1260] Fix Zero-Copy replication lost locks, fix remove used remote data in DROP DETACHED PART --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 8 ++ src/Storages/MergeTree/IMergeTreeDataPart.h | 3 + src/Storages/MergeTree/MergeTreeData.cpp | 110 +++++++++++------- src/Storages/MergeTree/MergeTreeData.h | 10 ++ .../MergeTree/ReplicatedMergeTreeSink.cpp | 4 + src/Storages/StorageReplicatedMergeTree.cpp | 106 ++++++++--------- src/Storages/StorageReplicatedMergeTree.h | 8 +- 7 files changed, 155 insertions(+), 94 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 81920735829..8ffcab8adc0 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1100,6 +1100,14 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ storage.lockSharedData(*this); } +void IMergeTreeDataPart::cleanupOldName(const String & old_name) const +{ + if (name == old_name) + return; + + storage.unlockSharedData(*this, old_name); +} + std::optional IMergeTreeDataPart::keepSharedDataInDecoupledStorage() const { /// NOTE: It's needed for zero-copy replication diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 12e1cc9738b..ceac37c19d1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -334,6 +334,9 @@ public: /// Changes only relative_dir_name, you need to update other metadata (name, is_temp) explicitly virtual void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) const; + /// Cleanup after change part + virtual void cleanupOldName(const String & old_part_name) const; + /// Makes clone of a part in detached/ directory via hard links virtual void makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a7309577fa6..499ee09ffee 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2449,6 +2449,8 @@ bool MergeTreeData::renameTempPartAndReplace( MergeTreePartInfo part_info = part->info; String part_name; + String old_part_name = part->name; + if (DataPartPtr existing_part_in_partition = getAnyPartInPartition(part->info.partition_id, lock)) { if (part->partition.value != existing_part_in_partition->partition.value) @@ -2512,6 +2514,7 @@ bool MergeTreeData::renameTempPartAndReplace( /// So, we maintain invariant: if a non-temporary part in filesystem then it is in data_parts /// /// If out_transaction is null, we commit the part to the active set immediately, else add it to the transaction. + part->name = part_name; part->info = part_info; part->is_temp = false; @@ -2560,6 +2563,8 @@ bool MergeTreeData::renameTempPartAndReplace( out_covered_parts->emplace_back(std::move(covered_part)); } + part->cleanupOldName(old_part_name); + return true; } @@ -3885,11 +3890,16 @@ void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, ContextPtr renamed_parts.tryRenameAll(); + String replica_name = getReplicaName(); + String zookeeper_name = getZooKeeperName(); + String zookeeper_path = getZooKeeperPath(); + for (auto & [old_name, new_name] : renamed_parts.old_and_new_names) { const auto & [path, disk] = renamed_parts.old_part_name_to_path_and_disk[old_name]; - disk->removeRecursive(fs::path(path) / "detached" / new_name / ""); - LOG_DEBUG(log, "Dropped detached part {}", old_name); + bool keep_shared = removeSharedDetachedPart(disk, fs::path(path) / "detached" / new_name / "", old_name, + zookeeper_name, replica_name, zookeeper_path); + LOG_DEBUG(log, "Dropped detached part {}, keep shared data: {}", old_name, keep_shared); old_name.clear(); } } @@ -5288,6 +5298,63 @@ PartitionCommandsResultInfo MergeTreeData::unfreezeAll( return unfreezePartitionsByMatcher([] (const String &) { return true; }, backup_name, local_context); } +bool MergeTreeData::removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name) +{ + bool keep_shared = false; + + if (disk->supportZeroCopyReplication()) + { + FreezeMetaData meta; + if (meta.load(disk, path) && meta.is_replicated) + return removeSharedDetachedPart(disk, path, part_name, meta.zookeeper_name, meta.replica_name, ""); + } + + disk->removeSharedRecursive(path, keep_shared); + + return keep_shared; +} + +bool MergeTreeData::removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, + const String & zookeeper_name, const String & replica_name, const String & zookeeper_path) +{ + bool keep_shared = false; + + if (disk->supportZeroCopyReplication()) + { + zkutil::ZooKeeperPtr zookeeper; + if (zookeeper_name == "default") + { + zookeeper = getContext()->getZooKeeper(); + } + else + { + zookeeper = getContext()->getAuxiliaryZooKeeper(zookeeper_name); + } + + if (zookeeper) + { + fs::path checksums = fs::path(path) / "checksums.txt"; + if (disk->exists(checksums)) + { + auto ref_count = disk->getRefCount(checksums); + if (ref_count == 0) + { + String id = disk->getUniqueId(checksums); + keep_shared = !StorageReplicatedMergeTree::unlockSharedDataById(id, part_name, + replica_name, disk, zookeeper, getContext()->getReplicatedMergeTreeSettings(), log, + zookeeper_path); + } + else + keep_shared = true; + } + } + } + + disk->removeSharedRecursive(path, keep_shared); + + return keep_shared; +} + PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn matcher, const String & backup_name, ContextPtr) { auto backup_path = fs::path("shadow") / escapeForFileName(backup_name) / relative_data_path; @@ -5316,42 +5383,7 @@ PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn const auto & path = it->path(); - bool keep_shared = false; - - if (disk->supportZeroCopyReplication()) - { - FreezeMetaData meta; - if (meta.load(disk, path) && meta.is_replicated) - { - zkutil::ZooKeeperPtr zookeeper; - if (meta.zookeeper_name == "default") - { - zookeeper = getContext()->getZooKeeper(); - } - else - { - zookeeper = getContext()->getAuxiliaryZooKeeper(meta.zookeeper_name); - } - - if (zookeeper) - { - fs::path checksums = fs::path(path) / "checksums.txt"; - if (disk->exists(checksums)) - { - auto ref_count = disk->getRefCount(checksums); - if (ref_count == 0) - { - String id = disk->getUniqueId(checksums); - keep_shared = !StorageReplicatedMergeTree::unlockSharedDataById(id, partition_directory, - meta.replica_name, disk, zookeeper, getContext()->getReplicatedMergeTreeSettings(), log, - nullptr); - } - } - } - } - } - - disk->removeSharedRecursive(path, keep_shared); + bool keep_shared = removeSharedDetachedPart(disk, path, partition_directory); result.push_back(PartitionCommandResultInfo{ .partition_id = partition_id, @@ -5361,7 +5393,7 @@ PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn .backup_name = backup_name, }); - LOG_DEBUG(log, "Unfreezed part by path {}, keep shared data {}", disk->getPath() + path, keep_shared); + LOG_DEBUG(log, "Unfreezed part by path {}, keep shared data: {}", disk->getPath() + path, keep_shared); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index ed9a7f058dd..6a6f616484d 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -867,11 +867,15 @@ public: /// Overridden in StorageReplicatedMergeTree virtual bool unlockSharedData(const IMergeTreeDataPart &) const { return true; } + /// Unlock same part with other (old) name + virtual bool unlockSharedData(const IMergeTreeDataPart &, const String &) const { return true; } + /// Fetch part only if some replica has it on shared storage like S3 /// Overridden in StorageReplicatedMergeTree virtual bool tryToFetchIfShared(const IMergeTreeDataPart &, const DiskPtr &, const String &) { return false; } virtual String getZooKeeperName() const { return ""; } + virtual String getZooKeeperPath() const { return ""; } /// Parts that currently submerging (merging to bigger parts) or emerging /// (to be appeared after merging finished). These two variables have to be used @@ -1174,6 +1178,12 @@ private: DataPartsVector & duplicate_parts_to_remove, MutableDataPartsVector & parts_from_wal, DataPartsLock & part_lock); + + /// Check shared data usage on other replicas for detached/freezed part + /// Remove local files and remote files if needed + bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name); + bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, + const String & zookeeper_name, const String & replica_name, const String & zookeeper_path); }; /// RAII struct to record big parts that are submerging or emerging. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index e3ca902b1bd..5027b861e18 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -222,6 +222,8 @@ void ReplicatedMergeTreeSink::commitPart( bool is_already_existing_part = false; + String old_part_name = part->name; + while (true) { /// Obtain incremental block number and lock it. The lock holds our intention to add the block to the filesystem. @@ -502,6 +504,8 @@ void ReplicatedMergeTreeSink::commitPart( waitForQuorum(zookeeper, part->name, quorum_info.status_path, quorum_info.is_active_node_value); } + + part->cleanupOldName(old_part_name); } void ReplicatedMergeTreeSink::onStart() diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d989c4b2030..572b5d7b4b9 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4111,13 +4111,8 @@ void StorageReplicatedMergeTree::startup() getContext()->getInterserverIOHandler().addEndpoint(data_parts_exchange_ptr->getId(replica_path), data_parts_exchange_ptr); convertZeroCopySchema(); - is_zero_copy_in_compatible_mode = isZeroCopySchemaInCompatibleMode(); - - if (!is_zero_copy_in_compatible_mode) - { /// All replicas have new version - cleanupOldZeroCopySchema(); - } + cleanupOldZeroCopySchema(); /// In this thread replica will be activated. restarting_thread.start(); @@ -7144,9 +7139,7 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part) String id = part.getUniqueId(); boost::replace_all(id, "/", "_"); - const String * zookeeper_path_ptr = is_zero_copy_in_compatible_mode ? &zookeeper_path : nullptr; - - Strings zc_zookeeper_paths = getZeroCopyRootPath(*getDefaultSettings(), zookeeper_path_ptr); + Strings zc_zookeeper_paths = getZeroCopyRootPath(*getDefaultSettings(), is_zero_copy_in_compatible_mode ? zookeeper_path : ""); for (const auto & zc_zookeeper_path : zc_zookeeper_paths) { String zookeeper_node = fs::path(zc_zookeeper_path) / zero_copy / "shared" / part.name / id / replica_name; @@ -7158,6 +7151,12 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part) bool StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part) const +{ + return unlockSharedData(part, part.name); +} + + +bool StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part, const String & name) const { if (!part.volume) return true; @@ -7170,25 +7169,23 @@ bool StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & par return true; auto ref_count = part.getRefCount(); - LOG_TRACE(log, "RefCount {} for part {}", ref_count, part.name); if (ref_count > 0) /// Keep part shard info for frozen backups return false; - const String * zookeeper_path_ptr = is_zero_copy_in_compatible_mode ? &zookeeper_path : nullptr; - - return unlockSharedDataById(part.getUniqueId(), part.name, replica_name, disk, zookeeper, *getDefaultSettings(), log, zookeeper_path_ptr); + return unlockSharedDataById(part.getUniqueId(), name, replica_name, disk, zookeeper, *getDefaultSettings(), log, + is_zero_copy_in_compatible_mode ? zookeeper_path : String("")); } bool StorageReplicatedMergeTree::unlockSharedDataById(String id, const String & part_name, - const String & replica_name_, DiskPtr disk, zkutil::ZooKeeperPtr zookeeper_, const MergeTreeSettings & settings, - Poco::Logger * logger, const String * zookeeper_path_ptr) + const String & replica_name_, DiskPtr disk, zkutil::ZooKeeperPtr zookeeper_ptr, const MergeTreeSettings & settings, + Poco::Logger * logger, const String & zookeeper_path_old) { boost::replace_all(id, "/", "_"); String zero_copy = fmt::format("zero_copy_{}", toString(disk->getType())); - Strings zc_zookeeper_paths = getZeroCopyRootPath(settings, zookeeper_path_ptr); + Strings zc_zookeeper_paths = getZeroCopyRootPath(settings, zookeeper_path_old); bool res = true; @@ -7200,10 +7197,10 @@ bool StorageReplicatedMergeTree::unlockSharedDataById(String id, const String & LOG_TRACE(logger, "Remove zookeeper lock {}", zookeeper_node); - zookeeper_->tryRemove(zookeeper_node); + zookeeper_ptr->tryRemove(zookeeper_node); Strings children; - zookeeper_->tryGetChildren(zookeeper_part_uniq_node, children); + zookeeper_ptr->tryGetChildren(zookeeper_part_uniq_node, children); if (!children.empty()) { @@ -7212,14 +7209,16 @@ bool StorageReplicatedMergeTree::unlockSharedDataById(String id, const String & continue; } - zookeeper_->tryRemove(zookeeper_part_uniq_node); + zookeeper_ptr->tryRemove(zookeeper_part_uniq_node); /// Even when we have lock with same part name, but with different uniq, we can remove files on S3 children.clear(); - zookeeper_->tryGetChildren(zookeeper_part_node, children); + zookeeper_ptr->tryGetChildren(zookeeper_part_node, children); if (children.empty()) + { /// Cleanup after last uniq removing - zookeeper_->tryRemove(zookeeper_part_node); + zookeeper_ptr->tryRemove(zookeeper_part_node); + } } return res; @@ -7257,9 +7256,7 @@ String StorageReplicatedMergeTree::getSharedDataReplica( String zero_copy = fmt::format("zero_copy_{}", toString(disk_type)); - const String * zookeeper_path_ptr = is_zero_copy_in_compatible_mode ? &zookeeper_path : nullptr; - - Strings zc_zookeeper_paths = getZeroCopyRootPath(*getDefaultSettings(), zookeeper_path_ptr); + Strings zc_zookeeper_paths = getZeroCopyRootPath(*getDefaultSettings(), is_zero_copy_in_compatible_mode ? zookeeper_path : ""); std::set replicas; @@ -7331,30 +7328,30 @@ String StorageReplicatedMergeTree::getSharedDataReplica( } -Strings StorageReplicatedMergeTree::getZeroCopyRootPath(const MergeTreeSettings & settings, const String * zookeeper_path_ptr) +Strings StorageReplicatedMergeTree::getZeroCopyRootPath(const MergeTreeSettings & settings, const String & zookeeper_path_old) { Strings res; res.push_back(settings.remote_fs_zero_copy_zookeeper_path); - if (zookeeper_path_ptr) /// Compatibility mode for cluster with old and new versions - res.push_back(*zookeeper_path_ptr); + if (!zookeeper_path_old.empty()) /// Compatibility mode for cluster with old and new versions + res.push_back(zookeeper_path_old); return res; } String StorageReplicatedMergeTree::findReplicaHavingPart( - const String & part_name, const String & zookeeper_path_, zkutil::ZooKeeper::Ptr zookeeper_) + const String & part_name, const String & zookeeper_path_, zkutil::ZooKeeper::Ptr zookeeper_ptr) { - Strings replicas = zookeeper_->getChildren(fs::path(zookeeper_path_) / "replicas"); + Strings replicas = zookeeper_ptr->getChildren(fs::path(zookeeper_path_) / "replicas"); /// Select replicas in uniformly random order. std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); for (const String & replica : replicas) { - if (zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica / "parts" / part_name) - && zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica / "is_active")) + if (zookeeper_ptr->exists(fs::path(zookeeper_path_) / "replicas" / replica / "parts" / part_name) + && zookeeper_ptr->exists(fs::path(zookeeper_path_) / "replicas" / replica / "is_active")) return fs::path(zookeeper_path_) / "replicas" / replica; } @@ -7594,6 +7591,8 @@ void StorageReplicatedMergeTree::convertZeroCopySchema() LOG_INFO(log, "Convert zero_copy version from {} to {} for {}", zero_copy_version, required_zero_copy_version, version_path.string()); + unsigned long converted_part_counter = 0; + for (auto const & disk_type : disk_types) { String zero_copy = fmt::format("zero_copy_{}", disk_type); @@ -7605,18 +7604,26 @@ void StorageReplicatedMergeTree::convertZeroCopySchema() auto old_shard_root = revert_to_version == 1 ? shard_root_v2 : shard_root_v1; auto new_shard_root = revert_to_version == 1 ? shard_root_v1 : shard_root_v2; - Coordination::Stat stat; - Strings parts = current_zookeeper->getChildren(old_shard_root, &stat); + Strings parts; + current_zookeeper->tryGetChildren(old_shard_root, parts); - for (const auto & part : parts) + for (const auto & part_name : parts) { - Strings ids = current_zookeeper->getChildren(old_shard_root / part, &stat); - for (const auto & id : ids) - { - if (current_zookeeper->exists(old_shard_root / part / id / replica_name)) + auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); + auto part = getPartIfExists(part_info, {MergeTreeDataPartState::Committed}); + + if (part) + { /// Do not move lost locks + Strings ids; + current_zookeeper->tryGetChildren(old_shard_root / part_name, ids); + for (const auto & id : ids) { - auto zookeeper_node = new_shard_root / part / id / replica_name; - createZeroCopyLockNode(current_zookeeper, zookeeper_node.string()); + if (current_zookeeper->exists(old_shard_root / part_name / id / replica_name)) + { + auto zookeeper_node = new_shard_root / part_name / id / replica_name; + createZeroCopyLockNode(current_zookeeper, zookeeper_node.string()); + ++converted_part_counter; + } } } } @@ -7630,13 +7637,16 @@ void StorageReplicatedMergeTree::convertZeroCopySchema() current_zookeeper->createOrUpdate(version_path / "cleanup_required", std::to_string(zero_copy_version), zkutil::CreateMode::Persistent); - LOG_INFO(log, "Convert zero_copy version from {} to {} for {} complete", zero_copy_version, required_zero_copy_version, - version_path.string()); + LOG_INFO(log, "Convert zero_copy version from {} to {} for {} complete, converted {} locks", zero_copy_version, required_zero_copy_version, + version_path.string(), converted_part_counter); } void StorageReplicatedMergeTree::cleanupOldZeroCopySchema() { + if (is_zero_copy_in_compatible_mode) + return; /// Some replicas have old version + if (!current_zookeeper) return; @@ -7672,17 +7682,7 @@ void StorageReplicatedMergeTree::cleanupOldZeroCopySchema() auto old_shard_root = fs::path(zookeeper_path) / zero_copy / "shared"; - Coordination::Stat stat; - Strings parts = current_zookeeper->getChildren(old_shard_root, &stat); - - for (const auto & part : parts) - { - Strings ids = current_zookeeper->getChildren(old_shard_root / part, &stat); - for (const auto & id : ids) - { - current_zookeeper->remove(old_shard_root / part / id / replica_name); - } - } + current_zookeeper->tryRemoveRecursive(old_shard_root); } current_zookeeper->remove(old_version_path); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 7fd1febc9a6..b6b6076206e 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -237,12 +237,15 @@ public: /// Return false if data is still used by another node bool unlockSharedData(const IMergeTreeDataPart & part) const override; + /// Unlock same part with other (old) name + bool unlockSharedData(const IMergeTreeDataPart & part, const String & name) const override; + /// Unlock shared data part in zookeeper by part id /// Return true if data unlocked /// Return false if data is still used by another node static bool unlockSharedDataById(String id, const String & part_name, const String & replica_name_, DiskPtr disk, zkutil::ZooKeeperPtr zookeeper_, const MergeTreeSettings & settings, Poco::Logger * logger, - const String * zookeeper_path_ptr); + const String & zookeeper_path_old); /// Fetch part only if some replica has it on shared storage like S3 bool tryToFetchIfShared(const IMergeTreeDataPart & part, const DiskPtr & disk, const String & path) override; @@ -272,6 +275,7 @@ public: bool createEmptyPartInsteadOfLost(zkutil::ZooKeeperPtr zookeeper, const String & lost_part_name); virtual String getZooKeeperName() const override { return zookeeper_name; } + virtual String getZooKeeperPath() const override { return zookeeper_path; } private: std::atomic_bool are_restoring_replica {false}; @@ -737,7 +741,7 @@ private: PartitionBlockNumbersHolder allocateBlockNumbersInAffectedPartitions( const MutationCommands & commands, ContextPtr query_context, const zkutil::ZooKeeperPtr & zookeeper) const; - static Strings getZeroCopyRootPath(const MergeTreeSettings & settings, const String * zookeeper_path_ptr = nullptr); + static Strings getZeroCopyRootPath(const MergeTreeSettings & settings, const String & zookeeper_path_old); /// Upgrave zero-copy version /// version 1 - lock for shared part inside table node in ZooKeeper From c8fe1dc4a78a0f71c40942955820d4f2268d972a Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Wed, 1 Dec 2021 16:12:40 +0300 Subject: [PATCH 0112/1260] Add tests for new zero-copy schema --- .../test_s3_zero_copy_replication/test.py | 99 ++++++++- .../test_s3_zero_copy_upgrade/__init__.py | 0 .../configs/config.d/s3.xml | 50 +++++ .../test_s3_zero_copy_upgrade/test.py | 198 ++++++++++++++++++ 4 files changed, 339 insertions(+), 8 deletions(-) create mode 100644 tests/integration/test_s3_zero_copy_upgrade/__init__.py create mode 100644 tests/integration/test_s3_zero_copy_upgrade/configs/config.d/s3.xml create mode 100644 tests/integration/test_s3_zero_copy_upgrade/test.py diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index 996b15dece6..160099a321d 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -269,6 +269,17 @@ def test_s3_zero_copy_with_ttl_delete(cluster, large_data, iterations): node2.query("DROP TABLE IF EXISTS ttl_delete_test NO DELAY") +def wait_mutations(node, table, seconds): + while seconds > 0: + seconds -= 1 + mutations = node.query(f"SELECT count() FROM system.mutations WHERE table='{table}' AND is_done=0") + if mutations == '0\n': + return + time.sleep(1) + mutations = node.query(f"SELECT count() FROM system.mutations WHERE table='{table}' AND is_done=0") + assert mutations == '0\n' + + def test_s3_zero_copy_unfreeze(cluster): node1 = cluster.instances["node1"] node2 = cluster.instances["node2"] @@ -288,11 +299,13 @@ def test_s3_zero_copy_unfreeze(cluster): node1.query("INSERT INTO unfreeze_test VALUES (0)") - node1.query("ALTER TABLE unfreeze_test FREEZE WITH NAME 'backup1'") - node2.query("ALTER TABLE unfreeze_test FREEZE WITH NAME 'backup2'") + node1.query("ALTER TABLE unfreeze_test FREEZE WITH NAME 'freeze_backup1'") + node2.query("ALTER TABLE unfreeze_test FREEZE WITH NAME 'freeze_backup2'") + wait_mutations(node1, "unfreeze_test", 10) + wait_mutations(node2, "unfreeze_test", 10) - objects01 = node1.get_backuped_s3_objects("s31", "backup1") - objects02 = node2.get_backuped_s3_objects("s31", "backup2") + objects01 = node1.get_backuped_s3_objects("s31", "freeze_backup1") + objects02 = node2.get_backuped_s3_objects("s31", "freeze_backup2") assert objects01 == objects02 @@ -300,18 +313,88 @@ def test_s3_zero_copy_unfreeze(cluster): node1.query("TRUNCATE TABLE unfreeze_test") - objects11 = node1.get_backuped_s3_objects("s31", "backup1") - objects12 = node2.get_backuped_s3_objects("s31", "backup2") + objects11 = node1.get_backuped_s3_objects("s31", "freeze_backup1") + objects12 = node2.get_backuped_s3_objects("s31", "freeze_backup2") assert objects01 == objects11 assert objects01 == objects12 check_objects_exisis(cluster, objects11) - node1.query("ALTER TABLE unfreeze_test UNFREEZE WITH NAME 'backup1'") + node1.query("ALTER TABLE unfreeze_test UNFREEZE WITH NAME 'freeze_backup1'") + wait_mutations(node1, "unfreeze_test", 10) check_objects_exisis(cluster, objects12) - node2.query("ALTER TABLE unfreeze_test UNFREEZE WITH NAME 'backup2'") + node2.query("ALTER TABLE unfreeze_test UNFREEZE WITH NAME 'freeze_backup2'") + wait_mutations(node2, "unfreeze_test", 10) check_objects_not_exisis(cluster, objects12) + + node1.query("DROP TABLE IF EXISTS unfreeze_test NO DELAY") + node2.query("DROP TABLE IF EXISTS unfreeze_test NO DELAY") + + +def test_s3_zero_copy_drop_detached(cluster): + node1 = cluster.instances["node1"] + node2 = cluster.instances["node2"] + + node1.query("DROP TABLE IF EXISTS drop_detached_test NO DELAY") + node2.query("DROP TABLE IF EXISTS drop_detached_test NO DELAY") + + node1.query( + """ + CREATE TABLE drop_detached_test ON CLUSTER test_cluster (d UInt64) + ENGINE=ReplicatedMergeTree('/clickhouse/tables/drop_detached_test', '{}') + ORDER BY d PARTITION BY d + SETTINGS storage_policy='s3' + """ + .format('{replica}') + ) + + node1.query("INSERT INTO drop_detached_test VALUES (0)") + node1.query("ALTER TABLE drop_detached_test FREEZE WITH NAME 'detach_backup1'") + node1.query("INSERT INTO drop_detached_test VALUES (1)") + node1.query("ALTER TABLE drop_detached_test FREEZE WITH NAME 'detach_backup2'") + + objects1 = node1.get_backuped_s3_objects("s31", "detach_backup1") + objects2 = node1.get_backuped_s3_objects("s31", "detach_backup2") + + objects_diff = list(set(objects2) - set(objects1)) + + node1.query("ALTER TABLE drop_detached_test UNFREEZE WITH NAME 'detach_backup2'") + node1.query("ALTER TABLE drop_detached_test UNFREEZE WITH NAME 'detach_backup1'") + + node1.query("ALTER TABLE drop_detached_test DETACH PARTITION '0'") + node1.query("ALTER TABLE drop_detached_test DETACH PARTITION '1'") + wait_mutations(node1, "drop_detached_test", 10) + wait_mutations(node2, "drop_detached_test", 10) + + check_objects_exisis(cluster, objects1) + check_objects_exisis(cluster, objects2) + + node2.query("ALTER TABLE drop_detached_test DROP DETACHED PARTITION '1'", settings={"allow_drop_detached": 1}) + wait_mutations(node1, "drop_detached_test", 10) + wait_mutations(node2, "drop_detached_test", 10) + + check_objects_exisis(cluster, objects1) + check_objects_exisis(cluster, objects2) + + node1.query("ALTER TABLE drop_detached_test DROP DETACHED PARTITION '1'", settings={"allow_drop_detached": 1}) + wait_mutations(node1, "drop_detached_test", 10) + wait_mutations(node2, "drop_detached_test", 10) + + check_objects_exisis(cluster, objects1) + check_objects_not_exisis(cluster, objects_diff) + + node1.query("ALTER TABLE drop_detached_test DROP DETACHED PARTITION '0'", settings={"allow_drop_detached": 1}) + wait_mutations(node1, "drop_detached_test", 10) + wait_mutations(node2, "drop_detached_test", 10) + + check_objects_exisis(cluster, objects1) + + node2.query("ALTER TABLE drop_detached_test DROP DETACHED PARTITION '0'", settings={"allow_drop_detached": 1}) + wait_mutations(node1, "drop_detached_test", 10) + wait_mutations(node2, "drop_detached_test", 10) + + check_objects_not_exisis(cluster, objects1) diff --git a/tests/integration/test_s3_zero_copy_upgrade/__init__.py b/tests/integration/test_s3_zero_copy_upgrade/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_s3_zero_copy_upgrade/configs/config.d/s3.xml b/tests/integration/test_s3_zero_copy_upgrade/configs/config.d/s3.xml new file mode 100644 index 00000000000..cbff39c23ce --- /dev/null +++ b/tests/integration/test_s3_zero_copy_upgrade/configs/config.d/s3.xml @@ -0,0 +1,50 @@ + + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + true + + + + + +
+ s3 +
+
+
+
+
+ + + 1024 + 1 + + + + + + + node1 + 9000 + + + + + node2 + 9000 + + + + + + + test_cluster + + +
diff --git a/tests/integration/test_s3_zero_copy_upgrade/test.py b/tests/integration/test_s3_zero_copy_upgrade/test.py new file mode 100644 index 00000000000..08f39b98787 --- /dev/null +++ b/tests/integration/test_s3_zero_copy_upgrade/test.py @@ -0,0 +1,198 @@ +import logging +import time +import kazoo + +import pytest +from helpers.cluster import ClickHouseCluster + +logging.getLogger().setLevel(logging.INFO) +logging.getLogger().addHandler(logging.StreamHandler()) + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance("node1", main_configs=["configs/config.d/s3.xml"], macros={'replica': '1'}, + image='yandex/clickhouse-server', tag='21.11.4.14', + stay_alive=True, with_installed_binary=True, + with_minio=True, + with_zookeeper=True) + cluster.add_instance("node2", main_configs=["configs/config.d/s3.xml"], macros={'replica': '2'}, + image='yandex/clickhouse-server', tag='21.11.4.14', + stay_alive=True, with_installed_binary=True, + with_minio=True, + with_zookeeper=True) + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +def get_large_objects_count(cluster, size=100, folder='data'): + minio = cluster.minio_client + counter = 0 + for obj in minio.list_objects(cluster.minio_bucket, '{}/'.format(folder)): + if obj.size is not None and obj.size >= size: + counter = counter + 1 + return counter + + +def check_objects_exisis(cluster, object_list, folder='data'): + minio = cluster.minio_client + for obj in object_list: + if obj: + minio.stat_object(cluster.minio_bucket, '{}/{}'.format(folder, obj)) + + +def check_objects_not_exisis(cluster, object_list, folder='data'): + minio = cluster.minio_client + for obj in object_list: + if obj: + try: + minio.stat_object(cluster.minio_bucket, '{}/{}'.format(folder, obj)) + except Exception as error: + assert "NoSuchKey" in str(error) + else: + assert False, "Object {} should not be exists".format(obj) + + +def wait_for_large_objects_count(cluster, expected, size=100, timeout=30): + while timeout > 0: + if get_large_objects_count(cluster, size=size) == expected: + return + timeout -= 1 + time.sleep(1) + assert get_large_objects_count(cluster, size=size) == expected + + +def wait_for_count_in_table(node, table, count, seconds): + while seconds > 0: + seconds -= 1 + res = node.query(f"SELECT count() FROM {table}") + if res == f"{count}\n": + return + time.sleep(1) + res = node.query(f"SELECT count() FROM {table}") + assert res == f"{count}\n" + + +def get_ids(zookeeper, zk_path): + ids = [] + + try: + zk_nodes = zookeeper.get_children(zk_path) + + for zk_node in zk_nodes: + part_ids = zookeeper.get_children(zk_path + "/" + zk_node) + assert len(part_ids) == 1 + ids += part_ids + except kazoo.exceptions.NoNodeError: + ids = [] + pass + + ids = list(set(ids)) + ids.sort() + return ids + + +def wait_mutations(node, table, seconds): + while seconds > 0: + seconds -= 1 + mutations = node.query(f"SELECT count() FROM system.mutations WHERE table='{table}' AND is_done=0") + if mutations == '0\n': + return + time.sleep(1) + mutations = node.query(f"SELECT count() FROM system.mutations WHERE table='{table}' AND is_done=0") + assert mutations == '0\n' + + +def test_s3_zero_copy_version_upgrade(cluster): + node1 = cluster.instances["node1"] + node2 = cluster.instances["node2"] + + zookeeper = cluster.get_kazoo_client("zoo1") + + node1.query("DROP TABLE IF EXISTS convert_test NO DELAY") + node2.query("DROP TABLE IF EXISTS convert_test NO DELAY") + + node1.query( + """ + CREATE TABLE convert_test ON CLUSTER test_cluster (d String) + ENGINE=ReplicatedMergeTree('/clickhouse/tables/convert_test', '{}') + ORDER BY d + PARTITION BY d + SETTINGS storage_policy='s3' + """ + .format('{replica}') + ) + + node1.query("INSERT INTO convert_test VALUES ('convert_part_1'),('convert_part_2'),('convert_part_3')") + wait_for_count_in_table(node2, "convert_test", 3, 10) + + zk_old_path = "/clickhouse/tables/convert_test/zero_copy_s3/shared" + zk_path = "/clickhouse/zero_copy/zero_copy_s3/shared" + + part_ids = get_ids(zookeeper, zk_old_path) + assert len(part_ids) == 3 + + ids = get_ids(zookeeper, zk_path) + assert len(ids) == 0 + + node1.restart_with_latest_version() + ids = get_ids(zookeeper, zk_path) + assert ids == part_ids + old_ids = get_ids(zookeeper, zk_old_path) + assert old_ids == part_ids + + node1.restart_clickhouse() + ids = get_ids(zookeeper, zk_path) + assert ids == part_ids + old_ids = get_ids(zookeeper, zk_old_path) + assert old_ids == part_ids + + node1.query("INSERT INTO convert_test VALUES ('convert_part_4')") + wait_for_count_in_table(node1, "convert_test", 4, 10) + wait_for_count_in_table(node2, "convert_test", 4, 10) + node2.query("INSERT INTO convert_test VALUES ('convert_part_5')") + wait_for_count_in_table(node1, "convert_test", 5, 10) + wait_for_count_in_table(node2, "convert_test", 5, 10) + + part_ids = get_ids(zookeeper, zk_path) + assert len(part_ids) == 5 + old_ids = get_ids(zookeeper, zk_old_path) + assert old_ids == part_ids + + node1.query("ALTER TABLE convert_test DETACH PARTITION 'convert_part_1'") + node1.query("ALTER TABLE convert_test DROP DETACHED PARTITION 'convert_part_1'", settings={"allow_drop_detached": 1}) + wait_for_count_in_table(node1, "convert_test", 4, 10) + wait_for_count_in_table(node2, "convert_test", 4, 10) + node2.query("ALTER TABLE convert_test DETACH PARTITION 'convert_part_2'") + node2.query("ALTER TABLE convert_test DROP DETACHED PARTITION 'convert_part_2'", settings={"allow_drop_detached": 1}) + wait_for_count_in_table(node1, "convert_test", 3, 10) + wait_for_count_in_table(node2, "convert_test", 3, 10) + wait_mutations(node1, "convert_test", 10) + wait_mutations(node2, "convert_test", 10) + + part_ids = get_ids(zookeeper, zk_path) + assert len(part_ids) == 4 + + node1.query("ALTER TABLE convert_test DROP DETACHED PARTITION 'convert_part_2'", settings={"allow_drop_detached": 1}) + wait_mutations(node1, "convert_test", 10) + + part_ids = get_ids(zookeeper, zk_path) + assert len(part_ids) == 3 + + node2.restart_with_latest_version() + ids = get_ids(zookeeper, zk_path) + assert ids == part_ids + old_ids = get_ids(zookeeper, zk_old_path) + assert len(old_ids) == 0 + + node1.query("DROP TABLE IF EXISTS convert_test NO DELAY") + node2.query("DROP TABLE IF EXISTS convert_test NO DELAY") + + zookeeper.stop() From 0e685c1aba487a78b411fd06b611e71ac4f30f93 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Wed, 1 Dec 2021 19:11:17 +0300 Subject: [PATCH 0113/1260] Fix types --- src/Disks/IDisk.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 34e5a229c0a..a04429e0acb 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -248,7 +248,7 @@ public: virtual void applyNewSettings(const Poco::Util::AbstractConfiguration &, ContextPtr, const String &, const DisksMap &) {} /// Open the local file for read and return ReadBufferFromFileBase object. - /// Overriden in IDiskRemote. + /// Overridden in IDiskRemote. /// Used for work with custom metadata. virtual std::unique_ptr readMetaFile( const String & path, @@ -256,7 +256,7 @@ public: std::optional size = {}) const; /// Open the local file for write and return WriteBufferFromFileBase object. - /// Overriden in IDiskRemote. + /// Overridden in IDiskRemote. /// Used for work with custom metadata. virtual std::unique_ptr writeMetaFile( const String & path, @@ -264,7 +264,7 @@ public: WriteMode mode = WriteMode::Rewrite); /// Return reference count for remote FS. - /// Overriden in IDiskRemote. + /// Overridden in IDiskRemote. virtual UInt32 getRefCount(const String &) const { return 0; } protected: From 9ec8272186156d0629d8112a51c803ab954bd68f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 2 Dec 2021 16:14:25 +0800 Subject: [PATCH 0114/1260] refactor hive text input format --- src/Core/Settings.h | 6 ++ src/Formats/FormatFactory.cpp | 6 ++ src/Formats/FormatSettings.h | 13 ++- .../Formats/Impl/ArrowBlockInputFormat.cpp | 13 +-- .../Formats/Impl/ArrowBlockInputFormat.h | 1 + .../Formats/Impl/ArrowBufferedStreams.cpp | 4 - .../Formats/Impl/ArrowColumnToCHColumn.cpp | 86 ++++++++++++------- .../Formats/Impl/ArrowColumnToCHColumn.h | 22 +++-- .../Formats/Impl/HiveTextRowInputFormat.cpp | 29 ++----- .../Formats/Impl/HiveTextRowInputFormat.h | 7 +- .../Formats/Impl/ORCBlockInputFormat.cpp | 13 +-- .../Formats/Impl/ORCBlockInputFormat.h | 1 + .../Formats/Impl/ParquetBlockInputFormat.cpp | 13 +-- .../Formats/Impl/ParquetBlockInputFormat.h | 1 + src/Storages/Hive/StorageHive.cpp | 7 +- 15 files changed, 138 insertions(+), 84 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f6035f66df3..4728f864343 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -585,6 +585,12 @@ class IColumn; M(Bool, input_format_orc_import_nested, false, "Allow to insert array of structs into Nested table in ORC input format.", 0) \ M(Bool, input_format_parquet_import_nested, false, "Allow to insert array of structs into Nested table in Parquet input format.", 0) \ M(Bool, input_format_allow_seeks, true, "Allow seeks while reading in ORC/Parquet/Arrow input formats", 0) \ + M(Bool, input_format_orc_allow_missing_columns, false, "Allow missing columns while reading ORC input formats", 0) \ + M(Bool, input_format_parquet_allow_missing_columns, false, "Allow missing columns while reading Parquet input formats", 0) \ + M(Bool, input_format_arrow_allow_missing_columns, false, "Allow missing columns while reading Arrow input formats", 0) \ + M(Char, input_format_hive_text_fields_delimiter, '\x01', "Delimiter between fields in Hive Text File", 0) \ + M(Char, input_format_hive_text_collection_items_delimiter, '\x02', "Delimiter between collection(array or map) items in Hive Text File", 0) \ + M(Char, input_format_hive_text_map_keys_delimiter, '\x03', "Delimiter between a pair of map key/values in Hive Text File", 0) \ \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 8e490fac301..d798be69150 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -61,6 +61,9 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.input_format_enum_as_number = settings.input_format_csv_enum_as_number; format_settings.csv.null_representation = settings.format_csv_null_representation; format_settings.csv.input_format_arrays_as_nested_csv = settings.input_format_csv_arrays_as_nested_csv; + format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; + format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; + format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; format_settings.custom.escaping_rule = settings.format_custom_escaping_rule; format_settings.custom.field_delimiter = settings.format_custom_field_delimiter; format_settings.custom.result_after_delimiter = settings.format_custom_result_after_delimiter; @@ -83,6 +86,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.decimal_trailing_zeros = settings.output_format_decimal_trailing_zeros; format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size; format_settings.parquet.import_nested = settings.input_format_parquet_import_nested; + format_settings.parquet.allow_missing_columns = settings.input_format_parquet_allow_missing_columns; format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? FormatSettings::Pretty::Charset::ASCII : FormatSettings::Pretty::Charset::UTF8; format_settings.pretty.color = settings.output_format_pretty_color; format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width; @@ -111,7 +115,9 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.write_statistics = settings.output_format_write_statistics; format_settings.arrow.low_cardinality_as_dictionary = settings.output_format_arrow_low_cardinality_as_dictionary; format_settings.arrow.import_nested = settings.input_format_arrow_import_nested; + format_settings.arrow.allow_missing_columns = settings.input_format_arrow_allow_missing_columns; format_settings.orc.import_nested = settings.input_format_orc_import_nested; + format_settings.orc.allow_missing_columns = settings.input_format_orc_allow_missing_columns; format_settings.defaults_for_omitted_fields = settings.input_format_defaults_for_omitted_fields; format_settings.capn_proto.enum_comparing_mode = settings.format_capn_proto_enum_comparising_mode; format_settings.seekable_read = settings.input_format_allow_seeks; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 6f1a403daeb..1aec5e28e87 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -71,6 +71,7 @@ struct FormatSettings UInt64 row_group_size = 1000000; bool low_cardinality_as_dictionary = false; bool import_nested = false; + bool allow_missing_columns = false; } arrow; struct @@ -93,10 +94,18 @@ struct FormatSettings bool input_format_enum_as_number = false; bool input_format_arrays_as_nested_csv = false; bool read_bool_as_uint8 = false; - Names input_field_names; String null_representation = "\\N"; } csv; + struct HiveText + { + char fields_delimiter = '\x01'; + char collection_items_delimiter = '\x02'; + char map_keys_delimiter = '\x03'; + bool read_bool_as_uint8 = true; + Names input_field_names; + } hive_text; + struct Custom { std::string result_before_delimiter; @@ -122,6 +131,7 @@ struct FormatSettings { UInt64 row_group_size = 1000000; bool import_nested = false; + bool allow_missing_columns = false; } parquet; struct Pretty @@ -200,6 +210,7 @@ struct FormatSettings struct { bool import_nested = false; + bool allow_missing_columns = false; } orc; /// For capnProto format we should determine how to diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index b6a43591398..33daa2bd69b 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -64,10 +64,12 @@ Chunk ArrowBlockInputFormat::generate() ++record_batch_current; - auto missing_column_indexes = arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result); - for (size_t row_idx = 0; row_idx < res.getNumRows(); ++row_idx) - for (const auto & column_idx : missing_column_indexes) - block_missing_values.setBit(column_idx, row_idx); + /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. + /// Otherwise fill the missing columns with zero values of its type. + if (format_settings.defaults_for_omitted_fields) + for (size_t row_idx = 0; row_idx < res.getNumRows(); ++row_idx) + for (const auto & column_idx : missing_columns) + block_missing_values.setBit(column_idx, row_idx); return res; } @@ -113,7 +115,8 @@ void ArrowBlockInputFormat::prepareReader() } arrow_column_to_ch_column = std::make_unique( - getPort().getHeader(), "Arrow", format_settings.arrow.import_nested, format_settings.defaults_for_omitted_fields); + getPort().getHeader(), "Arrow", format_settings.arrow.import_nested, format_settings.arrow.allow_missing_columns); + missing_columns = arrow_column_to_ch_column->getMissingColumns(*schema); if (stream) record_batch_total = -1; diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index c0e78ff7e42..a77daf96a52 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -41,6 +41,7 @@ private: int record_batch_total = 0; int record_batch_current = 0; + std::vector missing_columns; BlockMissingValues block_missing_values; const FormatSettings format_settings; diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp index 474071a1b9e..1bbce33eedd 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp @@ -5,7 +5,6 @@ #include "ArrowBufferedStreams.h" #if USE_ARROW || USE_ORC || USE_PARQUET -#include #include #include #include @@ -13,9 +12,6 @@ #include #include #include -#include -#include -#include #include diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index dc25ff19193..a844054b517 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -272,7 +272,7 @@ static ColumnPtr readByteMapFromArrowColumn(std::shared_ptr for (size_t value_i = 0; value_i != static_cast(chunk->length()); ++value_i) bytemap_data.emplace_back(chunk->IsNull(value_i)); } - return nullmap_column; + return std::move(nullmap_column); } static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptr & arrow_column) @@ -290,7 +290,7 @@ static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptr & arrow_column) @@ -505,21 +505,30 @@ static Block arrowSchemaToCHHeader(const arrow::Schema & schema, const std::stri } ArrowColumnToCHColumn::ArrowColumnToCHColumn( - const arrow::Schema & schema, const std::string & format_name_, bool import_nested_, bool defaults_for_omitted_fields_) + const arrow::Schema & schema, + const std::string & format_name_, + bool import_nested_, + bool allow_missing_columns_) : header(arrowSchemaToCHHeader(schema, format_name_)) , format_name(format_name_) , import_nested(import_nested_) - , defaults_for_omitted_fields(defaults_for_omitted_fields_) + , allow_missing_columns(allow_missing_columns_) { } ArrowColumnToCHColumn::ArrowColumnToCHColumn( - const Block & header_, const std::string & format_name_, bool import_nested_, bool defaults_for_omitted_fields_) - : header(header_), format_name(format_name_), import_nested(import_nested_), defaults_for_omitted_fields(defaults_for_omitted_fields_) + const Block & header_, + const std::string & format_name_, + bool import_nested_, + bool allow_missing_columns_) + : header(header_) + , format_name(format_name_) + , import_nested(import_nested_) + , allow_missing_columns(allow_missing_columns_) { } -std::vector ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table) +void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table) { NameToColumnPtr name_to_column_ptr; for (const auto& column_name : table->ColumnNames()) @@ -528,23 +537,15 @@ std::vector ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std: name_to_column_ptr[column_name] = arrow_column; } - return arrowColumnsToCHChunk(res, name_to_column_ptr); + arrowColumnsToCHChunk(res, name_to_column_ptr); } -std::vector ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & name_to_column_ptr) +void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & name_to_column_ptr) { Columns columns_list; - - if (name_to_column_ptr.empty()) - return {}; - UInt64 num_rows = name_to_column_ptr.begin()->second->length(); - columns_list.reserve(header.rows()); - std::unordered_map nested_tables; - std::vector missing_column_indexes; - missing_column_indexes.reserve(header.columns()); for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i) { const ColumnWithTypeAndName & header_column = header.getByPosition(column_i); @@ -569,20 +570,15 @@ std::vector ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, Na if (!read_from_nested) { - missing_column_indexes.push_back(column_i); - if (defaults_for_omitted_fields) - { - ColumnWithTypeAndName column; - column.name = header_column.name; - column.type = header_column.type; - column.column = header_column.column->cloneResized(num_rows); - columns_list.push_back(std::move(column.column)); - continue; - } - else - { + if (!allow_missing_columns) throw Exception{ErrorCodes::THERE_IS_NO_COLUMN, "Column '{}' is not presented in input data.", header_column.name}; - } + + ColumnWithTypeAndName column; + column.name = header_column.name; + column.type = header_column.type; + column.column = header_column.column->cloneResized(num_rows); + columns_list.push_back(std::move(column.column)); + continue; } } @@ -610,7 +606,35 @@ std::vector ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, Na } res.setColumns(columns_list, num_rows); - return missing_column_indexes; } + +std::vector ArrowColumnToCHColumn::getMissingColumns(const arrow::Schema & schema) const +{ + std::vector missing_columns; + auto block_from_arrow = arrowSchemaToCHHeader(schema, format_name); + auto flatten_block_from_arrow = Nested::flatten(block_from_arrow); + for (size_t i = 0, columns = header.columns(); i < columns; ++i) + { + const auto & column = header.getByPosition(i); + bool read_from_nested = false; + String nested_table_name = Nested::extractTableName(column.name); + if (!block_from_arrow.has(column.name)) + { + if (import_nested && block_from_arrow.has(nested_table_name)) + read_from_nested = flatten_block_from_arrow.has(column.name); + + if (!read_from_nested) + { + if (!allow_missing_columns) + throw Exception{ErrorCodes::THERE_IS_NO_COLUMN, "Column '{}' is not presented in input data.", column.name}; + + missing_columns.push_back(i); + } + } + } + return missing_columns; } + +} + #endif diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 761b389ee03..df11fb15c3e 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -21,22 +21,34 @@ class ArrowColumnToCHColumn public: using NameToColumnPtr = std::unordered_map>; - ArrowColumnToCHColumn(const Block & header_, const std::string & format_name_, bool import_nested_, bool defaults_for_omitted_fields_); + ArrowColumnToCHColumn( + const Block & header_, + const std::string & format_name_, + bool import_nested_, + bool allow_missing_columns_); /// Constructor that create header by arrow schema. It will be useful for inserting /// data from file without knowing table structure. - ArrowColumnToCHColumn(const arrow::Schema & schema, const std::string & format_name, bool import_nested_, bool defaults_for_omitted_fields_); + ArrowColumnToCHColumn( + const arrow::Schema & schema, + const std::string & format_name, + bool import_nested_, + bool allow_missing_columns_); /// Convert arrow::Table to chunk. Returns missing header columns not exists in arrow::Table. - std::vector arrowTableToCHChunk(Chunk & res, std::shared_ptr & table); + void arrowTableToCHChunk(Chunk & res, std::shared_ptr & table); - std::vector arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & name_to_column_ptr); + void arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & name_to_column_ptr); + + /// Get missing columns that exists in header but not in arrow::Schema + std::vector getMissingColumns(const arrow::Schema & schema) const; private: const Block header; const std::string format_name; bool import_nested; - bool defaults_for_omitted_fields; + /// If false, throw exception if some columns in header not exists in arrow table. + bool allow_missing_columns; /// Map {column name : dictionary column}. /// To avoid converting dictionary from Arrow Dictionary diff --git a/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp b/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp index 8500962be75..aeeb35a349c 100644 --- a/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp @@ -14,32 +14,16 @@ HiveTextRowInputFormat::HiveTextRowInputFormat( const Block & header_, ReadBuffer & in_, const Params & params_, const FormatSettings & format_settings_) : CSVRowInputFormat(header_, buf, params_, true, false, format_settings_) , buf(in_) + , input_field_names(format_settings_.hive_text.input_field_names) { } -void HiveTextRowInputFormat::readPrefix() -{ - std::vector read_columns(data_types.size(), false); - /// For Hive Text file, read the first row to get exact number of columns. - auto values = readNames(); - input_field_names = format_settings.csv.input_field_names; - input_field_names.resize(values.size()); - for (const auto & column_name : input_field_names) - addInputColumn(column_name, read_columns); - - for (size_t i = 0; i != read_columns.size(); ++i) - { - if (!read_columns[i]) - column_mapping->not_presented_columns.push_back(i); - } -} - std::vector HiveTextRowInputFormat::readNames() { - PeekableReadBufferCheckpoint checkpoint{buf}; + PeekableReadBufferCheckpoint checkpoint{buf, true}; auto values = readHeaderRow(); - buf.rollbackToCheckpoint(); - return values; + input_field_names.resize(values.size()); + return input_field_names; } std::vector HiveTextRowInputFormat::readTypes() @@ -53,7 +37,10 @@ void registerInputFormatHiveText(FormatFactory & factory) "HiveText", [](ReadBuffer & buf, const Block & sample, const RowInputFormatParams & params, const FormatSettings & settings) { - return std::make_shared(sample, buf, params, settings); + FormatSettings settings_copy = settings; + settings_copy.csv.delimiter = settings_copy.hive_text.fields_delimiter; + settings_copy.csv.read_bool_as_uint8 = settings_copy.hive_text.read_bool_as_uint8; + return std::make_shared(sample, buf, params, settings_copy); }); } } diff --git a/src/Processors/Formats/Impl/HiveTextRowInputFormat.h b/src/Processors/Formats/Impl/HiveTextRowInputFormat.h index 4985c6edba2..53bbf3bc427 100644 --- a/src/Processors/Formats/Impl/HiveTextRowInputFormat.h +++ b/src/Processors/Formats/Impl/HiveTextRowInputFormat.h @@ -15,12 +15,15 @@ namespace DB class HiveTextRowInputFormat : public CSVRowInputFormat { public: - HiveTextRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_, const FormatSettings & format_settings_); + HiveTextRowInputFormat( + const Block & header_, + ReadBuffer & in_, + const Params & params_, + const FormatSettings & format_settings_); String getName() const override { return "HiveTextRowInputFormat"; } protected: - void readPrefix() override; std::vector readNames() override; std::vector readTypes() override; diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 55fddfd7693..aef8e5909cf 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -69,10 +69,12 @@ Chunk ORCBlockInputFormat::generate() name_to_column_ptr[column_name] = arrow_column; } - auto missing_column_indexes = arrow_column_to_ch_column->arrowColumnsToCHChunk(res, name_to_column_ptr); - for (size_t row_idx = 0; row_idx < res.getNumRows(); ++row_idx) - for (const auto & column_idx : missing_column_indexes) - block_missing_values.setBit(column_idx, row_idx); + /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. + /// Otherwise fill the missing columns with zero values of its type. + if (format_settings.defaults_for_omitted_fields) + for (size_t row_idx = 0; row_idx < res.getNumRows(); ++row_idx) + for (const auto & column_idx : missing_columns) + block_missing_values.setBit(column_idx, row_idx); batch_reader.reset(); return res; @@ -126,7 +128,8 @@ void ORCBlockInputFormat::prepareReader() THROW_ARROW_NOT_OK(file_reader->ReadSchema(&schema)); arrow_column_to_ch_column = std::make_unique( - getPort().getHeader(), "ORC", format_settings.orc.import_nested, format_settings.defaults_for_omitted_fields); + getPort().getHeader(), "ORC", format_settings.orc.import_nested, format_settings.orc.allow_missing_columns); + missing_columns = arrow_column_to_ch_column->getMissingColumns(*schema); std::unordered_set nested_table_names; if (format_settings.orc.import_nested) diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h index c8747fb8d36..39a24e747bd 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -50,6 +50,7 @@ private: // indices of columns to read from ORC file std::vector include_indices; + std::vector missing_columns; BlockMissingValues block_missing_values; const FormatSettings format_settings; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 2c5799f9d7a..05528c67dd3 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -56,10 +56,12 @@ Chunk ParquetBlockInputFormat::generate() ++row_group_current; - auto missing_column_indexes = arrow_column_to_ch_column->arrowTableToCHChunk(res, table); - for (size_t row_idx = 0; row_idx < res.getNumRows(); ++row_idx) - for (const auto & column_idx : missing_column_indexes) - block_missing_values.setBit(column_idx, row_idx); + /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. + /// Otherwise fill the missing columns with zero values of its type. + if (format_settings.defaults_for_omitted_fields) + for (size_t row_idx = 0; row_idx < res.getNumRows(); ++row_idx) + for (const auto & column_idx : missing_columns) + block_missing_values.setBit(column_idx, row_idx); return res; } @@ -111,7 +113,8 @@ void ParquetBlockInputFormat::prepareReader() THROW_ARROW_NOT_OK(file_reader->GetSchema(&schema)); arrow_column_to_ch_column = std::make_unique( - getPort().getHeader(), "Parquet", format_settings.parquet.import_nested, format_settings.defaults_for_omitted_fields); + getPort().getHeader(), "Parquet", format_settings.parquet.import_nested, format_settings.parquet.allow_missing_columns); + missing_columns = arrow_column_to_ch_column->getMissingColumns(*schema); std::unordered_set nested_table_names; if (format_settings.parquet.import_nested) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index 4597a52d8ee..9beafcbf5a8 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -36,6 +36,7 @@ private: std::vector column_indices; std::unique_ptr arrow_column_to_ch_column; int row_group_current = 0; + std::vector missing_columns; BlockMissingValues block_missing_values; const FormatSettings format_settings; }; diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index a8200f89f2b..0bfeddaa759 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -131,10 +131,7 @@ public: } /// Initialize format settings - format_settings.csv.delimiter = '\x01'; - format_settings.csv.input_field_names = text_input_field_names; - format_settings.csv.read_bool_as_uint8 = true; - format_settings.defaults_for_omitted_fields = true; + format_settings.hive_text.input_field_names = text_input_field_names; } String getName() const override { return "Hive"; } @@ -373,7 +370,7 @@ ASTPtr StorageHive::extractKeyExpressionList(const ASTPtr & node) /// Primary key consists of one column. auto res = std::make_shared(); res->children.push_back(node); - return res; + return std::move(res); } } From c8f536a0a4f8b4cea605038fc7fb5a2acc74b2ea Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 2 Dec 2021 18:10:58 +0800 Subject: [PATCH 0115/1260] Use read/write buffer to read/write files --- src/IO/RemoteReadBufferCache.cpp | 186 ++++++++++++------------------- src/IO/RemoteReadBufferCache.h | 58 ++++------ 2 files changed, 93 insertions(+), 151 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 27683326b7d..157e5de78fe 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -111,8 +111,8 @@ RemoteCacheController::RemoteCacheController( if (remote_read_buffer) { // setup local files - out_file = std::make_unique(fs::path(local_path_) / "data.bin", std::ios::out | std::ios::binary); - out_file->flush(); + data_file_writer = std::make_unique((fs::path(local_path_) / "data.bin").string()); + data_file_writer->sync(); file_meta_data.save(local_path_ / "meta.txt"); @@ -141,7 +141,7 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs return RemoteReadBufferCacheError::OK; } else - more_data_signal.wait(lock, [this, end_offset_] { return this->file_meta_data.status == RemoteFileMetadata::DOWNLOADED || current_offset >= end_offset_; }); + more_data_signal.wait(lock, [this, end_offset_] { return file_meta_data.status == RemoteFileMetadata::DOWNLOADED || current_offset >= end_offset_; }); } lock.unlock(); return RemoteReadBufferCacheError::OK; @@ -156,7 +156,7 @@ void RemoteCacheController::backgroundDownload() { size_t bytes = remote_read_buffer->available(); - out_file->write(remote_read_buffer->position(), bytes); + data_file_writer->write(remote_read_buffer->position(), bytes); remote_read_buffer->position() += bytes; total_bytes += bytes; before_unflush_bytes += bytes; @@ -175,8 +175,7 @@ void RemoteCacheController::backgroundDownload() current_offset += total_bytes; file_meta_data.status = RemoteFileMetadata::DOWNLOADED; flush(true); - out_file->close(); - out_file.reset(); + data_file_writer.reset(); remote_read_buffer.reset(); lock.unlock(); more_data_signal.notify_all(); @@ -186,9 +185,9 @@ void RemoteCacheController::backgroundDownload() void RemoteCacheController::flush(bool need_flush_meta_data_) { - if (out_file) + if (data_file_writer) { - out_file->flush(); + data_file_writer->sync(); } if (!need_flush_meta_data_) @@ -202,79 +201,43 @@ RemoteCacheController::~RemoteCacheController() = default; void RemoteCacheController::close() { // delete directory - LOG_TRACE(log, "Removing all local cache. local path: {}, file meta data:{}", local_path.string(), file_meta_data.toString()); + LOG_TRACE(log, "Removing the local cache. local path: {}, file meta data:{}", local_path.string(), file_meta_data.toString()); std::filesystem::remove_all(local_path); } -std::pair RemoteCacheController::allocFile() +std::unique_ptr RemoteCacheController::allocFile() { - std::filesystem::path result_local_path = local_path / "data.bin"; + ReadSettings settings; + settings.local_fs_prefetch = false; + settings.local_fs_method = LocalFSReadMethod::read; + auto file_buffer = createReadBufferFromFileBase((local_path / "data.bin").string(), settings); - FILE * fs = fopen(result_local_path.string().c_str(), "r"); - if (!fs) - throw Exception(ErrorCodes::BAD_GET, "Alloc file failed, error code: {} local path: {}", errno, local_path.string()); - - std::lock_guard lock{mutex}; - opened_file_streams.insert(fs); - return {fs, result_local_path}; -} - -void RemoteCacheController::deallocFile(FILE * file_stream) -{ + if (file_buffer) { std::lock_guard lock{mutex}; - auto it = opened_file_streams.find(file_stream); - if (it == opened_file_streams.end()) - throw Exception( + opened_file_buffer_refs.insert(reinterpret_cast(file_buffer.get())); + } + return file_buffer; +} + +void RemoteCacheController::deallocFile(std::unique_ptr file_buffer) +{ + if (!file_buffer) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Try to release a null file buffer for ", local_path.string()); + } + auto buffer_ref = reinterpret_cast(file_buffer.get()); + std::lock_guard lock{mutex}; + auto it = opened_file_buffer_refs.find(buffer_ref); + if (it == opened_file_buffer_refs.end()) + { + throw Exception( ErrorCodes::BAD_ARGUMENTS, "Try to deallocate file with invalid handler remote path: {}, local path: {}", file_meta_data.remote_path, local_path.string()); - opened_file_streams.erase(it); } - fclose(file_stream); -} - -LocalCachedFileReader::LocalCachedFileReader(RemoteCacheController * cache_controller_) - : cache_controller(cache_controller_) - , file_size(cache_controller_->getFileMetaData().file_size) - , offset(0) -{ - std::tie(file_stream, local_path) = cache_controller->allocFile(); -} - -LocalCachedFileReader::~LocalCachedFileReader() -{ - cache_controller->deallocFile(file_stream); -} - -size_t LocalCachedFileReader::read(char * buf, size_t size) -{ - auto wret = cache_controller->waitMoreData(offset, offset + size); - if (wret != RemoteReadBufferCacheError::OK) - return 0; - std::lock_guard lock(mutex); - auto ret_size = fread(buf, 1, size, file_stream); - offset += ret_size; - return ret_size; -} - -off_t LocalCachedFileReader::seek(off_t off) -{ - cache_controller->waitMoreData(off, 0); - std::lock_guard lock(mutex); - auto ret = fseek(file_stream, off, SEEK_SET); - if (ret < 0) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Seek file {} with size {} to offset {} failed: {}", getPath().string(), getSize(), off, errnoToString(errno)); - - offset = off; - return off; -} - -size_t LocalCachedFileReader::getSize() -{ - return file_size; + opened_file_buffer_refs.erase(it); } // the size need be equal to the original buffer @@ -282,7 +245,10 @@ RemoteReadBuffer::RemoteReadBuffer(size_t buff_size) : BufferWithOwnMemorydeallocFile(std::move(file_buffer)); +} std::unique_ptr RemoteReadBuffer::create(ContextPtr context, const RemoteFileMetadata & remote_file_meta, std::unique_ptr read_buffer) { @@ -303,28 +269,40 @@ std::unique_ptr RemoteReadBuffer::create(ContextPtr context, c const auto & remote_path = remote_file_meta.remote_path; auto remote_read_buffer = std::make_unique(buff_size); - auto * raw_rbp = read_buffer.release(); - std::shared_ptr srb(raw_rbp); + auto * raw_readbuffer_ptr = read_buffer.release(); + std::shared_ptr shared_readbuffer_ptr(raw_readbuffer_ptr); RemoteReadBufferCacheError error; - std::tie(remote_read_buffer->file_reader, error) = RemoteReadBufferCache::instance().createReader(context, remote_file_meta, srb); - if (remote_read_buffer->file_reader == nullptr) + std::tie(remote_read_buffer->file_cache_controller, error) = RemoteReadBufferCache::instance().createReader(context, remote_file_meta, shared_readbuffer_ptr); + if (remote_read_buffer->file_cache_controller == nullptr) { LOG_ERROR(log, "Failed to allocate local file for remote path: {}, reason: {}", remote_path, error); - remote_read_buffer->original_read_buffer = srb; + remote_read_buffer->original_read_buffer = shared_readbuffer_ptr; + } + else + { + remote_read_buffer->file_buffer = remote_read_buffer->file_cache_controller->allocFile(); + if (!remote_read_buffer->file_buffer) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Create file readbuffer failed. {}", + remote_read_buffer->file_cache_controller->getLocalPath().string()); } return remote_read_buffer; } bool RemoteReadBuffer::nextImpl() { - if (file_reader) + if (file_buffer) { - int bytes_read = file_reader->read(internal_buffer.begin(), internal_buffer.size()); - if (bytes_read) - working_buffer.resize(bytes_read); - else - return false; + auto start_offset = file_buffer->getPosition(); + auto end_offset = file_buffer->internalBuffer().size(); + file_cache_controller->waitMoreData(start_offset, end_offset); + + auto status = file_buffer->next(); + if (status) + BufferBase::set(file_buffer->buffer().begin(), + file_buffer->buffer().size(), + file_buffer->offset()); + return status; } else { @@ -339,39 +317,25 @@ bool RemoteReadBuffer::nextImpl() BufferBase::set(original_read_buffer->buffer().begin(), original_read_buffer->buffer().size(), original_read_buffer->offset()); return status; } - return true; } off_t RemoteReadBuffer::seek(off_t offset, int whence) { - off_t pos_in_file = file_reader->getOffset(); - off_t new_pos; - if (whence == SEEK_SET) - new_pos = offset; - else if (whence == SEEK_CUR) - new_pos = pos_in_file - available() + offset; - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expects SEEK_SET or SEEK_CUR as whence but given {}", whence); - - /// Position is unchanged. - if (off_t(new_pos + available()) == pos_in_file) - return new_pos; - - if (new_pos <= pos_in_file && new_pos >= pos_in_file - static_cast(working_buffer.size())) - { - /// Position is still inside buffer. - pos = working_buffer.begin() + (new_pos - (pos_in_file - working_buffer.size())); - return new_pos; - } - - pos = working_buffer.end(); - auto ret_off = file_reader->seek(new_pos); - return ret_off; + /* + * Need to wait here. For example, the current file has been download at position X, but here we try to seek to + * postition Y ( Y > X), it would fail. + */ + file_cache_controller->waitMoreData(offset, offset + file_buffer->internalBuffer().size()); + auto ret = file_buffer->seek(offset, whence); + BufferBase::set(file_buffer->buffer().begin(), + file_buffer->buffer().size(), + file_buffer->offset()); + return ret; } off_t RemoteReadBuffer::getPosition() { - return file_reader->getOffset() - available(); + return file_buffer->getPosition(); } RemoteReadBufferCache::RemoteReadBufferCache() = default; @@ -450,7 +414,7 @@ String RemoteReadBufferCache::calculateLocalPath(const RemoteFileMetadata & meta return fs::path(root_dir) / hashcode_str.substr(0, 3) / hashcode_str; } -std::pair, RemoteReadBufferCacheError> +std::pair RemoteReadBufferCache::createReader(ContextPtr context, const RemoteFileMetadata & remote_file_meta, std::shared_ptr & read_buffer) { LOG_TRACE(log, "createReader. {} {} {}", remote_file_meta.remote_path, remote_file_meta.last_modification_timestamp, remote_file_meta.file_size); @@ -483,9 +447,7 @@ RemoteReadBufferCache::createReader(ContextPtr context, const RemoteFileMetadata { // move the key to the list end keys.splice(keys.end(), keys, cache_iter->second.key_iterator); - return { - std::make_shared(cache_iter->second.cache_controller.get()), - RemoteReadBufferCacheError::OK}; + return { cache_iter->second.cache_controller, RemoteReadBufferCacheError::OK}; } } @@ -497,7 +459,7 @@ RemoteReadBufferCache::createReader(ContextPtr context, const RemoteFileMetadata { keys.splice(keys.end(), keys, cache_iter->second.key_iterator); return { - std::make_shared(cache_iter->second.cache_controller.get()), + cache_iter->second.cache_controller, RemoteReadBufferCacheError::OK}; } else @@ -526,7 +488,7 @@ RemoteReadBufferCache::createReader(ContextPtr context, const RemoteFileMetadata cache_cell.key_iterator = keys.insert(keys.end(), local_path); caches[local_path] = cache_cell; - return {std::make_shared(cache_controller.get()), RemoteReadBufferCacheError::OK}; + return {cache_controller, RemoteReadBufferCacheError::OK}; } bool RemoteReadBufferCache::clearLocalCache() diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 978add14de3..20cb2eb8877 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -10,6 +10,11 @@ #include #include #include +#include +#include +#include +#include +#include #include #include #include @@ -81,8 +86,8 @@ public: * Called by LocalCachedFileReader, must be used in pair * The second value of the return tuple is the local_path to store file. */ - std::pair allocFile(); - void deallocFile(FILE * file_stream); + std::unique_ptr allocFile(); + void deallocFile(std::unique_ptr buffer); /** * when allocFile be called, count++. deallocFile be called, count--. @@ -91,7 +96,8 @@ public: inline bool closable() { std::lock_guard lock{mutex}; - return opened_file_streams.empty() && remote_read_buffer == nullptr; + //return opened_file_streams.empty() && remote_read_buffer == nullptr; + return opened_file_buffer_refs.empty() && remote_read_buffer == nullptr; } void close(); @@ -130,7 +136,7 @@ private: std::mutex mutex; std::condition_variable more_data_signal; - std::set opened_file_streams; + std::set opened_file_buffer_refs; // refer to a buffer address // meta info RemoteFileMetadata file_meta_data; @@ -141,40 +147,11 @@ private: size_t current_offset; std::shared_ptr remote_read_buffer; - std::unique_ptr out_file; + std::unique_ptr data_file_writer; Poco::Logger * log = &Poco::Logger::get("RemoteCacheController"); }; - -/** - * access local cached files by RemoteCacheController, and be used in RemoteReadBuffer - */ -class LocalCachedFileReader -{ -public: - LocalCachedFileReader(RemoteCacheController * cache_controller_); - ~LocalCachedFileReader(); - - // expect to read size bytes into buf, return is the real bytes read - size_t read(char * buf, size_t size); - off_t seek(off_t offset); - - inline std::filesystem::path getPath() const { return local_path; } - inline off_t getOffset() const { return static_cast(offset); } - size_t getSize(); - - -private: - RemoteCacheController * cache_controller; - size_t file_size; - size_t offset; - - std::mutex mutex; - FILE * file_stream; - std::filesystem::path local_path; - - Poco::Logger * log = &Poco::Logger::get("LocalCachedFileReader"); -}; +using RemoteCacheControllerPtr = std::shared_ptr; /* * FIXME:RemoteReadBuffer derive from SeekableReadBufferWithSize may cause some risks, since it's not seekable in some cases @@ -188,13 +165,16 @@ public: static std::unique_ptr create(ContextPtr contex, const RemoteFileMetadata & remote_file_meta, std::unique_ptr read_buffer); bool nextImpl() override; - inline bool seekable() { return file_reader != nullptr && file_reader->getSize() > 0; } + inline bool seekable() { return !file_buffer && file_cache_controller->getFileMetaData().file_size > 0; } off_t seek(off_t off, int whence) override; off_t getPosition() override; - std::optional getTotalSize() override { return file_reader->getSize(); } + std::optional getTotalSize() override { return file_cache_controller->getFileMetaData().file_size; } private: - std::shared_ptr file_reader; + std::shared_ptr file_cache_controller; + std::unique_ptr file_buffer; + + // in case local cache don't work, this buffer is setted; std::shared_ptr original_read_buffer; }; @@ -209,7 +189,7 @@ public: inline bool isInitialized() const { return initialized; } - std::pair, RemoteReadBufferCacheError> + std::pair createReader(ContextPtr context, const RemoteFileMetadata & remote_file_meta, std::shared_ptr & read_buffer); void updateTotalSize(size_t size) { total_size += size; } From 1c9d0b86cc7d04a6e00975d8a572dbb695b854a9 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 2 Dec 2021 18:19:15 +0800 Subject: [PATCH 0116/1260] fixed code style --- src/IO/RemoteReadBufferCache.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 157e5de78fe..33c3fd0dfa7 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -22,7 +22,6 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int BAD_GET; extern const int LOGICAL_ERROR; extern const int CANNOT_CREATE_DIRECTORY; } @@ -322,7 +321,7 @@ bool RemoteReadBuffer::nextImpl() off_t RemoteReadBuffer::seek(off_t offset, int whence) { /* - * Need to wait here. For example, the current file has been download at position X, but here we try to seek to + * Need to wait here. For example, the current file has been download at position X, but here we try to seek to * postition Y ( Y > X), it would fail. */ file_cache_controller->waitMoreData(offset, offset + file_buffer->internalBuffer().size()); From e2d1ed15682c88a8ae816d93a09d21fab4cf7aed Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 2 Dec 2021 20:51:19 +0800 Subject: [PATCH 0117/1260] fix error --- src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp | 2 ++ src/Processors/Formats/Impl/ORCBlockInputFormat.cpp | 1 + src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 2 ++ 3 files changed, 5 insertions(+) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index 33daa2bd69b..f61ad9bd0c2 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -64,6 +64,8 @@ Chunk ArrowBlockInputFormat::generate() ++record_batch_current; + arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result); + /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. /// Otherwise fill the missing columns with zero values of its type. if (format_settings.defaults_for_omitted_fields) diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index aef8e5909cf..eb895b66388 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -68,6 +68,7 @@ Chunk ORCBlockInputFormat::generate() std::shared_ptr arrow_column = std::make_shared(vec); name_to_column_ptr[column_name] = arrow_column; } + arrow_column_to_ch_column->arrowColumnsToCHChunk(res, name_to_column_ptr); /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. /// Otherwise fill the missing columns with zero values of its type. diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 05528c67dd3..73ff253263e 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -56,6 +56,8 @@ Chunk ParquetBlockInputFormat::generate() ++row_group_current; + arrow_column_to_ch_column->arrowTableToCHChunk(res, table); + /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. /// Otherwise fill the missing columns with zero values of its type. if (format_settings.defaults_for_omitted_fields) From 01cac015279b9f43afeb237fa0372e30a69f897d Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 2 Dec 2021 20:54:49 +0800 Subject: [PATCH 0118/1260] modify permission of RowInputFormatWithNamesAndTypes methods --- src/Processors/Formats/RowInputFormatWithNamesAndTypes.h | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index 16d8a182ef4..fa283bd5c3c 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -33,9 +33,7 @@ public: void resetParser() override; protected: - void readPrefix() override; - bool readRow(MutableColumns & columns, RowReadExtension & ext) override; - + /// Read single field from input. Return false if there was no real value and we inserted default value. virtual bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) = 0; @@ -71,17 +69,20 @@ protected: /// Read row with types and return the list of them. virtual std::vector readTypes() = 0; - virtual void addInputColumn(const String & column_name, std::vector & read_columns); const FormatSettings format_settings; DataTypes data_types; bool end_of_stream = false; private: + void readPrefix() override; + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override; void tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) override; void setupAllColumnsByTableSchema(); + void addInputColumn(const String & column_name, std::vector & read_columns); void insertDefaultsForNotSeenColumns(MutableColumns & columns, RowReadExtension & ext); bool with_names; From 98bae1b3130d8bbc189c863a7537389bb20b4f65 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 2 Dec 2021 16:25:45 +0300 Subject: [PATCH 0119/1260] Fix tests --- src/Storages/MergeTree/MergeTreeData.cpp | 26 ++++++++++++++----- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../test_log_lz4_streaming/test.py | 2 +- 3 files changed, 21 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 5bd13179c09..c9336a8419e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3877,7 +3877,7 @@ void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, ContextPtr for (auto & [old_name, new_name, disk] : renamed_parts.old_and_new_names) { bool keep_shared = removeSharedDetachedPart(disk, fs::path(relative_data_path) / "detached" / new_name / "", old_name, - zookeeper_name, replica_name, zookeeper_path); + zookeeper_name, replica_name, zookeeper_path, supportsReplication()); LOG_DEBUG(log, "Dropped detached part {}, keep shared data: {}", old_name, keep_shared); old_name.clear(); } @@ -5299,8 +5299,9 @@ bool MergeTreeData::removeSharedDetachedPart(DiskPtr disk, const String & path, if (disk->supportZeroCopyReplication()) { FreezeMetaData meta; - if (meta.load(disk, path) && meta.is_replicated) - return removeSharedDetachedPart(disk, path, part_name, meta.zookeeper_name, meta.replica_name, ""); + if (meta.load(disk, path)) + return removeSharedDetachedPart(disk, path, part_name, meta.zookeeper_name, meta.replica_name, + "", meta.is_replicated); } disk->removeSharedRecursive(path, keep_shared); @@ -5309,20 +5310,31 @@ bool MergeTreeData::removeSharedDetachedPart(DiskPtr disk, const String & path, } bool MergeTreeData::removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, - const String & zookeeper_name, const String & replica_name, const String & zookeeper_path) + const String & zookeeper_name, const String & replica_name, const String & zookeeper_path, bool is_replicated) { bool keep_shared = false; - if (disk->supportZeroCopyReplication()) + if (is_replicated && disk->supportZeroCopyReplication()) { + static constexpr auto default_zookeeper_name = "default"; zkutil::ZooKeeperPtr zookeeper; - if (zookeeper_name == "default") + if (zookeeper_name == default_zookeeper_name) { zookeeper = getContext()->getZooKeeper(); } else { - zookeeper = getContext()->getAuxiliaryZooKeeper(zookeeper_name); + try + { + zookeeper = getContext()->getAuxiliaryZooKeeper(zookeeper_name); + } + catch (const Exception & e) + { + if (e.code() != ErrorCodes::BAD_ARGUMENTS) + throw; + /// No more stored non-default zookeeper + zookeeper = nullptr; + } } if (zookeeper) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 171958be35f..fc21dd87569 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1185,7 +1185,7 @@ private: /// Remove local files and remote files if needed bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name); bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, - const String & zookeeper_name, const String & replica_name, const String & zookeeper_path); + const String & zookeeper_name, const String & replica_name, const String & zookeeper_path, bool is_replicated); }; /// RAII struct to record big parts that are submerging or emerging. diff --git a/tests/integration/test_log_lz4_streaming/test.py b/tests/integration/test_log_lz4_streaming/test.py index 7f2f22f28c9..75b46a378c5 100644 --- a/tests/integration/test_log_lz4_streaming/test.py +++ b/tests/integration/test_log_lz4_streaming/test.py @@ -18,7 +18,7 @@ def started_cluster(): def check_log_file(): - assert node.file_exists("/var/log/clickhouse-server/clickhouse-server.log.lz4") + assert node.path_exists("/var/log/clickhouse-server/clickhouse-server.log.lz4") lz4_output = node.exec_in_container(["bash", "-c", "lz4 -t /var/log/clickhouse-server/clickhouse-server.log.lz4 2>&1"], user='root') assert lz4_output.count('Error') == 0, lz4_output From 57f5fdc2fa58e5d6c158327418a6772d19250d28 Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 3 Dec 2021 17:08:57 +0000 Subject: [PATCH 0120/1260] update --- tests/queries/0_stateless/00700_decimal_compare.sql | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/00700_decimal_compare.sql b/tests/queries/0_stateless/00700_decimal_compare.sql index a557446a224..2ffd931820b 100644 --- a/tests/queries/0_stateless/00700_decimal_compare.sql +++ b/tests/queries/0_stateless/00700_decimal_compare.sql @@ -46,18 +46,18 @@ SELECT greatest(a, 0), greatest(b, 0), greatest(g, 0) FROM decimal ORDER BY a; SELECT (a, d, g) = (b, e, h), (a, d, g) != (b, e, h) FROM decimal ORDER BY a; SELECT (a, d, g) = (c, f, i), (a, d, g) != (c, f, i) FROM decimal ORDER BY a; -SELECT toUInt32(2147483648) AS x, a == x FROM decimal WHERE a = 42; -- { serverError 407 } +SELECT toUInt32(2147483648) AS x, a == x FROM decimal WHERE a = 42; -- { serverError 407 } SELECT toUInt64(2147483648) AS x, b == x, x == ((b - 42) + x) FROM decimal WHERE a = 42; -SELECT toUInt64(9223372036854775808) AS x, b == x FROM decimal WHERE a = 42; -- { serverError 407 } +SELECT toUInt64(9223372036854775808) AS x, b == x FROM decimal WHERE a = 42; -- { serverError 407 } SELECT toUInt64(9223372036854775808) AS x, c == x, x == ((c - 42) + x) FROM decimal WHERE a = 42; SELECT g = 10000, (g - g + 10000) == 10000 FROM decimal WHERE a = 42; SELECT 10000 = g, 10000 = (g - g + 10000) FROM decimal WHERE a = 42; -SELECT g = 30000 FROM decimal WHERE a = 42; -- { serverError 407 } -SELECT 30000 = g FROM decimal WHERE a = 42; -- { serverError 407 } +SELECT g = 30000 FROM decimal WHERE a = 42; -- { serverError 407 } +SELECT 30000 = g FROM decimal WHERE a = 42; -- { serverError 407 } SELECT h = 30000, (h - g + 30000) = 30000 FROM decimal WHERE a = 42; SELECT 30000 = h, 30000 = (h - g + 30000) FROM decimal WHERE a = 42; -SELECT h = 10000000000 FROM decimal WHERE a = 42; -- { serverError 407 } +SELECT h = 10000000000 FROM decimal WHERE a = 42; -- { serverError 407 } SELECT i = 10000000000, (i - g + 10000000000) = 10000000000 FROM decimal WHERE a = 42; SELECT 10000000000 = i, 10000000000 = (i - g + 10000000000) FROM decimal WHERE a = 42; From c678c8101e9f63b1389330f84ebc66f3295767e7 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 4 Dec 2021 16:41:35 +0800 Subject: [PATCH 0121/1260] fix some bugs --- .../Formats/RowInputFormatWithNamesAndTypes.h | 1 - src/Processors/examples/CMakeLists.txt | 6 +++++- .../examples/comma_separated_streams.cpp | 17 +++++++---------- src/Storages/Hive/StorageHive.cpp | 2 +- 4 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index fa283bd5c3c..baa474f9206 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -33,7 +33,6 @@ public: void resetParser() override; protected: - /// Read single field from input. Return false if there was no real value and we inserted default value. virtual bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) = 0; diff --git a/src/Processors/examples/CMakeLists.txt b/src/Processors/examples/CMakeLists.txt index 6f78d611f45..82f8c754373 100644 --- a/src/Processors/examples/CMakeLists.txt +++ b/src/Processors/examples/CMakeLists.txt @@ -1,2 +1,6 @@ add_executable (comma_separated_streams comma_separated_streams.cpp) -target_link_libraries (comma_separated_streams PRIVATE dbms) \ No newline at end of file +target_link_libraries (comma_separated_streams PRIVATE dbms) +#target_include_directories(comma_separated_streams SYSTEM BEFORE PRIVATE +# ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src +# ${ClickHouse_SOURCE_DIR}/contrib/arrow-cmake/cpp/src) +# dbms_target_link_libraries(PRIVATE ${ARROW_LIBRARY}) diff --git a/src/Processors/examples/comma_separated_streams.cpp b/src/Processors/examples/comma_separated_streams.cpp index a615957561c..a8c925354d9 100644 --- a/src/Processors/examples/comma_separated_streams.cpp +++ b/src/Processors/examples/comma_separated_streams.cpp @@ -1,15 +1,12 @@ #include - #include #include #include #include - #include #include - -#include +#include #include #include #include @@ -17,7 +14,7 @@ using namespace DB; -int main(int, char **) +int main() try { Block sample; @@ -83,7 +80,10 @@ try WriteBufferFromFile out_buf("test_out"); FormatSettings format_settings; - format_settings.csv.input_field_names = + format_settings.with_names_use_header = true; + format_settings.skip_unknown_fields = true; + format_settings.csv.delimiter = '\x01'; + format_settings.hive_text.input_field_names = { "d", "e", @@ -96,11 +96,9 @@ try "i", "j", }; - format_settings.csv.delimiter = '\x01'; - format_settings.with_names_use_header = true; RowInputFormatParams in_params{DEFAULT_INSERT_BLOCK_SIZE}; - InputFormatPtr input_format = std::make_shared(sample, in_buf, in_params, true, false, format_settings); + InputFormatPtr input_format = std::make_shared(sample, in_buf, in_params, format_settings); auto pipeline = QueryPipeline(std::move(input_format)); auto reader = std::make_unique(pipeline); @@ -111,7 +109,6 @@ try { output_format->write(res); } - output_format->flush(); return 0; } catch (...) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 0bfeddaa759..d0a0991a16a 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -599,7 +599,7 @@ Pipe StorageHive::read( format_name, compression_method, metadata_snapshot->getSampleBlock(), - getContext(), + context_, max_block_size, text_input_field_names)); } From 01940c3f0154eecfff3ae1b9ad1982fb015667e0 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 3 Dec 2021 13:58:19 +0800 Subject: [PATCH 0122/1260] coding refinement 1. make remote file meta data be a abstract class 2. make cache controller download process be started by RemoteReadBufferCache --- src/IO/RemoteFileMetaDataBase.cpp | 35 +++++ src/IO/RemoteFileMetaDataBase.h | 87 +++++++++++ src/IO/RemoteReadBufferCache.cpp | 190 +++++++++++++------------ src/IO/RemoteReadBufferCache.h | 77 ++++------ src/Storages/Hive/HiveFileMetaData.cpp | 53 +++++++ src/Storages/Hive/HiveFileMetaData.h | 29 ++++ src/Storages/Hive/StorageHive.cpp | 6 +- 7 files changed, 332 insertions(+), 145 deletions(-) create mode 100644 src/IO/RemoteFileMetaDataBase.cpp create mode 100644 src/IO/RemoteFileMetaDataBase.h create mode 100644 src/Storages/Hive/HiveFileMetaData.cpp create mode 100644 src/Storages/Hive/HiveFileMetaData.h diff --git a/src/IO/RemoteFileMetaDataBase.cpp b/src/IO/RemoteFileMetaDataBase.cpp new file mode 100644 index 00000000000..e041bdce29a --- /dev/null +++ b/src/IO/RemoteFileMetaDataBase.cpp @@ -0,0 +1,35 @@ +#include +#include +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +RemoteFileMetaDataBase::~RemoteFileMetaDataBase() {} + +RemoteFileMetaDataFactory & RemoteFileMetaDataFactory::instance() +{ + static RemoteFileMetaDataFactory g_factory; + return g_factory; +} + +RemoteFileMetaDataBasePtr RemoteFileMetaDataFactory::create_class(const String & class_name) +{ + auto it = class_creators.find(class_name); + if (it == class_creators.end()) + return nullptr; + return (it->second)(); +} + +void RemoteFileMetaDataFactory::register_class(const String & class_name, ClassCreator creator) +{ + auto it = class_creators.find(class_name); + if (it != class_creators.end()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Class ({}) has been registered. It is a fatal error.", class_name); + } + class_creators[class_name] = creator; +} +} diff --git a/src/IO/RemoteFileMetaDataBase.h b/src/IO/RemoteFileMetaDataBase.h new file mode 100644 index 00000000000..d2acbd0af96 --- /dev/null +++ b/src/IO/RemoteFileMetaDataBase.h @@ -0,0 +1,87 @@ +#pragma once +#include +#include +#include +#include +#include + +namespace DB +{ +class RemoteFileMetaDataBase +{ +public: + RemoteFileMetaDataBase() = default; + RemoteFileMetaDataBase(const String & schema_, + const String & cluster_, + const String & remote_path_, + size_t file_size_, + UInt64 last_modification_timestamp_): + schema(schema_) + ,cluster(cluster_) + ,remote_path(remote_path_) + ,file_size(file_size_) + ,last_modification_timestamp(last_modification_timestamp_) + { + } + virtual ~RemoteFileMetaDataBase(); + virtual String getClassName() = 0; //class name + // methods for basic information + inline String getSchema() { return schema; } + inline String getCluster() { return cluster; } + inline size_t getFileSize() { return file_size; } + inline String getRemotePath() { return remote_path; } + inline UInt64 getLastModificationTimestamp() { return last_modification_timestamp; } + // create a new object + virtual std::shared_ptr clone() = 0; + + // deserialize + virtual bool fromString(const String &buf) = 0; + // serialize + virtual String toString() = 0; + // to compare two meta datas for detecting file changes + virtual bool equal(std::shared_ptr b) = 0; +protected: + String schema; + String cluster; + String remote_path; + size_t file_size = 0; + UInt64 last_modification_timestamp = 0; +}; + +using RemoteFileMetaDataBasePtr = std::shared_ptr; + +/* + * How to register a subclass into factory and use it ? + * 1) define your own subclass derive from RemoteFileMetaDataBase. Notice! the getClassName() must be the same + as your subclass name. + * 2) in a .cpp file, call REGISTTER_REMOTE_FILE_META_DATA_CLASS(subclass). + 3) call RemoteFileMetaDataFactory::instance().create_class(subclass_name) where you want to make a new object + */ + +class RemoteFileMetaDataFactory : private boost::noncopyable +{ +public: + using ClassCreator = std::function; + ~RemoteFileMetaDataFactory() = default; + + static RemoteFileMetaDataFactory & instance(); + RemoteFileMetaDataBasePtr create_class(const String & class_name); + void register_class(const String &class_name, ClassCreator creator); +protected: + RemoteFileMetaDataFactory() = default; + +private: + std::unordered_map class_creators; +}; + +// this should be used in a .cpp file. All the subclasses will finish the registeration before the main() +#define REGISTTER_REMOTE_FILE_META_DATA_CLASS(meta_data_class) \ + class FileMetaDataFactory##meta_data_class{\ + public:\ + FileMetaDataFactory##meta_data_class(){\ + auto creator = []() -> RemoteFileMetaDataBasePtr { return std::make_shared(); };\ + RemoteFileMetaDataFactory::instance().register_class(#meta_data_class, creator);\ + }\ + };\ + static FileMetaDataFactory##meta_data_class g_file_meta_data_factory_instance##meta_data_class; +} diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 33c3fd0dfa7..3a482e90bad 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -26,104 +26,85 @@ namespace ErrorCodes extern const int CANNOT_CREATE_DIRECTORY; } -bool RemoteFileMetadata::load(const std::filesystem::path & local_path) +bool RemoteCacheController::loadInnerInformation(const fs::path & file_path) { - auto * log = &Poco::Logger::get("RemoteFileMetadata"); - if (!std::filesystem::exists(local_path)) - { - LOG_ERROR(log, "file path not exists:{}", local_path.string()); + if (!fs::exists(file_path)) return false; - } - std::ifstream meta_fs(local_path.string()); - Poco::JSON::Parser meta_data_parser; - auto meta_data_jobj = meta_data_parser.parse(meta_fs).extract(); - remote_path = meta_data_jobj->get("remote_path").convert(); - schema = meta_data_jobj->get("schema").convert(); - cluster = meta_data_jobj->get("cluster").convert(); - status = static_cast(meta_data_jobj->get("status").convert()); - last_modification_timestamp = meta_data_jobj->get("last_modification_timestamp").convert(); - file_size = meta_data_jobj->get("file_size").convert(); - meta_fs.close(); - + std::ifstream info_file(file_path); + Poco::JSON::Parser info_parser; + auto info_jobj = info_parser.parse(info_file).extract(); + file_status = static_cast(info_jobj->get("file_status").convert()); + meta_data_class = info_jobj->get("meta_data_class").convert(); + info_file.close(); return true; } -void RemoteFileMetadata::save(const std::filesystem::path & local_path) const -{ - std::ofstream meta_file(local_path.string(), std::ios::out); - meta_file << toString(); - meta_file.close(); -} - -String RemoteFileMetadata::toString() const -{ - Poco::JSON::Object jobj; - jobj.set("schema", schema); - jobj.set("cluster", cluster); - jobj.set("remote_path", remote_path); - jobj.set("status", static_cast(status)); - jobj.set("last_modification_timestamp", last_modification_timestamp); - jobj.set("file_size", file_size); - std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - jobj.stringify(buf); - return buf.str(); -} - std::shared_ptr RemoteCacheController::recover(const std::filesystem::path & local_path_) { auto * log = &Poco::Logger::get("RemoteCacheController"); - if (!std::filesystem::exists(local_path_) || !std::filesystem::exists(local_path_ / "data.bin")) + if (!std::filesystem::exists(local_path_ / "data.bin")) { LOG_TRACE(log, "Invalid cached directory:{}", local_path_.string()); return nullptr; } - RemoteFileMetadata remote_file_meta_data; - if (!remote_file_meta_data.load(local_path_ / "meta.txt") || remote_file_meta_data.status != RemoteFileMetadata::DOWNLOADED) + auto cache_controller = std::make_shared(nullptr, local_path_, 0); + if (!cache_controller->loadInnerInformation(local_path_ / "info.txt") + || cache_controller->file_status != DOWNLOADED) { - LOG_INFO(log, "recover cached file failed. local path:{}, file meta data:{}", local_path_.string(), remote_file_meta_data.toString()); + LOG_INFO(log, "recover cached file failed. local path:{}", local_path_.string()); return nullptr; } - auto cache_controller = std::make_shared(nullptr, remote_file_meta_data, local_path_, 0, nullptr); - cache_controller->current_offset = remote_file_meta_data.file_size; + cache_controller->file_meta_data_ptr = RemoteFileMetaDataFactory::instance().create_class(cache_controller->meta_data_class); + if (!cache_controller->file_meta_data_ptr) + { + // do not load this invalid cached file and clear it + LOG_ERROR(log, "Cannot create the meta data class : {}. The cached file is invalid and will be remove. path:{}", + cache_controller->meta_data_class, + local_path_.string()); + fs::remove_all(local_path_); + return nullptr; + } + std::ifstream meta_data_file(local_path_ / "meta_data.txt"); + if (!cache_controller->file_meta_data_ptr->fromString(std::string((std::istreambuf_iterator(meta_data_file)), + std::istreambuf_iterator()))) + { + LOG_ERROR(log, "Cannot load the meta data. The cached file is invalid and will be remove. path:{}", + local_path_.string()); + fs::remove_all(local_path_); + return nullptr; + } - RemoteReadBufferCache::instance().updateTotalSize(cache_controller->file_meta_data.file_size); + cache_controller->current_offset = fs::file_size(local_path_ / "data.bin"); + + RemoteReadBufferCache::instance().updateTotalSize(cache_controller->file_meta_data_ptr->getFileSize()); return cache_controller; } RemoteCacheController::RemoteCacheController( - ContextPtr context, - const RemoteFileMetadata & file_meta_data_, + RemoteFileMetaDataBasePtr file_meta_data_, const std::filesystem::path & local_path_, - size_t cache_bytes_before_flush_, - std::shared_ptr read_buffer_) - : file_meta_data(file_meta_data_) + size_t cache_bytes_before_flush_) + : file_meta_data_ptr(file_meta_data_) , local_path(local_path_) , valid(true) , local_cache_bytes_read_before_flush(cache_bytes_before_flush_) , current_offset(0) - , remote_read_buffer(read_buffer_) { - /// readbuffer == nullptr if `RemoteCacheController` is created in `initOnce`, when metadata and local cache already exist. - if (remote_read_buffer) + if (file_meta_data_ptr) { - // setup local files - data_file_writer = std::make_unique((fs::path(local_path_) / "data.bin").string()); - data_file_writer->sync(); - - file_meta_data.save(local_path_ / "meta.txt"); - - download_task_holder = context->getSchedulePool().createTask("download remote file", [this]{ this->backgroundDownload(); }); - download_task_holder->activateAndSchedule(); + std::ofstream meta_data_file(local_path_ / "meta_data.txt", std::ios::out); + meta_data_file << file_meta_data_ptr->toString(); + meta_data_file.close(); } } RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offset_, size_t end_offset_) { std::unique_lock lock{mutex}; - if (file_meta_data.status == RemoteFileMetadata::DOWNLOADED) + if (file_status == DOWNLOADED) { // finish reading if (start_offset_ >= current_offset) @@ -140,15 +121,29 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs return RemoteReadBufferCacheError::OK; } else - more_data_signal.wait(lock, [this, end_offset_] { return file_meta_data.status == RemoteFileMetadata::DOWNLOADED || current_offset >= end_offset_; }); + more_data_signal.wait(lock, [this, end_offset_] { return file_status == DOWNLOADED || current_offset >= end_offset_; }); } lock.unlock(); return RemoteReadBufferCacheError::OK; } -void RemoteCacheController::backgroundDownload() +bool RemoteCacheController::checkFileChanged(RemoteFileMetaDataBasePtr file_meta_data_) { - file_meta_data.status = RemoteFileMetadata::DOWNLOADING; + return !file_meta_data_ptr->equal(file_meta_data_); +} + +void RemoteCacheController::startBackgroundDownload(std::shared_ptr input_readbuffer, BackgroundSchedulePool & thread_pool) +{ + data_file_writer = std::make_unique((fs::path(local_path) / "data.bin").string()); + flush(true); + download_task_holder = thread_pool.createTask("download remote file", + [this,input_readbuffer]{ backgroundDownload(input_readbuffer); }); + download_task_holder->activateAndSchedule(); +} + +void RemoteCacheController::backgroundDownload(std::shared_ptr remote_read_buffer) +{ + file_status = DOWNLOADING; size_t before_unflush_bytes = 0; size_t total_bytes = 0; while (!remote_read_buffer->eof()) @@ -172,27 +167,33 @@ void RemoteCacheController::backgroundDownload() } std::unique_lock lock(mutex); current_offset += total_bytes; - file_meta_data.status = RemoteFileMetadata::DOWNLOADED; + file_status = DOWNLOADED; flush(true); data_file_writer.reset(); - remote_read_buffer.reset(); lock.unlock(); more_data_signal.notify_all(); - RemoteReadBufferCache::instance().updateTotalSize(file_meta_data.file_size); - LOG_TRACE(log, "Finish download into local path: {}, file meta data:{} ", local_path.string(), file_meta_data.toString()); + RemoteReadBufferCache::instance().updateTotalSize(file_meta_data_ptr->getFileSize()); + LOG_TRACE(log, "Finish download into local path: {}, file meta data:{} ", local_path.string(), file_meta_data_ptr->toString()); } -void RemoteCacheController::flush(bool need_flush_meta_data_) +void RemoteCacheController::flush(bool need_flush_status) { if (data_file_writer) { + LOG_DEBUG(&Poco::Logger::get("RemoteCacheController"),"flush file. offset:{}, file:{}. total_size:{}", current_offset, local_path.string(), file_meta_data_ptr->getFileSize()); data_file_writer->sync(); } - - if (!need_flush_meta_data_) - return; - - file_meta_data.save(local_path / "meta.txt"); + if (need_flush_status) + { + Poco::JSON::Object jobj; + jobj.set("file_status", static_cast(file_status)); + jobj.set("meta_data_class", meta_data_class); + std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + jobj.stringify(buf); + std::ofstream info_file(local_path / "info.txt"); + info_file << buf.str(); + info_file.close(); + } } RemoteCacheController::~RemoteCacheController() = default; @@ -200,7 +201,7 @@ RemoteCacheController::~RemoteCacheController() = default; void RemoteCacheController::close() { // delete directory - LOG_TRACE(log, "Removing the local cache. local path: {}, file meta data:{}", local_path.string(), file_meta_data.toString()); + LOG_TRACE(log, "Removing the local cache. local path: {}", local_path.string()); std::filesystem::remove_all(local_path); } @@ -233,7 +234,7 @@ void RemoteCacheController::deallocFile(std::unique_ptr throw Exception( ErrorCodes::BAD_ARGUMENTS, "Try to deallocate file with invalid handler remote path: {}, local path: {}", - file_meta_data.remote_path, + file_meta_data_ptr->getRemotePath(), local_path.string()); } opened_file_buffer_refs.erase(it); @@ -249,7 +250,7 @@ RemoteReadBuffer::~RemoteReadBuffer() file_cache_controller->deallocFile(std::move(file_buffer)); } -std::unique_ptr RemoteReadBuffer::create(ContextPtr context, const RemoteFileMetadata & remote_file_meta, std::unique_ptr read_buffer) +std::unique_ptr RemoteReadBuffer::create(ContextPtr context, RemoteFileMetaDataBasePtr remote_file_meta_data, std::unique_ptr read_buffer) { auto * log = &Poco::Logger::get("RemoteReadBuffer"); size_t buff_size = DBMS_DEFAULT_BUFFER_SIZE; @@ -266,13 +267,13 @@ std::unique_ptr RemoteReadBuffer::create(ContextPtr context, c if (buff_size == 0) buff_size = DBMS_DEFAULT_BUFFER_SIZE; - const auto & remote_path = remote_file_meta.remote_path; + auto remote_path = remote_file_meta_data->getRemotePath(); auto remote_read_buffer = std::make_unique(buff_size); auto * raw_readbuffer_ptr = read_buffer.release(); std::shared_ptr shared_readbuffer_ptr(raw_readbuffer_ptr); RemoteReadBufferCacheError error; - std::tie(remote_read_buffer->file_cache_controller, error) = RemoteReadBufferCache::instance().createReader(context, remote_file_meta, shared_readbuffer_ptr); + std::tie(remote_read_buffer->file_cache_controller, error) = RemoteReadBufferCache::instance().createReader(context, remote_file_meta_data, shared_readbuffer_ptr); if (remote_read_buffer->file_cache_controller == nullptr) { LOG_ERROR(log, "Failed to allocate local file for remote path: {}, reason: {}", remote_path, error); @@ -293,7 +294,10 @@ bool RemoteReadBuffer::nextImpl() if (file_buffer) { auto start_offset = file_buffer->getPosition(); - auto end_offset = file_buffer->internalBuffer().size(); + auto end_offset = start_offset + file_buffer->internalBuffer().size(); + LOG_DEBUG(&Poco::Logger::get("RemoteReadBuffer"), "nextImpl. start:{}, end:{}, file:{}, total_size:{}, remote_path:{}", + start_offset, end_offset, file_buffer->getFileName(), file_cache_controller->getFileMetaData()->getFileSize(), + file_cache_controller->getFileMetaData()->getRemotePath()); file_cache_controller->waitMoreData(start_offset, end_offset); auto status = file_buffer->next(); @@ -322,9 +326,11 @@ off_t RemoteReadBuffer::seek(off_t offset, int whence) { /* * Need to wait here. For example, the current file has been download at position X, but here we try to seek to - * postition Y ( Y > X), it would fail. + * postition Y (Y > X), it would fail. */ file_cache_controller->waitMoreData(offset, offset + file_buffer->internalBuffer().size()); + LOG_DEBUG(&Poco::Logger::get("RemoteReadBuffer"), "seek. offset:{}. file:{}, total_size:{}", offset, file_buffer->getFileName(), + file_cache_controller->getFileMetaData()->getFileSize()); auto ret = file_buffer->seek(offset, whence); BufferBase::set(file_buffer->buffer().begin(), file_buffer->buffer().size(), @@ -405,18 +411,18 @@ void RemoteReadBufferCache::initOnce( recover_task_holder->activateAndSchedule(); } -String RemoteReadBufferCache::calculateLocalPath(const RemoteFileMetadata & meta) const +String RemoteReadBufferCache::calculateLocalPath(RemoteFileMetaDataBasePtr meta_data) const { - String full_path = meta.schema + ":" + meta.cluster + ":" + meta.remote_path; + String full_path = meta_data->getSchema() + ":" + meta_data->getCluster() + ":" + meta_data->getRemotePath(); UInt128 hashcode = sipHash128(full_path.c_str(), full_path.size()); String hashcode_str = getHexUIntLowercase(hashcode); return fs::path(root_dir) / hashcode_str.substr(0, 3) / hashcode_str; } std::pair -RemoteReadBufferCache::createReader(ContextPtr context, const RemoteFileMetadata & remote_file_meta, std::shared_ptr & read_buffer) +RemoteReadBufferCache::createReader(ContextPtr context, RemoteFileMetaDataBasePtr remote_file_meta_data, std::shared_ptr & read_buffer) { - LOG_TRACE(log, "createReader. {} {} {}", remote_file_meta.remote_path, remote_file_meta.last_modification_timestamp, remote_file_meta.file_size); + LOG_TRACE(log, "createReader. {}", remote_file_meta_data->toString()); // If something is wrong on startup, rollback to read from the original ReadBuffer if (!isInitialized()) { @@ -424,15 +430,15 @@ RemoteReadBufferCache::createReader(ContextPtr context, const RemoteFileMetadata return {nullptr, RemoteReadBufferCacheError::NOT_INIT}; } - auto remote_path = remote_file_meta.remote_path; - const auto & last_modification_timestamp = remote_file_meta.last_modification_timestamp; - auto local_path = calculateLocalPath(remote_file_meta); + auto remote_path = remote_file_meta_data->getRemotePath(); + const auto & last_modification_timestamp = remote_file_meta_data->getLastModificationTimestamp(); + auto local_path = calculateLocalPath(remote_file_meta_data); std::lock_guard lock(mutex); auto cache_iter = caches.find(local_path); if (cache_iter != caches.end()) { // if the file has been update on remote side, we need to redownload it - if (cache_iter->second.cache_controller->getLastModificationTimestamp() != last_modification_timestamp) + if (cache_iter->second.cache_controller->checkFileChanged(remote_file_meta_data)) { LOG_TRACE( log, @@ -479,9 +485,9 @@ RemoteReadBufferCache::createReader(ContextPtr context, const RemoteFileMetadata fs::create_directories(local_path); - // pass a session context into RemoteCacheController is not a good idea auto cache_controller - = std::make_shared(context->getGlobalContext(), remote_file_meta, local_path, local_cache_bytes_read_before_flush, read_buffer); + = std::make_shared(remote_file_meta_data, local_path, local_cache_bytes_read_before_flush); + cache_controller->startBackgroundDownload(read_buffer, context->getSchedulePool()); CacheCell cache_cell; cache_cell.cache_controller = cache_controller; cache_cell.key_iterator = keys.insert(keys.end(), local_path); diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 20cb2eb8877..4bd1d758a41 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -16,6 +16,7 @@ #include #include #include +#include #include #include @@ -31,51 +32,20 @@ enum class RemoteReadBufferCacheError : int8_t END_OF_FILE = 20, }; -struct RemoteFileMetadata +class RemoteCacheController { - enum LocalStatus +public: + enum LocalFileStatus { TO_DOWNLOAD = 0, DOWNLOADING = 1, DOWNLOADED = 2, }; - RemoteFileMetadata(): last_modification_timestamp(0l), file_size(0), status(TO_DOWNLOAD){} - RemoteFileMetadata( - const String & schema_, - const String & cluster_, - const String & path_, - UInt64 last_modification_timestamp_, - size_t file_size_) - : schema(schema_) - , cluster(cluster_) - , remote_path(path_) - , last_modification_timestamp(last_modification_timestamp_) - , file_size(file_size_) - , status(TO_DOWNLOAD) - { - } - bool load(const std::filesystem::path & local_path); - void save(const std::filesystem::path & local_path) const; - String toString() const; - - String schema; // Hive, S2 etc. - String cluster; - String remote_path; - UInt64 last_modification_timestamp; - size_t file_size; - LocalStatus status; -}; - -class RemoteCacheController -{ -public: RemoteCacheController( - ContextPtr context, - const RemoteFileMetadata & file_meta_data_, + RemoteFileMetaDataBasePtr file_meta_data_, const std::filesystem::path & local_path_, - size_t cache_bytes_before_flush_, - std::shared_ptr read_buffer_); + size_t cache_bytes_before_flush_); ~RemoteCacheController(); // recover from local disk @@ -97,7 +67,7 @@ public: { std::lock_guard lock{mutex}; //return opened_file_streams.empty() && remote_read_buffer == nullptr; - return opened_file_buffer_refs.empty() && remote_read_buffer == nullptr; + return opened_file_buffer_refs.empty() && file_status == DOWNLOADED; } void close(); @@ -111,9 +81,10 @@ public: inline size_t size() const { return current_offset; } inline const std::filesystem::path & getLocalPath() { return local_path; } - inline const String & getRemotePath() const { return file_meta_data.remote_path; } + inline String getRemotePath() const { return file_meta_data_ptr->getRemotePath(); } - inline UInt64 getLastModificationTimestamp() const { return file_meta_data.last_modification_timestamp; } + inline UInt64 getLastModificationTimestamp() const { return file_meta_data_ptr->getLastModificationTimestamp(); } + bool checkFileChanged(RemoteFileMetaDataBasePtr file_meta_data_); inline void markInvalid() { std::lock_guard lock(mutex); @@ -124,29 +95,33 @@ public: std::lock_guard lock(mutex); return valid; } - const RemoteFileMetadata & getFileMetaData() { return file_meta_data; } + RemoteFileMetaDataBasePtr getFileMetaData() { return file_meta_data_ptr; } + + void startBackgroundDownload(std::shared_ptr input_readbuffer, BackgroundSchedulePool & thread_pool); private: - // flush file and meta info into disk - void flush(bool need_flush_meta_data_ = false); + // flush file and status information + void flush(bool need_flush_status = false); + bool loadInnerInformation(const std::filesystem::path & file_path); BackgroundSchedulePool::TaskHolder download_task_holder; - void backgroundDownload(); + void backgroundDownload(std::shared_ptr remote_read_buffer); std::mutex mutex; std::condition_variable more_data_signal; std::set opened_file_buffer_refs; // refer to a buffer address - // meta info - RemoteFileMetadata file_meta_data; + String meta_data_class; + LocalFileStatus file_status = TO_DOWNLOAD; // for tracking download process + RemoteFileMetaDataBasePtr file_meta_data_ptr; std::filesystem::path local_path; bool valid; size_t local_cache_bytes_read_before_flush; size_t current_offset; - std::shared_ptr remote_read_buffer; + //std::shared_ptr remote_read_buffer; std::unique_ptr data_file_writer; Poco::Logger * log = &Poco::Logger::get("RemoteCacheController"); @@ -162,13 +137,13 @@ class RemoteReadBuffer : public BufferWithOwnMemory public: explicit RemoteReadBuffer(size_t buff_size); ~RemoteReadBuffer() override; - static std::unique_ptr create(ContextPtr contex, const RemoteFileMetadata & remote_file_meta, std::unique_ptr read_buffer); + static std::unique_ptr create(ContextPtr contex, RemoteFileMetaDataBasePtr remote_file_meta_data, std::unique_ptr read_buffer); bool nextImpl() override; - inline bool seekable() { return !file_buffer && file_cache_controller->getFileMetaData().file_size > 0; } + inline bool seekable() { return !file_buffer && file_cache_controller->getFileMetaData()->getFileSize() > 0; } off_t seek(off_t off, int whence) override; off_t getPosition() override; - std::optional getTotalSize() override { return file_cache_controller->getFileMetaData().file_size; } + std::optional getTotalSize() override { return file_cache_controller->getFileMetaData()->getFileSize(); } private: std::shared_ptr file_cache_controller; @@ -190,7 +165,7 @@ public: inline bool isInitialized() const { return initialized; } std::pair - createReader(ContextPtr context, const RemoteFileMetadata & remote_file_meta, std::shared_ptr & read_buffer); + createReader(ContextPtr context, RemoteFileMetaDataBasePtr remote_file_meta_data, std::shared_ptr & read_buffer); void updateTotalSize(size_t size) { total_size += size; } @@ -217,7 +192,7 @@ private: std::list keys; std::map caches; - String calculateLocalPath(const RemoteFileMetadata & meta) const; + String calculateLocalPath(RemoteFileMetaDataBasePtr meta) const; BackgroundSchedulePool::TaskHolder recover_task_holder; void recoverTask(); diff --git a/src/Storages/Hive/HiveFileMetaData.cpp b/src/Storages/Hive/HiveFileMetaData.cpp new file mode 100644 index 00000000000..5045e4f0364 --- /dev/null +++ b/src/Storages/Hive/HiveFileMetaData.cpp @@ -0,0 +1,53 @@ +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +HiveFileMetaData::~HiveFileMetaData() = default; + +String HiveFileMetaData::toString() +{ + Poco::JSON::Object jobj; + jobj.set("schema", schema); + jobj.set("cluster", cluster); + jobj.set("remote_path", remote_path); + jobj.set("last_modification_timestamp", last_modification_timestamp); + jobj.set("file_size", file_size); + std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + jobj.stringify(buf); + return buf.str(); + +} + +bool HiveFileMetaData::fromString(const String &buf) +{ + std::stringstream istream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + istream << buf; + Poco::JSON::Parser parser; + auto jobj = parser.parse(istream).extract(); + remote_path = jobj->get("remote_path").convert(); + schema = jobj->get("schema").convert(); + cluster = jobj->get("cluster").convert(); + last_modification_timestamp = jobj->get("last_modification_timestamp").convert(); + file_size =jobj->get("file_size").convert(); + return true; +} + +bool HiveFileMetaData::equal(RemoteFileMetaDataBasePtr meta_data) +{ + auto real_meta_data = std::dynamic_pointer_cast(meta_data); + if (!real_meta_data) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid meta data class"); + return last_modification_timestamp == real_meta_data->last_modification_timestamp; +} + +REGISTTER_REMOTE_FILE_META_DATA_CLASS(HiveFileMetaData) + +} diff --git a/src/Storages/Hive/HiveFileMetaData.h b/src/Storages/Hive/HiveFileMetaData.h new file mode 100644 index 00000000000..c56fa4d1727 --- /dev/null +++ b/src/Storages/Hive/HiveFileMetaData.h @@ -0,0 +1,29 @@ +#pragma once +#include +namespace DB +{ +class HiveFileMetaData : public RemoteFileMetaDataBase +{ +public: + HiveFileMetaData() = default; + HiveFileMetaData(const String & schema_, + const String & cluster_, + const String & remote_path_, + size_t file_size_, + UInt64 last_modification_timestamp_): + RemoteFileMetaDataBase(schema_, cluster_, remote_path_, file_size_, last_modification_timestamp_){} + ~HiveFileMetaData() override; + + String getClassName() override { return "HiveFileMetaData"; } + + RemoteFileMetaDataBasePtr clone() override + { + auto result = std::make_shared(schema, cluster, remote_path, file_size, last_modification_timestamp); + return result; + } + String toString() override; + bool fromString(const String &buf) override; + bool equal(RemoteFileMetaDataBasePtr meta_data) override; + +}; +} diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index d0a0991a16a..54d692e48a6 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -38,6 +38,7 @@ #include #include #include +#include #include #include @@ -168,9 +169,10 @@ public: // Use local cache for remote filesystem if enabled. std::unique_ptr remote_read_buf; - if (RemoteReadBufferCache::instance().isInitialized() && getContext()->getSettingsRef().use_local_cache_for_remote_fs) + bool x = false; + if (RemoteReadBufferCache::instance().isInitialized() && getContext()->getSettingsRef().use_local_cache_for_remote_fs && x) remote_read_buf = RemoteReadBuffer::create(getContext(), - {"Hive", getNameNodeCluster(hdfs_namenode_url), uri_with_path, curr_file->getLastModTs(), curr_file->getSize()}, + std::make_shared("Hive", getNameNodeCluster(hdfs_namenode_url), uri_with_path, curr_file->getLastModTs(), curr_file->getSize()), std::move(raw_read_buf)); else remote_read_buf = std::move(raw_read_buf); From 3c56a5deacca2d2dc3a4a745a352378b6d383299 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 3 Dec 2021 17:29:35 +0800 Subject: [PATCH 0123/1260] refector 1. Make the lru cache pilicy in RemoteReadBufferCache into Common/UnreleasableLRUCache.h 2. If try to use local cached file fail, the RemoteReadBuffer::create() return the original ReadBuffer 3. Remove LocalFileReader 4. RemoteCacheController does not hold a context now, and the download process starts in RemoteReadBufferCache once a new RemoteCacheController is create successly 5. Make an abstract class RemoteFileMetaDataBase for descripting a remote file meta data. 6. Remote file meta data need to a version, can cache multi-version for a file now --- src/Common/UnreleasableLRUCache.h | 283 +++++++++++++++++++++++++ src/IO/RemoteFileMetaDataBase.h | 31 ++- src/IO/RemoteReadBufferCache.cpp | 213 ++++++------------- src/IO/RemoteReadBufferCache.h | 49 +++-- src/Storages/Hive/HiveFileMetaData.cpp | 14 +- src/Storages/Hive/HiveFileMetaData.h | 12 +- src/Storages/Hive/StorageHive.cpp | 5 +- 7 files changed, 408 insertions(+), 199 deletions(-) create mode 100644 src/Common/UnreleasableLRUCache.h diff --git a/src/Common/UnreleasableLRUCache.h b/src/Common/UnreleasableLRUCache.h new file mode 100644 index 00000000000..3b5eb27543e --- /dev/null +++ b/src/Common/UnreleasableLRUCache.h @@ -0,0 +1,283 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +#include +#include + + +namespace DB +{ + +template +struct TrivialUnreleasableLRUCacheWeightFunction +{ + size_t operator()(const T &) const + { + return 1; + } +}; + +enum class CacheEvictStatus +{ + CAN_EVITCT, // a key can be evicted + TERMINATE_EVICT, // stop the evicting process + SKIP_EVICT, // skip current value and keep iterating +}; + +template +struct TrivialUnreleasableLRUCacheEvitPolicy +{ + CacheEvictStatus canRelease(const T &) + { + return CacheEvictStatus::CAN_EVITCT; + } + + void release(T & ) + { + } +}; + +/* + * Another version LRU Cache。 + * A value can only be evicted or be updated if it is releasable. If there is no releasable value, + * insert or update will fail. + */ +template , + typename WeightFunction = TrivialUnreleasableLRUCacheWeightFunction, + typename EvictPolicy = TrivialUnreleasableLRUCacheEvitPolicy> +class UnreleasableLRUCache +{ +public: + using Key = TKey; + using Mapped = TMapped; + using MappedPtr = std::shared_ptr; + + /** Initialize LRUCache with max_size and max_elements_size. + * max_elements_size == 0 means no elements size restrictions. + */ + UnreleasableLRUCache(size_t max_size_, size_t max_elements_size_ = 0) + : max_size(std::max(static_cast(1), max_size_)) + , max_elements_size(max_elements_size_) + {} + + MappedPtr get(const Key & key) + { + std::lock_guard lock(mutex); + + auto res = getImpl(key, lock); + if (res) + ++hits; + else + ++misses; + return res; + } + + /* + * Fail on two cases + * 1) the key exists, but the old value is not releasable + * 2) the key not exists, but there is not enough space for it after trying to evict some least recently used values. + */ + bool set(const Key & key, const MappedPtr & mapped) + { + std::lock_guard lock(mutex); + + return setImpl(key, mapped, lock); + } + + void getStats(size_t & out_hits, size_t & out_misses) const + { + std::lock_guard lock(mutex); + out_hits = hits; + out_misses = misses; + } + + size_t weight() const + { + std::lock_guard lock(mutex); + return current_size; + } + + size_t count() const + { + std::lock_guard lock(mutex); + return cells.size(); + } + + size_t maxSize() const + { + return max_size; + } + + void reset() + { + std::lock_guard lock(mutex); + queue.clear(); + cells.clear(); + current_size = 0; + hits = 0; + misses = 0; + } + + virtual ~UnreleasableLRUCache() {} + +protected: + using LRUQueue = std::list; + using LRUQueueIterator = typename LRUQueue::iterator; + + struct Cell + { + MappedPtr value; + size_t size; + LRUQueueIterator queue_iterator; + }; + + using Cells = std::unordered_map; + + Cells cells; + + mutable std::mutex mutex; +private: + + LRUQueue queue; + + /// Total weight of values. + size_t current_size = 0; + const size_t max_size; + const size_t max_elements_size; + + std::atomic hits {0}; + std::atomic misses {0}; + + WeightFunction weight_function; + EvictPolicy evict_policy; + + MappedPtr getImpl(const Key & key, [[maybe_unused]] std::lock_guard & cache_lock) + { + auto it = cells.find(key); + if (it == cells.end()) + { + return MappedPtr(); + } + + Cell & cell = it->second; + + /// Move the key to the end of the queue. The iterator remains valid. + queue.splice(queue.end(), queue, cell.queue_iterator); + + return cell.value; + } + + bool setImpl(const Key & key, const MappedPtr & mapped, [[maybe_unused]] std::lock_guard & cache_lock) + { + auto [it, inserted] = cells.emplace(std::piecewise_construct, + std::forward_as_tuple(key), + std::forward_as_tuple()); + + Cell & cell = it->second; + + if (inserted) + { + auto weight = mapped ? weight_function(*mapped) : 0; + if (!removeOverflow(weight)) + { + // cannot insert this new value + cells.erase(it); + return false; + } + + try + { + cell.queue_iterator = queue.insert(queue.end(), key); + } + catch (...) + { + cells.erase(it); + throw; + } + } + else + { + if (evict_policy.canRelease(*cell.value) != CacheEvictStatus::CAN_EVITCT) + { + // the old value is not releasable + return false; + } + evict_policy.release(*cell.value); + current_size -= cell.size; + queue.splice(queue.end(), queue, cell.queue_iterator); + } + + cell.value = mapped; + cell.size = cell.value ? weight_function(*cell.value) : 0; + current_size += cell.size; + + removeOverflow(0); + return true; + } + + // Or make your own implementation + virtual bool removeOverflow(size_t more_size) + { + size_t current_weight_lost = 0; + size_t queue_size = cells.size(); + + auto key_it = queue.begin(); + while ((current_size + more_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) && (queue_size > 1) + && key_it != queue.end()) + { + const Key & key = *key_it; + + auto it = cells.find(key); + if (it == cells.end()) + { + LOG_ERROR(&Poco::Logger::get("UnreleasableLRUCache"), "UnreleasableLRUCache became inconsistent. There must be a bug in it."); + abort(); + } + + const auto & cell = it->second; + auto cache_evict_status = evict_policy.canRelease(*(cell.value)); + if (cache_evict_status == CacheEvictStatus::CAN_EVITCT) + { + evict_policy.release(*(cell.value)); + current_size -= cell.size; + current_weight_lost += cell.size; + + cells.erase(it); + key_it = queue.erase(key_it); + --queue_size; + } + else if (cache_evict_status == CacheEvictStatus::SKIP_EVICT) + { + key_it++; + continue; + } + else if (cache_evict_status == CacheEvictStatus::TERMINATE_EVICT) + { + break; + } + } + + onRemoveOverflowWeightLoss(current_weight_lost); + + if (current_size > (1ull << 63)) + { + LOG_ERROR(&Poco::Logger::get("UnreleasableLRUCache"), "UnreleasableLRUCache became inconsistent. There must be a bug in it."); + abort(); + } + return !(current_size + more_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)); + } + + /// Override this method if you want to track how much weight was lost in removeOverflow method. + virtual void onRemoveOverflowWeightLoss(size_t /*weight_loss*/) {} +}; + + +} diff --git a/src/IO/RemoteFileMetaDataBase.h b/src/IO/RemoteFileMetaDataBase.h index d2acbd0af96..51706f0a8d7 100644 --- a/src/IO/RemoteFileMetaDataBase.h +++ b/src/IO/RemoteFileMetaDataBase.h @@ -24,22 +24,21 @@ public: { } virtual ~RemoteFileMetaDataBase(); - virtual String getClassName() = 0; //class name + virtual String getClassName() const = 0; //class name // methods for basic information - inline String getSchema() { return schema; } - inline String getCluster() { return cluster; } - inline size_t getFileSize() { return file_size; } - inline String getRemotePath() { return remote_path; } - inline UInt64 getLastModificationTimestamp() { return last_modification_timestamp; } - // create a new object - virtual std::shared_ptr clone() = 0; + inline String getSchema() const { return schema; } + inline String getCluster() const { return cluster; } + inline size_t getFileSize() const { return file_size; } + inline String getRemotePath() const { return remote_path; } + inline UInt64 getLastModificationTimestamp() const { return last_modification_timestamp; } // deserialize virtual bool fromString(const String &buf) = 0; // serialize - virtual String toString() = 0; - // to compare two meta datas for detecting file changes - virtual bool equal(std::shared_ptr b) = 0; + virtual String toString() const = 0; + + // used for comparing two file meta datas are the same or not. + virtual String getVersion() const = 0; protected: String schema; String cluster; @@ -51,11 +50,11 @@ protected: using RemoteFileMetaDataBasePtr = std::shared_ptr; /* - * How to register a subclass into factory and use it ? - * 1) define your own subclass derive from RemoteFileMetaDataBase. Notice! the getClassName() must be the same - as your subclass name. - * 2) in a .cpp file, call REGISTTER_REMOTE_FILE_META_DATA_CLASS(subclass). - 3) call RemoteFileMetaDataFactory::instance().create_class(subclass_name) where you want to make a new object + * How to register a subclass into the factory and use it ? + * 1) define your own subclass derive from RemoteFileMetaDataBase. Notice! the getClassName() must be the same + * as your subclass name. + * 2) in a .cpp file, call REGISTTER_REMOTE_FILE_META_DATA_CLASS(subclass), + * 3) call RemoteFileMetaDataFactory::instance().create_class(subclass_name) where you want to make a new object */ class RemoteFileMetaDataFactory : private boost::noncopyable diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 3a482e90bad..6dd4d4be66c 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -53,18 +53,19 @@ std::shared_ptr RemoteCacheController::recover(const std: if (!cache_controller->loadInnerInformation(local_path_ / "info.txt") || cache_controller->file_status != DOWNLOADED) { - LOG_INFO(log, "recover cached file failed. local path:{}", local_path_.string()); + LOG_INFO(log, "Recover cached file failed. local path:{}", local_path_.string()); return nullptr; } cache_controller->file_meta_data_ptr = RemoteFileMetaDataFactory::instance().create_class(cache_controller->meta_data_class); if (!cache_controller->file_meta_data_ptr) { - // do not load this invalid cached file and clear it + // do not load this invalid cached file and clear it. the clear action is in + // RemoteReadBufferCache::recoverCachedFilesMetaData(), because deleting directories during iteration will + // cause unexpected behaviors LOG_ERROR(log, "Cannot create the meta data class : {}. The cached file is invalid and will be remove. path:{}", cache_controller->meta_data_class, local_path_.string()); - fs::remove_all(local_path_); return nullptr; } std::ifstream meta_data_file(local_path_ / "meta_data.txt"); @@ -73,7 +74,6 @@ std::shared_ptr RemoteCacheController::recover(const std: { LOG_ERROR(log, "Cannot load the meta data. The cached file is invalid and will be remove. path:{}", local_path_.string()); - fs::remove_all(local_path_); return nullptr; } @@ -93,6 +93,8 @@ RemoteCacheController::RemoteCacheController( , local_cache_bytes_read_before_flush(cache_bytes_before_flush_) , current_offset(0) { + // on recover, file_meta_data_ptr is null, but it will be allocated after loading from meta_data.txt + // when we allocate a whole new file cache , file_meta_data_ptr must not be null. if (file_meta_data_ptr) { std::ofstream meta_data_file(local_path_ / "meta_data.txt", std::ios::out); @@ -129,7 +131,7 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs bool RemoteCacheController::checkFileChanged(RemoteFileMetaDataBasePtr file_meta_data_) { - return !file_meta_data_ptr->equal(file_meta_data_); + return !(file_meta_data_ptr->getVersion() == file_meta_data_->getVersion()); } void RemoteCacheController::startBackgroundDownload(std::shared_ptr input_readbuffer, BackgroundSchedulePool & thread_pool) @@ -180,7 +182,6 @@ void RemoteCacheController::flush(bool need_flush_status) { if (data_file_writer) { - LOG_DEBUG(&Poco::Logger::get("RemoteCacheController"),"flush file. offset:{}, file:{}. total_size:{}", current_offset, local_path.string(), file_meta_data_ptr->getFileSize()); data_file_writer->sync(); } if (need_flush_status) @@ -196,7 +197,11 @@ void RemoteCacheController::flush(bool need_flush_status) } } -RemoteCacheController::~RemoteCacheController() = default; +RemoteCacheController::~RemoteCacheController() +{ + if (download_task_holder) + download_task_holder->deactivate(); +} void RemoteCacheController::close() { @@ -240,17 +245,17 @@ void RemoteCacheController::deallocFile(std::unique_ptr opened_file_buffer_refs.erase(it); } -// the size need be equal to the original buffer RemoteReadBuffer::RemoteReadBuffer(size_t buff_size) : BufferWithOwnMemory(buff_size) { } RemoteReadBuffer::~RemoteReadBuffer() { - file_cache_controller->deallocFile(std::move(file_buffer)); + if (file_cache_controller) + file_cache_controller->deallocFile(std::move(file_buffer)); } -std::unique_ptr RemoteReadBuffer::create(ContextPtr context, RemoteFileMetaDataBasePtr remote_file_meta_data, std::unique_ptr read_buffer) +std::unique_ptr RemoteReadBuffer::create(ContextPtr context, RemoteFileMetaDataBasePtr remote_file_meta_data, std::unique_ptr read_buffer) { auto * log = &Poco::Logger::get("RemoteReadBuffer"); size_t buff_size = DBMS_DEFAULT_BUFFER_SIZE; @@ -269,15 +274,14 @@ std::unique_ptr RemoteReadBuffer::create(ContextPtr context, R auto remote_path = remote_file_meta_data->getRemotePath(); auto remote_read_buffer = std::make_unique(buff_size); - auto * raw_readbuffer_ptr = read_buffer.release(); - std::shared_ptr shared_readbuffer_ptr(raw_readbuffer_ptr); RemoteReadBufferCacheError error; - std::tie(remote_read_buffer->file_cache_controller, error) = RemoteReadBufferCache::instance().createReader(context, remote_file_meta_data, shared_readbuffer_ptr); + std::tie(remote_read_buffer->file_cache_controller, read_buffer, error) = RemoteReadBufferCache::instance().createReader(context, remote_file_meta_data, read_buffer); if (remote_read_buffer->file_cache_controller == nullptr) { - LOG_ERROR(log, "Failed to allocate local file for remote path: {}, reason: {}", remote_path, error); - remote_read_buffer->original_read_buffer = shared_readbuffer_ptr; + LOG_ERROR(log, "Failed to allocate local file for remote path: {}, reason: {}.", remote_path, error); + // read_buffer is the input one. + return read_buffer; } else { @@ -286,51 +290,33 @@ std::unique_ptr RemoteReadBuffer::create(ContextPtr context, R throw Exception(ErrorCodes::LOGICAL_ERROR, "Create file readbuffer failed. {}", remote_read_buffer->file_cache_controller->getLocalPath().string()); } + remote_read_buffer->remote_file_size = remote_file_meta_data->getFileSize(); return remote_read_buffer; } bool RemoteReadBuffer::nextImpl() { - if (file_buffer) - { - auto start_offset = file_buffer->getPosition(); - auto end_offset = start_offset + file_buffer->internalBuffer().size(); - LOG_DEBUG(&Poco::Logger::get("RemoteReadBuffer"), "nextImpl. start:{}, end:{}, file:{}, total_size:{}, remote_path:{}", - start_offset, end_offset, file_buffer->getFileName(), file_cache_controller->getFileMetaData()->getFileSize(), - file_cache_controller->getFileMetaData()->getRemotePath()); - file_cache_controller->waitMoreData(start_offset, end_offset); + auto start_offset = file_buffer->getPosition(); + auto end_offset = start_offset + file_buffer->internalBuffer().size(); + file_cache_controller->waitMoreData(start_offset, end_offset); - auto status = file_buffer->next(); - if (status) - BufferBase::set(file_buffer->buffer().begin(), - file_buffer->buffer().size(), - file_buffer->offset()); - return status; - } - else - { - // In the case we cannot use local cache, read from the original readbuffer directly - if (!original_read_buffer) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Original read buffer is not initialized. It's a bug"); - - auto status = original_read_buffer->next(); - // We don't need to worry about the memory buffer allocated in RemoteReadBuffer, since it is owned by - // BufferWithOwnMemory, BufferWithOwnMemory would release it. - if (status) - BufferBase::set(original_read_buffer->buffer().begin(), original_read_buffer->buffer().size(), original_read_buffer->offset()); - return status; - } + auto status = file_buffer->next(); + if (status) + BufferBase::set(file_buffer->buffer().begin(), + file_buffer->buffer().size(), + file_buffer->offset()); + return status; } off_t RemoteReadBuffer::seek(off_t offset, int whence) { + if (!file_buffer) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot call seek() in this buffer. It's a bug!"); /* * Need to wait here. For example, the current file has been download at position X, but here we try to seek to * postition Y (Y > X), it would fail. */ file_cache_controller->waitMoreData(offset, offset + file_buffer->internalBuffer().size()); - LOG_DEBUG(&Poco::Logger::get("RemoteReadBuffer"), "seek. offset:{}. file:{}, total_size:{}", offset, file_buffer->getFileName(), - file_cache_controller->getFileMetaData()->getFileSize()); auto ret = file_buffer->seek(offset, whence); BufferBase::set(file_buffer->buffer().begin(), file_buffer->buffer().size(), @@ -360,15 +346,24 @@ void RemoteReadBufferCache::recoverCachedFilesMetaData( { if (current_depth >= max_depth) { + std::vector invalid_pathes; for (auto const & dir : fs::directory_iterator{current_path}) { String path = dir.path(); auto cache_controller = RemoteCacheController::recover(path); if (!cache_controller) + { + invalid_pathes.emplace_back(path); continue; - auto & cell = caches[path]; - cell.cache_controller = cache_controller; - cell.key_iterator = keys.insert(keys.end(), path); + } + if (!lru_caches->set(path, cache_controller)) + { + invalid_pathes.emplace_back(path); + } + } + for (auto & path : invalid_pathes) + { + fs::remove_all(path); } return; } @@ -384,7 +379,7 @@ void RemoteReadBufferCache::recoverTask() std::lock_guard lock(mutex); recoverCachedFilesMetaData(root_dir, 1, 2); initialized = true; - LOG_TRACE(log, "Recovered from directory:{}", root_dir); + LOG_INFO(log, "Recovered from directory:{}", root_dir); } void RemoteReadBufferCache::initOnce( @@ -394,8 +389,8 @@ void RemoteReadBufferCache::initOnce( LOG_INFO( log, "Initializing local cache for remote data sources. Local cache root path: {}, cache size limit: {}", root_dir_, limit_size_); root_dir = root_dir_; - limit_size = limit_size_; local_cache_bytes_read_before_flush = bytes_read_before_flush_; + lru_caches = std::make_unique(limit_size_); /// create if root_dir not exists if (!fs::exists(fs::path(root_dir) / "")) @@ -413,130 +408,62 @@ void RemoteReadBufferCache::initOnce( String RemoteReadBufferCache::calculateLocalPath(RemoteFileMetaDataBasePtr meta_data) const { - String full_path = meta_data->getSchema() + ":" + meta_data->getCluster() + ":" + meta_data->getRemotePath(); + // add version into the full_path, and not block to read the new version + String full_path = meta_data->getSchema() + ":" + meta_data->getCluster() + ":" + meta_data->getRemotePath() + + ":" + meta_data->getVersion(); UInt128 hashcode = sipHash128(full_path.c_str(), full_path.size()); String hashcode_str = getHexUIntLowercase(hashcode); return fs::path(root_dir) / hashcode_str.substr(0, 3) / hashcode_str; } -std::pair -RemoteReadBufferCache::createReader(ContextPtr context, RemoteFileMetaDataBasePtr remote_file_meta_data, std::shared_ptr & read_buffer) +std::tuple, RemoteReadBufferCacheError> +RemoteReadBufferCache::createReader(ContextPtr context, RemoteFileMetaDataBasePtr remote_file_meta_data, std::unique_ptr & read_buffer) { - LOG_TRACE(log, "createReader. {}", remote_file_meta_data->toString()); // If something is wrong on startup, rollback to read from the original ReadBuffer if (!isInitialized()) { LOG_ERROR(log, "RemoteReadBufferCache has not been initialized"); - return {nullptr, RemoteReadBufferCacheError::NOT_INIT}; + return {nullptr, std::move(read_buffer), RemoteReadBufferCacheError::NOT_INIT}; } auto remote_path = remote_file_meta_data->getRemotePath(); const auto & last_modification_timestamp = remote_file_meta_data->getLastModificationTimestamp(); auto local_path = calculateLocalPath(remote_file_meta_data); std::lock_guard lock(mutex); - auto cache_iter = caches.find(local_path); - if (cache_iter != caches.end()) + auto cache = lru_caches->get(local_path); + if (cache) { - // if the file has been update on remote side, we need to redownload it - if (cache_iter->second.cache_controller->checkFileChanged(remote_file_meta_data)) + // the remote file has been updated, need to redownload + if (!cache->isValid() || cache->checkFileChanged(remote_file_meta_data)) { LOG_TRACE( log, "Remote file ({}) has been updated. Last saved modification time: {}, actual last modification time: {}", remote_path, - std::to_string(cache_iter->second.cache_controller->getLastModificationTimestamp()), + std::to_string(cache->getLastModificationTimestamp()), std::to_string(last_modification_timestamp)); - cache_iter->second.cache_controller->markInvalid(); + cache->markInvalid(); } else { - // move the key to the list end - keys.splice(keys.end(), keys, cache_iter->second.key_iterator); - return { cache_iter->second.cache_controller, RemoteReadBufferCacheError::OK}; + return {cache, nullptr, RemoteReadBufferCacheError::OK}; } } - auto clear_ret = clearLocalCache(); - cache_iter = caches.find(local_path); - if (cache_iter != caches.end()) + if (!fs::exists(local_path)) + fs::create_directories(local_path); + + // cache is not found or is invalid + auto new_cache = std::make_shared(remote_file_meta_data, local_path, local_cache_bytes_read_before_flush); + if (!lru_caches->set(local_path, new_cache)) { - if (cache_iter->second.cache_controller->isValid()) - { - keys.splice(keys.end(), keys, cache_iter->second.key_iterator); - return { - cache_iter->second.cache_controller, - RemoteReadBufferCacheError::OK}; - } - else - { - // maybe someone is holding this file - LOG_INFO(log, "The remote file {} has been updated, but the previous readers do not finish reading.", - remote_path); - return {nullptr, RemoteReadBufferCacheError::FILE_INVALID}; - } + LOG_ERROR(log, "Insert the new cache failed. new file size:{}, current total size:{}", + remote_file_meta_data->getFileSize(), + lru_caches->weight()); + return {nullptr, std::move(read_buffer), RemoteReadBufferCacheError::DISK_FULL}; } - - // reach the disk capacity limit - if (!clear_ret) - { - LOG_INFO(log, "Reached local cache capacity limit size ({})", limit_size); - return {nullptr, RemoteReadBufferCacheError::DISK_FULL}; - } - - fs::create_directories(local_path); - - auto cache_controller - = std::make_shared(remote_file_meta_data, local_path, local_cache_bytes_read_before_flush); - cache_controller->startBackgroundDownload(read_buffer, context->getSchedulePool()); - CacheCell cache_cell; - cache_cell.cache_controller = cache_controller; - cache_cell.key_iterator = keys.insert(keys.end(), local_path); - caches[local_path] = cache_cell; - - return {cache_controller, RemoteReadBufferCacheError::OK}; -} - -bool RemoteReadBufferCache::clearLocalCache() -{ - // clear closable cache from the list head - for (auto it = keys.begin(); it != keys.end();) - { - auto cache_it = caches.find(*it); - if (cache_it == caches.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Found no entry in local cache with key: {}", *it); - - auto cache_controller = cache_it->second.cache_controller; - if (!cache_controller->isValid() && cache_controller->closable()) - { - LOG_TRACE(log, "Clear invalid cache entry with key {} from local cache", *it); - total_size - = total_size > cache_it->second.cache_controller->size() ? total_size - cache_it->second.cache_controller->size() : 0; - cache_controller->close(); - it = keys.erase(it); - caches.erase(cache_it); - continue; - } - - // if enough disk space is release, just to iterate the remained caches and clear the invalid ones. - if (total_size > limit_size && cache_controller->closable()) - { - total_size = total_size > cache_controller->size() ? total_size - cache_controller->size() : 0; - cache_controller->close(); - caches.erase(cache_it); - it = keys.erase(it); - LOG_TRACE( - log, - "clear local file {} for {}. key size:{}. next{}", - cache_controller->getLocalPath().string(), - cache_controller->getRemotePath(), - keys.size(), - *it); - } - else - it++; - } - LOG_TRACE(log, "After clear local cache, keys size:{}, total_size:{}, limit size:{}", keys.size(), total_size, limit_size); - return total_size < limit_size; + new_cache->startBackgroundDownload(std::move(read_buffer), context->getSchedulePool()); + return {new_cache, nullptr, RemoteReadBufferCacheError::OK}; } } diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 4bd1d758a41..8f524c236c2 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -96,6 +97,7 @@ public: return valid; } RemoteFileMetaDataBasePtr getFileMetaData() { return file_meta_data_ptr; } + inline size_t getFileSize() const { return file_meta_data_ptr->getFileSize(); } void startBackgroundDownload(std::shared_ptr input_readbuffer, BackgroundSchedulePool & thread_pool); @@ -137,25 +139,47 @@ class RemoteReadBuffer : public BufferWithOwnMemory public: explicit RemoteReadBuffer(size_t buff_size); ~RemoteReadBuffer() override; - static std::unique_ptr create(ContextPtr contex, RemoteFileMetaDataBasePtr remote_file_meta_data, std::unique_ptr read_buffer); + static std::unique_ptr create(ContextPtr contex, RemoteFileMetaDataBasePtr remote_file_meta_data, std::unique_ptr read_buffer); bool nextImpl() override; - inline bool seekable() { return !file_buffer && file_cache_controller->getFileMetaData()->getFileSize() > 0; } + inline bool seekable() { return file_buffer && remote_file_size > 0; } off_t seek(off_t off, int whence) override; off_t getPosition() override; - std::optional getTotalSize() override { return file_cache_controller->getFileMetaData()->getFileSize(); } + std::optional getTotalSize() override { return remote_file_size; } private: std::shared_ptr file_cache_controller; std::unique_ptr file_buffer; + size_t remote_file_size = 0; +}; - // in case local cache don't work, this buffer is setted; - std::shared_ptr original_read_buffer; +struct RemoteFileCacheWeightFunction +{ + size_t operator()(const RemoteCacheController & cache) const + { + return cache.getFileSize(); + } +}; + +struct RemoteFileCacheEvictPolicy +{ + CacheEvictStatus canRelease(RemoteCacheController & cache) const + { + if (cache.closable()) + return CacheEvictStatus::CAN_EVITCT; + return CacheEvictStatus::SKIP_EVICT; + } + void release(RemoteCacheController & cache) + { + cache.close(); + } }; class RemoteReadBufferCache { public: + using CacheType = UnreleasableLRUCache, + RemoteFileCacheWeightFunction, RemoteFileCacheEvictPolicy>; ~RemoteReadBufferCache(); // global instance static RemoteReadBufferCache & instance(); @@ -164,8 +188,8 @@ public: inline bool isInitialized() const { return initialized; } - std::pair - createReader(ContextPtr context, RemoteFileMetaDataBasePtr remote_file_meta_data, std::shared_ptr & read_buffer); + std::tuple, RemoteReadBufferCacheError> + createReader(ContextPtr context, RemoteFileMetaDataBasePtr remote_file_meta_data, std::unique_ptr & read_buffer); void updateTotalSize(size_t size) { total_size += size; } @@ -175,23 +199,15 @@ protected: private: // root directory of local cache for remote filesystem String root_dir; - size_t limit_size = 0; size_t local_cache_bytes_read_before_flush = 0; std::atomic initialized = false; std::atomic total_size; std::mutex mutex; + std::unique_ptr lru_caches; Poco::Logger * log = &Poco::Logger::get("RemoteReadBufferCache"); - struct CacheCell - { - std::list::iterator key_iterator; - std::shared_ptr cache_controller; - }; - std::list keys; - std::map caches; - String calculateLocalPath(RemoteFileMetaDataBasePtr meta) const; BackgroundSchedulePool::TaskHolder recover_task_holder; @@ -200,7 +216,6 @@ private: const std::filesystem::path & current_path, size_t current_depth, size_t max_depth); - bool clearLocalCache(); }; } diff --git a/src/Storages/Hive/HiveFileMetaData.cpp b/src/Storages/Hive/HiveFileMetaData.cpp index 5045e4f0364..7bb2ba760b0 100644 --- a/src/Storages/Hive/HiveFileMetaData.cpp +++ b/src/Storages/Hive/HiveFileMetaData.cpp @@ -5,14 +5,9 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - HiveFileMetaData::~HiveFileMetaData() = default; -String HiveFileMetaData::toString() +String HiveFileMetaData::toString() const { Poco::JSON::Object jobj; jobj.set("schema", schema); @@ -40,12 +35,9 @@ bool HiveFileMetaData::fromString(const String &buf) return true; } -bool HiveFileMetaData::equal(RemoteFileMetaDataBasePtr meta_data) +String HiveFileMetaData::getVersion() const { - auto real_meta_data = std::dynamic_pointer_cast(meta_data); - if (!real_meta_data) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid meta data class"); - return last_modification_timestamp == real_meta_data->last_modification_timestamp; + return std::to_string(getLastModificationTimestamp()); } REGISTTER_REMOTE_FILE_META_DATA_CLASS(HiveFileMetaData) diff --git a/src/Storages/Hive/HiveFileMetaData.h b/src/Storages/Hive/HiveFileMetaData.h index c56fa4d1727..52890b93333 100644 --- a/src/Storages/Hive/HiveFileMetaData.h +++ b/src/Storages/Hive/HiveFileMetaData.h @@ -14,16 +14,10 @@ public: RemoteFileMetaDataBase(schema_, cluster_, remote_path_, file_size_, last_modification_timestamp_){} ~HiveFileMetaData() override; - String getClassName() override { return "HiveFileMetaData"; } + String getClassName() const override { return "HiveFileMetaData"; } - RemoteFileMetaDataBasePtr clone() override - { - auto result = std::make_shared(schema, cluster, remote_path, file_size, last_modification_timestamp); - return result; - } - String toString() override; + String toString() const override; bool fromString(const String &buf) override; - bool equal(RemoteFileMetaDataBasePtr meta_data) override; - + String getVersion() const override; }; } diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 54d692e48a6..259763aca1c 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -169,10 +169,9 @@ public: // Use local cache for remote filesystem if enabled. std::unique_ptr remote_read_buf; - bool x = false; - if (RemoteReadBufferCache::instance().isInitialized() && getContext()->getSettingsRef().use_local_cache_for_remote_fs && x) + if (RemoteReadBufferCache::instance().isInitialized() && getContext()->getSettingsRef().use_local_cache_for_remote_fs) remote_read_buf = RemoteReadBuffer::create(getContext(), - std::make_shared("Hive", getNameNodeCluster(hdfs_namenode_url), uri_with_path, curr_file->getLastModTs(), curr_file->getSize()), + std::make_shared("Hive", getNameNodeCluster(hdfs_namenode_url), uri_with_path, curr_file->getSize(), curr_file->getLastModTs()), std::move(raw_read_buf)); else remote_read_buf = std::move(raw_read_buf); From ddac2a33bdbfa7c6f05f01cf50773e96a86bee4b Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 6 Dec 2021 16:46:13 +0800 Subject: [PATCH 0124/1260] fixed code style --- src/IO/RemoteFileMetaDataBase.cpp | 4 ++-- src/IO/RemoteFileMetaDataBase.h | 8 ++++---- src/IO/RemoteReadBufferCache.cpp | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/IO/RemoteFileMetaDataBase.cpp b/src/IO/RemoteFileMetaDataBase.cpp index e041bdce29a..40653a68bcb 100644 --- a/src/IO/RemoteFileMetaDataBase.cpp +++ b/src/IO/RemoteFileMetaDataBase.cpp @@ -15,7 +15,7 @@ RemoteFileMetaDataFactory & RemoteFileMetaDataFactory::instance() return g_factory; } -RemoteFileMetaDataBasePtr RemoteFileMetaDataFactory::create_class(const String & class_name) +RemoteFileMetaDataBasePtr RemoteFileMetaDataFactory::createClass(const String & class_name) { auto it = class_creators.find(class_name); if (it == class_creators.end()) @@ -23,7 +23,7 @@ RemoteFileMetaDataBasePtr RemoteFileMetaDataFactory::create_class(const String & return (it->second)(); } -void RemoteFileMetaDataFactory::register_class(const String & class_name, ClassCreator creator) +void RemoteFileMetaDataFactory::registerClass(const String & class_name, ClassCreator creator) { auto it = class_creators.find(class_name); if (it != class_creators.end()) diff --git a/src/IO/RemoteFileMetaDataBase.h b/src/IO/RemoteFileMetaDataBase.h index 51706f0a8d7..09127cf56e8 100644 --- a/src/IO/RemoteFileMetaDataBase.h +++ b/src/IO/RemoteFileMetaDataBase.h @@ -54,7 +54,7 @@ using RemoteFileMetaDataBasePtr = std::shared_ptr; * 1) define your own subclass derive from RemoteFileMetaDataBase. Notice! the getClassName() must be the same * as your subclass name. * 2) in a .cpp file, call REGISTTER_REMOTE_FILE_META_DATA_CLASS(subclass), - * 3) call RemoteFileMetaDataFactory::instance().create_class(subclass_name) where you want to make a new object + * 3) call RemoteFileMetaDataFactory::instance().createClass(subclass_name) where you want to make a new object */ class RemoteFileMetaDataFactory : private boost::noncopyable @@ -64,8 +64,8 @@ public: ~RemoteFileMetaDataFactory() = default; static RemoteFileMetaDataFactory & instance(); - RemoteFileMetaDataBasePtr create_class(const String & class_name); - void register_class(const String &class_name, ClassCreator creator); + RemoteFileMetaDataBasePtr createClass(const String & class_name); + void registerClass(const String &class_name, ClassCreator creator); protected: RemoteFileMetaDataFactory() = default; @@ -79,7 +79,7 @@ private: public:\ FileMetaDataFactory##meta_data_class(){\ auto creator = []() -> RemoteFileMetaDataBasePtr { return std::make_shared(); };\ - RemoteFileMetaDataFactory::instance().register_class(#meta_data_class, creator);\ + RemoteFileMetaDataFactory::instance().registerClass(#meta_data_class, creator);\ }\ };\ static FileMetaDataFactory##meta_data_class g_file_meta_data_factory_instance##meta_data_class; diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 6dd4d4be66c..9bf095b8ee6 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -57,7 +57,7 @@ std::shared_ptr RemoteCacheController::recover(const std: return nullptr; } - cache_controller->file_meta_data_ptr = RemoteFileMetaDataFactory::instance().create_class(cache_controller->meta_data_class); + cache_controller->file_meta_data_ptr = RemoteFileMetaDataFactory::instance().createClass(cache_controller->meta_data_class); if (!cache_controller->file_meta_data_ptr) { // do not load this invalid cached file and clear it. the clear action is in From 5913d67553368a92e29b1cd1df9dae22e5187ff4 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 6 Dec 2021 16:51:49 +0800 Subject: [PATCH 0125/1260] fixed code style --- src/IO/RemoteFileMetaDataBase.h | 2 +- src/IO/RemoteReadBufferCache.h | 1 - src/Storages/Hive/HiveFileMetaData.h | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/IO/RemoteFileMetaDataBase.h b/src/IO/RemoteFileMetaDataBase.h index 09127cf56e8..85f29ba68ab 100644 --- a/src/IO/RemoteFileMetaDataBase.h +++ b/src/IO/RemoteFileMetaDataBase.h @@ -24,7 +24,7 @@ public: { } virtual ~RemoteFileMetaDataBase(); - virtual String getClassName() const = 0; //class name + virtual String getName() const = 0; //class name // methods for basic information inline String getSchema() const { return schema; } inline String getCluster() const { return cluster; } diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 8f524c236c2..276b82312b0 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -142,7 +142,6 @@ public: static std::unique_ptr create(ContextPtr contex, RemoteFileMetaDataBasePtr remote_file_meta_data, std::unique_ptr read_buffer); bool nextImpl() override; - inline bool seekable() { return file_buffer && remote_file_size > 0; } off_t seek(off_t off, int whence) override; off_t getPosition() override; std::optional getTotalSize() override { return remote_file_size; } diff --git a/src/Storages/Hive/HiveFileMetaData.h b/src/Storages/Hive/HiveFileMetaData.h index 52890b93333..be48f004dcf 100644 --- a/src/Storages/Hive/HiveFileMetaData.h +++ b/src/Storages/Hive/HiveFileMetaData.h @@ -14,7 +14,7 @@ public: RemoteFileMetaDataBase(schema_, cluster_, remote_path_, file_size_, last_modification_timestamp_){} ~HiveFileMetaData() override; - String getClassName() const override { return "HiveFileMetaData"; } + String getName() const override { return "HiveFileMetaData"; } String toString() const override; bool fromString(const String &buf) override; From dd70209623120b658262926a7a12c1dcf387733c Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 6 Dec 2021 18:19:36 +0800 Subject: [PATCH 0126/1260] fixed code style --- ...taDataBase.cpp => IRemoteFileMetadata.cpp} | 12 +-- ...leMetaDataBase.h => IRemoteFileMetadata.h} | 46 ++++----- src/IO/RemoteReadBufferCache.cpp | 96 ++++++++++--------- src/IO/RemoteReadBufferCache.h | 30 +++--- src/Storages/Hive/HiveFileMetaData.h | 23 ----- src/Storages/Hive/StorageHive.cpp | 4 +- ...leMetaData.cpp => StorageHiveMetadata.cpp} | 12 +-- src/Storages/Hive/StorageHiveMetadata.h | 28 ++++++ 8 files changed, 125 insertions(+), 126 deletions(-) rename src/IO/{RemoteFileMetaDataBase.cpp => IRemoteFileMetadata.cpp} (61%) rename src/IO/{RemoteFileMetaDataBase.h => IRemoteFileMetadata.h} (54%) delete mode 100644 src/Storages/Hive/HiveFileMetaData.h rename src/Storages/Hive/{HiveFileMetaData.cpp => StorageHiveMetadata.cpp} (77%) create mode 100644 src/Storages/Hive/StorageHiveMetadata.h diff --git a/src/IO/RemoteFileMetaDataBase.cpp b/src/IO/IRemoteFileMetadata.cpp similarity index 61% rename from src/IO/RemoteFileMetaDataBase.cpp rename to src/IO/IRemoteFileMetadata.cpp index 40653a68bcb..80d3999078e 100644 --- a/src/IO/RemoteFileMetaDataBase.cpp +++ b/src/IO/IRemoteFileMetadata.cpp @@ -1,4 +1,4 @@ -#include +#include #include namespace DB { @@ -7,15 +7,15 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -RemoteFileMetaDataBase::~RemoteFileMetaDataBase() {} +IRemoteFileMetadata::~IRemoteFileMetadata() {} -RemoteFileMetaDataFactory & RemoteFileMetaDataFactory::instance() +RemoteFileMetadataFactory & RemoteFileMetadataFactory::instance() { - static RemoteFileMetaDataFactory g_factory; + static RemoteFileMetadataFactory g_factory; return g_factory; } -RemoteFileMetaDataBasePtr RemoteFileMetaDataFactory::createClass(const String & class_name) +IRemoteFileMetadataPtr RemoteFileMetadataFactory::createClass(const String & class_name) { auto it = class_creators.find(class_name); if (it == class_creators.end()) @@ -23,7 +23,7 @@ RemoteFileMetaDataBasePtr RemoteFileMetaDataFactory::createClass(const String & return (it->second)(); } -void RemoteFileMetaDataFactory::registerClass(const String & class_name, ClassCreator creator) +void RemoteFileMetadataFactory::registerClass(const String & class_name, ClassCreator creator) { auto it = class_creators.find(class_name); if (it != class_creators.end()) diff --git a/src/IO/RemoteFileMetaDataBase.h b/src/IO/IRemoteFileMetadata.h similarity index 54% rename from src/IO/RemoteFileMetaDataBase.h rename to src/IO/IRemoteFileMetadata.h index 85f29ba68ab..becf90f4920 100644 --- a/src/IO/RemoteFileMetaDataBase.h +++ b/src/IO/IRemoteFileMetadata.h @@ -7,27 +7,21 @@ namespace DB { -class RemoteFileMetaDataBase +class IRemoteFileMetadata { public: - RemoteFileMetaDataBase() = default; - RemoteFileMetaDataBase(const String & schema_, - const String & cluster_, - const String & remote_path_, + IRemoteFileMetadata() = default; + IRemoteFileMetadata(const String & remote_path_, size_t file_size_, UInt64 last_modification_timestamp_): - schema(schema_) - ,cluster(cluster_) - ,remote_path(remote_path_) + remote_path(remote_path_) ,file_size(file_size_) ,last_modification_timestamp(last_modification_timestamp_) { } - virtual ~RemoteFileMetaDataBase(); + virtual ~IRemoteFileMetadata(); virtual String getName() const = 0; //class name // methods for basic information - inline String getSchema() const { return schema; } - inline String getCluster() const { return cluster; } inline size_t getFileSize() const { return file_size; } inline String getRemotePath() const { return remote_path; } inline UInt64 getLastModificationTimestamp() const { return last_modification_timestamp; } @@ -40,34 +34,32 @@ public: // used for comparing two file meta datas are the same or not. virtual String getVersion() const = 0; protected: - String schema; - String cluster; String remote_path; size_t file_size = 0; UInt64 last_modification_timestamp = 0; }; -using RemoteFileMetaDataBasePtr = std::shared_ptr; +using IRemoteFileMetadataPtr = std::shared_ptr; /* * How to register a subclass into the factory and use it ? - * 1) define your own subclass derive from RemoteFileMetaDataBase. Notice! the getClassName() must be the same + * 1) define your own subclass derive from IRemoteFileMetadata. Notice! the getClassName() must be the same * as your subclass name. * 2) in a .cpp file, call REGISTTER_REMOTE_FILE_META_DATA_CLASS(subclass), - * 3) call RemoteFileMetaDataFactory::instance().createClass(subclass_name) where you want to make a new object + * 3) call RemoteFileMetadataFactory::instance().createClass(subclass_name) where you want to make a new object */ -class RemoteFileMetaDataFactory : private boost::noncopyable +class RemoteFileMetadataFactory : private boost::noncopyable { public: - using ClassCreator = std::function; - ~RemoteFileMetaDataFactory() = default; + using ClassCreator = std::function; + ~RemoteFileMetadataFactory() = default; - static RemoteFileMetaDataFactory & instance(); - RemoteFileMetaDataBasePtr createClass(const String & class_name); + static RemoteFileMetadataFactory & instance(); + IRemoteFileMetadataPtr createClass(const String & class_name); void registerClass(const String &class_name, ClassCreator creator); protected: - RemoteFileMetaDataFactory() = default; + RemoteFileMetadataFactory() = default; private: std::unordered_map class_creators; @@ -75,12 +67,12 @@ private: // this should be used in a .cpp file. All the subclasses will finish the registeration before the main() #define REGISTTER_REMOTE_FILE_META_DATA_CLASS(meta_data_class) \ - class FileMetaDataFactory##meta_data_class{\ + class FileMetadataFactory##meta_data_class{\ public:\ - FileMetaDataFactory##meta_data_class(){\ - auto creator = []() -> RemoteFileMetaDataBasePtr { return std::make_shared(); };\ - RemoteFileMetaDataFactory::instance().registerClass(#meta_data_class, creator);\ + FileMetadataFactory##meta_data_class(){\ + auto creator = []() -> IRemoteFileMetadataPtr { return std::make_shared(); };\ + RemoteFileMetadataFactory::instance().registerClass(#meta_data_class, creator);\ }\ };\ - static FileMetaDataFactory##meta_data_class g_file_meta_data_factory_instance##meta_data_class; + static FileMetadataFactory##meta_data_class g_file_meta_data_factory_instance##meta_data_class; } diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 9bf095b8ee6..594a2ea450c 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -32,9 +32,9 @@ bool RemoteCacheController::loadInnerInformation(const fs::path & file_path) return false; std::ifstream info_file(file_path); Poco::JSON::Parser info_parser; - auto info_jobj = info_parser.parse(info_file).extract(); - file_status = static_cast(info_jobj->get("file_status").convert()); - meta_data_class = info_jobj->get("meta_data_class").convert(); + auto info_json = info_parser.parse(info_file).extract(); + file_status = static_cast(info_json->get("file_status").convert()); + metadata_class = info_json->get("metadata_class").convert(); info_file.close(); return true; } @@ -57,19 +57,19 @@ std::shared_ptr RemoteCacheController::recover(const std: return nullptr; } - cache_controller->file_meta_data_ptr = RemoteFileMetaDataFactory::instance().createClass(cache_controller->meta_data_class); - if (!cache_controller->file_meta_data_ptr) + cache_controller->file_metadata_ptr = RemoteFileMetadataFactory::instance().createClass(cache_controller->metadata_class); + if (!cache_controller->file_metadata_ptr) { // do not load this invalid cached file and clear it. the clear action is in - // RemoteReadBufferCache::recoverCachedFilesMetaData(), because deleting directories during iteration will + // RemoteReadBufferCache::recoverCachedFilesMetadata(), because deleting directories during iteration will // cause unexpected behaviors LOG_ERROR(log, "Cannot create the meta data class : {}. The cached file is invalid and will be remove. path:{}", - cache_controller->meta_data_class, + cache_controller->metadata_class, local_path_.string()); return nullptr; } - std::ifstream meta_data_file(local_path_ / "meta_data.txt"); - if (!cache_controller->file_meta_data_ptr->fromString(std::string((std::istreambuf_iterator(meta_data_file)), + std::ifstream metadata_file(local_path_ / "metadata.txt"); + if (!cache_controller->file_metadata_ptr->fromString(std::string((std::istreambuf_iterator(metadata_file)), std::istreambuf_iterator()))) { LOG_ERROR(log, "Cannot load the meta data. The cached file is invalid and will be remove. path:{}", @@ -79,27 +79,28 @@ std::shared_ptr RemoteCacheController::recover(const std: cache_controller->current_offset = fs::file_size(local_path_ / "data.bin"); - RemoteReadBufferCache::instance().updateTotalSize(cache_controller->file_meta_data_ptr->getFileSize()); + RemoteReadBufferCache::instance().updateTotalSize(cache_controller->file_metadata_ptr->getFileSize()); return cache_controller; } RemoteCacheController::RemoteCacheController( - RemoteFileMetaDataBasePtr file_meta_data_, + IRemoteFileMetadataPtr file_metadata_, const std::filesystem::path & local_path_, size_t cache_bytes_before_flush_) - : file_meta_data_ptr(file_meta_data_) + : file_metadata_ptr(file_metadata_) , local_path(local_path_) , valid(true) , local_cache_bytes_read_before_flush(cache_bytes_before_flush_) , current_offset(0) { - // on recover, file_meta_data_ptr is null, but it will be allocated after loading from meta_data.txt - // when we allocate a whole new file cache , file_meta_data_ptr must not be null. - if (file_meta_data_ptr) + // on recover, file_metadata_ptr is null, but it will be allocated after loading from metadata.txt + // when we allocate a whole new file cache , file_metadata_ptr must not be null. + if (file_metadata_ptr) { - std::ofstream meta_data_file(local_path_ / "meta_data.txt", std::ios::out); - meta_data_file << file_meta_data_ptr->toString(); - meta_data_file.close(); + auto metadata_file_writer = std::make_unique((local_path_ / "metadata.txt").string()); + auto str_buf = file_metadata_ptr->toString(); + metadata_file_writer->write(str_buf.c_str(), str_buf.size()); + metadata_file_writer->close(); } } @@ -129,21 +130,22 @@ RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offs return RemoteReadBufferCacheError::OK; } -bool RemoteCacheController::checkFileChanged(RemoteFileMetaDataBasePtr file_meta_data_) +bool RemoteCacheController::checkFileChanged(IRemoteFileMetadataPtr file_metadata_) { - return !(file_meta_data_ptr->getVersion() == file_meta_data_->getVersion()); + return !(file_metadata_ptr->getVersion() == file_metadata_->getVersion()); } -void RemoteCacheController::startBackgroundDownload(std::shared_ptr input_readbuffer, BackgroundSchedulePool & thread_pool) +void RemoteCacheController::startBackgroundDownload(std::unique_ptr in_readbuffer_, BackgroundSchedulePool & thread_pool) { data_file_writer = std::make_unique((fs::path(local_path) / "data.bin").string()); flush(true); + ReadBufferPtr in_readbuffer(in_readbuffer_.release()); download_task_holder = thread_pool.createTask("download remote file", - [this,input_readbuffer]{ backgroundDownload(input_readbuffer); }); + [this, in_readbuffer]{ backgroundDownload(in_readbuffer); }); download_task_holder->activateAndSchedule(); } -void RemoteCacheController::backgroundDownload(std::shared_ptr remote_read_buffer) +void RemoteCacheController::backgroundDownload(ReadBufferPtr remote_read_buffer) { file_status = DOWNLOADING; size_t before_unflush_bytes = 0; @@ -174,8 +176,8 @@ void RemoteCacheController::backgroundDownload(std::shared_ptr remot data_file_writer.reset(); lock.unlock(); more_data_signal.notify_all(); - RemoteReadBufferCache::instance().updateTotalSize(file_meta_data_ptr->getFileSize()); - LOG_TRACE(log, "Finish download into local path: {}, file meta data:{} ", local_path.string(), file_meta_data_ptr->toString()); + RemoteReadBufferCache::instance().updateTotalSize(file_metadata_ptr->getFileSize()); + LOG_TRACE(log, "Finish download into local path: {}, file meta data:{} ", local_path.string(), file_metadata_ptr->toString()); } void RemoteCacheController::flush(bool need_flush_status) @@ -186,14 +188,14 @@ void RemoteCacheController::flush(bool need_flush_status) } if (need_flush_status) { + auto file_writer = std::make_unique(local_path / "info.txt"); Poco::JSON::Object jobj; jobj.set("file_status", static_cast(file_status)); - jobj.set("meta_data_class", meta_data_class); + jobj.set("metadata_class", metadata_class); std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM jobj.stringify(buf); - std::ofstream info_file(local_path / "info.txt"); - info_file << buf.str(); - info_file.close(); + file_writer->write(buf.str().c_str(), buf.str().size()); + file_writer->close(); } } @@ -239,7 +241,7 @@ void RemoteCacheController::deallocFile(std::unique_ptr throw Exception( ErrorCodes::BAD_ARGUMENTS, "Try to deallocate file with invalid handler remote path: {}, local path: {}", - file_meta_data_ptr->getRemotePath(), + file_metadata_ptr->getRemotePath(), local_path.string()); } opened_file_buffer_refs.erase(it); @@ -255,7 +257,7 @@ RemoteReadBuffer::~RemoteReadBuffer() file_cache_controller->deallocFile(std::move(file_buffer)); } -std::unique_ptr RemoteReadBuffer::create(ContextPtr context, RemoteFileMetaDataBasePtr remote_file_meta_data, std::unique_ptr read_buffer) +std::unique_ptr RemoteReadBuffer::create(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer) { auto * log = &Poco::Logger::get("RemoteReadBuffer"); size_t buff_size = DBMS_DEFAULT_BUFFER_SIZE; @@ -272,11 +274,11 @@ std::unique_ptr RemoteReadBuffer::create(ContextPtr context, RemoteF if (buff_size == 0) buff_size = DBMS_DEFAULT_BUFFER_SIZE; - auto remote_path = remote_file_meta_data->getRemotePath(); + auto remote_path = remote_file_metadata->getRemotePath(); auto remote_read_buffer = std::make_unique(buff_size); RemoteReadBufferCacheError error; - std::tie(remote_read_buffer->file_cache_controller, read_buffer, error) = RemoteReadBufferCache::instance().createReader(context, remote_file_meta_data, read_buffer); + std::tie(remote_read_buffer->file_cache_controller, read_buffer, error) = RemoteReadBufferCache::instance().createReader(context, remote_file_metadata, read_buffer); if (remote_read_buffer->file_cache_controller == nullptr) { LOG_ERROR(log, "Failed to allocate local file for remote path: {}, reason: {}.", remote_path, error); @@ -290,7 +292,7 @@ std::unique_ptr RemoteReadBuffer::create(ContextPtr context, RemoteF throw Exception(ErrorCodes::LOGICAL_ERROR, "Create file readbuffer failed. {}", remote_read_buffer->file_cache_controller->getLocalPath().string()); } - remote_read_buffer->remote_file_size = remote_file_meta_data->getFileSize(); + remote_read_buffer->remote_file_size = remote_file_metadata->getFileSize(); return remote_read_buffer; } @@ -339,7 +341,7 @@ RemoteReadBufferCache & RemoteReadBufferCache::instance() return instance; } -void RemoteReadBufferCache::recoverCachedFilesMetaData( +void RemoteReadBufferCache::recoverCachedFilesMetadata( const fs::path & current_path, size_t current_depth, size_t max_depth) @@ -370,14 +372,14 @@ void RemoteReadBufferCache::recoverCachedFilesMetaData( for (auto const & dir : fs::directory_iterator{current_path}) { - recoverCachedFilesMetaData(dir.path(), current_depth + 1, max_depth); + recoverCachedFilesMetadata(dir.path(), current_depth + 1, max_depth); } } void RemoteReadBufferCache::recoverTask() { std::lock_guard lock(mutex); - recoverCachedFilesMetaData(root_dir, 1, 2); + recoverCachedFilesMetadata(root_dir, 1, 2); initialized = true; LOG_INFO(log, "Recovered from directory:{}", root_dir); } @@ -406,18 +408,18 @@ void RemoteReadBufferCache::initOnce( recover_task_holder->activateAndSchedule(); } -String RemoteReadBufferCache::calculateLocalPath(RemoteFileMetaDataBasePtr meta_data) const +String RemoteReadBufferCache::calculateLocalPath(IRemoteFileMetadataPtr metadata) const { // add version into the full_path, and not block to read the new version - String full_path = meta_data->getSchema() + ":" + meta_data->getCluster() + ":" + meta_data->getRemotePath() - + ":" + meta_data->getVersion(); + String full_path = metadata->getName() + ":" + metadata->getRemotePath() + + ":" + metadata->getVersion(); UInt128 hashcode = sipHash128(full_path.c_str(), full_path.size()); String hashcode_str = getHexUIntLowercase(hashcode); return fs::path(root_dir) / hashcode_str.substr(0, 3) / hashcode_str; } std::tuple, RemoteReadBufferCacheError> -RemoteReadBufferCache::createReader(ContextPtr context, RemoteFileMetaDataBasePtr remote_file_meta_data, std::unique_ptr & read_buffer) +RemoteReadBufferCache::createReader(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr & read_buffer) { // If something is wrong on startup, rollback to read from the original ReadBuffer if (!isInitialized()) @@ -426,15 +428,15 @@ RemoteReadBufferCache::createReader(ContextPtr context, RemoteFileMetaDataBasePt return {nullptr, std::move(read_buffer), RemoteReadBufferCacheError::NOT_INIT}; } - auto remote_path = remote_file_meta_data->getRemotePath(); - const auto & last_modification_timestamp = remote_file_meta_data->getLastModificationTimestamp(); - auto local_path = calculateLocalPath(remote_file_meta_data); + auto remote_path = remote_file_metadata->getRemotePath(); + const auto & last_modification_timestamp = remote_file_metadata->getLastModificationTimestamp(); + auto local_path = calculateLocalPath(remote_file_metadata); std::lock_guard lock(mutex); auto cache = lru_caches->get(local_path); if (cache) { // the remote file has been updated, need to redownload - if (!cache->isValid() || cache->checkFileChanged(remote_file_meta_data)) + if (!cache->isValid() || cache->checkFileChanged(remote_file_metadata)) { LOG_TRACE( log, @@ -454,11 +456,11 @@ RemoteReadBufferCache::createReader(ContextPtr context, RemoteFileMetaDataBasePt fs::create_directories(local_path); // cache is not found or is invalid - auto new_cache = std::make_shared(remote_file_meta_data, local_path, local_cache_bytes_read_before_flush); + auto new_cache = std::make_shared(remote_file_metadata, local_path, local_cache_bytes_read_before_flush); if (!lru_caches->set(local_path, new_cache)) { LOG_ERROR(log, "Insert the new cache failed. new file size:{}, current total size:{}", - remote_file_meta_data->getFileSize(), + remote_file_metadata->getFileSize(), lru_caches->weight()); return {nullptr, std::move(read_buffer), RemoteReadBufferCacheError::DISK_FULL}; } diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 276b82312b0..3c96945f271 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include #include @@ -44,7 +44,7 @@ public: }; RemoteCacheController( - RemoteFileMetaDataBasePtr file_meta_data_, + IRemoteFileMetadataPtr file_metadata_, const std::filesystem::path & local_path_, size_t cache_bytes_before_flush_); ~RemoteCacheController(); @@ -82,10 +82,10 @@ public: inline size_t size() const { return current_offset; } inline const std::filesystem::path & getLocalPath() { return local_path; } - inline String getRemotePath() const { return file_meta_data_ptr->getRemotePath(); } + inline String getRemotePath() const { return file_metadata_ptr->getRemotePath(); } - inline UInt64 getLastModificationTimestamp() const { return file_meta_data_ptr->getLastModificationTimestamp(); } - bool checkFileChanged(RemoteFileMetaDataBasePtr file_meta_data_); + inline UInt64 getLastModificationTimestamp() const { return file_metadata_ptr->getLastModificationTimestamp(); } + bool checkFileChanged(IRemoteFileMetadataPtr file_metadata_); inline void markInvalid() { std::lock_guard lock(mutex); @@ -96,10 +96,10 @@ public: std::lock_guard lock(mutex); return valid; } - RemoteFileMetaDataBasePtr getFileMetaData() { return file_meta_data_ptr; } - inline size_t getFileSize() const { return file_meta_data_ptr->getFileSize(); } + IRemoteFileMetadataPtr getFileMetadata() { return file_metadata_ptr; } + inline size_t getFileSize() const { return file_metadata_ptr->getFileSize(); } - void startBackgroundDownload(std::shared_ptr input_readbuffer, BackgroundSchedulePool & thread_pool); + void startBackgroundDownload(std::unique_ptr in_readbuffer_, BackgroundSchedulePool & thread_pool); private: // flush file and status information @@ -107,16 +107,16 @@ private: bool loadInnerInformation(const std::filesystem::path & file_path); BackgroundSchedulePool::TaskHolder download_task_holder; - void backgroundDownload(std::shared_ptr remote_read_buffer); + void backgroundDownload(ReadBufferPtr remote_read_buffer); std::mutex mutex; std::condition_variable more_data_signal; std::set opened_file_buffer_refs; // refer to a buffer address - String meta_data_class; + String metadata_class; LocalFileStatus file_status = TO_DOWNLOAD; // for tracking download process - RemoteFileMetaDataBasePtr file_meta_data_ptr; + IRemoteFileMetadataPtr file_metadata_ptr; std::filesystem::path local_path; bool valid; @@ -139,7 +139,7 @@ class RemoteReadBuffer : public BufferWithOwnMemory public: explicit RemoteReadBuffer(size_t buff_size); ~RemoteReadBuffer() override; - static std::unique_ptr create(ContextPtr contex, RemoteFileMetaDataBasePtr remote_file_meta_data, std::unique_ptr read_buffer); + static std::unique_ptr create(ContextPtr contex, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer); bool nextImpl() override; off_t seek(off_t off, int whence) override; @@ -188,7 +188,7 @@ public: inline bool isInitialized() const { return initialized; } std::tuple, RemoteReadBufferCacheError> - createReader(ContextPtr context, RemoteFileMetaDataBasePtr remote_file_meta_data, std::unique_ptr & read_buffer); + createReader(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr & read_buffer); void updateTotalSize(size_t size) { total_size += size; } @@ -207,11 +207,11 @@ private: Poco::Logger * log = &Poco::Logger::get("RemoteReadBufferCache"); - String calculateLocalPath(RemoteFileMetaDataBasePtr meta) const; + String calculateLocalPath(IRemoteFileMetadataPtr meta) const; BackgroundSchedulePool::TaskHolder recover_task_holder; void recoverTask(); - void recoverCachedFilesMetaData( + void recoverCachedFilesMetadata( const std::filesystem::path & current_path, size_t current_depth, size_t max_depth); diff --git a/src/Storages/Hive/HiveFileMetaData.h b/src/Storages/Hive/HiveFileMetaData.h deleted file mode 100644 index be48f004dcf..00000000000 --- a/src/Storages/Hive/HiveFileMetaData.h +++ /dev/null @@ -1,23 +0,0 @@ -#pragma once -#include -namespace DB -{ -class HiveFileMetaData : public RemoteFileMetaDataBase -{ -public: - HiveFileMetaData() = default; - HiveFileMetaData(const String & schema_, - const String & cluster_, - const String & remote_path_, - size_t file_size_, - UInt64 last_modification_timestamp_): - RemoteFileMetaDataBase(schema_, cluster_, remote_path_, file_size_, last_modification_timestamp_){} - ~HiveFileMetaData() override; - - String getName() const override { return "HiveFileMetaData"; } - - String toString() const override; - bool fromString(const String &buf) override; - String getVersion() const override; -}; -} diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 259763aca1c..bd53ce22925 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -38,7 +38,7 @@ #include #include #include -#include +#include #include #include @@ -171,7 +171,7 @@ public: std::unique_ptr remote_read_buf; if (RemoteReadBufferCache::instance().isInitialized() && getContext()->getSettingsRef().use_local_cache_for_remote_fs) remote_read_buf = RemoteReadBuffer::create(getContext(), - std::make_shared("Hive", getNameNodeCluster(hdfs_namenode_url), uri_with_path, curr_file->getSize(), curr_file->getLastModTs()), + std::make_shared("Hive", getNameNodeCluster(hdfs_namenode_url), uri_with_path, curr_file->getSize(), curr_file->getLastModTs()), std::move(raw_read_buf)); else remote_read_buf = std::move(raw_read_buf); diff --git a/src/Storages/Hive/HiveFileMetaData.cpp b/src/Storages/Hive/StorageHiveMetadata.cpp similarity index 77% rename from src/Storages/Hive/HiveFileMetaData.cpp rename to src/Storages/Hive/StorageHiveMetadata.cpp index 7bb2ba760b0..c7a4b909b7a 100644 --- a/src/Storages/Hive/HiveFileMetaData.cpp +++ b/src/Storages/Hive/StorageHiveMetadata.cpp @@ -1,13 +1,13 @@ -#include +#include #include #include #include namespace DB { -HiveFileMetaData::~HiveFileMetaData() = default; +StorageHiveMetadata::~StorageHiveMetadata() = default; -String HiveFileMetaData::toString() const +String StorageHiveMetadata::toString() const { Poco::JSON::Object jobj; jobj.set("schema", schema); @@ -21,7 +21,7 @@ String HiveFileMetaData::toString() const } -bool HiveFileMetaData::fromString(const String &buf) +bool StorageHiveMetadata::fromString(const String &buf) { std::stringstream istream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM istream << buf; @@ -35,11 +35,11 @@ bool HiveFileMetaData::fromString(const String &buf) return true; } -String HiveFileMetaData::getVersion() const +String StorageHiveMetadata::getVersion() const { return std::to_string(getLastModificationTimestamp()); } -REGISTTER_REMOTE_FILE_META_DATA_CLASS(HiveFileMetaData) +REGISTTER_REMOTE_FILE_META_DATA_CLASS(StorageHiveMetadata) } diff --git a/src/Storages/Hive/StorageHiveMetadata.h b/src/Storages/Hive/StorageHiveMetadata.h new file mode 100644 index 00000000000..76785ba8b6a --- /dev/null +++ b/src/Storages/Hive/StorageHiveMetadata.h @@ -0,0 +1,28 @@ +#pragma once +#include +namespace DB +{ +class StorageHiveMetadata : public IRemoteFileMetadata +{ +public: + StorageHiveMetadata() = default; + StorageHiveMetadata(const String & schema_, + const String & cluster_, + const String & remote_path_, + size_t file_size_, + UInt64 last_modification_timestamp_): + IRemoteFileMetadata(remote_path_, file_size_, last_modification_timestamp_),schema(schema_), cluster(cluster_){} + ~StorageHiveMetadata() override; + + String getName() const override { return "StorageHiveMetadata"; } + String getSchema() const { return schema; } + String getCluster() const { return cluster; } + + String toString() const override; + bool fromString(const String &buf) override; + String getVersion() const override; +private: + String schema; + String cluster; +}; +} From d4dcbd59fbc174ec226fca9a448b66db705b5ba0 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 6 Dec 2021 19:59:15 +0800 Subject: [PATCH 0127/1260] fixed code style --- src/IO/IRemoteFileMetadata.cpp | 5 +++-- src/IO/IRemoteFileMetadata.h | 6 +++--- src/IO/RemoteReadBufferCache.cpp | 2 +- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/IO/IRemoteFileMetadata.cpp b/src/IO/IRemoteFileMetadata.cpp index 80d3999078e..cd9aec68123 100644 --- a/src/IO/IRemoteFileMetadata.cpp +++ b/src/IO/IRemoteFileMetadata.cpp @@ -5,6 +5,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } IRemoteFileMetadata::~IRemoteFileMetadata() {} @@ -15,11 +16,11 @@ RemoteFileMetadataFactory & RemoteFileMetadataFactory::instance() return g_factory; } -IRemoteFileMetadataPtr RemoteFileMetadataFactory::createClass(const String & class_name) +IRemoteFileMetadataPtr RemoteFileMetadataFactory::get(const String & class_name) { auto it = class_creators.find(class_name); if (it == class_creators.end()) - return nullptr; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not found metadata class:{}", class_name); return (it->second)(); } diff --git a/src/IO/IRemoteFileMetadata.h b/src/IO/IRemoteFileMetadata.h index becf90f4920..92830cd481e 100644 --- a/src/IO/IRemoteFileMetadata.h +++ b/src/IO/IRemoteFileMetadata.h @@ -27,7 +27,7 @@ public: inline UInt64 getLastModificationTimestamp() const { return last_modification_timestamp; } // deserialize - virtual bool fromString(const String &buf) = 0; + virtual bool fromString(const String & buf) = 0; // serialize virtual String toString() const = 0; @@ -43,7 +43,7 @@ using IRemoteFileMetadataPtr = std::shared_ptr; /* * How to register a subclass into the factory and use it ? - * 1) define your own subclass derive from IRemoteFileMetadata. Notice! the getClassName() must be the same + * 1) define your own subclass derive from IRemoteFileMetadata. Notice! the getName() must be the same * as your subclass name. * 2) in a .cpp file, call REGISTTER_REMOTE_FILE_META_DATA_CLASS(subclass), * 3) call RemoteFileMetadataFactory::instance().createClass(subclass_name) where you want to make a new object @@ -56,7 +56,7 @@ public: ~RemoteFileMetadataFactory() = default; static RemoteFileMetadataFactory & instance(); - IRemoteFileMetadataPtr createClass(const String & class_name); + IRemoteFileMetadataPtr get(const String & class_name); void registerClass(const String &class_name, ClassCreator creator); protected: RemoteFileMetadataFactory() = default; diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 594a2ea450c..36680063c5f 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -57,7 +57,7 @@ std::shared_ptr RemoteCacheController::recover(const std: return nullptr; } - cache_controller->file_metadata_ptr = RemoteFileMetadataFactory::instance().createClass(cache_controller->metadata_class); + cache_controller->file_metadata_ptr = RemoteFileMetadataFactory::instance().get(cache_controller->metadata_class); if (!cache_controller->file_metadata_ptr) { // do not load this invalid cached file and clear it. the clear action is in From f083c5594c81eb4f90acba53e1673aa2f8c4c7d2 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 7 Dec 2021 09:23:38 +0800 Subject: [PATCH 0128/1260] update config.xml in hive integration test --- tests/integration/test_hive_query/configs/config.xml | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_hive_query/configs/config.xml b/tests/integration/test_hive_query/configs/config.xml index 2aca007ef9b..a895d417a2d 100644 --- a/tests/integration/test_hive_query/configs/config.xml +++ b/tests/integration/test_hive_query/configs/config.xml @@ -11,9 +11,12 @@ - /clickhouse_local_cache - 207374182400 - 1000 + + true + /data/clickhouse_local_cache + 207374182400 + 1048576 + /etc/clickhouse-server/extra_conf.d/hdfs-site.xml From 0c1733817b674970d786125eb65d22911bb4b72c Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 7 Dec 2021 10:57:05 +0800 Subject: [PATCH 0129/1260] update LRUCache set() --- src/Common/LRUCache.h | 96 ++++++++-- src/Common/UnreleasableLRUCache.h | 283 ------------------------------ src/IO/RemoteReadBufferCache.h | 10 +- 3 files changed, 89 insertions(+), 300 deletions(-) delete mode 100644 src/Common/UnreleasableLRUCache.h diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index bbc09fd3aff..4b53ef40c01 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -22,13 +22,37 @@ struct TrivialWeightFunction } }; +enum class LRUCacheEvictStatus +{ + CAN_EVITCT, // a key can be evicted + TERMINATE_EVICT, // stop the evicting process + SKIP_EVICT, // skip current value and keep iterating +}; + +template +struct TrivialLRUCacheEvitPolicy +{ + inline LRUCacheEvictStatus canRelease(const T &) const + { + return LRUCacheEvictStatus::CAN_EVITCT; + } + + inline void release(T & ) + { + } +}; + /// Thread-safe cache that evicts entries which are not used for a long time. /// WeightFunction is a functor that takes Mapped as a parameter and returns "weight" (approximate size) /// of that value. /// Cache starts to evict entries when their total weight exceeds max_size. /// Value weight should not change after insertion. -template , typename WeightFunction = TrivialWeightFunction> +template , + typename WeightFunction = TrivialWeightFunction, + typename EvictPolicy = TrivialLRUCacheEvitPolicy> class LRUCache { public: @@ -57,11 +81,15 @@ public: return res; } - void set(const Key & key, const MappedPtr & mapped) + /** + * set() will fail if there is no space left and no keys could be evicted. + * In some cases, a key can be only evicted when it is not refered by anyone. + */ + bool set(const Key & key, const MappedPtr & mapped) { std::lock_guard lock(mutex); - setImpl(key, mapped, lock); + return setImpl(key, mapped, lock); } /// If the value for the key is in the cache, returns it. If it is not, calls load_func() to @@ -117,6 +145,8 @@ public: auto token_it = insert_tokens.find(key); if (token_it != insert_tokens.end() && token_it->second.get() == token) { + // setImpl() may fail, but the final behavior seems not be affected + // next call of getOrSet() will still call load_func() setImpl(key, token->value, cache_lock); result = true; } @@ -259,6 +289,7 @@ private: std::atomic misses {0}; WeightFunction weight_function; + EvictPolicy evict_policy; MappedPtr getImpl(const Key & key, [[maybe_unused]] std::lock_guard & cache_lock) { @@ -276,7 +307,7 @@ private: return cell.value; } - void setImpl(const Key & key, const MappedPtr & mapped, [[maybe_unused]] std::lock_guard & cache_lock) + bool setImpl(const Key & key, const MappedPtr & mapped, [[maybe_unused]] std::lock_guard & cache_lock) { auto [it, inserted] = cells.emplace(std::piecewise_construct, std::forward_as_tuple(key), @@ -286,6 +317,15 @@ private: if (inserted) { + auto weight = mapped ? weight_function(*mapped) : 0; + // move removeOverflow() ahead here. In default, the final result is the same as the old implementation + if (!removeOverflow(weight)) + { + // cannot find enough space to put in the new value + cells.erase(it); + return false; + } + try { cell.queue_iterator = queue.insert(queue.end(), key); @@ -298,6 +338,13 @@ private: } else { + if (evict_policy.canRelease(*cell.value) != LRUCacheEvictStatus::CAN_EVITCT) + { + // the old value is refered by someone, cannot release now + // in default policy, it is always CAN_EVITCT. + return false; + } + evict_policy.release(*cell.value); // release the old value. this action is empty in default policy. current_size -= cell.size; queue.splice(queue.end(), queue, cell.queue_iterator); } @@ -307,16 +354,20 @@ private: current_size += cell.size; removeOverflow(); + return true; } - void removeOverflow() + bool removeOverflow(size_t more_size = 0) { size_t current_weight_lost = 0; size_t queue_size = cells.size(); + auto key_it = queue.begin(); - while ((current_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) && (queue_size > 1)) + while ((current_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) + && (queue_size > 1) + && key_it != queue.end()) { - const Key & key = queue.front(); + const Key & key = *key_it; auto it = cells.find(key); if (it == cells.end()) @@ -326,13 +377,33 @@ private: } const auto & cell = it->second; + auto evict_status = evict_policy.canRelease(*cell.value);// in default, it is CAN_EVITCT + if (evict_status == LRUCacheEvictStatus::CAN_EVITCT) + { + // always call release() before erasing an element + // in default, it's an empty action + evict_policy.release(*cell.value); - current_size -= cell.size; - current_weight_lost += cell.size; + current_size -= cell.size; + current_weight_lost += cell.size; - cells.erase(it); - queue.pop_front(); - --queue_size; + cells.erase(it); + queue.pop_front(); + --queue_size; + } + else if (evict_status == LRUCacheEvictStatus::SKIP_EVICT) + { + // skip this element and try to evict the remaining ones. + key_it++; + continue; + } + else if (evict_status == LRUCacheEvictStatus::TERMINATE_EVICT) + { + // maybe we want to stop this iteration once we meet the first unreleasable element + break; + } + LOG_ERROR(&Poco::Logger::get("LRUCache"), "This condition branch should not be reached."); + abort(); } onRemoveOverflowWeightLoss(current_weight_lost); @@ -342,6 +413,7 @@ private: LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); abort(); } + return !(current_size + more_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)); } /// Override this method if you want to track how much weight was lost in removeOverflow method. diff --git a/src/Common/UnreleasableLRUCache.h b/src/Common/UnreleasableLRUCache.h deleted file mode 100644 index 3b5eb27543e..00000000000 --- a/src/Common/UnreleasableLRUCache.h +++ /dev/null @@ -1,283 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include - -#include -#include - - -namespace DB -{ - -template -struct TrivialUnreleasableLRUCacheWeightFunction -{ - size_t operator()(const T &) const - { - return 1; - } -}; - -enum class CacheEvictStatus -{ - CAN_EVITCT, // a key can be evicted - TERMINATE_EVICT, // stop the evicting process - SKIP_EVICT, // skip current value and keep iterating -}; - -template -struct TrivialUnreleasableLRUCacheEvitPolicy -{ - CacheEvictStatus canRelease(const T &) - { - return CacheEvictStatus::CAN_EVITCT; - } - - void release(T & ) - { - } -}; - -/* - * Another version LRU Cache。 - * A value can only be evicted or be updated if it is releasable. If there is no releasable value, - * insert or update will fail. - */ -template , - typename WeightFunction = TrivialUnreleasableLRUCacheWeightFunction, - typename EvictPolicy = TrivialUnreleasableLRUCacheEvitPolicy> -class UnreleasableLRUCache -{ -public: - using Key = TKey; - using Mapped = TMapped; - using MappedPtr = std::shared_ptr; - - /** Initialize LRUCache with max_size and max_elements_size. - * max_elements_size == 0 means no elements size restrictions. - */ - UnreleasableLRUCache(size_t max_size_, size_t max_elements_size_ = 0) - : max_size(std::max(static_cast(1), max_size_)) - , max_elements_size(max_elements_size_) - {} - - MappedPtr get(const Key & key) - { - std::lock_guard lock(mutex); - - auto res = getImpl(key, lock); - if (res) - ++hits; - else - ++misses; - return res; - } - - /* - * Fail on two cases - * 1) the key exists, but the old value is not releasable - * 2) the key not exists, but there is not enough space for it after trying to evict some least recently used values. - */ - bool set(const Key & key, const MappedPtr & mapped) - { - std::lock_guard lock(mutex); - - return setImpl(key, mapped, lock); - } - - void getStats(size_t & out_hits, size_t & out_misses) const - { - std::lock_guard lock(mutex); - out_hits = hits; - out_misses = misses; - } - - size_t weight() const - { - std::lock_guard lock(mutex); - return current_size; - } - - size_t count() const - { - std::lock_guard lock(mutex); - return cells.size(); - } - - size_t maxSize() const - { - return max_size; - } - - void reset() - { - std::lock_guard lock(mutex); - queue.clear(); - cells.clear(); - current_size = 0; - hits = 0; - misses = 0; - } - - virtual ~UnreleasableLRUCache() {} - -protected: - using LRUQueue = std::list; - using LRUQueueIterator = typename LRUQueue::iterator; - - struct Cell - { - MappedPtr value; - size_t size; - LRUQueueIterator queue_iterator; - }; - - using Cells = std::unordered_map; - - Cells cells; - - mutable std::mutex mutex; -private: - - LRUQueue queue; - - /// Total weight of values. - size_t current_size = 0; - const size_t max_size; - const size_t max_elements_size; - - std::atomic hits {0}; - std::atomic misses {0}; - - WeightFunction weight_function; - EvictPolicy evict_policy; - - MappedPtr getImpl(const Key & key, [[maybe_unused]] std::lock_guard & cache_lock) - { - auto it = cells.find(key); - if (it == cells.end()) - { - return MappedPtr(); - } - - Cell & cell = it->second; - - /// Move the key to the end of the queue. The iterator remains valid. - queue.splice(queue.end(), queue, cell.queue_iterator); - - return cell.value; - } - - bool setImpl(const Key & key, const MappedPtr & mapped, [[maybe_unused]] std::lock_guard & cache_lock) - { - auto [it, inserted] = cells.emplace(std::piecewise_construct, - std::forward_as_tuple(key), - std::forward_as_tuple()); - - Cell & cell = it->second; - - if (inserted) - { - auto weight = mapped ? weight_function(*mapped) : 0; - if (!removeOverflow(weight)) - { - // cannot insert this new value - cells.erase(it); - return false; - } - - try - { - cell.queue_iterator = queue.insert(queue.end(), key); - } - catch (...) - { - cells.erase(it); - throw; - } - } - else - { - if (evict_policy.canRelease(*cell.value) != CacheEvictStatus::CAN_EVITCT) - { - // the old value is not releasable - return false; - } - evict_policy.release(*cell.value); - current_size -= cell.size; - queue.splice(queue.end(), queue, cell.queue_iterator); - } - - cell.value = mapped; - cell.size = cell.value ? weight_function(*cell.value) : 0; - current_size += cell.size; - - removeOverflow(0); - return true; - } - - // Or make your own implementation - virtual bool removeOverflow(size_t more_size) - { - size_t current_weight_lost = 0; - size_t queue_size = cells.size(); - - auto key_it = queue.begin(); - while ((current_size + more_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) && (queue_size > 1) - && key_it != queue.end()) - { - const Key & key = *key_it; - - auto it = cells.find(key); - if (it == cells.end()) - { - LOG_ERROR(&Poco::Logger::get("UnreleasableLRUCache"), "UnreleasableLRUCache became inconsistent. There must be a bug in it."); - abort(); - } - - const auto & cell = it->second; - auto cache_evict_status = evict_policy.canRelease(*(cell.value)); - if (cache_evict_status == CacheEvictStatus::CAN_EVITCT) - { - evict_policy.release(*(cell.value)); - current_size -= cell.size; - current_weight_lost += cell.size; - - cells.erase(it); - key_it = queue.erase(key_it); - --queue_size; - } - else if (cache_evict_status == CacheEvictStatus::SKIP_EVICT) - { - key_it++; - continue; - } - else if (cache_evict_status == CacheEvictStatus::TERMINATE_EVICT) - { - break; - } - } - - onRemoveOverflowWeightLoss(current_weight_lost); - - if (current_size > (1ull << 63)) - { - LOG_ERROR(&Poco::Logger::get("UnreleasableLRUCache"), "UnreleasableLRUCache became inconsistent. There must be a bug in it."); - abort(); - } - return !(current_size + more_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)); - } - - /// Override this method if you want to track how much weight was lost in removeOverflow method. - virtual void onRemoveOverflowWeightLoss(size_t /*weight_loss*/) {} -}; - - -} diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index 3c96945f271..dd9217f924b 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include @@ -162,11 +162,11 @@ struct RemoteFileCacheWeightFunction struct RemoteFileCacheEvictPolicy { - CacheEvictStatus canRelease(RemoteCacheController & cache) const + LRUCacheEvictStatus canRelease(RemoteCacheController & cache) const { if (cache.closable()) - return CacheEvictStatus::CAN_EVITCT; - return CacheEvictStatus::SKIP_EVICT; + return LRUCacheEvictStatus::CAN_EVITCT; + return LRUCacheEvictStatus::SKIP_EVICT; } void release(RemoteCacheController & cache) { @@ -177,7 +177,7 @@ struct RemoteFileCacheEvictPolicy class RemoteReadBufferCache { public: - using CacheType = UnreleasableLRUCache, + using CacheType = LRUCache, RemoteFileCacheWeightFunction, RemoteFileCacheEvictPolicy>; ~RemoteReadBufferCache(); // global instance From 07045bc45131c8869f5b51339364e0fb69b548ec Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 7 Dec 2021 11:18:44 +0800 Subject: [PATCH 0130/1260] fixed a exception message --- src/IO/RemoteReadBufferCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 36680063c5f..357c85de5a6 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -231,7 +231,7 @@ void RemoteCacheController::deallocFile(std::unique_ptr { if (!file_buffer) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Try to release a null file buffer for ", local_path.string()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Try to release a null file buffer for {}", local_path.string()); } auto buffer_ref = reinterpret_cast(file_buffer.get()); std::lock_guard lock{mutex}; From fcea033a072dfcc03c605baef228184940eaf254 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 7 Dec 2021 12:17:47 +0800 Subject: [PATCH 0131/1260] fixed code style --- src/Common/LRUCache.h | 14 +++++++------- src/IO/IRemoteFileMetadata.cpp | 14 +++++++------- src/IO/IRemoteFileMetadata.h | 4 ++-- 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 4b53ef40c01..cf550abd7cc 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -37,7 +37,7 @@ struct TrivialLRUCacheEvitPolicy return LRUCacheEvictStatus::CAN_EVITCT; } - inline void release(T & ) + inline void release(T &) { } }; @@ -317,9 +317,9 @@ private: if (inserted) { - auto weight = mapped ? weight_function(*mapped) : 0; + auto value_weight = mapped ? weight_function(*mapped) : 0; // move removeOverflow() ahead here. In default, the final result is the same as the old implementation - if (!removeOverflow(weight)) + if (!removeOverflow(value_weight)) { // cannot find enough space to put in the new value cells.erase(it); @@ -352,8 +352,8 @@ private: cell.value = mapped; cell.size = cell.value ? weight_function(*cell.value) : 0; current_size += cell.size; - removeOverflow(); + return true; } @@ -363,9 +363,9 @@ private: size_t queue_size = cells.size(); auto key_it = queue.begin(); - while ((current_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) + while ((current_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) && (queue_size > 1) - && key_it != queue.end()) + && (key_it != queue.end())) { const Key & key = *key_it; @@ -379,7 +379,7 @@ private: const auto & cell = it->second; auto evict_status = evict_policy.canRelease(*cell.value);// in default, it is CAN_EVITCT if (evict_status == LRUCacheEvictStatus::CAN_EVITCT) - { + { // always call release() before erasing an element // in default, it's an empty action evict_policy.release(*cell.value); diff --git a/src/IO/IRemoteFileMetadata.cpp b/src/IO/IRemoteFileMetadata.cpp index cd9aec68123..f9c6bac56f6 100644 --- a/src/IO/IRemoteFileMetadata.cpp +++ b/src/IO/IRemoteFileMetadata.cpp @@ -16,21 +16,21 @@ RemoteFileMetadataFactory & RemoteFileMetadataFactory::instance() return g_factory; } -IRemoteFileMetadataPtr RemoteFileMetadataFactory::get(const String & class_name) +IRemoteFileMetadataPtr RemoteFileMetadataFactory::get(const String & name) { - auto it = class_creators.find(class_name); + auto it = class_creators.find(name); if (it == class_creators.end()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not found metadata class:{}", class_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not found metadata class:{}", name); return (it->second)(); } -void RemoteFileMetadataFactory::registerClass(const String & class_name, ClassCreator creator) +void RemoteFileMetadataFactory::registerClass(const String & name, ClassCreator creator) { - auto it = class_creators.find(class_name); + auto it = class_creators.find(name); if (it != class_creators.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Class ({}) has been registered. It is a fatal error.", class_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Class ({}) has been registered. It is a fatal error.", name); } - class_creators[class_name] = creator; + class_creators[name] = creator; } } diff --git a/src/IO/IRemoteFileMetadata.h b/src/IO/IRemoteFileMetadata.h index 92830cd481e..a9125844896 100644 --- a/src/IO/IRemoteFileMetadata.h +++ b/src/IO/IRemoteFileMetadata.h @@ -56,8 +56,8 @@ public: ~RemoteFileMetadataFactory() = default; static RemoteFileMetadataFactory & instance(); - IRemoteFileMetadataPtr get(const String & class_name); - void registerClass(const String &class_name, ClassCreator creator); + IRemoteFileMetadataPtr get(const String & name); + void registerClass(const String &name, ClassCreator creator); protected: RemoteFileMetadataFactory() = default; From 562a0499a14b3c397f6dd801869b3754b866f7d0 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 7 Dec 2021 12:29:20 +0800 Subject: [PATCH 0132/1260] fixed code style --- src/IO/IRemoteFileMetadata.h | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/IO/IRemoteFileMetadata.h b/src/IO/IRemoteFileMetadata.h index a9125844896..1275ad5ab5c 100644 --- a/src/IO/IRemoteFileMetadata.h +++ b/src/IO/IRemoteFileMetadata.h @@ -66,13 +66,13 @@ private: }; // this should be used in a .cpp file. All the subclasses will finish the registeration before the main() -#define REGISTTER_REMOTE_FILE_META_DATA_CLASS(meta_data_class) \ - class FileMetadataFactory##meta_data_class{\ +#define REGISTTER_REMOTE_FILE_META_DATA_CLASS(metadata_class) \ + class FileMetadataFactory##metadata_class{\ public:\ - FileMetadataFactory##meta_data_class(){\ - auto creator = []() -> IRemoteFileMetadataPtr { return std::make_shared(); };\ - RemoteFileMetadataFactory::instance().registerClass(#meta_data_class, creator);\ + FileMetadataFactory##metadata_class(){\ + auto creator = []() -> IRemoteFileMetadataPtr { return std::make_shared(); };\ + RemoteFileMetadataFactory::instance().registerClass(#metadata_class, creator);\ }\ };\ - static FileMetadataFactory##meta_data_class g_file_meta_data_factory_instance##meta_data_class; + static FileMetadataFactory##metadata_class g_file_metadata_factory_instance##metadata_class; } From 15504a4bfad35d436645355b0f142b0638c4f926 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 7 Dec 2021 12:32:50 +0800 Subject: [PATCH 0133/1260] fixed code style --- src/IO/IRemoteFileMetadata.h | 2 +- src/IO/RemoteReadBufferCache.cpp | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/IO/IRemoteFileMetadata.h b/src/IO/IRemoteFileMetadata.h index 1275ad5ab5c..e6a3e2438f6 100644 --- a/src/IO/IRemoteFileMetadata.h +++ b/src/IO/IRemoteFileMetadata.h @@ -31,7 +31,7 @@ public: // serialize virtual String toString() const = 0; - // used for comparing two file meta datas are the same or not. + // used for comparing two file metadatas are the same or not. virtual String getVersion() const = 0; protected: String remote_path; diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 357c85de5a6..426b68df11a 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -63,7 +63,7 @@ std::shared_ptr RemoteCacheController::recover(const std: // do not load this invalid cached file and clear it. the clear action is in // RemoteReadBufferCache::recoverCachedFilesMetadata(), because deleting directories during iteration will // cause unexpected behaviors - LOG_ERROR(log, "Cannot create the meta data class : {}. The cached file is invalid and will be remove. path:{}", + LOG_ERROR(log, "Cannot create the metadata class : {}. The cached file is invalid and will be remove. path:{}", cache_controller->metadata_class, local_path_.string()); return nullptr; @@ -72,7 +72,7 @@ std::shared_ptr RemoteCacheController::recover(const std: if (!cache_controller->file_metadata_ptr->fromString(std::string((std::istreambuf_iterator(metadata_file)), std::istreambuf_iterator()))) { - LOG_ERROR(log, "Cannot load the meta data. The cached file is invalid and will be remove. path:{}", + LOG_ERROR(log, "Cannot load the metadata. The cached file is invalid and will be remove. path:{}", local_path_.string()); return nullptr; } @@ -177,7 +177,7 @@ void RemoteCacheController::backgroundDownload(ReadBufferPtr remote_read_buffer) lock.unlock(); more_data_signal.notify_all(); RemoteReadBufferCache::instance().updateTotalSize(file_metadata_ptr->getFileSize()); - LOG_TRACE(log, "Finish download into local path: {}, file meta data:{} ", local_path.string(), file_metadata_ptr->toString()); + LOG_TRACE(log, "Finish download into local path: {}, file metadata:{} ", local_path.string(), file_metadata_ptr->toString()); } void RemoteCacheController::flush(bool need_flush_status) @@ -404,7 +404,7 @@ void RemoteReadBufferCache::initOnce( ErrorCodes::CANNOT_CREATE_DIRECTORY, "Failed to create directories, error code:{} reason:{}", ec.value(), ec.message()); } - recover_task_holder = context->getSchedulePool().createTask("recover local cache meta data for remote files", [this]{ recoverTask(); }); + recover_task_holder = context->getSchedulePool().createTask("recover local cache metadata for remote files", [this]{ recoverTask(); }); recover_task_holder->activateAndSchedule(); } From b6c230606e0f8b7dd13384fe747c62f2cc1ff85a Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 7 Dec 2021 14:46:17 +0800 Subject: [PATCH 0134/1260] fixed config error in server startup --- programs/server/config.xml | 3 +-- src/IO/RemoteReadBufferCache.h | 3 ++- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index e1c2e3ca3fc..d5dc338279f 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1257,8 +1257,7 @@ true /var/lib/clickhouse/local_cache - 53687091200 + 559096952 1048576 - 1024 diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index dd9217f924b..a9243f92e97 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -20,6 +20,7 @@ #include #include #include +#include namespace DB @@ -174,7 +175,7 @@ struct RemoteFileCacheEvictPolicy } }; -class RemoteReadBufferCache +class RemoteReadBufferCache : private boost::noncopyable { public: using CacheType = LRUCache, From 5a8c56bd559d8f527fed78afb3bc99b94ede1c31 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 7 Dec 2021 16:35:08 +0800 Subject: [PATCH 0135/1260] fixed erros on creating dir on startup --- programs/server/config.xml | 2 +- src/IO/RemoteReadBufferCache.cpp | 6 +++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index d5dc338279f..80df17b03ec 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1256,7 +1256,7 @@ true - /var/lib/clickhouse/local_cache + local_cache 559096952 1048576 diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 426b68df11a..7219bda2efd 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -378,7 +378,6 @@ void RemoteReadBufferCache::recoverCachedFilesMetadata( void RemoteReadBufferCache::recoverTask() { - std::lock_guard lock(mutex); recoverCachedFilesMetadata(root_dir, 1, 2); initialized = true; LOG_INFO(log, "Recovered from directory:{}", root_dir); @@ -388,6 +387,11 @@ void RemoteReadBufferCache::initOnce( ContextPtr context, const String & root_dir_, size_t limit_size_, size_t bytes_read_before_flush_) { + std::lock_guard lock(mutex); + if (isInitialized()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot initialize RemoteReadBufferCache twice"); + } LOG_INFO( log, "Initializing local cache for remote data sources. Local cache root path: {}, cache size limit: {}", root_dir_, limit_size_); root_dir = root_dir_; From 54cc65ede8ba6d26197ee25194f194a7f40ba70f Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 7 Dec 2021 17:12:26 +0800 Subject: [PATCH 0136/1260] fixed code style --- src/Common/LRUCache.h | 47 ++++++++++++++------------------ src/IO/RemoteReadBufferCache.cpp | 8 ++---- 2 files changed, 23 insertions(+), 32 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index cf550abd7cc..1f3b7fb09d2 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -25,7 +25,6 @@ struct TrivialWeightFunction enum class LRUCacheEvictStatus { CAN_EVITCT, // a key can be evicted - TERMINATE_EVICT, // stop the evicting process SKIP_EVICT, // skip current value and keep iterating }; @@ -357,13 +356,13 @@ private: return true; } - bool removeOverflow(size_t more_size = 0) + bool removeOverflow(size_t required_size_to_remove = 0) { size_t current_weight_lost = 0; size_t queue_size = cells.size(); auto key_it = queue.begin(); - while ((current_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) + while ((current_size + required_size_to_remove > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) && (queue_size > 1) && (key_it != queue.end())) { @@ -378,32 +377,28 @@ private: const auto & cell = it->second; auto evict_status = evict_policy.canRelease(*cell.value);// in default, it is CAN_EVITCT - if (evict_status == LRUCacheEvictStatus::CAN_EVITCT) + switch (evict_status) { - // always call release() before erasing an element - // in default, it's an empty action - evict_policy.release(*cell.value); + case LRUCacheEvictStatus::CAN_EVITCT: + { + // always call release() before erasing an element + // in default, it's an empty action + evict_policy.release(*cell.value); - current_size -= cell.size; - current_weight_lost += cell.size; + current_size -= cell.size; + current_weight_lost += cell.size; - cells.erase(it); - queue.pop_front(); - --queue_size; + cells.erase(it); + queue.pop_front(); + --queue_size; + } + break; + case LRUCacheEvictStatus::SKIP_EVICT: + { + key_it++; + } + break; } - else if (evict_status == LRUCacheEvictStatus::SKIP_EVICT) - { - // skip this element and try to evict the remaining ones. - key_it++; - continue; - } - else if (evict_status == LRUCacheEvictStatus::TERMINATE_EVICT) - { - // maybe we want to stop this iteration once we meet the first unreleasable element - break; - } - LOG_ERROR(&Poco::Logger::get("LRUCache"), "This condition branch should not be reached."); - abort(); } onRemoveOverflowWeightLoss(current_weight_lost); @@ -413,7 +408,7 @@ private: LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); abort(); } - return !(current_size + more_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)); + return !(current_size + required_size_to_remove > max_size || (max_elements_size != 0 && queue_size > max_elements_size)); } /// Override this method if you want to track how much weight was lost in removeOverflow method. diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index 7219bda2efd..f5f52de1963 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -399,13 +399,9 @@ void RemoteReadBufferCache::initOnce( lru_caches = std::make_unique(limit_size_); /// create if root_dir not exists - if (!fs::exists(fs::path(root_dir) / "")) + if (!fs::exists(fs::path(root_dir))) { - std::error_code ec; - bool success = fs::create_directories(fs::path(root_dir) / "", ec); - if (!success) - throw Exception( - ErrorCodes::CANNOT_CREATE_DIRECTORY, "Failed to create directories, error code:{} reason:{}", ec.value(), ec.message()); + fs::create_directories(fs::path(root_dir)); } recover_task_holder = context->getSchedulePool().createTask("recover local cache metadata for remote files", [this]{ recoverTask(); }); From 8ab2777dd5bc4804f3e9cc42f259cab0759408b1 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 7 Dec 2021 15:51:47 +0000 Subject: [PATCH 0137/1260] fix --- src/Functions/FunctionsComparison.h | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 3f5146935cb..dcdbb3738f8 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -687,7 +687,7 @@ private: return (res = DecimalComparison::apply(col_left, col_right)) != nullptr; }; - if (!callOnBasicTypes(left_number, right_number, call)) + if (!callOnBasicTypes(left_number, right_number, call)) throw Exception("Wrong call for " + getName() + " with " + col_left.type->getName() + " and " + col_right.type->getName(), ErrorCodes::LOGICAL_ERROR); @@ -1175,8 +1175,11 @@ public: const bool left_is_num = col_left_untyped->isNumeric(); const bool right_is_num = col_right_untyped->isNumeric(); - const bool left_is_string = isStringOrFixedString(which_left); - const bool right_is_string = isStringOrFixedString(which_right); + const bool left_is_float = which_left.isFloat(); + const bool right_is_float = which_right.isFloat(); + + const bool left_is_string = which_left.isStringOrFixedString(); + const bool right_is_string = which_right.isStringOrFixedString(); bool date_and_datetime = (which_left.idx != which_right.idx) && (which_left.isDate() || which_left.isDate32() || which_left.isDateTime() || which_left.isDateTime64()) && (which_right.isDate() || which_right.isDate32() || which_right.isDateTime() || which_right.isDateTime64()); @@ -1237,6 +1240,16 @@ public: throw Exception( "No operation " + getName() + " between " + left_type->getName() + " and " + right_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (left_is_float) + { + ColumnPtr left_converted = castColumn(col_with_type_and_name_left, right_type); + return executeDecimal({left_converted, right_type, "left"}, col_with_type_and_name_right); + } + if (right_is_float) + { + ColumnPtr right_converted = castColumn(col_with_type_and_name_right, left_type); + return executeDecimal(col_with_type_and_name_left, {right_converted, left_type, "right"}); + } return executeDecimal(col_with_type_and_name_left, col_with_type_and_name_right); } From 38020a33e4d265d4941f4c0ab76aca86cd9e6c5c Mon Sep 17 00:00:00 2001 From: lgbo Date: Wed, 8 Dec 2021 10:07:04 +0800 Subject: [PATCH 0138/1260] some refinement --- src/Common/LRUCache.h | 48 ++++++++++++-------------------- src/IO/RemoteReadBufferCache.cpp | 5 ++-- src/IO/RemoteReadBufferCache.h | 6 ++-- 3 files changed, 22 insertions(+), 37 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 1f3b7fb09d2..183004cb8e5 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -22,18 +22,12 @@ struct TrivialWeightFunction } }; -enum class LRUCacheEvictStatus -{ - CAN_EVITCT, // a key can be evicted - SKIP_EVICT, // skip current value and keep iterating -}; - template struct TrivialLRUCacheEvitPolicy { - inline LRUCacheEvictStatus canRelease(const T &) const + inline bool canRelease(const T &) const { - return LRUCacheEvictStatus::CAN_EVITCT; + return true; } inline void release(T &) @@ -337,10 +331,10 @@ private: } else { - if (evict_policy.canRelease(*cell.value) != LRUCacheEvictStatus::CAN_EVITCT) + if (!evict_policy.canRelease(*cell.value)) { // the old value is refered by someone, cannot release now - // in default policy, it is always CAN_EVITCT. + // in default policy, it is always true. return false; } evict_policy.release(*cell.value); // release the old value. this action is empty in default policy. @@ -351,7 +345,6 @@ private: cell.value = mapped; cell.size = cell.value ? weight_function(*cell.value) : 0; current_size += cell.size; - removeOverflow(); return true; } @@ -376,28 +369,23 @@ private: } const auto & cell = it->second; - auto evict_status = evict_policy.canRelease(*cell.value);// in default, it is CAN_EVITCT - switch (evict_status) + auto can_evict = evict_policy.canRelease(*cell.value);// in default, it is true + if (can_evict) { - case LRUCacheEvictStatus::CAN_EVITCT: - { - // always call release() before erasing an element - // in default, it's an empty action - evict_policy.release(*cell.value); + // always call release() before erasing an element + // in default, it's an empty action + evict_policy.release(*cell.value); - current_size -= cell.size; - current_weight_lost += cell.size; + current_size -= cell.size; + current_weight_lost += cell.size; - cells.erase(it); - queue.pop_front(); - --queue_size; - } - break; - case LRUCacheEvictStatus::SKIP_EVICT: - { - key_it++; - } - break; + cells.erase(it); + queue.pop_front(); + --queue_size; + } + else + { + key_it++; } } diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index f5f52de1963..d7781ee27b1 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -66,7 +66,7 @@ std::shared_ptr RemoteCacheController::recover(const std: LOG_ERROR(log, "Cannot create the metadata class : {}. The cached file is invalid and will be remove. path:{}", cache_controller->metadata_class, local_path_.string()); - return nullptr; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid metadata class:{}", cache_controller->metadata_class); } std::ifstream metadata_file(local_path_ / "metadata.txt"); if (!cache_controller->file_metadata_ptr->fromString(std::string((std::istreambuf_iterator(metadata_file)), @@ -74,7 +74,7 @@ std::shared_ptr RemoteCacheController::recover(const std: { LOG_ERROR(log, "Cannot load the metadata. The cached file is invalid and will be remove. path:{}", local_path_.string()); - return nullptr; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid metadata file({}) for meta class {}", local_path_.string(), cache_controller->metadata_class); } cache_controller->current_offset = fs::file_size(local_path_ / "data.bin"); @@ -215,7 +215,6 @@ void RemoteCacheController::close() std::unique_ptr RemoteCacheController::allocFile() { ReadSettings settings; - settings.local_fs_prefetch = false; settings.local_fs_method = LocalFSReadMethod::read; auto file_buffer = createReadBufferFromFileBase((local_path / "data.bin").string(), settings); diff --git a/src/IO/RemoteReadBufferCache.h b/src/IO/RemoteReadBufferCache.h index a9243f92e97..21126601278 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/IO/RemoteReadBufferCache.h @@ -163,11 +163,9 @@ struct RemoteFileCacheWeightFunction struct RemoteFileCacheEvictPolicy { - LRUCacheEvictStatus canRelease(RemoteCacheController & cache) const + bool canRelease(RemoteCacheController & cache) const { - if (cache.closable()) - return LRUCacheEvictStatus::CAN_EVITCT; - return LRUCacheEvictStatus::SKIP_EVICT; + return cache.closable(); } void release(RemoteCacheController & cache) { From d0c5a887a35a2b89ff8adff600a44ba797235792 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 8 Dec 2021 13:16:55 +0000 Subject: [PATCH 0139/1260] update tests --- .../02124_comparison_betwwen_decimal_and_float.reference | 7 ++++++- .../02124_comparison_betwwen_decimal_and_float.sql | 6 ++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.reference b/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.reference index eebbb0f7d4c..6ca3014f7ee 100644 --- a/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.reference +++ b/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.reference @@ -1,8 +1,8 @@ 0 1 0 -1 0 +1 0 0 1 @@ -11,3 +11,8 @@ 0 0 1 +0 +0 +1 +1 +1 diff --git a/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.sql b/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.sql index a2ab6c120f0..19808d6fbd9 100644 --- a/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.sql +++ b/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.sql @@ -1,14 +1,20 @@ select CAST(1.0, 'Decimal(15,2)') > CAST(1, 'Float64'); select CAST(1.0, 'Decimal(15,2)') = CAST(1, 'Float64'); select CAST(1.0, 'Decimal(15,2)') < CAST(1, 'Float64'); +select CAST(1.0, 'Decimal(15,2)') != CAST(1, 'Float64'); select CAST(1.0, 'Decimal(15,2)') > CAST(-1, 'Float64'); select CAST(1.0, 'Decimal(15,2)') = CAST(-1, 'Float64'); select CAST(1.0, 'Decimal(15,2)') < CAST(-1, 'Float64'); +select CAST(1.0, 'Decimal(15,2)') != CAST(-1, 'Float64'); select CAST(1.0, 'Decimal(15,2)') > CAST(1, 'Float32'); select CAST(1.0, 'Decimal(15,2)') = CAST(1, 'Float32'); select CAST(1.0, 'Decimal(15,2)') < CAST(1, 'Float32'); +select CAST(1.0, 'Decimal(15,2)') != CAST(1, 'Float32'); select CAST(1.0, 'Decimal(15,2)') > CAST(-1, 'Float32'); select CAST(1.0, 'Decimal(15,2)') = CAST(-1, 'Float32'); select CAST(1.0, 'Decimal(15,2)') < CAST(-1, 'Float32'); +select CAST(1.0, 'Decimal(15,2)') != CAST(-1, 'Float32'); SELECT toDecimal32('11.00', 2) > 1.; + +SELECT 0.1 > 0.1::Decimal256(70); From 0eef092ded352e9b9087c6362d7b443e416f9939 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 9 Dec 2021 16:57:33 +0800 Subject: [PATCH 0140/1260] fix building --- src/IO/RemoteReadBufferCache.cpp | 1 - src/Storages/Hive/StorageHive.cpp | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/IO/RemoteReadBufferCache.cpp index d7781ee27b1..9df0fa25329 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/IO/RemoteReadBufferCache.cpp @@ -23,7 +23,6 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; - extern const int CANNOT_CREATE_DIRECTORY; } bool RemoteCacheController::loadInnerInformation(const fs::path & file_path) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index bd53ce22925..49abc8287a4 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -371,7 +371,7 @@ ASTPtr StorageHive::extractKeyExpressionList(const ASTPtr & node) /// Primary key consists of one column. auto res = std::make_shared(); res->children.push_back(node); - return std::move(res); + return res; } } From 9fe648ef8e2e6e7a55007c4147d2ea0a6164325d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 2 Dec 2021 13:30:15 +0100 Subject: [PATCH 0141/1260] Force PipeLineExecutor creators to pass a QueryStatus This is to avoid future issues --- src/Core/ExternalTable.cpp | 2 +- src/Processors/Executors/PipelineExecutor.h | 2 +- .../tests/gtest_exception_on_incorrect_pipeline.cpp | 6 ++++-- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index b4adbcc0662..3b515fab5c9 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -169,7 +169,7 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, processors.push_back(std::move(sink)); processors.push_back(std::move(exception_handling)); - auto executor = std::make_shared(processors); + auto executor = std::make_shared(processors, getContext()->getProcessListElement()); executor->execute(/*num_threads = */ 1); /// We are ready to receive the next file, for this we clear all the information received diff --git a/src/Processors/Executors/PipelineExecutor.h b/src/Processors/Executors/PipelineExecutor.h index 12f2bd8b75b..0b1fe5dedf6 100644 --- a/src/Processors/Executors/PipelineExecutor.h +++ b/src/Processors/Executors/PipelineExecutor.h @@ -26,7 +26,7 @@ public: /// During pipeline execution new processors can appear. They will be added to existing set. /// /// Explicit graph representation is built in constructor. Throws if graph is not correct. - explicit PipelineExecutor(Processors & processors, QueryStatus * elem = nullptr); + explicit PipelineExecutor(Processors & processors, QueryStatus * elem); ~PipelineExecutor(); /// Execute pipeline in multiple threads. Must be called once. diff --git a/src/Processors/tests/gtest_exception_on_incorrect_pipeline.cpp b/src/Processors/tests/gtest_exception_on_incorrect_pipeline.cpp index df3901e2eb1..ee661b39fac 100644 --- a/src/Processors/tests/gtest_exception_on_incorrect_pipeline.cpp +++ b/src/Processors/tests/gtest_exception_on_incorrect_pipeline.cpp @@ -27,7 +27,8 @@ TEST(Processors, PortsConnected) processors.emplace_back(std::move(source)); processors.emplace_back(std::move(sink)); - PipelineExecutor executor(processors); + QueryStatus * element = nullptr; + PipelineExecutor executor(processors, element); executor.execute(1); } @@ -51,7 +52,8 @@ TEST(Processors, PortsNotConnected) try { - PipelineExecutor executor(processors); + QueryStatus * element = nullptr; + PipelineExecutor executor(processors, element); executor.execute(1); ASSERT_TRUE(false) << "Should have thrown."; } From edf4095139e2c0518fa6985dd82b23762f25e8bd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 2 Dec 2021 18:18:52 +0100 Subject: [PATCH 0142/1260] Add QueryStatus to BuildQueryPipelineSettings and use it --- .../QueryPlan/BuildQueryPipelineSettings.cpp | 10 +++------- src/Processors/QueryPlan/BuildQueryPipelineSettings.h | 4 ++-- src/Processors/QueryPlan/QueryPlan.cpp | 2 ++ src/QueryPipeline/QueryPipelineBuilder.cpp | 1 + 4 files changed, 8 insertions(+), 9 deletions(-) diff --git a/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp b/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp index 2480673d65e..59ed1226787 100644 --- a/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp +++ b/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp @@ -6,16 +6,12 @@ namespace DB { -BuildQueryPipelineSettings BuildQueryPipelineSettings::fromSettings(const Settings & from) +BuildQueryPipelineSettings BuildQueryPipelineSettings::fromContext(ContextPtr from) { BuildQueryPipelineSettings settings; - settings.actions_settings = ExpressionActionsSettings::fromSettings(from, CompileExpressions::yes); + settings.actions_settings = ExpressionActionsSettings::fromSettings(from->getSettingsRef(), CompileExpressions::yes); + settings.process_list_element = from->getProcessListElement(); return settings; } -BuildQueryPipelineSettings BuildQueryPipelineSettings::fromContext(ContextPtr from) -{ - return fromSettings(from->getSettingsRef()); -} - } diff --git a/src/Processors/QueryPlan/BuildQueryPipelineSettings.h b/src/Processors/QueryPlan/BuildQueryPipelineSettings.h index c3282d43778..f106782ae5b 100644 --- a/src/Processors/QueryPlan/BuildQueryPipelineSettings.h +++ b/src/Processors/QueryPlan/BuildQueryPipelineSettings.h @@ -8,14 +8,14 @@ namespace DB { struct Settings; +class QueryStatus; struct BuildQueryPipelineSettings { ExpressionActionsSettings actions_settings; + QueryStatus * process_list_element = nullptr; const ExpressionActionsSettings & getActionsSettings() const { return actions_settings; } - - static BuildQueryPipelineSettings fromSettings(const Settings & from); static BuildQueryPipelineSettings fromContext(ContextPtr from); }; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index f319e562bfb..6affecb5617 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -180,6 +180,8 @@ QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline( for (auto & context : interpreter_context) last_pipeline->addInterpreterContext(std::move(context)); + last_pipeline->setProcessListElement(build_pipeline_settings.process_list_element); + return last_pipeline; } diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 40c64046560..dba7c7cb8f7 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -560,6 +560,7 @@ QueryPipeline QueryPipelineBuilder::getPipeline(QueryPipelineBuilder builder) { QueryPipeline res(std::move(builder.pipe)); res.setNumThreads(builder.getNumThreads()); + res.setProcessListElement(builder.process_list_element); return res; } From 180055b577bda453830518c6dbbe44a95ea2005d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 6 Dec 2021 12:50:46 +0100 Subject: [PATCH 0143/1260] Test that scalar queries can be killed --- .../02132_kill_scalar_queries.reference | 4 ++++ .../0_stateless/02132_kill_scalar_queries.sh | 21 +++++++++++++++++++ 2 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/02132_kill_scalar_queries.reference create mode 100755 tests/queries/0_stateless/02132_kill_scalar_queries.sh diff --git a/tests/queries/0_stateless/02132_kill_scalar_queries.reference b/tests/queries/0_stateless/02132_kill_scalar_queries.reference new file mode 100644 index 00000000000..7d0d98e85f0 --- /dev/null +++ b/tests/queries/0_stateless/02132_kill_scalar_queries.reference @@ -0,0 +1,4 @@ +OK +finished default_TEST02132KILL_QUERY1 default select (SELECT max(number) from system.numbers) + 1; +OK +finished default_TEST02132KILL_QUERY2 default SELECT (SELECT number FROM system.numbers WHERE number = 1000000000000); diff --git a/tests/queries/0_stateless/02132_kill_scalar_queries.sh b/tests/queries/0_stateless/02132_kill_scalar_queries.sh new file mode 100755 index 00000000000..ae7348a1f0b --- /dev/null +++ b/tests/queries/0_stateless/02132_kill_scalar_queries.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash +# Ref: https://github.com/ClickHouse/ClickHouse/issues/1576 +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +function wait_for_query_to_start() +{ + while [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() FROM system.processes WHERE query_id = '$1'") == 0 ]]; do sleep 0.1; done +} + +QUERY_1_ID="${CLICKHOUSE_DATABASE}_TEST02132KILL_QUERY1" +(${CLICKHOUSE_CLIENT} --query_id="${QUERY_1_ID}" --query='select (SELECT max(number) from system.numbers) + 1;' 2>&1 | grep -q "Code: 394." && echo 'OK' || echo 'FAIL' ||: ) & +wait_for_query_to_start "${QUERY_1_ID}" +${CLICKHOUSE_CLIENT} --query="KILL QUERY WHERE query_id='${QUERY_1_ID}' SYNC" + +QUERY_2_ID="${CLICKHOUSE_DATABASE}_TEST02132KILL_QUERY2" +(${CLICKHOUSE_CLIENT} --query_id="${QUERY_2_ID}" --query='SELECT (SELECT number FROM system.numbers WHERE number = 1000000000000);' 2>&1 | grep -q "Code: 394." && echo 'OK' || echo 'FAIL' ||: ) & +wait_for_query_to_start "${QUERY_2_ID}" +${CLICKHOUSE_CLIENT} --query="KILL QUERY WHERE query_id='${QUERY_2_ID}' SYNC" + From 0158133b62876665d6b4db17bfe02b355aa92bf5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 6 Dec 2021 15:02:43 +0100 Subject: [PATCH 0144/1260] SourceWithProgress: Don't use process_list_elem without checking it --- src/Interpreters/ProcessList.h | 2 +- src/Processors/Sources/SourceWithProgress.cpp | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 9c826bde061..0b17874836f 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -94,7 +94,7 @@ protected: ExecutionSpeedLimits limits; OverflowMode overflow_mode; - QueryPriorities::Handle priority_handle; + QueryPriorities::Handle priority_handle = nullptr; std::atomic is_killed { false }; diff --git a/src/Processors/Sources/SourceWithProgress.cpp b/src/Processors/Sources/SourceWithProgress.cpp index 9b7a5c6a762..60c39c919f6 100644 --- a/src/Processors/Sources/SourceWithProgress.cpp +++ b/src/Processors/Sources/SourceWithProgress.cpp @@ -26,6 +26,8 @@ SourceWithProgress::SourceWithProgress(Block header, bool enable_auto_progress) void SourceWithProgress::setProcessListElement(QueryStatus * elem) { process_list_elem = elem; + if (!elem) + return; /// Update total_rows_approx as soon as possible. /// From 3fc4167c54bb5399fddf9c60d73f89a977b0379d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 9 Dec 2021 14:00:12 +0100 Subject: [PATCH 0145/1260] Rework how progress is reported in views --- src/Interpreters/ProcessList.cpp | 5 +- src/Interpreters/ThreadStatusExt.cpp | 11 +- src/Interpreters/executeQuery.cpp | 4 +- .../Transforms/CountingTransform.cpp | 11 +- .../Transforms/buildPushingToViewsChain.cpp | 29 ++-- ...query_views_log_current_database.reference | 136 ++++++++++-------- ...01927_query_views_log_current_database.sql | 24 +++- .../02125_query_views_log.reference | 4 +- 8 files changed, 138 insertions(+), 86 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index a4583685a90..67871c630d8 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -411,9 +411,8 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even res.read_bytes = progress_in.read_bytes; res.total_rows = progress_in.total_rows_to_read; - /// TODO: Use written_rows and written_bytes when real time progress is implemented - res.written_rows = progress_out.read_rows; - res.written_bytes = progress_out.read_bytes; + res.written_rows = progress_out.written_rows; + res.written_bytes = progress_out.written_bytes; if (thread_group) { diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index fc6aa15a1e8..55e94f709e6 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -28,6 +28,8 @@ namespace ProfileEvents { extern const Event InsertedRows; extern const Event InsertedBytes; + extern const Event SelectedRows; + extern const Event SelectedBytes; } @@ -447,9 +449,8 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String elem.read_rows = progress_in.read_rows.load(std::memory_order_relaxed); elem.read_bytes = progress_in.read_bytes.load(std::memory_order_relaxed); - /// TODO: Use written_rows and written_bytes when run time progress is implemented - elem.written_rows = progress_out.read_rows.load(std::memory_order_relaxed); - elem.written_bytes = progress_out.read_bytes.load(std::memory_order_relaxed); + elem.written_rows = progress_out.written_rows.load(std::memory_order_relaxed); + elem.written_bytes = progress_out.written_bytes.load(std::memory_order_relaxed); elem.memory_usage = memory_tracker.get(); elem.peak_memory_usage = memory_tracker.getPeak(); @@ -520,8 +521,8 @@ void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo) auto events = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); element.read_rows = progress_in.read_rows.load(std::memory_order_relaxed); element.read_bytes = progress_in.read_bytes.load(std::memory_order_relaxed); - element.written_rows = (*events)[ProfileEvents::InsertedRows]; - element.written_bytes = (*events)[ProfileEvents::InsertedBytes]; + element.written_rows = progress_out.written_rows.load(std::memory_order_relaxed); + element.written_bytes = progress_out.written_bytes.load(std::memory_order_relaxed); element.peak_memory_usage = memory_tracker.getPeak() > 0 ? memory_tracker.getPeak() : 0; if (query_context_ptr->getSettingsRef().log_profile_events != 0) { diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index b69bbcc6332..b8da7c97c3e 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -819,8 +819,8 @@ static std::tuple executeQueryImpl( else /// will be used only for ordinary INSERT queries { auto progress_out = process_list_elem->getProgressOut(); - elem.result_rows = progress_out.read_rows; - elem.result_bytes = progress_out.read_bytes; + elem.result_rows = progress_out.written_rows; + elem.result_bytes = progress_out.written_rows; } if (elem.read_rows != 0) diff --git a/src/Processors/Transforms/CountingTransform.cpp b/src/Processors/Transforms/CountingTransform.cpp index 88ecbe6adc3..78e30557139 100644 --- a/src/Processors/Transforms/CountingTransform.cpp +++ b/src/Processors/Transforms/CountingTransform.cpp @@ -18,20 +18,21 @@ namespace DB void CountingTransform::transform(Chunk & chunk) { - Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); + Progress local_progress{WriteProgress(chunk.getNumRows(), chunk.bytes())}; progress.incrementPiecewiseAtomically(local_progress); //std::cerr << "============ counting adding progress for " << static_cast(thread_status) << ' ' << chunk.getNumRows() << " rows\n"; if (thread_status) { - thread_status->performance_counters.increment(ProfileEvents::InsertedRows, local_progress.read_rows); - thread_status->performance_counters.increment(ProfileEvents::InsertedBytes, local_progress.read_bytes); + thread_status->performance_counters.increment(ProfileEvents::InsertedRows, local_progress.written_rows); + thread_status->performance_counters.increment(ProfileEvents::InsertedBytes, local_progress.written_bytes); + thread_status->progress_out.incrementPiecewiseAtomically(local_progress); } else { - ProfileEvents::increment(ProfileEvents::InsertedRows, local_progress.read_rows); - ProfileEvents::increment(ProfileEvents::InsertedBytes, local_progress.read_bytes); + ProfileEvents::increment(ProfileEvents::InsertedRows, local_progress.written_rows); + ProfileEvents::increment(ProfileEvents::InsertedBytes, local_progress.written_bytes); } if (process_elem) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 503f34593c7..eb5e1eddc24 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -14,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -23,6 +25,14 @@ #include #include +namespace ProfileEvents +{ + extern const Event InsertedBytes; + extern const Event InsertedRows; + extern const Event SelectedBytes; + extern const Event SelectedRows; +} + namespace DB { @@ -436,13 +446,6 @@ static void process(Block & block, ViewRuntimeData & view, const ViewsData & vie pipeline.getHeader(), std::make_shared(std::move(converting)))); - pipeline.setProgressCallback([context](const Progress & progress) - { - CurrentThread::updateProgressIn(progress); - if (auto callback = context->getProgressCallback()) - callback(progress); - }); - auto query_pipeline = QueryPipelineBuilder::getPipeline(std::move(pipeline)); PullingPipelineExecutor executor(query_pipeline); if (!executor.pull(block)) @@ -570,7 +573,11 @@ void PushingToLiveViewSink::consume(Chunk chunk) { Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); StorageLiveView::writeIntoLiveView(live_view, getHeader().cloneWithColumns(chunk.detachColumns()), context); - CurrentThread::updateProgressIn(local_progress); + auto process = context->getProcessListElement(); + if (process) + process->updateProgressIn(local_progress); + ProfileEvents::increment(ProfileEvents::SelectedRows, local_progress.read_rows); + ProfileEvents::increment(ProfileEvents::SelectedBytes, local_progress.read_bytes); } @@ -589,7 +596,11 @@ void PushingToWindowViewSink::consume(Chunk chunk) Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); StorageWindowView::writeIntoWindowView( window_view, getHeader().cloneWithColumns(chunk.detachColumns()), context); - CurrentThread::updateProgressIn(local_progress); + auto process = context->getProcessListElement(); + if (process) + process->updateProgressIn(local_progress); + ProfileEvents::increment(ProfileEvents::SelectedRows, local_progress.read_rows); + ProfileEvents::increment(ProfileEvents::SelectedBytes, local_progress.read_bytes); } diff --git a/tests/queries/0_stateless/01927_query_views_log_current_database.reference b/tests/queries/0_stateless/01927_query_views_log_current_database.reference index ff9eca2d97f..eaa1e98c55c 100644 --- a/tests/queries/0_stateless/01927_query_views_log_current_database.reference +++ b/tests/queries/0_stateless/01927_query_views_log_current_database.reference @@ -1,70 +1,94 @@ Row 1: ────── -stage: Query log rows -read_rows: 100 -written_rows: 201 -databases: ['_table_function','default'] -tables: ['_table_function.numbers','default.table_a','default.table_b','default.table_b_live_view','default.table_c'] -views: ['default.matview_a_to_b','default.matview_b_to_c','default.table_b_live_view'] -sleep_calls: 200 -sleep_us: 298 +stage: Query log rows +read_rows: 400 +written_rows: 201 +databases: ['_table_function','default'] +tables: ['_table_function.numbers','default.table_a','default.table_b','default.table_b_live_view','default.table_c'] +views: ['default.matview_a_to_b','default.matview_b_to_c','default.table_b_live_view'] +sleep_calls: 200 +sleep_us: 298 +profile_select_rows: 400 +profile_select_bytes: 5200 +profile_insert_rows: 201 +profile_insert_bytes: 2808 Row 1: ────── -stage: Depending views -view_name: default.matview_a_to_b -view_type: Materialized -status: QueryFinish -view_target: default.table_b -view_query: SELECT toFloat64(a) AS a, b + sleepEachRow(0.000001) AS count FROM default.table_a -read_rows: 100 -written_rows: 100 -sleep_calls: 100 -sleep_us: 99 +stage: Depending views +view_name: default.matview_a_to_b +view_type: Materialized +status: QueryFinish +view_target: default.table_b +view_query: SELECT toFloat64(a) AS a, b + sleepEachRow(0.000001) AS count FROM default.table_a +read_rows: 100 +written_rows: 100 +sleep_calls: 100 +sleep_us: 99 +profile_select_rows: 100 +profile_select_bytes: 2000 +profile_insert_rows: 100 +profile_insert_bytes: 800 Row 2: ────── -stage: Depending views -view_name: default.matview_b_to_c -view_type: Materialized -status: QueryFinish -view_target: default.table_c -view_query: SELECT sum(a + sleepEachRow(0.000002)) AS a FROM default.table_b -read_rows: 100 -written_rows: 1 -sleep_calls: 100 -sleep_us: 199 +stage: Depending views +view_name: default.matview_b_to_c +view_type: Materialized +status: QueryFinish +view_target: default.table_c +view_query: SELECT sum(a + sleepEachRow(0.000002)) AS a FROM default.table_b +read_rows: 100 +written_rows: 1 +sleep_calls: 100 +sleep_us: 199 +profile_select_rows: 100 +profile_select_bytes: 800 +profile_insert_rows: 1 +profile_insert_bytes: 8 Row 3: ────── -stage: Depending views -view_name: default.table_b_live_view -view_type: Live -status: QueryFinish -view_target: default.table_b_live_view -view_query: SELECT sum(a + b) FROM default.table_b -read_rows: 100 -written_rows: 0 -sleep_calls: 0 -sleep_us: 0 +stage: Depending views +view_name: default.table_b_live_view +view_type: Live +status: QueryFinish +view_target: default.table_b_live_view +view_query: SELECT sum(a + b) FROM default.table_b +read_rows: 100 +written_rows: 0 +sleep_calls: 0 +sleep_us: 0 +profile_select_rows: 100 +profile_select_bytes: 1600 +profile_insert_rows: 0 +profile_insert_bytes: 0 Row 1: ────── -stage: Query log rows 2 -read_rows: 50 -written_rows: 100 -databases: ['_table_function','default'] -tables: ['_table_function.numbers','default.table_d','default.table_e','default.table_f'] -views: ['default.matview_join_d_e'] -sleep_calls: 50 -sleep_us: 150 +stage: Query log rows 2 +read_rows: 100 +written_rows: 100 +databases: ['_table_function','default'] +tables: ['_table_function.numbers','default.table_d','default.table_e','default.table_f'] +views: ['default.matview_join_d_e'] +sleep_calls: 50 +sleep_us: 150 +profile_select_rows: 100 +profile_select_bytes: 800 +profile_insert_rows: 100 +profile_insert_bytes: 1600 Row 1: ────── -stage: Depending views 2 -view_name: default.matview_join_d_e -view_type: Materialized -status: QueryFinish -view_target: default.table_f -view_query: SELECT table_d.a AS a, table_e.count + sleepEachRow(0.000003) AS count FROM default.table_d LEFT JOIN default.table_e ON table_d.a = table_e.a -read_rows: 50 -written_rows: 50 -sleep_calls: 50 -sleep_us: 150 +stage: Depending views 2 +view_name: default.matview_join_d_e +view_type: Materialized +status: QueryFinish +view_target: default.table_f +view_query: SELECT table_d.a AS a, table_e.count + sleepEachRow(0.000003) AS count FROM default.table_d LEFT JOIN default.table_e ON table_d.a = table_e.a +read_rows: 50 +written_rows: 50 +sleep_calls: 50 +sleep_us: 150 +profile_select_rows: 50 +profile_select_bytes: 400 +profile_insert_rows: 50 +profile_insert_bytes: 800 diff --git a/tests/queries/0_stateless/01927_query_views_log_current_database.sql b/tests/queries/0_stateless/01927_query_views_log_current_database.sql index 40ab8c8e16a..fbfbeab0167 100644 --- a/tests/queries/0_stateless/01927_query_views_log_current_database.sql +++ b/tests/queries/0_stateless/01927_query_views_log_current_database.sql @@ -45,7 +45,11 @@ SELECT arraySort(tables) as tables, arraySort(views) as views, ProfileEvents['SleepFunctionCalls'] as sleep_calls, - ProfileEvents['SleepFunctionMicroseconds'] as sleep_us + ProfileEvents['SleepFunctionMicroseconds'] as sleep_us, + ProfileEvents['SelectedRows'] as profile_select_rows, + ProfileEvents['SelectedBytes'] as profile_select_bytes, + ProfileEvents['InsertedRows'] as profile_insert_rows, + ProfileEvents['InsertedBytes'] as profile_insert_bytes FROM system.query_log WHERE query like '-- INSERT 1%INSERT INTO table_a%' AND current_database = currentDatabase() @@ -62,7 +66,11 @@ SELECT read_rows, written_rows, ProfileEvents['SleepFunctionCalls'] as sleep_calls, - ProfileEvents['SleepFunctionMicroseconds'] as sleep_us + ProfileEvents['SleepFunctionMicroseconds'] as sleep_us, + ProfileEvents['SelectedRows'] as profile_select_rows, + ProfileEvents['SelectedBytes'] as profile_select_bytes, + ProfileEvents['InsertedRows'] as profile_insert_rows, + ProfileEvents['InsertedBytes'] as profile_insert_bytes FROM system.query_views_log WHERE initial_query_id = ( @@ -85,7 +93,11 @@ SELECT arraySort(tables) as tables, arraySort(views) as views, ProfileEvents['SleepFunctionCalls'] as sleep_calls, - ProfileEvents['SleepFunctionMicroseconds'] as sleep_us + ProfileEvents['SleepFunctionMicroseconds'] as sleep_us, + ProfileEvents['SelectedRows'] as profile_select_rows, + ProfileEvents['SelectedBytes'] as profile_select_bytes, + ProfileEvents['InsertedRows'] as profile_insert_rows, + ProfileEvents['InsertedBytes'] as profile_insert_bytes FROM system.query_log WHERE query like '-- INSERT 2%INSERT INTO table_d%' AND current_database = currentDatabase() @@ -102,7 +114,11 @@ SELECT read_rows, written_rows, ProfileEvents['SleepFunctionCalls'] as sleep_calls, - ProfileEvents['SleepFunctionMicroseconds'] as sleep_us + ProfileEvents['SleepFunctionMicroseconds'] as sleep_us, + ProfileEvents['SelectedRows'] as profile_select_rows, + ProfileEvents['SelectedBytes'] as profile_select_bytes, + ProfileEvents['InsertedRows'] as profile_insert_rows, + ProfileEvents['InsertedBytes'] as profile_insert_bytes FROM system.query_views_log WHERE initial_query_id = ( diff --git a/tests/queries/0_stateless/02125_query_views_log.reference b/tests/queries/0_stateless/02125_query_views_log.reference index 3ae4af9b4d0..fac70027113 100644 --- a/tests/queries/0_stateless/02125_query_views_log.reference +++ b/tests/queries/0_stateless/02125_query_views_log.reference @@ -18,7 +18,7 @@ written_bytes: 4000000 select read_rows, read_bytes, written_rows, written_bytes from system.query_log where type = 'QueryFinish' and query_kind = 'Insert' and current_database = currentDatabase() format Vertical; Row 1: ────── -read_rows: 1000000 -read_bytes: 8000000 +read_rows: 3000000 +read_bytes: 16000000 written_rows: 3000000 written_bytes: 12000000 From 10f18b8d7b9fe67f401002b447fc10712d6c793a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 9 Dec 2021 18:15:11 +0100 Subject: [PATCH 0146/1260] Adapt tests to count properly --- .../queries/0_stateless/00945_bloom_filter_index.sql | 8 ++++---- ...ncremental_streaming_from_2_src_with_feedback.sql | 9 ++++++--- .../0_stateless/01583_const_column_in_set_index.sql | 2 +- .../0_stateless/01585_use_index_for_global_in.sql | 4 +++- ...01585_use_index_for_global_in_with_null.reference | 8 ++++++++ .../01585_use_index_for_global_in_with_null.sql | 12 ++++++++++++ .../0_stateless/01748_partition_id_pruning.sql | 4 ++-- 7 files changed, 36 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/00945_bloom_filter_index.sql b/tests/queries/0_stateless/00945_bloom_filter_index.sql index f45c4c04290..d72f5ad1c6d 100644 --- a/tests/queries/0_stateless/00945_bloom_filter_index.sql +++ b/tests/queries/0_stateless/00945_bloom_filter_index.sql @@ -14,10 +14,10 @@ SELECT COUNT() FROM single_column_bloom_filter WHERE i32 IN (1, 2) SETTINGS max_ SELECT COUNT() FROM single_column_bloom_filter WHERE (i32, i32) IN ((1, 2), (2, 3)) SETTINGS max_rows_to_read = 6; SELECT COUNT() FROM single_column_bloom_filter WHERE (i32, i64) IN ((1, 1), (2, 2)) SETTINGS max_rows_to_read = 6; SELECT COUNT() FROM single_column_bloom_filter WHERE (i64, (i64, i32)) IN ((1, (1, 1)), (2, (2, 2))) SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM single_column_bloom_filter WHERE i32 IN (SELECT arrayJoin([toInt32(1), toInt32(2)])) SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM single_column_bloom_filter WHERE (i32, i32) IN (SELECT arrayJoin([(toInt32(1), toInt32(2)), (toInt32(2), toInt32(3))])) SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM single_column_bloom_filter WHERE (i32, i64) IN (SELECT arrayJoin([(toInt32(1), toUInt64(1)), (toInt32(2), toUInt64(2))])) SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM single_column_bloom_filter WHERE (i64, (i64, i32)) IN (SELECT arrayJoin([(toUInt64(1), (toUInt64(1), toInt32(1))), (toUInt64(2), (toUInt64(2), toInt32(2)))])) SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM single_column_bloom_filter WHERE i32 IN (SELECT arrayJoin([toInt32(1), toInt32(2)])) SETTINGS max_rows_to_read = 7; +SELECT COUNT() FROM single_column_bloom_filter WHERE (i32, i32) IN (SELECT arrayJoin([(toInt32(1), toInt32(2)), (toInt32(2), toInt32(3))])) SETTINGS max_rows_to_read = 7; +SELECT COUNT() FROM single_column_bloom_filter WHERE (i32, i64) IN (SELECT arrayJoin([(toInt32(1), toUInt64(1)), (toInt32(2), toUInt64(2))])) SETTINGS max_rows_to_read = 7; +SELECT COUNT() FROM single_column_bloom_filter WHERE (i64, (i64, i32)) IN (SELECT arrayJoin([(toUInt64(1), (toUInt64(1), toInt32(1))), (toUInt64(2), (toUInt64(2), toInt32(2)))])) SETTINGS max_rows_to_read = 7; WITH (1, 2) AS liter_prepared_set SELECT COUNT() FROM single_column_bloom_filter WHERE i32 IN liter_prepared_set SETTINGS max_rows_to_read = 6; WITH ((1, 2), (2, 3)) AS liter_prepared_set SELECT COUNT() FROM single_column_bloom_filter WHERE (i32, i32) IN liter_prepared_set SETTINGS max_rows_to_read = 6; WITH ((1, 1), (2, 2)) AS liter_prepared_set SELECT COUNT() FROM single_column_bloom_filter WHERE (i32, i64) IN liter_prepared_set SETTINGS max_rows_to_read = 6; diff --git a/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql b/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql index a653206fe18..0bc5fcd1db8 100644 --- a/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql +++ b/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql @@ -89,8 +89,11 @@ INSERT INTO checkouts SELECT number as id, '2000-01-01 10:00:00' from numbers(50 -- by this time we should have 3 parts for target_table because of prev inserts -- and we plan to make two more inserts. With index_granularity=128 and max id=1000 -- we expect to read not more than: +-- 1000 rows read from numbers(1000) in the INSERT itself +-- 1000 rows in the `IN (SELECT id FROM table)` in the mat views -- (1000/128) marks per part * (3 + 2) parts * 128 granularity = 5120 rows -set max_rows_to_read = 5120; +-- Total: 7120 +set max_rows_to_read = 7120; INSERT INTO logins SELECT number as id, '2000-01-01 11:00:00' from numbers(1000); INSERT INTO checkouts SELECT number as id, '2000-01-01 11:10:00' from numbers(1000); @@ -98,8 +101,8 @@ INSERT INTO checkouts SELECT number as id, '2000-01-01 11:10:00' from numbers(10 -- by this time we should have 5 parts for target_table because of prev inserts -- and we plan to make two more inserts. With index_granularity=128 and max id=1 -- we expect to read not more than: --- 1 mark per part * (5 + 2) parts * 128 granularity = 896 rows -set max_rows_to_read = 896; +-- 1 mark per part * (5 + 2) parts * 128 granularity + 1 (numbers(1)) = 897 rows +set max_rows_to_read = 897; INSERT INTO logins SELECT number+2 as id, '2001-01-01 11:10:01' from numbers(1); INSERT INTO checkouts SELECT number+2 as id, '2001-01-01 11:10:02' from numbers(1); diff --git a/tests/queries/0_stateless/01583_const_column_in_set_index.sql b/tests/queries/0_stateless/01583_const_column_in_set_index.sql index e40249eaf08..b781efb0f13 100644 --- a/tests/queries/0_stateless/01583_const_column_in_set_index.sql +++ b/tests/queries/0_stateless/01583_const_column_in_set_index.sql @@ -3,7 +3,7 @@ drop table if exists insub; create table insub (i int, j int) engine MergeTree order by i settings index_granularity = 1; insert into insub select number a, a + 2 from numbers(10); -SET max_rows_to_read = 2; +SET max_rows_to_read = 12; -- 10 from numbers + 2 from table select * from insub where i in (select toInt32(3) from numbers(10)); drop table if exists insub; diff --git a/tests/queries/0_stateless/01585_use_index_for_global_in.sql b/tests/queries/0_stateless/01585_use_index_for_global_in.sql index a0a5b90ac1f..1dd7609350f 100644 --- a/tests/queries/0_stateless/01585_use_index_for_global_in.sql +++ b/tests/queries/0_stateless/01585_use_index_for_global_in.sql @@ -8,10 +8,12 @@ create table xp_d as xp engine Distributed(test_shard_localhost, currentDatabase insert into xp select number, number + 2 from numbers(10); -set max_rows_to_read = 2; +set max_rows_to_read = 4; -- 2 from numbers, 2 from tables select * from xp where i in (select * from numbers(2)); select * from xp where i global in (select * from numbers(2)); select * from xp_d where i in (select * from numbers(2)); + +set max_rows_to_read = 6; -- 2 from numbers, 2 from GLOBAL temp table (pushed from numbers), 2 from local xp select * from xp_d where i global in (select * from numbers(2)); drop table if exists xp; diff --git a/tests/queries/0_stateless/01585_use_index_for_global_in_with_null.reference b/tests/queries/0_stateless/01585_use_index_for_global_in_with_null.reference index de0116f9eaa..0cb1993057f 100644 --- a/tests/queries/0_stateless/01585_use_index_for_global_in_with_null.reference +++ b/tests/queries/0_stateless/01585_use_index_for_global_in_with_null.reference @@ -14,6 +14,14 @@ 1 3 0 2 1 3 +0 2 +1 3 +0 2 +1 3 +0 2 +1 3 +0 2 +1 3 \N 100 \N 100 \N 100 diff --git a/tests/queries/0_stateless/01585_use_index_for_global_in_with_null.sql b/tests/queries/0_stateless/01585_use_index_for_global_in_with_null.sql index 6129c92c888..d4147a445ec 100644 --- a/tests/queries/0_stateless/01585_use_index_for_global_in_with_null.sql +++ b/tests/queries/0_stateless/01585_use_index_for_global_in_with_null.sql @@ -12,17 +12,29 @@ insert into xp select null, 100; optimize table xp final; set max_rows_to_read = 2; +select * from xp where i in [0, 1]; +select * from xp where i global in [0, 1]; +select * from xp_d where i in [0, 1]; +select * from xp_d where i global in [0, 1]; + +set max_rows_to_read = 4; -- 2 in the subquery, 2 in the query itself select * from xp where i in (select * from numbers(2)); select * from xp where i global in (select * from numbers(2)); select * from xp_d where i in (select * from numbers(2)); + +set max_rows_to_read = 6; -- 2 subquery, 2 from global temp table (GLOBAL IN), 2 from local xp table select * from xp_d where i global in (select * from numbers(2)); set transform_null_in = 1; +set max_rows_to_read = 4; -- 2 in the subquery, 2 in the query itself select * from xp where i in (select * from numbers(2)); select * from xp where i global in (select * from numbers(2)); select * from xp_d where i in (select * from numbers(2)); + +set max_rows_to_read = 6; -- 2 subquery, 2 from global temp table (GLOBAL IN), 2 from local xp table select * from xp_d where i global in (select * from numbers(2)); +set max_rows_to_read = 0; -- No rows should be read select * from xp where i in (null); select * from xp where i global in (null); select * from xp_d where i in (null); diff --git a/tests/queries/0_stateless/01748_partition_id_pruning.sql b/tests/queries/0_stateless/01748_partition_id_pruning.sql index 17a405e17ad..e0d45884c60 100644 --- a/tests/queries/0_stateless/01748_partition_id_pruning.sql +++ b/tests/queries/0_stateless/01748_partition_id_pruning.sql @@ -8,12 +8,12 @@ set max_rows_to_read = 3; select * from x where _partition_id = partitionId(1); -set max_rows_to_read = 4; -- one row for subquery +set max_rows_to_read = 5; -- one row for subquery + subquery select * from x where _partition_id in (select partitionId(number + 1) from numbers(1)); -- trivial count optimization test -set max_rows_to_read = 1; -- one row for subquery +set max_rows_to_read = 2; -- one row for subquery + subquery itself select count() from x where _partition_id in (select partitionId(number + 1) from numbers(1)); drop table x; From 3a7e524f9ef55f1f2a1195a46f54ff52876632e0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 9 Dec 2021 18:24:28 +0100 Subject: [PATCH 0147/1260] Style --- src/Interpreters/ThreadStatusExt.cpp | 8 -------- src/Processors/Transforms/buildPushingToViewsChain.cpp | 2 -- 2 files changed, 10 deletions(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 55e94f709e6..b3720b89eaa 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -24,14 +24,6 @@ # include #endif -namespace ProfileEvents -{ - extern const Event InsertedRows; - extern const Event InsertedBytes; - extern const Event SelectedRows; - extern const Event SelectedBytes; -} - /// Implement some methods of ThreadStatus and CurrentThread here to avoid extra linking dependencies in clickhouse_common_io /// TODO It doesn't make sense. diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index eb5e1eddc24..a70b2699f56 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -27,8 +27,6 @@ namespace ProfileEvents { - extern const Event InsertedBytes; - extern const Event InsertedRows; extern const Event SelectedBytes; extern const Event SelectedRows; } From 10ce56c8b530aca66f9542adc122acab2bd9d107 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 9 Dec 2021 19:55:38 +0100 Subject: [PATCH 0148/1260] Avoid flakyness due to print order --- .../queries/0_stateless/02132_kill_scalar_queries.reference | 2 -- tests/queries/0_stateless/02132_kill_scalar_queries.sh | 6 ++++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02132_kill_scalar_queries.reference b/tests/queries/0_stateless/02132_kill_scalar_queries.reference index 7d0d98e85f0..a598447cff5 100644 --- a/tests/queries/0_stateless/02132_kill_scalar_queries.reference +++ b/tests/queries/0_stateless/02132_kill_scalar_queries.reference @@ -1,4 +1,2 @@ -OK finished default_TEST02132KILL_QUERY1 default select (SELECT max(number) from system.numbers) + 1; -OK finished default_TEST02132KILL_QUERY2 default SELECT (SELECT number FROM system.numbers WHERE number = 1000000000000); diff --git a/tests/queries/0_stateless/02132_kill_scalar_queries.sh b/tests/queries/0_stateless/02132_kill_scalar_queries.sh index ae7348a1f0b..dd4d40d60a0 100755 --- a/tests/queries/0_stateless/02132_kill_scalar_queries.sh +++ b/tests/queries/0_stateless/02132_kill_scalar_queries.sh @@ -10,12 +10,14 @@ function wait_for_query_to_start() } QUERY_1_ID="${CLICKHOUSE_DATABASE}_TEST02132KILL_QUERY1" -(${CLICKHOUSE_CLIENT} --query_id="${QUERY_1_ID}" --query='select (SELECT max(number) from system.numbers) + 1;' 2>&1 | grep -q "Code: 394." && echo 'OK' || echo 'FAIL' ||: ) & +(${CLICKHOUSE_CLIENT} --query_id="${QUERY_1_ID}" --query='select (SELECT max(number) from system.numbers) + 1;' 2>&1 | grep -q "Code: 394." || echo 'FAIL') & wait_for_query_to_start "${QUERY_1_ID}" ${CLICKHOUSE_CLIENT} --query="KILL QUERY WHERE query_id='${QUERY_1_ID}' SYNC" QUERY_2_ID="${CLICKHOUSE_DATABASE}_TEST02132KILL_QUERY2" -(${CLICKHOUSE_CLIENT} --query_id="${QUERY_2_ID}" --query='SELECT (SELECT number FROM system.numbers WHERE number = 1000000000000);' 2>&1 | grep -q "Code: 394." && echo 'OK' || echo 'FAIL' ||: ) & +(${CLICKHOUSE_CLIENT} --query_id="${QUERY_2_ID}" --query='SELECT (SELECT number FROM system.numbers WHERE number = 1000000000000);' 2>&1 | grep -q "Code: 394." || echo 'FAIL') & wait_for_query_to_start "${QUERY_2_ID}" ${CLICKHOUSE_CLIENT} --query="KILL QUERY WHERE query_id='${QUERY_2_ID}' SYNC" +wait + From 6487201beab82d70f9706a2005e6fb74b06230f2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 9 Dec 2021 23:40:03 +0100 Subject: [PATCH 0149/1260] Adapt 01504_rocksdb --- tests/queries/0_stateless/01504_rocksdb.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01504_rocksdb.sql b/tests/queries/0_stateless/01504_rocksdb.sql index 9f9e6c3b1ac..f79f31139fe 100644 --- a/tests/queries/0_stateless/01504_rocksdb.sql +++ b/tests/queries/0_stateless/01504_rocksdb.sql @@ -34,7 +34,7 @@ INSERT INTO 01504_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 01504_test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 01504_test_memory) B USING a ORDER BY a; -CREATE TEMPORARY TABLE keys AS SELECT * FROM numbers(1000); +CREATE TEMPORARY TABLE keys AS SELECT * FROM system.numbers LIMIT 1 OFFSET 4; SET max_rows_to_read = 2; SELECT dummy == (1,1.2) FROM 01504_test WHERE k IN (1, 3) OR k IN (1) OR k IN (3, 1) OR k IN [1] OR k IN [1, 3] ; From 02b877c978f73e8077ed53b2bfef55a8a3895052 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 10 Dec 2021 10:27:53 +0800 Subject: [PATCH 0150/1260] move remote buffer from src/IO to src/Storages --- programs/server/Server.cpp | 2 +- src/Storages/Hive/StorageHive.cpp | 2 +- src/Storages/Hive/StorageHiveMetadata.h | 2 +- src/{IO => Storages}/IRemoteFileMetadata.cpp | 2 +- src/{IO => Storages}/IRemoteFileMetadata.h | 0 src/{IO => Storages}/RemoteReadBufferCache.cpp | 2 +- src/{IO => Storages}/RemoteReadBufferCache.h | 2 +- 7 files changed, 6 insertions(+), 6 deletions(-) rename src/{IO => Storages}/IRemoteFileMetadata.cpp (95%) rename src/{IO => Storages}/IRemoteFileMetadata.h (100%) rename src/{IO => Storages}/RemoteReadBufferCache.cpp (99%) rename src/{IO => Storages}/RemoteReadBufferCache.h (99%) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index e3d62351421..c0c3608d426 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -43,7 +43,6 @@ #include #include #include -#include #include #include #include @@ -58,6 +57,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 49abc8287a4..817454bb790 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -21,7 +21,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/Hive/StorageHiveMetadata.h b/src/Storages/Hive/StorageHiveMetadata.h index 76785ba8b6a..453db188503 100644 --- a/src/Storages/Hive/StorageHiveMetadata.h +++ b/src/Storages/Hive/StorageHiveMetadata.h @@ -1,5 +1,5 @@ #pragma once -#include +#include namespace DB { class StorageHiveMetadata : public IRemoteFileMetadata diff --git a/src/IO/IRemoteFileMetadata.cpp b/src/Storages/IRemoteFileMetadata.cpp similarity index 95% rename from src/IO/IRemoteFileMetadata.cpp rename to src/Storages/IRemoteFileMetadata.cpp index f9c6bac56f6..bde083d65d7 100644 --- a/src/IO/IRemoteFileMetadata.cpp +++ b/src/Storages/IRemoteFileMetadata.cpp @@ -1,4 +1,4 @@ -#include +#include #include namespace DB { diff --git a/src/IO/IRemoteFileMetadata.h b/src/Storages/IRemoteFileMetadata.h similarity index 100% rename from src/IO/IRemoteFileMetadata.h rename to src/Storages/IRemoteFileMetadata.h diff --git a/src/IO/RemoteReadBufferCache.cpp b/src/Storages/RemoteReadBufferCache.cpp similarity index 99% rename from src/IO/RemoteReadBufferCache.cpp rename to src/Storages/RemoteReadBufferCache.cpp index 9df0fa25329..99dc91eec3b 100644 --- a/src/IO/RemoteReadBufferCache.cpp +++ b/src/Storages/RemoteReadBufferCache.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include namespace fs = std::filesystem; diff --git a/src/IO/RemoteReadBufferCache.h b/src/Storages/RemoteReadBufferCache.h similarity index 99% rename from src/IO/RemoteReadBufferCache.h rename to src/Storages/RemoteReadBufferCache.h index 21126601278..126c8249be6 100644 --- a/src/IO/RemoteReadBufferCache.h +++ b/src/Storages/RemoteReadBufferCache.h @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include #include #include From f38ddb4f53fb358aa93e065968478b38188e9346 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 10 Dec 2021 10:33:08 +0100 Subject: [PATCH 0151/1260] Mark affected tests as short --- tests/performance/set_index.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/performance/set_index.xml b/tests/performance/set_index.xml index 1fb7cf967f3..2a8bb925c7d 100644 --- a/tests/performance/set_index.xml +++ b/tests/performance/set_index.xml @@ -3,9 +3,9 @@ INSERT INTO test_in SELECT number FROM numbers(500000000) - SELECT count() FROM test_in WHERE a IN (SELECT rand(1) FROM numbers(200000)) SETTINGS max_rows_to_read = 1, read_overflow_mode = 'break' + SELECT count() FROM test_in WHERE a IN (SELECT rand(1) FROM numbers(200000)) SETTINGS max_rows_to_read = 1, read_overflow_mode = 'break' - SELECT count() FROM test_in WHERE toInt64(a) IN (SELECT toInt64(rand(1)) FROM numbers(200000)) settings max_rows_to_read=1, read_overflow_mode='break' + SELECT count() FROM test_in WHERE toInt64(a) IN (SELECT toInt64(rand(1)) FROM numbers(200000)) settings max_rows_to_read=1, read_overflow_mode='break' SELECT count() FROM test_in WHERE -toInt64(a) IN (SELECT toInt64(rand(1)) FROM numbers(200000)) settings max_rows_to_read=1, read_overflow_mode='break' @@ -13,7 +13,7 @@ SELECT count() FROM test_in WHERE -toInt64(a) NOT IN (SELECT toInt64(rand(1)) FROM numbers(200000)) settings max_rows_to_read=1, read_overflow_mode='break' SELECT (rand(), rand()) IN ((17258, 93148), (4508, 52749), (68660, 70017), (77797, 23528), (1136, 37393), (53237, 15379), (68370, 73211), (15782, 54962), (59432, 45415), (68396, 920), (96154, 21016), (12700, 26887), (88016, 43191), (68153, 51575), (91315, 40005), (18070, 73178), (86, 631), (77717, 20324), (3227, 76188), (74960, 43147), (77538, 19628), (82292, 6525), (24293, 12566), (85244, 96287), (93982, 1329), (38064, 54723), (83999, 45810), (71921, 53673), (88638, 9669), (1959, 39535), (82235, 95796), (27907, 90975), (42383, 91015), (9948, 91514), (81712, 47309), (400, 25808), (31791, 46948), (39740, 36098), (25943, 84598), (99598, 52939), (77134, 15845), (40313, 72174), (85017, 94036), (36595, 14303), (83961, 68078), (55792, 72759), (73574, 43606), (9853, 63560), (28580, 56721), (74804, 41025), (32095, 55657), (52881, 63416), (91368, 90310), (23922, 38883), (30592, 10758), (66448, 61183), (31880, 96697), (11362, 20633), (75331, 2015), (71129, 8785), (1115, 70955), (7886, 83698), (18961, 84556), (16677, 43028), (37347, 70220), (31699, 71244), (10578, 96159), (67600, 39041), (78791, 86687), (21545, 54174), (68774, 37637), (46132, 81768), (98413, 20605), (2960, 23665), (31507, 35719), (96209, 18368), (60558, 38035), (21952, 3264), (11834, 86458), (21651, 17650), (86276, 36087), (18818, 24849), (61951, 3390), (59637, 62545), (30346, 72253), (36281, 2992), (78340, 49872), (94326, 93723), (3416, 94405), (12272, 8741), (22600, 22095), (57636, 37106), (38702, 14889), (70238, 11276), (17325, 60648), (16492, 41271), (52100, 1304), (93416, 7795), (57209, 71008), (48010, 36078), (20384, 74420), (77440, 34439), (69224, 45099), (30374, 33884), (49038, 90140), (1154, 84725), (64926, 86985), (91746, 73472), (59757, 75755), (45860, 71557), (45833, 36526), (74618, 73598), (91360, 65168), (58029, 30793), (56332, 14973), (99943, 96877), (97454, 6450), (64502, 77301), (73182, 31853), (76809, 83964), (82916, 86188), (78736, 65427), (36495, 7422), (76196, 2804), (96117, 61093), (9177, 26099), (52942, 63007), (48578, 47876), (50638, 89903), (7113, 97316), (35301, 12750), (47807, 7254), (38217, 55418), (56970, 41687), (20527, 62886), (358, 14021), (64018, 18582), (91740, 21683), (81967, 53589), (45437, 38450), (45476, 67752), (76851, 72072), (7304, 60091), (40097, 12897), (39906, 29247), (84262, 58734), (30857, 43791), (56087, 78929), (20498, 45954), (48726, 500), (62723, 43763), (28368, 30756), (74048, 52403), (15045, 95926), (75542, 55384), (52543, 22525), (56001, 6935), (11431, 46745), (77731, 7310), (36718, 59909), (32235, 91254), (92417, 25917), (21782, 79277), (46378, 87536), (35324, 26075), (6310, 76915), (1551, 69473), (50642, 68865), (55190, 72934), (49780, 21873), (99466, 29686), (90761, 13179), (72959, 57033), (20020, 90200), (46186, 79105), (73871, 52382), (59559, 38801), (59916, 16082), (33610, 94966), (46001, 45225), (86679, 26469), (77245, 91929), (32887, 36623), (11179, 46898), (87881, 68087), (45438, 47991), (24950, 94525), (91664, 51656), (43914, 47805), (15736, 96156), (56346, 20283), (85053, 48931), (17790, 26179), (96195, 55728), (43765, 54807), (44988, 89269), (55911, 99411), (52446, 47397), (28346, 65442), (96669, 68226), (66194, 26848), (37276, 55864), (14116, 41583), (18058, 16317), (93136, 85318), (35616, 86252), (29222, 29969), (33386, 85372), (71094, 44238), (27733, 31838), (64626, 16692), (52904, 97899), (97619, 12663), (50165, 4688), (67557, 44053), (69184, 66269), (73164, 89705), (39822, 15169), (65499, 72808), (30068, 63697), (30154, 64235), (97016, 58716), (94366, 36592), (1592, 16261), (87985, 52102), (12554, 23652), (15909, 25292), (2527, 91531), (92139, 36031), (28986, 30032), (3038, 56314), (32239, 26707), (15973, 34901), (70246, 39680), (82529, 38132), (45827, 74783), (53665, 64111), (55218, 84170), (20466, 16130), (55734, 71203), (31438, 96906), (66338, 85858), (35988, 68511), (78391, 15191), (80747, 59213), (5357, 11546), (16822, 16607), (36607, 41106), (74949, 30739), (45726, 64887), (1524, 54847), (37371, 89195), (28726, 27788), (22600, 44777), (53999, 63625), (84304, 98338), (49260, 76480), (74564, 53907), (89867, 97096), (60157, 61299), (17165, 10146), (56334, 36268), (62114, 49222), (22715, 23620), (42830, 11539), (41091, 69151), (75471, 68364), (18681, 43249), (42738, 63219), (35474, 98454), (76815, 46024), (66310, 36521), (86095, 77013), (63693, 77319), (80731, 63031), (95478, 92387), (23787, 63724), (46299, 68994), (4800, 2460), (9663, 80639), (77231, 85814), (81615, 11311), (35638, 27340), (13598, 14322), (30657, 17238), (90957, 96846), (69962, 52140), (41681, 65962), (96836, 58177), (36190, 11623), (4231, 40500), (43049, 41949), (71177, 98492), (30193, 39750), (19744, 33204), (63358, 30210), (45638, 58918), (43641, 38741), (35598, 40932), (33238, 36236), (50835, 20968), (25099, 34071), (84986, 88456), (35333, 1529), (79771, 23985), (647, 61658), (9424, 11743), (77766, 31528), (77811, 86973), (76403, 74377), (55568, 79251), (68858, 20762), (68520, 66773), (93598, 89823), (8080, 82539), (87760, 52247), (25191, 16905), (17837, 8339), (85177, 59050), (51680, 77374), (3287, 43018), (43479, 62141), (34909, 46322), (11869, 5885), (96193, 58417), (101, 47460), (34937, 88582), (83216, 88388), (28571, 15292), (66683, 62613), (34478, 8924), (2680, 89973), (62438, 44460), (11724, 4791), (5383, 72888), (88206, 67586), (8124, 21690), (28779, 75789), (66791, 4757), (6176, 47760), (6403, 78084), (78122, 35446), (99494, 73608), (39691, 89098), (59182, 19484), (25389, 98963), (96487, 3692), (76222, 67381), (21199, 50358), (95998, 58137), (28777, 43913), (14176, 60117), (52257, 81703), (14604, 13438), (71301, 14401), (19758, 66914), (15506, 29873), (87205, 29449), (93295, 15930), (63651, 11287), (19785, 15966), (30795, 75112), (69462, 37655), (18793, 85764), (36240, 31236), (98153, 73724), (72491, 4223), (66930, 35048), (25686, 13269), (13940, 13259), (69163, 11235), (1183, 86961), (54323, 67315), (85044, 60872), (48875, 3683), (43052, 92861), (87574, 32969), (92552, 80564), (94832, 47682), (72011, 80994), (60182, 917), (97788, 34169), (66432, 47940), (87468, 80954), (35385, 68758), (50555, 63710), (55311, 44337), (87065, 26514), (84581, 98736), (23212, 56499), (75120, 72447), (56087, 38285), (58171, 45629), (28401, 44319), (70432, 27883), (18891, 14646), (26206, 49924), (79957, 44914), (56064, 27529), (99090, 29197), (49435, 340), (53525, 65601), (76998, 88349), (50416, 70860), (42506, 75290), (34024, 13295), (86663, 46523), (88814, 231), (57809, 21), (84914, 84771), (43042, 66892), (17288, 33908), (4934, 63195), (50590, 1516), (97843, 80208), (20091, 86717), (71566, 15929), (19531, 23634), (41646, 45549), (89226, 82902), (96683, 63386), (31072, 53788), (51135, 41099), (78912, 65609), (36094, 23603), (88403, 51455), (73795, 47066), (26448, 82852), (22829, 2894), (30041, 92548), (27733, 20608), (70180, 19892), (51650, 63440), (76328, 13666), (40514, 6677), (2786, 51059), (40809, 16499), (10857, 82541), (78221, 61067), (17982, 51969), (85369, 66965), (47153, 47149), (43965, 75796), (82725, 60767), (42407, 97249), (51475, 81224), (60957, 89414), (33065, 21663), (36601, 5290), (95842, 67301), (64630, 60398), (55212, 35638), (41750, 44235), (75260, 82400), (91291, 25843), (6477, 8311), (14919, 52306), (66220, 33180), (45736, 2313), (37450, 64444), (98614, 61344), (75007, 50946), (56701, 28117), (66632, 5174), (92323, 76613), (6796, 73695), (33696, 76280), (86876, 5614), (50863, 67993), (36068, 17049), (91912, 34271), (70706, 1904), (97798, 41117), (68154, 72483), (83862, 25578), (61643, 17204), (69974, 64232), (77926, 19637), (64901, 88988), (71424, 91703), (91655, 17147), (46872, 56530), (44189, 98087), (95939, 54420), (72651, 68785), (67624, 84875), (92587, 87663), (65275, 81256), (53798, 2506), (14702, 3638), (71291, 50452), (14909, 13903), (66965, 26606), (14127, 60345), (35306, 1738), (77234, 10468), (53521, 41218), (80681, 82583), (44227, 26521), (32263, 21482), (82270, 56963), (50580, 80567), (11593, 22346), (20074, 26867), (73126, 28667), (62996, 24317), (20295, 57163), (1506, 57668), (69567, 45236), (43366, 26001), (88052, 40181), (1599, 89349), (36789, 1579), (39895, 46673), (30381, 3206), (31723, 5625), (19252, 31317), (16932, 77149), (48794, 34409), (55986, 30328), (47551, 75088), (57363, 78365), (95221, 63385), (26449, 5733), (96588, 53077), (52980, 41140), (8187, 85947), (36723, 26520), (23579, 38909), (33350, 19275), (63930, 19357), (43536, 59941), (31117, 77322), (44638, 94812), (44730, 99097), (95108, 48170), (57813, 49503), (79959, 89436), (86980, 62031), (8275, 44009), (36666, 94645), (22064, 38882), (40471, 16939), (31156, 11337), (13101, 96977), (17906, 26835), (89861, 51405), (73369, 67946), (99141, 58572), (27131, 98703), (15900, 43412), (51768, 93125), (78579, 46689), (23029, 13895), (60870, 55830), (22553, 8236), (76449, 96207), (83766, 51024), (27630, 50614), (53484, 90104), (77626, 21944), (46755, 41583), (53616, 34240), (94159, 44415), (13914, 90059), (44387, 89012), (27499, 64579), (83415, 30809), (77558, 82619), (88880, 9814), (8466, 4424), (43598, 91921), (24695, 3349), (46295, 65208), (51256, 82461), (49126, 93012), (16186, 96585), (43284, 22655), (93130, 90393), (77495, 34372), (85509, 65856), (86662, 61906), (50988, 44393), (29828, 17737), (91651, 35308), (29796, 49716), (14019, 87751), (29688, 71207), (82845, 19100), (11989, 50132), (21158, 99905), (54732, 42547), (32314, 12851), (46405, 43794), (87849, 45643), (53524, 21212), (61925, 75491), (12498, 21937), (30185, 69475), (48421, 52487), (15112, 90935), (33187, 17801), (61704, 25514), (17889, 23917), (18758, 57197), (7693, 47232), (47905, 24618), (11494, 78950), (95662, 54561), (8075, 33909), (90427, 46065), (73962, 19821), (50691, 79400), (58218, 4881), (94106, 2509), (60633, 55169), (49600, 83054), (23339, 13270), (70262, 58946), (48417, 97266), (27629, 46905), (74465, 75514), (41687, 2564), (12814, 19492), (78899, 30168), (17745, 35206), (37972, 35296), (22288, 80001), From 5ef68fc479df9eeae335b17163935a365e01052d Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 11 Dec 2021 15:50:59 +0800 Subject: [PATCH 0152/1260] fix building --- src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index a844054b517..2caa0e397c1 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -272,7 +272,7 @@ static ColumnPtr readByteMapFromArrowColumn(std::shared_ptr for (size_t value_i = 0; value_i != static_cast(chunk->length()); ++value_i) bytemap_data.emplace_back(chunk->IsNull(value_i)); } - return std::move(nullmap_column); + return nullmap_column; } static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptr & arrow_column) @@ -290,7 +290,7 @@ static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptr & arrow_column) From 2f7edf82abc029fa61aff3d54366010d7a87f792 Mon Sep 17 00:00:00 2001 From: gulige Date: Mon, 13 Dec 2021 13:53:12 +0800 Subject: [PATCH 0153/1260] avoid inconsistent state in Block In the member function insert(position, elem) to class Block, if function checkColumnStructure throws an exception for columns with same name but unidentical structure, it will result in the mismatch between index_by_name and data. However we could not presume upon that caller will not use this block object anymore. So it seems better to guarantee consistency innerly. --- src/Core/Block.cpp | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 168ee346626..9db3040de4d 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -139,15 +139,17 @@ void Block::insert(size_t position, ColumnWithTypeAndName elem) if (elem.name.empty()) throw Exception("Column name in Block cannot be empty", ErrorCodes::AMBIGUOUS_COLUMN_NAME); - for (auto & name_pos : index_by_name) - if (name_pos.second >= position) - ++name_pos.second; - - auto [it, inserted] = index_by_name.emplace(elem.name, position); + auto [it, inserted] = index_by_name.emplace(elem.name, (size_t)-1); if (!inserted) checkColumnStructure(data[it->second], elem, "(columns with identical name must have identical structure)", true, ErrorCodes::AMBIGUOUS_COLUMN_NAME); + for (auto & name_pos : index_by_name) + if (name_pos.second == (size_t)-1) + name_pos.second = position; + else if (name_pos.second >= position) + ++name_pos.second; + data.emplace(data.begin() + position, std::move(elem)); } From bd138f5bfa519e0333aa0a24061a025b8a90b3e8 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 13 Dec 2021 17:56:51 +0300 Subject: [PATCH 0154/1260] [docs/faq] How do I contribute code to ClickHouse? --- .../how-do-i-contribute-code-to-clickhouse.md | 15 +++++++++++++++ 1 file changed, 15 insertions(+) create mode 100644 docs/en/faq/general/how-do-i-contribute-code-to-clickhouse.md diff --git a/docs/en/faq/general/how-do-i-contribute-code-to-clickhouse.md b/docs/en/faq/general/how-do-i-contribute-code-to-clickhouse.md new file mode 100644 index 00000000000..731dc9dface --- /dev/null +++ b/docs/en/faq/general/how-do-i-contribute-code-to-clickhouse.md @@ -0,0 +1,15 @@ +--- +title: How do I contribute code to ClickHouse? +toc_hidden: true +toc_priority: 120 +--- + +# How do I contribute code to ClickHouse? {#how-do-i-contribute-code-to-clickhouse} + +ClickHouse is an open-source project [developed on GitHub](https://github.com/ClickHouse/ClickHouse). + +As customary, contribution instructions are published in [CONTRIBUTING.md](https://github.com/ClickHouse/ClickHouse/blob/master/CONTRIBUTING.md) file in the root of the source code repository. + +If you want to suggest a substantial change to ClickHouse, consider [opening a GitHub issue](https://github.com/ClickHouse/ClickHouse/issues/new/choose) explaining what you want to do, to discuss it with maintainers and community first. [Examples of such RFC issues](https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aissue+is%3Aopen+rfc). + +If your contributions are security related, please check out [our security policy](https://github.com/ClickHouse/ClickHouse/security/policy/) too. From 5c83be9221727e6958f1b581d69358289e2930ec Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 13 Dec 2021 17:58:22 +0300 Subject: [PATCH 0155/1260] Update index.md --- docs/en/faq/general/index.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/faq/general/index.md b/docs/en/faq/general/index.md index cd2368be1cf..51fff9a53ae 100644 --- a/docs/en/faq/general/index.md +++ b/docs/en/faq/general/index.md @@ -17,6 +17,7 @@ Questions: - [What is OLAP?](../../faq/general/olap.md) - [What is a columnar database?](../../faq/general/columnar-database.md) - [Why not use something like MapReduce?](../../faq/general/mapreduce.md) +- [How do I contribute code to ClickHouse?](../../faq/general/how-do-i-contribute-code-to-clickhouse.md) !!! info "Don’t see what you were looking for?" Check out [other F.A.Q. categories](../../faq/index.md) or browse around main documentation articles found in the left sidebar. From d9e454423991048a2de48adf25a3870a07b0ca71 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 30 Nov 2021 17:39:29 +0100 Subject: [PATCH 0156/1260] Adapt to arrow 6 --- contrib/arrow-cmake/CMakeLists.txt | 212 ++++++++++-------- .../Formats/Impl/ORCBlockInputFormat.cpp | 33 ++- 2 files changed, 131 insertions(+), 114 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 231185462dc..9c13a5dd133 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -1,5 +1,22 @@ set (CMAKE_CXX_STANDARD 17) +set(ARROW_VERSION "6.0.1") +string(REGEX MATCH "^[0-9]+\\.[0-9]+\\.[0-9]+" ARROW_BASE_VERSION "${ARROW_VERSION}") + +set(ARROW_VERSION_MAJOR "6") +set(ARROW_VERSION_MINOR "0") +set(ARROW_VERSION_PATCH "1") + +if(ARROW_VERSION_MAJOR STREQUAL "0") + # Arrow 0.x.y => SO version is "x", full SO version is "x.y.0" + set(ARROW_SO_VERSION "${ARROW_VERSION_MINOR}") + set(ARROW_FULL_SO_VERSION "${ARROW_SO_VERSION}.${ARROW_VERSION_PATCH}.0") +else() + # Arrow 1.x.y => SO version is "10x", full SO version is "10x.y.0" + math(EXPR ARROW_SO_VERSION "${ARROW_VERSION_MAJOR} * 100 + ${ARROW_VERSION_MINOR}") + set(ARROW_FULL_SO_VERSION "${ARROW_SO_VERSION}.${ARROW_VERSION_PATCH}.0") +endif() + # === thrift set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/thrift/lib/cpp") @@ -93,6 +110,9 @@ add_subdirectory(${FLATBUFFERS_SRC_DIR} "${FLATBUFFERS_BINARY_DIR}") message(STATUS "FLATBUFFERS_LIBRARY: ${FLATBUFFERS_LIBRARY}") +# === hdfs +set(HDFS_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libhdfs3/include/hdfs/") + # 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. @@ -114,6 +134,7 @@ configure_file("${ORC_INCLUDE_DIR}/orc/orc-config.hh.in" "${ORC_BUILD_INCLUDE_DI configure_file("${ORC_SOURCE_SRC_DIR}/Adaptor.hh.in" "${ORC_BUILD_INCLUDE_DIR}/Adaptor.hh") +# ARROW_ORC + adapters/orc/CMakefiles set(ORC_SRCS "${ARROW_SRC_DIR}/arrow/adapters/orc/adapter.cc" "${ARROW_SRC_DIR}/arrow/adapters/orc/adapter_util.cc" @@ -150,28 +171,8 @@ set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src/arrow") configure_file("${LIBRARY_DIR}/util/config.h.cmake" "${CMAKE_CURRENT_BINARY_DIR}/cpp/src/arrow/util/config.h") -# arrow/cpp/src/arrow/CMakeLists.txt +# arrow/cpp/src/arrow/CMakeLists.txt (ARROW_SRCS + ARROW_COMPUTE) set(ARROW_SRCS - "${LIBRARY_DIR}/buffer.cc" - "${LIBRARY_DIR}/builder.cc" - "${LIBRARY_DIR}/chunked_array.cc" - "${LIBRARY_DIR}/compare.cc" - "${LIBRARY_DIR}/datum.cc" - "${LIBRARY_DIR}/device.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/array_base.cc" "${LIBRARY_DIR}/array/array_binary.cc" "${LIBRARY_DIR}/array/array_decimal.cc" @@ -191,25 +192,94 @@ set(ARROW_SRCS "${LIBRARY_DIR}/array/diff.cc" "${LIBRARY_DIR}/array/util.cc" "${LIBRARY_DIR}/array/validate.cc" + "${LIBRARY_DIR}/builder.cc" + "${LIBRARY_DIR}/buffer.cc" + "${LIBRARY_DIR}/chunked_array.cc" + "${LIBRARY_DIR}/compare.cc" + "${LIBRARY_DIR}/config.cc" + "${LIBRARY_DIR}/datum.cc" + "${LIBRARY_DIR}/device.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.cc" + "${LIBRARY_DIR}/table_builder.cc" + "${LIBRARY_DIR}/tensor.cc" + "${LIBRARY_DIR}/tensor/coo_converter.cc" + "${LIBRARY_DIR}/tensor/csf_converter.cc" + "${LIBRARY_DIR}/tensor/csx_converter.cc" + "${LIBRARY_DIR}/type.cc" + "${LIBRARY_DIR}/visitor.cc" + "${LIBRARY_DIR}/c/bridge.cc" + "${LIBRARY_DIR}/io/buffered.cc" + "${LIBRARY_DIR}/io/caching.cc" + "${LIBRARY_DIR}/io/compressed.cc" + "${LIBRARY_DIR}/io/file.cc" + "${LIBRARY_DIR}/io/hdfs.cc" + "${LIBRARY_DIR}/io/hdfs_internal.cc" + "${LIBRARY_DIR}/io/interfaces.cc" + "${LIBRARY_DIR}/io/memory.cc" + "${LIBRARY_DIR}/io/slow.cc" + "${LIBRARY_DIR}/io/stdio.cc" + "${LIBRARY_DIR}/io/transform.cc" + "${LIBRARY_DIR}/util/async_util.cc" + "${LIBRARY_DIR}/util/basic_decimal.cc" + "${LIBRARY_DIR}/util/bit_block_counter.cc" + "${LIBRARY_DIR}/util/bit_run_reader.cc" + "${LIBRARY_DIR}/util/bit_util.cc" + "${LIBRARY_DIR}/util/bitmap.cc" + "${LIBRARY_DIR}/util/bitmap_builders.cc" + "${LIBRARY_DIR}/util/bitmap_ops.cc" + "${LIBRARY_DIR}/util/bpacking.cc" + "${LIBRARY_DIR}/util/cancel.cc" + "${LIBRARY_DIR}/util/compression.cc" + "${LIBRARY_DIR}/util/counting_semaphore.cc" + "${LIBRARY_DIR}/util/cpu_info.cc" + "${LIBRARY_DIR}/util/decimal.cc" + "${LIBRARY_DIR}/util/delimiting.cc" + "${LIBRARY_DIR}/util/formatting.cc" + "${LIBRARY_DIR}/util/future.cc" + "${LIBRARY_DIR}/util/int_util.cc" + "${LIBRARY_DIR}/util/io_util.cc" + "${LIBRARY_DIR}/util/logging.cc" + "${LIBRARY_DIR}/util/key_value_metadata.cc" + "${LIBRARY_DIR}/util/memory.cc" + "${LIBRARY_DIR}/util/mutex.cc" + "${LIBRARY_DIR}/util/string.cc" + "${LIBRARY_DIR}/util/string_builder.cc" + "${LIBRARY_DIR}/util/task_group.cc" + "${LIBRARY_DIR}/util/tdigest.cc" + "${LIBRARY_DIR}/util/thread_pool.cc" + "${LIBRARY_DIR}/util/time.cc" + "${LIBRARY_DIR}/util/trie.cc" + "${LIBRARY_DIR}/util/unreachable.cc" + "${LIBRARY_DIR}/util/uri.cc" + "${LIBRARY_DIR}/util/utf8.cc" + "${LIBRARY_DIR}/util/value_parsing.cc" + "${LIBRARY_DIR}/vendored/base64.cpp" "${LIBRARY_DIR}/compute/api_aggregate.cc" "${LIBRARY_DIR}/compute/api_scalar.cc" "${LIBRARY_DIR}/compute/api_vector.cc" "${LIBRARY_DIR}/compute/cast.cc" "${LIBRARY_DIR}/compute/exec.cc" + "${LIBRARY_DIR}/compute/exec/aggregate_node.cc" + "${LIBRARY_DIR}/compute/exec/exec_plan.cc" + "${LIBRARY_DIR}/compute/exec/expression.cc" + "${LIBRARY_DIR}/compute/exec/filter_node.cc" + "${LIBRARY_DIR}/compute/exec/project_node.cc" + "${LIBRARY_DIR}/compute/exec/source_node.cc" + "${LIBRARY_DIR}/compute/exec/sink_node.cc" + "${LIBRARY_DIR}/compute/exec/order_by_impl.cc" "${LIBRARY_DIR}/compute/function.cc" "${LIBRARY_DIR}/compute/function_internal.cc" "${LIBRARY_DIR}/compute/kernel.cc" "${LIBRARY_DIR}/compute/registry.cc" - - "${LIBRARY_DIR}/compute/exec/exec_plan.cc" - "${LIBRARY_DIR}/compute/exec/expression.cc" - "${LIBRARY_DIR}/compute/exec/key_compare.cc" - "${LIBRARY_DIR}/compute/exec/key_encode.cc" - "${LIBRARY_DIR}/compute/exec/key_hash.cc" - "${LIBRARY_DIR}/compute/exec/key_map.cc" - "${LIBRARY_DIR}/compute/exec/util.cc" - "${LIBRARY_DIR}/compute/kernels/aggregate_basic.cc" "${LIBRARY_DIR}/compute/kernels/aggregate_mode.cc" "${LIBRARY_DIR}/compute/kernels/aggregate_quantile.cc" @@ -227,83 +297,32 @@ set(ARROW_SRCS "${LIBRARY_DIR}/compute/kernels/scalar_cast_string.cc" "${LIBRARY_DIR}/compute/kernels/scalar_cast_temporal.cc" "${LIBRARY_DIR}/compute/kernels/scalar_compare.cc" - "${LIBRARY_DIR}/compute/kernels/scalar_fill_null.cc" - "${LIBRARY_DIR}/compute/kernels/scalar_if_else.cc" "${LIBRARY_DIR}/compute/kernels/scalar_nested.cc" "${LIBRARY_DIR}/compute/kernels/scalar_set_lookup.cc" "${LIBRARY_DIR}/compute/kernels/scalar_string.cc" - "${LIBRARY_DIR}/compute/kernels/scalar_temporal.cc" + "${LIBRARY_DIR}/compute/kernels/scalar_temporal_binary.cc" + "${LIBRARY_DIR}/compute/kernels/scalar_temporal_unary.cc" "${LIBRARY_DIR}/compute/kernels/scalar_validity.cc" + "${LIBRARY_DIR}/compute/kernels/scalar_if_else.cc" "${LIBRARY_DIR}/compute/kernels/util_internal.cc" + "${LIBRARY_DIR}/compute/kernels/vector_array_sort.cc" "${LIBRARY_DIR}/compute/kernels/vector_hash.cc" "${LIBRARY_DIR}/compute/kernels/vector_nested.cc" "${LIBRARY_DIR}/compute/kernels/vector_replace.cc" "${LIBRARY_DIR}/compute/kernels/vector_selection.cc" "${LIBRARY_DIR}/compute/kernels/vector_sort.cc" + "${LIBRARY_DIR}/compute/kernels/row_encoder.cc" + "${LIBRARY_DIR}/compute/exec/union_node.cc" + "${LIBRARY_DIR}/compute/exec/key_hash.cc" + "${LIBRARY_DIR}/compute/exec/key_map.cc" + "${LIBRARY_DIR}/compute/exec/key_compare.cc" + "${LIBRARY_DIR}/compute/exec/key_encode.cc" + "${LIBRARY_DIR}/compute/exec/util.cc" + "${LIBRARY_DIR}/compute/exec/hash_join_dict.cc" + "${LIBRARY_DIR}/compute/exec/hash_join.cc" + "${LIBRARY_DIR}/compute/exec/hash_join_node.cc" + "${LIBRARY_DIR}/compute/exec/task_util.cc" - "${LIBRARY_DIR}/csv/chunker.cc" - "${LIBRARY_DIR}/csv/column_builder.cc" - "${LIBRARY_DIR}/csv/column_decoder.cc" - "${LIBRARY_DIR}/csv/converter.cc" - "${LIBRARY_DIR}/csv/options.cc" - "${LIBRARY_DIR}/csv/parser.cc" - "${LIBRARY_DIR}/csv/reader.cc" - "${LIBRARY_DIR}/csv/writer.cc" - - "${LIBRARY_DIR}/ipc/dictionary.cc" - "${LIBRARY_DIR}/ipc/feather.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/writer.cc" - - "${LIBRARY_DIR}/io/buffered.cc" - "${LIBRARY_DIR}/io/caching.cc" - "${LIBRARY_DIR}/io/compressed.cc" - "${LIBRARY_DIR}/io/file.cc" - "${LIBRARY_DIR}/io/interfaces.cc" - "${LIBRARY_DIR}/io/memory.cc" - "${LIBRARY_DIR}/io/slow.cc" - "${LIBRARY_DIR}/io/stdio.cc" - "${LIBRARY_DIR}/io/transform.cc" - - "${LIBRARY_DIR}/tensor/coo_converter.cc" - "${LIBRARY_DIR}/tensor/csf_converter.cc" - "${LIBRARY_DIR}/tensor/csx_converter.cc" - - "${LIBRARY_DIR}/util/basic_decimal.cc" - "${LIBRARY_DIR}/util/bit_block_counter.cc" - "${LIBRARY_DIR}/util/bit_run_reader.cc" - "${LIBRARY_DIR}/util/bit_util.cc" - "${LIBRARY_DIR}/util/bitmap_builders.cc" - "${LIBRARY_DIR}/util/bitmap_ops.cc" - "${LIBRARY_DIR}/util/bitmap.cc" - "${LIBRARY_DIR}/util/bpacking.cc" - "${LIBRARY_DIR}/util/cancel.cc" - "${LIBRARY_DIR}/util/compression.cc" - "${LIBRARY_DIR}/util/cpu_info.cc" - "${LIBRARY_DIR}/util/decimal.cc" - "${LIBRARY_DIR}/util/delimiting.cc" - "${LIBRARY_DIR}/util/formatting.cc" - "${LIBRARY_DIR}/util/future.cc" - "${LIBRARY_DIR}/util/int_util.cc" - "${LIBRARY_DIR}/util/io_util.cc" - "${LIBRARY_DIR}/util/key_value_metadata.cc" - "${LIBRARY_DIR}/util/logging.cc" - "${LIBRARY_DIR}/util/memory.cc" - "${LIBRARY_DIR}/util/mutex.cc" - "${LIBRARY_DIR}/util/string_builder.cc" - "${LIBRARY_DIR}/util/string.cc" - "${LIBRARY_DIR}/util/task_group.cc" - "${LIBRARY_DIR}/util/tdigest.cc" - "${LIBRARY_DIR}/util/thread_pool.cc" - "${LIBRARY_DIR}/util/time.cc" - "${LIBRARY_DIR}/util/trie.cc" - "${LIBRARY_DIR}/util/utf8.cc" - "${LIBRARY_DIR}/util/value_parsing.cc" - - "${LIBRARY_DIR}/vendored/base64.cpp" ${ORC_SRCS} ) @@ -373,6 +392,7 @@ target_include_directories(${ARROW_LIBRARY} PRIVATE SYSTEM ${ORC_BUILD_INCLUDE_D target_include_directories(${ARROW_LIBRARY} PRIVATE SYSTEM ${ORC_ADDITION_SOURCE_DIR}) target_include_directories(${ARROW_LIBRARY} PRIVATE SYSTEM ${ARROW_SRC_DIR}) target_include_directories(${ARROW_LIBRARY} PRIVATE SYSTEM ${FLATBUFFERS_INCLUDE_DIR}) +target_include_directories(${ARROW_LIBRARY} PRIVATE SYSTEM ${HDFS_INCLUDE_DIR}) # === parquet diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 8768e2f5f14..9fa360c6592 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -19,13 +19,6 @@ namespace ErrorCodes extern const int CANNOT_READ_ALL_DATA; } -#define THROW_ARROW_NOT_OK(status) \ - do \ - { \ - if (::arrow::Status _s = (status); !_s.ok()) \ - throw Exception(_s.ToString(), ErrorCodes::BAD_ARGUMENTS); \ - } while (false) - ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_) : IInputFormat(std::move(header_), in_), format_settings(format_settings_) { @@ -40,14 +33,12 @@ Chunk ORCBlockInputFormat::generate() if (!batch_reader) { - arrow::Status reader_status = file_reader->NextStripeReader( - DBMS_DEFAULT_BUFFER_SIZE, include_indices, &batch_reader); - if (!reader_status.ok()) - throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, - "Failed to create batch reader: {}", - reader_status.ToString()); - if (!batch_reader) - return res; + auto result = file_reader->NextStripeReader(DBMS_DEFAULT_BUFFER_SIZE, include_indices); + if (!result.ok()) + { + throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Failed to create batch reader: {}", result.status().ToString()); + } + batch_reader.reset(result->get()); } std::shared_ptr batch_result; @@ -107,12 +98,18 @@ static size_t countIndicesForType(std::shared_ptr type) void ORCBlockInputFormat::prepareReader() { - THROW_ARROW_NOT_OK(arrow::adapters::orc::ORCFileReader::Open(asArrowFile(*in, format_settings), arrow::default_memory_pool(), &file_reader)); + auto result = arrow::adapters::orc::ORCFileReader::Open(asArrowFile(*in, format_settings), arrow::default_memory_pool()); + if (!result.ok()) + throw Exception(result.status().ToString(), ErrorCodes::BAD_ARGUMENTS); + file_reader.reset(result->get()); stripe_total = file_reader->NumberOfStripes(); stripe_current = 0; - std::shared_ptr schema; - THROW_ARROW_NOT_OK(file_reader->ReadSchema(&schema)); + auto read_schema_result = file_reader->ReadSchema(); + if (!read_schema_result.ok()) + throw Exception(read_schema_result.status().ToString(), ErrorCodes::BAD_ARGUMENTS); + + std::shared_ptr schema = *read_schema_result; arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), "ORC", format_settings.orc.import_nested); From 61d959df8fb453a7f45df4a58f6958c4103d9a38 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 1 Dec 2021 12:19:24 +0100 Subject: [PATCH 0157/1260] Fix arrow build --- contrib/arrow-cmake/CMakeLists.txt | 11 ++++++++++- src/Processors/Formats/Impl/ORCBlockInputFormat.cpp | 11 +++++------ 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 9c13a5dd133..627e905385d 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -171,7 +171,7 @@ set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src/arrow") configure_file("${LIBRARY_DIR}/util/config.h.cmake" "${CMAKE_CURRENT_BINARY_DIR}/cpp/src/arrow/util/config.h") -# arrow/cpp/src/arrow/CMakeLists.txt (ARROW_SRCS + ARROW_COMPUTE) +# arrow/cpp/src/arrow/CMakeLists.txt (ARROW_SRCS + ARROW_COMPUTE + ARROW_IPC) set(ARROW_SRCS "${LIBRARY_DIR}/array/array_base.cc" "${LIBRARY_DIR}/array/array_binary.cc" @@ -262,6 +262,7 @@ set(ARROW_SRCS "${LIBRARY_DIR}/util/utf8.cc" "${LIBRARY_DIR}/util/value_parsing.cc" "${LIBRARY_DIR}/vendored/base64.cpp" + "${LIBRARY_DIR}/vendored/datetime/tz.cpp" "${LIBRARY_DIR}/compute/api_aggregate.cc" "${LIBRARY_DIR}/compute/api_scalar.cc" @@ -323,6 +324,14 @@ set(ARROW_SRCS "${LIBRARY_DIR}/compute/exec/hash_join_node.cc" "${LIBRARY_DIR}/compute/exec/task_util.cc" + "${LIBRARY_DIR}/ipc/dictionary.cc" + "${LIBRARY_DIR}/ipc/feather.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/writer.cc" + ${ORC_SRCS} ) diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 9fa360c6592..690e2b08a76 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -35,10 +35,10 @@ Chunk ORCBlockInputFormat::generate() { auto result = file_reader->NextStripeReader(DBMS_DEFAULT_BUFFER_SIZE, include_indices); if (!result.ok()) - { throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Failed to create batch reader: {}", result.status().ToString()); - } - batch_reader.reset(result->get()); + batch_reader = std::move(result).ValueOrDie(); + if (!batch_reader) + return res; } std::shared_ptr batch_result; @@ -101,15 +101,14 @@ void ORCBlockInputFormat::prepareReader() auto result = arrow::adapters::orc::ORCFileReader::Open(asArrowFile(*in, format_settings), arrow::default_memory_pool()); if (!result.ok()) throw Exception(result.status().ToString(), ErrorCodes::BAD_ARGUMENTS); - file_reader.reset(result->get()); + file_reader = std::move(result).ValueOrDie(); stripe_total = file_reader->NumberOfStripes(); stripe_current = 0; auto read_schema_result = file_reader->ReadSchema(); if (!read_schema_result.ok()) throw Exception(read_schema_result.status().ToString(), ErrorCodes::BAD_ARGUMENTS); - - std::shared_ptr schema = *read_schema_result; + std::shared_ptr schema = std::move(read_schema_result).ValueOrDie(); arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), "ORC", format_settings.orc.import_nested); From 1a0e0dba3feb93a76e1ccd44a8f07c01012d9f02 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 1 Dec 2021 14:13:37 +0100 Subject: [PATCH 0158/1260] Adapt parquet reference --- tests/queries/0_stateless/00900_long_parquet_load.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00900_long_parquet_load.reference b/tests/queries/0_stateless/00900_long_parquet_load.reference index 421def88e41..89da3c6fa43 100644 --- a/tests/queries/0_stateless/00900_long_parquet_load.reference +++ b/tests/queries/0_stateless/00900_long_parquet_load.reference @@ -89,7 +89,7 @@ idx10 ['This','is','a','test'] 23 24 === Try load data from datapage_v2.snappy.parquet -Code: 33. DB::ParsingEx---tion: Error while reading Parquet data: IOError: Not yet implemented: Unsupported encoding.: While executing ParquetBlockInputFormat: data for INSERT was parsed from stdin: (in query: INSERT INTO parquet_load FORMAT Parquet). (CANNOT_READ_ALL_DATA) +Code: 33. DB::ParsingEx---tion: Error while reading Parquet data: IOError: Unknown encoding type.: While executing ParquetBlockInputFormat: data for INSERT was parsed from stdin: (in query: INSERT INTO parquet_load FORMAT Parquet). (CANNOT_READ_ALL_DATA) === Try load data from datatype-date32.parquet 1925-01-01 From cc6f7a71e530a440b04bb2dae2363935dc112ae3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 1 Dec 2021 14:14:28 +0100 Subject: [PATCH 0159/1260] Fix String reinterpret https://github.com/ClickHouse/ClickHouse/issues/32018#issuecomment-983595475 --- src/Core/Field.h | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/src/Core/Field.h b/src/Core/Field.h index a9fb73393cf..0cafdbe5361 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -808,11 +808,29 @@ auto & Field::safeGet() template T & Field::reinterpret() { + assert(which != Types::String); // See specialization for char using ValueType = std::decay_t; ValueType * MAY_ALIAS ptr = reinterpret_cast(&storage); return *ptr; } +// Specialize reinterpreting to char (used in ColumnUnique) to make sure Strings are reinterpreted correctly +// inline to avoid multiple definitions +template <> +inline char & Field::reinterpret() +{ + using ValueType = std::decay_t; + if (which == Types::String) + { + // For String we want to return a pointer to the data, not the start of the class + // as the layout of std::string depends on the STD version and options + ValueType * MAY_ALIAS ptr = reinterpret_cast(reinterpret_cast(&storage)->data()); + return *ptr; + } + ValueType * MAY_ALIAS ptr = reinterpret_cast(&storage); + return *ptr; +} + template T get(const Field & field) { From 1c26033383a3542a363daa45218f86214b9f0c7f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 1 Dec 2021 16:49:42 +0100 Subject: [PATCH 0160/1260] Update contrib/s2geometry-cmake/CMakeLists.txt --- contrib/s2geometry-cmake/CMakeLists.txt | 71 ++++++++++++++++++------- 1 file changed, 51 insertions(+), 20 deletions(-) diff --git a/contrib/s2geometry-cmake/CMakeLists.txt b/contrib/s2geometry-cmake/CMakeLists.txt index 41d570c9afd..e5ef3755ab7 100644 --- a/contrib/s2geometry-cmake/CMakeLists.txt +++ b/contrib/s2geometry-cmake/CMakeLists.txt @@ -1,8 +1,12 @@ set(S2_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/s2geometry/src") +set(ABSL_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/abseil-cpp") +if(NOT EXISTS "${ABSL_SOURCE_DIR}/CMakeLists.txt") + message(FATAL_ERROR " submodule contrib/abseil-cpp is missing. To fix try run: \n git submodule update --init --recursive") +endif() + + set(S2_SRCS - "${S2_SOURCE_DIR}/s2/base/stringprintf.cc" - "${S2_SOURCE_DIR}/s2/base/strtoint.cc" "${S2_SOURCE_DIR}/s2/encoded_s2cell_id_vector.cc" "${S2_SOURCE_DIR}/s2/encoded_s2point_vector.cc" "${S2_SOURCE_DIR}/s2/encoded_s2shape_index.cc" @@ -14,11 +18,14 @@ set(S2_SRCS "${S2_SOURCE_DIR}/s2/s1chord_angle.cc" "${S2_SOURCE_DIR}/s2/s1interval.cc" "${S2_SOURCE_DIR}/s2/s2boolean_operation.cc" + "${S2_SOURCE_DIR}/s2/s2buffer_operation.cc" "${S2_SOURCE_DIR}/s2/s2builder.cc" "${S2_SOURCE_DIR}/s2/s2builder_graph.cc" "${S2_SOURCE_DIR}/s2/s2builderutil_closed_set_normalizer.cc" "${S2_SOURCE_DIR}/s2/s2builderutil_find_polygon_degeneracies.cc" + "${S2_SOURCE_DIR}/s2/s2builderutil_get_snapped_winding_delta.cc" "${S2_SOURCE_DIR}/s2/s2builderutil_lax_polygon_layer.cc" + "${S2_SOURCE_DIR}/s2/s2builderutil_lax_polyline_layer.cc" "${S2_SOURCE_DIR}/s2/s2builderutil_s2point_vector_layer.cc" "${S2_SOURCE_DIR}/s2/s2builderutil_s2polygon_layer.cc" "${S2_SOURCE_DIR}/s2/s2builderutil_s2polyline_layer.cc" @@ -44,7 +51,6 @@ set(S2_SRCS "${S2_SOURCE_DIR}/s2/s2edge_crossings.cc" "${S2_SOURCE_DIR}/s2/s2edge_distances.cc" "${S2_SOURCE_DIR}/s2/s2edge_tessellator.cc" - "${S2_SOURCE_DIR}/s2/s2error.cc" "${S2_SOURCE_DIR}/s2/s2furthest_edge_query.cc" "${S2_SOURCE_DIR}/s2/s2latlng.cc" "${S2_SOURCE_DIR}/s2/s2latlng_rect.cc" @@ -55,6 +61,7 @@ set(S2_SRCS "${S2_SOURCE_DIR}/s2/s2loop.cc" "${S2_SOURCE_DIR}/s2/s2loop_measures.cc" "${S2_SOURCE_DIR}/s2/s2measures.cc" + "${S2_SOURCE_DIR}/s2/s2memory_tracker.cc" "${S2_SOURCE_DIR}/s2/s2metrics.cc" "${S2_SOURCE_DIR}/s2/s2max_distance_targets.cc" "${S2_SOURCE_DIR}/s2/s2min_distance_targets.cc" @@ -82,28 +89,15 @@ set(S2_SRCS "${S2_SOURCE_DIR}/s2/s2shapeutil_build_polygon_boundaries.cc" "${S2_SOURCE_DIR}/s2/s2shapeutil_coding.cc" "${S2_SOURCE_DIR}/s2/s2shapeutil_contains_brute_force.cc" + "${S2_SOURCE_DIR}/s2/s2shapeutil_conversion.cc" "${S2_SOURCE_DIR}/s2/s2shapeutil_edge_iterator.cc" "${S2_SOURCE_DIR}/s2/s2shapeutil_get_reference_point.cc" "${S2_SOURCE_DIR}/s2/s2shapeutil_range_iterator.cc" "${S2_SOURCE_DIR}/s2/s2shapeutil_visit_crossing_edge_pairs.cc" "${S2_SOURCE_DIR}/s2/s2text_format.cc" "${S2_SOURCE_DIR}/s2/s2wedge_relations.cc" - "${S2_SOURCE_DIR}/s2/strings/ostringstream.cc" + "${S2_SOURCE_DIR}/s2/s2winding_operation.cc" "${S2_SOURCE_DIR}/s2/strings/serialize.cc" - # ClickHouse doesn't use strings from abseil. - # So, there is no duplicate symbols. - "${S2_SOURCE_DIR}/s2/third_party/absl/base/dynamic_annotations.cc" - "${S2_SOURCE_DIR}/s2/third_party/absl/base/internal/raw_logging.cc" - "${S2_SOURCE_DIR}/s2/third_party/absl/base/internal/throw_delegate.cc" - "${S2_SOURCE_DIR}/s2/third_party/absl/numeric/int128.cc" - "${S2_SOURCE_DIR}/s2/third_party/absl/strings/ascii.cc" - "${S2_SOURCE_DIR}/s2/third_party/absl/strings/match.cc" - "${S2_SOURCE_DIR}/s2/third_party/absl/strings/numbers.cc" - "${S2_SOURCE_DIR}/s2/third_party/absl/strings/str_cat.cc" - "${S2_SOURCE_DIR}/s2/third_party/absl/strings/str_split.cc" - "${S2_SOURCE_DIR}/s2/third_party/absl/strings/string_view.cc" - "${S2_SOURCE_DIR}/s2/third_party/absl/strings/strip.cc" - "${S2_SOURCE_DIR}/s2/third_party/absl/strings/internal/memutil.cc" "${S2_SOURCE_DIR}/s2/util/bits/bit-interleave.cc" "${S2_SOURCE_DIR}/s2/util/bits/bits.cc" "${S2_SOURCE_DIR}/s2/util/coding/coder.cc" @@ -111,17 +105,54 @@ set(S2_SRCS "${S2_SOURCE_DIR}/s2/util/math/exactfloat/exactfloat.cc" "${S2_SOURCE_DIR}/s2/util/math/mathutil.cc" "${S2_SOURCE_DIR}/s2/util/units/length-units.cc" + + + # ClickHouse doesn't use strings from abseil. + # So, there is no duplicate symbols. +# "${ABSL_SOURCE_DIR}/absl/base/internal/raw_logging.cc" +# "${ABSL_SOURCE_DIR}/absl/base/internal/throw_delegate.cc" +# "${ABSL_SOURCE_DIR}/absl/numeric/int128.cc" +# "${ABSL_SOURCE_DIR}/absl/strings/ascii.cc" +# "${ABSL_SOURCE_DIR}/absl/strings/match.cc" +# "${ABSL_SOURCE_DIR}/absl/strings/numbers.cc" +# "${ABSL_SOURCE_DIR}/absl/strings/str_cat.cc" +# "${ABSL_SOURCE_DIR}/absl/strings/str_split.cc" +# "${ABSL_SOURCE_DIR}/absl/strings/string_view.cc" +# "${ABSL_SOURCE_DIR}/absl/strings/strip.cc" +# "${ABSL_SOURCE_DIR}/absl/strings/internal/memutil.cc" ) add_library(s2 ${S2_SRCS}) - -set_property(TARGET s2 PROPERTY CXX_STANDARD 11) +set_property(TARGET s2 PROPERTY CXX_STANDARD 17) if (OPENSSL_FOUND) target_link_libraries(s2 PRIVATE ${OPENSSL_LIBRARIES}) endif() +target_link_libraries(s2 PRIVATE + absl::base + absl::btree + absl::config + absl::core_headers + absl::dynamic_annotations + absl::endian + absl::fixed_array + absl::flat_hash_map + absl::flat_hash_set + absl::hash + absl::inlined_vector + absl::int128 + absl::log_severity + absl::memory + absl::span + absl::str_format + absl::strings + absl::type_traits + absl::utility + ) + target_include_directories(s2 SYSTEM BEFORE PUBLIC "${S2_SOURCE_DIR}/") +target_include_directories(s2 SYSTEM PUBLIC "${ABSL_SOURCE_DIR}") if(M_LIBRARY) target_link_libraries(s2 PRIVATE ${M_LIBRARY}) From fe9a2c7e04b758ba6df98bb5c1d50b0ebab64506 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 1 Dec 2021 16:59:09 +0100 Subject: [PATCH 0161/1260] Abseil: Use upstream lts_2021_11_02 branch --- .gitmodules | 4 ++-- contrib/abseil-cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.gitmodules b/.gitmodules index a9178493cdf..9b5875cf81d 100644 --- a/.gitmodules +++ b/.gitmodules @@ -190,8 +190,8 @@ url = https://github.com/xz-mirror/xz [submodule "contrib/abseil-cpp"] path = contrib/abseil-cpp - url = https://github.com/ClickHouse-Extras/abseil-cpp.git - branch = lts_2020_02_25 + url = https://github.com/abseil/abseil-cpp.git + branch = lts_2021_11_02 [submodule "contrib/dragonbox"] path = contrib/dragonbox url = https://github.com/ClickHouse-Extras/dragonbox.git diff --git a/contrib/abseil-cpp b/contrib/abseil-cpp index b004a8a0241..215105818df 160000 --- a/contrib/abseil-cpp +++ b/contrib/abseil-cpp @@ -1 +1 @@ -Subproject commit b004a8a02418b83de8b686caa0b0f6e39ac2191f +Subproject commit 215105818dfde3174fe799600bb0f3cae233d0bf From 229b8658db18d5efd546054ad735e492e99e9575 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 1 Dec 2021 18:39:34 +0100 Subject: [PATCH 0162/1260] s2 cleanup --- contrib/s2geometry-cmake/CMakeLists.txt | 15 +-------------- 1 file changed, 1 insertion(+), 14 deletions(-) diff --git a/contrib/s2geometry-cmake/CMakeLists.txt b/contrib/s2geometry-cmake/CMakeLists.txt index e5ef3755ab7..e2b0f20f408 100644 --- a/contrib/s2geometry-cmake/CMakeLists.txt +++ b/contrib/s2geometry-cmake/CMakeLists.txt @@ -106,20 +106,6 @@ set(S2_SRCS "${S2_SOURCE_DIR}/s2/util/math/mathutil.cc" "${S2_SOURCE_DIR}/s2/util/units/length-units.cc" - - # ClickHouse doesn't use strings from abseil. - # So, there is no duplicate symbols. -# "${ABSL_SOURCE_DIR}/absl/base/internal/raw_logging.cc" -# "${ABSL_SOURCE_DIR}/absl/base/internal/throw_delegate.cc" -# "${ABSL_SOURCE_DIR}/absl/numeric/int128.cc" -# "${ABSL_SOURCE_DIR}/absl/strings/ascii.cc" -# "${ABSL_SOURCE_DIR}/absl/strings/match.cc" -# "${ABSL_SOURCE_DIR}/absl/strings/numbers.cc" -# "${ABSL_SOURCE_DIR}/absl/strings/str_cat.cc" -# "${ABSL_SOURCE_DIR}/absl/strings/str_split.cc" -# "${ABSL_SOURCE_DIR}/absl/strings/string_view.cc" -# "${ABSL_SOURCE_DIR}/absl/strings/strip.cc" -# "${ABSL_SOURCE_DIR}/absl/strings/internal/memutil.cc" ) add_library(s2 ${S2_SRCS}) @@ -129,6 +115,7 @@ if (OPENSSL_FOUND) target_link_libraries(s2 PRIVATE ${OPENSSL_LIBRARIES}) endif() +# Copied from contrib/s2geometry/CMakeLists target_link_libraries(s2 PRIVATE absl::base absl::btree From 352c7b1cd3d4e3f686a4cec9e6ea6dcfbbeedff5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 2 Dec 2021 19:39:22 +0100 Subject: [PATCH 0163/1260] arrow: Point to upstream instead of fork --- .gitmodules | 4 ++-- contrib/arrow | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.gitmodules b/.gitmodules index 9b5875cf81d..cf396b4cfcf 100644 --- a/.gitmodules +++ b/.gitmodules @@ -54,8 +54,8 @@ url = https://github.com/ClickHouse-Extras/Turbo-Base64.git [submodule "contrib/arrow"] path = contrib/arrow - url = https://github.com/ClickHouse-Extras/arrow - branch = clickhouse-arrow-2.0.0 + url = https://github.com/apache/arrow + branch = blessed/release-6.0.1 [submodule "contrib/thrift"] path = contrib/thrift url = https://github.com/apache/thrift.git diff --git a/contrib/arrow b/contrib/arrow index 078e21bad34..aa9a7a698e3 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit 078e21bad344747b7656ef2d7a4f7410a0a303eb +Subproject commit aa9a7a698e33e278abe053f4634170b3b026e48e From f959380fd143ba09daeabf536c4a7e7e66c4f654 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 3 Dec 2021 09:36:55 +0100 Subject: [PATCH 0164/1260] Update submodules for libc++13 compatibility --- contrib/boost | 2 +- contrib/s2geometry | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/boost b/contrib/boost index fcb058e1459..a959abf9d6c 160000 --- a/contrib/boost +++ b/contrib/boost @@ -1 +1 @@ -Subproject commit fcb058e1459ac273ecfe7cdf72791cb1479115af +Subproject commit a959abf9d6c1c488a90a89bc57229fc07c3e396b diff --git a/contrib/s2geometry b/contrib/s2geometry index 38b7a290f92..471fe9dc931 160000 --- a/contrib/s2geometry +++ b/contrib/s2geometry @@ -1 +1 @@ -Subproject commit 38b7a290f927cc372218c2094602b83e35b18c05 +Subproject commit 471fe9dc931a4bb560333545186e9b5da168ac83 From 9bd50ebb08fcb6bff896e7cc7e538ac8bd8daa3f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 9 Dec 2021 20:17:46 +0100 Subject: [PATCH 0165/1260] Arrow missing deps --- contrib/arrow-cmake/CMakeLists.txt | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 627e905385d..0fc376b7208 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -264,6 +264,23 @@ set(ARROW_SRCS "${LIBRARY_DIR}/vendored/base64.cpp" "${LIBRARY_DIR}/vendored/datetime/tz.cpp" + "${LIBRARY_DIR}/vendored/musl/strptime.c" + "${LIBRARY_DIR}/vendored/uriparser/UriCommon.c" + "${LIBRARY_DIR}/vendored/uriparser/UriCompare.c" + "${LIBRARY_DIR}/vendored/uriparser/UriEscape.c" + "${LIBRARY_DIR}/vendored/uriparser/UriFile.c" + "${LIBRARY_DIR}/vendored/uriparser/UriIp4Base.c" + "${LIBRARY_DIR}/vendored/uriparser/UriIp4.c" + "${LIBRARY_DIR}/vendored/uriparser/UriMemory.c" + "${LIBRARY_DIR}/vendored/uriparser/UriNormalizeBase.c" + "${LIBRARY_DIR}/vendored/uriparser/UriNormalize.c" + "${LIBRARY_DIR}/vendored/uriparser/UriParseBase.c" + "${LIBRARY_DIR}/vendored/uriparser/UriParse.c" + "${LIBRARY_DIR}/vendored/uriparser/UriQuery.c" + "${LIBRARY_DIR}/vendored/uriparser/UriRecompose.c" + "${LIBRARY_DIR}/vendored/uriparser/UriResolve.c" + "${LIBRARY_DIR}/vendored/uriparser/UriShorten.c" + "${LIBRARY_DIR}/compute/api_aggregate.cc" "${LIBRARY_DIR}/compute/api_scalar.cc" "${LIBRARY_DIR}/compute/api_vector.cc" From f249a35e4c8a561ae85901165654d6eb7934cbe4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 9 Dec 2021 20:31:34 +0100 Subject: [PATCH 0166/1260] Shut up abseil --- contrib/abseil-cpp-cmake/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/contrib/abseil-cpp-cmake/CMakeLists.txt b/contrib/abseil-cpp-cmake/CMakeLists.txt index c8cb512066a..65e4c24ff5a 100644 --- a/contrib/abseil-cpp-cmake/CMakeLists.txt +++ b/contrib/abseil-cpp-cmake/CMakeLists.txt @@ -2,6 +2,8 @@ set(ABSL_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/abseil-cpp") if(NOT EXISTS "${ABSL_ROOT_DIR}/CMakeLists.txt") message(FATAL_ERROR " submodule third_party/abseil-cpp is missing. To fix try run: \n git submodule update --init --recursive") endif() +set(BUILD_TESTING OFF) +set(ABSL_PROPAGATE_CXX_STD ON) add_subdirectory("${ABSL_ROOT_DIR}" "${ClickHouse_BINARY_DIR}/contrib/abseil-cpp") add_library(abseil_swiss_tables INTERFACE) From 13dd5395870f45fa65608f45978961688d774308 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 10 Dec 2021 15:42:52 +0100 Subject: [PATCH 0167/1260] Prepare for boost 1.78 --- contrib/boost-cmake/CMakeLists.txt | 22 ++++------------------ 1 file changed, 4 insertions(+), 18 deletions(-) diff --git a/contrib/boost-cmake/CMakeLists.txt b/contrib/boost-cmake/CMakeLists.txt index 057a893e926..4a21b8a0e2d 100644 --- a/contrib/boost-cmake/CMakeLists.txt +++ b/contrib/boost-cmake/CMakeLists.txt @@ -1,9 +1,7 @@ option (USE_INTERNAL_BOOST_LIBRARY "Use internal Boost library" ON) if (NOT USE_INTERNAL_BOOST_LIBRARY) - # 1.70 like in contrib/boost - # 1.71 on CI - set(BOOST_VERSION 1.71) + set(BOOST_VERSION 1.78) find_package(Boost ${BOOST_VERSION} COMPONENTS system @@ -66,9 +64,11 @@ if (NOT EXTERNAL_BOOST_FOUND) set (SRCS_FILESYSTEM "${LIBRARY_DIR}/libs/filesystem/src/codecvt_error_category.cpp" + "${LIBRARY_DIR}/libs/filesystem/src/directory.cpp" + "${LIBRARY_DIR}/libs/filesystem/src/exception.cpp" "${LIBRARY_DIR}/libs/filesystem/src/operations.cpp" - "${LIBRARY_DIR}/libs/filesystem/src/path_traits.cpp" "${LIBRARY_DIR}/libs/filesystem/src/path.cpp" + "${LIBRARY_DIR}/libs/filesystem/src/path_traits.cpp" "${LIBRARY_DIR}/libs/filesystem/src/portability.cpp" "${LIBRARY_DIR}/libs/filesystem/src/unique_path.cpp" "${LIBRARY_DIR}/libs/filesystem/src/utf8_codecvt_facet.cpp" @@ -126,24 +126,11 @@ if (NOT EXTERNAL_BOOST_FOUND) # regex set (SRCS_REGEX - "${LIBRARY_DIR}/libs/regex/src/c_regex_traits.cpp" - "${LIBRARY_DIR}/libs/regex/src/cpp_regex_traits.cpp" - "${LIBRARY_DIR}/libs/regex/src/cregex.cpp" - "${LIBRARY_DIR}/libs/regex/src/fileiter.cpp" - "${LIBRARY_DIR}/libs/regex/src/icu.cpp" - "${LIBRARY_DIR}/libs/regex/src/instances.cpp" - "${LIBRARY_DIR}/libs/regex/src/internals.hpp" "${LIBRARY_DIR}/libs/regex/src/posix_api.cpp" "${LIBRARY_DIR}/libs/regex/src/regex_debug.cpp" - "${LIBRARY_DIR}/libs/regex/src/regex_raw_buffer.cpp" - "${LIBRARY_DIR}/libs/regex/src/regex_traits_defaults.cpp" "${LIBRARY_DIR}/libs/regex/src/regex.cpp" "${LIBRARY_DIR}/libs/regex/src/static_mutex.cpp" - "${LIBRARY_DIR}/libs/regex/src/usinstances.cpp" - "${LIBRARY_DIR}/libs/regex/src/w32_regex_traits.cpp" - "${LIBRARY_DIR}/libs/regex/src/wc_regex_traits.cpp" "${LIBRARY_DIR}/libs/regex/src/wide_posix_api.cpp" - "${LIBRARY_DIR}/libs/regex/src/winstances.cpp" ) add_library (_boost_regex ${SRCS_REGEX}) @@ -166,7 +153,6 @@ if (NOT EXTERNAL_BOOST_FOUND) set (SRCS_CONTEXT "${LIBRARY_DIR}/libs/context/src/dummy.cpp" - "${LIBRARY_DIR}/libs/context/src/execution_context.cpp" "${LIBRARY_DIR}/libs/context/src/posix/stack_traits.cpp" ) From 04471d72fd019738fbc06ac6aeb33d3c3fa794eb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 10 Dec 2021 16:15:04 +0100 Subject: [PATCH 0168/1260] TEMPORAL: Try updating everything to boost 1.78 --- .gitmodules | 2 +- contrib/boost | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index cf396b4cfcf..1053cc12475 100644 --- a/.gitmodules +++ b/.gitmodules @@ -48,7 +48,7 @@ branch = v3.13.0.1 [submodule "contrib/boost"] path = contrib/boost - url = https://github.com/ClickHouse-Extras/boost.git + url = https://github.com/Algunenano/boost.git [submodule "contrib/base64"] path = contrib/base64 url = https://github.com/ClickHouse-Extras/Turbo-Base64.git diff --git a/contrib/boost b/contrib/boost index a959abf9d6c..cf9517d7e46 160000 --- a/contrib/boost +++ b/contrib/boost @@ -1 +1 @@ -Subproject commit a959abf9d6c1c488a90a89bc57229fc07c3e396b +Subproject commit cf9517d7e46109105646fef3906c80632622bbb9 From e659ef6b36a505a514de727b197f57785f70cde3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 10 Dec 2021 18:06:20 +0100 Subject: [PATCH 0169/1260] TEMPORAL: patch NuRaft SSL mock to support boost 1.78.0 --- .gitmodules | 2 +- contrib/NuRaft | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index 1053cc12475..eadb76e4b38 100644 --- a/.gitmodules +++ b/.gitmodules @@ -207,7 +207,7 @@ branch = MergeWithUpstream [submodule "contrib/NuRaft"] path = contrib/NuRaft - url = https://github.com/ClickHouse-Extras/NuRaft.git + url = https://github.com/Algunenano/NuRaft.git [submodule "contrib/nanodbc"] path = contrib/nanodbc url = https://github.com/ClickHouse-Extras/nanodbc.git diff --git a/contrib/NuRaft b/contrib/NuRaft index d10351f312c..8572660b5d5 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit d10351f312c1ae1ca3fdda433693dfbef3acfece +Subproject commit 8572660b5d581ba365c679d66e57d221fa0cd1fb From af4beb4cb8ca895e175a8dae7cc717229cdec06f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 13 Dec 2021 17:52:17 +0100 Subject: [PATCH 0170/1260] Update boost asio with msan disabling commit --- contrib/boost | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boost b/contrib/boost index cf9517d7e46..65112f83c56 160000 --- a/contrib/boost +++ b/contrib/boost @@ -1 +1 @@ -Subproject commit cf9517d7e46109105646fef3906c80632622bbb9 +Subproject commit 65112f83c5657de9c06bc838071e7229eab60821 From 8208c3d49cdfd7fa60a96282247ae0e05d44e3c7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 13 Dec 2021 18:54:31 +0100 Subject: [PATCH 0171/1260] Update protobuf to ignore protoc MSAN errors --- contrib/protobuf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/protobuf b/contrib/protobuf index c1c5d020260..6bb70196c53 160000 --- a/contrib/protobuf +++ b/contrib/protobuf @@ -1 +1 @@ -Subproject commit c1c5d02026059f4c3cb51aaa08e82288d3e08b89 +Subproject commit 6bb70196c5360268d9f021bb7936fb0b551724c2 From f1535c7226fb28d27e5137612abc48ab5219633e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 14 Dec 2021 10:41:51 +0800 Subject: [PATCH 0172/1260] fix conflict --- tests/integration/helpers/cluster.py | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index ba365f55305..5f6940ebfd9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1903,11 +1903,7 @@ class ClickHouseInstance: self, cluster, base_path, name, base_config_dir, custom_main_configs, custom_user_configs, custom_dictionaries, macros, with_zookeeper, zookeeper_config_path, with_mysql_client, with_mysql, with_mysql8, with_mysql_cluster, with_kafka, with_kerberized_kafka, -<<<<<<< HEAD - with_rabbitmq, with_nginx, with_kerberized_hdfs, with_mongo, with_redis, with_minio, with_jdbc_bridge,with_hive, -======= - with_rabbitmq, with_nginx, with_kerberized_hdfs, with_mongo, with_redis, with_minio, with_azurite, with_jdbc_bridge, ->>>>>>> ck/master + with_rabbitmq, with_nginx, with_kerberized_hdfs, with_mongo, with_redis, with_minio, with_azurite, with_jdbc_bridge, with_hive with_cassandra, server_bin_path, odbc_bridge_bin_path, library_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, with_postgres, with_postgres_cluster, clickhouse_start_command=CLICKHOUSE_START_COMMAND, main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True, @@ -1955,6 +1951,7 @@ class ClickHouseInstance: self.with_azurite = with_azurite self.with_cassandra = with_cassandra self.with_jdbc_bridge = with_jdbc_bridge + self.with_hive = with_hive self.main_config_name = main_config_name self.users_config_name = users_config_name From 2f8998880d7ac81266ad80149d5956cb87a3fede Mon Sep 17 00:00:00 2001 From: gulige Date: Tue, 14 Dec 2021 10:41:51 +0800 Subject: [PATCH 0173/1260] to clear up compiler warning about "C-style cast". use std::npos to replace (size_t)-1 --- src/Core/Block.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 9db3040de4d..3208aefcdab 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -139,13 +139,13 @@ void Block::insert(size_t position, ColumnWithTypeAndName elem) if (elem.name.empty()) throw Exception("Column name in Block cannot be empty", ErrorCodes::AMBIGUOUS_COLUMN_NAME); - auto [it, inserted] = index_by_name.emplace(elem.name, (size_t)-1); + auto [it, inserted] = index_by_name.emplace(elem.name, std::npos); if (!inserted) checkColumnStructure(data[it->second], elem, "(columns with identical name must have identical structure)", true, ErrorCodes::AMBIGUOUS_COLUMN_NAME); for (auto & name_pos : index_by_name) - if (name_pos.second == (size_t)-1) + if (name_pos.second == std::npos) name_pos.second = position; else if (name_pos.second >= position) ++name_pos.second; From 5fa84c79c3cf2bbae8143e530ae82c3c00b840ca Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 14 Dec 2021 14:04:52 +0800 Subject: [PATCH 0174/1260] fixed bugs --- src/Storages/RemoteReadBufferCache.cpp | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Storages/RemoteReadBufferCache.cpp b/src/Storages/RemoteReadBufferCache.cpp index 99dc91eec3b..05cf2127a24 100644 --- a/src/Storages/RemoteReadBufferCache.cpp +++ b/src/Storages/RemoteReadBufferCache.cpp @@ -55,8 +55,15 @@ std::shared_ptr RemoteCacheController::recover(const std: LOG_INFO(log, "Recover cached file failed. local path:{}", local_path_.string()); return nullptr; } - - cache_controller->file_metadata_ptr = RemoteFileMetadataFactory::instance().get(cache_controller->metadata_class); + try + { + cache_controller->file_metadata_ptr = RemoteFileMetadataFactory::instance().get(cache_controller->metadata_class); + } + catch(...) + { + LOG_ERROR(log, "Get metadata class failed for {}", cache_controller->metadata_class); + cache_controller->file_metadata_ptr = nullptr; + } if (!cache_controller->file_metadata_ptr) { // do not load this invalid cached file and clear it. the clear action is in @@ -96,6 +103,7 @@ RemoteCacheController::RemoteCacheController( // when we allocate a whole new file cache , file_metadata_ptr must not be null. if (file_metadata_ptr) { + metadata_class = file_metadata_ptr->getName(); auto metadata_file_writer = std::make_unique((local_path_ / "metadata.txt").string()); auto str_buf = file_metadata_ptr->toString(); metadata_file_writer->write(str_buf.c_str(), str_buf.size()); @@ -214,7 +222,7 @@ void RemoteCacheController::close() std::unique_ptr RemoteCacheController::allocFile() { ReadSettings settings; - settings.local_fs_method = LocalFSReadMethod::read; + //settings.local_fs_method = LocalFSReadMethod::read; auto file_buffer = createReadBufferFromFileBase((local_path / "data.bin").string(), settings); if (file_buffer) @@ -299,6 +307,7 @@ bool RemoteReadBuffer::nextImpl() auto start_offset = file_buffer->getPosition(); auto end_offset = start_offset + file_buffer->internalBuffer().size(); file_cache_controller->waitMoreData(start_offset, end_offset); + //LOG_TRACE(&Poco::Logger::get("RemoteReadBuffer"), "nextImpl, {}->{}", start_offset, end_offset); auto status = file_buffer->next(); if (status) From b30e250eedd493cf0b25bf94dcecb291027591e8 Mon Sep 17 00:00:00 2001 From: cmsxbc Date: Sun, 12 Dec 2021 16:07:06 +0800 Subject: [PATCH 0175/1260] add max_concurrent_select_queries and max_concurrent_insert_queries --- .../settings.md | 39 +++++++++- programs/server/Server.cpp | 6 ++ src/Interpreters/ProcessList.cpp | 54 +++++++++++++- src/Interpreters/ProcessList.h | 34 ++++++++- .../__init__.py | 0 .../configs/concurrent_insert_restriction.xml | 3 + .../configs/concurrent_select_restriction.xml | 3 + .../test.py | 74 +++++++++++++++++++ 8 files changed, 209 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_concurrent_queries_restriction_by_query_kind/__init__.py create mode 100644 tests/integration/test_concurrent_queries_restriction_by_query_kind/configs/concurrent_insert_restriction.xml create mode 100644 tests/integration/test_concurrent_queries_restriction_by_query_kind/configs/concurrent_select_restriction.xml create mode 100644 tests/integration/test_concurrent_queries_restriction_by_query_kind/test.py diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 8a38fbce083..490543498be 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -640,7 +640,8 @@ On hosts with low RAM and swap, you possibly need setting `max_server_memory_usa ## max_concurrent_queries {#max-concurrent-queries} -The maximum number of simultaneously processed queries related to MergeTree table. Queries may be limited by other settings: [max_concurrent_queries_for_user](#max-concurrent-queries-for-user), [max_concurrent_queries_for_all_users](#max-concurrent-queries-for-all-users), [min_marks_to_honor_max_concurrent_queries](#min-marks-to-honor-max-concurrent-queries). +The maximum number of simultaneously processed queries related to MergeTree table. +Queries may be limited by other settings: [max_concurrent_insert_queries](#max-concurrent-insert-queries), [max_concurrent_select_queries](#max-concurrent-select-queries), [max_concurrent_queries_for_user](#max-concurrent-queries-for-user), [max_concurrent_queries_for_all_users](#max-concurrent-queries-for-all-users), [min_marks_to_honor_max_concurrent_queries](#min-marks-to-honor-max-concurrent-queries). !!! info "Note" These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. @@ -656,6 +657,42 @@ Possible values: 100 ``` +## max_concurrent_insert_queries {#max-concurrent-insert-queries} + +The maximum number of simultaneously processed insert queries. + +!!! info "Note" + These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. + +Possible values: + +- Positive integer. +- 0 — Disabled. + +**Example** + +``` xml +100 +``` + +## max_concurrent_select_queries {#max-concurrent-select-queries} + +The maximum number of simultaneously processed select queries. + +!!! info "Note" + These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. + +Possible values: + +- Positive integer. +- 0 — Disabled. + +**Example** + +``` xml +100 +``` + ## max_concurrent_queries_for_user {#max-concurrent-queries-for-user} The maximum number of simultaneously processed queries related to MergeTree table per user. diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 14075f9fbf2..d5b8eaf2557 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -859,6 +859,12 @@ if (ThreadFuzzer::instance().isEffective()) if (config->has("max_concurrent_queries")) global_context->getProcessList().setMaxSize(config->getInt("max_concurrent_queries", 0)); + if (config->has("max_concurrent_insert_queries")) + global_context->getProcessList().setMaxInsertQueriesAmount(config->getInt("max_concurrent_insert_queries", 0)); + + if (config->has("max_concurrent_select_queries")) + global_context->getProcessList().setMaxSelectQueriesAmount(config->getInt("max_concurrent_select_queries", 0)); + if (config->has("keeper_server")) global_context->updateKeeperConfiguration(*config); diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index a4583685a90..40b1b5fd40e 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -86,6 +86,20 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as throw Exception("Too many simultaneous queries. Maximum: " + toString(max_size), ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES); } + String query_kind{ast->getQueryKindString()}; + if (!is_unlimited_query) + { + auto amount = getQueryKindAmount(query_kind); + if (max_insert_queries_amount && query_kind == "Insert" && amount >= max_insert_queries_amount) + throw Exception(ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, + "Too many simultaneous insert queries. Maximum: {}, current: {}", + max_insert_queries_amount, amount); + if (max_select_queries_amount && query_kind == "Select" && amount >= max_select_queries_amount) + throw Exception(ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, + "Too many simultaneous select queries. Maximum: {}, current: {}", + max_select_queries_amount, amount); + } + { /** * `max_size` check above is controlled by `max_concurrent_queries` server setting and is a "hard" limit for how many @@ -176,7 +190,9 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as } auto process_it = processes.emplace(processes.end(), - query_context, query_, client_info, priorities.insert(settings.priority)); + query_context, query_, client_info, priorities.insert(settings.priority), query_kind); + + increaseQueryKindAmount(query_kind); res = std::make_shared(*this, process_it); @@ -242,6 +258,7 @@ ProcessListEntry::~ProcessListEntry() String user = it->getClientInfo().current_user; String query_id = it->getClientInfo().current_query_id; + String query_kind = it->query_kind; const QueryStatus * process_list_element_ptr = &*it; @@ -273,6 +290,9 @@ ProcessListEntry::~ProcessListEntry() LOG_ERROR(&Poco::Logger::get("ProcessList"), "Logical error: cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser"); std::terminate(); } + + parent.decreaseQueryKindAmount(query_kind); + parent.have_space.notify_all(); /// If there are no more queries for the user, then we will reset memory tracker and network throttler. @@ -286,11 +306,12 @@ ProcessListEntry::~ProcessListEntry() QueryStatus::QueryStatus( - ContextPtr context_, const String & query_, const ClientInfo & client_info_, QueryPriorities::Handle && priority_handle_) + ContextPtr context_, const String & query_, const ClientInfo & client_info_, QueryPriorities::Handle && priority_handle_, const String & query_kind_) : WithContext(context_) , query(query_) , client_info(client_info_) , priority_handle(std::move(priority_handle_)) + , query_kind(query_kind_) { auto settings = getContext()->getSettings(); limits.max_execution_time = settings.max_execution_time; @@ -485,4 +506,33 @@ ProcessList::UserInfo ProcessList::getUserInfo(bool get_profile_events) const return per_user_infos; } +void ProcessList::increaseQueryKindAmount(const String & query_kind) +{ + auto found = query_kind_amounts.find(query_kind); + if (found == query_kind_amounts.end()) + query_kind_amounts[query_kind] = 1; + else + found->second += 1; +} + +void ProcessList::decreaseQueryKindAmount(const String & query_kind) +{ + auto found = query_kind_amounts.find(query_kind); + /// TODO: we could just rebuild the map, as we have saved all query_kind. + if (found == query_kind_amounts.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong query kind amount: decrease before increase on '{}'", query_kind); + else if (found->second == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong query kind amount: decrease to negative on '{}'", query_kind, found->second); + else + found->second -= 1; + +} +ProcessList::QueryAmount ProcessList::getQueryKindAmount(const String & query_kind) +{ + auto found = query_kind_amounts.find(query_kind); + if (found == query_kind_amounts.end()) + return 0; + return found->second; +} + } diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 9c826bde061..208aed8d364 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -118,13 +118,17 @@ protected: ProcessListForUser * user_process_list = nullptr; + String query_kind; + public: QueryStatus( ContextPtr context_, const String & query_, const ClientInfo & client_info_, - QueryPriorities::Handle && priority_handle_); + QueryPriorities::Handle && priority_handle_, + const String & query_kind_ + ); ~QueryStatus(); @@ -256,6 +260,7 @@ class ProcessList public: using Element = QueryStatus; using Entry = ProcessListEntry; + using QueryAmount = UInt64; /// list, for iterators not to invalidate. NOTE: could replace with cyclic buffer, but not worth. using Container = std::list; @@ -265,6 +270,8 @@ public: /// User -> queries using UserToQueries = std::unordered_map; + using QueryKindToAmount = std::unordered_map; + protected: friend class ProcessListEntry; @@ -287,6 +294,19 @@ protected: /// Call under lock. Finds process with specified current_user and current_query_id. QueryStatus * tryGetProcessListElement(const String & current_query_id, const String & current_user); + /// limit for insert. 0 means no limit. Otherwise, when limit exceeded, an exception is thrown. + size_t max_insert_queries_amount = 0; + + /// limit for select. 0 means no limit. Otherwise, when limit exceeded, an exception is thrown. + size_t max_select_queries_amount = 0; + + /// amount of queries by query kind. + QueryKindToAmount query_kind_amounts; + + void increaseQueryKindAmount(const String & query_kind); + void decreaseQueryKindAmount(const String & query_kind); + QueryAmount getQueryKindAmount(const String & query_kind); + public: using EntryPtr = std::shared_ptr; @@ -312,6 +332,18 @@ public: max_size = max_size_; } + void setMaxInsertQueriesAmount(size_t max_insert_queries_amount_) + { + std::lock_guard lock(mutex); + max_insert_queries_amount = max_insert_queries_amount_; + } + + void setMaxSelectQueriesAmount(size_t max_select_queries_amount_) + { + std::lock_guard lock(mutex); + max_select_queries_amount = max_select_queries_amount_; + } + /// Try call cancel() for input and output streams of query with specified id and user CancellationCode sendCancelToQuery(const String & current_query_id, const String & current_user, bool kill = false); diff --git a/tests/integration/test_concurrent_queries_restriction_by_query_kind/__init__.py b/tests/integration/test_concurrent_queries_restriction_by_query_kind/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_concurrent_queries_restriction_by_query_kind/configs/concurrent_insert_restriction.xml b/tests/integration/test_concurrent_queries_restriction_by_query_kind/configs/concurrent_insert_restriction.xml new file mode 100644 index 00000000000..7753c579902 --- /dev/null +++ b/tests/integration/test_concurrent_queries_restriction_by_query_kind/configs/concurrent_insert_restriction.xml @@ -0,0 +1,3 @@ + + 2 + diff --git a/tests/integration/test_concurrent_queries_restriction_by_query_kind/configs/concurrent_select_restriction.xml b/tests/integration/test_concurrent_queries_restriction_by_query_kind/configs/concurrent_select_restriction.xml new file mode 100644 index 00000000000..c8f081e6804 --- /dev/null +++ b/tests/integration/test_concurrent_queries_restriction_by_query_kind/configs/concurrent_select_restriction.xml @@ -0,0 +1,3 @@ + + 2 + diff --git a/tests/integration/test_concurrent_queries_restriction_by_query_kind/test.py b/tests/integration/test_concurrent_queries_restriction_by_query_kind/test.py new file mode 100644 index 00000000000..d13fc467500 --- /dev/null +++ b/tests/integration/test_concurrent_queries_restriction_by_query_kind/test.py @@ -0,0 +1,74 @@ +import time +from multiprocessing.dummy import Pool + +import pytest +from helpers.cluster import ClickHouseCluster + + +cluster = ClickHouseCluster(__file__) +node_insert = cluster.add_instance('node_insert', main_configs=['configs/concurrent_insert_restriction.xml']) +node_select = cluster.add_instance('node_select', main_configs=['configs/concurrent_select_restriction.xml']) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + node_select.query("create table test_concurrent_insert (x UInt64) ENGINE = MergeTree() order by tuple()") + node_insert.query("create table test_concurrent_insert (x UInt64) ENGINE = MergeTree() order by tuple()") + yield cluster + finally: + cluster.shutdown() + + +def execute_with_background(node, sql, background_sql, background_times, wait_times=3): + r = None + for _ in range(wait_times): + r = node.query('show processlist', stdin='') + if not r.strip(): + break + time.sleep(1) + else: + assert False, "there are unknown background queries: {}".format(r) + for _ in range(background_times): + node.get_query_request(background_sql, stdin='') + time.sleep(0.5) # wait background to start. + return node.query(sql, stdin='') + + +def common_pattern(node, query_kind, restricted_sql, normal_sql, limit): + # restriction is working + with pytest.raises(Exception, match=r".*Too many simultaneous {} queries.*".format(query_kind)): + execute_with_background(node, restricted_sql, restricted_sql, limit) + + # different query kind is independent + execute_with_background(node, normal_sql, restricted_sql, limit) + + # normal + execute_with_background(node, restricted_sql, '', 0) + + +def test_select(started_cluster): + common_pattern( + node_select, 'select', + 'select sleep(3)', + 'insert into test_concurrent_insert values (0)', + 2 + ) + + # subquery is not counted + execute_with_background( + node_select, + 'select sleep(3)', + 'insert into test_concurrent_insert select sleep(3)', + 2 + ) + + +def test_insert(started_cluster): + common_pattern( + node_insert, 'insert', + 'insert into test_concurrent_insert select sleep(3)', + 'select 1', + 2 + ) From 0b8bc3f6de11ddc962168a84573db41581277195 Mon Sep 17 00:00:00 2001 From: cmsxbc Date: Tue, 14 Dec 2021 08:44:37 +0800 Subject: [PATCH 0176/1260] wait more times for clear background queries. --- .../test.py | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_concurrent_queries_restriction_by_query_kind/test.py b/tests/integration/test_concurrent_queries_restriction_by_query_kind/test.py index d13fc467500..2d16d9157f6 100644 --- a/tests/integration/test_concurrent_queries_restriction_by_query_kind/test.py +++ b/tests/integration/test_concurrent_queries_restriction_by_query_kind/test.py @@ -36,16 +36,16 @@ def execute_with_background(node, sql, background_sql, background_times, wait_ti return node.query(sql, stdin='') -def common_pattern(node, query_kind, restricted_sql, normal_sql, limit): +def common_pattern(node, query_kind, restricted_sql, normal_sql, limit, wait_times): # restriction is working with pytest.raises(Exception, match=r".*Too many simultaneous {} queries.*".format(query_kind)): - execute_with_background(node, restricted_sql, restricted_sql, limit) + execute_with_background(node, restricted_sql, restricted_sql, limit, wait_times) # different query kind is independent - execute_with_background(node, normal_sql, restricted_sql, limit) + execute_with_background(node, normal_sql, restricted_sql, limit, wait_times) # normal - execute_with_background(node, restricted_sql, '', 0) + execute_with_background(node, restricted_sql, '', 0, wait_times) def test_select(started_cluster): @@ -53,7 +53,8 @@ def test_select(started_cluster): node_select, 'select', 'select sleep(3)', 'insert into test_concurrent_insert values (0)', - 2 + 2, + 10 ) # subquery is not counted @@ -61,7 +62,8 @@ def test_select(started_cluster): node_select, 'select sleep(3)', 'insert into test_concurrent_insert select sleep(3)', - 2 + 2, + 10 ) @@ -70,5 +72,6 @@ def test_insert(started_cluster): node_insert, 'insert', 'insert into test_concurrent_insert select sleep(3)', 'select 1', - 2 + 2, + 10 ) From 248e933fc4542a4f630600e0f565a08d566caae2 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 14 Dec 2021 16:06:30 +0800 Subject: [PATCH 0177/1260] Add test for local cache --- src/Common/ProfileEvents.cpp | 1 + src/Storages/RemoteReadBufferCache.cpp | 10 +++++++-- tests/integration/helpers/cluster.py | 2 +- .../test_hive_query/configs/config.xml | 7 +++++++ tests/integration/test_hive_query/test.py | 21 +++++++++++++++++++ 5 files changed, 38 insertions(+), 3 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 982523a3ef2..9b762028f17 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -274,6 +274,7 @@ M(ThreadPoolReaderPageCacheMissElapsedMicroseconds, "Time spent reading data inside the asynchronous job in ThreadPoolReader - when read was not done from page cache.") \ \ M(AsynchronousReadWaitMicroseconds, "Time spent in waiting for asynchronous reads.") \ + M(ExternalDataSourceLocalCacheReadBytes, "Bytes read from local cache buffer in RemoteReadBufferCache")\ namespace ProfileEvents diff --git a/src/Storages/RemoteReadBufferCache.cpp b/src/Storages/RemoteReadBufferCache.cpp index 05cf2127a24..aa334215575 100644 --- a/src/Storages/RemoteReadBufferCache.cpp +++ b/src/Storages/RemoteReadBufferCache.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -16,7 +17,10 @@ #include namespace fs = std::filesystem; - +namespace ProfileEvents +{ + extern const Event ExternalDataSourceLocalCacheReadBytes; +} namespace DB { namespace ErrorCodes @@ -307,13 +311,15 @@ bool RemoteReadBuffer::nextImpl() auto start_offset = file_buffer->getPosition(); auto end_offset = start_offset + file_buffer->internalBuffer().size(); file_cache_controller->waitMoreData(start_offset, end_offset); - //LOG_TRACE(&Poco::Logger::get("RemoteReadBuffer"), "nextImpl, {}->{}", start_offset, end_offset); auto status = file_buffer->next(); if (status) + { BufferBase::set(file_buffer->buffer().begin(), file_buffer->buffer().size(), file_buffer->offset()); + ProfileEvents::increment(ProfileEvents::ExternalDataSourceLocalCacheReadBytes, file_buffer->available()); + } return status; } diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 5f6940ebfd9..33d0aa53fe9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1903,7 +1903,7 @@ class ClickHouseInstance: self, cluster, base_path, name, base_config_dir, custom_main_configs, custom_user_configs, custom_dictionaries, macros, with_zookeeper, zookeeper_config_path, with_mysql_client, with_mysql, with_mysql8, with_mysql_cluster, with_kafka, with_kerberized_kafka, - with_rabbitmq, with_nginx, with_kerberized_hdfs, with_mongo, with_redis, with_minio, with_azurite, with_jdbc_bridge, with_hive + with_rabbitmq, with_nginx, with_kerberized_hdfs, with_mongo, with_redis, with_minio, with_azurite, with_jdbc_bridge, with_hive, with_cassandra, server_bin_path, odbc_bridge_bin_path, library_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, with_postgres, with_postgres_cluster, clickhouse_start_command=CLICKHOUSE_START_COMMAND, main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True, diff --git a/tests/integration/test_hive_query/configs/config.xml b/tests/integration/test_hive_query/configs/config.xml index a895d417a2d..d7593bee803 100644 --- a/tests/integration/test_hive_query/configs/config.xml +++ b/tests/integration/test_hive_query/configs/config.xml @@ -22,4 +22,11 @@ /etc/clickhouse-server/extra_conf.d/hdfs-site.xml + + system +
metric_log
+ 1000 + 1000 + + diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index a30f0b03070..456b2f645c0 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -1,6 +1,7 @@ import logging import os +import time import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV @@ -26,6 +27,7 @@ def started_cluster(): def test_create_parquet_table(started_cluster): logging.info('Start testing creating hive table ...') node = started_cluster.instances['h0_0_0'] + node.query("set input_format_parquet_allow_missing_columns = true") result = node.query(""" CREATE TABLE default.demo_parquet (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) """) @@ -89,3 +91,22 @@ def test_text_count(started_cluster): 2021-11-16 2 """ assert result == expected_result + +def test_parquet_groupby_witch_cache(started_cluster): + logging.info('Start testing groupby ...') + node = started_cluster.instances['h0_0_0'] + result = node.query(""" + SELECT day, count(*) FROM default.demo_parquet group by day order by day + """) + expected_result = """2021-11-01 1 +2021-11-05 2 +2021-11-11 1 +2021-11-16 2 +""" + assert result == expected_result +def test_cache_read_bytes(started_cluster): + node = started_cluster.instances['h0_0_0'] + time.sleep(3) + result = node.query("select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0") + logging.info("Read bytes from cache:{}".format(result)) + assert result.strip() != '0' From ca3f7425a4d423a5ea04a57e9a39d6fa12c25bb9 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 14 Dec 2021 17:37:31 +0800 Subject: [PATCH 0178/1260] fix code --- contrib/thrift-cmake/CMakeLists.txt | 18 -- programs/server/config.xml | 7 - .../Serializations/SerializationNumber.cpp | 4 +- src/Formats/FormatSettings.h | 2 - src/IO/ReadHelpers.h | 28 +-- .../Formats/Impl/HiveTextRowInputFormat.cpp | 1 - src/Processors/examples/CMakeLists.txt | 6 +- src/Storages/Hive/HiveCommon.cpp | 205 +++++++++--------- src/Storages/Hive/HiveCommon.h | 70 +++--- src/Storages/Hive/HiveFile.cpp | 15 +- src/Storages/Hive/StorageHive.cpp | 61 +++--- 11 files changed, 196 insertions(+), 221 deletions(-) diff --git a/contrib/thrift-cmake/CMakeLists.txt b/contrib/thrift-cmake/CMakeLists.txt index 26e6b4d085c..c1abccdc271 100644 --- a/contrib/thrift-cmake/CMakeLists.txt +++ b/contrib/thrift-cmake/CMakeLists.txt @@ -45,21 +45,3 @@ include("${ClickHouse_SOURCE_DIR}/contrib/thrift/build/cmake/ConfigureChecks.cma add_library(${THRIFT_LIBRARY} ${thriftcpp_SOURCES} ${thriftcpp_threads_SOURCES}) target_include_directories(${THRIFT_LIBRARY} SYSTEM PUBLIC "${THRIFT_INCLUDE_DIR}" ${CMAKE_CURRENT_BINARY_DIR}) target_link_libraries (${THRIFT_LIBRARY} PUBLIC boost::headers_only) - -# set(COMPILER_DIR "${ClickHouse_SOURCE_DIR}/contrib/thrift/compiler/cpp") -# configure_file(${COMPILER_DIR}/src/thrift/version.h.in ${COMPILER_DIR}/src/thrift/version.h) -# set(thrift_compiler_SOURCES -# ${COMPILER_DIR}/src/thrift/main.cc -# ${COMPILER_DIR}/src/thrift/audit/t_audit.cpp -# ${COMPILER_DIR}/src/thrift/generate/t_cpp_generator.cc -# ${COMPILER_DIR}/src/thrift/common.cc -# ${COMPILER_DIR}/src/thrift/generate/t_generator.cc -# ${COMPILER_DIR}/src/thrift/parse/t_typedef.cc -# ${COMPILER_DIR}/src/thrift/parse/parse.cc -# ${COMPILER_DIR}/src/thrift/version.h -# ${COMPILER_DIR}/src/thrift/thrifty.cc -# ${COMPILER_DIR}/src/thrift/thriftl.cc -# ${COMPILER_DIR}/src/thrift/thrifty.hh -# ) -# add_executable(${THRIFT_COMPILER} ${thrift_compiler_SOURCES}) -# include_directories(${THRIFT_COMPILER} SYSTEM ${COMPILER_DIR}/src) diff --git a/programs/server/config.xml b/programs/server/config.xml index 357d2eec061..9a2a6d7729f 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1285,11 +1285,4 @@ --> - - - true - local_cache - 559096952 - 1048576 - diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index 6b220f27387..c5e2b31e043 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -95,10 +95,10 @@ void SerializationNumber::deserializeTextJSON(IColumn & column, ReadBuffer & } template -void SerializationNumber::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +void SerializationNumber::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & /*settings*/) const { FieldType x; - readCSV(x, istr, settings.csv.read_bool_as_uint8); + readCSV(x, istr); assert_cast &>(column).getData().push_back(x); } diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index a228f912593..e40396c65c2 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -96,7 +96,6 @@ struct FormatSettings bool crlf_end_of_line = false; bool input_format_enum_as_number = false; bool input_format_arrays_as_nested_csv = false; - bool read_bool_as_uint8 = false; String null_representation = "\\N"; } csv; @@ -105,7 +104,6 @@ struct FormatSettings char fields_delimiter = '\x01'; char collection_items_delimiter = '\x02'; char map_keys_delimiter = '\x03'; - bool read_bool_as_uint8 = true; Names input_field_names; } hive_text; diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index a3f6f6255d4..61508583c22 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1071,11 +1071,9 @@ inline void readDoubleQuoted(LocalDateTime & x, ReadBuffer & buf) assertChar('"', buf); } - /// CSV, for numbers, dates: quotes are optional, no special escaping rules. -/// read_bool_as_uint8 enable read "t" and "f" as UInt8 value in Hive TEXT File. template -inline void readCSVSimple(T & x, ReadBuffer & buf, bool read_bool_as_uint8 = false) +inline void readCSVSimple(T & x, ReadBuffer & buf) { if (buf.eof()) throwReadAfterEOF(); @@ -1085,31 +1083,17 @@ inline void readCSVSimple(T & x, ReadBuffer & buf, bool read_bool_as_uint8 = fal if (maybe_quote == '\'' || maybe_quote == '\"') ++buf.position(); - if constexpr (std::is_same_v) - { - if (read_bool_as_uint8 && (*buf.position() == 't' || *buf.position() == 'f')) - { - bool v = false; - readBoolTextWord(v, buf); - x = v ? 1 : 0; - } - else - { - readText(x, buf); - } - } - else - { - readText(x, buf); - } + readText(x, buf); if (maybe_quote == '\'' || maybe_quote == '\"') assertChar(maybe_quote, buf); } template -inline std::enable_if_t, void> -readCSV(T & x, ReadBuffer & buf, bool read_bool_as_uint8 = false) { readCSVSimple(x, buf, read_bool_as_uint8); } +inline std::enable_if_t, void> readCSV(T & x, ReadBuffer & buf) +{ + readCSVSimple(x, buf); +} inline void readCSV(String & x, ReadBuffer & buf, const FormatSettings::CSV & settings) { readCSVString(x, buf, settings); } inline void readCSV(LocalDate & x, ReadBuffer & buf) { readCSVSimple(x, buf); } diff --git a/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp b/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp index aeeb35a349c..d2e83cc6bf5 100644 --- a/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp @@ -39,7 +39,6 @@ void registerInputFormatHiveText(FormatFactory & factory) { FormatSettings settings_copy = settings; settings_copy.csv.delimiter = settings_copy.hive_text.fields_delimiter; - settings_copy.csv.read_bool_as_uint8 = settings_copy.hive_text.read_bool_as_uint8; return std::make_shared(sample, buf, params, settings_copy); }); } diff --git a/src/Processors/examples/CMakeLists.txt b/src/Processors/examples/CMakeLists.txt index 82f8c754373..6f78d611f45 100644 --- a/src/Processors/examples/CMakeLists.txt +++ b/src/Processors/examples/CMakeLists.txt @@ -1,6 +1,2 @@ add_executable (comma_separated_streams comma_separated_streams.cpp) -target_link_libraries (comma_separated_streams PRIVATE dbms) -#target_include_directories(comma_separated_streams SYSTEM BEFORE PRIVATE -# ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src -# ${ClickHouse_SOURCE_DIR}/contrib/arrow-cmake/cpp/src) -# dbms_target_link_libraries(PRIVATE ${ARROW_LIBRARY}) +target_link_libraries (comma_separated_streams PRIVATE dbms) \ No newline at end of file diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 51f665f8ab0..a47c4a5a87b 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -13,9 +13,35 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -std::shared_ptr HiveMetastoreClient::getTableMetadata(const std::string & db_name, const std::string & table_name) +bool HiveMetastoreClient::shouldUpdateTableMetadata( + const String & db_name, const String & table_name, const std::vector & partitions) { - LOG_TRACE(log, "get table meta:" + db_name + ":" + table_name); + String cache_key = getCacheKey(db_name, table_name); + std::shared_ptr metadata = table_metadata_cache.get(cache_key); + if (!metadata) + return true; + + auto old_partiton_infos = metadata->getPartitionInfos(); + if (old_partiton_infos.size() != partitions.size()) + return true; + + for (const auto & partition : partitions) + { + auto it = old_partiton_infos.find(partition.sd.location); + if (it == old_partiton_infos.end()) + return true; + + const auto & old_partition_info = it->second; + if (!old_partition_info.haveSameParameters(partition)) + return true; + } + return false; +} + +std::shared_ptr +HiveMetastoreClient::getTableMetadata(const String & db_name, const String & table_name) +{ + LOG_TRACE(log, "Get table metadata for {}.{}", db_name, table_name); std::lock_guard lock{mutex}; auto table = std::make_shared(); @@ -24,71 +50,66 @@ std::shared_ptr HiveMetastoreClient::get { client->get_table(*table, db_name, table_name); - /** - * query the latest partition info to check new change - */ + /// Query the latest partition info to check new change. client->get_partitions(partitions, db_name, table_name, -1); } catch (apache::thrift::transport::TTransportException & e) { setExpired(); - throw Exception("Hive Metastore expired because " + String(e.what()), ErrorCodes::NO_HIVEMETASTORE); + throw Exception(ErrorCodes::NO_HIVEMETASTORE, "Hive Metastore expired because {}", String(e.what())); } - std::string cache_key = db_name + "." + table_name; - std::shared_ptr result = table_meta_cache.get(cache_key); - bool update_cache = false; - std::map old_partition_infos; - std::map new_partition_infos; - if (result) - { - old_partition_infos = result->getPartitionInfos(); - if (old_partition_infos.size() != partitions.size()) - update_cache = true; - } - else - { - update_cache = true; - } + bool update_cache = shouldUpdateTableMetadata(db_name, table_name, partitions); + String cache_key = db_name + "." + table_name; - for (const auto & partition : partitions) - { - auto & partition_info = new_partition_infos[partition.sd.location]; - partition_info.partition = partition; - - // query files under the partition by hdfs api is costly, we reuse the files in case the partition has no change - if (result) - { - auto it = old_partition_infos.find(partition.sd.location); - if (it != old_partition_infos.end() && it->second.equal(partition)) - partition_info.files = it->second.files; - else - update_cache = true; - } - } - - /** - * FIXME: force to update. - * we have found some cases under which the partition's meta don't update if the table changed - */ - //invalid_meta = true; + std::shared_ptr metadata = table_metadata_cache.get(cache_key); if (update_cache) { - LOG_INFO(log, "reload hive partition meta info:" + db_name + ":" + table_name); - result = std::make_shared(db_name, table_name, table, std::move(new_partition_infos), getContext()); - table_meta_cache.set(cache_key, result); + LOG_INFO(log, "Reload hive partition metadata info for {}.{}", db_name, table_name); + + /// Generate partition infos from partitions and old partition infos(if exists). + std::map new_partition_infos; + if (metadata) + { + auto & old_partiton_infos = metadata->getPartitionInfos(); + for (const auto & partition : partitions) + { + auto it = old_partiton_infos.find(partition.sd.location); + if (it == old_partiton_infos.end() || !it->second.haveSameParameters(partition) || !it->second.initialized) + { + new_partition_infos.emplace(partition.sd.location, PartitionInfo(partition)); + continue; + } + else + { + PartitionInfo new_partition_info(partition); + new_partition_info.files = std::move(it->second.files); + new_partition_info.initialized = true; + } + } + } + else + { + for (const auto & partition : partitions) + new_partition_infos.emplace(partition.sd.location, PartitionInfo(partition)); + } + + metadata = std::make_shared( + db_name, table_name, table, std::move(new_partition_infos), getContext()); + table_metadata_cache.set(cache_key, metadata); } - return result; + return metadata; } -void HiveMetastoreClient::clearTableMetadata(const std::string & db_name, const std::string & table_name) +void HiveMetastoreClient::clearTableMetadata(const String & db_name, const String & table_name) { + String cache_key = getCacheKey(db_name, table_name); + std::lock_guard lock{mutex}; - std::string cache_key = db_name + "." + table_name; - std::shared_ptr meta = table_meta_cache.get(cache_key); - if (meta) - table_meta_cache.set(cache_key, nullptr); + std::shared_ptr metadata = table_metadata_cache.get(cache_key); + if (metadata) + table_metadata_cache.set(cache_key, nullptr); } void HiveMetastoreClient::setClient(std::shared_ptr client_) @@ -98,90 +119,54 @@ void HiveMetastoreClient::setClient(std::shared_ptrfirst != it2->first || it1->second != it2->second) return false; } - return (it1 == partition.parameters.end() && it2 == other.parameters.end()); + return (it1 == partition.parameters.cend() && it2 == other.parameters.cend()); } -std::vector HiveMetastoreClient::HiveTableMetadata::getPartitions() +std::vector HiveMetastoreClient::HiveTableMetadata::getPartitions() const { std::vector result; std::lock_guard lock{mutex}; - for (const auto & partition : partition_infos) - result.emplace_back(partition.second.partition); + for (const auto & partition_info : partition_infos) + result.emplace_back(partition_info.second.partition); return result; } -std::vector HiveMetastoreClient::HiveTableMetadata::getLocationFiles(const std::string & location) +std::vector HiveMetastoreClient::HiveTableMetadata::getFilesByLocation(const HDFSFSPtr & fs, const String & location) { - std::map::const_iterator it; + LOG_TRACE(log, "List directory {}", location); + std::map::iterator it; if (!empty_partition_keys) { std::lock_guard lock{mutex}; it = partition_infos.find(location); if (it == partition_infos.end()) - throw Exception("invalid location " + location, ErrorCodes::BAD_ARGUMENTS); - return it->second.files; - } + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid location {}", location); - auto fs_builder = createHDFSBuilder(getNameNodeUrl(table->sd.location), getContext()->getGlobalContext()->getConfigRef()); - auto fs = createHDFSFS(fs_builder.get()); - Poco::URI uri(location); - HDFSFileInfo dir_info; - dir_info.file_info = hdfsListDirectory(fs.get(), uri.getPath().c_str(), &dir_info.length); - - std::vector result; - result.reserve(dir_info.length); - for (int i = 0; i < dir_info.length; ++i) - { - auto & file_info = dir_info.file_info[i]; - /// skip directories and empty files, mKind value 'D' represents directory, otherwise file - if (file_info.mKind != 'D' && file_info.mSize > 0) - result.emplace_back(String(file_info.mName), file_info.mLastMod, file_info.mSize); - } - - if (!empty_partition_keys) - { - std::lock_guard lock{mutex}; - partition_infos[location].files = result; - } - return result; -} - -std::vector HiveMetastoreClient::HiveTableMetadata::getLocationFiles(const HDFSFSPtr & fs, const std::string & location) -{ - LOG_TRACE(&Poco::Logger::get("HiveMetastoreClient"), "ls {}", location); - std::map::const_iterator it; - bool x = false; - if (!empty_partition_keys && x) - { - std::lock_guard lock{mutex}; - it = partition_infos.find(location); - if (it == partition_infos.end()) - throw Exception("invalid location " + location, ErrorCodes::BAD_ARGUMENTS); - - LOG_TRACE(&Poco::Logger::get("HiveMetastoreClient"), "empty_partition_keys {} {}", location, it->second.files.size()); - return it->second.files; + if (it->second.initialized) + { + LOG_TRACE(log, "Get {} files under directory {}", it->second.files.size(), location); + return it->second.files; + } } Poco::URI location_uri(location); HDFSFileInfo ls; ls.file_info = hdfsListDirectory(fs.get(), location_uri.getPath().c_str(), &ls.length); std::vector result; - LOG_TRACE(&Poco::Logger::get("HiveMetastoreClient"), "ls result. {} {}", ls.length, location); for (int i = 0; i < ls.length; ++i) { auto & file_info = ls.file_info[i]; - LOG_TRACE(&Poco::Logger::get("HiveMetastoreClient"), "get file:{} {} {}", file_info.mName, file_info.mKind, file_info.mSize); if (file_info.mKind != 'D' && file_info.mSize > 0) result.emplace_back(String(file_info.mName), file_info.mLastMod, file_info.mSize); } @@ -189,8 +174,14 @@ std::vector HiveMetastoreClient::HiveTableMetadat if (!empty_partition_keys) { std::lock_guard lock{mutex}; - partition_infos[location].files = result; + it = partition_infos.find(location); + if (it == partition_infos.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid location {}", location); + + it->second.files = result; + it->second.initialized = true; } + LOG_TRACE(log, "Get {} files under directory {}", result.size(), location); return result; } @@ -211,7 +202,7 @@ HiveMetastoreClientPtr HiveMetastoreClientFactory::getOrCreate(const String & na auto it = clients.find(name); if (it == clients.end() || it->second->isExpired()) { - // connect to hive metastore + /// Connect to hive metastore Poco::URI hive_metastore_url(name); const auto & host = hive_metastore_url.getHost(); auto port = hive_metastore_url.getPort(); diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index 22749faf75d..ea58f4d7230 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -19,32 +19,37 @@ class HiveMetastoreClient : public WithContext public: struct FileInfo { - std::string path; - UInt64 last_modify_time; // in ms + String path; + UInt64 last_modify_time; /// In ms size_t size; FileInfo() = default; - //FileInfo(const FileInfo & b) : path(b.path), last_modify_time(b.last_modify_time), size(b.size){} - FileInfo(const std::string & path_, UInt64 last_modify_time_, size_t size_) : path(path_), last_modify_time(last_modify_time_), size(size_) {} + FileInfo(const String & path_, UInt64 last_modify_time_, size_t size_) + : path(path_), last_modify_time(last_modify_time_), size(size_) + { + } }; struct PartitionInfo { Apache::Hadoop::Hive::Partition partition; std::vector files; + bool initialized = false; /// If true, files are initialized. - bool equal(const Apache::Hadoop::Hive::Partition & other); + explicit PartitionInfo(const Apache::Hadoop::Hive::Partition & partition_): partition(partition_) {} + bool haveSameParameters(const Apache::Hadoop::Hive::Partition & other) const; }; - // use for speeding up query metadata + + /// Used for speeding up metadata query process. struct HiveTableMetadata : public WithContext { public: HiveTableMetadata( - const std::string & db_name_, - const std::string & table_name_, + const String & db_name_, + const String & table_name_, std::shared_ptr table_, - std::map && partition_infos_, + const std::map & partition_infos_, ContextPtr context_) : WithContext(context_) , db_name(db_name_) @@ -54,41 +59,56 @@ public: , empty_partition_keys(table->partitionKeys.empty()) { } - std::vector getPartitions(); - inline std::map getPartitionInfos() + + + std::map & getPartitionInfos() { std::lock_guard lock{mutex}; return partition_infos; } - std::vector getLocationFiles(const std::string & location); - std::vector getLocationFiles(const HDFSFSPtr & fs, const std::string & location); - inline std::shared_ptr getTable() { return table; } + + std::shared_ptr getTable() const + { + std::lock_guard lock{mutex}; + return table; + } + + std::vector getPartitions() const; + + std::vector getFilesByLocation(const HDFSFSPtr & fs, const String & location); private: - std::string db_name; - std::string table_name; + String db_name; + String table_name; - std::mutex mutex; + mutable std::mutex mutex; std::shared_ptr table; - std::map partition_infos; + std::map partition_infos; const bool empty_partition_keys; + + Poco::Logger * log = &Poco::Logger::get("HiveMetastoreClient"); }; explicit HiveMetastoreClient(std::shared_ptr client_, ContextPtr context_) - : WithContext(context_), client(client_), table_meta_cache(1000) + : WithContext(context_), client(client_), table_metadata_cache(1000) { } - std::shared_ptr getTableMetadata(const std::string & db_name, const std::string & table_name); - void clearTableMetadata(const std::string & db_name, const std::string & table_name); + std::shared_ptr getTableMetadata(const String & db_name, const String & table_name); + void clearTableMetadata(const String & db_name, const String & table_name); void setClient(std::shared_ptr client_); - inline bool isExpired() const { return expired; } - inline void setExpired() { expired = true; } - inline void clearExpired() { expired = false; } + bool isExpired() const { return expired; } + void setExpired() { expired = true; } + void clearExpired() { expired = false; } private: + static String getCacheKey(const String & db_name, const String & table_name) { return db_name + "." + table_name; } + + bool shouldUpdateTableMetadata( + const String & db_name, const String & table_name, const std::vector & partitions); + std::shared_ptr client; - LRUCache table_meta_cache; + LRUCache table_metadata_cache; mutable std::mutex mutex; std::atomic expired{false}; diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 8ecb903c7f3..7d5b224a854 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -21,6 +21,11 @@ #include #include +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + namespace DB { @@ -114,11 +119,13 @@ Range HiveOrcFile::buildRange(const orc::ColumnStatistics * col_stats) void HiveOrcFile::prepareReader() { // TODO To be implemented + throw Exception("Unimplemented HiveOrcFile::prepareReader", ErrorCodes::NOT_IMPLEMENTED); } void HiveOrcFile::prepareColumnMapping() { // TODO To be implemented + throw Exception("Unimplemented HiveOrcFile::prepareColumnMapping", ErrorCodes::NOT_IMPLEMENTED); } bool HiveOrcFile::hasMinMaxIndex() const @@ -130,13 +137,14 @@ bool HiveOrcFile::hasMinMaxIndex() const std::unique_ptr HiveOrcFile::buildMinMaxIndex(const orc::Statistics * /*statistics*/) { // TODO To be implemented - return {}; + throw Exception("Unimplemented HiveOrcFile::buildMinMaxIndex", ErrorCodes::NOT_IMPLEMENTED); } void HiveOrcFile::loadMinMaxIndex() { // TODO To be implemented + throw Exception("Unimplemented HiveOrcFile::loadMinMaxIndex", ErrorCodes::NOT_IMPLEMENTED); } bool HiveOrcFile::hasSubMinMaxIndex() const @@ -148,23 +156,26 @@ bool HiveOrcFile::hasSubMinMaxIndex() const void HiveOrcFile::loadSubMinMaxIndex() { // TODO To be implemented + throw Exception("Unimplemented HiveOrcFile::loadSubMinMaxIndex", ErrorCodes::NOT_IMPLEMENTED); } bool HiveParquetFile::hasSubMinMaxIndex() const { // TODO To be implemented - return false; + throw Exception("Unimplemented HiveParquetFile::hasSubMinMaxIndex", ErrorCodes::NOT_IMPLEMENTED); } void HiveParquetFile::prepareReader() { // TODO To be implemented + throw Exception("Unimplemented HiveParquetFile::prepareReader", ErrorCodes::NOT_IMPLEMENTED); } void HiveParquetFile::loadSubMinMaxIndex() { // TODO To be implemented + throw Exception("Unimplemented HiveParquetFile::loadSubMinMaxIndex", ErrorCodes::NOT_IMPLEMENTED); } } diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 817454bb790..8579e7bb528 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -60,7 +60,7 @@ using HiveFiles = std::vector; static std::string getBaseName(const String & path) { size_t basename_start = path.rfind('/'); - return std::string{&path[basename_start + 1], path.length() - basename_start - 1}; + return path.substr(basename_start + 1); } class HiveSource : public SourceWithProgress, WithContext @@ -167,7 +167,7 @@ public: } } - // Use local cache for remote filesystem if enabled. + /// Use local cache for remote filesystem if enabled. std::unique_ptr remote_read_buf; if (RemoteReadBufferCache::instance().isInitialized() && getContext()->getSettingsRef().use_local_cache_for_remote_fs) remote_read_buf = RemoteReadBuffer::create(getContext(), @@ -298,10 +298,10 @@ StorageHive::StorageHive( format_name = "HiveText"; break; case FileFormat::RC_FILE: - // TODO to be implemented + /// TODO to be implemented throw Exception("Unsopported hive format rc_file", ErrorCodes::NOT_IMPLEMENTED); case FileFormat::SEQUENCE_FILE: - // TODO to be implemented + /// TODO to be implemented throw Exception("Unsopported hive format sequence_file", ErrorCodes::NOT_IMPLEMENTED); case FileFormat::AVRO: format_name = "Avro"; @@ -314,7 +314,7 @@ StorageHive::StorageHive( break; } - // Need to specify text_input_fields_names from table_schema for TextInputFormated Hive table + /// Need to specify text_input_fields_names from table_schema for TextInputFormated Hive table if (format_name == "HiveText") { size_t i = 0; @@ -421,19 +421,21 @@ Pipe StorageHive::read( auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url, getContext()); auto hive_table_metadata = hive_metastore_client->getTableMetadata(hive_database, hive_table); - // List files under partition directory in HDFS - auto list_paths = [hive_table_metadata, &fs](const String & path) { return hive_table_metadata->getLocationFiles(fs, path); }; + /// List files under partition directory in HDFS + auto list_paths = [hive_table_metadata, &fs](const String & path) { return hive_table_metadata->getFilesByLocation(fs, path); }; std::vector partitions = hive_table_metadata->getPartitions(); - HiveFiles hive_files; // hive files to read - std::mutex hive_files_mutex; // Mutext to protect hive_files, which maybe appended in multiple threads + /// Hive files to read + HiveFiles hive_files; + /// Mutext to protect hive_files, which maybe appended in multiple threads + std::mutex hive_files_mutex; auto append_hive_files = [&](const HiveMetastoreClient::FileInfo & hfile, const FieldVector & fields) { - LOG_TRACE(log, "append hive file:{}", hfile.path); + LOG_TRACE(log, "Append hive file {}", hfile.path); String filename = getBaseName(hfile.path); - // Skip temporary files starts with '.' + /// Skip temporary files starts with '.' if (filename.find('.') == 0) return; @@ -448,7 +450,7 @@ Pipe StorageHive::read( storage_settings, context_); - // Load file level minmax index and apply + /// Load file level minmax index and apply const KeyCondition hivefile_key_condition(query_info, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); if (file->hasMinMaxIndex()) { @@ -456,12 +458,12 @@ Pipe StorageHive::read( if (!hivefile_key_condition.checkInHyperrectangle(file->getMinMaxIndex()->hyperrectangle, hivefile_name_types.getTypes()) .can_be_true) { - LOG_DEBUG(log, "skip file:{} index:{}", file->getPath(), file->describeMinMaxIndex(file->getMinMaxIndex())); + LOG_TRACE(log, "Skip hive file {} by index {}", file->getPath(), file->describeMinMaxIndex(file->getMinMaxIndex())); return; } } - // Load sub-file level minmax index and apply + /// Load sub-file level minmax index and apply std::set skip_splits; if (file->hasSubMinMaxIndex()) { @@ -472,7 +474,7 @@ Pipe StorageHive::read( if (!hivefile_key_condition.checkInHyperrectangle(sub_minmax_idxes[i]->hyperrectangle, hivefile_name_types.getTypes()) .can_be_true) { - LOG_DEBUG(log, "skip split:{} in file {}", i, file->getPath()); + LOG_TRACE(log, "Skip split {} of hive file {}", i, file->getPath()); skip_splits.insert(i); } } @@ -491,13 +493,15 @@ Pipe StorageHive::read( const auto partition_names = partition_name_types.getNames(); const auto partition_types = partition_name_types.getTypes(); - for (const auto & p : partitions) + for (const auto & partition : partitions) { - auto f = [&]() + auto process_partition = [&]() { - // Skip partition "__HIVE_DEFAULT_PARTITION__" + LOG_DEBUG(log, "Process partition {}", boost::join(partition.values, ",")); + + /// Skip partition "__HIVE_DEFAULT_PARTITION__" bool has_default_partition = false; - for (const auto & value : p.values) + for (const auto & value : partition.values) { if (value == "__HIVE_DEFAULT_PARTITION__") { @@ -507,22 +511,21 @@ Pipe StorageHive::read( } if (has_default_partition) { - //LOG_DEBUG(log, "skip partition:__HIVE_DEFAULT_PARTITION__"); return; } std::vector ranges; WriteBufferFromOwnString wb; - if (p.values.size() != partition_names.size()) + if (partition.values.size() != partition_names.size()) throw Exception( - fmt::format("Partition value size not match, expect {}, but got {}", partition_names.size(), p.values.size()), + fmt::format("Partition value size not match, expect {}, but got {}", partition_names.size(), partition.values.size()), ErrorCodes::INVALID_PARTITION_VALUE); - for (size_t i = 0; i < p.values.size(); ++i) + for (size_t i = 0; i < partition.values.size(); ++i) { if (i != 0) writeString(",", wb); - writeString(p.values[i], wb); + writeString(partition.values[i], wb); } writeString("\n", wb); @@ -545,22 +548,20 @@ Pipe StorageHive::read( const KeyCondition partition_key_condition(query_info, getContext(), partition_names, partition_minmax_idx_expr); if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true) { - //LOG_DEBUG(log, "skip partition:{}", boost::algorithm::join(p.values, "|")); return; } - LOG_TRACE(log, "list location:{}", p.sd.location); - auto paths = list_paths(p.sd.location); + auto paths = list_paths(partition.sd.location); for (const auto & path : paths) { append_hive_files(path, fields); } }; - pool.scheduleOrThrowOnError(f); + pool.scheduleOrThrowOnError(process_partition); } pool.wait(); } - else if (partition_name_types.empty()) // Partition keys is empty + else if (partition_name_types.empty()) /// Partition keys is empty { auto paths = list_paths(hive_table_metadata->getTable()->sd.location); for (const auto & path : paths) @@ -569,7 +570,7 @@ Pipe StorageHive::read( } pool.wait(); } - else // Partition keys is not empty but partitions is empty + else /// Partition keys is not empty but partitions is empty { return {}; } From a0b90fa9ec15fc58f7ee8019fc5a388f00cc725a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 14 Dec 2021 18:44:15 +0800 Subject: [PATCH 0179/1260] fix bugs and add some docs --- .../table-engines/integrations/hive.md | 317 ++++++++++++++++-- src/Storages/Hive/HiveFile.cpp | 3 +- 2 files changed, 286 insertions(+), 34 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/hive.md b/docs/en/engines/table-engines/integrations/hive.md index ce3828555e7..b804b9c2279 100644 --- a/docs/en/engines/table-engines/integrations/hive.md +++ b/docs/en/engines/table-engines/integrations/hive.md @@ -41,8 +41,29 @@ The table structure can differ from the original Hive table structure: ## Usage Example {#usage-example} -Table in Hive: +### How to Use Local Cache for HDFS Filesystem +We strongly advice you to enable local cache for remote filesystems. Benchmark shows that its almost 2x faster with cache. +Before using cache, add it to `config.xml` +``` xml + + true + local_cache + 559096952 + 1048576 + +``` + +- enable: ClickHouse will maintain local cache for remote filesystem(HDFS) after startup if true. +- root_dir: Required. The root directory to store local cache files for remote filesystem. +- limit_size: Required. The maximum size(in bytes) of local cache files. +- bytes_read_before_flush: Control bytes before flush to local filesystem when downloading file from remote filesystem. The default value is 1MB. + +When ClickHouse is started up with local cache for remote filesystem enabled, users can still choose not to use cache with `settings use_local_cache_for_remote_fs = 0` in their query. `use_local_cache_for_remote_fs` is `false` in default. + +### Query Hive Table with ORC Input Format + +#### Create Table in Hive ``` text hive > CREATE TABLE `test`.`test_orc`( `f_tinyint` tinyint, @@ -75,8 +96,7 @@ OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat' LOCATION 'hdfs://testcluster/data/hive/test.db/test_orc' -TBLPROPERTIES ( - 'transient_lastDdlTime'='1631944221') + OK Time taken: 0.51 seconds @@ -90,42 +110,46 @@ OK Time taken: 0.295 seconds, Fetched: 1 row(s) ``` +#### Create Table in ClickHouse Table in ClickHouse, retrieving data from the Hive table created above: ``` sql -CREATE TABLE `test`.`test_orc`( - `f_tinyint` Int8, - `f_smallint` Int16, - `f_int` Int32, - `f_integer` Int32, - `f_bigint` Int64, - `f_float` Float32, - `f_double` Float64, - `f_decimal` Float64, - `f_timestamp` DateTime, - `f_date` Date, - `f_string` String, - `f_varchar` String, - `f_bool` UInt8, - `f_binary` String, - `f_array_int` Array(Int32), - `f_array_string` Array(String), - `f_array_float` Array(Float32), - `f_array_array_int` Array(Array(Int32)), - `f_array_array_string` Array(Array(String)), - `f_array_array_float` Array(Array(Float32)), - day String -) -ENGINE = Hive('thrift://localhost:9083', 'test', 'test_orc') -PARTITION BY day +CREATE TABLE test.test_orc +( + `f_tinyint` Int8, + `f_smallint` Int16, + `f_int` Int32, + `f_integer` Int32, + `f_bigint` Int64, + `f_float` Float32, + `f_double` Float64, + `f_decimal` Float64, + `f_timestamp` DateTime, + `f_date` Date, + `f_string` String, + `f_varchar` String, + `f_bool` Bool, + `f_binary` String, + `f_array_int` Array(Int32), + `f_array_string` Array(String), + `f_array_float` Array(Float32), + `f_array_array_int` Array(Array(Int32)), + `f_array_array_string` Array(Array(String)), + `f_array_array_float` Array(Array(Float32)), + `day` String +) +ENGINE = Hive('thrift://202.168.117.26:9083', 'test', 'test_orc') +PARTITION BY day + ``` ``` sql -SELECT * FROM test.test_orc \G +SELECT * FROM test.test_orc settings input_format_orc_allow_missing_columns = 1\G ``` ``` text SELECT * -FROM test_orc +FROM test.test_orc +SETTINGS input_format_orc_allow_missing_columns = 1 Query id: c3eaffdc-78ab-43cd-96a4-4acc5b480658 @@ -139,11 +163,11 @@ f_bigint: 5 f_float: 6.11 f_double: 7.22 f_decimal: 8 -f_timestamp: 2021-11-05 20:38:16 -f_date: 2021-11-05 +f_timestamp: 2021-12-04 04:00:44 +f_date: 2021-12-03 f_string: hello world f_varchar: hello world -f_bool: 1 +f_bool: true f_binary: hello world f_array_int: [1,2,3] f_array_string: ['hello world','hello world'] @@ -153,5 +177,232 @@ f_array_array_string: [['a','b'],['c','d']] f_array_array_float: [[1.11,2.22],[3.33,4.44]] day: 2021-09-18 + 1 rows in set. Elapsed: 0.078 sec. ``` + +### Query Hive Table with Parquet Input Format + +#### Create Table in Hive +``` text +hive > +CREATE TABLE `test`.`test_parquet`( + `f_tinyint` tinyint, + `f_smallint` smallint, + `f_int` int, + `f_integer` int, + `f_bigint` bigint, + `f_float` float, + `f_double` double, + `f_decimal` decimal(10,0), + `f_timestamp` timestamp, + `f_date` date, + `f_string` string, + `f_varchar` varchar(100), + `f_char` char(100), + `f_bool` boolean, + `f_binary` binary, + `f_array_int` array, + `f_array_string` array, + `f_array_float` array, + `f_array_array_int` array>, + `f_array_array_string` array>, + `f_array_array_float` array>) +PARTITIONED BY ( + `day` string) +ROW FORMAT SERDE + 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' +STORED AS INPUTFORMAT + 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' +LOCATION + 'hdfs://testcluster/data/hive/test.db/test_parquet' +OK +Time taken: 0.51 seconds + +hive > insert into test.test_parquet partition(day='2021-09-18') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, current_timestamp(), current_date(), 'hello world', 'hello world', 'hello world', true, 'hello world', array(1, 2, 3), array('hello world', 'hello world'), array(float(1.1), float(1.2)), array(array(1, 2), array(3, 4)), array(array('a', 'b'), array('c', 'd')), array(array(float(1.11), float(2.22)), array(float(3.33), float(4.44))); +OK +Time taken: 36.025 seconds + +hive > select * from test.test_parquet; +OK +1 2 3 4 5 6.11 7.22 8 2021-12-14 17:54:56.743 2021-12-14 hello world hello world hello world true hello world [1,2,3] ["hello world","hello world"] [1.1,1.2] [[1,2],[3,4]] [["a","b"],["c","d"]] [[1.11,2.22],[3.33,4.44]] 2021-09-18 +Time taken: 0.766 seconds, Fetched: 1 row(s) +``` + +#### Create Table in ClickHouse +Table in ClickHouse, retrieving data from the Hive table created above: +``` sql +CREATE TABLE test.test_parquet +( + `f_tinyint` Int8, + `f_smallint` Int16, + `f_int` Int32, + `f_integer` Int32, + `f_bigint` Int64, + `f_float` Float32, + `f_double` Float64, + `f_decimal` Float64, + `f_timestamp` DateTime, + `f_date` Date, + `f_string` String, + `f_varchar` String, + `f_char` String, + `f_bool` Bool, + `f_binary` String, + `f_array_int` Array(Int32), + `f_array_string` Array(String), + `f_array_float` Array(Float32), + `f_array_array_int` Array(Array(Int32)), + `f_array_array_string` Array(Array(String)), + `f_array_array_float` Array(Array(Float32)), + `day` String +) +ENGINE = Hive('thrift://localhost:9083', 'test', 'test_parquet') +PARTITION BY day +``` + +``` sql +SELECT * FROM test.test_parquet settings input_format_parquet_allow_missing_columns = 1\G +``` + +``` text +SELECT * +FROM test_parquet +SETTINGS input_format_parquet_allow_missing_columns = 1 + +Query id: 4e35cf02-c7b2-430d-9b81-16f438e5fca9 + +Row 1: +────── +f_tinyint: 1 +f_smallint: 2 +f_int: 3 +f_integer: 4 +f_bigint: 5 +f_float: 6.11 +f_double: 7.22 +f_decimal: 8 +f_timestamp: 2021-12-14 17:54:56 +f_date: 2021-12-14 +f_string: hello world +f_varchar: hello world +f_char: hello world +f_bool: true +f_binary: hello world +f_array_int: [1,2,3] +f_array_string: ['hello world','hello world'] +f_array_float: [1.1,1.2] +f_array_array_int: [[1,2],[3,4]] +f_array_array_string: [['a','b'],['c','d']] +f_array_array_float: [[1.11,2.22],[3.33,4.44]] +day: 2021-09-18 + +1 rows in set. Elapsed: 0.357 sec. +``` + +### Query Hive Table with Text Input Format +#### Create Table in Hive +``` text +hive > +CREATE TABLE `test`.`test_text`( + `f_tinyint` tinyint, + `f_smallint` smallint, + `f_int` int, + `f_integer` int, + `f_bigint` bigint, + `f_float` float, + `f_double` double, + `f_decimal` decimal(10,0), + `f_timestamp` timestamp, + `f_date` date, + `f_string` string, + `f_varchar` varchar(100), + `f_char` char(100), + `f_bool` boolean, + `f_binary` binary, + `f_array_int` array, + `f_array_string` array, + `f_array_float` array, + `f_array_array_int` array>, + `f_array_array_string` array>, + `f_array_array_float` array>) +PARTITIONED BY ( + `day` string) +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +STORED AS INPUTFORMAT + 'org.apache.hadoop.mapred.TextInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +LOCATION + 'hdfs://testcluster/data/hive/test.db/test_text' +Time taken: 0.1 seconds, Fetched: 34 row(s) + + +hive > insert into test.test_text partition(day='2021-09-18') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, current_timestamp(), current_date(), 'hello world', 'hello world', 'hello world', true, 'hello world', array(1, 2, 3), array('hello world', 'hello world'), array(float(1.1), float(1.2)), array(array(1, 2), array(3, 4)), array(array('a', 'b'), array('c', 'd')), array(array(float(1.11), float(2.22)), array(float(3.33), float(4.44))); +OK +Time taken: 36.025 seconds + +hive > select * from test.test_text; +OK +1 2 3 4 5 6.11 7.22 8 2021-12-14 18:11:17.239 2021-12-14 hello world hello world hello world true hello world [1,2,3] ["hello world","hello world"] [1.1,1.2] [[1,2],[3,4]] [["a","b"],["c","d"]] [[1.11,2.22],[3.33,4.44]] 2021-09-18 +Time taken: 0.624 seconds, Fetched: 1 row(s) +``` + +#### Create Table in ClickHouse + +Table in ClickHouse, retrieving data from the Hive table created above: +``` sql +CREATE TABLE test.test_text +( + `f_tinyint` Int8, + `f_smallint` Int16, + `f_int` Int32, + `f_integer` Int32, + `f_bigint` Int64, + `f_float` Float32, + `f_double` Float64, + `f_decimal` Float64, + `f_timestamp` DateTime, + `f_date` Date, + `f_string` String, + `f_varchar` String, + `f_char` String, + `f_bool` Bool, + `day` String +) +ENGINE = Hive('thrift://localhost:9083', 'test', 'test_text') +PARTITION BY day +``` + +``` sql +SELECT * FROM test.test_text settings input_format_skip_unknown_fields = 1, input_format_with_names_use_header = 1, date_time_input_format = 'best_effort'\G +``` + +``` text +SELECT * +FROM test.test_text +SETTINGS input_format_skip_unknown_fields = 1, input_format_with_names_use_header = 1, date_time_input_format = 'best_effort' + +Query id: 55b79d35-56de-45b9-8be6-57282fbf1f44 + +Row 1: +────── +f_tinyint: 1 +f_smallint: 2 +f_int: 3 +f_integer: 4 +f_bigint: 5 +f_float: 6.11 +f_double: 7.22 +f_decimal: 8 +f_timestamp: 2021-12-14 18:11:17 +f_date: 2021-12-14 +f_string: hello world +f_varchar: hello world +f_char: hello world +f_bool: true +day: 2021-09-18 +``` diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 7d5b224a854..c2ab6b4593b 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -149,6 +149,7 @@ void HiveOrcFile::loadMinMaxIndex() bool HiveOrcFile::hasSubMinMaxIndex() const { + // TODO To be implemented return false; } @@ -162,7 +163,7 @@ void HiveOrcFile::loadSubMinMaxIndex() bool HiveParquetFile::hasSubMinMaxIndex() const { // TODO To be implemented - throw Exception("Unimplemented HiveParquetFile::hasSubMinMaxIndex", ErrorCodes::NOT_IMPLEMENTED); + return false; } void HiveParquetFile::prepareReader() From b0638b04484560bedce18b455851c87c47432976 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 14 Dec 2021 13:47:35 +0100 Subject: [PATCH 0180/1260] Cherry-pick other previous changes to boost libs --- contrib/boost | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boost b/contrib/boost index 65112f83c56..ccc2047cc75 160000 --- a/contrib/boost +++ b/contrib/boost @@ -1 +1 @@ -Subproject commit 65112f83c5657de9c06bc838071e7229eab60821 +Subproject commit ccc2047cc754dcf02ee03667425e90f2e9b8e293 From c90bfaf78c8ff742f11c24f7715c386ebeaee19b Mon Sep 17 00:00:00 2001 From: gulige Date: Wed, 15 Dec 2021 10:23:34 +0800 Subject: [PATCH 0181/1260] code refactoring as @kitaisreal suggested Better to not use emplace with std::npos, just find element, if it already exists then checkColumnStructure, after that we can safely update index_by_name and insert element. --- src/Core/Block.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 3208aefcdab..319bb4202ac 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -139,15 +139,15 @@ void Block::insert(size_t position, ColumnWithTypeAndName elem) if (elem.name.empty()) throw Exception("Column name in Block cannot be empty", ErrorCodes::AMBIGUOUS_COLUMN_NAME); - auto [it, inserted] = index_by_name.emplace(elem.name, std::npos); - if (!inserted) + auto it = index_by_name.find(elem.name); + if (index_by_name.end() != it) + { checkColumnStructure(data[it->second], elem, "(columns with identical name must have identical structure)", true, ErrorCodes::AMBIGUOUS_COLUMN_NAME); + } for (auto & name_pos : index_by_name) - if (name_pos.second == std::npos) - name_pos.second = position; - else if (name_pos.second >= position) + if (name_pos.second >= position) ++name_pos.second; data.emplace(data.begin() + position, std::move(elem)); From c0aa1c340557a68832b3879c1aed05889a164022 Mon Sep 17 00:00:00 2001 From: gulige Date: Wed, 15 Dec 2021 10:58:58 +0800 Subject: [PATCH 0182/1260] missing code refactoring as @kitaisreal suggested --- src/Core/Block.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 319bb4202ac..5a586e25534 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -145,6 +145,10 @@ void Block::insert(size_t position, ColumnWithTypeAndName elem) checkColumnStructure(data[it->second], elem, "(columns with identical name must have identical structure)", true, ErrorCodes::AMBIGUOUS_COLUMN_NAME); } + else + { + index_by_name.emplace(elem.name, position); + } for (auto & name_pos : index_by_name) if (name_pos.second >= position) From 574bee9040a80c233124a4c7fd1501449871bae0 Mon Sep 17 00:00:00 2001 From: gulige Date: Wed, 15 Dec 2021 11:48:12 +0800 Subject: [PATCH 0183/1260] fix code refactoring as @kitaisreal suggested --- src/Core/Block.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 5a586e25534..e9993f0bd46 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -145,15 +145,13 @@ void Block::insert(size_t position, ColumnWithTypeAndName elem) checkColumnStructure(data[it->second], elem, "(columns with identical name must have identical structure)", true, ErrorCodes::AMBIGUOUS_COLUMN_NAME); } - else - { - index_by_name.emplace(elem.name, position); - } for (auto & name_pos : index_by_name) if (name_pos.second >= position) ++name_pos.second; + index_by_name.emplace(elem.name, position); + data.emplace(data.begin() + position, std::move(elem)); } From f600b9f525e6d987b3bb29763f1f94cb195cd1ba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 15 Dec 2021 10:57:24 +0100 Subject: [PATCH 0184/1260] Cherry-pick old fix for m1 --- contrib/boost | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boost b/contrib/boost index ccc2047cc75..79b2fdaa512 160000 --- a/contrib/boost +++ b/contrib/boost @@ -1 +1 @@ -Subproject commit ccc2047cc754dcf02ee03667425e90f2e9b8e293 +Subproject commit 79b2fdaa512fde9683ecbaaf1bb86601f73d1135 From fce807b2490b81ca6702e455378fa88fc832592e Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 17 Dec 2021 23:09:26 +0300 Subject: [PATCH 0185/1260] DOCSUP-17856: [CLICKHOUSEDOCS] Document the merge_tree_min_rows_for_concurrent_read_for_remote_filesystem and merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem setting --- docs/en/operations/settings/settings.md | 28 +++++++++++++++++++++---- docs/ru/operations/settings/settings.md | 28 ++++++++++++++++++++++--- 2 files changed, 49 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e56625fe948..a3f326d2e50 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -817,9 +817,19 @@ If the number of rows to be read from a file of a [MergeTree](../../engines/tabl Possible values: -- Any positive integer. +- Positive integer. -Default value: 163840. +Default value: `163840`. + +## merge_tree_min_rows_for_concurrent_read_for_remote_filesystem {#merge-tree-min-rows-for-concurrent-read-for-remote-filesystem} + +If at least as many lines are read from one file, the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) engine can reads parallelized, when reading from remote filesystem. + +Possible values: + +- Positive integer. + +Default value: `163840`. ## merge_tree_min_bytes_for_concurrent_read {#setting-merge-tree-min-bytes-for-concurrent-read} @@ -827,9 +837,19 @@ If the number of bytes to read from one file of a [MergeTree](../../engines/tabl Possible value: -- Any positive integer. +- Positive integer. -Default value: 251658240. +Default value: `251658240`. + +## merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem {#merge-tree-min-bytes-for-concurrent-read-for-remote-filesystem} + +If at least as many bytes are read from one file, the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) engine can reads parallelized, when reading from remote filesystem. + +Possible values: + +- Positive integer. + +Default value: `251658240`. ## merge_tree_min_rows_for_seek {#setting-merge-tree-min-rows-for-seek} diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 1b4da512c9f..6a6474d03e9 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -739,9 +739,20 @@ ClickHouse может парсить только базовый формат `Y Возможные значения: -- Любое положительное целое число. +- Положительное целое число. -Значение по умолчанию: 163840. +Значение по умолчанию: `163840`. + + +## merge_tree_min_rows_for_concurrent_read_for_remote_filesystem {#merge-tree-min-rows-for-concurrent-read-for-remote-filesystem} + +Если из одного файла считывается указанное количество строк, движок [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) может выполнять параллельное чтение из удаленной файловой системы. + +Возможные значения: + +- Положительное целое число. + +Значение по умолчанию: `163840`. ## merge_tree_min_bytes_for_concurrent_read {#setting-merge-tree-min-bytes-for-concurrent-read} @@ -751,7 +762,18 @@ ClickHouse может парсить только базовый формат `Y - Положительное целое число. -Значение по умолчанию: 251658240. +Значение по умолчанию: `251658240`. + + +## merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem {#merge-tree-min-bytes-for-concurrent-read-for-remote-filesystem} + +Если из одного файла считывается указанное число байтов, движок [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) может выполнять параллельное чтение из удаленной файловой системы. + +Возможное значение: + +- Положительное целое число. + +Значение по умолчанию: `251658240`. ## merge_tree_min_rows_for_seek {#setting-merge-tree-min-rows-for-seek} From 305fa316fac58d16bca86f9bf60336281893ee2b Mon Sep 17 00:00:00 2001 From: Alexey Date: Sat, 18 Dec 2021 08:06:47 +0000 Subject: [PATCH 0186/1260] en draft --- .../settings.md | 34 +++++++++++++++++-- 1 file changed, 31 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 717ab4e14b7..462bc078603 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -443,18 +443,46 @@ By default, the authentication is not used. This section contains the following parameters: -- `user` — username. -- `password` — password. +- `user` — Username. +- `password` — Password. +- `allow_empty` — Allows connection of replicas without authentication as well as authenticated replicas. Default value: `false`. +- `old` — Previous `user` and `password` used during credential rotation. Several `old` sections can be specified. -**Example** +**Credential Rotation** + +ClickHouse supports dynamic interserver credential rotation without the need of shutting down all replicas to update their configuration at once. Credentials can be changed in several steps. + +To enable authentication if it was not used before, set `interserver_http_credentials.allow_empty` to `true` and add credentials. This allows connections with authentication and without it. By default, `allow_empty` is set to `false` and unauthenticaded connections are refused if any credentials are specified in the configuration. + +``` xml + + admin + 111 + true + +``` + +Then set `allow_empty` to `false` or remove this setting to require authentication with new credentials. + +To change credentials, move previous user name and password to `interserver_http_credentials.old` section and update `username` and `password` with new values. At this step a server uses new credentials to connect to other replicas and accepts connections with new and old credentials. ``` xml admin 222 + + admin + 111 + + + temp + 000 + ``` +When new credentials are applied to all replicas, old credentials may be removed. + ## keep_alive_timeout {#keep-alive-timeout} The number of seconds that ClickHouse waits for incoming requests before closing the connection. Defaults to 10 seconds. From cf4d41cf799d90901383e2ead3409e5e4d78a190 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Sat, 18 Dec 2021 17:52:44 +0300 Subject: [PATCH 0187/1260] Update settings.md --- docs/ru/operations/settings/settings.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 6a6474d03e9..03d0aee4fef 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -764,7 +764,6 @@ ClickHouse может парсить только базовый формат `Y Значение по умолчанию: `251658240`. - ## merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem {#merge-tree-min-bytes-for-concurrent-read-for-remote-filesystem} Если из одного файла считывается указанное число байтов, движок [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) может выполнять параллельное чтение из удаленной файловой системы. From f13c018c50f7b124ef3ad9057b724af97e3f2610 Mon Sep 17 00:00:00 2001 From: Alexey Date: Sat, 18 Dec 2021 18:25:42 +0000 Subject: [PATCH 0188/1260] en updated --- .../server-configuration-parameters/settings.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 462bc078603..b2fadc5d702 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -443,16 +443,16 @@ By default, the authentication is not used. This section contains the following parameters: -- `user` — Username. +- `user` — User name. - `password` — Password. -- `allow_empty` — Allows connection of replicas without authentication as well as authenticated replicas. Default value: `false`. -- `old` — Previous `user` and `password` used during credential rotation. Several `old` sections can be specified. +- `allow_empty` — If `true`, then other replicas are allowed to connect without authentication even if credentials are set. If `allow_empty` is `false`, then connections without authentication are refused. Default value: `false`. +- `old` — Contains old `user` and `password` used during credential rotation. Several `old` sections can be specified. **Credential Rotation** -ClickHouse supports dynamic interserver credential rotation without the need of shutting down all replicas to update their configuration at once. Credentials can be changed in several steps. +ClickHouse supports dynamic interserver credential rotation without the need of shutting down all replicas at the same time to update their configuration. Credentials can be changed in several steps. -To enable authentication if it was not used before, set `interserver_http_credentials.allow_empty` to `true` and add credentials. This allows connections with authentication and without it. By default, `allow_empty` is set to `false` and unauthenticaded connections are refused if any credentials are specified in the configuration. +To enable authentication, set `interserver_http_credentials.allow_empty` to `true` and add credentials. This allows connections with authentication and without it. ``` xml @@ -462,9 +462,9 @@ To enable authentication if it was not used before, set `interserver_http_creden ``` -Then set `allow_empty` to `false` or remove this setting to require authentication with new credentials. +After configuring all replicas set `allow_empty` to `false` or remove this setting. It makes authentication with new credentials mandatory. -To change credentials, move previous user name and password to `interserver_http_credentials.old` section and update `username` and `password` with new values. At this step a server uses new credentials to connect to other replicas and accepts connections with new and old credentials. +To change existing credentials, move previous user name and password to `interserver_http_credentials.old` section and update `user` and `password` with new values. At this step a server uses new credentials to connect to other replicas and accepts connections with new or old credentials. ``` xml From cfed9b9c599a719106a852b3c57a293de8ae59d4 Mon Sep 17 00:00:00 2001 From: Alexey Date: Sat, 18 Dec 2021 19:00:32 +0000 Subject: [PATCH 0189/1260] updates --- .../en/operations/server-configuration-parameters/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index b2fadc5d702..39dbf5c2e66 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -450,7 +450,7 @@ This section contains the following parameters: **Credential Rotation** -ClickHouse supports dynamic interserver credential rotation without the need of shutting down all replicas at the same time to update their configuration. Credentials can be changed in several steps. +ClickHouse supports dynamic interserver credential rotation without the need of stopping all replicas at the same time to update their configuration. Credentials can be changed in several steps. To enable authentication, set `interserver_http_credentials.allow_empty` to `true` and add credentials. This allows connections with authentication and without it. @@ -464,7 +464,7 @@ To enable authentication, set `interserver_http_credentials.allow_empty` to `tru After configuring all replicas set `allow_empty` to `false` or remove this setting. It makes authentication with new credentials mandatory. -To change existing credentials, move previous user name and password to `interserver_http_credentials.old` section and update `user` and `password` with new values. At this step a server uses new credentials to connect to other replicas and accepts connections with new or old credentials. +To change existing credentials, move the user name and the password to `interserver_http_credentials.old` section and update `user` and `password` with new values. At this step a server uses new credentials to connect to other replicas and accepts connections with new or old credentials. ``` xml From 2cebda874da8087011a6e19c47c004657605ebd3 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 20 Dec 2021 09:26:32 +0800 Subject: [PATCH 0190/1260] remove debug modification --- utils/check-style/check-style | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 3b29f63d990..4816525c0e4 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -139,7 +139,7 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.xml' | xargs xmllint --noout --nonet # FIXME: for now only clickhouse-test -#pylint --rcfile=$ROOT_PATH/.pylintrc --persistent=no --score=n $ROOT_PATH/tests/clickhouse-test $ROOT_PATH/tests/ci/*.py +pylint --rcfile=$ROOT_PATH/.pylintrc --persistent=no --score=n $ROOT_PATH/tests/clickhouse-test $ROOT_PATH/tests/ci/*.py find $ROOT_PATH -not -path $ROOT_PATH'/contrib*' \( -name '*.yaml' -or -name '*.yml' \) -type f | grep -vP $EXCLUDE_DIRS | From 244a8b2cbf9bf939a3f7301abda7b0d984367fe4 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 20 Dec 2021 11:49:45 +0800 Subject: [PATCH 0191/1260] fixed typro --- tests/integration/test_hive_query/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index 456b2f645c0..bbb1b5f3c9a 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -92,7 +92,7 @@ def test_text_count(started_cluster): """ assert result == expected_result -def test_parquet_groupby_witch_cache(started_cluster): +def test_parquet_groupby_with_cache(started_cluster): logging.info('Start testing groupby ...') node = started_cluster.instances['h0_0_0'] result = node.query(""" From e9ab9d5b59d885c104c1112fd49a381b638bd952 Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 20 Dec 2021 08:25:42 +0300 Subject: [PATCH 0192/1260] Update docs/en/operations/server-configuration-parameters/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 39dbf5c2e66..d4d6b8f7d06 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -445,7 +445,7 @@ This section contains the following parameters: - `user` — User name. - `password` — Password. -- `allow_empty` — If `true`, then other replicas are allowed to connect without authentication even if credentials are set. If `allow_empty` is `false`, then connections without authentication are refused. Default value: `false`. +- `allow_empty` — If `true`, then other replicas are allowed to connect without authentication even if credentials are set. If `false`, then connections without authentication are refused. Default value: `false`. - `old` — Contains old `user` and `password` used during credential rotation. Several `old` sections can be specified. **Credential Rotation** From 8cc3fd6310f76c896da76e81b1fbcb7a4411b87e Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 20 Dec 2021 08:26:09 +0300 Subject: [PATCH 0193/1260] Update docs/en/operations/server-configuration-parameters/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index d4d6b8f7d06..c7da6ca939d 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -448,7 +448,7 @@ This section contains the following parameters: - `allow_empty` — If `true`, then other replicas are allowed to connect without authentication even if credentials are set. If `false`, then connections without authentication are refused. Default value: `false`. - `old` — Contains old `user` and `password` used during credential rotation. Several `old` sections can be specified. -**Credential Rotation** +**Credentials Rotation** ClickHouse supports dynamic interserver credential rotation without the need of stopping all replicas at the same time to update their configuration. Credentials can be changed in several steps. From 87b607d16b846211dda186019574e85f95cd3b38 Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 20 Dec 2021 08:26:33 +0300 Subject: [PATCH 0194/1260] Update docs/en/operations/server-configuration-parameters/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index c7da6ca939d..b2d17d76311 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -450,7 +450,7 @@ This section contains the following parameters: **Credentials Rotation** -ClickHouse supports dynamic interserver credential rotation without the need of stopping all replicas at the same time to update their configuration. Credentials can be changed in several steps. +ClickHouse supports dynamic interserver credentials rotation without stopping all replicas at the same time to update their configuration. Credentials can be changed in several steps. To enable authentication, set `interserver_http_credentials.allow_empty` to `true` and add credentials. This allows connections with authentication and without it. From 708fea2bbee413289cc6b31f0048a6c1de497648 Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 20 Dec 2021 08:27:25 +0300 Subject: [PATCH 0195/1260] Update docs/en/operations/server-configuration-parameters/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index b2d17d76311..b728a0303f6 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -464,7 +464,7 @@ To enable authentication, set `interserver_http_credentials.allow_empty` to `tru After configuring all replicas set `allow_empty` to `false` or remove this setting. It makes authentication with new credentials mandatory. -To change existing credentials, move the user name and the password to `interserver_http_credentials.old` section and update `user` and `password` with new values. At this step a server uses new credentials to connect to other replicas and accepts connections with new or old credentials. +To change existing credentials, move the user name and the password to `interserver_http_credentials.old` section and update `user` and `password` with new values. At this point the server uses new credentials to connect to other replicas and accepts connections with either new or old credentials. ``` xml From 71924586014f3006f371528f4b97dc3fbeab863d Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 20 Dec 2021 18:10:15 +0800 Subject: [PATCH 0196/1260] fix code typo --- src/Core/Settings.h | 2 +- src/Storages/Hive/HiveCommon.cpp | 11 +++++------ src/Storages/Hive/HiveCommon.h | 7 +++++-- src/Storages/Hive/StorageHive.cpp | 2 +- 4 files changed, 12 insertions(+), 10 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6166f6f6a6a..08bdbf3d248 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -540,7 +540,7 @@ class IColumn; \ M(Bool, force_remove_data_recursively_on_drop, false, "Recursively remove data on DROP query. Avoids 'Directory not empty' error, but may silently remove detached data", 0) \ M(Bool, check_table_dependencies, true, "Check that DDL query (such as DROP TABLE or RENAME) will not break dependencies", 0) \ - M(Bool, use_local_cache_for_remote_fs, true, "Use local cache for remote filesystem like HDFS or S3", 0) \ + M(Bool, use_local_cache_for_remote_storage, true, "Use local cache for remote storage like HDFS or S3, it's used for remote table engine only", 0) \ \ /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index a47c4a5a87b..68aba13b726 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -17,7 +17,7 @@ bool HiveMetastoreClient::shouldUpdateTableMetadata( const String & db_name, const String & table_name, const std::vector & partitions) { String cache_key = getCacheKey(db_name, table_name); - std::shared_ptr metadata = table_metadata_cache.get(cache_key); + HiveTableMetadataPtr metadata = table_metadata_cache.get(cache_key); if (!metadata) return true; @@ -38,8 +38,7 @@ bool HiveMetastoreClient::shouldUpdateTableMetadata( return false; } -std::shared_ptr -HiveMetastoreClient::getTableMetadata(const String & db_name, const String & table_name) +HiveMetastoreClient::HiveTableMetadataPtr HiveMetastoreClient::getTableMetadata(const String & db_name, const String & table_name) { LOG_TRACE(log, "Get table metadata for {}.{}", db_name, table_name); std::lock_guard lock{mutex}; @@ -60,9 +59,9 @@ HiveMetastoreClient::getTableMetadata(const String & db_name, const String & tab } bool update_cache = shouldUpdateTableMetadata(db_name, table_name, partitions); - String cache_key = db_name + "." + table_name; + String cache_key = getCacheKey(db_name, table_name); - std::shared_ptr metadata = table_metadata_cache.get(cache_key); + HiveTableMetadataPtr metadata = table_metadata_cache.get(cache_key); if (update_cache) { @@ -107,7 +106,7 @@ void HiveMetastoreClient::clearTableMetadata(const String & db_name, const Strin String cache_key = getCacheKey(db_name, table_name); std::lock_guard lock{mutex}; - std::shared_ptr metadata = table_metadata_cache.get(cache_key); + HiveTableMetadataPtr metadata = table_metadata_cache.get(cache_key); if (metadata) table_metadata_cache.set(cache_key, nullptr); } diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index ea58f4d7230..ee222f82877 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -14,9 +14,11 @@ namespace DB { + class HiveMetastoreClient : public WithContext { public: + struct FileInfo { String path; @@ -89,12 +91,14 @@ public: Poco::Logger * log = &Poco::Logger::get("HiveMetastoreClient"); }; + using HiveTableMetadataPtr = std::shared_ptr; + explicit HiveMetastoreClient(std::shared_ptr client_, ContextPtr context_) : WithContext(context_), client(client_), table_metadata_cache(1000) { } - std::shared_ptr getTableMetadata(const String & db_name, const String & table_name); + HiveTableMetadataPtr getTableMetadata(const String & db_name, const String & table_name); void clearTableMetadata(const String & db_name, const String & table_name); void setClient(std::shared_ptr client_); bool isExpired() const { return expired; } @@ -116,7 +120,6 @@ private: }; using HiveMetastoreClientPtr = std::shared_ptr; - class HiveMetastoreClientFactory final : private boost::noncopyable { public: diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 8579e7bb528..db689cf9bcf 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -169,7 +169,7 @@ public: /// Use local cache for remote filesystem if enabled. std::unique_ptr remote_read_buf; - if (RemoteReadBufferCache::instance().isInitialized() && getContext()->getSettingsRef().use_local_cache_for_remote_fs) + if (RemoteReadBufferCache::instance().isInitialized() && getContext()->getSettingsRef().use_local_cache_for_remote_storage) remote_read_buf = RemoteReadBuffer::create(getContext(), std::make_shared("Hive", getNameNodeCluster(hdfs_namenode_url), uri_with_path, curr_file->getSize(), curr_file->getLastModTs()), std::move(raw_read_buf)); From 970466ea4ed787ff6702e7d315fbeb3a4dc35772 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 20 Dec 2021 18:30:40 +0800 Subject: [PATCH 0197/1260] revert irrelate contribs --- contrib/base64 | 2 +- contrib/libhdfs3 | 2 +- contrib/replxx | 2 +- contrib/sysroot | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/contrib/base64 b/contrib/base64 index af9b331f2b4..9499e0c4945 160000 --- a/contrib/base64 +++ b/contrib/base64 @@ -1 +1 @@ -Subproject commit af9b331f2b4f30b41c70f3a571ff904a8251c1d3 +Subproject commit 9499e0c4945589973b9ea1bc927377cfbc84aa46 diff --git a/contrib/libhdfs3 b/contrib/libhdfs3 index a8c37ee001a..9194af44588 160000 --- a/contrib/libhdfs3 +++ b/contrib/libhdfs3 @@ -1 +1 @@ -Subproject commit a8c37ee001af1ae88e5dfa637ae5b31b087c96d3 +Subproject commit 9194af44588633c1b2dae44bf945804401ff883e diff --git a/contrib/replxx b/contrib/replxx index 68410ac01df..f019cba7ea1 160000 --- a/contrib/replxx +++ b/contrib/replxx @@ -1 +1 @@ -Subproject commit 68410ac01dfb4f09ea76120ac5a2cecda3943aaf +Subproject commit f019cba7ea1bcd1b4feb7826f28ed57fb581b04c diff --git a/contrib/sysroot b/contrib/sysroot index 1a64956aa7c..410845187f5 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit 1a64956aa7c280448be6526251bb2b8e6d380ab1 +Subproject commit 410845187f582c5e6692b53dddbe43efbb728734 From 71353ceedecc01b9e66d480e360ddb49513c3307 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 20 Dec 2021 19:54:20 +0800 Subject: [PATCH 0198/1260] refactor metadata class factory --- programs/server/Server.cpp | 2 + src/Storages/Hive/StorageHiveMetadata.cpp | 9 ++++- src/Storages/IRemoteFileMetadata.cpp | 29 --------------- src/Storages/IRemoteFileMetadata.h | 35 ------------------ src/Storages/RemoteFileMetadataFactory.cpp | 43 ++++++++++++++++++++++ src/Storages/RemoteFileMetadataFactory.h | 27 ++++++++++++++ src/Storages/RemoteReadBufferCache.cpp | 1 + 7 files changed, 81 insertions(+), 65 deletions(-) create mode 100644 src/Storages/RemoteFileMetadataFactory.cpp create mode 100644 src/Storages/RemoteFileMetadataFactory.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 5ed0beb6568..f7b76b333c0 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -58,6 +58,7 @@ #include #include #include +#include #include #include #include @@ -482,6 +483,7 @@ int Server::main(const std::vector & /*args*/) registerDictionaries(); registerDisks(); registerFormats(); + registerRemoteFileMatadataCreators(); CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); diff --git a/src/Storages/Hive/StorageHiveMetadata.cpp b/src/Storages/Hive/StorageHiveMetadata.cpp index c7a4b909b7a..379d8721d6e 100644 --- a/src/Storages/Hive/StorageHiveMetadata.cpp +++ b/src/Storages/Hive/StorageHiveMetadata.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -40,6 +41,12 @@ String StorageHiveMetadata::getVersion() const return std::to_string(getLastModificationTimestamp()); } -REGISTTER_REMOTE_FILE_META_DATA_CLASS(StorageHiveMetadata) +void registerStorageHiveMetadataCreator() +{ + auto & factory = RemoteFileMetadataFactory::instance(); + auto creator = []() -> IRemoteFileMetadataPtr { return std::make_shared(); }; + factory.registerRemoteFileMatadataCreator("StorageHiveMetadata", creator); +} + } diff --git a/src/Storages/IRemoteFileMetadata.cpp b/src/Storages/IRemoteFileMetadata.cpp index bde083d65d7..9d2a93ab185 100644 --- a/src/Storages/IRemoteFileMetadata.cpp +++ b/src/Storages/IRemoteFileMetadata.cpp @@ -1,36 +1,7 @@ #include -#include namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; -} IRemoteFileMetadata::~IRemoteFileMetadata() {} -RemoteFileMetadataFactory & RemoteFileMetadataFactory::instance() -{ - static RemoteFileMetadataFactory g_factory; - return g_factory; -} - -IRemoteFileMetadataPtr RemoteFileMetadataFactory::get(const String & name) -{ - auto it = class_creators.find(name); - if (it == class_creators.end()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not found metadata class:{}", name); - return (it->second)(); -} - -void RemoteFileMetadataFactory::registerClass(const String & name, ClassCreator creator) -{ - auto it = class_creators.find(name); - if (it != class_creators.end()) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Class ({}) has been registered. It is a fatal error.", name); - } - class_creators[name] = creator; -} } diff --git a/src/Storages/IRemoteFileMetadata.h b/src/Storages/IRemoteFileMetadata.h index e6a3e2438f6..383f9f01927 100644 --- a/src/Storages/IRemoteFileMetadata.h +++ b/src/Storages/IRemoteFileMetadata.h @@ -40,39 +40,4 @@ protected: }; using IRemoteFileMetadataPtr = std::shared_ptr; - -/* - * How to register a subclass into the factory and use it ? - * 1) define your own subclass derive from IRemoteFileMetadata. Notice! the getName() must be the same - * as your subclass name. - * 2) in a .cpp file, call REGISTTER_REMOTE_FILE_META_DATA_CLASS(subclass), - * 3) call RemoteFileMetadataFactory::instance().createClass(subclass_name) where you want to make a new object - */ - -class RemoteFileMetadataFactory : private boost::noncopyable -{ -public: - using ClassCreator = std::function; - ~RemoteFileMetadataFactory() = default; - - static RemoteFileMetadataFactory & instance(); - IRemoteFileMetadataPtr get(const String & name); - void registerClass(const String &name, ClassCreator creator); -protected: - RemoteFileMetadataFactory() = default; - -private: - std::unordered_map class_creators; -}; - -// this should be used in a .cpp file. All the subclasses will finish the registeration before the main() -#define REGISTTER_REMOTE_FILE_META_DATA_CLASS(metadata_class) \ - class FileMetadataFactory##metadata_class{\ - public:\ - FileMetadataFactory##metadata_class(){\ - auto creator = []() -> IRemoteFileMetadataPtr { return std::make_shared(); };\ - RemoteFileMetadataFactory::instance().registerClass(#metadata_class, creator);\ - }\ - };\ - static FileMetadataFactory##metadata_class g_file_metadata_factory_instance##metadata_class; } diff --git a/src/Storages/RemoteFileMetadataFactory.cpp b/src/Storages/RemoteFileMetadataFactory.cpp new file mode 100644 index 00000000000..fac1e5648d6 --- /dev/null +++ b/src/Storages/RemoteFileMetadataFactory.cpp @@ -0,0 +1,43 @@ +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; +} + +RemoteFileMetadataFactory & RemoteFileMetadataFactory::instance() +{ + static RemoteFileMetadataFactory g_factory; + return g_factory; +} + +IRemoteFileMetadataPtr RemoteFileMetadataFactory::get(const String & name) +{ + auto it = class_creators.find(name); + if (it == class_creators.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not found metadata class:{}", name); + return (it->second)(); +} + +void RemoteFileMetadataFactory::registerRemoteFileMatadataCreator(const String & name, MetadataCreator creator) +{ + auto it = class_creators.find(name); + if (it != class_creators.end()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Class ({}) has been registered. It is a fatal error.", name); + } + class_creators[name] = creator; +} + +void registerStorageHiveMetadataCreator(); + +void registerRemoteFileMatadataCreators() +{ + registerStorageHiveMetadataCreator(); +} +} diff --git a/src/Storages/RemoteFileMetadataFactory.h b/src/Storages/RemoteFileMetadataFactory.h new file mode 100644 index 00000000000..507da9ea08b --- /dev/null +++ b/src/Storages/RemoteFileMetadataFactory.h @@ -0,0 +1,27 @@ +#pragma once +#include +#include +#include +#include +namespace DB +{ + +class RemoteFileMetadataFactory : private boost::noncopyable +{ +public: + using MetadataCreator = std::function; + ~RemoteFileMetadataFactory() = default; + + static RemoteFileMetadataFactory & instance(); + IRemoteFileMetadataPtr get(const String & name); + void registerRemoteFileMatadataCreator(const String &name, MetadataCreator creator); +protected: + RemoteFileMetadataFactory() = default; + +private: + std::unordered_map class_creators; +}; + +void registerRemoteFileMatadataCreators(); + +} diff --git a/src/Storages/RemoteReadBufferCache.cpp b/src/Storages/RemoteReadBufferCache.cpp index aa334215575..2252f84dfd7 100644 --- a/src/Storages/RemoteReadBufferCache.cpp +++ b/src/Storages/RemoteReadBufferCache.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include namespace fs = std::filesystem; From 869cdbeb4472a10352da8cbd2e183b1c11f6ab62 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Fri, 3 Dec 2021 13:25:14 +0800 Subject: [PATCH 0199/1260] added table function hdfsCluster --- .../table-functions/hdfsCluster.md | 58 +++ src/Storages/HDFS/StorageHDFS.cpp | 337 ++++++++++-------- src/Storages/HDFS/StorageHDFS.h | 68 +++- src/Storages/HDFS/StorageHDFSCluster.cpp | 149 ++++++++ src/Storages/HDFS/StorageHDFSCluster.h | 55 +++ src/TableFunctions/TableFunctionHDFS.h | 2 +- .../TableFunctionHDFSCluster.cpp | 118 ++++++ src/TableFunctions/TableFunctionHDFSCluster.h | 54 +++ src/TableFunctions/registerTableFunctions.cpp | 1 + src/TableFunctions/registerTableFunctions.h | 1 + tests/integration/test_storage_hdfs/test.py | 22 ++ 11 files changed, 710 insertions(+), 155 deletions(-) create mode 100644 docs/en/sql-reference/table-functions/hdfsCluster.md create mode 100644 src/Storages/HDFS/StorageHDFSCluster.cpp create mode 100644 src/Storages/HDFS/StorageHDFSCluster.h create mode 100644 src/TableFunctions/TableFunctionHDFSCluster.cpp create mode 100644 src/TableFunctions/TableFunctionHDFSCluster.h diff --git a/docs/en/sql-reference/table-functions/hdfsCluster.md b/docs/en/sql-reference/table-functions/hdfsCluster.md new file mode 100644 index 00000000000..6183fe83c38 --- /dev/null +++ b/docs/en/sql-reference/table-functions/hdfsCluster.md @@ -0,0 +1,58 @@ +--- +toc_priority: 55 +toc_title: hdfsCluster +--- + +# hdfsCluster Table Function {#hdfsCluster-table-function} + +Allows processing files from HDFS in parallel from many nodes in a specified cluster. On initiator it creates a connection to all nodes in the cluster, discloses asterics in HDFS file path, and dispatches each file dynamically. On the worker node it asks the initiator about the next task to process and processes it. This is repeated until all tasks are finished. + +**Syntax** + +``` sql +hdfsCluster(cluster_name, URI, format, structure) +``` + +**Arguments** + +- `cluster_name` — Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers. +- `URI` — URI to a file or a bunch of files. Supports following wildcards in readonly mode: `*`, `?`, `{'abc','def'}` and `{N..M}` where `N`, `M` — numbers, `abc`, `def` — strings. For more information see [Wildcards In Path](../../engines/table-engines/integrations/s3.md#wildcards-in-path). +- `format` — The [format](../../interfaces/formats.md#formats) of the file. +- `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. + +**Returned value** + +A table with the specified structure for reading data in the specified file. + +**Examples** + +1. Suppose that we have a ClickHouse cluster named `cluster_simple`, and several files with following URIs on HDFS: + +- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ +- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ + +2. Query the amount of rows in these files: + +``` sql +SELECT count(*) +FROM hdfsCluster('cluster_simple', 'hdfs://hdfs1:9000/{some,another}_dir/some_file_{1..3}', 'TSV', 'name String, value UInt32') +``` + +3. Query the amount of rows in all files of these two directories: + +``` sql +SELECT count(*) +FROM hdfsCluster('cluster_simple', 'hdfs://hdfs1:9000/{some,another}_dir/*', 'TSV', 'name String, value UInt32') +``` + +!!! warning "Warning" + If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. + +**See Also** + +- [HDFS engine](../../engines/table-engines/integrations/hdfs.md) +- [HDFS table function](../../sql-reference/table-functions/hdfs.md) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 70aa3d28174..68a4b835bfc 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include #include #include @@ -54,6 +53,8 @@ namespace ErrorCodes extern const int ACCESS_DENIED; } +Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match); + StorageHDFS::StorageHDFS( const String & uri_, const StorageID & table_id_, @@ -62,13 +63,15 @@ StorageHDFS::StorageHDFS( const ConstraintsDescription & constraints_, const String & comment, ContextPtr context_, - const String & compression_method_ = "", + const String & compression_method_, + const bool distributed_processing_, ASTPtr partition_by_) : IStorage(table_id_) , WithContext(context_) , uri(uri_) , format_name(format_name_) , compression_method(compression_method_) + , distributed_processing(distributed_processing_) , partition_by(partition_by_) { context_->getRemoteHostFilter().checkURL(Poco::URI(uri)); @@ -81,154 +84,178 @@ StorageHDFS::StorageHDFS( setInMemoryMetadata(storage_metadata); } -using StorageHDFSPtr = std::shared_ptr; - -class HDFSSource : public SourceWithProgress, WithContext +class HDFSSource::DisclosedGlobIterator::Impl { public: - struct SourcesInfo + Impl(ContextPtr context_, const String & uri) { - std::vector uris; - std::atomic next_uri_to_read = 0; + const size_t begin_of_path = uri.find('/', uri.find("//") + 2); + const String path_from_uri = uri.substr(begin_of_path); + const String uri_without_path = uri.substr(0, begin_of_path); /// ends without '/' - bool need_path_column = false; - bool need_file_column = false; - }; - - using SourcesInfoPtr = std::shared_ptr; - - static Block getHeader(const StorageMetadataPtr & metadata_snapshot, bool need_path_column, bool need_file_column) - { - auto header = metadata_snapshot->getSampleBlock(); - - /// Note: AddingDefaultsBlockInputStream doesn't change header. - - if (need_path_column) - header.insert({DataTypeString().createColumn(), std::make_shared(), "_path"}); - if (need_file_column) - header.insert({DataTypeString().createColumn(), std::make_shared(), "_file"}); - - return header; + HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context_->getGlobalContext()->getConfigRef()); + HDFSFSPtr fs = createHDFSFS(builder.get()); + std::lock_guard lock(mutex); + uris = LSWithRegexpMatching("/", fs, path_from_uri); + for(size_t i=0; igetSampleBlock(); + /// Note: AddingDefaultsBlockInputStream doesn't change header. + if (need_path_column) + header.insert({DataTypeString().createColumn(), std::make_shared(), "_path"}); + if (need_file_column) + header.insert({DataTypeString().createColumn(), std::make_shared(), "_file"}); + return header; +} + +Block HDFSSource::getBlockForSource( + const StorageHDFSPtr & storage, + const StorageMetadataPtr & metadata_snapshot, + const ColumnsDescription & columns_description, + bool need_path_column, + bool need_file_column) +{ + if (storage->isColumnOriented()) + return metadata_snapshot->getSampleBlockForColumns( + columns_description.getNamesOfPhysical(), storage->getVirtuals(), storage->getStorageID()); + else + return getHeader(metadata_snapshot, need_path_column, need_file_column); +} + +HDFSSource::DisclosedGlobIterator::DisclosedGlobIterator(ContextPtr context_, const String & uri) + : pimpl(std::make_shared(context_, uri)) {} + +String HDFSSource::DisclosedGlobIterator::next() +{ + return pimpl->next(); +} + + +HDFSSource::HDFSSource( + StorageHDFSPtr storage_, + const StorageMetadataPtr & metadata_snapshot_, + ContextPtr context_, + UInt64 max_block_size_, + bool need_path_column_, + bool need_file_column_, + std::shared_ptr file_iterator_, + ColumnsDescription columns_description_) + : SourceWithProgress(getBlockForSource(storage_, metadata_snapshot_, columns_description_, need_path_column_, need_file_column_)) + , WithContext(context_) + , storage(std::move(storage_)) + , metadata_snapshot(metadata_snapshot_) + , max_block_size(max_block_size_) + , need_path_column(need_path_column_) + , need_file_column(need_file_column_) + , file_iterator(file_iterator_) + , columns_description(std::move(columns_description_)) +{ + initialize(); +} + +bool HDFSSource::initialize() +{ + current_path = (*file_iterator)(); + if (current_path.empty()) + return false; + const size_t begin_of_path = current_path.find('/', current_path.find("//") + 2); + const String path_from_uri = current_path.substr(begin_of_path); + const String uri_without_path = current_path.substr(0, begin_of_path); + + auto compression = chooseCompressionMethod(path_from_uri, storage->compression_method); + read_buf = wrapReadBufferWithCompressionMethod(std::make_unique(uri_without_path, path_from_uri, getContext()->getGlobalContext()->getConfigRef()), compression); + + auto get_block_for_format = [&]() -> Block { if (storage->isColumnOriented()) - return metadata_snapshot->getSampleBlockForColumns( - columns_description.getNamesOfPhysical(), storage->getVirtuals(), storage->getStorageID()); - else - return getHeader(metadata_snapshot, files_info->need_path_column, files_info->need_file_column); - } + return metadata_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); + return metadata_snapshot->getSampleBlock(); + }; - HDFSSource( - StorageHDFSPtr storage_, - const StorageMetadataPtr & metadata_snapshot_, - ContextPtr context_, - UInt64 max_block_size_, - SourcesInfoPtr source_info_, - String uri_without_path_, - ColumnsDescription columns_description_) - : SourceWithProgress(getBlockForSource(storage_, metadata_snapshot_, columns_description_, source_info_)) - , WithContext(context_) - , storage(std::move(storage_)) - , metadata_snapshot(metadata_snapshot_) - , source_info(std::move(source_info_)) - , uri_without_path(std::move(uri_without_path_)) - , max_block_size(max_block_size_) - , columns_description(std::move(columns_description_)) - { - } + auto input_format = getContext()->getInputFormat(storage->format_name, *read_buf, get_block_for_format(), max_block_size); - String getName() const override + QueryPipelineBuilder builder; + builder.init(Pipe(input_format)); + if (columns_description.hasDefaults()) { - return "HDFS"; - } - - Chunk generate() override - { - while (true) + builder.addSimpleTransform([&](const Block & header) { - if (!reader) - { - auto pos = source_info->next_uri_to_read.fetch_add(1); - if (pos >= source_info->uris.size()) - return {}; + return std::make_shared(header, columns_description, *input_format, getContext()); + }); + } + pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); + reader = std::make_unique(*pipeline); + return true; +} - auto path = source_info->uris[pos]; - current_path = uri_without_path + path; +String HDFSSource::getName() const +{ + return "HDFSSource"; +} - auto compression = chooseCompressionMethod(path, storage->compression_method); - read_buf = wrapReadBufferWithCompressionMethod(std::make_unique(uri_without_path, path, getContext()->getGlobalContext()->getConfigRef()), compression); +Chunk HDFSSource::generate() +{ + if (!reader) + return {}; - auto get_block_for_format = [&]() -> Block - { - if (storage->isColumnOriented()) - return metadata_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); - return metadata_snapshot->getSampleBlock(); - }; - auto input_format = getContext()->getInputFormat(storage->format_name, *read_buf, get_block_for_format(), max_block_size); + Chunk chunk; + if (reader->pull(chunk)) + { + Columns columns = chunk.getColumns(); + UInt64 num_rows = chunk.getNumRows(); - QueryPipelineBuilder builder; - builder.init(Pipe(input_format)); - if (columns_description.hasDefaults()) - { - builder.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, columns_description, *input_format, getContext()); - }); - } - pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); - reader = std::make_unique(*pipeline); - } - - Block res; - if (reader->pull(res)) - { - Columns columns = res.getColumns(); - UInt64 num_rows = res.rows(); - - /// Enrich with virtual columns. - if (source_info->need_path_column) - { - auto column = DataTypeString().createColumnConst(num_rows, current_path); - columns.push_back(column->convertToFullColumnIfConst()); - } - - if (source_info->need_file_column) - { - size_t last_slash_pos = current_path.find_last_of('/'); - auto file_name = current_path.substr(last_slash_pos + 1); - - auto column = DataTypeString().createColumnConst(num_rows, std::move(file_name)); - columns.push_back(column->convertToFullColumnIfConst()); - } - - return Chunk(std::move(columns), num_rows); - } - - reader.reset(); - pipeline.reset(); - read_buf.reset(); + /// Enrich with virtual columns. + if (need_path_column) + { + auto column = DataTypeString().createColumnConst(num_rows, current_path); + columns.push_back(column->convertToFullColumnIfConst()); } + + if (need_file_column) + { + size_t last_slash_pos = current_path.find_last_of('/'); + auto file_name = current_path.substr(last_slash_pos + 1); + + auto column = DataTypeString().createColumnConst(num_rows, std::move(file_name)); + columns.push_back(column->convertToFullColumnIfConst()); + } + + return Chunk(std::move(columns), num_rows); } -private: - StorageHDFSPtr storage; - StorageMetadataPtr metadata_snapshot; - SourcesInfoPtr source_info; - String uri_without_path; - UInt64 max_block_size; - ColumnsDescription columns_description; + reader.reset(); + pipeline.reset(); + read_buf.reset(); + + if (!initialize()) + return {}; + return generate(); +} - std::unique_ptr read_buf; - std::unique_ptr pipeline; - std::unique_ptr reader; - String current_path; -}; class HDFSSink : public SinkToStorage { @@ -300,7 +327,6 @@ public: private: const String uri; - const String format; const Block sample_block; ContextPtr context; @@ -367,29 +393,33 @@ Pipe StorageHDFS::read( size_t max_block_size, unsigned num_streams) { - const size_t begin_of_path = uri.find('/', uri.find("//") + 2); - const String path_from_uri = uri.substr(begin_of_path); - const String uri_without_path = uri.substr(0, begin_of_path); - - HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context_->getGlobalContext()->getConfigRef()); - HDFSFSPtr fs = createHDFSFS(builder.get()); - - auto sources_info = std::make_shared(); - sources_info->uris = LSWithRegexpMatching("/", fs, path_from_uri); - - if (sources_info->uris.empty()) - LOG_WARNING(log, "No file in HDFS matches the path: {}", uri); - + bool need_path_column = false; + bool need_file_column = false; for (const auto & column : column_names) { if (column == "_path") - sources_info->need_path_column = true; + need_path_column = true; if (column == "_file") - sources_info->need_file_column = true; + need_file_column = true; } - if (num_streams > sources_info->uris.size()) - num_streams = sources_info->uris.size(); + std::shared_ptr iterator_wrapper{nullptr}; + if (distributed_processing) + { + iterator_wrapper = std::make_shared( + [callback = context_->getReadTaskCallback()]() -> String { + return callback(); + }); + } + else + { + /// Iterate through disclosed globs and make a source for each file + auto glob_iterator = std::make_shared(context_, uri); + iterator_wrapper = std::make_shared([glob_iterator]() + { + return glob_iterator->next(); + }); + } Pipes pipes; auto this_ptr = std::static_pointer_cast(shared_from_this()); @@ -409,8 +439,9 @@ Pipe StorageHDFS::read( metadata_snapshot, context_, max_block_size, - sources_info, - uri_without_path, + need_path_column, + need_file_column, + iterator_wrapper, get_columns_for_format())); } return Pipe::unitePipes(std::move(pipes)); @@ -443,13 +474,13 @@ SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataP } } -void StorageHDFS::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, ContextPtr context_, TableExclusiveLockHolder &) +void StorageHDFS::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) { const size_t begin_of_path = uri.find('/', uri.find("//") + 2); const String path = uri.substr(begin_of_path); const String url = uri.substr(0, begin_of_path); - HDFSBuilderWrapper builder = createHDFSBuilder(url + "/", context_->getGlobalContext()->getConfigRef()); + HDFSBuilderWrapper builder = createHDFSBuilder(url + "/", local_context->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); int ret = hdfsDelete(fs.get(), path.data(), 0); @@ -488,7 +519,7 @@ void registerStorageHDFS(StorageFactory & factory) partition_by = args.storage_def->partition_by->clone(); return StorageHDFS::create( - url, args.table_id, format_name, args.columns, args.constraints, args.comment, args.getContext(), compression_method, partition_by); + url, args.table_id, format_name, args.columns, args.constraints, args.comment, args.getContext(), compression_method, false, partition_by); }, { .supports_sort_order = true, // for partition by diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index db6b078265d..2a9a25188ae 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -4,6 +4,7 @@ #if USE_HDFS +#include #include #include #include @@ -54,17 +55,82 @@ protected: const ConstraintsDescription & constraints_, const String & comment, ContextPtr context_, - const String & compression_method_, + const String & compression_method_ = "", + bool distributed_processing_ = false, ASTPtr partition_by = nullptr); private: const String uri; String format_name; String compression_method; + const bool distributed_processing; ASTPtr partition_by; Poco::Logger * log = &Poco::Logger::get("StorageHDFS"); }; + +class PullingPipelineExecutor; + +class HDFSSource : public SourceWithProgress, WithContext +{ +public: + class DisclosedGlobIterator + { + public: + DisclosedGlobIterator(ContextPtr context_, const String & uri_); + String next(); + private: + class Impl; + /// shared_ptr to have copy constructor + std::shared_ptr pimpl; + }; + + using IteratorWrapper = std::function; + using StorageHDFSPtr = std::shared_ptr; + + static Block getHeader( + const StorageMetadataPtr & metadata_snapshot, + bool need_path_column, + bool need_file_column); + + static Block getBlockForSource( + const StorageHDFSPtr & storage, + const StorageMetadataPtr & metadata_snapshot, + const ColumnsDescription & columns_description, + bool need_path_column, + bool need_file_column); + + HDFSSource( + StorageHDFSPtr storage_, + const StorageMetadataPtr & metadata_snapshot_, + ContextPtr context_, + UInt64 max_block_size_, + bool need_path_column_, + bool need_file_column_, + std::shared_ptr file_iterator_, + ColumnsDescription columns_description_); + + String getName() const override; + + Chunk generate() override; + +private: + StorageHDFSPtr storage; + StorageMetadataPtr metadata_snapshot; + UInt64 max_block_size; + bool need_path_column; + bool need_file_column; + std::shared_ptr file_iterator; + ColumnsDescription columns_description; + + std::unique_ptr read_buf; + std::unique_ptr pipeline; + std::unique_ptr reader; + String current_path; + + /// Recreate ReadBuffer and PullingPipelineExecutor for each file. + bool initialize(); +}; } #endif diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp new file mode 100644 index 00000000000..f2f3c77b3b9 --- /dev/null +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -0,0 +1,149 @@ +#include + +#if USE_HDFS + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +namespace DB +{ +StorageHDFSCluster::StorageHDFSCluster( + String cluster_name_, + const String & uri_, + const StorageID & table_id_, + const String & format_name_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & compression_method_) + : IStorage(table_id_) + , cluster_name(cluster_name_) + , uri(uri_) + , format_name(format_name_) + , compression_method(compression_method_) +{ + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); + setInMemoryMetadata(storage_metadata); +} + +/// The code executes on initiator +Pipe StorageHDFSCluster::read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t /*max_block_size*/, + unsigned /*num_streams*/) +{ + auto cluster = context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettings()); + + auto iterator = std::make_shared(context, uri); + auto callback = std::make_shared([iterator]() mutable -> String + { + return iterator->next(); + }); + + /// Calculate the header. This is significant, because some columns could be thrown away in some cases like query with count(*) + Block header = + InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + + const Scalars & scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{}; + + Pipes pipes; + + const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; + + for (const auto & replicas : cluster->getShardsAddresses()) + { + /// There will be only one replica, because we consider each replica as a shard + for (const auto & node : replicas) + { + auto connection = std::make_shared( + node.host_name, node.port, context->getGlobalContext()->getCurrentDatabase(), + node.user, node.password, node.cluster, node.cluster_secret, + "HDFSClusterInititiator", + node.compression, + node.secure + ); + + + /// For unknown reason global context is passed to IStorage::read() method + /// So, task_identifier is passed as constructor argument. It is more obvious. + auto remote_query_executor = std::make_shared( + connection, + queryToString(query_info.query), + header, + context, + /*throttler=*/nullptr, + scalars, + Tables(), + processed_stage, + callback); + + pipes.emplace_back(std::make_shared(remote_query_executor, add_agg_info, false)); + } + } + + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); + return Pipe::unitePipes(std::move(pipes)); +} + +QueryProcessingStage::Enum StorageHDFSCluster::getQueryProcessingStage( + ContextPtr context, QueryProcessingStage::Enum to_stage, const StorageMetadataPtr &, SelectQueryInfo &) const +{ + /// Initiator executes query on remote node. + if (context->getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY) + if (to_stage >= QueryProcessingStage::Enum::WithMergeableState) + return QueryProcessingStage::Enum::WithMergeableState; + + /// Follower just reads the data. + return QueryProcessingStage::Enum::FetchColumns; +} + + +NamesAndTypesList StorageHDFSCluster::getVirtuals() const +{ + return NamesAndTypesList{ + {"_path", std::make_shared()}, + {"_file", std::make_shared()} + }; +} + + +} + +#endif diff --git a/src/Storages/HDFS/StorageHDFSCluster.h b/src/Storages/HDFS/StorageHDFSCluster.h new file mode 100644 index 00000000000..0e568a9faf8 --- /dev/null +++ b/src/Storages/HDFS/StorageHDFSCluster.h @@ -0,0 +1,55 @@ +#pragma once + +#include + +#if USE_HDFS + +#include +#include + +#include + +#include +#include +#include + +namespace DB +{ + +class Context; + +class StorageHDFSCluster : public shared_ptr_helper, public IStorage +{ + friend struct shared_ptr_helper; +public: + std::string getName() const override { return "HDFSCluster"; } + + Pipe read(const Names &, const StorageMetadataPtr &, SelectQueryInfo &, + ContextPtr, QueryProcessingStage::Enum, size_t /*max_block_size*/, unsigned /*num_streams*/) override; + + QueryProcessingStage::Enum + getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageMetadataPtr &, SelectQueryInfo &) const override; + + NamesAndTypesList getVirtuals() const override; + +protected: + StorageHDFSCluster( + String cluster_name_, + const String & uri_, + const StorageID & table_id_, + const String & format_name_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & compression_method_); + +private: + String cluster_name; + String uri; + String format_name; + String compression_method; +}; + + +} + +#endif diff --git a/src/TableFunctions/TableFunctionHDFS.h b/src/TableFunctions/TableFunctionHDFS.h index d9ee9b47868..70bdc67efc8 100644 --- a/src/TableFunctions/TableFunctionHDFS.h +++ b/src/TableFunctions/TableFunctionHDFS.h @@ -12,7 +12,7 @@ namespace DB class Context; -/* hdfs(name_node_ip:name_node_port, format, structure) - creates a temporary storage from hdfs file +/* hdfs(URI, format, structure) - creates a temporary storage from hdfs files * */ class TableFunctionHDFS : public ITableFunctionFileLike diff --git a/src/TableFunctions/TableFunctionHDFSCluster.cpp b/src/TableFunctions/TableFunctionHDFSCluster.cpp new file mode 100644 index 00000000000..241e52d4d2c --- /dev/null +++ b/src/TableFunctions/TableFunctionHDFSCluster.cpp @@ -0,0 +1,118 @@ +#include + +#if USE_HDFS + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "registerTableFunctions.h" + +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +void TableFunctionHDFSCluster::parseArguments(const ASTPtr & ast_function, ContextPtr context) +{ + /// Parse args + ASTs & args_func = ast_function->children; + + if (args_func.size() != 1) + throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + ASTs & args = args_func.at(0)->children; + + const auto message = fmt::format( + "The signature of table function {} shall be the following:\n" \ + " - cluster, uri, format, structure", + " - cluster, uri, format, structure, compression_method", + getName()); + + if (args.size() < 4 || args.size() > 5) + throw Exception(message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + + /// This arguments are always the first + cluster_name = args[0]->as().value.safeGet(); + uri = args[1]->as().value.safeGet(); + format = args[2]->as().value.safeGet(); + structure = args[3]->as().value.safeGet(); + if(args.size()>=5) + { + compression_method = args[4]->as().value.safeGet(); + } +} + + +ColumnsDescription TableFunctionHDFSCluster::getActualTableStructure(ContextPtr context) const +{ + return parseColumnsListFromString(structure, context); +} + +StoragePtr TableFunctionHDFSCluster::executeImpl( + const ASTPtr & /*function*/, ContextPtr context, + const std::string & table_name, ColumnsDescription /*cached_columns*/) const +{ + StoragePtr storage; + if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) + { + /// On worker node this uri won't contains globs + storage = StorageHDFS::create( + uri, + StorageID(getDatabaseName(), table_name), + format, + getActualTableStructure(context), + ConstraintsDescription{}, + String{}, + context, + compression_method, + /*distributed_processing=*/true, + nullptr); + } + else + { + storage = StorageHDFSCluster::create( + cluster_name, uri, StorageID(getDatabaseName(), table_name), + format, getActualTableStructure(context), ConstraintsDescription{}, + compression_method); + } + + storage->startup(); + + return storage; +} + + +void registerTableFunctionHDFSCluster(TableFunctionFactory & factory) +{ + factory.registerFunction(); +} + + +} + +#endif diff --git a/src/TableFunctions/TableFunctionHDFSCluster.h b/src/TableFunctions/TableFunctionHDFSCluster.h new file mode 100644 index 00000000000..58d1c3d9b05 --- /dev/null +++ b/src/TableFunctions/TableFunctionHDFSCluster.h @@ -0,0 +1,54 @@ +#pragma once + +#include + +#if USE_HDFS + +#include + + +namespace DB +{ + +class Context; + +/** + * hdfsCluster(cluster, URI, format, structure, compression_method) + * A table function, which allows to process many files from HDFS on a specific cluster + * On initiator it creates a connection to _all_ nodes in cluster, discloses asterics + * in HDFS file path and dispatch each file dynamically. + * On worker node it asks initiator about next task to process, processes it. + * This is repeated until the tasks are finished. + */ +class TableFunctionHDFSCluster : public ITableFunction +{ +public: + static constexpr auto name = "hdfsCluster"; + std::string getName() const override + { + return name; + } + bool hasStaticStructure() const override { return true; } + +protected: + StoragePtr executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns) const override; + + const char * getStorageTypeName() const override { return "HDFSCluster"; } + + ColumnsDescription getActualTableStructure(ContextPtr) const override; + void parseArguments(const ASTPtr &, ContextPtr) override; + + String cluster_name; + String uri; + String format; + String structure; + String compression_method = "auto"; +}; + +} + +#endif diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index b4aab3e5c55..ea5c2c75f94 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -28,6 +28,7 @@ void registerTableFunctions() #if USE_HDFS registerTableFunctionHDFS(factory); + registerTableFunctionHDFSCluster(factory); #endif registerTableFunctionODBC(factory); diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index 8dbb5ebb5fa..8ddd9b7c8ab 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -26,6 +26,7 @@ void registerTableFunctionCOS(TableFunctionFactory & factory); #if USE_HDFS void registerTableFunctionHDFS(TableFunctionFactory & factory); +void registerTableFunctionHDFSCluster(TableFunctionFactory & factory); #endif void registerTableFunctionODBC(TableFunctionFactory & factory); diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index ede1dafefb1..33ce94a7a29 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -323,6 +323,28 @@ def test_read_table_with_default(started_cluster): +def test_hdfsCluster(started_cluster): + hdfs_api = started_cluster.hdfs_api + fs = HdfsClient(hosts=started_cluster.hdfs_ip) + dir = '/test_hdfsCluster' + exists = fs.exists(dir) + if exists: + fs.delete(dir, recursive=True) + fs.mkdirs(dir) + hdfs_api.write_data("/test_hdfsCluster/file1", "1\n") + hdfs_api.write_data("/test_hdfsCluster/file2", "2\n") + hdfs_api.write_data("/test_hdfsCluster/file3", "3\n") + + actual = node1.query("select id, _file as file_name, _path as file_path from hdfs('hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') order by id") + expected = "1\tfile1\thdfs://hdfs1:9000/test_hdfsCluster/file1\n2\tfile2\thdfs://hdfs1:9000/test_hdfsCluster/file2\n3\tfile3\thdfs://hdfs1:9000/test_hdfsCluster/file3\n" + assert actual == expected + + actual = node1.query("select id, _file as file_name, _path as file_path from hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') order by id") + expected = "1\tfile1\thdfs://hdfs1:9000/test_hdfsCluster/file1\n2\tfile2\thdfs://hdfs1:9000/test_hdfsCluster/file2\n3\tfile3\thdfs://hdfs1:9000/test_hdfsCluster/file3\n" + assert actual == expected + fs.delete(dir, recursive=True) + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From f0b9a4327ae4f1658af58953304629bcdf776326 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Fri, 17 Dec 2021 14:03:20 +0300 Subject: [PATCH 0200/1260] Use table UUID in zero-copy shared label in ZooKeeper --- src/Disks/DiskDecorator.h | 2 + src/Disks/IDisk.cpp | 6 ++ src/Disks/IDisk.h | 2 + src/Disks/IDiskRemote.cpp | 6 ++ src/Disks/IDiskRemote.h | 3 + src/Storages/IStorage.h | 3 + src/Storages/MergeTree/MergeTreeData.cpp | 23 +++-- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 98 ++++++++++++++----- src/Storages/StorageReplicatedMergeTree.h | 12 ++- .../test_s3_zero_copy_replication/test.py | 1 + .../test_s3_zero_copy_upgrade/test.py | 37 +++++-- 12 files changed, 154 insertions(+), 41 deletions(-) diff --git a/src/Disks/DiskDecorator.h b/src/Disks/DiskDecorator.h index 5890a772ed7..f6724279577 100644 --- a/src/Disks/DiskDecorator.h +++ b/src/Disks/DiskDecorator.h @@ -80,6 +80,8 @@ public: size_t buf_size, WriteMode mode) override { return delegate->writeMetaFile(path, buf_size, mode); } + void removeMetaFileIfExists(const String & path) override { delegate->removeMetaFileIfExists(path); } + UInt32 getRefCount(const String & path) const override { return delegate->getRefCount(path); } protected: diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 404f759d6cb..b1d7b33fec3 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -104,4 +104,10 @@ std::unique_ptr IDisk::writeMetaFile( return writeFile(path, buf_size, mode); } +void IDisk::removeMetaFileIfExists(const String & path) +{ + LOG_TRACE(&Poco::Logger::get("IDisk"), "Remove local metafile: {}", path); + removeFileIfExists(path); +} + } diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index a04429e0acb..28d3f0f615d 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -263,6 +263,8 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, WriteMode mode = WriteMode::Rewrite); + virtual void removeMetaFileIfExists(const String & path); + /// Return reference count for remote FS. /// Overridden in IDiskRemote. virtual UInt32 getRefCount(const String &) const { return 0; } diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index 0162b0fc8da..4d78b5d8818 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -519,6 +519,12 @@ std::unique_ptr IDiskRemote::writeMetaFile( return metadata_disk->writeFile(path, buf_size, mode); } +void IDiskRemote::removeMetaFileIfExists(const String & path) +{ + LOG_TRACE(log, "Remove metafile: {}", path); + return metadata_disk->removeFileIfExists(path); +} + UInt32 IDiskRemote::getRefCount(const String & path) const { auto meta = readMeta(path); diff --git a/src/Disks/IDiskRemote.h b/src/Disks/IDiskRemote.h index 9bfaead3def..124b85edd70 100644 --- a/src/Disks/IDiskRemote.h +++ b/src/Disks/IDiskRemote.h @@ -146,6 +146,9 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, WriteMode mode = WriteMode::Rewrite) override; + virtual void removeMetaFileIfExists( + const String & path) override; + UInt32 getRefCount(const String & path) const override; protected: diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 8240b7182d8..2318e28de6b 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -113,6 +113,9 @@ public: /// The name of the table. StorageID getStorageID() const; + /// Unique ID, synchronized between replicas for replicated storage + virtual String getTableUniqID() const { return ""; } + /// Returns true if the storage receives data from a remote server or servers. virtual bool isRemote() const { return false; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c9336a8419e..a39f8ba7b7f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -62,6 +62,7 @@ #include #include +#include #include #include @@ -3873,11 +3874,12 @@ void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, ContextPtr String replica_name = getReplicaName(); String zookeeper_name = getZooKeeperName(); String zookeeper_path = getZooKeeperPath(); + String table_uuid = getTableUniqID(); for (auto & [old_name, new_name, disk] : renamed_parts.old_and_new_names) { bool keep_shared = removeSharedDetachedPart(disk, fs::path(relative_data_path) / "detached" / new_name / "", old_name, - zookeeper_name, replica_name, zookeeper_path, supportsReplication()); + table_uuid, zookeeper_name, replica_name, zookeeper_path, supportsReplication()); LOG_DEBUG(log, "Dropped detached part {}, keep shared data: {}", old_name, keep_shared); old_name.clear(); } @@ -5124,6 +5126,7 @@ public: is_remote = part.storage.isRemote(); replica_name = part.storage.getReplicaName(); zookeeper_name = part.storage.getZooKeeperName(); + table_uuid = part.storage.getTableUniqID(); } void save(DiskPtr disk, const String & path) const @@ -5140,6 +5143,8 @@ public: buffer->write("\n", 1); writeString(zookeeper_name, *buffer); buffer->write("\n", 1); + writeString(table_uuid, *buffer); + buffer->write("\n", 1); } bool load(DiskPtr disk, const String & path) @@ -5163,12 +5168,14 @@ public: DB::assertChar('\n', *buffer); readString(zookeeper_name, *buffer); DB::assertChar('\n', *buffer); + readString(table_uuid, *buffer); + DB::assertChar('\n', *buffer); return true; } - void clean(DiskPtr disk, const String & path) + static void clean(DiskPtr disk, const String & path) { - disk->removeFileIfExists(getFileName(path)); + disk->removeMetaFileIfExists(getFileName(path)); } private: @@ -5183,6 +5190,7 @@ public: bool is_remote; String replica_name; String zookeeper_name; + String table_uuid; }; PartitionCommandsResultInfo MergeTreeData::freezePartition( @@ -5300,8 +5308,11 @@ bool MergeTreeData::removeSharedDetachedPart(DiskPtr disk, const String & path, { FreezeMetaData meta; if (meta.load(disk, path)) - return removeSharedDetachedPart(disk, path, part_name, meta.zookeeper_name, meta.replica_name, + { + FreezeMetaData::clean(disk, path); + return removeSharedDetachedPart(disk, path, part_name, meta.table_uuid, meta.zookeeper_name, meta.replica_name, "", meta.is_replicated); + } } disk->removeSharedRecursive(path, keep_shared); @@ -5309,7 +5320,7 @@ bool MergeTreeData::removeSharedDetachedPart(DiskPtr disk, const String & path, return keep_shared; } -bool MergeTreeData::removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, +bool MergeTreeData::removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid, const String & zookeeper_name, const String & replica_name, const String & zookeeper_path, bool is_replicated) { bool keep_shared = false; @@ -5346,7 +5357,7 @@ bool MergeTreeData::removeSharedDetachedPart(DiskPtr disk, const String & path, if (ref_count == 0) { String id = disk->getUniqueId(checksums); - keep_shared = !StorageReplicatedMergeTree::unlockSharedDataById(id, part_name, + keep_shared = !StorageReplicatedMergeTree::unlockSharedDataById(id, table_uuid, part_name, replica_name, disk, zookeeper, getContext()->getReplicatedMergeTreeSettings(), log, zookeeper_path); } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index fc21dd87569..6c77735e5dc 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1184,7 +1184,7 @@ private: /// Check shared data usage on other replicas for detached/freezed part /// Remove local files and remote files if needed bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name); - bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, + bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid, const String & zookeeper_name, const String & replica_name, const String & zookeeper_path, bool is_replicated); }; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7c60a9c17d3..56bc3f933a6 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7124,6 +7124,42 @@ std::unique_ptr StorageReplicatedMergeTree::getDefaultSetting } +String StorageReplicatedMergeTree::getTableUniqID() const +{ + if (table_global_id == UUIDHelpers::Nil) + { + zkutil::ZooKeeperPtr zookeeper = getZooKeeper(); + String zookeeper_table_id_path = fs::path(zookeeper_path) / "table_id"; + String id; + if (!zookeeper->tryGet(zookeeper_table_id_path, id)) + { + UUID table_id_candidate; + auto storage_id = getStorageID(); + if (storage_id.uuid != UUIDHelpers::Nil) + table_id_candidate = storage_id.uuid; + else + table_id_candidate = UUIDHelpers::generateV4(); + + id = toString(table_id_candidate); + + auto code = zookeeper->tryCreate(zookeeper_table_id_path, id, zkutil::CreateMode::Persistent); + if (code == Coordination::Error::ZNODEEXISTS) + { /// Other replica create node early + id = zookeeper->get(zookeeper_table_id_path); + } + else if (code != Coordination::Error::ZOK) + { + throw zkutil::KeeperException(code, zookeeper_table_id_path); + } + } + + table_global_id = parseFromString(id); + } + + return toString(table_global_id); +} + + void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part) const { if (!part.volume) @@ -7131,7 +7167,6 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part) DiskPtr disk = part.volume->getDisk(); if (!disk || !disk->supportZeroCopyReplication()) return; - String zero_copy = fmt::format("zero_copy_{}", toString(disk->getType())); zkutil::ZooKeeperPtr zookeeper = tryGetZooKeeper(); if (!zookeeper) @@ -7140,10 +7175,11 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part) String id = part.getUniqueId(); boost::replace_all(id, "/", "_"); - Strings zc_zookeeper_paths = getZeroCopyRootPath(*getDefaultSettings(), is_zero_copy_in_compatible_mode ? zookeeper_path : ""); + Strings zc_zookeeper_paths = getZeroCopyPartPath(*getDefaultSettings(), disk->getType(), getTableUniqID(), + part.name, is_zero_copy_in_compatible_mode ? zookeeper_path : ""); for (const auto & zc_zookeeper_path : zc_zookeeper_paths) { - String zookeeper_node = fs::path(zc_zookeeper_path) / zero_copy / "shared" / part.name / id / replica_name; + String zookeeper_node = fs::path(zc_zookeeper_path) / id / replica_name; LOG_TRACE(log, "Set zookeeper lock {}", zookeeper_node); createZeroCopyLockNode(zookeeper, zookeeper_node); @@ -7173,27 +7209,24 @@ bool StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & par if (ref_count > 0) /// Keep part shard info for frozen backups return false; - return unlockSharedDataById(part.getUniqueId(), name, replica_name, disk, zookeeper, *getDefaultSettings(), log, + return unlockSharedDataById(part.getUniqueId(), getTableUniqID(), name, replica_name, disk, zookeeper, *getDefaultSettings(), log, is_zero_copy_in_compatible_mode ? zookeeper_path : String("")); } -bool StorageReplicatedMergeTree::unlockSharedDataById(String id, const String & part_name, +bool StorageReplicatedMergeTree::unlockSharedDataById(String id, const String & table_uuid, const String & part_name, const String & replica_name_, DiskPtr disk, zkutil::ZooKeeperPtr zookeeper_ptr, const MergeTreeSettings & settings, Poco::Logger * logger, const String & zookeeper_path_old) { boost::replace_all(id, "/", "_"); - String zero_copy = fmt::format("zero_copy_{}", toString(disk->getType())); - - Strings zc_zookeeper_paths = getZeroCopyRootPath(settings, zookeeper_path_old); + Strings zc_zookeeper_paths = getZeroCopyPartPath(settings, disk->getType(), table_uuid, part_name, zookeeper_path_old); bool res = true; for (const auto & zc_zookeeper_path : zc_zookeeper_paths) { - String zookeeper_part_node = fs::path(zc_zookeeper_path) / zero_copy / "shared" / part_name; - String zookeeper_part_uniq_node = fs::path(zookeeper_part_node) / id; + String zookeeper_part_uniq_node = fs::path(zc_zookeeper_path) / id; String zookeeper_node = fs::path(zookeeper_part_uniq_node) / replica_name_; LOG_TRACE(logger, "Remove zookeeper lock {}", zookeeper_node); @@ -7210,15 +7243,28 @@ bool StorageReplicatedMergeTree::unlockSharedDataById(String id, const String & continue; } - zookeeper_ptr->tryRemove(zookeeper_part_uniq_node); + auto e = zookeeper_ptr->tryRemove(zookeeper_part_uniq_node); + + LOG_TRACE(logger, "Remove parent zookeeper lock {} : {}", zookeeper_part_uniq_node, e != Coordination::Error::ZNOTEMPTY); /// Even when we have lock with same part name, but with different uniq, we can remove files on S3 children.clear(); + String zookeeper_part_node = fs::path(zookeeper_part_uniq_node).parent_path(); zookeeper_ptr->tryGetChildren(zookeeper_part_node, children); if (children.empty()) { /// Cleanup after last uniq removing - zookeeper_ptr->tryRemove(zookeeper_part_node); + e = zookeeper_ptr->tryRemove(zookeeper_part_node); + + LOG_TRACE(logger, "Remove parent zookeeper lock {} : {}", zookeeper_part_node, e != Coordination::Error::ZNOTEMPTY); + } + else + { + LOG_TRACE(logger, "Can't remove parent zookeeper lock {} : {}", zookeeper_part_node, children.size()); + for (auto & c : children) + { + LOG_TRACE(logger, "Child node {}", c); + } } } @@ -7255,22 +7301,19 @@ String StorageReplicatedMergeTree::getSharedDataReplica( if (!zookeeper) return best_replica; - String zero_copy = fmt::format("zero_copy_{}", toString(disk_type)); - - Strings zc_zookeeper_paths = getZeroCopyRootPath(*getDefaultSettings(), is_zero_copy_in_compatible_mode ? zookeeper_path : ""); + Strings zc_zookeeper_paths = getZeroCopyPartPath(*getDefaultSettings(), disk_type, getTableUniqID(), part.name, + is_zero_copy_in_compatible_mode ? zookeeper_path : ""); std::set replicas; for (const auto & zc_zookeeper_path : zc_zookeeper_paths) { - String zookeeper_part_node = fs::path(zc_zookeeper_path) / zero_copy / "shared" / part.name; - Strings ids; - zookeeper->tryGetChildren(zookeeper_part_node, ids); + zookeeper->tryGetChildren(zc_zookeeper_path, ids); for (const auto & id : ids) { - String zookeeper_part_uniq_node = fs::path(zookeeper_part_node) / id; + String zookeeper_part_uniq_node = fs::path(zc_zookeeper_path) / id; Strings id_replicas; zookeeper->tryGetChildren(zookeeper_part_uniq_node, id_replicas); LOG_TRACE(log, "Found zookeper replicas for {}: {}", zookeeper_part_uniq_node, id_replicas.size()); @@ -7329,13 +7372,20 @@ String StorageReplicatedMergeTree::getSharedDataReplica( } -Strings StorageReplicatedMergeTree::getZeroCopyRootPath(const MergeTreeSettings & settings, const String & zookeeper_path_old) +Strings StorageReplicatedMergeTree::getZeroCopyPartPath(const MergeTreeSettings & settings, DiskType disk_type, const String & table_uuid, + const String & part_name, const String & zookeeper_path_old) { Strings res; - res.push_back(settings.remote_fs_zero_copy_zookeeper_path); - if (!zookeeper_path_old.empty()) /// Compatibility mode for cluster with old and new versions - res.push_back(zookeeper_path_old); + String zero_copy = fmt::format("zero_copy_{}", toString(disk_type)); + + String new_path = fs::path(settings.remote_fs_zero_copy_zookeeper_path.toString()) / zero_copy / table_uuid / part_name; + res.push_back(new_path); + if (!zookeeper_path_old.empty()) + { /// Compatibility mode for cluster with old and new versions + String old_path = fs::path(zookeeper_path_old) / zero_copy / "shared" / part_name; + res.push_back(old_path); + } return res; } @@ -7610,7 +7660,7 @@ void StorageReplicatedMergeTree::convertZeroCopySchema() auto shard_root_v1 = fs::path(zookeeper_path) / zero_copy / "shared"; auto shard_root_v2 = fs::path(getDefaultSettings()->remote_fs_zero_copy_zookeeper_path.toString()) - / zero_copy / "shared"; + / zero_copy / getTableUniqID(); auto old_shard_root = revert_to_version == 1 ? shard_root_v2 : shard_root_v1; auto new_shard_root = revert_to_version == 1 ? shard_root_v1 : shard_root_v2; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index b6b6076206e..94692c9836c 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -243,7 +244,7 @@ public: /// Unlock shared data part in zookeeper by part id /// Return true if data unlocked /// Return false if data is still used by another node - static bool unlockSharedDataById(String id, const String & part_name, const String & replica_name_, + static bool unlockSharedDataById(String id, const String & table_uuid, const String & part_name, const String & replica_name_, DiskPtr disk, zkutil::ZooKeeperPtr zookeeper_, const MergeTreeSettings & settings, Poco::Logger * logger, const String & zookeeper_path_old); @@ -277,6 +278,8 @@ public: virtual String getZooKeeperName() const override { return zookeeper_name; } virtual String getZooKeeperPath() const override { return zookeeper_path; } + virtual String getTableUniqID() const override; + private: std::atomic_bool are_restoring_replica {false}; @@ -741,7 +744,8 @@ private: PartitionBlockNumbersHolder allocateBlockNumbersInAffectedPartitions( const MutationCommands & commands, ContextPtr query_context, const zkutil::ZooKeeperPtr & zookeeper) const; - static Strings getZeroCopyRootPath(const MergeTreeSettings & settings, const String & zookeeper_path_old); + static Strings getZeroCopyPartPath(const MergeTreeSettings & settings, DiskType disk_type, const String & table_uuid, + const String & part_name, const String & zookeeper_path_old); /// Upgrave zero-copy version /// version 1 - lock for shared part inside table node in ZooKeeper @@ -772,6 +776,10 @@ protected: std::unique_ptr settings_, bool has_force_restore_data_flag, bool allow_renaming_); + + /// Global ID, synced via ZooKeeper between replicas + /// mutable because can getted from ZooKeeper when required + mutable UUID table_global_id; }; String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info); diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index 160099a321d..c705bdf1863 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -270,6 +270,7 @@ def test_s3_zero_copy_with_ttl_delete(cluster, large_data, iterations): def wait_mutations(node, table, seconds): + time.sleep(1) while seconds > 0: seconds -= 1 mutations = node.query(f"SELECT count() FROM system.mutations WHERE table='{table}' AND is_done=0") diff --git a/tests/integration/test_s3_zero_copy_upgrade/test.py b/tests/integration/test_s3_zero_copy_upgrade/test.py index 08f39b98787..7f7c4f9c607 100644 --- a/tests/integration/test_s3_zero_copy_upgrade/test.py +++ b/tests/integration/test_s3_zero_copy_upgrade/test.py @@ -99,7 +99,28 @@ def get_ids(zookeeper, zk_path): return ids +def get_ids_new(zookeeper, zk_path): + ids = [] + + try: + zk_tables = zookeeper.get_children(zk_path) + for zk_table in zk_tables: + zk_nodes = zookeeper.get_children(zk_path + "/" + zk_table) + for zk_node in zk_nodes: + part_ids = zookeeper.get_children(zk_path + "/" + zk_table + "/" + zk_node) + assert len(part_ids) == 1 + ids += part_ids + except kazoo.exceptions.NoNodeError: + ids = [] + pass + + ids = list(set(ids)) + ids.sort() + return ids + + def wait_mutations(node, table, seconds): + time.sleep(1) while seconds > 0: seconds -= 1 mutations = node.query(f"SELECT count() FROM system.mutations WHERE table='{table}' AND is_done=0") @@ -134,22 +155,22 @@ def test_s3_zero_copy_version_upgrade(cluster): wait_for_count_in_table(node2, "convert_test", 3, 10) zk_old_path = "/clickhouse/tables/convert_test/zero_copy_s3/shared" - zk_path = "/clickhouse/zero_copy/zero_copy_s3/shared" + zk_path = "/clickhouse/zero_copy/zero_copy_s3" part_ids = get_ids(zookeeper, zk_old_path) assert len(part_ids) == 3 - ids = get_ids(zookeeper, zk_path) + ids = get_ids_new(zookeeper, zk_path) assert len(ids) == 0 node1.restart_with_latest_version() - ids = get_ids(zookeeper, zk_path) + ids = get_ids_new(zookeeper, zk_path) assert ids == part_ids old_ids = get_ids(zookeeper, zk_old_path) assert old_ids == part_ids node1.restart_clickhouse() - ids = get_ids(zookeeper, zk_path) + ids = get_ids_new(zookeeper, zk_path) assert ids == part_ids old_ids = get_ids(zookeeper, zk_old_path) assert old_ids == part_ids @@ -161,7 +182,7 @@ def test_s3_zero_copy_version_upgrade(cluster): wait_for_count_in_table(node1, "convert_test", 5, 10) wait_for_count_in_table(node2, "convert_test", 5, 10) - part_ids = get_ids(zookeeper, zk_path) + part_ids = get_ids_new(zookeeper, zk_path) assert len(part_ids) == 5 old_ids = get_ids(zookeeper, zk_old_path) assert old_ids == part_ids @@ -177,17 +198,17 @@ def test_s3_zero_copy_version_upgrade(cluster): wait_mutations(node1, "convert_test", 10) wait_mutations(node2, "convert_test", 10) - part_ids = get_ids(zookeeper, zk_path) + part_ids = get_ids_new(zookeeper, zk_path) assert len(part_ids) == 4 node1.query("ALTER TABLE convert_test DROP DETACHED PARTITION 'convert_part_2'", settings={"allow_drop_detached": 1}) wait_mutations(node1, "convert_test", 10) - part_ids = get_ids(zookeeper, zk_path) + part_ids = get_ids_new(zookeeper, zk_path) assert len(part_ids) == 3 node2.restart_with_latest_version() - ids = get_ids(zookeeper, zk_path) + ids = get_ids_new(zookeeper, zk_path) assert ids == part_ids old_ids = get_ids(zookeeper, zk_old_path) assert len(old_ids) == 0 From d9c13900f58a7de012551e8721b23e372974788c Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 20 Dec 2021 14:31:56 +0000 Subject: [PATCH 0201/1260] Fix build + style --- src/Storages/HDFS/StorageHDFS.cpp | 4 +--- src/Storages/HDFS/StorageHDFS.h | 4 ++-- src/Storages/HDFS/StorageHDFSCluster.cpp | 2 +- src/TableFunctions/TableFunctionHDFSCluster.cpp | 4 +--- 4 files changed, 5 insertions(+), 9 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 68a4b835bfc..ce9fc5c8129 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -97,10 +97,8 @@ public: HDFSFSPtr fs = createHDFSFS(builder.get()); std::lock_guard lock(mutex); uris = LSWithRegexpMatching("/", fs, path_from_uri); - for(size_t i=0; i; static Block getHeader( - const StorageMetadataPtr & metadata_snapshot, - bool need_path_column, + const StorageMetadataPtr & metadata_snapshot, + bool need_path_column, bool need_file_column); static Block getBlockForSource( diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index f2f3c77b3b9..ba1cc045fbf 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -112,7 +112,7 @@ Pipe StorageHDFSCluster::read( scalars, Tables(), processed_stage, - callback); + RemoteQueryExecutor::Extension{.task_iterator = callback}); pipes.emplace_back(std::make_shared(remote_query_executor, add_agg_info, false)); } diff --git a/src/TableFunctions/TableFunctionHDFSCluster.cpp b/src/TableFunctions/TableFunctionHDFSCluster.cpp index 241e52d4d2c..ca1ac6a11cd 100644 --- a/src/TableFunctions/TableFunctionHDFSCluster.cpp +++ b/src/TableFunctions/TableFunctionHDFSCluster.cpp @@ -61,10 +61,8 @@ void TableFunctionHDFSCluster::parseArguments(const ASTPtr & ast_function, Conte uri = args[1]->as().value.safeGet(); format = args[2]->as().value.safeGet(); structure = args[3]->as().value.safeGet(); - if(args.size()>=5) - { + if (args.size() >= 5) compression_method = args[4]->as().value.safeGet(); - } } From 79086764ff9acfa73778f2b894d0edb73c2e3670 Mon Sep 17 00:00:00 2001 From: Alexey Date: Mon, 20 Dec 2021 15:52:41 +0000 Subject: [PATCH 0202/1260] new interserver_http_credentials description variant --- .../server-configuration-parameters/settings.md | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index b728a0303f6..6a6d54b9f5a 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -435,15 +435,22 @@ Similar to `interserver_http_host`, except that this hostname can be used by oth ## interserver_http_credentials {#server-settings-interserver-http-credentials} -The username and password used to authenticate during [replication](../../engines/table-engines/mergetree-family/replication.md) with the Replicated\* engines. These credentials are used only for communication between replicas and are unrelated to credentials for ClickHouse clients. The server is checking these credentials for connecting replicas and use the same credentials when connecting to other replicas. So, these credentials should be set the same for all replicas in a cluster. -By default, the authentication is not used. +A username and a password used to connect to other servers during [replication](../../engines/table-engines/mergetree-family/replication.md). +Also the server expects these credentials for incoming connections from other replicas. So, these credentials should be the same for all replicas in a cluster. + +By default, if `interserver_http_credentials` section is omitted, authentication is not used during replication. + +These credentials are used for communication between replicas and they are unrelated to credentials for ClickHouse clients. + +!!! note "Note" + `interserver_http_credentials` do not relate to credentials for ClickHouse clients. !!! note "Note" These credentials are common for replication through `HTTP` and `HTTPS`. -This section contains the following parameters: +The section contains the following parameters: -- `user` — User name. +- `user` — Username. - `password` — Password. - `allow_empty` — If `true`, then other replicas are allowed to connect without authentication even if credentials are set. If `false`, then connections without authentication are refused. Default value: `false`. - `old` — Contains old `user` and `password` used during credential rotation. Several `old` sections can be specified. @@ -464,7 +471,7 @@ To enable authentication, set `interserver_http_credentials.allow_empty` to `tru After configuring all replicas set `allow_empty` to `false` or remove this setting. It makes authentication with new credentials mandatory. -To change existing credentials, move the user name and the password to `interserver_http_credentials.old` section and update `user` and `password` with new values. At this point the server uses new credentials to connect to other replicas and accepts connections with either new or old credentials. +To change existing credentials, move the username and the password to `interserver_http_credentials.old` section and update `user` and `password` with new values. At this point the server uses new credentials to connect to other replicas and accepts connections with either new or old credentials. ``` xml From c42f03beb615a3cef5600c119013bc5d7209165b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 20 Dec 2021 17:55:40 +0100 Subject: [PATCH 0203/1260] =?UTF-8?q?Use=20ClickHouse-Extras=20for=20boost?= =?UTF-8?q?=C2=B7?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .gitmodules | 2 +- contrib/boost | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index a5482f28e46..676e8d3d9aa 100644 --- a/.gitmodules +++ b/.gitmodules @@ -48,7 +48,7 @@ branch = v3.13.0.1 [submodule "contrib/boost"] path = contrib/boost - url = https://github.com/Algunenano/boost.git + url = https://github.com/ClickHouse-Extras/boost.git [submodule "contrib/base64"] path = contrib/base64 url = https://github.com/ClickHouse-Extras/Turbo-Base64.git diff --git a/contrib/boost b/contrib/boost index 79b2fdaa512..c0807e83f28 160000 --- a/contrib/boost +++ b/contrib/boost @@ -1 +1 @@ -Subproject commit 79b2fdaa512fde9683ecbaaf1bb86601f73d1135 +Subproject commit c0807e83f2824e8dd67a15b355496a9b784cdcd5 From 36164a72c20f91f3a567831a83e04dcd4efa2200 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 20 Dec 2021 18:17:10 +0100 Subject: [PATCH 0204/1260] Missing include --- src/Columns/ColumnSparse.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index a4b770f8f6b..e9bdc3971c0 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -8,6 +8,7 @@ #include #include +#include namespace DB { From c724b074aee8914b9dbd95ae891fd61bb5b9a100 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Mon, 20 Dec 2021 20:23:25 +0300 Subject: [PATCH 0205/1260] Remove zero-copy version converter --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 179 +------------- src/Storages/StorageReplicatedMergeTree.h | 11 - .../test_s3_zero_copy_upgrade/__init__.py | 0 .../configs/config.d/s3.xml | 50 ---- .../test_s3_zero_copy_upgrade/test.py | 219 ------------------ 6 files changed, 5 insertions(+), 456 deletions(-) delete mode 100644 tests/integration/test_s3_zero_copy_upgrade/__init__.py delete mode 100644 tests/integration/test_s3_zero_copy_upgrade/configs/config.d/s3.xml delete mode 100644 tests/integration/test_s3_zero_copy_upgrade/test.py diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 03e7a562218..e2fc6246808 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -126,7 +126,7 @@ struct Settings; M(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \ M(Bool, allow_remote_fs_zero_copy_replication, true, "Allow Zero-copy replication over remote fs.", 0) \ M(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for Zero-copy table-independet info.", 0) \ - M(UInt64, need_revert_zero_copy_version, 0, "Revert Zero-copy to old version", 0) \ + M(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during convertion process.", 0) \ M(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm.", 0) \ M(Bool, assign_part_uuids, false, "Generate UUIDs for parts. Before enabling check that all replicas support new format.", 0) \ M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited. This setting is the default that can be overridden by the query-level setting with the same name.", 0) \ diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 56bc3f933a6..4c554c33e78 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -156,8 +156,6 @@ static const auto QUEUE_UPDATE_ERROR_SLEEP_MS = 1 * 1000; static const auto MUTATIONS_FINALIZING_SLEEP_MS = 1 * 1000; static const auto MUTATIONS_FINALIZING_IDLE_SLEEP_MS = 5 * 1000; -static const int CURRENT_ZERO_COPY_VERSION = 2; - void StorageReplicatedMergeTree::setZooKeeper() { /// Every ReplicatedMergeTree table is using only one ZooKeeper session. @@ -4111,10 +4109,6 @@ void StorageReplicatedMergeTree::startup() assert(prev_ptr == nullptr); getContext()->getInterserverIOHandler().addEndpoint(data_parts_exchange_ptr->getId(replica_path), data_parts_exchange_ptr); - convertZeroCopySchema(); - is_zero_copy_in_compatible_mode = isZeroCopySchemaInCompatibleMode(); - cleanupOldZeroCopySchema(); - /// In this thread replica will be activated. restarting_thread.start(); @@ -7176,7 +7170,7 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part) boost::replace_all(id, "/", "_"); Strings zc_zookeeper_paths = getZeroCopyPartPath(*getDefaultSettings(), disk->getType(), getTableUniqID(), - part.name, is_zero_copy_in_compatible_mode ? zookeeper_path : ""); + part.name, zookeeper_path); for (const auto & zc_zookeeper_path : zc_zookeeper_paths) { String zookeeper_node = fs::path(zc_zookeeper_path) / id / replica_name; @@ -7210,7 +7204,7 @@ bool StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & par return false; return unlockSharedDataById(part.getUniqueId(), getTableUniqID(), name, replica_name, disk, zookeeper, *getDefaultSettings(), log, - is_zero_copy_in_compatible_mode ? zookeeper_path : String("")); + zookeeper_path); } @@ -7302,7 +7296,7 @@ String StorageReplicatedMergeTree::getSharedDataReplica( return best_replica; Strings zc_zookeeper_paths = getZeroCopyPartPath(*getDefaultSettings(), disk_type, getTableUniqID(), part.name, - is_zero_copy_in_compatible_mode ? zookeeper_path : ""); + zookeeper_path); std::set replicas; @@ -7381,7 +7375,7 @@ Strings StorageReplicatedMergeTree::getZeroCopyPartPath(const MergeTreeSettings String new_path = fs::path(settings.remote_fs_zero_copy_zookeeper_path.toString()) / zero_copy / table_uuid / part_name; res.push_back(new_path); - if (!zookeeper_path_old.empty()) + if (settings.remote_fs_zero_copy_path_compatible_mode && !zookeeper_path_old.empty()) { /// Compatibility mode for cluster with old and new versions String old_path = fs::path(zookeeper_path_old) / zero_copy / "shared" / part_name; res.push_back(old_path); @@ -7611,148 +7605,6 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP } -void StorageReplicatedMergeTree::convertZeroCopySchema() -{ - if (!current_zookeeper) - return; - - int zero_copy_version = 1; - - auto version_path = fs::path(getDefaultSettings()->remote_fs_zero_copy_zookeeper_path.toString()) / "version" / replica_name; - - if (current_zookeeper->exists(version_path)) - zero_copy_version = parse(current_zookeeper->get(version_path)); - - /// Emergency parameter to restore zero-copy marks on old paths - int revert_to_version = getDefaultSettings()->need_revert_zero_copy_version; - - if (!revert_to_version && zero_copy_version >= CURRENT_ZERO_COPY_VERSION) - return; - - if (revert_to_version && zero_copy_version <= revert_to_version) - return; - - int required_zero_copy_version = revert_to_version ? revert_to_version : CURRENT_ZERO_COPY_VERSION; - - auto storage_policy = getStoragePolicy(); - if (!storage_policy) - return; - - auto disks = storage_policy->getDisks(); - - std::set disk_types; - - for (const auto & disk : disks) - if (disk->supportZeroCopyReplication()) - disk_types.insert(toString(disk->getType())); - - if (disk_types.empty()) - return; - - LOG_INFO(log, "Convert zero_copy version from {} to {} for {}", zero_copy_version, required_zero_copy_version, - version_path.string()); - - unsigned long converted_part_counter = 0; - - for (auto const & disk_type : disk_types) - { - String zero_copy = fmt::format("zero_copy_{}", disk_type); - - auto shard_root_v1 = fs::path(zookeeper_path) / zero_copy / "shared"; - auto shard_root_v2 = fs::path(getDefaultSettings()->remote_fs_zero_copy_zookeeper_path.toString()) - / zero_copy / getTableUniqID(); - - auto old_shard_root = revert_to_version == 1 ? shard_root_v2 : shard_root_v1; - auto new_shard_root = revert_to_version == 1 ? shard_root_v1 : shard_root_v2; - - Strings parts; - current_zookeeper->tryGetChildren(old_shard_root, parts); - - for (const auto & part_name : parts) - { - auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); - auto part = getPartIfExists(part_info, {MergeTreeDataPartState::Committed}); - - if (part) - { /// Do not move lost locks - Strings ids; - current_zookeeper->tryGetChildren(old_shard_root / part_name, ids); - for (const auto & id : ids) - { - if (current_zookeeper->exists(old_shard_root / part_name / id / replica_name)) - { - auto zookeeper_node = new_shard_root / part_name / id / replica_name; - createZeroCopyLockNode(current_zookeeper, zookeeper_node.string()); - ++converted_part_counter; - } - } - } - } - } - - current_zookeeper->createAncestors(version_path); - - current_zookeeper->createOrUpdate(version_path, std::to_string(required_zero_copy_version), - zkutil::CreateMode::Persistent); - - current_zookeeper->createOrUpdate(version_path / "cleanup_required", std::to_string(zero_copy_version), - zkutil::CreateMode::Persistent); - - LOG_INFO(log, "Convert zero_copy version from {} to {} for {} complete, converted {} locks", zero_copy_version, required_zero_copy_version, - version_path.string(), converted_part_counter); -} - - -void StorageReplicatedMergeTree::cleanupOldZeroCopySchema() -{ - if (is_zero_copy_in_compatible_mode) - return; /// Some replicas have old version - - if (!current_zookeeper) - return; - - auto old_version_path = fs::path(getDefaultSettings()->remote_fs_zero_copy_zookeeper_path.toString()) / "version" / replica_name / "cleanup_required"; - - if (!current_zookeeper->exists(old_version_path)) - return; - - auto zero_copy_version = parse(current_zookeeper->get(old_version_path)); - - if (zero_copy_version == 1) - { - auto storage_policy = getStoragePolicy(); - if (!storage_policy) - return; - - auto disks = storage_policy->getDisks(); - - std::set disk_types; - - for (const auto & disk : disks) - if (disk->supportZeroCopyReplication()) - disk_types.insert(toString(disk->getType())); - - if (disk_types.empty()) - return; - - LOG_INFO(log, "Cleanup zero_copy version {}", zero_copy_version); - - for (auto const & disk_type : disk_types) - { - String zero_copy = fmt::format("zero_copy_{}", disk_type); - - auto old_shard_root = fs::path(zookeeper_path) / zero_copy / "shared"; - - current_zookeeper->tryRemoveRecursive(old_shard_root); - } - - current_zookeeper->remove(old_version_path); - - LOG_INFO(log, "Cleanup zero_copy version {} complete", zero_copy_version); - } -} - - void StorageReplicatedMergeTree::createZeroCopyLockNode(const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_node) { /// In rare case other replica can remove path between createAncestors and createIfNotExists @@ -7776,27 +7628,4 @@ void StorageReplicatedMergeTree::createZeroCopyLockNode(const zkutil::ZooKeeperP } -bool StorageReplicatedMergeTree::isZeroCopySchemaInCompatibleMode() const -{ - if (!current_zookeeper) - return false; - - auto version_root_path = fs::path(getDefaultSettings()->remote_fs_zero_copy_zookeeper_path.toString()) / "version"; - - Strings replicas = current_zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); - - for (const auto & replica : replicas) - { - if (!current_zookeeper->exists(version_root_path / replica)) - return true; - int zero_copy_version = parse(current_zookeeper->get(version_root_path / replica)); - if (zero_copy_version < CURRENT_ZERO_COPY_VERSION) - return true; - /// If version is greater that current then other replica has new version. - /// In that case other replica with new version should be in compatible mode. - } - - return false; -} - } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 94692c9836c..9feca393506 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -747,19 +747,8 @@ private: static Strings getZeroCopyPartPath(const MergeTreeSettings & settings, DiskType disk_type, const String & table_uuid, const String & part_name, const String & zookeeper_path_old); - /// Upgrave zero-copy version - /// version 1 - lock for shared part inside table node in ZooKeeper - /// version 2 - lock for shared part in separate node - void convertZeroCopySchema(); - - void cleanupOldZeroCopySchema(); - static void createZeroCopyLockNode(const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_node); - bool isZeroCopySchemaInCompatibleMode() const; - - bool is_zero_copy_in_compatible_mode = false; - protected: /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. */ diff --git a/tests/integration/test_s3_zero_copy_upgrade/__init__.py b/tests/integration/test_s3_zero_copy_upgrade/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_s3_zero_copy_upgrade/configs/config.d/s3.xml b/tests/integration/test_s3_zero_copy_upgrade/configs/config.d/s3.xml deleted file mode 100644 index cbff39c23ce..00000000000 --- a/tests/integration/test_s3_zero_copy_upgrade/configs/config.d/s3.xml +++ /dev/null @@ -1,50 +0,0 @@ - - - - - - s3 - http://minio1:9001/root/data/ - minio - minio123 - true - - - - - -
- s3 -
-
-
-
-
- - - 1024 - 1 - - - - - - - node1 - 9000 - - - - - node2 - 9000 - - - - - - - test_cluster - - -
diff --git a/tests/integration/test_s3_zero_copy_upgrade/test.py b/tests/integration/test_s3_zero_copy_upgrade/test.py deleted file mode 100644 index 7f7c4f9c607..00000000000 --- a/tests/integration/test_s3_zero_copy_upgrade/test.py +++ /dev/null @@ -1,219 +0,0 @@ -import logging -import time -import kazoo - -import pytest -from helpers.cluster import ClickHouseCluster - -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - - -@pytest.fixture(scope="module") -def cluster(): - try: - cluster = ClickHouseCluster(__file__) - cluster.add_instance("node1", main_configs=["configs/config.d/s3.xml"], macros={'replica': '1'}, - image='yandex/clickhouse-server', tag='21.11.4.14', - stay_alive=True, with_installed_binary=True, - with_minio=True, - with_zookeeper=True) - cluster.add_instance("node2", main_configs=["configs/config.d/s3.xml"], macros={'replica': '2'}, - image='yandex/clickhouse-server', tag='21.11.4.14', - stay_alive=True, with_installed_binary=True, - with_minio=True, - with_zookeeper=True) - logging.info("Starting cluster...") - cluster.start() - logging.info("Cluster started") - - yield cluster - finally: - cluster.shutdown() - - -def get_large_objects_count(cluster, size=100, folder='data'): - minio = cluster.minio_client - counter = 0 - for obj in minio.list_objects(cluster.minio_bucket, '{}/'.format(folder)): - if obj.size is not None and obj.size >= size: - counter = counter + 1 - return counter - - -def check_objects_exisis(cluster, object_list, folder='data'): - minio = cluster.minio_client - for obj in object_list: - if obj: - minio.stat_object(cluster.minio_bucket, '{}/{}'.format(folder, obj)) - - -def check_objects_not_exisis(cluster, object_list, folder='data'): - minio = cluster.minio_client - for obj in object_list: - if obj: - try: - minio.stat_object(cluster.minio_bucket, '{}/{}'.format(folder, obj)) - except Exception as error: - assert "NoSuchKey" in str(error) - else: - assert False, "Object {} should not be exists".format(obj) - - -def wait_for_large_objects_count(cluster, expected, size=100, timeout=30): - while timeout > 0: - if get_large_objects_count(cluster, size=size) == expected: - return - timeout -= 1 - time.sleep(1) - assert get_large_objects_count(cluster, size=size) == expected - - -def wait_for_count_in_table(node, table, count, seconds): - while seconds > 0: - seconds -= 1 - res = node.query(f"SELECT count() FROM {table}") - if res == f"{count}\n": - return - time.sleep(1) - res = node.query(f"SELECT count() FROM {table}") - assert res == f"{count}\n" - - -def get_ids(zookeeper, zk_path): - ids = [] - - try: - zk_nodes = zookeeper.get_children(zk_path) - - for zk_node in zk_nodes: - part_ids = zookeeper.get_children(zk_path + "/" + zk_node) - assert len(part_ids) == 1 - ids += part_ids - except kazoo.exceptions.NoNodeError: - ids = [] - pass - - ids = list(set(ids)) - ids.sort() - return ids - - -def get_ids_new(zookeeper, zk_path): - ids = [] - - try: - zk_tables = zookeeper.get_children(zk_path) - for zk_table in zk_tables: - zk_nodes = zookeeper.get_children(zk_path + "/" + zk_table) - for zk_node in zk_nodes: - part_ids = zookeeper.get_children(zk_path + "/" + zk_table + "/" + zk_node) - assert len(part_ids) == 1 - ids += part_ids - except kazoo.exceptions.NoNodeError: - ids = [] - pass - - ids = list(set(ids)) - ids.sort() - return ids - - -def wait_mutations(node, table, seconds): - time.sleep(1) - while seconds > 0: - seconds -= 1 - mutations = node.query(f"SELECT count() FROM system.mutations WHERE table='{table}' AND is_done=0") - if mutations == '0\n': - return - time.sleep(1) - mutations = node.query(f"SELECT count() FROM system.mutations WHERE table='{table}' AND is_done=0") - assert mutations == '0\n' - - -def test_s3_zero_copy_version_upgrade(cluster): - node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] - - zookeeper = cluster.get_kazoo_client("zoo1") - - node1.query("DROP TABLE IF EXISTS convert_test NO DELAY") - node2.query("DROP TABLE IF EXISTS convert_test NO DELAY") - - node1.query( - """ - CREATE TABLE convert_test ON CLUSTER test_cluster (d String) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/convert_test', '{}') - ORDER BY d - PARTITION BY d - SETTINGS storage_policy='s3' - """ - .format('{replica}') - ) - - node1.query("INSERT INTO convert_test VALUES ('convert_part_1'),('convert_part_2'),('convert_part_3')") - wait_for_count_in_table(node2, "convert_test", 3, 10) - - zk_old_path = "/clickhouse/tables/convert_test/zero_copy_s3/shared" - zk_path = "/clickhouse/zero_copy/zero_copy_s3" - - part_ids = get_ids(zookeeper, zk_old_path) - assert len(part_ids) == 3 - - ids = get_ids_new(zookeeper, zk_path) - assert len(ids) == 0 - - node1.restart_with_latest_version() - ids = get_ids_new(zookeeper, zk_path) - assert ids == part_ids - old_ids = get_ids(zookeeper, zk_old_path) - assert old_ids == part_ids - - node1.restart_clickhouse() - ids = get_ids_new(zookeeper, zk_path) - assert ids == part_ids - old_ids = get_ids(zookeeper, zk_old_path) - assert old_ids == part_ids - - node1.query("INSERT INTO convert_test VALUES ('convert_part_4')") - wait_for_count_in_table(node1, "convert_test", 4, 10) - wait_for_count_in_table(node2, "convert_test", 4, 10) - node2.query("INSERT INTO convert_test VALUES ('convert_part_5')") - wait_for_count_in_table(node1, "convert_test", 5, 10) - wait_for_count_in_table(node2, "convert_test", 5, 10) - - part_ids = get_ids_new(zookeeper, zk_path) - assert len(part_ids) == 5 - old_ids = get_ids(zookeeper, zk_old_path) - assert old_ids == part_ids - - node1.query("ALTER TABLE convert_test DETACH PARTITION 'convert_part_1'") - node1.query("ALTER TABLE convert_test DROP DETACHED PARTITION 'convert_part_1'", settings={"allow_drop_detached": 1}) - wait_for_count_in_table(node1, "convert_test", 4, 10) - wait_for_count_in_table(node2, "convert_test", 4, 10) - node2.query("ALTER TABLE convert_test DETACH PARTITION 'convert_part_2'") - node2.query("ALTER TABLE convert_test DROP DETACHED PARTITION 'convert_part_2'", settings={"allow_drop_detached": 1}) - wait_for_count_in_table(node1, "convert_test", 3, 10) - wait_for_count_in_table(node2, "convert_test", 3, 10) - wait_mutations(node1, "convert_test", 10) - wait_mutations(node2, "convert_test", 10) - - part_ids = get_ids_new(zookeeper, zk_path) - assert len(part_ids) == 4 - - node1.query("ALTER TABLE convert_test DROP DETACHED PARTITION 'convert_part_2'", settings={"allow_drop_detached": 1}) - wait_mutations(node1, "convert_test", 10) - - part_ids = get_ids_new(zookeeper, zk_path) - assert len(part_ids) == 3 - - node2.restart_with_latest_version() - ids = get_ids_new(zookeeper, zk_path) - assert ids == part_ids - old_ids = get_ids(zookeeper, zk_old_path) - assert len(old_ids) == 0 - - node1.query("DROP TABLE IF EXISTS convert_test NO DELAY") - node2.query("DROP TABLE IF EXISTS convert_test NO DELAY") - - zookeeper.stop() From f39011150b55dba840c57a77e394634a5be77891 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Mon, 20 Dec 2021 20:24:33 +0300 Subject: [PATCH 0206/1260] Add zero-copy version converted script --- utils/zero_copy/zero_copy_schema_converter.py | 129 ++++++++++++++++++ 1 file changed, 129 insertions(+) create mode 100755 utils/zero_copy/zero_copy_schema_converter.py diff --git a/utils/zero_copy/zero_copy_schema_converter.py b/utils/zero_copy/zero_copy_schema_converter.py new file mode 100755 index 00000000000..922c7b342b4 --- /dev/null +++ b/utils/zero_copy/zero_copy_schema_converter.py @@ -0,0 +1,129 @@ +#!/usr/bin/env python3 +import argparse +import socket +import uuid +from kazoo.client import KazooClient + +def parse_args(): + """ + Parse command-line arguments. + """ + parser = argparse.ArgumentParser() + parser.add_argument('--hosts', default=socket.getfqdn() + ':2181', help='ZooKeeper hosts (host:port,host:port,...)') + parser.add_argument('-s', '--secure', default=False, action='store_true', help='Use secure connection') + parser.add_argument('--cert', default='', help='Client TLS certificate file') + parser.add_argument('--key', default='', help='Client TLS key file') + parser.add_argument('--ca', default='', help='Client TLS ca file') + parser.add_argument('-u', '--user', default='', help='ZooKeeper ACL user') + parser.add_argument('-p', '--password', default='', help='ZooKeeper ACL password') + parser.add_argument('-r', '--root', default='/clickhouse', help='ZooKeeper root path for ClickHouse') + parser.add_argument('-z', '--zcroot', default='zero_copy', help='ZooKeeper node for new zero-copy data') + parser.add_argument('--dryrun', default=False, action='store_true', help='Do not perfome any actions') + parser.add_argument('--cleanup', default=False, action='store_true', help='Clean old nodes') + parser.add_argument('-v', '--verbose', action='store_true', default=False, help='Verbose mode') + + return parser.parse_args() + + +# Several folders to euristic that zookeepr node is folder node +# May be false positive when someone creates set of tables with same paths +table_nodes = ['alter_partition_version', 'block_numbers', 'blocks', 'columns', 'leader_election'] +zc_nodes = ['zero_copy_s3', 'zero_copy_hdfs'] + + +def convert_node(client, args, path, zc_node): + base_path = f'{path}/{zc_node}/shared' + parts = client.get_children(base_path) + table_id_path = f'{path}/table_id' + table_id = '' + if client.exists(table_id_path): + table_id = client.get(table_id_path)[0].decode('UTF-8') + else: + table_id = str(uuid.uuid4()) + if args.verbose: + print(f'Make table_id "{table_id_path}" = "{table_id}"') + if not args.dryrun: + client.create(table_id_path, bytes(table_id, 'UTF-8')) + for part in parts: + part_path = f'{base_path}/{part}' + uniq_ids = client.get_children(part_path) + for uniq_id in uniq_ids: + uniq_path = f'{part_path}/{uniq_id}' + replicas = client.get_children(uniq_path) + for replica in replicas: + replica_path = f'{uniq_path}/{replica}' + new_path = f'{args.root}/{args.zcroot}/{zc_node}/{table_id}/{part}/{uniq_id}/{replica}' + if not client.exists(new_path): + if args.verbose: + print(f'Make node "{new_path}"') + if not args.dryrun: + client.ensure_path(f'{args.root}/{args.zcroot}/{zc_node}/{table_id}/{part}/{uniq_id}') + client.create(new_path, value=b'lock') + if args.cleanup: + if args.verbose: + print(f'Remove node "{replica_path}"') + if not args.dryrun: + client.delete(replica_path) + if args.cleanup and not args.dryrun: + client.delete(uniq_path) + if args.cleanup and not args.dryrun: + client.delete(part_path) + if args.cleanup and not args.dryrun: + client.delete(base_path) + client.delete(f'{path}/{zc_node}') + + +def convert_table(client, args, path, nodes): + print(f'Convert table nodes by path "{path}"') + for zc_node in zc_nodes: + if zc_node in nodes: + convert_node(client, args, path, zc_node) + + +def is_like_a_table(nodes): + for tn in table_nodes: + if tn not in nodes: + return False + return True + + +def scan_recursive(client, args, path): + nodes = client.get_children(path) + if is_like_a_table(nodes): + convert_table(client, args, path, nodes) + else: + for node in nodes: + scan_recursive(client, args, f'{path}/{node}') + + +def scan(client, args): + nodes = client.get_children(args.root) + for node in nodes: + if node != args.zcroot: + scan_recursive(client, args, f'{args.root}/{node}') + + +def get_client(args): + client = KazooClient(connection_retry=3, + command_retry=3, + timeout=1, + hosts=args.hosts, + use_ssl=args.secure, + certfile=args.cert, + keyfile=args.key, + ca=args.ca + ) + client.start() + if (args.user and args.password): + client.add_auth('digest', f'{args.user}:{args.password}') + return client + + +def main(): + args = parse_args() + client = get_client(args) + scan(client, args) + + +if __name__ == '__main__': + main() From 5a63f311fc592071665b0a6926741ade121ac684 Mon Sep 17 00:00:00 2001 From: Alexey Date: Mon, 20 Dec 2021 20:05:55 +0000 Subject: [PATCH 0207/1260] en edits --- .../server-configuration-parameters/settings.md | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 6a6d54b9f5a..9b0edddb4b7 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -435,18 +435,15 @@ Similar to `interserver_http_host`, except that this hostname can be used by oth ## interserver_http_credentials {#server-settings-interserver-http-credentials} -A username and a password used to connect to other servers during [replication](../../engines/table-engines/mergetree-family/replication.md). -Also the server expects these credentials for incoming connections from other replicas. So, these credentials should be the same for all replicas in a cluster. +A username and a password used to connect to other servers during [replication](../../engines/table-engines/mergetree-family/replication.md). Also the server authenticate other replicas using these credentials. So, `interserver_http_credentials` must be the same for all replicas in a cluster. By default, if `interserver_http_credentials` section is omitted, authentication is not used during replication. -These credentials are used for communication between replicas and they are unrelated to credentials for ClickHouse clients. +!!! note "Note" + `interserver_http_credentials` settings do not relate to a ClickHouse client credentials [configuration](../../interfaces/cli.md#configuration_files). !!! note "Note" - `interserver_http_credentials` do not relate to credentials for ClickHouse clients. - -!!! note "Note" - These credentials are common for replication through `HTTP` and `HTTPS`. + These credentials are common for replication via `HTTP` and `HTTPS`. The section contains the following parameters: From 2c67516432665b703a4ff26b978e6e2e4a07d83e Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 21 Dec 2021 09:44:04 +0800 Subject: [PATCH 0208/1260] 1. fixed code style 2. fixed bugs in lru cache --- src/Common/LRUCache.h | 12 ++++++++---- src/Storages/IRemoteFileMetadata.cpp | 7 ------- src/Storages/IRemoteFileMetadata.h | 2 +- src/Storages/RemoteFileMetadataFactory.cpp | 2 +- 4 files changed, 10 insertions(+), 13 deletions(-) delete mode 100644 src/Storages/IRemoteFileMetadata.cpp diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 183004cb8e5..81604ee6be5 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -331,13 +331,14 @@ private: } else { - if (!evict_policy.canRelease(*cell.value)) + if (cell.value && !evict_policy.canRelease(*cell.value)) { // the old value is refered by someone, cannot release now // in default policy, it is always true. return false; } - evict_policy.release(*cell.value); // release the old value. this action is empty in default policy. + if (cell.value) + evict_policy.release(*cell.value); // release the old value. this action is empty in default policy. current_size -= cell.size; queue.splice(queue.end(), queue, cell.queue_iterator); } @@ -369,12 +370,15 @@ private: } const auto & cell = it->second; - auto can_evict = evict_policy.canRelease(*cell.value);// in default, it is true + bool can_evict = true; + if (cell.value) + can_evict = evict_policy.canRelease(*cell.value);// in default, it is true if (can_evict) { // always call release() before erasing an element // in default, it's an empty action - evict_policy.release(*cell.value); + if (cell.value) + evict_policy.release(*cell.value); current_size -= cell.size; current_weight_lost += cell.size; diff --git a/src/Storages/IRemoteFileMetadata.cpp b/src/Storages/IRemoteFileMetadata.cpp deleted file mode 100644 index 9d2a93ab185..00000000000 --- a/src/Storages/IRemoteFileMetadata.cpp +++ /dev/null @@ -1,7 +0,0 @@ -#include -namespace DB -{ - -IRemoteFileMetadata::~IRemoteFileMetadata() {} - -} diff --git a/src/Storages/IRemoteFileMetadata.h b/src/Storages/IRemoteFileMetadata.h index 383f9f01927..a2b4ac2a9be 100644 --- a/src/Storages/IRemoteFileMetadata.h +++ b/src/Storages/IRemoteFileMetadata.h @@ -19,7 +19,7 @@ public: ,last_modification_timestamp(last_modification_timestamp_) { } - virtual ~IRemoteFileMetadata(); + virtual ~IRemoteFileMetadata() = default; virtual String getName() const = 0; //class name // methods for basic information inline size_t getFileSize() const { return file_size; } diff --git a/src/Storages/RemoteFileMetadataFactory.cpp b/src/Storages/RemoteFileMetadataFactory.cpp index fac1e5648d6..fba93bb9dc2 100644 --- a/src/Storages/RemoteFileMetadataFactory.cpp +++ b/src/Storages/RemoteFileMetadataFactory.cpp @@ -29,7 +29,7 @@ void RemoteFileMetadataFactory::registerRemoteFileMatadataCreator(const String & auto it = class_creators.find(name); if (it != class_creators.end()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Class ({}) has been registered. It is a fatal error.", name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Metadata class ({}) has already been registered.", name); } class_creators[name] = creator; } From 2e2354be401ef9fa168d4412a0890d8c127f35c0 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 21 Dec 2021 10:39:59 +0800 Subject: [PATCH 0209/1260] 1. add tryDel and getOrTrySet in LRUCache 2. fixed bugs in LRUCache for evicting elements --- src/Common/LRUCache.h | 52 ++++++++++++++++++++++++++------ src/Storages/Hive/HiveCommon.cpp | 8 ++++- 2 files changed, 50 insertions(+), 10 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 81604ee6be5..98d14a31a21 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -85,6 +85,16 @@ public: return setImpl(key, mapped, lock); } + /// Returns std::pair of the cached value and a bool indicating whether the value was produced during this call. + template + std::pair getOrSet(const Key & key, LoadFunc && load_func) + { + MappedPtr value = nullptr; + bool is_value_loaded = false, is_value_updated = false; + std::tie(value, is_value_loaded, is_value_updated) = getOrTrySet(key, std::move(load_func)); + return std::make_pair(value, is_value_loaded); + } + /// If the value for the key is in the cache, returns it. If it is not, calls load_func() to /// produce it, saves the result in the cache and returns it. /// Only one of several concurrent threads calling getOrSet() will call load_func(), @@ -92,9 +102,12 @@ public: /// Exceptions occurring in load_func will be propagated to the caller. Another thread from the /// set of concurrent threads will then try to call its load_func etc. /// - /// Returns std::pair of the cached value and a bool indicating whether the value was produced during this call. + /// return std::tuple is , where + /// - is_value_loaded indicates whether the value was produce during this call + /// - is_value_updated indicates whether the value is updated in the cache when is_value_loaded = true. + /// if is_value_loaded = false, is_value_updated = false template - std::pair getOrSet(const Key & key, LoadFunc && load_func) + std::tuple getOrTrySet(const Key &key, LoadFunc && load_func) { InsertTokenHolder token_holder; { @@ -104,7 +117,7 @@ public: if (val) { ++hits; - return std::make_pair(val, false); + return {val, false, false}; } auto & token = insert_tokens[key]; @@ -124,7 +137,7 @@ public: { /// Another thread already produced the value while we waited for token->mutex. ++hits; - return std::make_pair(token->value, false); + return {token->value, false, false}; } ++misses; @@ -134,20 +147,41 @@ public: /// Insert the new value only if the token is still in present in insert_tokens. /// (The token may be absent because of a concurrent reset() call). - bool result = false; + bool is_value_loaded = false; + bool is_value_updated = true; auto token_it = insert_tokens.find(key); if (token_it != insert_tokens.end() && token_it->second.get() == token) { // setImpl() may fail, but the final behavior seems not be affected // next call of getOrSet() will still call load_func() - setImpl(key, token->value, cache_lock); - result = true; + is_value_updated = setImpl(key, token->value, cache_lock); + is_value_loaded = true; } if (!token->cleaned_up) token_holder.cleanup(token_lock, cache_lock); + + return {token->value, is_value_loaded, is_value_updated}; + } - return std::make_pair(token->value, result); + /// If key is not in cache or the element can be released, return is true. otherwise, return is false + bool tryDel(const Key &key) + { + std::lock_guard loc(mutex); + auto it = cells.find(key); + if (it == cells.end()) + return true; + auto & cell = it->second; + if (cell.value) + { + if (!evict_policy.canRelease(*cell.value)) + return false; + evict_policy.release(*cell.value); + } + current_size -= cell.size; + cells.erase(it); + queue.erase(cell.queue_iterator); + return true; } void getStats(size_t & out_hits, size_t & out_misses) const @@ -384,7 +418,7 @@ private: current_weight_lost += cell.size; cells.erase(it); - queue.pop_front(); + key_it = queue.erase(key_it); --queue_size; } else diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 68aba13b726..a61e7515236 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -11,6 +11,7 @@ namespace ErrorCodes { extern const int NO_HIVEMETASTORE; extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; } bool HiveMetastoreClient::shouldUpdateTableMetadata( @@ -108,7 +109,12 @@ void HiveMetastoreClient::clearTableMetadata(const String & db_name, const Strin std::lock_guard lock{mutex}; HiveTableMetadataPtr metadata = table_metadata_cache.get(cache_key); if (metadata) - table_metadata_cache.set(cache_key, nullptr); + { + if (!table_metadata_cache.tryDel(cache_key)) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Try to clear table metadata failed."); + } + } } void HiveMetastoreClient::setClient(std::shared_ptr client_) From eb9e9bd47a830e7581d9940d9e3c96947f6e0b32 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 21 Dec 2021 10:50:25 +0800 Subject: [PATCH 0210/1260] fixed a bug --- src/Common/LRUCache.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 98d14a31a21..01b4c93e439 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -148,7 +148,7 @@ public: /// Insert the new value only if the token is still in present in insert_tokens. /// (The token may be absent because of a concurrent reset() call). bool is_value_loaded = false; - bool is_value_updated = true; + bool is_value_updated = false; auto token_it = insert_tokens.find(key); if (token_it != insert_tokens.end() && token_it->second.get() == token) { From fb1214efeb04bf5ff1de399bceda47e0197e7d2e Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 21 Dec 2021 10:56:57 +0800 Subject: [PATCH 0211/1260] fixed code style --- src/Common/LRUCache.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 01b4c93e439..e9eb180dfce 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -165,7 +165,7 @@ public: } /// If key is not in cache or the element can be released, return is true. otherwise, return is false - bool tryDel(const Key &key) + bool tryDel(const Key & key) { std::lock_guard loc(mutex); auto it = cells.find(key); From 44c0f0e68057c90038c153287b6bb00ca0b9a534 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 21 Dec 2021 10:59:23 +0800 Subject: [PATCH 0212/1260] rewrite comments --- src/Common/LRUCache.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index e9eb180dfce..88f73480885 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -97,7 +97,7 @@ public: /// If the value for the key is in the cache, returns it. If it is not, calls load_func() to /// produce it, saves the result in the cache and returns it. - /// Only one of several concurrent threads calling getOrSet() will call load_func(), + /// Only one of several concurrent threads calling getOrTrySet() will call load_func(), /// others will wait for that call to complete and will use its result (this helps prevent cache stampede). /// Exceptions occurring in load_func will be propagated to the caller. Another thread from the /// set of concurrent threads will then try to call its load_func etc. @@ -153,7 +153,7 @@ public: if (token_it != insert_tokens.end() && token_it->second.get() == token) { // setImpl() may fail, but the final behavior seems not be affected - // next call of getOrSet() will still call load_func() + // next call of getOrTrySet() will still call load_func() is_value_updated = setImpl(key, token->value, cache_lock); is_value_loaded = true; } From 40aa359201313c5693f368f20461e229e65cf29e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 21 Dec 2021 14:44:31 +0800 Subject: [PATCH 0213/1260] refactor storage hive --- src/Storages/Hive/HiveCommon.cpp | 10 +- src/Storages/Hive/HiveCommon.h | 11 +- src/Storages/Hive/HiveFile.cpp | 3 +- src/Storages/Hive/HiveFile.h | 65 +++++-- src/Storages/Hive/HiveSettings.h | 2 +- src/Storages/Hive/StorageHive.cpp | 313 ++++++++++++++++-------------- src/Storages/Hive/StorageHive.h | 87 +++------ 7 files changed, 260 insertions(+), 231 deletions(-) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 68aba13b726..7acb2a27cdb 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -1,10 +1,12 @@ -#include -#include -#include -#include #include #if USE_HIVE + +#include +#include +#include + + namespace DB { namespace ErrorCodes diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index ee222f82877..e88e67b0257 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -6,10 +6,11 @@ #include #include -#include #include -#include + +#include #include +#include namespace DB @@ -131,9 +132,9 @@ private: std::mutex mutex; std::map clients; - const int conn_timeout_ms = 60000; - const int recv_timeout_ms = 60000; - const int send_timeout_ms = 60000; + const int conn_timeout_ms = 10000; + const int recv_timeout_ms = 10000; + const int send_timeout_ms = 10000; }; } diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index c2ab6b4593b..b0cfa9809e1 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -3,9 +3,8 @@ #if USE_HIVE #include -#include -#include #include + #include #include #include diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index 0be2d7a9573..63cca2562eb 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -7,13 +7,12 @@ #include #include -#include #include +#include #include #include #include -#include #include namespace orc @@ -50,6 +49,49 @@ namespace ErrorCodes class IHiveFile : public WithContext { public: + using MinMaxIndex = IMergeTreeDataPart::MinMaxIndex; + using MinMaxIndexPtr = std::shared_ptr; + + enum class FileFormat + { + RC_FILE, + TEXT, + LZO_TEXT, + SEQUENCE_FILE, + AVRO, + PARQUET, + ORC, + }; + + inline static const String RCFILE_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.RCFileInputFormat"; + inline static const String TEXT_INPUT_FORMAT = "org.apache.hadoop.mapred.TextInputFormat"; + inline static const String LZO_TEXT_INPUT_FORMAT = "com.hadoop.mapred.DeprecatedLzoTextInputFormat"; + inline static const String SEQUENCE_INPUT_FORMAT = "org.apache.hadoop.mapred.SequenceFileInputFormat"; + inline static const String PARQUET_INPUT_FORMAT = "com.cloudera.impala.hive.serde.ParquetInputFormat"; + inline static const String MR_PARQUET_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"; + inline static const String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"; + inline static const String ORC_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"; + inline static const std::map VALID_HDFS_FORMATS = { + {RCFILE_INPUT_FORMAT, FileFormat::RC_FILE}, + {TEXT_INPUT_FORMAT, FileFormat::TEXT}, + {LZO_TEXT_INPUT_FORMAT, FileFormat::LZO_TEXT}, + {SEQUENCE_INPUT_FORMAT, FileFormat::SEQUENCE_FILE}, + {PARQUET_INPUT_FORMAT, FileFormat::PARQUET}, + {MR_PARQUET_INPUT_FORMAT, FileFormat::PARQUET}, + {AVRO_INPUT_FORMAT, FileFormat::AVRO}, + {ORC_INPUT_FORMAT, FileFormat::ORC}, + }; + + static inline bool isFormatClass(const String & format_class) { return VALID_HDFS_FORMATS.count(format_class) > 0; } + static inline FileFormat toFileFormat(const String & format_class) + { + if (isFormatClass(format_class)) + { + return VALID_HDFS_FORMATS.find(format_class)->second; + } + throw Exception("Unsupported hdfs file format " + format_class, ErrorCodes::NOT_IMPLEMENTED); + } + IHiveFile( const FieldVector & values_, const String & namenode_url_, @@ -71,7 +113,6 @@ public: } virtual ~IHiveFile() = default; - using FileFormat = StorageHive::FileFormat; virtual FileFormat getFormat() const = 0; virtual String getName() const = 0; @@ -89,7 +130,7 @@ public: throw Exception("Method loadMinMaxIndex is not supported by hive file:" + getName(), ErrorCodes::NOT_IMPLEMENTED); } - virtual std::shared_ptr getMinMaxIndex() const { return minmax_idx; } + virtual MinMaxIndexPtr getMinMaxIndex() const { return minmax_idx; } // Do hive file contains sub-file level minmax index? virtual bool hasSubMinMaxIndex() const { return false; } @@ -99,13 +140,13 @@ public: throw Exception("Method loadSubMinMaxIndex is not supported by hive file:" + getName(), ErrorCodes::NOT_IMPLEMENTED); } - virtual const std::vector> & getSubMinMaxIndexes() const { return sub_minmax_idxes; } + virtual const std::vector & getSubMinMaxIndexes() const { return sub_minmax_idxes; } virtual void setSkipSplits(const std::set & splits) { skip_splits = splits; } virtual const std::set & getSkipSplits() const { return skip_splits; } - inline std::string describeMinMaxIndex(const std::shared_ptr & idx) const + inline std::string describeMinMaxIndex(const MinMaxIndexPtr & idx) const { if (!idx) return ""; @@ -130,12 +171,15 @@ protected: UInt64 last_modify_time; size_t size; NamesAndTypesList index_names_and_types; - std::shared_ptr minmax_idx; - std::vector> sub_minmax_idxes; + MinMaxIndexPtr minmax_idx; + std::vector sub_minmax_idxes; std::set skip_splits; // skip splits for this file after applying minmax index (if any) std::shared_ptr storage_settings; }; +using HiveFilePtr = std::shared_ptr; +using HiveFiles = std::vector; + class HiveTextFile : public IHiveFile { public: @@ -152,7 +196,6 @@ public: { } - using FileFormat = StorageHive::FileFormat; virtual FileFormat getFormat() const override { return FileFormat::TEXT; } virtual String getName() const override { return "TEXT"; } }; @@ -173,7 +216,6 @@ public: { } - using FileFormat = StorageHive::FileFormat; virtual FileFormat getFormat() const override { return FileFormat::ORC; } virtual String getName() const override { return "ORC"; } virtual bool hasMinMaxIndex() const override; @@ -183,7 +225,7 @@ public: virtual void loadSubMinMaxIndex() override; protected: - virtual std::unique_ptr buildMinMaxIndex(const orc::Statistics * statistics); + virtual std::unique_ptr buildMinMaxIndex(const orc::Statistics * statistics); virtual Range buildRange(const orc::ColumnStatistics * col_stats); virtual void prepareReader(); virtual void prepareColumnMapping(); @@ -208,7 +250,6 @@ public: { } - using FileFormat = StorageHive::FileFormat; virtual FileFormat getFormat() const override { return FileFormat::PARQUET; } virtual String getName() const override { return "PARQUET"; } diff --git a/src/Storages/Hive/HiveSettings.h b/src/Storages/Hive/HiveSettings.h index 406595b4741..ed430ba97cd 100644 --- a/src/Storages/Hive/HiveSettings.h +++ b/src/Storages/Hive/HiveSettings.h @@ -4,9 +4,9 @@ #if USE_HIVE +#include #include #include -#include namespace DB { diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index db689cf9bcf..81f6649a716 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -2,20 +2,18 @@ #if USE_HIVE +#include #include #include -#include +#include + #include -#include -#include +#include #include #include #include #include -#include #include -#include -#include #include #include #include @@ -23,26 +21,20 @@ #include #include #include +#include #include #include -#include -#include #include #include #include -#include #include #include -#include #include -#include #include -#include #include #include #include - namespace DB { namespace ErrorCodes @@ -54,8 +46,6 @@ namespace ErrorCodes extern const int CANNOT_OPEN_FILE; } -using HiveFilePtr = std::shared_ptr; -using HiveFiles = std::vector; static std::string getBaseName(const String & path) { @@ -63,9 +53,10 @@ static std::string getBaseName(const String & path) return path.substr(basename_start + 1); } -class HiveSource : public SourceWithProgress, WithContext +class StorageHiveSource : public SourceWithProgress, WithContext { public: + using FileFormat = StorageHive::FileFormat; struct SourcesInfo { HiveMetastoreClientPtr hive_metastore_client; @@ -102,7 +93,7 @@ public: return columns_description; } - HiveSource( + StorageHiveSource( SourcesInfoPtr source_info_, String hdfs_namenode_url_, String format_, @@ -167,7 +158,7 @@ public: } } - /// Use local cache for remote filesystem if enabled. + /// Use local cache for remote storage if enabled. std::unique_ptr remote_read_buf; if (RemoteReadBufferCache::instance().isInitialized() && getContext()->getSettingsRef().use_local_cache_for_remote_storage) remote_read_buf = RemoteReadBuffer::create(getContext(), @@ -176,7 +167,7 @@ public: else remote_read_buf = std::move(raw_read_buf); - if (curr_file->getFormat() == StorageHive::FileFormat::TEXT) + if (curr_file->getFormat() == FileFormat::TEXT) read_buf = wrapReadBufferWithCompressionMethod(std::move(remote_read_buf), compression); else read_buf = std::move(remote_read_buf); @@ -290,7 +281,7 @@ StorageHive::StorageHive( hdfs_namenode_url = getNameNodeUrl(hive_table_metadata->getTable()->sd.location); table_schema = hive_table_metadata->getTable()->sd.cols; - FileFormat hdfs_file_format = toFileFormat(hive_table_metadata->getTable()->sd.inputFormat); + FileFormat hdfs_file_format = IHiveFile::toFileFormat(hive_table_metadata->getTable()->sd.inputFormat); switch (hdfs_file_format) { case FileFormat::TEXT: @@ -341,6 +332,8 @@ void StorageHive::initMinMaxIndexExpression() /// Add all columns used in the partition key to the min-max index. partition_name_types = partition_key_expr->getRequiredColumnsWithTypes(); + partition_names = partition_name_types.getNames(); + partition_types = partition_name_types.getTypes(); partition_minmax_idx_expr = std::make_shared( std::make_shared(partition_name_types), ExpressionActionsSettings::fromContext(getContext())); } @@ -407,6 +400,137 @@ HiveFilePtr createHiveFile( return hive_file; } +std::vector StorageHive::collectHiveFilesFromPartition( + const Apache::Hadoop::Hive::Partition & partition, + SelectQueryInfo & query_info, + HiveTableMetadataPtr hive_table_metadata, + const HDFSFSPtr & fs, + ContextPtr context_) +{ + LOG_DEBUG(log, "Collect hive files from partition {}", boost::join(partition.values, ",")); + + /// Skip partition "__HIVE_DEFAULT_PARTITION__" + bool has_default_partition = false; + for (const auto & value : partition.values) + { + if (value == "__HIVE_DEFAULT_PARTITION__") + { + has_default_partition = true; + break; + } + } + if (has_default_partition) + return {}; + + /// Check partition values + if (partition.values.size() != partition_names.size()) + throw Exception( + fmt::format("Partition value size not match, expect {}, but got {}", partition_names.size(), partition.values.size()), + ErrorCodes::INVALID_PARTITION_VALUE); + + /// Join partition values in CSV format + WriteBufferFromOwnString wb; + for (size_t i = 0; i < partition.values.size(); ++i) + { + if (i != 0) + writeString(",", wb); + writeString(partition.values[i], wb); + } + writeString("\n", wb); + + ReadBufferFromString buffer(wb.str()); + auto format = FormatFactory::instance().getInputFormat( + "CSV", buffer, partition_key_expr->getSampleBlock(), getContext(), getContext()->getSettingsRef().max_block_size); + auto pipeline = QueryPipeline(std::move(format)); + auto reader = std::make_unique(pipeline); + Block block; + if (!reader->pull(block) || !block.rows()) + throw Exception("Could not parse partition value: " + wb.str(), ErrorCodes::INVALID_PARTITION_VALUE); + + std::vector ranges; + ranges.reserve(partition_names.size()); + FieldVector fields(partition_names.size()); + for (size_t i = 0; i < partition_names.size(); ++i) + { + block.getByPosition(i).column->get(0, fields[i]); + ranges.emplace_back(fields[i]); + } + + const KeyCondition partition_key_condition(query_info, getContext(), partition_names, partition_minmax_idx_expr); + if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true) + return {}; + + auto file_infos = listDirectory(partition.sd.location, hive_table_metadata, fs); + std::vector hive_files; + hive_files.reserve(file_infos.size()); + for (const auto & file_info : file_infos) + { + auto hive_file = createHiveFileIfValid(file_info, fields, query_info, context_); + if (hive_file) + hive_files.push_back(hive_file); + } + return hive_files; +} + +std::vector +StorageHive::listDirectory(const String & path, HiveTableMetadataPtr hive_table_metadata, const HDFSFSPtr & fs) +{ + return hive_table_metadata->getFilesByLocation(fs, path); +} + +HiveFilePtr StorageHive::createHiveFileIfValid( + const FileInfo & file_info, const FieldVector & fields, SelectQueryInfo & query_info, ContextPtr context_) +{ + LOG_TRACE(log, "Append hive file {}", file_info.path); + String filename = getBaseName(file_info.path); + /// Skip temporary files starts with '.' + if (filename.find('.') == 0) + return {}; + + auto hive_file = createHiveFile( + format_name, + fields, + hdfs_namenode_url, + file_info.path, + file_info.last_modify_time, + file_info.size, + hivefile_name_types, + storage_settings, + context_); + + /// Load file level minmax index and apply + const KeyCondition hivefile_key_condition(query_info, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); + if (hive_file->hasMinMaxIndex()) + { + hive_file->loadMinMaxIndex(); + if (!hivefile_key_condition.checkInHyperrectangle(hive_file->getMinMaxIndex()->hyperrectangle, hivefile_name_types.getTypes()) + .can_be_true) + { + LOG_TRACE(log, "Skip hive file {} by index {}", hive_file->getPath(), hive_file->describeMinMaxIndex(hive_file->getMinMaxIndex())); + return {}; + } + } + + /// Load sub-file level minmax index and apply + if (hive_file->hasSubMinMaxIndex()) + { + std::set skip_splits; + hive_file->loadSubMinMaxIndex(); + const auto & sub_minmax_idxes = hive_file->getSubMinMaxIndexes(); + for (size_t i = 0; i < sub_minmax_idxes.size(); ++i) + { + if (!hivefile_key_condition.checkInHyperrectangle(sub_minmax_idxes[i]->hyperrectangle, hivefile_name_types.getTypes()) + .can_be_true) + { + LOG_TRACE(log, "Skip split {} of hive file {}", i, hive_file->getPath()); + skip_splits.insert(i); + } + } + hive_file->setSkipSplits(skip_splits); + } + return hive_file; +} + Pipe StorageHive::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -421,161 +545,50 @@ Pipe StorageHive::read( auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url, getContext()); auto hive_table_metadata = hive_metastore_client->getTableMetadata(hive_database, hive_table); - /// List files under partition directory in HDFS - auto list_paths = [hive_table_metadata, &fs](const String & path) { return hive_table_metadata->getFilesByLocation(fs, path); }; - std::vector partitions = hive_table_metadata->getPartitions(); /// Hive files to read HiveFiles hive_files; /// Mutext to protect hive_files, which maybe appended in multiple threads std::mutex hive_files_mutex; - auto append_hive_files = [&](const HiveMetastoreClient::FileInfo & hfile, const FieldVector & fields) - { - LOG_TRACE(log, "Append hive file {}", hfile.path); - String filename = getBaseName(hfile.path); - - /// Skip temporary files starts with '.' - if (filename.find('.') == 0) - return; - - auto file = createHiveFile( - format_name, - fields, - hdfs_namenode_url, - hfile.path, - hfile.last_modify_time, - hfile.size, - hivefile_name_types, - storage_settings, - context_); - - /// Load file level minmax index and apply - const KeyCondition hivefile_key_condition(query_info, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); - if (file->hasMinMaxIndex()) - { - file->loadMinMaxIndex(); - if (!hivefile_key_condition.checkInHyperrectangle(file->getMinMaxIndex()->hyperrectangle, hivefile_name_types.getTypes()) - .can_be_true) - { - LOG_TRACE(log, "Skip hive file {} by index {}", file->getPath(), file->describeMinMaxIndex(file->getMinMaxIndex())); - return; - } - } - - /// Load sub-file level minmax index and apply - std::set skip_splits; - if (file->hasSubMinMaxIndex()) - { - file->loadSubMinMaxIndex(); - const auto & sub_minmax_idxes = file->getSubMinMaxIndexes(); - for (size_t i = 0; i < sub_minmax_idxes.size(); ++i) - { - if (!hivefile_key_condition.checkInHyperrectangle(sub_minmax_idxes[i]->hyperrectangle, hivefile_name_types.getTypes()) - .can_be_true) - { - LOG_TRACE(log, "Skip split {} of hive file {}", i, file->getPath()); - skip_splits.insert(i); - } - } - file->setSkipSplits(skip_splits); - } - - { - std::lock_guard lock{hive_files_mutex}; - hive_files.push_back(file); - } - }; - ThreadPool pool{num_streams}; if (!partitions.empty()) { - const auto partition_names = partition_name_types.getNames(); - const auto partition_types = partition_name_types.getTypes(); - for (const auto & partition : partitions) { - auto process_partition = [&]() + pool.scheduleOrThrowOnError([&]() { - LOG_DEBUG(log, "Process partition {}", boost::join(partition.values, ",")); - - /// Skip partition "__HIVE_DEFAULT_PARTITION__" - bool has_default_partition = false; - for (const auto & value : partition.values) + auto hive_files_in_partition = collectHiveFilesFromPartition(partition, query_info, hive_table_metadata, fs, context_); + if (!hive_files_in_partition.empty()) { - if (value == "__HIVE_DEFAULT_PARTITION__") - { - has_default_partition = true; - break; - } + std::lock_guard lock(hive_files_mutex); + hive_files.insert(std::end(hive_files), std::begin(hive_files_in_partition), std::end(hive_files_in_partition)); } - if (has_default_partition) - { - return; - } - - std::vector ranges; - WriteBufferFromOwnString wb; - if (partition.values.size() != partition_names.size()) - throw Exception( - fmt::format("Partition value size not match, expect {}, but got {}", partition_names.size(), partition.values.size()), - ErrorCodes::INVALID_PARTITION_VALUE); - - for (size_t i = 0; i < partition.values.size(); ++i) - { - if (i != 0) - writeString(",", wb); - writeString(partition.values[i], wb); - } - writeString("\n", wb); - - ReadBufferFromString buffer(wb.str()); - auto format = FormatFactory::instance().getInputFormat( - "CSV", buffer, partition_key_expr->getSampleBlock(), getContext(), getContext()->getSettingsRef().max_block_size); - auto pipeline = QueryPipeline(std::move(format)); - auto reader = std::make_unique(pipeline); - Block block; - if (!reader->pull(block) || !block.rows()) - throw Exception("Could not parse partition value: " + wb.str(), ErrorCodes::INVALID_PARTITION_VALUE); - - FieldVector fields(partition_names.size()); - for (size_t i = 0; i < partition_names.size(); ++i) - { - block.getByPosition(i).column->get(0, fields[i]); - ranges.emplace_back(fields[i]); - } - - const KeyCondition partition_key_condition(query_info, getContext(), partition_names, partition_minmax_idx_expr); - if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true) - { - return; - } - - auto paths = list_paths(partition.sd.location); - for (const auto & path : paths) - { - append_hive_files(path, fields); - } - }; - pool.scheduleOrThrowOnError(process_partition); + }); } pool.wait(); } else if (partition_name_types.empty()) /// Partition keys is empty { - auto paths = list_paths(hive_table_metadata->getTable()->sd.location); - for (const auto & path : paths) + auto file_infos = listDirectory(hive_table_metadata->getTable()->sd.location, hive_table_metadata, fs); + for (const auto & file_info : file_infos) { - pool.scheduleOrThrowOnError([&] { append_hive_files(path, {}); }); + pool.scheduleOrThrowOnError([&] + { + auto hive_file = createHiveFileIfValid(file_info, {}, query_info, context_); + if (hive_file) + { + std::lock_guard lock(hive_files_mutex); + hive_files.push_back(hive_file); + } + }); } pool.wait(); } else /// Partition keys is not empty but partitions is empty - { return {}; - } - auto sources_info = std::make_shared(); + auto sources_info = std::make_shared(); sources_info->hive_files = std::move(hive_files); sources_info->database = hive_database; sources_info->table_name = hive_table; @@ -595,7 +608,7 @@ Pipe StorageHive::read( Pipes pipes; for (size_t i = 0; i < num_streams; ++i) { - pipes.emplace_back(std::make_shared( + pipes.emplace_back(std::make_shared( sources_info, hdfs_namenode_url, format_name, diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 007f25539af..0527934ca12 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -5,23 +5,19 @@ #if USE_HIVE #include +#include + #include #include - #include -#include #include -#include -#include +#include +#include +#include namespace DB { -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - class HiveSettings; /** * This class represents table engine for external hdfs files. @@ -32,48 +28,6 @@ class StorageHive final : public shared_ptr_helper, public IStorage friend struct shared_ptr_helper; public: - enum class FileFormat - { - RC_FILE, - TEXT, - LZO_TEXT, - SEQUENCE_FILE, - AVRO, - PARQUET, - ORC, - }; - - // TODO: json support - inline static const String RCFILE_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.RCFileInputFormat"; - inline static const String TEXT_INPUT_FORMAT = "org.apache.hadoop.mapred.TextInputFormat"; - inline static const String LZO_TEXT_INPUT_FORMAT = "com.hadoop.mapred.DeprecatedLzoTextInputFormat"; - inline static const String SEQUENCE_INPUT_FORMAT = "org.apache.hadoop.mapred.SequenceFileInputFormat"; - inline static const String PARQUET_INPUT_FORMAT = "com.cloudera.impala.hive.serde.ParquetInputFormat"; - inline static const String MR_PARQUET_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"; - inline static const String AVRO_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"; - inline static const String ORC_INPUT_FORMAT = "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"; - inline static const std::map VALID_HDFS_FORMATS = { - {RCFILE_INPUT_FORMAT, FileFormat::RC_FILE}, - {TEXT_INPUT_FORMAT, FileFormat::TEXT}, - {LZO_TEXT_INPUT_FORMAT, FileFormat::LZO_TEXT}, - {SEQUENCE_INPUT_FORMAT, FileFormat::SEQUENCE_FILE}, - {PARQUET_INPUT_FORMAT, FileFormat::PARQUET}, - {MR_PARQUET_INPUT_FORMAT, FileFormat::PARQUET}, - {AVRO_INPUT_FORMAT, FileFormat::AVRO}, - {ORC_INPUT_FORMAT, FileFormat::ORC}, - }; - - static inline bool isFormatClass(const String & format_class) { return VALID_HDFS_FORMATS.count(format_class) > 0; } - static inline FileFormat toFileFormat(const String & format_class) - { - if (isFormatClass(format_class)) - { - return VALID_HDFS_FORMATS.find(format_class)->second; - } - throw Exception("Unsupported hdfs file format " + format_class, ErrorCodes::NOT_IMPLEMENTED); - } - - String getName() const override { return "Hive"; } bool supportsIndexForIn() const override { return true; } @@ -100,6 +54,7 @@ public: NamesAndTypesList getVirtuals() const override; protected: + friend class StorageHiveSource; StorageHive( const String & hive_metastore_url_, const String & hive_database_, @@ -112,21 +67,37 @@ protected: std::unique_ptr storage_settings_, ContextPtr context_); - void initMinMaxIndexExpression(); +private: + using FileFormat = IHiveFile::FileFormat; + using FileInfo = HiveMetastoreClient::FileInfo; + using HiveTableMetadataPtr = HiveMetastoreClient::HiveTableMetadataPtr; + static ASTPtr extractKeyExpressionList(const ASTPtr & node); -private: + static std::vector listDirectory(const String & path, HiveTableMetadataPtr hive_table_metadata, const HDFSFSPtr & fs); + + void initMinMaxIndexExpression(); + + std::vector collectHiveFilesFromPartition( + const Apache::Hadoop::Hive::Partition & partition, + SelectQueryInfo & query_info, + HiveTableMetadataPtr hive_table_metadata, + const HDFSFSPtr & fs, + ContextPtr context_); + + HiveFilePtr + createHiveFileIfValid(const FileInfo & file_info, const FieldVector & fields, SelectQueryInfo & query_info, ContextPtr context_); + String hive_metastore_url; - // hive database and table + /// Hive database and table String hive_database; String hive_table; - // hive table meta + /// Hive table meta std::vector table_schema; - Names text_input_field_names; // Defines schema of hive file, only used when text input format is TEXT + Names text_input_field_names; /// Defines schema of hive file, only used when text input format is TEXT - // hdfs relative information String hdfs_namenode_url; String format_name; @@ -134,6 +105,8 @@ private: const ASTPtr partition_by_ast; NamesAndTypesList partition_name_types; + Names partition_names; + DataTypes partition_types; ExpressionActionsPtr partition_key_expr; ExpressionActionsPtr partition_minmax_idx_expr; From 2fd63f93811b088588b38d8e5d35955b2fa03919 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 21 Dec 2021 15:00:47 +0800 Subject: [PATCH 0214/1260] rename some symboles --- src/Storages/Hive/StorageHive.cpp | 6 +++--- src/Storages/Hive/StorageHive.h | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 81f6649a716..46599a18961 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -465,7 +465,7 @@ std::vector StorageHive::collectHiveFilesFromPartition( hive_files.reserve(file_infos.size()); for (const auto & file_info : file_infos) { - auto hive_file = createHiveFileIfValid(file_info, fields, query_info, context_); + auto hive_file = createHiveFileIfNeeded(file_info, fields, query_info, context_); if (hive_file) hive_files.push_back(hive_file); } @@ -478,7 +478,7 @@ StorageHive::listDirectory(const String & path, HiveTableMetadataPtr hive_table_ return hive_table_metadata->getFilesByLocation(fs, path); } -HiveFilePtr StorageHive::createHiveFileIfValid( +HiveFilePtr StorageHive::createHiveFileIfNeeded( const FileInfo & file_info, const FieldVector & fields, SelectQueryInfo & query_info, ContextPtr context_) { LOG_TRACE(log, "Append hive file {}", file_info.path); @@ -575,7 +575,7 @@ Pipe StorageHive::read( { pool.scheduleOrThrowOnError([&] { - auto hive_file = createHiveFileIfValid(file_info, {}, query_info, context_); + auto hive_file = createHiveFileIfNeeded(file_info, {}, query_info, context_); if (hive_file) { std::lock_guard lock(hive_files_mutex); diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 0527934ca12..9629629e057 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -86,7 +86,7 @@ private: ContextPtr context_); HiveFilePtr - createHiveFileIfValid(const FileInfo & file_info, const FieldVector & fields, SelectQueryInfo & query_info, ContextPtr context_); + createHiveFileIfNeeded(const FileInfo & file_info, const FieldVector & fields, SelectQueryInfo & query_info, ContextPtr context_); String hive_metastore_url; From 1f59ce760cbfa832fb024dab94b70443706babc0 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 21 Dec 2021 19:04:57 +0800 Subject: [PATCH 0215/1260] typo --- src/Storages/RemoteReadBufferCache.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/RemoteReadBufferCache.cpp b/src/Storages/RemoteReadBufferCache.cpp index 2252f84dfd7..be25345625e 100644 --- a/src/Storages/RemoteReadBufferCache.cpp +++ b/src/Storages/RemoteReadBufferCache.cpp @@ -362,22 +362,22 @@ void RemoteReadBufferCache::recoverCachedFilesMetadata( { if (current_depth >= max_depth) { - std::vector invalid_pathes; + std::vector invalid_paths; for (auto const & dir : fs::directory_iterator{current_path}) { String path = dir.path(); auto cache_controller = RemoteCacheController::recover(path); if (!cache_controller) { - invalid_pathes.emplace_back(path); + invalid_paths.emplace_back(path); continue; } if (!lru_caches->set(path, cache_controller)) { - invalid_pathes.emplace_back(path); + invalid_paths.emplace_back(path); } } - for (auto & path : invalid_pathes) + for (auto & path : invalid_paths) { fs::remove_all(path); } From 627bd2cec4b461b3d57fbcd57916a66cb4d1e775 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 21 Dec 2021 15:07:10 +0300 Subject: [PATCH 0216/1260] Add Keeper Jepsen check to new CI --- .github/workflows/main.yml | 34 ++++++ tests/ci/keeper_jepsen_check.py | 186 ++++++++++++++++++++++++++++++++ 2 files changed, 220 insertions(+) create mode 100644 tests/ci/keeper_jepsen_check.py diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 4cda4eac33e..c9a9beda598 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -199,6 +199,39 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH + KeeperJepsenRelease: + needs: [BuilderBinRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/keeper_jepsen + REPORTS_PATH=${{runner.temp}}/keeper_jepsen + REPO_COPY=${{runner.temp}}/keeper_jepsen/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 keeper_jepsen_check.py + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH ######################################################################################### #################################### ORDINARY BUILDS #################################### ######################################################################################### @@ -2919,6 +2952,7 @@ jobs: - SplitBuildSmokeTest - CompatibilityCheck - IntegrationTestsFlakyCheck + - KeeperJepsenRelease runs-on: [self-hosted, style-checker] steps: - name: Clear repository diff --git a/tests/ci/keeper_jepsen_check.py b/tests/ci/keeper_jepsen_check.py new file mode 100644 index 00000000000..d7b881b0802 --- /dev/null +++ b/tests/ci/keeper_jepsen_check.py @@ -0,0 +1,186 @@ +#!/usr/bin/env python3 + +import time +import logging +import os + +import boto3 +from github import Github + +from env_helper import REPORTS_PATH, REPO_COPY, TEMP_PATH +from stopwatch import Stopwatch +from upload_result_helper import upload_results +from s3_helper import S3Helper +from get_robot_token import get_best_robot_token +from pr_info import PRInfo +from compress_files import compress_fast +from commit_status_helper import post_commit_status +from docker_pull_helper import get_image_with_version +from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse +from tee_popen import TeePopen +from ssh import SSHKey +from build_download_helper import get_build_name_for_check, get_build_urls + +JEPSEN_GROUP_NAME = 'jepsen_group' +DESIRED_INSTANCE_COUNT = 3 +IMAGE_NAME = 'clickhouse/keeper-jepsen-test' +CHECK_NAME = 'ClickHouse Keeper Jepsen (actions)' + + +SUCCESSFUL_TESTS_ANCHOR = "# Successful tests" +INTERMINATE_TESTS_ANCHOR = "# Indeterminate tests" +CRASHED_TESTS_ANCHOR = "# Crashed tests" +FAILED_TESTS_ANCHOR = "# Failed tests" + +def _parse_jepsen_output(path): + test_results = [] + current_type = '' + with open(path, 'r') as f: + for line in f: + if SUCCESSFUL_TESTS_ANCHOR in line: + current_type = 'OK' + elif INTERMINATE_TESTS_ANCHOR in line or CRASHED_TESTS_ANCHOR in line: + current_type = 'ERROR' + elif FAILED_TESTS_ANCHOR in line: + current_type = 'FAIL' + + if (line.startswith('store/clickhouse-keeper') or line.startswith('clickhouse-keeper')) and current_type: + test_results.append((line.strip(), current_type)) + + return test_results + +def get_autoscaling_group_instances_ids(asg_client, group_name): + group_description = asg_client.describe_auto_scaling_groups(AutoScalingGroupNames=[group_name]) + our_group = group_description['AutoScalingGroups'][0] + instance_ids = [] + for instance in our_group['Instances']: + if instance['LifecycleState'] == 'InService' and instance['HealthStatus'] == 'Healthy': + instance_ids.append(instance['InstanceId']) + + return instance_ids + +def get_instances_addresses(ec2_client, instance_ids): + ec2_response = ec2_client.describe_instances(InstanceIds = instance_ids) + instance_ips = [] + for instances in ec2_response['Reservations']: + for ip in instances['Instances']: + instance_ips.append(ip['PrivateIpAddress']) + return instance_ips + + +def prepare_autoscaling_group_and_get_hostnames(): + asg_client = boto3.client('autoscaling') + asg_client.set_desired_capacity(AutoScalingGroupName=JEPSEN_GROUP_NAME, DesiredCapacity=DESIRED_INSTANCE_COUNT) + + instances = get_autoscaling_group_instances_ids(asg_client, JEPSEN_GROUP_NAME) + counter = 0 + while len(instances) < DESIRED_INSTANCE_COUNT: + time.sleep(5) + instances = get_autoscaling_group_instances_ids(asg_client, JEPSEN_GROUP_NAME) + counter += 1 + if counter > 30: + raise Exception("Cannot wait autoscaling group") + + ec2_client = boto3.client('ec2') + return get_instances_addresses(ec2_client, instances) + + +def clear_autoscaling_group(): + asg_client = boto3.client('autoscaling') + asg_client.set_desired_capacity(AutoScalingGroupName=JEPSEN_GROUP_NAME, DesiredCapacity=0) + instances = get_autoscaling_group_instances_ids(asg_client, JEPSEN_GROUP_NAME) + counter = 0 + while len(instances) > 0: + time.sleep(5) + instances = get_autoscaling_group_instances_ids(asg_client, JEPSEN_GROUP_NAME) + counter += 1 + if counter > 30: + raise Exception("Cannot wait autoscaling group") + + +def save_nodes_to_file(instances, temp_path): + nodes_path = os.path.join(temp_path, "nodes.txt") + with open(nodes_path, 'w') as f: + f.write("\n".join(instances)) + f.flush() + return nodes_path + +def get_run_command(ssh_auth_sock, ssh_sock_dir, pr_info, nodes_path, repo_path, build_url, result_path, docker_image): + return f"docker run --network=host -v '{ssh_sock_dir}:{ssh_sock_dir}' -e SSH_AUTH_SOCK={ssh_auth_sock} " \ + f"-e PR_TO_TEST={pr_info.number} -e SHA_TO_TEST={pr_info.sha} -v '{nodes_path}:/nodes.txt' -v {result_path}:/test_output " \ + f"-e 'CLICKHOUSE_PACKAGE={build_url}' -v '{repo_path}:/ch' -e 'CLICKHOUSE_REPO_PATH=/ch' -e NODES_USERNAME=ubuntu {docker_image}" + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() + + pr_info = PRInfo() + + gh = Github(get_best_robot_token()) + + if not os.path.exists(TEMP_PATH): + os.makedirs(TEMP_PATH) + + result_path = os.path.join(TEMP_PATH, "result_path") + if not os.path.exists(result_path): + os.makedirs(result_path) + + instances = prepare_autoscaling_group_and_get_hostnames() + nodes_path = save_nodes_to_file(instances, TEMP_PATH) + + docker_image = get_image_with_version(REPORTS_PATH, IMAGE_NAME) + + build_name = get_build_name_for_check(CHECK_NAME) + urls = get_build_urls(build_name, REPORTS_PATH) + if not urls: + raise Exception("No build URLs found") + + for url in urls: + if url.endswith('/clickhouse'): + build_url = url + break + else: + raise Exception("Cannot binary clickhouse among build results") + + with SSHKey("ROBOT_CLICKHOUSE_SSH_KEY"): + ssh_auth_sock = os.environ['SSH_AUTH_SOCK'] + auth_sock_dir = os.path.dirname(ssh_auth_sock) + cmd = get_run_command(ssh_auth_sock, auth_sock_dir, pr_info, nodes_path, REPO_COPY, build_url, result_path, docker_image) + logging.info("Going to run jepsen: %s", cmd) + + run_log_path = os.path.join(TEMP_PATH, "runlog.log") + + with TeePopen(cmd, run_log_path) as process: + retcode = process.wait() + if retcode == 0: + logging.info("Run successfully") + else: + logging.info("Run failed") + + status = 'success' + description = 'No invalid analysis found ヽ(‘ー`)ノ' + jepsen_log_path = os.path.join(result_path, 'jepsen_run_all_tests.log') + additional_data = [] + try: + test_result = _parse_jepsen_output(jepsen_log_path) + if any(r[1] == 'FAIL' for r in test_result): + status = 'failure' + description = 'Found invalid analysis (ノಥ益ಥ)ノ ┻━┻' + compress_fast(os.path.join(result_path, 'store'), 'jepsen_store.tar.gz') + additional_data.append(os.path.join(TEMP_PATH, 'jepsen_store.tar.gz')) + except: + status = 'failure' + description = 'No Jepsen output log' + test_result = [('No Jepsen output log', 'FAIL')] + + s3_helper = S3Helper('https://s3.amazonaws.com') + report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_result, [run_log_path] + additional_data, CHECK_NAME) + + print(f"::notice ::Report url: {report_url}") + post_commit_status(gh, pr_info.sha, CHECK_NAME, description, status, report_url) + + ch_helper = ClickHouseHelper() + prepared_events = prepare_tests_results_for_clickhouse(pr_info, test_result, status, stopwatch.duration_seconds, stopwatch.start_time_str, report_url, CHECK_NAME) + ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events) + clear_autoscaling_group() From 157d4a7f7ce7d8b2640cfe63c044661b7cbdce60 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 21 Dec 2021 15:08:33 +0300 Subject: [PATCH 0217/1260] Add SSH key --- .github/workflows/main.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index c9a9beda598..99288846e02 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -208,6 +208,9 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/keeper_jepsen REPORTS_PATH=${{runner.temp}}/keeper_jepsen + ROBOT_CLICKHOUSE_SSH_KEY< Date: Tue, 21 Dec 2021 16:05:02 +0300 Subject: [PATCH 0218/1260] Don't use too powerful key --- .github/workflows/main.yml | 3 --- tests/ci/keeper_jepsen_check.py | 4 +++- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 99288846e02..c9a9beda598 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -208,9 +208,6 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/keeper_jepsen REPORTS_PATH=${{runner.temp}}/keeper_jepsen - ROBOT_CLICKHOUSE_SSH_KEY< Date: Tue, 21 Dec 2021 16:16:01 +0300 Subject: [PATCH 0219/1260] Group params --- tests/ci/keeper_jepsen_check.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/ci/keeper_jepsen_check.py b/tests/ci/keeper_jepsen_check.py index 004143f8895..a25be94ddcc 100644 --- a/tests/ci/keeper_jepsen_check.py +++ b/tests/ci/keeper_jepsen_check.py @@ -7,12 +7,11 @@ import os import boto3 from github import Github -from get_robot_token import get_parameter_from_ssm from env_helper import REPORTS_PATH, REPO_COPY, TEMP_PATH from stopwatch import Stopwatch from upload_result_helper import upload_results from s3_helper import S3Helper -from get_robot_token import get_best_robot_token +from get_robot_token import get_best_robot_token, get_parameter_from_ssm from pr_info import PRInfo from compress_files import compress_fast from commit_status_helper import post_commit_status From 3ab250a869c98e6f7c06396391f8d9d95e9733fc Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 21 Dec 2021 16:47:23 +0300 Subject: [PATCH 0220/1260] Add region name: --- tests/ci/keeper_jepsen_check.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/keeper_jepsen_check.py b/tests/ci/keeper_jepsen_check.py index a25be94ddcc..380bd501d3d 100644 --- a/tests/ci/keeper_jepsen_check.py +++ b/tests/ci/keeper_jepsen_check.py @@ -69,7 +69,7 @@ def get_instances_addresses(ec2_client, instance_ids): def prepare_autoscaling_group_and_get_hostnames(): - asg_client = boto3.client('autoscaling') + asg_client = boto3.client('autoscaling', region_name='us-east-1') asg_client.set_desired_capacity(AutoScalingGroupName=JEPSEN_GROUP_NAME, DesiredCapacity=DESIRED_INSTANCE_COUNT) instances = get_autoscaling_group_instances_ids(asg_client, JEPSEN_GROUP_NAME) @@ -81,7 +81,7 @@ def prepare_autoscaling_group_and_get_hostnames(): if counter > 30: raise Exception("Cannot wait autoscaling group") - ec2_client = boto3.client('ec2') + ec2_client = boto3.client('ec2', region_name='us-east-1') return get_instances_addresses(ec2_client, instances) From 33cbfc89efdd96bed99371203fd3b06ce24b9a01 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Tue, 21 Dec 2021 17:29:50 +0300 Subject: [PATCH 0221/1260] Move logic for replicated part to StorageReplicatedMergeTree class --- src/Storages/MergeTree/MergeTreeData.cpp | 169 ++------------------ src/Storages/MergeTree/MergeTreeData.h | 14 +- src/Storages/StorageReplicatedMergeTree.cpp | 165 +++++++++++++++++++ src/Storages/StorageReplicatedMergeTree.h | 12 +- 4 files changed, 191 insertions(+), 169 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9bdb01fd0a8..4b3c84a323c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3889,15 +3889,9 @@ void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, ContextPtr renamed_parts.tryRenameAll(); - String replica_name = getReplicaName(); - String zookeeper_name = getZooKeeperName(); - String zookeeper_path = getZooKeeperPath(); - String table_uuid = getTableUniqID(); - for (auto & [old_name, new_name, disk] : renamed_parts.old_and_new_names) { - bool keep_shared = removeSharedDetachedPart(disk, fs::path(relative_data_path) / "detached" / new_name / "", old_name, - table_uuid, zookeeper_name, replica_name, zookeeper_path, supportsReplication()); + bool keep_shared = removeDetachedPart(disk, fs::path(relative_data_path) / "detached" / new_name / "", old_name, false); LOG_DEBUG(log, "Dropped detached part {}, keep shared data: {}", old_name, keep_shared); old_name.clear(); } @@ -5135,82 +5129,6 @@ MergeTreeData::MatcherFn MergeTreeData::getPartitionMatcher(const ASTPtr & parti }; } -struct FreezeMetaData -{ -public: - void fill(const IMergeTreeDataPart & part) - { - is_replicated = part.storage.supportsReplication(); - is_remote = part.storage.isRemote(); - replica_name = part.storage.getReplicaName(); - zookeeper_name = part.storage.getZooKeeperName(); - table_uuid = part.storage.getTableUniqID(); - } - - void save(DiskPtr disk, const String & path) const - { - auto file_path = getFileName(path); - auto buffer = disk->writeMetaFile(file_path); - writeIntText(version, *buffer); - buffer->write("\n", 1); - writeBoolText(is_replicated, *buffer); - buffer->write("\n", 1); - writeBoolText(is_remote, *buffer); - buffer->write("\n", 1); - writeString(replica_name, *buffer); - buffer->write("\n", 1); - writeString(zookeeper_name, *buffer); - buffer->write("\n", 1); - writeString(table_uuid, *buffer); - buffer->write("\n", 1); - } - - bool load(DiskPtr disk, const String & path) - { - auto file_path = getFileName(path); - if (!disk->exists(file_path)) - return false; - auto buffer = disk->readMetaFile(file_path); - readIntText(version, *buffer); - if (version != 1) - { - LOG_ERROR(&Poco::Logger::get("FreezeMetaData"), "Unknown freezed metadata version: {}", version); - return false; - } - DB::assertChar('\n', *buffer); - readBoolText(is_replicated, *buffer); - DB::assertChar('\n', *buffer); - readBoolText(is_remote, *buffer); - DB::assertChar('\n', *buffer); - readString(replica_name, *buffer); - DB::assertChar('\n', *buffer); - readString(zookeeper_name, *buffer); - DB::assertChar('\n', *buffer); - readString(table_uuid, *buffer); - DB::assertChar('\n', *buffer); - return true; - } - - static void clean(DiskPtr disk, const String & path) - { - disk->removeMetaFileIfExists(getFileName(path)); - } - -private: - static String getFileName(const String & path) - { - return fs::path(path) / "frozen_metadata.txt"; - } - -public: - int version = 1; - bool is_replicated; - bool is_remote; - String replica_name; - String zookeeper_name; - String table_uuid; -}; - PartitionCommandsResultInfo MergeTreeData::freezePartition( const ASTPtr & partition_ast, const StorageMetadataPtr & metadata_snapshot, @@ -5277,12 +5195,7 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( localBackup(disk, src_part_path, backup_part_path); - if (disk->supportZeroCopyReplication()) - { - FreezeMetaData meta; - meta.fill(*part); - meta.save(disk, backup_part_path); - } + freezeMetaData(disk, part, backup_part_path); disk->removeFileIfExists(fs::path(backup_part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); @@ -5301,6 +5214,11 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( return result; } +void MergeTreeData::freezeMetaData(DiskPtr, DataPartPtr, String) const +{ + +} + PartitionCommandsResultInfo MergeTreeData::unfreezePartition( const ASTPtr & partition, const String & backup_name, @@ -5318,76 +5236,11 @@ PartitionCommandsResultInfo MergeTreeData::unfreezeAll( return unfreezePartitionsByMatcher([] (const String &) { return true; }, backup_name, local_context); } -bool MergeTreeData::removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name) +bool MergeTreeData::removeDetachedPart(DiskPtr disk, const String & path, const String &, bool) { - bool keep_shared = false; + disk->removeRecursive(path); - if (disk->supportZeroCopyReplication()) - { - FreezeMetaData meta; - if (meta.load(disk, path)) - { - FreezeMetaData::clean(disk, path); - return removeSharedDetachedPart(disk, path, part_name, meta.table_uuid, meta.zookeeper_name, meta.replica_name, - "", meta.is_replicated); - } - } - - disk->removeSharedRecursive(path, keep_shared); - - return keep_shared; -} - -bool MergeTreeData::removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid, - const String & zookeeper_name, const String & replica_name, const String & zookeeper_path, bool is_replicated) -{ - bool keep_shared = false; - - if (is_replicated && disk->supportZeroCopyReplication()) - { - static constexpr auto default_zookeeper_name = "default"; - zkutil::ZooKeeperPtr zookeeper; - if (zookeeper_name == default_zookeeper_name) - { - zookeeper = getContext()->getZooKeeper(); - } - else - { - try - { - zookeeper = getContext()->getAuxiliaryZooKeeper(zookeeper_name); - } - catch (const Exception & e) - { - if (e.code() != ErrorCodes::BAD_ARGUMENTS) - throw; - /// No more stored non-default zookeeper - zookeeper = nullptr; - } - } - - if (zookeeper) - { - fs::path checksums = fs::path(path) / "checksums.txt"; - if (disk->exists(checksums)) - { - auto ref_count = disk->getRefCount(checksums); - if (ref_count == 0) - { - String id = disk->getUniqueId(checksums); - keep_shared = !StorageReplicatedMergeTree::unlockSharedDataById(id, table_uuid, part_name, - replica_name, disk, zookeeper, getContext()->getReplicatedMergeTreeSettings(), log, - zookeeper_path); - } - else - keep_shared = true; - } - } - } - - disk->removeSharedRecursive(path, keep_shared); - - return keep_shared; + return false; } PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn matcher, const String & backup_name, ContextPtr) @@ -5418,7 +5271,7 @@ PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn const auto & path = it->path(); - bool keep_shared = removeSharedDetachedPart(disk, path, partition_directory); + bool keep_shared = removeDetachedPart(disk, path, partition_directory, true); result.push_back(PartitionCommandResultInfo{ .partition_id = partition_id, diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 77bcf4925d0..deee7556b68 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -879,8 +879,12 @@ public: /// Overridden in StorageReplicatedMergeTree virtual bool tryToFetchIfShared(const IMergeTreeDataPart &, const DiskPtr &, const String &) { return false; } - virtual String getZooKeeperName() const { return ""; } - virtual String getZooKeeperPath() const { return ""; } + /// Check shared data usage on other replicas for detached/freezed part + /// Remove local files and remote files if needed + virtual bool removeDetachedPart(DiskPtr disk, const String & path, const String & part_name, bool is_freezed); + + /// Store some metadata for freezed part if needed + virtual void freezeMetaData(DiskPtr disk, DataPartPtr part, String backup_part_path) const; /// Parts that currently submerging (merging to bigger parts) or emerging /// (to be appeared after merging finished). These two variables have to be used @@ -1183,12 +1187,6 @@ private: DataPartsVector & duplicate_parts_to_remove, MutableDataPartsVector & parts_from_wal, DataPartsLock & part_lock); - - /// Check shared data usage on other replicas for detached/freezed part - /// Remove local files and remote files if needed - bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name); - bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid, - const String & zookeeper_name, const String & replica_name, const String & zookeeper_path, bool is_replicated); }; /// RAII struct to record big parts that are submerging or emerging. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ccbee608a08..6df941a59b0 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7619,4 +7619,169 @@ void StorageReplicatedMergeTree::createZeroCopyLockNode(const zkutil::ZooKeeperP } +struct FreezeMetaData +{ +public: + void fill(const StorageReplicatedMergeTree & storage) + { + is_replicated = storage.supportsReplication(); + is_remote = storage.isRemote(); + replica_name = storage.getReplicaName(); + zookeeper_name = storage.getZooKeeperName(); + table_uuid = storage.getTableUniqID(); + } + + void save(DiskPtr disk, const String & path) const + { + auto file_path = getFileName(path); + auto buffer = disk->writeMetaFile(file_path); + writeIntText(version, *buffer); + buffer->write("\n", 1); + writeBoolText(is_replicated, *buffer); + buffer->write("\n", 1); + writeBoolText(is_remote, *buffer); + buffer->write("\n", 1); + writeString(replica_name, *buffer); + buffer->write("\n", 1); + writeString(zookeeper_name, *buffer); + buffer->write("\n", 1); + writeString(table_uuid, *buffer); + buffer->write("\n", 1); + } + + bool load(DiskPtr disk, const String & path) + { + auto file_path = getFileName(path); + if (!disk->exists(file_path)) + return false; + auto buffer = disk->readMetaFile(file_path); + readIntText(version, *buffer); + if (version != 1) + { + LOG_ERROR(&Poco::Logger::get("FreezeMetaData"), "Unknown freezed metadata version: {}", version); + return false; + } + DB::assertChar('\n', *buffer); + readBoolText(is_replicated, *buffer); + DB::assertChar('\n', *buffer); + readBoolText(is_remote, *buffer); + DB::assertChar('\n', *buffer); + readString(replica_name, *buffer); + DB::assertChar('\n', *buffer); + readString(zookeeper_name, *buffer); + DB::assertChar('\n', *buffer); + readString(table_uuid, *buffer); + DB::assertChar('\n', *buffer); + return true; + } + + static void clean(DiskPtr disk, const String & path) + { + disk->removeMetaFileIfExists(getFileName(path)); + } + +private: + static String getFileName(const String & path) + { + return fs::path(path) / "frozen_metadata.txt"; + } + +public: + int version = 1; + bool is_replicated; + bool is_remote; + String replica_name; + String zookeeper_name; + String table_uuid; +}; + + +bool StorageReplicatedMergeTree::removeDetachedPart(DiskPtr disk, const String & path, const String & part_name, bool is_freezed) +{ + if (disk->supportZeroCopyReplication()) + { + if (is_freezed) + { + FreezeMetaData meta; + if (meta.load(disk, path)) + { + FreezeMetaData::clean(disk, path); + return removeSharedDetachedPart(disk, path, part_name, meta.table_uuid, meta.zookeeper_name, meta.replica_name, ""); + } + } + else + { + String table_uuid = getTableUniqID(); + + return removeSharedDetachedPart(disk, path, part_name, table_uuid, zookeeper_name, replica_name, zookeeper_path); + } + } + + disk->removeRecursive(path); + + return false; +} + + +bool StorageReplicatedMergeTree::removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid, + const String & detached_zookeeper_name, const String & detached_replica_name, const String & detached_zookeeper_path) +{ + bool keep_shared = false; + + static constexpr auto default_zookeeper_name = "default"; + zkutil::ZooKeeperPtr zookeeper; + if (detached_zookeeper_name == default_zookeeper_name) + { + zookeeper = getContext()->getZooKeeper(); + } + else + { + try + { + zookeeper = getContext()->getAuxiliaryZooKeeper(detached_zookeeper_name); + } + catch (const Exception & e) + { + if (e.code() != ErrorCodes::BAD_ARGUMENTS) + throw; + /// No more stored non-default zookeeper + zookeeper = nullptr; + } + } + + if (zookeeper) + { + fs::path checksums = fs::path(path) / "checksums.txt"; + if (disk->exists(checksums)) + { + auto ref_count = disk->getRefCount(checksums); + if (ref_count == 0) + { + String id = disk->getUniqueId(checksums); + keep_shared = !StorageReplicatedMergeTree::unlockSharedDataById(id, table_uuid, part_name, + detached_replica_name, disk, zookeeper, getContext()->getReplicatedMergeTreeSettings(), log, + detached_zookeeper_path); + } + else + keep_shared = true; + } + } + + disk->removeSharedRecursive(path, keep_shared); + + return keep_shared; +} + + +void StorageReplicatedMergeTree::freezeMetaData(DiskPtr disk, DataPartPtr, String backup_part_path) const +{ + if (disk->supportZeroCopyReplication()) + { + FreezeMetaData meta; + meta.fill(*this); + meta.save(disk, backup_part_path); + } +} + + } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index a93930a0957..4f8c4617388 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -237,7 +237,7 @@ public: /// Return false if data is still used by another node bool unlockSharedData(const IMergeTreeDataPart & part) const override; - /// Unlock same part with other (old) name + /// Remove lock with old name for shared data part after rename bool unlockSharedData(const IMergeTreeDataPart & part, const String & name) const override; /// Unlock shared data part in zookeeper by part id @@ -274,8 +274,7 @@ public: bool createEmptyPartInsteadOfLost(zkutil::ZooKeeperPtr zookeeper, const String & lost_part_name); - virtual String getZooKeeperName() const override { return zookeeper_name; } - virtual String getZooKeeperPath() const override { return zookeeper_path; } + String getZooKeeperName() const { return zookeeper_name; } virtual String getTableUniqID() const override; @@ -742,6 +741,13 @@ private: static void createZeroCopyLockNode(const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_node); + bool removeDetachedPart(DiskPtr disk, const String & path, const String & part_name, bool is_freezed) override; + + bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid, + const String & zookeeper_name, const String & replica_name, const String & zookeeper_path); + + void freezeMetaData(DiskPtr disk, DataPartPtr part, String backup_part_path) const override; + protected: /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. */ From 6a531c384eaf7bf9f1ed35354c98797ef787626c Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 21 Dec 2021 18:31:13 +0300 Subject: [PATCH 0222/1260] Fix --- .github/workflows/main.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index c9a9beda598..37a01f1a3a1 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -207,7 +207,7 @@ jobs: run: | cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/keeper_jepsen - REPORTS_PATH=${{runner.temp}}/keeper_jepsen + REPORTS_PATH=${{runner.temp}}/reports_dir REPO_COPY=${{runner.temp}}/keeper_jepsen/ClickHouse EOF - name: Download json reports From 27fc5b2330f686319b71597e5a171f34dcc2c619 Mon Sep 17 00:00:00 2001 From: lehasm Date: Tue, 21 Dec 2021 18:32:08 +0300 Subject: [PATCH 0223/1260] Update docs/en/operations/server-configuration-parameters/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 9b0edddb4b7..a8c006cdbe1 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -435,7 +435,7 @@ Similar to `interserver_http_host`, except that this hostname can be used by oth ## interserver_http_credentials {#server-settings-interserver-http-credentials} -A username and a password used to connect to other servers during [replication](../../engines/table-engines/mergetree-family/replication.md). Also the server authenticate other replicas using these credentials. So, `interserver_http_credentials` must be the same for all replicas in a cluster. +A username and a password used to connect to other servers during [replication](../../engines/table-engines/mergetree-family/replication.md). Also the server authenticates other replicas using these credentials. So, `interserver_http_credentials` must be the same for all replicas in a cluster. By default, if `interserver_http_credentials` section is omitted, authentication is not used during replication. From e88b97dafb9f5cbe5f4c9be08e88b950bb67d3e1 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Tue, 21 Dec 2021 17:55:20 +0300 Subject: [PATCH 0224/1260] Fix typos --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 6 +++--- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- utils/zero_copy/zero_copy_schema_converter.py | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index eb2188e17f6..4c9373dd15c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1153,12 +1153,12 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ storage.lockSharedData(*this); } -void IMergeTreeDataPart::cleanupOldName(const String & old_name) const +void IMergeTreeDataPart::cleanupOldName(const String & old_part_name) const { - if (name == old_name) + if (name == old_part_name) return; - storage.unlockSharedData(*this, old_name); + storage.unlockSharedData(*this, old_part_name); } std::optional IMergeTreeDataPart::keepSharedDataInDecoupledStorage() const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 58cf82d295c..04eb3e1a6c0 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -873,7 +873,7 @@ public: /// Overridden in StorageReplicatedMergeTree virtual bool unlockSharedData(const IMergeTreeDataPart &) const { return true; } - /// Unlock same part with other (old) name + /// Remove lock with old name for shared data part after rename virtual bool unlockSharedData(const IMergeTreeDataPart &, const String &) const { return true; } /// Fetch part only if some replica has it on shared storage like S3 diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 2bd7bee2f25..6861599a1ac 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -127,7 +127,7 @@ struct Settings; M(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \ M(Bool, allow_remote_fs_zero_copy_replication, true, "Allow Zero-copy replication over remote fs.", 0) \ M(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for Zero-copy table-independet info.", 0) \ - M(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during convertion process.", 0) \ + M(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during conversion process.", 0) \ M(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm.", 0) \ M(Bool, assign_part_uuids, false, "Generate UUIDs for parts. Before enabling check that all replicas support new format.", 0) \ M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited. This setting is the default that can be overridden by the query-level setting with the same name.", 0) \ diff --git a/utils/zero_copy/zero_copy_schema_converter.py b/utils/zero_copy/zero_copy_schema_converter.py index 922c7b342b4..6fdd03add5a 100755 --- a/utils/zero_copy/zero_copy_schema_converter.py +++ b/utils/zero_copy/zero_copy_schema_converter.py @@ -18,14 +18,14 @@ def parse_args(): parser.add_argument('-p', '--password', default='', help='ZooKeeper ACL password') parser.add_argument('-r', '--root', default='/clickhouse', help='ZooKeeper root path for ClickHouse') parser.add_argument('-z', '--zcroot', default='zero_copy', help='ZooKeeper node for new zero-copy data') - parser.add_argument('--dryrun', default=False, action='store_true', help='Do not perfome any actions') + parser.add_argument('--dryrun', default=False, action='store_true', help='Do not perform any actions') parser.add_argument('--cleanup', default=False, action='store_true', help='Clean old nodes') parser.add_argument('-v', '--verbose', action='store_true', default=False, help='Verbose mode') return parser.parse_args() -# Several folders to euristic that zookeepr node is folder node +# Several folders to heuristic that zookeepr node is folder node # May be false positive when someone creates set of tables with same paths table_nodes = ['alter_partition_version', 'block_numbers', 'blocks', 'columns', 'leader_election'] zc_nodes = ['zero_copy_s3', 'zero_copy_hdfs'] From ca9526e3a4bc66fde0abb83b8175196a39b97836 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 22 Dec 2021 00:39:51 +0300 Subject: [PATCH 0225/1260] Stupid fix --- tests/ci/keeper_jepsen_check.py | 3 +-- tests/ci/ssh.py | 11 +++++++++-- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/tests/ci/keeper_jepsen_check.py b/tests/ci/keeper_jepsen_check.py index 380bd501d3d..347e01908e6 100644 --- a/tests/ci/keeper_jepsen_check.py +++ b/tests/ci/keeper_jepsen_check.py @@ -143,8 +143,7 @@ if __name__ == "__main__": else: raise Exception("Cannot binary clickhouse among build results") - os.environ['JEPSEN_KEY'] = get_parameter_from_ssm("jepsen_ssh_key") - with SSHKey("JEPSEN_KEY"): + with SSHKey(key_value=get_parameter_from_ssm("jepsen_ssh_key")): ssh_auth_sock = os.environ['SSH_AUTH_SOCK'] auth_sock_dir = os.path.dirname(ssh_auth_sock) cmd = get_run_command(ssh_auth_sock, auth_sock_dir, pr_info, nodes_path, REPO_COPY, build_url, result_path, docker_image) diff --git a/tests/ci/ssh.py b/tests/ci/ssh.py index 1c0515364a8..f6309e31d0f 100644 --- a/tests/ci/ssh.py +++ b/tests/ci/ssh.py @@ -102,8 +102,15 @@ class SSHAgent: return stdout class SSHKey: - def __init__(self, key_name): - self.key = os.getenv(key_name) + def __init__(self, key_name=None, key_value=None): + if key_name is None and key_value is None: + raise Exception("Either key_name or key_value must be specified") + if key_name is not None and key_value is not None: + raise Exception("key_name or key_value must be specified") + if key_name is not None: + self.key = os.getenv(key_name) + else: + self.key = key_value self._key_pub = None self._ssh_agent = SSHAgent() From 2d226e287a55e5f8b59eae36182e4cc480aa711a Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 22 Dec 2021 15:11:39 +0800 Subject: [PATCH 0226/1260] update comment in LRUCache::set --- src/Common/LRUCache.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 88f73480885..5a38e8225c1 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -75,9 +75,9 @@ public: } /** - * set() will fail if there is no space left and no keys could be evicted. - * In some cases, a key can be only evicted when it is not refered by anyone. - */ + * set() will fail (return false) if there is no space left and no keys could be evicted. + * Eviction permission of each key is defined by EvictPolicy. In default policy there is no restriction. + */ bool set(const Key & key, const MappedPtr & mapped) { std::lock_guard lock(mutex); From 7e714641acf953fefd0289c209214f27b2ece757 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 22 Dec 2021 13:46:41 +0300 Subject: [PATCH 0227/1260] Add newline --- tests/ci/keeper_jepsen_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/keeper_jepsen_check.py b/tests/ci/keeper_jepsen_check.py index 347e01908e6..8207d783b0a 100644 --- a/tests/ci/keeper_jepsen_check.py +++ b/tests/ci/keeper_jepsen_check.py @@ -143,7 +143,7 @@ if __name__ == "__main__": else: raise Exception("Cannot binary clickhouse among build results") - with SSHKey(key_value=get_parameter_from_ssm("jepsen_ssh_key")): + with SSHKey(key_value=get_parameter_from_ssm("jepsen_ssh_key") + '\n'): ssh_auth_sock = os.environ['SSH_AUTH_SOCK'] auth_sock_dir = os.path.dirname(ssh_auth_sock) cmd = get_run_command(ssh_auth_sock, auth_sock_dir, pr_info, nodes_path, REPO_COPY, build_url, result_path, docker_image) From 709d63c59beaffb73eccd374e98aaea5ca35bedf Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 22 Dec 2021 16:05:07 +0300 Subject: [PATCH 0228/1260] Better test --- .github/workflows/main.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index e1fe9b485b2..0adbf1b784e 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -201,7 +201,7 @@ jobs: sudo rm -fr $TEMP_PATH KeeperJepsenRelease: needs: [BuilderBinRelease] - runs-on: [self-hosted, func-tester] + runs-on: [self-hosted, style-checker] steps: - name: Set envs run: | @@ -219,7 +219,7 @@ jobs: sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - - name: Functional test + - name: Jepsen Test run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH From 7cb6079fa22cf4afd582c8a30dff1ea2962e76d8 Mon Sep 17 00:00:00 2001 From: Alexey Date: Wed, 22 Dec 2021 15:52:47 +0000 Subject: [PATCH 0229/1260] ru --- .../settings.md | 40 +++++++++++++++++-- 1 file changed, 36 insertions(+), 4 deletions(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 1b0c7fc5897..0c8999cf046 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -436,26 +436,58 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ## interserver_http_credentials {#server-settings-interserver-http-credentials} -Имя пользователя и пароль, использующиеся для аутентификации при [репликации](../../operations/server-configuration-parameters/settings.md) движками Replicated\*. Это имя пользователя и пароль используются только для взаимодействия между репликами кластера и никак не связаны с аутентификацией клиентов ClickHouse. Сервер проверяет совпадение имени и пароля для соединяющихся с ним реплик, а также использует это же имя и пароль для соединения с другими репликами. Соответственно, эти имя и пароль должны быть прописаны одинаковыми для всех реплик кластера. -По умолчанию аутентификация не используется. +Имя пользователя и пароль, использующиеся для подключения к другим серверам при [репликации](../../engines/table-engines/mergetree-family/replication.md) движками Replicated\*. Сервер использует эти же учетные данные при аутентификации других реплик. Поэтому настройки `interserver_http_credentials` должны быть заданы одинаковыми для всех реплик кластера. + +По умолчанию, если секция `interserver_http_credentials` не задана в конфигурации, аутентификация при репликации не используется. !!! note "Примечание" - Эти учетные данные являются общими для обмена данными по протоколам `HTTP` и `HTTPS`. + Настройки `interserver_http_credentials` не относятся к [конфигурации](../../interfaces/cli.md#configuration_files) учетных данных клиента ClickHouse. + +!!! note "Примечание" + Учетные данные в `interserver_http_credentials` являются общими для репликации по `HTTP` и `HTTPS`. Раздел содержит следующие параметры: - `user` — имя пользователя. - `password` — пароль. +- `allow_empty` — Если `true`, то другие реплики могут подключаться без аутентификации, даже если учетные данные заданы. Если `false`, то подключения без аутентификации не допускаются. Значение по умолчанию: `false`. +- `old` — секция содержит старые значения `user` и `password`, которые используются в процессе изменения учетных данных. Можно указывать несколько секций `old`. -**Пример конфигурации** +**Изменение учетных данных** + +ClickHouse поддерживает динамическое изменение учетных данных. При этом не требуется одновременно останавливать все реплики, чтобы обновить конфигурацию. Изменение учетные данные выполняется за несколько шагов. + +Чтобы включить аутентификацию, установите `interserver_http_credentials.allow_empty` в `true` и задайте учетные данные. С такой конфигурацией разрешены подключения как с аутентификацией, так и без нее. + +``` xml + + admin + 111 + true + +``` + +После конфигурации всех реплик установите `allow_empty` в значение `false` или удалите эту настройку. Это сделает аутентификацию с новыми учетными данными обязательной. + +Чтобы изменить учетные данные, перенесите имя пользователя и пароль в секцию `interserver_http_credentials.old` и укажите в `user` и `password` новые значения. Сервер будет использовать новые учетные данные при подключении к другим репликам и при этом разрешать подключения как с новыми, так и со старыми учетными данными. ``` xml admin 222 + + admin + 111 + + + temp + 000 + ``` +Когда новые учетные данные заданы на всех репликах, старые учетные данные можно удалить из конфигурации. + ## keep_alive_timeout {#keep-alive-timeout} Время в секундах, в течение которого ClickHouse ожидает входящих запросов прежде чем закрыть соединение. Значение по умолчанию: 10 секунд. From be8187df09e62d1a748a2b218acaf03c67949684 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 22 Dec 2021 19:16:54 +0300 Subject: [PATCH 0230/1260] Fix report upload --- tests/ci/keeper_jepsen_check.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/ci/keeper_jepsen_check.py b/tests/ci/keeper_jepsen_check.py index 8207d783b0a..6f049eb7a57 100644 --- a/tests/ci/keeper_jepsen_check.py +++ b/tests/ci/keeper_jepsen_check.py @@ -167,8 +167,9 @@ if __name__ == "__main__": if any(r[1] == 'FAIL' for r in test_result): status = 'failure' description = 'Found invalid analysis (ノಥ益ಥ)ノ ┻━┻' - compress_fast(os.path.join(result_path, 'store'), 'jepsen_store.tar.gz') - additional_data.append(os.path.join(TEMP_PATH, 'jepsen_store.tar.gz')) + + compress_fast(os.path.join(result_path, 'store'), os.path.join(result_path, 'jepsen_store.tar.gz')) + additional_data.append(os.path.join(result_path, 'jepsen_store.tar.gz')) except: status = 'failure' description = 'No Jepsen output log' From 2470a8d380923f6d3da4a38778f1b97f2001c06c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 22 Dec 2021 18:26:50 +0100 Subject: [PATCH 0231/1260] Move tests --- ...lar_queries.reference => 02136_kill_scalar_queries.reference} | 0 ...02132_kill_scalar_queries.sh => 02136_kill_scalar_queries.sh} | 1 - 2 files changed, 1 deletion(-) rename tests/queries/0_stateless/{02132_kill_scalar_queries.reference => 02136_kill_scalar_queries.reference} (100%) rename tests/queries/0_stateless/{02132_kill_scalar_queries.sh => 02136_kill_scalar_queries.sh} (99%) diff --git a/tests/queries/0_stateless/02132_kill_scalar_queries.reference b/tests/queries/0_stateless/02136_kill_scalar_queries.reference similarity index 100% rename from tests/queries/0_stateless/02132_kill_scalar_queries.reference rename to tests/queries/0_stateless/02136_kill_scalar_queries.reference diff --git a/tests/queries/0_stateless/02132_kill_scalar_queries.sh b/tests/queries/0_stateless/02136_kill_scalar_queries.sh similarity index 99% rename from tests/queries/0_stateless/02132_kill_scalar_queries.sh rename to tests/queries/0_stateless/02136_kill_scalar_queries.sh index dd4d40d60a0..382f6555c66 100755 --- a/tests/queries/0_stateless/02132_kill_scalar_queries.sh +++ b/tests/queries/0_stateless/02136_kill_scalar_queries.sh @@ -20,4 +20,3 @@ wait_for_query_to_start "${QUERY_2_ID}" ${CLICKHOUSE_CLIENT} --query="KILL QUERY WHERE query_id='${QUERY_2_ID}' SYNC" wait - From 781fe363919b2708c121268ff88fc51fd4b2c075 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 22 Dec 2021 18:55:16 +0100 Subject: [PATCH 0232/1260] Add some basic tests to read rows with scalar subqueries --- .../02136_scalar_subquery_metrics.reference | 9 +++++++++ .../0_stateless/02136_scalar_subquery_metrics.sql | 13 +++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/02136_scalar_subquery_metrics.reference create mode 100644 tests/queries/0_stateless/02136_scalar_subquery_metrics.sql diff --git a/tests/queries/0_stateless/02136_scalar_subquery_metrics.reference b/tests/queries/0_stateless/02136_scalar_subquery_metrics.reference new file mode 100644 index 00000000000..7bef11d008f --- /dev/null +++ b/tests/queries/0_stateless/02136_scalar_subquery_metrics.reference @@ -0,0 +1,9 @@ +#02136_scalar_subquery_1 999 +#02136_scalar_subquery_2 999 0 +#02136_scalar_subquery_3 999 999 +#02136_scalar_subquery_4 999 +#02136_scalar_subquery_4 999 +1001 SELECT \'#02136_scalar_subquery_1\', (SELECT max(number) FROM numbers(1000)) as n; +2001 SELECT \'#02136_scalar_subquery_2\', (SELECT max(number) FROM numbers(1000)) as n, (SELECT min(number) FROM numbers(1000)) as n2; +1001 SELECT \'#02136_scalar_subquery_3\', (SELECT max(number) FROM numbers(1000)) as n, (SELECT max(number) FROM numbers(1000)) as n2; +1002 SELECT \'#02136_scalar_subquery_4\', (SELECT max(number) FROM numbers(1000)) as n FROM system.numbers LIMIT 2; diff --git a/tests/queries/0_stateless/02136_scalar_subquery_metrics.sql b/tests/queries/0_stateless/02136_scalar_subquery_metrics.sql new file mode 100644 index 00000000000..180610288aa --- /dev/null +++ b/tests/queries/0_stateless/02136_scalar_subquery_metrics.sql @@ -0,0 +1,13 @@ +SELECT '#02136_scalar_subquery_1', (SELECT max(number) FROM numbers(1000)) as n; +SELECT '#02136_scalar_subquery_2', (SELECT max(number) FROM numbers(1000)) as n, (SELECT min(number) FROM numbers(1000)) as n2; +SELECT '#02136_scalar_subquery_3', (SELECT max(number) FROM numbers(1000)) as n, (SELECT max(number) FROM numbers(1000)) as n2; -- Cached +SELECT '#02136_scalar_subquery_4', (SELECT max(number) FROM numbers(1000)) as n FROM system.numbers LIMIT 2; -- Cached + +SYSTEM FLUSH LOGS; +SELECT read_rows, query FROM system.query_log +WHERE + event_date > yesterday() + AND type = 'QueryFinish' + AND current_database == currentDatabase() + AND query LIKE 'SELECT ''#02136_scalar_subquery_%' +ORDER BY query ASC; From 02f7c878cc18dabf49083b11d0b29078f92363ae Mon Sep 17 00:00:00 2001 From: Alexey Date: Wed, 22 Dec 2021 19:52:50 +0000 Subject: [PATCH 0233/1260] minor updates --- .../ru/operations/server-configuration-parameters/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 0c8999cf046..19716dc8947 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -450,7 +450,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part - `user` — имя пользователя. - `password` — пароль. -- `allow_empty` — Если `true`, то другие реплики могут подключаться без аутентификации, даже если учетные данные заданы. Если `false`, то подключения без аутентификации не допускаются. Значение по умолчанию: `false`. +- `allow_empty` — Если `true`, то другие реплики могут подключаться без аутентификации, даже если учетные данные заданы. Если `false`, то подключение без аутентификации не допускается. Значение по умолчанию: `false`. - `old` — секция содержит старые значения `user` и `password`, которые используются в процессе изменения учетных данных. Можно указывать несколько секций `old`. **Изменение учетных данных** @@ -592,7 +592,7 @@ ClickHouse поддерживает динамическое изменение - `debug` - Настроить клиентскую библиотеку Sentry в debug режим. - `tmp_path` - Путь в файловой системе для временного хранения состояния отчетов о сбоях перед отправкой на сервер Sentry. -**Рекомендованые настройки** +**Рекомендованные настройки** ``` xml From c7e54b399051a42c2e53dac1ffa70990698a59f2 Mon Sep 17 00:00:00 2001 From: Alexey Date: Wed, 22 Dec 2021 20:00:31 +0000 Subject: [PATCH 0234/1260] On ru CLI page two anchor names copied from en page --- docs/ru/interfaces/cli.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index bbb66b70371..e2ca1a86284 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -114,7 +114,7 @@ $ clickhouse-client --param_tbl="numbers" --param_db="system" --param_col="numbe Параметры в конфигурационных файлах переопределяют значения по умолчанию. -### Параметры командной строки {#parametry-komandnoi-stroki} +### Параметры командной строки {#command-line-options} - `--host, -h` — имя сервера, по умолчанию — ‘localhost’. Вы можете использовать как имя, так и IPv4 или IPv6 адрес. - `--port` — порт для подключения, по умолчанию — 9000. Обратите внимание: для HTTP-интерфейса и нативного интерфейса используются разные порты. @@ -136,7 +136,7 @@ $ clickhouse-client --param_tbl="numbers" --param_db="system" --param_col="numbe Начиная с версии 20.5, в `clickhouse-client` есть автоматическая подсветка синтаксиса (включена всегда). -### Конфигурационные файлы {#konfiguratsionnye-faily} +### Конфигурационные файлы {#configuration_files} `clickhouse—client` использует первый существующий файл из: From dbb13036bc3438216882206142e78f78e84f992e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 22 Dec 2021 23:13:45 +0100 Subject: [PATCH 0235/1260] Hook progress callbacks with subqueries too --- src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp | 1 + src/Processors/QueryPlan/BuildQueryPipelineSettings.h | 2 ++ src/Processors/QueryPlan/QueryPlan.cpp | 1 + 3 files changed, 4 insertions(+) diff --git a/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp b/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp index 59ed1226787..fb3ed7f80fc 100644 --- a/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp +++ b/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp @@ -11,6 +11,7 @@ BuildQueryPipelineSettings BuildQueryPipelineSettings::fromContext(ContextPtr fr BuildQueryPipelineSettings settings; settings.actions_settings = ExpressionActionsSettings::fromSettings(from->getSettingsRef(), CompileExpressions::yes); settings.process_list_element = from->getProcessListElement(); + settings.progress_callback = from->getProgressCallback(); return settings; } diff --git a/src/Processors/QueryPlan/BuildQueryPipelineSettings.h b/src/Processors/QueryPlan/BuildQueryPipelineSettings.h index f106782ae5b..fadbd061fbd 100644 --- a/src/Processors/QueryPlan/BuildQueryPipelineSettings.h +++ b/src/Processors/QueryPlan/BuildQueryPipelineSettings.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -14,6 +15,7 @@ struct BuildQueryPipelineSettings { ExpressionActionsSettings actions_settings; QueryStatus * process_list_element = nullptr; + ProgressCallback progress_callback = nullptr; const ExpressionActionsSettings & getActionsSettings() const { return actions_settings; } static BuildQueryPipelineSettings fromContext(ContextPtr from); diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 6affecb5617..a271ef78dfa 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -180,6 +180,7 @@ QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline( for (auto & context : interpreter_context) last_pipeline->addInterpreterContext(std::move(context)); + last_pipeline->setProgressCallback(build_pipeline_settings.progress_callback); last_pipeline->setProcessListElement(build_pipeline_settings.process_list_element); return last_pipeline; From 9bb88c26d8b35c2ac83ad18a2565d99bcc567d42 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 22 Dec 2021 23:14:23 +0100 Subject: [PATCH 0236/1260] Add existing progress to the record of the output format progress --- src/Formats/FormatFactory.cpp | 27 ++++++++++++++++++++++----- 1 file changed, 22 insertions(+), 5 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 09e0876bb4f..74ccecda5c7 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -1,17 +1,18 @@ #include #include -#include -#include #include #include +#include +#include #include #include -#include #include -#include #include +#include +#include #include +#include #include #include @@ -234,6 +235,18 @@ InputFormatPtr FormatFactory::getInputFormat( return format; } +static void addExistingProgressToOutputFormat(OutputFormatPtr format, ContextPtr context) +{ + auto element_id = context->getProcessListElement(); + if (element_id) + { + /// While preparing the query there might have been progress (for example in subscalar subqueries) so add it here + auto current_progress = element_id->getProgressIn(); + Progress read_progress{current_progress.read_rows, current_progress.read_bytes, current_progress.total_rows_to_read}; + format->onProgress(read_progress); + } +} + OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible( const String & name, WriteBuffer & buf, @@ -262,7 +275,9 @@ OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible( if (context->hasQueryContext() && settings.log_queries) context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Format, name); - return std::make_shared(builder); + auto format = std::make_shared(builder); + addExistingProgressToOutputFormat(format, context); + return format; } return getOutputFormat(name, buf, sample, context, callback, _format_settings); @@ -302,6 +317,8 @@ OutputFormatPtr FormatFactory::getOutputFormat( if (auto * mysql = typeid_cast(format.get())) mysql->setContext(context); + addExistingProgressToOutputFormat(format, context); + return format; } From d205f9ecab98a8a25f5a9f76059c062d2aa9c0bc Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 23 Dec 2021 11:50:26 +0800 Subject: [PATCH 0237/1260] codes refactor --- programs/server/Server.cpp | 4 +- src/CMakeLists.txt | 1 + src/Common/ErrorCodes.cpp | 4 + src/Common/LRUCache.h | 46 +- .../Cache/ExternalDataSourceCache.cpp | 261 ++++++++++ src/Storages/Cache/ExternalDataSourceCache.h | 97 ++++ src/Storages/Cache/RemoteCacheController.cpp | 249 +++++++++ src/Storages/Cache/RemoteCacheController.h | 114 ++++ src/Storages/Cache/RemoteFileCachePolicy.h | 25 + src/Storages/Hive/HiveCommon.cpp | 2 +- src/Storages/Hive/StorageHive.cpp | 4 +- src/Storages/Hive/StorageHiveMetadata.cpp | 2 +- src/Storages/Hive/StorageHiveMetadata.h | 9 +- src/Storages/IRemoteFileMetadata.h | 15 +- src/Storages/RemoteReadBufferCache.cpp | 485 ------------------ src/Storages/RemoteReadBufferCache.h | 219 -------- 16 files changed, 783 insertions(+), 754 deletions(-) create mode 100644 src/Storages/Cache/ExternalDataSourceCache.cpp create mode 100644 src/Storages/Cache/ExternalDataSourceCache.h create mode 100644 src/Storages/Cache/RemoteCacheController.cpp create mode 100644 src/Storages/Cache/RemoteCacheController.h create mode 100644 src/Storages/Cache/RemoteFileCachePolicy.h delete mode 100644 src/Storages/RemoteReadBufferCache.cpp delete mode 100644 src/Storages/RemoteReadBufferCache.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index f7b76b333c0..974d7b1adb1 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -57,7 +57,7 @@ #include #include #include -#include +#include #include #include #include @@ -529,7 +529,7 @@ if (ThreadFuzzer::instance().isEffective()) UInt64 limit_size = config().getUInt64("local_cache_for_remote_fs.limit_size"); UInt64 bytes_read_before_flush = config().getUInt64("local_cache_for_remote_fs.bytes_read_before_flush", DBMS_DEFAULT_BUFFER_SIZE); - RemoteReadBufferCache::instance().initOnce(global_context, root_dir, limit_size, bytes_read_before_flush); + ExternalDataSourceCache::instance().initOnce(global_context, root_dir, limit_size, bytes_read_before_flush); } } diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 958cfd6d9f9..ada53ca0d24 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -115,6 +115,7 @@ if (USE_HDFS) add_headers_and_sources(dbms Disks/HDFS) endif() +add_headers_and_sources(dbms Storages/Cache) if (USE_HIVE) add_headers_and_sources(dbms Storages/Hive) endif() diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 1ba512f7be3..bac5ab9a46f 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -606,6 +606,10 @@ M(650, SNAPPY_UNCOMPRESS_FAILED) \ M(651, SNAPPY_COMPRESS_FAILED) \ M(652, NO_HIVEMETASTORE) \ + M(653, NOT_INIT)\ + M(654, DISK_OVERFLOW)\ + M(655, FILE_BROKEN)\ + M(656, END_OF_FILE)\ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 5a38e8225c1..22ed498bd60 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -25,12 +25,14 @@ struct TrivialWeightFunction template struct TrivialLRUCacheEvitPolicy { - inline bool canRelease(const T &) const + // To note that the arg maybe is null + inline bool canRelease(std::shared_ptr) const { return true; } - inline void release(T &) + // To note that the arg is null + inline void release(std::shared_ptr) { } }; @@ -89,10 +91,8 @@ public: template std::pair getOrSet(const Key & key, LoadFunc && load_func) { - MappedPtr value = nullptr; - bool is_value_loaded = false, is_value_updated = false; - std::tie(value, is_value_loaded, is_value_updated) = getOrTrySet(key, std::move(load_func)); - return std::make_pair(value, is_value_loaded); + auto [value, is_loaded, _] = getOrTrySet(key, std::move(load_func)); + return std::make_pair(value, is_loaded); } /// If the value for the key is in the cache, returns it. If it is not, calls load_func() to @@ -165,19 +165,17 @@ public: } /// If key is not in cache or the element can be released, return is true. otherwise, return is false - bool tryDel(const Key & key) + bool tryRemove(const Key & key) { std::lock_guard loc(mutex); auto it = cells.find(key); if (it == cells.end()) return true; auto & cell = it->second; - if (cell.value) - { - if (!evict_policy.canRelease(*cell.value)) - return false; - evict_policy.release(*cell.value); - } + if (!evict_policy.canRelease(cell.value)) + return false; + evict_policy.release(cell.value); + current_size -= cell.size; cells.erase(it); queue.erase(cell.queue_iterator); @@ -365,14 +363,13 @@ private: } else { - if (cell.value && !evict_policy.canRelease(*cell.value)) + if (!evict_policy.canRelease(cell.value)) { // the old value is refered by someone, cannot release now // in default policy, it is always true. return false; } - if (cell.value) - evict_policy.release(*cell.value); // release the old value. this action is empty in default policy. + evict_policy.release(cell.value); // release the old value. this action is empty in default policy. current_size -= cell.size; queue.splice(queue.end(), queue, cell.queue_iterator); } @@ -389,10 +386,9 @@ private: size_t current_weight_lost = 0; size_t queue_size = cells.size(); auto key_it = queue.begin(); - - while ((current_size + required_size_to_remove > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) - && (queue_size > 1) - && (key_it != queue.end())) + auto is_overflow = [&] { return (current_size + required_size_to_remove > max_size || (max_elements_size != 0 && queue_size > max_elements_size)); + }; + while (is_overflow() && (queue_size > 1) && (key_it != queue.end())) { const Key & key = *key_it; @@ -404,15 +400,11 @@ private: } const auto & cell = it->second; - bool can_evict = true; - if (cell.value) - can_evict = evict_policy.canRelease(*cell.value);// in default, it is true - if (can_evict) + if (evict_policy.canRelease(cell.value))// in default, it is true { // always call release() before erasing an element // in default, it's an empty action - if (cell.value) - evict_policy.release(*cell.value); + evict_policy.release(cell.value); current_size -= cell.size; current_weight_lost += cell.size; @@ -434,7 +426,7 @@ private: LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); abort(); } - return !(current_size + required_size_to_remove > max_size || (max_elements_size != 0 && queue_size > max_elements_size)); + return !is_overflow(); } /// Override this method if you want to track how much weight was lost in removeOverflow method. diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp new file mode 100644 index 00000000000..3a8dfd9bcaa --- /dev/null +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -0,0 +1,261 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace fs = std::filesystem; +namespace ProfileEvents +{ + extern const Event ExternalDataSourceLocalCacheReadBytes; +} +namespace DB +{ +namespace fs = std::filesystem; +namespace ErrorCodes +{ + extern const int OK; + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; + extern const int NOT_INIT; + extern const int DISK_OVERFLOW; + extern const int FILE_BROKEN; + extern const int END_OF_FILE; +} + + +RemoteReadBuffer::RemoteReadBuffer(size_t buff_size) : BufferWithOwnMemory(buff_size) +{ +} + +RemoteReadBuffer::~RemoteReadBuffer() +{ + if (file_cache_controller) + file_cache_controller->deallocFile(std::move(file_buffer)); +} + +std::unique_ptr RemoteReadBuffer::create(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer) +{ + auto * log = &Poco::Logger::get("RemoteReadBuffer"); + size_t buff_size = DBMS_DEFAULT_BUFFER_SIZE; + if (read_buffer) + buff_size = read_buffer->internalBuffer().size(); + /* + * in the new implement of ReadBufferFromHDFS, buffer size is 0. + * + * in the common case, we don't read bytes from readbuffer directly, so set buff_size = DBMS_DEFAULT_BUFFER_SIZE + * is OK. + * + * we need be careful with the case without local file reader. + */ + if (buff_size == 0) + buff_size = DBMS_DEFAULT_BUFFER_SIZE; + + auto remote_path = remote_file_metadata->remote_path; + auto remote_read_buffer = std::make_unique(buff_size); + ErrorCodes::ErrorCode error; + + std::tie(remote_read_buffer->file_cache_controller, read_buffer, error) = ExternalDataSourceCache::instance().createReader(context, remote_file_metadata, read_buffer); + if (remote_read_buffer->file_cache_controller == nullptr) + { + LOG_ERROR(log, "Failed to allocate local file for remote path: {}, reason: {}.", remote_path, error); + // read_buffer is the input one. + return read_buffer; + } + else + { + remote_read_buffer->file_buffer = remote_read_buffer->file_cache_controller->allocFile(); + if (!remote_read_buffer->file_buffer) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Create file readbuffer failed. {}", + remote_read_buffer->file_cache_controller->getLocalPath().string()); + } + remote_read_buffer->remote_file_size = remote_file_metadata->file_size; + return remote_read_buffer; +} + +bool RemoteReadBuffer::nextImpl() +{ + auto start_offset = file_buffer->getPosition(); + auto end_offset = start_offset + file_buffer->internalBuffer().size(); + file_cache_controller->waitMoreData(start_offset, end_offset); + + auto status = file_buffer->next(); + if (status) + { + BufferBase::set(file_buffer->buffer().begin(), + file_buffer->buffer().size(), + file_buffer->offset()); + ProfileEvents::increment(ProfileEvents::ExternalDataSourceLocalCacheReadBytes, file_buffer->available()); + } + return status; +} + +off_t RemoteReadBuffer::seek(off_t offset, int whence) +{ + if (!file_buffer) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot call seek() in this buffer. It's a bug!"); + /* + * Need to wait here. For example, the current file has been download at position X, but here we try to seek to + * postition Y (Y > X), it would fail. + */ + file_cache_controller->waitMoreData(offset, offset + file_buffer->internalBuffer().size()); + auto ret = file_buffer->seek(offset, whence); + BufferBase::set(file_buffer->buffer().begin(), + file_buffer->buffer().size(), + file_buffer->offset()); + return ret; +} + +off_t RemoteReadBuffer::getPosition() +{ + return file_buffer->getPosition(); +} + +ExternalDataSourceCache::ExternalDataSourceCache() = default; + +ExternalDataSourceCache::~ExternalDataSourceCache() = default; + +ExternalDataSourceCache & ExternalDataSourceCache::instance() +{ + static ExternalDataSourceCache instance; + return instance; +} + +void ExternalDataSourceCache::recoverCachedFilesMetadata( + const fs::path & current_path, + size_t current_depth, + size_t max_depth) +{ + if (current_depth >= max_depth) + { + std::vector invalid_paths; + for (auto const & dir : fs::directory_iterator{current_path}) + { + String path = dir.path(); + auto cache_controller = RemoteCacheController::recover(path); + if (!cache_controller) + { + invalid_paths.emplace_back(path); + continue; + } + if (!lru_caches->set(path, cache_controller)) + { + invalid_paths.emplace_back(path); + } + } + for (auto & path : invalid_paths) + { + fs::remove_all(path); + } + return; + } + + for (auto const & dir : fs::directory_iterator{current_path}) + { + recoverCachedFilesMetadata(dir.path(), current_depth + 1, max_depth); + } +} + +void ExternalDataSourceCache::recoverTask() +{ + recoverCachedFilesMetadata(root_dir, 1, 2); + initialized = true; + LOG_INFO(log, "Recovered from directory:{}", root_dir); +} + +void ExternalDataSourceCache::initOnce( + ContextPtr context, + const String & root_dir_, size_t limit_size_, size_t bytes_read_before_flush_) +{ + std::lock_guard lock(mutex); + if (isInitialized()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot initialize ExternalDataSourceCache twice"); + } + LOG_INFO( + log, "Initializing local cache for remote data sources. Local cache root path: {}, cache size limit: {}", root_dir_, limit_size_); + root_dir = root_dir_; + local_cache_bytes_read_before_flush = bytes_read_before_flush_; + lru_caches = std::make_unique(limit_size_); + + /// create if root_dir not exists + if (!fs::exists(fs::path(root_dir))) + { + fs::create_directories(fs::path(root_dir)); + } + + recover_task_holder = context->getSchedulePool().createTask("recover local cache metadata for remote files", [this]{ recoverTask(); }); + recover_task_holder->activateAndSchedule(); +} + +String ExternalDataSourceCache::calculateLocalPath(IRemoteFileMetadataPtr metadata) const +{ + // add version into the full_path, and not block to read the new version + String full_path = metadata->getName() + ":" + metadata->remote_path + + ":" + metadata->getVersion(); + UInt128 hashcode = sipHash128(full_path.c_str(), full_path.size()); + String hashcode_str = getHexUIntLowercase(hashcode); + return fs::path(root_dir) / hashcode_str.substr(0, 3) / hashcode_str; +} + +std::tuple, ErrorCodes::ErrorCode> +ExternalDataSourceCache::createReader(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr & read_buffer) +{ + // If something is wrong on startup, rollback to read from the original ReadBuffer + if (!isInitialized()) + { + LOG_ERROR(log, "ExternalDataSourceCache has not been initialized"); + return {nullptr, std::move(read_buffer), ErrorCodes::NOT_INIT}; + } + + auto remote_path = remote_file_metadata->remote_path; + const auto & last_modification_timestamp = remote_file_metadata->last_modification_timestamp; + auto local_path = calculateLocalPath(remote_file_metadata); + std::lock_guard lock(mutex); + auto cache = lru_caches->get(local_path); + if (cache) + { + // the remote file has been updated, need to redownload + if (!cache->isValid() || cache->isModified(remote_file_metadata)) + { + LOG_TRACE( + log, + "Remote file ({}) has been updated. Last saved modification time: {}, actual last modification time: {}", + remote_path, + std::to_string(cache->getLastModificationTimestamp()), + std::to_string(last_modification_timestamp)); + cache->markInvalid(); + } + else + { + return {cache, nullptr, ErrorCodes::OK}; + } + } + + if (!fs::exists(local_path)) + fs::create_directories(local_path); + + // cache is not found or is invalid + auto new_cache = std::make_shared(remote_file_metadata, local_path, local_cache_bytes_read_before_flush); + if (!lru_caches->set(local_path, new_cache)) + { + LOG_ERROR(log, "Insert the new cache failed. new file size:{}, current total size:{}", + remote_file_metadata->file_size, + lru_caches->weight()); + return {nullptr, std::move(read_buffer), ErrorCodes::DISK_OVERFLOW}; + } + new_cache->startBackgroundDownload(std::move(read_buffer), context->getSchedulePool()); + return {new_cache, nullptr, ErrorCodes::OK}; +} + +} diff --git a/src/Storages/Cache/ExternalDataSourceCache.h b/src/Storages/Cache/ExternalDataSourceCache.h new file mode 100644 index 00000000000..468cf3c300d --- /dev/null +++ b/src/Storages/Cache/ExternalDataSourceCache.h @@ -0,0 +1,97 @@ +#pragma once +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/* + * FIXME:RemoteReadBuffer derive from SeekableReadBufferWithSize may cause some risks, since it's not seekable in some cases + * But SeekableReadBuffer is not a interface which make it hard to fixup. + */ +class RemoteReadBuffer : public BufferWithOwnMemory +{ +public: + explicit RemoteReadBuffer(size_t buff_size); + ~RemoteReadBuffer() override; + static std::unique_ptr create(ContextPtr contex, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer); + + bool nextImpl() override; + off_t seek(off_t off, int whence) override; + off_t getPosition() override; + std::optional getTotalSize() override { return remote_file_size; } + +private: + std::shared_ptr file_cache_controller; + std::unique_ptr file_buffer; + size_t remote_file_size = 0; +}; + +class ExternalDataSourceCache : private boost::noncopyable +{ +public: + using CacheType = LRUCache, + RemoteFileCacheWeightFunction, RemoteFileCacheEvictPolicy>; + ~ExternalDataSourceCache(); + // global instance + static ExternalDataSourceCache & instance(); + + void initOnce(ContextPtr context, const String & root_dir_, size_t limit_size_, size_t bytes_read_before_flush_); + + inline bool isInitialized() const { return initialized; } + + std::tuple, ErrorCodes::ErrorCode> + createReader(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr & read_buffer); + + void updateTotalSize(size_t size) { total_size += size; } + +protected: + ExternalDataSourceCache(); + +private: + // root directory of local cache for remote filesystem + String root_dir; + size_t local_cache_bytes_read_before_flush = 0; + + std::atomic initialized = false; + std::atomic total_size; + std::mutex mutex; + std::unique_ptr lru_caches; + + Poco::Logger * log = &Poco::Logger::get("ExternalDataSourceCache"); + + String calculateLocalPath(IRemoteFileMetadataPtr meta) const; + + BackgroundSchedulePool::TaskHolder recover_task_holder; + void recoverTask(); + void recoverCachedFilesMetadata( + const std::filesystem::path & current_path, + size_t current_depth, + size_t max_depth); +}; + +} diff --git a/src/Storages/Cache/RemoteCacheController.cpp b/src/Storages/Cache/RemoteCacheController.cpp new file mode 100644 index 00000000000..b2d36ee38bc --- /dev/null +++ b/src/Storages/Cache/RemoteCacheController.cpp @@ -0,0 +1,249 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace fs = std::filesystem; +namespace ErrorCodes +{ + extern const int OK; + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; + extern const int END_OF_FILE; +} + +bool RemoteCacheController::loadInnerInformation(const fs::path & file_path) +{ + if (!fs::exists(file_path)) + return false; + std::ifstream info_file(file_path); + Poco::JSON::Parser info_parser; + auto info_json = info_parser.parse(info_file).extract(); + file_status = static_cast(info_json->get("file_status").convert()); + metadata_class = info_json->get("metadata_class").convert(); + info_file.close(); + return true; +} + +std::shared_ptr RemoteCacheController::recover(const std::filesystem::path & local_path_) +{ + auto * log = &Poco::Logger::get("RemoteCacheController"); + + if (!std::filesystem::exists(local_path_ / "data.bin")) + { + LOG_TRACE(log, "Invalid cached directory:{}", local_path_.string()); + return nullptr; + } + + auto cache_controller = std::make_shared(nullptr, local_path_, 0); + if (!cache_controller->loadInnerInformation(local_path_ / "info.txt") + || cache_controller->file_status != DOWNLOADED) + { + LOG_INFO(log, "Recover cached file failed. local path:{}", local_path_.string()); + return nullptr; + } + try + { + cache_controller->file_metadata_ptr = RemoteFileMetadataFactory::instance().get(cache_controller->metadata_class); + } + catch(...) + { + LOG_ERROR(log, "Get metadata class failed for {}", cache_controller->metadata_class); + cache_controller->file_metadata_ptr = nullptr; + } + if (!cache_controller->file_metadata_ptr) + { + // do not load this invalid cached file and clear it. the clear action is in + // ExternalDataSourceCache::recoverCachedFilesMetadata(), because deleting directories during iteration will + // cause unexpected behaviors + LOG_ERROR(log, "Cannot create the metadata class : {}. The cached file is invalid and will be remove. path:{}", + cache_controller->metadata_class, + local_path_.string()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid metadata class:{}", cache_controller->metadata_class); + } + std::ifstream metadata_file(local_path_ / "metadata.txt"); + if (!cache_controller->file_metadata_ptr->fromString(std::string((std::istreambuf_iterator(metadata_file)), + std::istreambuf_iterator()))) + { + LOG_ERROR(log, "Cannot load the metadata. The cached file is invalid and will be remove. path:{}", + local_path_.string()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid metadata file({}) for meta class {}", local_path_.string(), cache_controller->metadata_class); + } + + cache_controller->current_offset = fs::file_size(local_path_ / "data.bin"); + + ExternalDataSourceCache::instance().updateTotalSize(cache_controller->file_metadata_ptr->file_size); + return cache_controller; +} + +RemoteCacheController::RemoteCacheController( + IRemoteFileMetadataPtr file_metadata_, + const std::filesystem::path & local_path_, + size_t cache_bytes_before_flush_) + : file_metadata_ptr(file_metadata_) + , local_path(local_path_) + , valid(true) + , local_cache_bytes_read_before_flush(cache_bytes_before_flush_) + , current_offset(0) +{ + // on recover, file_metadata_ptr is null, but it will be allocated after loading from metadata.txt + // when we allocate a whole new file cache , file_metadata_ptr must not be null. + if (file_metadata_ptr) + { + metadata_class = file_metadata_ptr->getName(); + auto metadata_file_writer = std::make_unique((local_path_ / "metadata.txt").string()); + auto str_buf = file_metadata_ptr->toString(); + metadata_file_writer->write(str_buf.c_str(), str_buf.size()); + metadata_file_writer->close(); + } +} + +ErrorCodes::ErrorCode RemoteCacheController::waitMoreData(size_t start_offset_, size_t end_offset_) +{ + std::unique_lock lock{mutex}; + if (file_status == DOWNLOADED) + { + // finish reading + if (start_offset_ >= current_offset) + { + lock.unlock(); + return ErrorCodes::END_OF_FILE; + } + } + else // block until more data is ready + { + if (current_offset >= end_offset_) + { + lock.unlock(); + return ErrorCodes::OK; + } + else + more_data_signal.wait(lock, [this, end_offset_] { return file_status == DOWNLOADED || current_offset >= end_offset_; }); + } + lock.unlock(); + return ErrorCodes::OK; +} + +bool RemoteCacheController::isModified(IRemoteFileMetadataPtr file_metadata_) +{ + return !(file_metadata_ptr->getVersion() == file_metadata_->getVersion()); +} + +void RemoteCacheController::startBackgroundDownload(std::unique_ptr in_readbuffer_, BackgroundSchedulePool & thread_pool) +{ + data_file_writer = std::make_unique((fs::path(local_path) / "data.bin").string()); + flush(true); + ReadBufferPtr in_readbuffer(in_readbuffer_.release()); + download_task_holder = thread_pool.createTask("download remote file", + [this, in_readbuffer]{ backgroundDownload(in_readbuffer); }); + download_task_holder->activateAndSchedule(); +} + +void RemoteCacheController::backgroundDownload(ReadBufferPtr remote_read_buffer) +{ + file_status = DOWNLOADING; + size_t before_unflush_bytes = 0; + size_t total_bytes = 0; + while (!remote_read_buffer->eof()) + { + size_t bytes = remote_read_buffer->available(); + + data_file_writer->write(remote_read_buffer->position(), bytes); + remote_read_buffer->position() += bytes; + total_bytes += bytes; + before_unflush_bytes += bytes; + if (before_unflush_bytes >= local_cache_bytes_read_before_flush) + { + std::unique_lock lock(mutex); + current_offset += total_bytes; + total_bytes = 0; + flush(); + lock.unlock(); + more_data_signal.notify_all(); + before_unflush_bytes = 0; + } + } + std::unique_lock lock(mutex); + current_offset += total_bytes; + file_status = DOWNLOADED; + flush(true); + data_file_writer.reset(); + lock.unlock(); + more_data_signal.notify_all(); + ExternalDataSourceCache::instance().updateTotalSize(file_metadata_ptr->file_size); + LOG_TRACE(log, "Finish download into local path: {}, file metadata:{} ", local_path.string(), file_metadata_ptr->toString()); +} + +void RemoteCacheController::flush(bool need_flush_status) +{ + if (data_file_writer) + { + data_file_writer->sync(); + } + if (need_flush_status) + { + auto file_writer = std::make_unique(local_path / "info.txt"); + Poco::JSON::Object jobj; + jobj.set("file_status", static_cast(file_status)); + jobj.set("metadata_class", metadata_class); + std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + jobj.stringify(buf); + file_writer->write(buf.str().c_str(), buf.str().size()); + file_writer->close(); + } +} + +RemoteCacheController::~RemoteCacheController() +{ + if (download_task_holder) + download_task_holder->deactivate(); +} + +void RemoteCacheController::close() +{ + // delete directory + LOG_TRACE(log, "Removing the local cache. local path: {}", local_path.string()); + std::filesystem::remove_all(local_path); +} + +std::unique_ptr RemoteCacheController::allocFile() +{ + ReadSettings settings; + //settings.local_fs_method = LocalFSReadMethod::read; + auto file_buffer = createReadBufferFromFileBase((local_path / "data.bin").string(), settings); + + if (file_buffer) + { + std::lock_guard lock{mutex}; + opened_file_buffer_refs.insert(reinterpret_cast(file_buffer.get())); + } + return file_buffer; +} + +void RemoteCacheController::deallocFile(std::unique_ptr file_buffer) +{ + if (!file_buffer) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Try to release a null file buffer for {}", local_path.string()); + } + auto buffer_ref = reinterpret_cast(file_buffer.get()); + std::lock_guard lock{mutex}; + auto it = opened_file_buffer_refs.find(buffer_ref); + if (it == opened_file_buffer_refs.end()) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Try to deallocate file with invalid handler remote path: {}, local path: {}", + file_metadata_ptr->remote_path, + local_path.string()); + } + opened_file_buffer_refs.erase(it); +} + +} diff --git a/src/Storages/Cache/RemoteCacheController.h b/src/Storages/Cache/RemoteCacheController.h new file mode 100644 index 00000000000..a07ab4a92c9 --- /dev/null +++ b/src/Storages/Cache/RemoteCacheController.h @@ -0,0 +1,114 @@ +#pragma once +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +class RemoteCacheController +{ +public: + enum LocalFileStatus + { + TO_DOWNLOAD = 0, + DOWNLOADING = 1, + DOWNLOADED = 2, + }; + + RemoteCacheController( + IRemoteFileMetadataPtr file_metadata_, + const std::filesystem::path & local_path_, + size_t cache_bytes_before_flush_); + ~RemoteCacheController(); + + // recover from local disk + static std::shared_ptr + recover(const std::filesystem::path & local_path); + + /** + * Called by LocalCachedFileReader, must be used in pair + * The second value of the return tuple is the local_path to store file. + */ + std::unique_ptr allocFile(); + void deallocFile(std::unique_ptr buffer); + + /** + * when allocFile be called, count++. deallocFile be called, count--. + * the local file could be deleted only count==0 + */ + inline bool closable() + { + std::lock_guard lock{mutex}; + return opened_file_buffer_refs.empty(); + } + void close(); + + /** + * called in LocalCachedFileReader read(), the reading process would be blocked until + * enough data be downloaded. + * If the file has finished download, the process would unblocked + */ + ErrorCodes::ErrorCode waitMoreData(size_t start_offset_, size_t end_offset_); + + inline size_t size() const { return current_offset; } + + inline const std::filesystem::path & getLocalPath() { return local_path; } + inline String getRemotePath() const { return file_metadata_ptr->remote_path; } + + inline UInt64 getLastModificationTimestamp() const { return file_metadata_ptr->last_modification_timestamp; } + bool isModified(IRemoteFileMetadataPtr file_metadata_); + inline void markInvalid() + { + std::lock_guard lock(mutex); + valid = false; + } + inline bool isValid() + { + std::lock_guard lock(mutex); + return valid; + } + IRemoteFileMetadataPtr getFileMetadata() { return file_metadata_ptr; } + inline size_t getFileSize() const { return file_metadata_ptr->file_size; } + + void startBackgroundDownload(std::unique_ptr in_readbuffer_, BackgroundSchedulePool & thread_pool); + +private: + // flush file and status information + void flush(bool need_flush_status = false); + bool loadInnerInformation(const std::filesystem::path & file_path); + + BackgroundSchedulePool::TaskHolder download_task_holder; + void backgroundDownload(ReadBufferPtr remote_read_buffer); + + std::mutex mutex; + std::condition_variable more_data_signal; + + std::set opened_file_buffer_refs; // refer to a buffer address + + String metadata_class; + LocalFileStatus file_status = TO_DOWNLOAD; // for tracking download process + IRemoteFileMetadataPtr file_metadata_ptr; + std::filesystem::path local_path; + + bool valid; + size_t local_cache_bytes_read_before_flush; + size_t current_offset; + + //std::shared_ptr remote_read_buffer; + std::unique_ptr data_file_writer; + + Poco::Logger * log = &Poco::Logger::get("RemoteCacheController"); +}; +using RemoteCacheControllerPtr = std::shared_ptr; + +} diff --git a/src/Storages/Cache/RemoteFileCachePolicy.h b/src/Storages/Cache/RemoteFileCachePolicy.h new file mode 100644 index 00000000000..4866247ee6e --- /dev/null +++ b/src/Storages/Cache/RemoteFileCachePolicy.h @@ -0,0 +1,25 @@ +#pragma once +namespace DB +{ +struct RemoteFileCacheWeightFunction +{ + size_t operator()(const RemoteCacheController & cache) const + { + return cache.getFileSize(); + } +}; + +struct RemoteFileCacheEvictPolicy +{ + bool canRelease(std::shared_ptr cache) const + { + return (!cache || cache->closable()); + } + void release(std::shared_ptr cache) + { + if (cache) + cache->close(); + } +}; + +} diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 6bb72f7aed8..4612e006258 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -112,7 +112,7 @@ void HiveMetastoreClient::clearTableMetadata(const String & db_name, const Strin HiveTableMetadataPtr metadata = table_metadata_cache.get(cache_key); if (metadata) { - if (!table_metadata_cache.tryDel(cache_key)) + if (!table_metadata_cache.tryRemove(cache_key)) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Try to clear table metadata failed."); } diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 46599a18961..f4536c07d93 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -19,7 +19,7 @@ #include #include #include -#include +#include #include #include #include @@ -160,7 +160,7 @@ public: /// Use local cache for remote storage if enabled. std::unique_ptr remote_read_buf; - if (RemoteReadBufferCache::instance().isInitialized() && getContext()->getSettingsRef().use_local_cache_for_remote_storage) + if (ExternalDataSourceCache::instance().isInitialized() && getContext()->getSettingsRef().use_local_cache_for_remote_storage) remote_read_buf = RemoteReadBuffer::create(getContext(), std::make_shared("Hive", getNameNodeCluster(hdfs_namenode_url), uri_with_path, curr_file->getSize(), curr_file->getLastModTs()), std::move(raw_read_buf)); diff --git a/src/Storages/Hive/StorageHiveMetadata.cpp b/src/Storages/Hive/StorageHiveMetadata.cpp index 379d8721d6e..69fb444c19c 100644 --- a/src/Storages/Hive/StorageHiveMetadata.cpp +++ b/src/Storages/Hive/StorageHiveMetadata.cpp @@ -38,7 +38,7 @@ bool StorageHiveMetadata::fromString(const String &buf) String StorageHiveMetadata::getVersion() const { - return std::to_string(getLastModificationTimestamp()); + return std::to_string(last_modification_timestamp); } void registerStorageHiveMetadataCreator() diff --git a/src/Storages/Hive/StorageHiveMetadata.h b/src/Storages/Hive/StorageHiveMetadata.h index 453db188503..60bbe2fe0aa 100644 --- a/src/Storages/Hive/StorageHiveMetadata.h +++ b/src/Storages/Hive/StorageHiveMetadata.h @@ -10,8 +10,11 @@ public: const String & cluster_, const String & remote_path_, size_t file_size_, - UInt64 last_modification_timestamp_): - IRemoteFileMetadata(remote_path_, file_size_, last_modification_timestamp_),schema(schema_), cluster(cluster_){} + UInt64 last_modification_timestamp_) : schema(schema_), cluster(cluster_){ + remote_path = remote_path_; + file_size = file_size_; + last_modification_timestamp = last_modification_timestamp_; + } ~StorageHiveMetadata() override; String getName() const override { return "StorageHiveMetadata"; } @@ -19,7 +22,7 @@ public: String getCluster() const { return cluster; } String toString() const override; - bool fromString(const String &buf) override; + bool fromString(const String & buf) override; String getVersion() const override; private: String schema; diff --git a/src/Storages/IRemoteFileMetadata.h b/src/Storages/IRemoteFileMetadata.h index a2b4ac2a9be..6634fc2a020 100644 --- a/src/Storages/IRemoteFileMetadata.h +++ b/src/Storages/IRemoteFileMetadata.h @@ -10,21 +10,8 @@ namespace DB class IRemoteFileMetadata { public: - IRemoteFileMetadata() = default; - IRemoteFileMetadata(const String & remote_path_, - size_t file_size_, - UInt64 last_modification_timestamp_): - remote_path(remote_path_) - ,file_size(file_size_) - ,last_modification_timestamp(last_modification_timestamp_) - { - } virtual ~IRemoteFileMetadata() = default; virtual String getName() const = 0; //class name - // methods for basic information - inline size_t getFileSize() const { return file_size; } - inline String getRemotePath() const { return remote_path; } - inline UInt64 getLastModificationTimestamp() const { return last_modification_timestamp; } // deserialize virtual bool fromString(const String & buf) = 0; @@ -33,7 +20,7 @@ public: // used for comparing two file metadatas are the same or not. virtual String getVersion() const = 0; -protected: + String remote_path; size_t file_size = 0; UInt64 last_modification_timestamp = 0; diff --git a/src/Storages/RemoteReadBufferCache.cpp b/src/Storages/RemoteReadBufferCache.cpp deleted file mode 100644 index be25345625e..00000000000 --- a/src/Storages/RemoteReadBufferCache.cpp +++ /dev/null @@ -1,485 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace fs = std::filesystem; -namespace ProfileEvents -{ - extern const Event ExternalDataSourceLocalCacheReadBytes; -} -namespace DB -{ -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int LOGICAL_ERROR; -} - -bool RemoteCacheController::loadInnerInformation(const fs::path & file_path) -{ - if (!fs::exists(file_path)) - return false; - std::ifstream info_file(file_path); - Poco::JSON::Parser info_parser; - auto info_json = info_parser.parse(info_file).extract(); - file_status = static_cast(info_json->get("file_status").convert()); - metadata_class = info_json->get("metadata_class").convert(); - info_file.close(); - return true; -} - -std::shared_ptr RemoteCacheController::recover(const std::filesystem::path & local_path_) -{ - auto * log = &Poco::Logger::get("RemoteCacheController"); - - if (!std::filesystem::exists(local_path_ / "data.bin")) - { - LOG_TRACE(log, "Invalid cached directory:{}", local_path_.string()); - return nullptr; - } - - auto cache_controller = std::make_shared(nullptr, local_path_, 0); - if (!cache_controller->loadInnerInformation(local_path_ / "info.txt") - || cache_controller->file_status != DOWNLOADED) - { - LOG_INFO(log, "Recover cached file failed. local path:{}", local_path_.string()); - return nullptr; - } - try - { - cache_controller->file_metadata_ptr = RemoteFileMetadataFactory::instance().get(cache_controller->metadata_class); - } - catch(...) - { - LOG_ERROR(log, "Get metadata class failed for {}", cache_controller->metadata_class); - cache_controller->file_metadata_ptr = nullptr; - } - if (!cache_controller->file_metadata_ptr) - { - // do not load this invalid cached file and clear it. the clear action is in - // RemoteReadBufferCache::recoverCachedFilesMetadata(), because deleting directories during iteration will - // cause unexpected behaviors - LOG_ERROR(log, "Cannot create the metadata class : {}. The cached file is invalid and will be remove. path:{}", - cache_controller->metadata_class, - local_path_.string()); - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid metadata class:{}", cache_controller->metadata_class); - } - std::ifstream metadata_file(local_path_ / "metadata.txt"); - if (!cache_controller->file_metadata_ptr->fromString(std::string((std::istreambuf_iterator(metadata_file)), - std::istreambuf_iterator()))) - { - LOG_ERROR(log, "Cannot load the metadata. The cached file is invalid and will be remove. path:{}", - local_path_.string()); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid metadata file({}) for meta class {}", local_path_.string(), cache_controller->metadata_class); - } - - cache_controller->current_offset = fs::file_size(local_path_ / "data.bin"); - - RemoteReadBufferCache::instance().updateTotalSize(cache_controller->file_metadata_ptr->getFileSize()); - return cache_controller; -} - -RemoteCacheController::RemoteCacheController( - IRemoteFileMetadataPtr file_metadata_, - const std::filesystem::path & local_path_, - size_t cache_bytes_before_flush_) - : file_metadata_ptr(file_metadata_) - , local_path(local_path_) - , valid(true) - , local_cache_bytes_read_before_flush(cache_bytes_before_flush_) - , current_offset(0) -{ - // on recover, file_metadata_ptr is null, but it will be allocated after loading from metadata.txt - // when we allocate a whole new file cache , file_metadata_ptr must not be null. - if (file_metadata_ptr) - { - metadata_class = file_metadata_ptr->getName(); - auto metadata_file_writer = std::make_unique((local_path_ / "metadata.txt").string()); - auto str_buf = file_metadata_ptr->toString(); - metadata_file_writer->write(str_buf.c_str(), str_buf.size()); - metadata_file_writer->close(); - } -} - -RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offset_, size_t end_offset_) -{ - std::unique_lock lock{mutex}; - if (file_status == DOWNLOADED) - { - // finish reading - if (start_offset_ >= current_offset) - { - lock.unlock(); - return RemoteReadBufferCacheError::END_OF_FILE; - } - } - else // block until more data is ready - { - if (current_offset >= end_offset_) - { - lock.unlock(); - return RemoteReadBufferCacheError::OK; - } - else - more_data_signal.wait(lock, [this, end_offset_] { return file_status == DOWNLOADED || current_offset >= end_offset_; }); - } - lock.unlock(); - return RemoteReadBufferCacheError::OK; -} - -bool RemoteCacheController::checkFileChanged(IRemoteFileMetadataPtr file_metadata_) -{ - return !(file_metadata_ptr->getVersion() == file_metadata_->getVersion()); -} - -void RemoteCacheController::startBackgroundDownload(std::unique_ptr in_readbuffer_, BackgroundSchedulePool & thread_pool) -{ - data_file_writer = std::make_unique((fs::path(local_path) / "data.bin").string()); - flush(true); - ReadBufferPtr in_readbuffer(in_readbuffer_.release()); - download_task_holder = thread_pool.createTask("download remote file", - [this, in_readbuffer]{ backgroundDownload(in_readbuffer); }); - download_task_holder->activateAndSchedule(); -} - -void RemoteCacheController::backgroundDownload(ReadBufferPtr remote_read_buffer) -{ - file_status = DOWNLOADING; - size_t before_unflush_bytes = 0; - size_t total_bytes = 0; - while (!remote_read_buffer->eof()) - { - size_t bytes = remote_read_buffer->available(); - - data_file_writer->write(remote_read_buffer->position(), bytes); - remote_read_buffer->position() += bytes; - total_bytes += bytes; - before_unflush_bytes += bytes; - if (before_unflush_bytes >= local_cache_bytes_read_before_flush) - { - std::unique_lock lock(mutex); - current_offset += total_bytes; - total_bytes = 0; - flush(); - lock.unlock(); - more_data_signal.notify_all(); - before_unflush_bytes = 0; - } - } - std::unique_lock lock(mutex); - current_offset += total_bytes; - file_status = DOWNLOADED; - flush(true); - data_file_writer.reset(); - lock.unlock(); - more_data_signal.notify_all(); - RemoteReadBufferCache::instance().updateTotalSize(file_metadata_ptr->getFileSize()); - LOG_TRACE(log, "Finish download into local path: {}, file metadata:{} ", local_path.string(), file_metadata_ptr->toString()); -} - -void RemoteCacheController::flush(bool need_flush_status) -{ - if (data_file_writer) - { - data_file_writer->sync(); - } - if (need_flush_status) - { - auto file_writer = std::make_unique(local_path / "info.txt"); - Poco::JSON::Object jobj; - jobj.set("file_status", static_cast(file_status)); - jobj.set("metadata_class", metadata_class); - std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - jobj.stringify(buf); - file_writer->write(buf.str().c_str(), buf.str().size()); - file_writer->close(); - } -} - -RemoteCacheController::~RemoteCacheController() -{ - if (download_task_holder) - download_task_holder->deactivate(); -} - -void RemoteCacheController::close() -{ - // delete directory - LOG_TRACE(log, "Removing the local cache. local path: {}", local_path.string()); - std::filesystem::remove_all(local_path); -} - -std::unique_ptr RemoteCacheController::allocFile() -{ - ReadSettings settings; - //settings.local_fs_method = LocalFSReadMethod::read; - auto file_buffer = createReadBufferFromFileBase((local_path / "data.bin").string(), settings); - - if (file_buffer) - { - std::lock_guard lock{mutex}; - opened_file_buffer_refs.insert(reinterpret_cast(file_buffer.get())); - } - return file_buffer; -} - -void RemoteCacheController::deallocFile(std::unique_ptr file_buffer) -{ - if (!file_buffer) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Try to release a null file buffer for {}", local_path.string()); - } - auto buffer_ref = reinterpret_cast(file_buffer.get()); - std::lock_guard lock{mutex}; - auto it = opened_file_buffer_refs.find(buffer_ref); - if (it == opened_file_buffer_refs.end()) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Try to deallocate file with invalid handler remote path: {}, local path: {}", - file_metadata_ptr->getRemotePath(), - local_path.string()); - } - opened_file_buffer_refs.erase(it); -} - -RemoteReadBuffer::RemoteReadBuffer(size_t buff_size) : BufferWithOwnMemory(buff_size) -{ -} - -RemoteReadBuffer::~RemoteReadBuffer() -{ - if (file_cache_controller) - file_cache_controller->deallocFile(std::move(file_buffer)); -} - -std::unique_ptr RemoteReadBuffer::create(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer) -{ - auto * log = &Poco::Logger::get("RemoteReadBuffer"); - size_t buff_size = DBMS_DEFAULT_BUFFER_SIZE; - if (read_buffer) - buff_size = read_buffer->internalBuffer().size(); - /* - * in the new implement of ReadBufferFromHDFS, buffer size is 0. - * - * in the common case, we don't read bytes from readbuffer directly, so set buff_size = DBMS_DEFAULT_BUFFER_SIZE - * is OK. - * - * we need be careful with the case without local file reader. - */ - if (buff_size == 0) - buff_size = DBMS_DEFAULT_BUFFER_SIZE; - - auto remote_path = remote_file_metadata->getRemotePath(); - auto remote_read_buffer = std::make_unique(buff_size); - RemoteReadBufferCacheError error; - - std::tie(remote_read_buffer->file_cache_controller, read_buffer, error) = RemoteReadBufferCache::instance().createReader(context, remote_file_metadata, read_buffer); - if (remote_read_buffer->file_cache_controller == nullptr) - { - LOG_ERROR(log, "Failed to allocate local file for remote path: {}, reason: {}.", remote_path, error); - // read_buffer is the input one. - return read_buffer; - } - else - { - remote_read_buffer->file_buffer = remote_read_buffer->file_cache_controller->allocFile(); - if (!remote_read_buffer->file_buffer) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Create file readbuffer failed. {}", - remote_read_buffer->file_cache_controller->getLocalPath().string()); - } - remote_read_buffer->remote_file_size = remote_file_metadata->getFileSize(); - return remote_read_buffer; -} - -bool RemoteReadBuffer::nextImpl() -{ - auto start_offset = file_buffer->getPosition(); - auto end_offset = start_offset + file_buffer->internalBuffer().size(); - file_cache_controller->waitMoreData(start_offset, end_offset); - - auto status = file_buffer->next(); - if (status) - { - BufferBase::set(file_buffer->buffer().begin(), - file_buffer->buffer().size(), - file_buffer->offset()); - ProfileEvents::increment(ProfileEvents::ExternalDataSourceLocalCacheReadBytes, file_buffer->available()); - } - return status; -} - -off_t RemoteReadBuffer::seek(off_t offset, int whence) -{ - if (!file_buffer) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot call seek() in this buffer. It's a bug!"); - /* - * Need to wait here. For example, the current file has been download at position X, but here we try to seek to - * postition Y (Y > X), it would fail. - */ - file_cache_controller->waitMoreData(offset, offset + file_buffer->internalBuffer().size()); - auto ret = file_buffer->seek(offset, whence); - BufferBase::set(file_buffer->buffer().begin(), - file_buffer->buffer().size(), - file_buffer->offset()); - return ret; -} - -off_t RemoteReadBuffer::getPosition() -{ - return file_buffer->getPosition(); -} - -RemoteReadBufferCache::RemoteReadBufferCache() = default; - -RemoteReadBufferCache::~RemoteReadBufferCache() = default; - -RemoteReadBufferCache & RemoteReadBufferCache::instance() -{ - static RemoteReadBufferCache instance; - return instance; -} - -void RemoteReadBufferCache::recoverCachedFilesMetadata( - const fs::path & current_path, - size_t current_depth, - size_t max_depth) -{ - if (current_depth >= max_depth) - { - std::vector invalid_paths; - for (auto const & dir : fs::directory_iterator{current_path}) - { - String path = dir.path(); - auto cache_controller = RemoteCacheController::recover(path); - if (!cache_controller) - { - invalid_paths.emplace_back(path); - continue; - } - if (!lru_caches->set(path, cache_controller)) - { - invalid_paths.emplace_back(path); - } - } - for (auto & path : invalid_paths) - { - fs::remove_all(path); - } - return; - } - - for (auto const & dir : fs::directory_iterator{current_path}) - { - recoverCachedFilesMetadata(dir.path(), current_depth + 1, max_depth); - } -} - -void RemoteReadBufferCache::recoverTask() -{ - recoverCachedFilesMetadata(root_dir, 1, 2); - initialized = true; - LOG_INFO(log, "Recovered from directory:{}", root_dir); -} - -void RemoteReadBufferCache::initOnce( - ContextPtr context, - const String & root_dir_, size_t limit_size_, size_t bytes_read_before_flush_) -{ - std::lock_guard lock(mutex); - if (isInitialized()) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot initialize RemoteReadBufferCache twice"); - } - LOG_INFO( - log, "Initializing local cache for remote data sources. Local cache root path: {}, cache size limit: {}", root_dir_, limit_size_); - root_dir = root_dir_; - local_cache_bytes_read_before_flush = bytes_read_before_flush_; - lru_caches = std::make_unique(limit_size_); - - /// create if root_dir not exists - if (!fs::exists(fs::path(root_dir))) - { - fs::create_directories(fs::path(root_dir)); - } - - recover_task_holder = context->getSchedulePool().createTask("recover local cache metadata for remote files", [this]{ recoverTask(); }); - recover_task_holder->activateAndSchedule(); -} - -String RemoteReadBufferCache::calculateLocalPath(IRemoteFileMetadataPtr metadata) const -{ - // add version into the full_path, and not block to read the new version - String full_path = metadata->getName() + ":" + metadata->getRemotePath() - + ":" + metadata->getVersion(); - UInt128 hashcode = sipHash128(full_path.c_str(), full_path.size()); - String hashcode_str = getHexUIntLowercase(hashcode); - return fs::path(root_dir) / hashcode_str.substr(0, 3) / hashcode_str; -} - -std::tuple, RemoteReadBufferCacheError> -RemoteReadBufferCache::createReader(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr & read_buffer) -{ - // If something is wrong on startup, rollback to read from the original ReadBuffer - if (!isInitialized()) - { - LOG_ERROR(log, "RemoteReadBufferCache has not been initialized"); - return {nullptr, std::move(read_buffer), RemoteReadBufferCacheError::NOT_INIT}; - } - - auto remote_path = remote_file_metadata->getRemotePath(); - const auto & last_modification_timestamp = remote_file_metadata->getLastModificationTimestamp(); - auto local_path = calculateLocalPath(remote_file_metadata); - std::lock_guard lock(mutex); - auto cache = lru_caches->get(local_path); - if (cache) - { - // the remote file has been updated, need to redownload - if (!cache->isValid() || cache->checkFileChanged(remote_file_metadata)) - { - LOG_TRACE( - log, - "Remote file ({}) has been updated. Last saved modification time: {}, actual last modification time: {}", - remote_path, - std::to_string(cache->getLastModificationTimestamp()), - std::to_string(last_modification_timestamp)); - cache->markInvalid(); - } - else - { - return {cache, nullptr, RemoteReadBufferCacheError::OK}; - } - } - - if (!fs::exists(local_path)) - fs::create_directories(local_path); - - // cache is not found or is invalid - auto new_cache = std::make_shared(remote_file_metadata, local_path, local_cache_bytes_read_before_flush); - if (!lru_caches->set(local_path, new_cache)) - { - LOG_ERROR(log, "Insert the new cache failed. new file size:{}, current total size:{}", - remote_file_metadata->getFileSize(), - lru_caches->weight()); - return {nullptr, std::move(read_buffer), RemoteReadBufferCacheError::DISK_FULL}; - } - new_cache->startBackgroundDownload(std::move(read_buffer), context->getSchedulePool()); - return {new_cache, nullptr, RemoteReadBufferCacheError::OK}; -} - -} diff --git a/src/Storages/RemoteReadBufferCache.h b/src/Storages/RemoteReadBufferCache.h deleted file mode 100644 index 126c8249be6..00000000000 --- a/src/Storages/RemoteReadBufferCache.h +++ /dev/null @@ -1,219 +0,0 @@ -#pragma once -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -enum class RemoteReadBufferCacheError : int8_t -{ - OK, - NOT_INIT = 10, - DISK_FULL = 11, - FILE_INVALID = 12, - END_OF_FILE = 20, -}; - -class RemoteCacheController -{ -public: - enum LocalFileStatus - { - TO_DOWNLOAD = 0, - DOWNLOADING = 1, - DOWNLOADED = 2, - }; - - RemoteCacheController( - IRemoteFileMetadataPtr file_metadata_, - const std::filesystem::path & local_path_, - size_t cache_bytes_before_flush_); - ~RemoteCacheController(); - - // recover from local disk - static std::shared_ptr - recover(const std::filesystem::path & local_path); - - /** - * Called by LocalCachedFileReader, must be used in pair - * The second value of the return tuple is the local_path to store file. - */ - std::unique_ptr allocFile(); - void deallocFile(std::unique_ptr buffer); - - /** - * when allocFile be called, count++. deallocFile be called, count--. - * the local file could be deleted only count==0 - */ - inline bool closable() - { - std::lock_guard lock{mutex}; - //return opened_file_streams.empty() && remote_read_buffer == nullptr; - return opened_file_buffer_refs.empty() && file_status == DOWNLOADED; - } - void close(); - - /** - * called in LocalCachedFileReader read(), the reading process would be blocked until - * enough data be downloaded. - * If the file has finished download, the process would unblocked - */ - RemoteReadBufferCacheError waitMoreData(size_t start_offset_, size_t end_offset_); - - inline size_t size() const { return current_offset; } - - inline const std::filesystem::path & getLocalPath() { return local_path; } - inline String getRemotePath() const { return file_metadata_ptr->getRemotePath(); } - - inline UInt64 getLastModificationTimestamp() const { return file_metadata_ptr->getLastModificationTimestamp(); } - bool checkFileChanged(IRemoteFileMetadataPtr file_metadata_); - inline void markInvalid() - { - std::lock_guard lock(mutex); - valid = false; - } - inline bool isValid() - { - std::lock_guard lock(mutex); - return valid; - } - IRemoteFileMetadataPtr getFileMetadata() { return file_metadata_ptr; } - inline size_t getFileSize() const { return file_metadata_ptr->getFileSize(); } - - void startBackgroundDownload(std::unique_ptr in_readbuffer_, BackgroundSchedulePool & thread_pool); - -private: - // flush file and status information - void flush(bool need_flush_status = false); - bool loadInnerInformation(const std::filesystem::path & file_path); - - BackgroundSchedulePool::TaskHolder download_task_holder; - void backgroundDownload(ReadBufferPtr remote_read_buffer); - - std::mutex mutex; - std::condition_variable more_data_signal; - - std::set opened_file_buffer_refs; // refer to a buffer address - - String metadata_class; - LocalFileStatus file_status = TO_DOWNLOAD; // for tracking download process - IRemoteFileMetadataPtr file_metadata_ptr; - std::filesystem::path local_path; - - bool valid; - size_t local_cache_bytes_read_before_flush; - size_t current_offset; - - //std::shared_ptr remote_read_buffer; - std::unique_ptr data_file_writer; - - Poco::Logger * log = &Poco::Logger::get("RemoteCacheController"); -}; -using RemoteCacheControllerPtr = std::shared_ptr; - -/* - * FIXME:RemoteReadBuffer derive from SeekableReadBufferWithSize may cause some risks, since it's not seekable in some cases - * But SeekableReadBuffer is not a interface which make it hard to fixup. - */ -class RemoteReadBuffer : public BufferWithOwnMemory -{ -public: - explicit RemoteReadBuffer(size_t buff_size); - ~RemoteReadBuffer() override; - static std::unique_ptr create(ContextPtr contex, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer); - - bool nextImpl() override; - off_t seek(off_t off, int whence) override; - off_t getPosition() override; - std::optional getTotalSize() override { return remote_file_size; } - -private: - std::shared_ptr file_cache_controller; - std::unique_ptr file_buffer; - size_t remote_file_size = 0; -}; - -struct RemoteFileCacheWeightFunction -{ - size_t operator()(const RemoteCacheController & cache) const - { - return cache.getFileSize(); - } -}; - -struct RemoteFileCacheEvictPolicy -{ - bool canRelease(RemoteCacheController & cache) const - { - return cache.closable(); - } - void release(RemoteCacheController & cache) - { - cache.close(); - } -}; - -class RemoteReadBufferCache : private boost::noncopyable -{ -public: - using CacheType = LRUCache, - RemoteFileCacheWeightFunction, RemoteFileCacheEvictPolicy>; - ~RemoteReadBufferCache(); - // global instance - static RemoteReadBufferCache & instance(); - - void initOnce(ContextPtr context, const String & root_dir_, size_t limit_size_, size_t bytes_read_before_flush_); - - inline bool isInitialized() const { return initialized; } - - std::tuple, RemoteReadBufferCacheError> - createReader(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr & read_buffer); - - void updateTotalSize(size_t size) { total_size += size; } - -protected: - RemoteReadBufferCache(); - -private: - // root directory of local cache for remote filesystem - String root_dir; - size_t local_cache_bytes_read_before_flush = 0; - - std::atomic initialized = false; - std::atomic total_size; - std::mutex mutex; - std::unique_ptr lru_caches; - - Poco::Logger * log = &Poco::Logger::get("RemoteReadBufferCache"); - - String calculateLocalPath(IRemoteFileMetadataPtr meta) const; - - BackgroundSchedulePool::TaskHolder recover_task_holder; - void recoverTask(); - void recoverCachedFilesMetadata( - const std::filesystem::path & current_path, - size_t current_depth, - size_t max_depth); -}; - -} From dc6f7858f8f199a73e235445f01025d3ec64281f Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 23 Dec 2021 03:55:40 +0000 Subject: [PATCH 0238/1260] refactor --- src/Core/DecimalComparison.h | 196 +++++++++++++++++++++------- src/Core/DecimalFloatComparison.h | 58 ++++++++ src/Functions/FunctionsComparison.h | 15 +-- src/Functions/IsOperation.h | 4 + 4 files changed, 209 insertions(+), 64 deletions(-) create mode 100644 src/Core/DecimalFloatComparison.h diff --git a/src/Core/DecimalComparison.h b/src/Core/DecimalComparison.h index 0398ec5cbac..62d7d72a924 100644 --- a/src/Core/DecimalComparison.h +++ b/src/Core/DecimalComparison.h @@ -1,17 +1,21 @@ #pragma once -#include -#include -#include -#include -#include -#include +#include #include #include -#include -#include /// TODO Core should not depend on Functions +#include +#include +#include +#include +#include +#include +#include /// TODO Core should not depend on Functions +#include +#include +#include + namespace DB { @@ -52,9 +56,14 @@ struct DecCompareInt using TypeB = Type; }; -/// -template typename Operation, bool _check_overflow = true, - bool _actual = is_decimal || is_decimal> +template < + typename A, + typename B, + template + typename Operation, + bool _check_overflow = true, + bool _actual = is_decimal || is_decimal, + bool _has_float = std::is_floating_point_v || std::is_floating_point_v> class DecimalComparison { public: @@ -221,48 +230,135 @@ private: template static NO_INLINE UInt8 apply(A a, B b, CompareInt scale [[maybe_unused]]) { - CompareInt x; - if constexpr (is_decimal) - x = a.value; - else - x = a; - - CompareInt y; - if constexpr (is_decimal) - y = b.value; - else - y = b; - - if constexpr (_check_overflow) + /// Decimal compares with Float + if constexpr (_has_float) { - bool overflow = false; - - if constexpr (sizeof(A) > sizeof(CompareInt)) - overflow |= (static_cast(x) != a); - if constexpr (sizeof(B) > sizeof(CompareInt)) - overflow |= (static_cast(y) != b); - if constexpr (is_unsigned_v) - overflow |= (x < 0); - if constexpr (is_unsigned_v) - overflow |= (y < 0); - - if constexpr (scale_left) - overflow |= common::mulOverflow(x, scale, x); - if constexpr (scale_right) - overflow |= common::mulOverflow(y, scale, y); - - if (overflow) - throw Exception("Can't compare decimal number due to overflow", ErrorCodes::DECIMAL_OVERFLOW); - } - else - { - if constexpr (scale_left) - x = common::mulIgnoreOverflow(x, scale); - if constexpr (scale_right) - y = common::mulIgnoreOverflow(y, scale); + if constexpr (IsOperation::equals) + { + if constexpr (std::is_floating_point_v && is_decimal) + { + CompareInt decimal_value = b.value; + return DecimalFloatComparison::equals(a, decimal_value, scale); + } + if constexpr (std::is_floating_point_v && is_decimal) + { + CompareInt decimal_value = a.value; + return DecimalFloatComparison::equals(b, decimal_value, scale); + } + } + if constexpr (IsOperation::not_equals) + { + if constexpr (std::is_floating_point_v && is_decimal) + { + CompareInt decimal_value = b.value; + return DecimalFloatComparison::notEquals(a, decimal_value, scale); + } + if constexpr (std::is_floating_point_v && is_decimal) + { + CompareInt decimal_value = a.value; + return DecimalFloatComparison::notEquals(b, decimal_value, scale); + } + } + if constexpr (IsOperation::less) + { + if constexpr (std::is_floating_point_v && is_decimal) + { + CompareInt decimal_value = b.value; + return DecimalFloatComparison::less(a, decimal_value, scale); + } + if constexpr (std::is_floating_point_v && is_decimal) + { + CompareInt decimal_value = a.value; + return DecimalFloatComparison::greater(b, decimal_value, scale); + } + } + if constexpr (IsOperation::less_or_equals) + { + if constexpr (std::is_floating_point_v && is_decimal) + { + CompareInt decimal_value = b.value; + return DecimalFloatComparison::lessOrEquals(a, decimal_value, scale); + } + if constexpr (std::is_floating_point_v && is_decimal) + { + CompareInt decimal_value = a.value; + return DecimalFloatComparison::greaterOrEquals(b, decimal_value, scale); + } + } + if constexpr (IsOperation::greater) + { + if constexpr (std::is_floating_point_v && is_decimal) + { + CompareInt decimal_value = b.value; + return DecimalFloatComparison::greater(a, decimal_value, scale); + } + if constexpr (std::is_floating_point_v && is_decimal) + { + CompareInt decimal_value = a.value; + return DecimalFloatComparison::less(b, decimal_value, scale); + } + } + if constexpr (IsOperation::greater_or_equals) + { + if constexpr (std::is_floating_point_v && is_decimal) + { + CompareInt decimal_value = b.value; + return DecimalFloatComparison::greaterOrEquals(a, decimal_value, scale); + } + if constexpr (std::is_floating_point_v && is_decimal) + { + CompareInt decimal_value = a.value; + return DecimalFloatComparison::lessOrEquals(b, decimal_value, scale); + } + } } - return Op::apply(x, y); + /// Decimal compares with Int + else + { + CompareInt x; + if constexpr (is_decimal) + x = a.value; + else + x = a; + + CompareInt y; + if constexpr (is_decimal) + y = b.value; + else + y = b; + + if constexpr (_check_overflow) + { + bool overflow = false; + + if constexpr (sizeof(A) > sizeof(CompareInt)) + overflow |= (static_cast(x) != a); + if constexpr (sizeof(B) > sizeof(CompareInt)) + overflow |= (static_cast(y) != b); + if constexpr (is_unsigned_v) + overflow |= (x < 0); + if constexpr (is_unsigned_v) + overflow |= (y < 0); + + if constexpr (scale_left) + overflow |= common::mulOverflow(x, scale, x); + if constexpr (scale_right) + overflow |= common::mulOverflow(y, scale, y); + + if (overflow) + throw Exception("Can't compare decimal number due to overflow", ErrorCodes::DECIMAL_OVERFLOW); + } + else + { + if constexpr (scale_left) + x = common::mulIgnoreOverflow(x, scale); + if constexpr (scale_right) + y = common::mulIgnoreOverflow(y, scale); + } + + return Op::apply(x, y); + } } template diff --git a/src/Core/DecimalFloatComparison.h b/src/Core/DecimalFloatComparison.h new file mode 100644 index 00000000000..8524478e312 --- /dev/null +++ b/src/Core/DecimalFloatComparison.h @@ -0,0 +1,58 @@ +#pragma once + +#include + +namespace DB +{ +struct DecimalFloatComparison +{ + template + static int compare(Float a, Int b, Int scale) + { + /// TODO need to implement comparison + if (a) + return -1; + if (b) + return 0; + if (scale) + return 1; + return 0; + } + + template + static bool equals(Float a, Int b, Int scale) + { + return compare(a, b, scale) == 0; + } + + template + static bool notEquals(Float a, Int b, Int scale) + { + return compare(a, b, scale) != 0; + } + + template + static bool less(Float a, Int b, Int scale) + { + return compare(a, b, scale) < 0; + } + + template + static bool greater(Float a, Int b, Int scale) + { + return compare(a, b, scale) > 0; + } + + template + static bool lessOrEquals(Float a, Int b, Int scale) + { + return compare(a, b, scale) <= 0; + } + + template + static bool greaterOrEquals(Float a, Int b, Int scale) + { + return compare(a, b, scale) >= 0; + } +}; +} diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index dcdbb3738f8..28ab01b77a4 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -687,7 +687,7 @@ private: return (res = DecimalComparison::apply(col_left, col_right)) != nullptr; }; - if (!callOnBasicTypes(left_number, right_number, call)) + if (!callOnBasicTypes(left_number, right_number, call)) throw Exception("Wrong call for " + getName() + " with " + col_left.type->getName() + " and " + col_right.type->getName(), ErrorCodes::LOGICAL_ERROR); @@ -1175,9 +1175,6 @@ public: const bool left_is_num = col_left_untyped->isNumeric(); const bool right_is_num = col_right_untyped->isNumeric(); - const bool left_is_float = which_left.isFloat(); - const bool right_is_float = which_right.isFloat(); - const bool left_is_string = which_left.isStringOrFixedString(); const bool right_is_string = which_right.isStringOrFixedString(); @@ -1240,16 +1237,6 @@ public: throw Exception( "No operation " + getName() + " between " + left_type->getName() + " and " + right_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (left_is_float) - { - ColumnPtr left_converted = castColumn(col_with_type_and_name_left, right_type); - return executeDecimal({left_converted, right_type, "left"}, col_with_type_and_name_right); - } - if (right_is_float) - { - ColumnPtr right_converted = castColumn(col_with_type_and_name_right, left_type); - return executeDecimal(col_with_type_and_name_left, {right_converted, left_type, "right"}); - } return executeDecimal(col_with_type_and_name_left, col_with_type_and_name_right); } diff --git a/src/Functions/IsOperation.h b/src/Functions/IsOperation.h index 369978fe271..0e4feeaf314 100644 --- a/src/Functions/IsOperation.h +++ b/src/Functions/IsOperation.h @@ -17,7 +17,9 @@ template struct GreatestBaseImpl; template struct ModuloImpl; template struct EqualsOp; template struct NotEqualsOp; +template struct LessOp; template struct LessOrEqualsOp; +template struct GreaterOp; template struct GreaterOrEqualsOp; template @@ -42,7 +44,9 @@ struct IsOperation { static constexpr bool equals = IsSameOperation::value; static constexpr bool not_equals = IsSameOperation::value; + static constexpr bool less = IsSameOperation::value; static constexpr bool less_or_equals = IsSameOperation::value; + static constexpr bool greater = IsSameOperation::value; static constexpr bool greater_or_equals = IsSameOperation::value; static constexpr bool plus = IsSameOperation::value; From c3736859acc44962d94e88e41f74d662bdff771a Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 23 Dec 2021 11:56:54 +0800 Subject: [PATCH 0239/1260] update some comments --- src/Common/LRUCache.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 22ed498bd60..345036775be 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -25,13 +25,13 @@ struct TrivialWeightFunction template struct TrivialLRUCacheEvitPolicy { - // To note that the arg maybe is null + // To note that the arg could be null inline bool canRelease(std::shared_ptr) const { return true; } - // To note that the arg is null + // To note that the arg could be null inline void release(std::shared_ptr) { } From e19fd5bc767b4bab704271ce88d188299bf55f21 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 23 Dec 2021 12:01:44 +0800 Subject: [PATCH 0240/1260] update some comments --- src/Storages/Cache/ExternalDataSourceCache.h | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Storages/Cache/ExternalDataSourceCache.h b/src/Storages/Cache/ExternalDataSourceCache.h index 468cf3c300d..740bd820d40 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.h +++ b/src/Storages/Cache/ExternalDataSourceCache.h @@ -29,10 +29,6 @@ namespace DB { -/* - * FIXME:RemoteReadBuffer derive from SeekableReadBufferWithSize may cause some risks, since it's not seekable in some cases - * But SeekableReadBuffer is not a interface which make it hard to fixup. - */ class RemoteReadBuffer : public BufferWithOwnMemory { public: From 2c77b7cf5f77d3acaa5b3c9f5ff12fa42fc31a71 Mon Sep 17 00:00:00 2001 From: lehasm Date: Thu, 23 Dec 2021 08:18:52 +0300 Subject: [PATCH 0241/1260] Update docs/ru/operations/server-configuration-parameters/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 19716dc8947..dc4e7664aa3 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -450,7 +450,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part - `user` — имя пользователя. - `password` — пароль. -- `allow_empty` — Если `true`, то другие реплики могут подключаться без аутентификации, даже если учетные данные заданы. Если `false`, то подключение без аутентификации не допускается. Значение по умолчанию: `false`. +- `allow_empty` — если `true`, то другие реплики могут подключаться без аутентификации, даже если учетные данные заданы. Если `false`, то подключение без аутентификации не допускается. Значение по умолчанию: `false`. - `old` — секция содержит старые значения `user` и `password`, которые используются в процессе изменения учетных данных. Можно указывать несколько секций `old`. **Изменение учетных данных** From 4151207a9876608b14b7725d1499105af20b0e86 Mon Sep 17 00:00:00 2001 From: lehasm Date: Thu, 23 Dec 2021 08:19:11 +0300 Subject: [PATCH 0242/1260] Update docs/ru/operations/server-configuration-parameters/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index dc4e7664aa3..41e31b46e4b 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -455,7 +455,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part **Изменение учетных данных** -ClickHouse поддерживает динамическое изменение учетных данных. При этом не требуется одновременно останавливать все реплики, чтобы обновить конфигурацию. Изменение учетные данные выполняется за несколько шагов. +ClickHouse поддерживает динамическое изменение учетных данных. При этом не требуется одновременно останавливать все реплики, чтобы обновить конфигурацию. Изменение учетных данных выполняется за несколько шагов. Чтобы включить аутентификацию, установите `interserver_http_credentials.allow_empty` в `true` и задайте учетные данные. С такой конфигурацией разрешены подключения как с аутентификацией, так и без нее. From a89eba6caa6cad9239a04fc73f43a9df95e720fa Mon Sep 17 00:00:00 2001 From: lehasm Date: Thu, 23 Dec 2021 08:19:51 +0300 Subject: [PATCH 0243/1260] Update docs/ru/operations/server-configuration-parameters/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 41e31b46e4b..07be839e742 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -457,7 +457,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ClickHouse поддерживает динамическое изменение учетных данных. При этом не требуется одновременно останавливать все реплики, чтобы обновить конфигурацию. Изменение учетных данных выполняется за несколько шагов. -Чтобы включить аутентификацию, установите `interserver_http_credentials.allow_empty` в `true` и задайте учетные данные. С такой конфигурацией разрешены подключения как с аутентификацией, так и без нее. +Чтобы включить аутентификацию, установите `interserver_http_credentials.allow_empty` в значение `true` и задайте учетные данные. С такой конфигурацией разрешены подключения как с аутентификацией, так и без нее. ``` xml From 2d8af8e13df80f14bbf72158a0440bfee9d2af8f Mon Sep 17 00:00:00 2001 From: lehasm Date: Thu, 23 Dec 2021 08:20:32 +0300 Subject: [PATCH 0244/1260] Update docs/ru/operations/server-configuration-parameters/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 07be839e742..8aeb131546c 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -469,7 +469,7 @@ ClickHouse поддерживает динамическое изменение После конфигурации всех реплик установите `allow_empty` в значение `false` или удалите эту настройку. Это сделает аутентификацию с новыми учетными данными обязательной. -Чтобы изменить учетные данные, перенесите имя пользователя и пароль в секцию `interserver_http_credentials.old` и укажите в `user` и `password` новые значения. Сервер будет использовать новые учетные данные при подключении к другим репликам и при этом разрешать подключения как с новыми, так и со старыми учетными данными. +Чтобы изменить учетные данные, перенесите имя пользователя и пароль в секцию `interserver_http_credentials.old` и укажите новые значения для `user` и `password`. Сервер будет использовать новые учетные данные при подключении к другим репликам и при этом будет разрешать подключения как с новыми, так и со старыми учетными данными. ``` xml From 6b979b1fcba0b3f420752e70d61a78156d8685d7 Mon Sep 17 00:00:00 2001 From: lehasm Date: Thu, 23 Dec 2021 08:21:07 +0300 Subject: [PATCH 0245/1260] Update docs/ru/operations/server-configuration-parameters/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 8aeb131546c..c250f70cac9 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -486,7 +486,7 @@ ClickHouse поддерживает динамическое изменение ``` -Когда новые учетные данные заданы на всех репликах, старые учетные данные можно удалить из конфигурации. +Когда новые учетные данные обновятся на всех репликах, старые учетные данные можно удалить из конфигурации. ## keep_alive_timeout {#keep-alive-timeout} From f0b22d2882cf9aed73a1fc2ce8ea9ca69ed3e6a2 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 23 Dec 2021 15:15:39 +0800 Subject: [PATCH 0246/1260] fixed code-style --- src/Common/LRUCache.h | 7 +++---- src/Storages/Cache/ExternalDataSourceCache.cpp | 3 --- src/Storages/Cache/RemoteCacheController.cpp | 4 ++-- src/Storages/IRemoteFileMetadata.h | 6 +++--- 4 files changed, 8 insertions(+), 12 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 345036775be..b9003d92470 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -160,7 +160,7 @@ public: if (!token->cleaned_up) token_holder.cleanup(token_lock, cache_lock); - + return {token->value, is_value_loaded, is_value_updated}; } @@ -175,7 +175,7 @@ public: if (!evict_policy.canRelease(cell.value)) return false; evict_policy.release(cell.value); - + current_size -= cell.size; cells.erase(it); queue.erase(cell.queue_iterator); @@ -386,8 +386,7 @@ private: size_t current_weight_lost = 0; size_t queue_size = cells.size(); auto key_it = queue.begin(); - auto is_overflow = [&] { return (current_size + required_size_to_remove > max_size || (max_elements_size != 0 && queue_size > max_elements_size)); - }; + auto is_overflow = [&] { return (current_size + required_size_to_remove > max_size || (max_elements_size != 0 && queue_size > max_elements_size)); }; while (is_overflow() && (queue_size > 1) && (key_it != queue.end())) { const Key & key = *key_it; diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp index 3a8dfd9bcaa..57815226a68 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.cpp +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -25,12 +25,9 @@ namespace fs = std::filesystem; namespace ErrorCodes { extern const int OK; - extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int NOT_INIT; extern const int DISK_OVERFLOW; - extern const int FILE_BROKEN; - extern const int END_OF_FILE; } diff --git a/src/Storages/Cache/RemoteCacheController.cpp b/src/Storages/Cache/RemoteCacheController.cpp index b2d36ee38bc..082e74e1f4f 100644 --- a/src/Storages/Cache/RemoteCacheController.cpp +++ b/src/Storages/Cache/RemoteCacheController.cpp @@ -52,9 +52,9 @@ std::shared_ptr RemoteCacheController::recover(const std: { cache_controller->file_metadata_ptr = RemoteFileMetadataFactory::instance().get(cache_controller->metadata_class); } - catch(...) + catch(const Exception & e) { - LOG_ERROR(log, "Get metadata class failed for {}", cache_controller->metadata_class); + LOG_ERROR(log, "Get metadata class failed for {}. {}", cache_controller->metadata_class, e.message()); cache_controller->file_metadata_ptr = nullptr; } if (!cache_controller->file_metadata_ptr) diff --git a/src/Storages/IRemoteFileMetadata.h b/src/Storages/IRemoteFileMetadata.h index 6634fc2a020..0b07103b786 100644 --- a/src/Storages/IRemoteFileMetadata.h +++ b/src/Storages/IRemoteFileMetadata.h @@ -1,8 +1,8 @@ #pragma once -#include -#include #include +#include #include +#include #include namespace DB @@ -20,7 +20,7 @@ public: // used for comparing two file metadatas are the same or not. virtual String getVersion() const = 0; - + String remote_path; size_t file_size = 0; UInt64 last_modification_timestamp = 0; From a00ab6551b772939e81dde23162643b6bd059474 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 23 Dec 2021 15:43:42 +0800 Subject: [PATCH 0247/1260] compile error --- src/Storages/RemoteFileMetadataFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/RemoteFileMetadataFactory.cpp b/src/Storages/RemoteFileMetadataFactory.cpp index fba93bb9dc2..461fc9b1c9e 100644 --- a/src/Storages/RemoteFileMetadataFactory.cpp +++ b/src/Storages/RemoteFileMetadataFactory.cpp @@ -34,7 +34,7 @@ void RemoteFileMetadataFactory::registerRemoteFileMatadataCreator(const String & class_creators[name] = creator; } -void registerStorageHiveMetadataCreator(); +extern void registerStorageHiveMetadataCreator(); void registerRemoteFileMatadataCreators() { From ccc39bad80313ec982b8f7da106ded0fa5da1575 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 23 Dec 2021 15:56:33 +0800 Subject: [PATCH 0248/1260] move IRemoteFileMetadata info Storages/Cache --- programs/server/Server.cpp | 2 +- src/Storages/Cache/ExternalDataSourceCache.cpp | 2 +- src/Storages/Cache/ExternalDataSourceCache.h | 2 +- src/Storages/{ => Cache}/IRemoteFileMetadata.h | 0 src/Storages/Cache/RemoteCacheController.cpp | 2 +- src/Storages/Cache/RemoteCacheController.h | 2 +- src/Storages/{ => Cache}/RemoteFileMetadataFactory.cpp | 2 +- src/Storages/{ => Cache}/RemoteFileMetadataFactory.h | 2 +- src/Storages/Hive/StorageHiveMetadata.cpp | 2 +- src/Storages/Hive/StorageHiveMetadata.h | 2 +- 10 files changed, 9 insertions(+), 9 deletions(-) rename src/Storages/{ => Cache}/IRemoteFileMetadata.h (100%) rename src/Storages/{ => Cache}/RemoteFileMetadataFactory.cpp (95%) rename src/Storages/{ => Cache}/RemoteFileMetadataFactory.h (93%) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 974d7b1adb1..ffe56aa04d4 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -58,7 +58,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp index 57815226a68..dc966691f58 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.cpp +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include namespace fs = std::filesystem; diff --git a/src/Storages/Cache/ExternalDataSourceCache.h b/src/Storages/Cache/ExternalDataSourceCache.h index 740bd820d40..9f350da65ca 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.h +++ b/src/Storages/Cache/ExternalDataSourceCache.h @@ -18,7 +18,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/IRemoteFileMetadata.h b/src/Storages/Cache/IRemoteFileMetadata.h similarity index 100% rename from src/Storages/IRemoteFileMetadata.h rename to src/Storages/Cache/IRemoteFileMetadata.h diff --git a/src/Storages/Cache/RemoteCacheController.cpp b/src/Storages/Cache/RemoteCacheController.cpp index 082e74e1f4f..3f866415a0b 100644 --- a/src/Storages/Cache/RemoteCacheController.cpp +++ b/src/Storages/Cache/RemoteCacheController.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Storages/Cache/RemoteCacheController.h b/src/Storages/Cache/RemoteCacheController.h index a07ab4a92c9..19dd49d5182 100644 --- a/src/Storages/Cache/RemoteCacheController.h +++ b/src/Storages/Cache/RemoteCacheController.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/RemoteFileMetadataFactory.cpp b/src/Storages/Cache/RemoteFileMetadataFactory.cpp similarity index 95% rename from src/Storages/RemoteFileMetadataFactory.cpp rename to src/Storages/Cache/RemoteFileMetadataFactory.cpp index 461fc9b1c9e..672e48627d4 100644 --- a/src/Storages/RemoteFileMetadataFactory.cpp +++ b/src/Storages/Cache/RemoteFileMetadataFactory.cpp @@ -1,4 +1,4 @@ -#include +#include #include namespace DB diff --git a/src/Storages/RemoteFileMetadataFactory.h b/src/Storages/Cache/RemoteFileMetadataFactory.h similarity index 93% rename from src/Storages/RemoteFileMetadataFactory.h rename to src/Storages/Cache/RemoteFileMetadataFactory.h index 507da9ea08b..5a08e469a6b 100644 --- a/src/Storages/RemoteFileMetadataFactory.h +++ b/src/Storages/Cache/RemoteFileMetadataFactory.h @@ -1,5 +1,5 @@ #pragma once -#include +#include #include #include #include diff --git a/src/Storages/Hive/StorageHiveMetadata.cpp b/src/Storages/Hive/StorageHiveMetadata.cpp index 69fb444c19c..666caf64d34 100644 --- a/src/Storages/Hive/StorageHiveMetadata.cpp +++ b/src/Storages/Hive/StorageHiveMetadata.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Storages/Hive/StorageHiveMetadata.h b/src/Storages/Hive/StorageHiveMetadata.h index 60bbe2fe0aa..7cdbac9a9c8 100644 --- a/src/Storages/Hive/StorageHiveMetadata.h +++ b/src/Storages/Hive/StorageHiveMetadata.h @@ -1,5 +1,5 @@ #pragma once -#include +#include namespace DB { class StorageHiveMetadata : public IRemoteFileMetadata From d29a4d05271012a5870447c179fdaac582e64ba0 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 23 Dec 2021 16:16:33 +0800 Subject: [PATCH 0249/1260] add trySet() in LRUCache --- src/Common/ErrorCodes.cpp | 1 + src/Common/LRUCache.h | 16 +++++++++++++--- src/Storages/Cache/ExternalDataSourceCache.cpp | 4 ++-- 3 files changed, 16 insertions(+), 5 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index bac5ab9a46f..f861d408a16 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -610,6 +610,7 @@ M(654, DISK_OVERFLOW)\ M(655, FILE_BROKEN)\ M(656, END_OF_FILE)\ + M(657, CANNOT_RELEASE)\ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index b9003d92470..2920c785dc8 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -6,12 +6,16 @@ #include #include #include - #include +#include namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_RELEASE; +} template struct TrivialWeightFunction @@ -76,11 +80,17 @@ public: return res; } + void set(const Key & key, const MappedPtr & mapped) + { + std::lock_guard lock(mutex); + if (!setImpl(key, mapped, lock)) + throw Exception(ErrorCodes::CANNOT_RELEASE, "Cannot release the key in LRUCache"); + } /** - * set() will fail (return false) if there is no space left and no keys could be evicted. + * trySet() will fail (return false) if there is no space left and no keys could be evicted. * Eviction permission of each key is defined by EvictPolicy. In default policy there is no restriction. */ - bool set(const Key & key, const MappedPtr & mapped) + bool trySet(const Key & key, const MappedPtr & mapped) { std::lock_guard lock(mutex); diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp index dc966691f58..872ce815e58 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.cpp +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -145,7 +145,7 @@ void ExternalDataSourceCache::recoverCachedFilesMetadata( invalid_paths.emplace_back(path); continue; } - if (!lru_caches->set(path, cache_controller)) + if (!lru_caches->trySet(path, cache_controller)) { invalid_paths.emplace_back(path); } @@ -244,7 +244,7 @@ ExternalDataSourceCache::createReader(ContextPtr context, IRemoteFileMetadataPtr // cache is not found or is invalid auto new_cache = std::make_shared(remote_file_metadata, local_path, local_cache_bytes_read_before_flush); - if (!lru_caches->set(local_path, new_cache)) + if (!lru_caches->trySet(local_path, new_cache)) { LOG_ERROR(log, "Insert the new cache failed. new file size:{}, current total size:{}", remote_file_metadata->file_size, From e5a0fba6387b399eeae0c666cd46287e651bd162 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 23 Dec 2021 09:19:10 +0100 Subject: [PATCH 0250/1260] Use ClickHouse-Extras for the submodules --- .gitmodules | 4 ++-- contrib/NuRaft | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.gitmodules b/.gitmodules index 676e8d3d9aa..5321712f1f1 100644 --- a/.gitmodules +++ b/.gitmodules @@ -54,7 +54,7 @@ url = https://github.com/ClickHouse-Extras/Turbo-Base64.git [submodule "contrib/arrow"] path = contrib/arrow - url = https://github.com/apache/arrow + url = https://github.com/ClickHouse-Extras/arrow.git branch = blessed/release-6.0.1 [submodule "contrib/thrift"] path = contrib/thrift @@ -207,7 +207,7 @@ branch = MergeWithUpstream [submodule "contrib/NuRaft"] path = contrib/NuRaft - url = https://github.com/Algunenano/NuRaft.git + url = https://github.com/ClickHouse-Extras/NuRaft.git [submodule "contrib/nanodbc"] path = contrib/nanodbc url = https://github.com/ClickHouse-Extras/nanodbc.git diff --git a/contrib/NuRaft b/contrib/NuRaft index 8572660b5d5..ff100a87131 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 8572660b5d581ba365c679d66e57d221fa0cd1fb +Subproject commit ff100a8713146e1ca4b4158dd6cc4eef9af47fc3 From abe79fb1c26507054b5a608822558ec746331b7f Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 23 Dec 2021 16:33:43 +0800 Subject: [PATCH 0251/1260] move Exception in LRUCache::set() --- src/Common/LRUCache.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 2920c785dc8..c503619a7d0 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -83,9 +83,9 @@ public: void set(const Key & key, const MappedPtr & mapped) { std::lock_guard lock(mutex); - if (!setImpl(key, mapped, lock)) - throw Exception(ErrorCodes::CANNOT_RELEASE, "Cannot release the key in LRUCache"); + setImpl(key, mapped, lock); } + /** * trySet() will fail (return false) if there is no space left and no keys could be evicted. * Eviction permission of each key is defined by EvictPolicy. In default policy there is no restriction. From 6bab5c86516ce03194daa0ea3fbaac1206fa6745 Mon Sep 17 00:00:00 2001 From: gulige Date: Thu, 23 Dec 2021 17:26:47 +0800 Subject: [PATCH 0252/1260] code refactoring as @alexey-milovidov suggested --- src/Core/Block.cpp | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 473be282410..b6f62968db6 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -145,19 +145,17 @@ void Block::insert(size_t position, ColumnWithTypeAndName elem) if (elem.name.empty()) throw Exception("Column name in Block cannot be empty", ErrorCodes::AMBIGUOUS_COLUMN_NAME); - auto it = index_by_name.find(elem.name); - if (index_by_name.end() != it) - { + auto [it, inserted] = index_by_name.emplace(elem.name, position); + if (!inserted) checkColumnStructure(data[it->second], elem, "(columns with identical name must have identical structure)", true, ErrorCodes::AMBIGUOUS_COLUMN_NAME); + + for (auto itr = index_by_name.begin(); itr != index_by_name.end(); ++itr) + { + if (itr->second >= position && (!inserted || itr != it)) + ++itr->second; } - for (auto & name_pos : index_by_name) - if (name_pos.second >= position) - ++name_pos.second; - - index_by_name.emplace(elem.name, position); - data.emplace(data.begin() + position, std::move(elem)); } From 9fdb061b7e3f4d12518919d0c508b002c37e4db5 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 23 Dec 2021 17:36:41 +0800 Subject: [PATCH 0253/1260] Add evict policy in LRUCache --- src/Common/ErrorCodes.cpp | 2 + src/Common/LRUCache.h | 140 +++++++++++++++++++++++++++++++------- 2 files changed, 119 insertions(+), 23 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 70d85433513..07e05683340 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -604,6 +604,8 @@ M(633, QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW) \ M(634, MONGODB_ERROR) \ \ + M(656, CANNOT_RELEASE) \ + \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index bbc09fd3aff..c503619a7d0 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -6,12 +6,16 @@ #include #include #include - #include +#include namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_RELEASE; +} template struct TrivialWeightFunction @@ -22,13 +26,32 @@ struct TrivialWeightFunction } }; +template +struct TrivialLRUCacheEvitPolicy +{ + // To note that the arg could be null + inline bool canRelease(std::shared_ptr) const + { + return true; + } + + // To note that the arg could be null + inline void release(std::shared_ptr) + { + } +}; + /// Thread-safe cache that evicts entries which are not used for a long time. /// WeightFunction is a functor that takes Mapped as a parameter and returns "weight" (approximate size) /// of that value. /// Cache starts to evict entries when their total weight exceeds max_size. /// Value weight should not change after insertion. -template , typename WeightFunction = TrivialWeightFunction> +template , + typename WeightFunction = TrivialWeightFunction, + typename EvictPolicy = TrivialLRUCacheEvitPolicy> class LRUCache { public: @@ -60,20 +83,41 @@ public: void set(const Key & key, const MappedPtr & mapped) { std::lock_guard lock(mutex); - setImpl(key, mapped, lock); } + /** + * trySet() will fail (return false) if there is no space left and no keys could be evicted. + * Eviction permission of each key is defined by EvictPolicy. In default policy there is no restriction. + */ + bool trySet(const Key & key, const MappedPtr & mapped) + { + std::lock_guard lock(mutex); + + return setImpl(key, mapped, lock); + } + + /// Returns std::pair of the cached value and a bool indicating whether the value was produced during this call. + template + std::pair getOrSet(const Key & key, LoadFunc && load_func) + { + auto [value, is_loaded, _] = getOrTrySet(key, std::move(load_func)); + return std::make_pair(value, is_loaded); + } + /// If the value for the key is in the cache, returns it. If it is not, calls load_func() to /// produce it, saves the result in the cache and returns it. - /// Only one of several concurrent threads calling getOrSet() will call load_func(), + /// Only one of several concurrent threads calling getOrTrySet() will call load_func(), /// others will wait for that call to complete and will use its result (this helps prevent cache stampede). /// Exceptions occurring in load_func will be propagated to the caller. Another thread from the /// set of concurrent threads will then try to call its load_func etc. /// - /// Returns std::pair of the cached value and a bool indicating whether the value was produced during this call. + /// return std::tuple is , where + /// - is_value_loaded indicates whether the value was produce during this call + /// - is_value_updated indicates whether the value is updated in the cache when is_value_loaded = true. + /// if is_value_loaded = false, is_value_updated = false template - std::pair getOrSet(const Key & key, LoadFunc && load_func) + std::tuple getOrTrySet(const Key &key, LoadFunc && load_func) { InsertTokenHolder token_holder; { @@ -83,7 +127,7 @@ public: if (val) { ++hits; - return std::make_pair(val, false); + return {val, false, false}; } auto & token = insert_tokens[key]; @@ -103,7 +147,7 @@ public: { /// Another thread already produced the value while we waited for token->mutex. ++hits; - return std::make_pair(token->value, false); + return {token->value, false, false}; } ++misses; @@ -113,18 +157,39 @@ public: /// Insert the new value only if the token is still in present in insert_tokens. /// (The token may be absent because of a concurrent reset() call). - bool result = false; + bool is_value_loaded = false; + bool is_value_updated = false; auto token_it = insert_tokens.find(key); if (token_it != insert_tokens.end() && token_it->second.get() == token) { - setImpl(key, token->value, cache_lock); - result = true; + // setImpl() may fail, but the final behavior seems not be affected + // next call of getOrTrySet() will still call load_func() + is_value_updated = setImpl(key, token->value, cache_lock); + is_value_loaded = true; } if (!token->cleaned_up) token_holder.cleanup(token_lock, cache_lock); - return std::make_pair(token->value, result); + return {token->value, is_value_loaded, is_value_updated}; + } + + /// If key is not in cache or the element can be released, return is true. otherwise, return is false + bool tryRemove(const Key & key) + { + std::lock_guard loc(mutex); + auto it = cells.find(key); + if (it == cells.end()) + return true; + auto & cell = it->second; + if (!evict_policy.canRelease(cell.value)) + return false; + evict_policy.release(cell.value); + + current_size -= cell.size; + cells.erase(it); + queue.erase(cell.queue_iterator); + return true; } void getStats(size_t & out_hits, size_t & out_misses) const @@ -259,6 +324,7 @@ private: std::atomic misses {0}; WeightFunction weight_function; + EvictPolicy evict_policy; MappedPtr getImpl(const Key & key, [[maybe_unused]] std::lock_guard & cache_lock) { @@ -276,7 +342,7 @@ private: return cell.value; } - void setImpl(const Key & key, const MappedPtr & mapped, [[maybe_unused]] std::lock_guard & cache_lock) + bool setImpl(const Key & key, const MappedPtr & mapped, [[maybe_unused]] std::lock_guard & cache_lock) { auto [it, inserted] = cells.emplace(std::piecewise_construct, std::forward_as_tuple(key), @@ -286,6 +352,15 @@ private: if (inserted) { + auto value_weight = mapped ? weight_function(*mapped) : 0; + // move removeOverflow() ahead here. In default, the final result is the same as the old implementation + if (!removeOverflow(value_weight)) + { + // cannot find enough space to put in the new value + cells.erase(it); + return false; + } + try { cell.queue_iterator = queue.insert(queue.end(), key); @@ -298,6 +373,13 @@ private: } else { + if (!evict_policy.canRelease(cell.value)) + { + // the old value is refered by someone, cannot release now + // in default policy, it is always true. + return false; + } + evict_policy.release(cell.value); // release the old value. this action is empty in default policy. current_size -= cell.size; queue.splice(queue.end(), queue, cell.queue_iterator); } @@ -306,17 +388,18 @@ private: cell.size = cell.value ? weight_function(*cell.value) : 0; current_size += cell.size; - removeOverflow(); + return true; } - void removeOverflow() + bool removeOverflow(size_t required_size_to_remove = 0) { size_t current_weight_lost = 0; size_t queue_size = cells.size(); - - while ((current_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) && (queue_size > 1)) + auto key_it = queue.begin(); + auto is_overflow = [&] { return (current_size + required_size_to_remove > max_size || (max_elements_size != 0 && queue_size > max_elements_size)); }; + while (is_overflow() && (queue_size > 1) && (key_it != queue.end())) { - const Key & key = queue.front(); + const Key & key = *key_it; auto it = cells.find(key); if (it == cells.end()) @@ -326,13 +409,23 @@ private: } const auto & cell = it->second; + if (evict_policy.canRelease(cell.value))// in default, it is true + { + // always call release() before erasing an element + // in default, it's an empty action + evict_policy.release(cell.value); - current_size -= cell.size; - current_weight_lost += cell.size; + current_size -= cell.size; + current_weight_lost += cell.size; - cells.erase(it); - queue.pop_front(); - --queue_size; + cells.erase(it); + key_it = queue.erase(key_it); + --queue_size; + } + else + { + key_it++; + } } onRemoveOverflowWeightLoss(current_weight_lost); @@ -342,6 +435,7 @@ private: LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); abort(); } + return !is_overflow(); } /// Override this method if you want to track how much weight was lost in removeOverflow method. From 8cf52918b922dbd37c6a8f6a8f470b4d89279726 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 Dec 2021 12:24:44 +0300 Subject: [PATCH 0254/1260] clickhouse local send profile info --- src/Client/LocalConnection.cpp | 36 +++++++++++++++---- src/Client/LocalConnection.h | 2 ++ ...53_clickhouse_local_profile_info.reference | 32 +++++++++++++++++ .../02153_clickhouse_local_profile_info.sh | 8 +++++ 4 files changed, 71 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02153_clickhouse_local_profile_info.reference create mode 100755 tests/queries/0_stateless/02153_clickhouse_local_profile_info.sh diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 9eaa9ce883a..8ad853950b2 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -214,15 +214,15 @@ bool LocalConnection::poll(size_t) if (next_packet_type) return true; - if (send_progress && (state->after_send_progress.elapsedMicroseconds() >= query_context->getSettingsRef().interactive_delay)) - { - state->after_send_progress.restart(); - next_packet_type = Protocol::Server::Progress; - return true; - } - if (!state->is_finished) { + if (send_progress && (state->after_send_progress.elapsedMicroseconds() >= query_context->getSettingsRef().interactive_delay)) + { + state->after_send_progress.restart(); + next_packet_type = Protocol::Server::Progress; + return true; + } + try { pollImpl(); @@ -282,6 +282,18 @@ bool LocalConnection::poll(size_t) } } + if (state->is_finished && !state->sent_profile_info) + { + state->sent_profile_info = true; + + if (state->executor) + { + next_packet_type = Protocol::Server::ProfileInfo; + state->profile_info = state->executor->getProfileInfo(); + return true; + } + } + if (state->is_finished) { finishQuery(); @@ -349,6 +361,16 @@ Packet LocalConnection::receivePacket() next_packet_type.reset(); break; } + case Protocol::Server::ProfileInfo: + { + if (state->profile_info) + { + packet.profile_info = std::move(*state->profile_info); + state->profile_info.reset(); + } + next_packet_type.reset(); + break; + } case Protocol::Server::TableColumns: { if (state->columns_description) diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index fbd054506e7..92c2af30c80 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -35,6 +35,7 @@ struct LocalQueryState /// Current block to be sent next. std::optional block; std::optional columns_description; + std::optional profile_info; /// Is request cancelled bool is_cancelled = false; @@ -43,6 +44,7 @@ struct LocalQueryState bool sent_totals = false; bool sent_extremes = false; bool sent_progress = false; + bool sent_profile_info = false; /// To output progress, the difference after the previous sending of progress. Progress progress; diff --git a/tests/queries/0_stateless/02153_clickhouse_local_profile_info.reference b/tests/queries/0_stateless/02153_clickhouse_local_profile_info.reference new file mode 100644 index 00000000000..2e1b607ac04 --- /dev/null +++ b/tests/queries/0_stateless/02153_clickhouse_local_profile_info.reference @@ -0,0 +1,32 @@ +{ + "meta": + [ + { + "name": "count()", + "type": "UInt64" + }, + { + "name": "n", + "type": "UInt8" + } + ], + + "data": + [ + { + "count()": "1", + "n": 1 + } + ], + + "totals": + { + "count()": "3", + "n": 0 + }, + + "rows": 1, + + "rows_before_limit_at_least": 3, + + "statistics": diff --git a/tests/queries/0_stateless/02153_clickhouse_local_profile_info.sh b/tests/queries/0_stateless/02153_clickhouse_local_profile_info.sh new file mode 100755 index 00000000000..65754d390fa --- /dev/null +++ b/tests/queries/0_stateless/02153_clickhouse_local_profile_info.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_LOCAL} --query "SELECT count(), arrayJoin([1, 2, 3]) AS n GROUP BY n WITH TOTALS ORDER BY n LIMIT 1 FORMAT JSON;" 2>&1 | head -32 + From 59bc87e4098dd020b9a7bada7917e28bbde73ee4 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 23 Dec 2021 17:53:19 +0800 Subject: [PATCH 0255/1260] remove unused errorcode --- src/Common/ErrorCodes.cpp | 2 -- src/Common/LRUCache.h | 5 ----- 2 files changed, 7 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 07e05683340..70d85433513 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -604,8 +604,6 @@ M(633, QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW) \ M(634, MONGODB_ERROR) \ \ - M(656, CANNOT_RELEASE) \ - \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index c503619a7d0..4919bbffc32 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -12,11 +12,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_RELEASE; -} - template struct TrivialWeightFunction { From f005d6d316c0de5b3af2db9cbe9f0b2b429fac02 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 23 Dec 2021 13:00:38 +0300 Subject: [PATCH 0256/1260] Update Block.cpp --- src/Core/Block.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index b6f62968db6..85eb6264220 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -145,15 +145,15 @@ void Block::insert(size_t position, ColumnWithTypeAndName elem) if (elem.name.empty()) throw Exception("Column name in Block cannot be empty", ErrorCodes::AMBIGUOUS_COLUMN_NAME); - auto [it, inserted] = index_by_name.emplace(elem.name, position); + auto [new_it, inserted] = index_by_name.emplace(elem.name, position); if (!inserted) - checkColumnStructure(data[it->second], elem, + checkColumnStructure(data[new_it->second], elem, "(columns with identical name must have identical structure)", true, ErrorCodes::AMBIGUOUS_COLUMN_NAME); - for (auto itr = index_by_name.begin(); itr != index_by_name.end(); ++itr) + for (auto it = index_by_name.begin(); it != index_by_name.end(); ++it) { - if (itr->second >= position && (!inserted || itr != it)) - ++itr->second; + if (it->second >= position && (!inserted || it != new_it)) + ++it->second; } data.emplace(data.begin() + position, std::move(elem)); From fe4ef69fc7108bd63fa23ae76f75e23a5f50630a Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 23 Dec 2021 18:01:58 +0800 Subject: [PATCH 0257/1260] remove unused header --- src/Common/LRUCache.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 4919bbffc32..c8a5dad5731 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -7,7 +7,6 @@ #include #include #include -#include namespace DB From e25b1f7361a4c67034fb63c224eda8795f3f4e21 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 23 Dec 2021 18:05:52 +0800 Subject: [PATCH 0258/1260] update comments --- src/Common/LRUCache.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index c8a5dad5731..b6cacc8f7ee 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -23,13 +23,11 @@ struct TrivialWeightFunction template struct TrivialLRUCacheEvitPolicy { - // To note that the arg could be null inline bool canRelease(std::shared_ptr) const { return true; } - // To note that the arg could be null inline void release(std::shared_ptr) { } From 5a9bd7fd41d599e1d04adb9760bd6fd38a8f211b Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 23 Dec 2021 18:13:56 +0800 Subject: [PATCH 0259/1260] fixe typos --- src/Common/LRUCache.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index b6cacc8f7ee..856a230674b 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -367,7 +367,7 @@ private: { if (!evict_policy.canRelease(cell.value)) { - // the old value is refered by someone, cannot release now + // the old value is referred by someone, cannot release now // in default policy, it is always true. return false; } From b94dbb64d65f2b452e5de9393940ffcd12adaba4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 23 Dec 2021 11:56:07 +0100 Subject: [PATCH 0260/1260] Improvements based on PR review --- src/Core/Field.h | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Core/Field.h b/src/Core/Field.h index 0cafdbe5361..7c09ec71749 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -819,16 +819,14 @@ T & Field::reinterpret() template <> inline char & Field::reinterpret() { - using ValueType = std::decay_t; if (which == Types::String) { // For String we want to return a pointer to the data, not the start of the class // as the layout of std::string depends on the STD version and options - ValueType * MAY_ALIAS ptr = reinterpret_cast(reinterpret_cast(&storage)->data()); + char * MAY_ALIAS ptr = reinterpret_cast(&storage)->data(); return *ptr; } - ValueType * MAY_ALIAS ptr = reinterpret_cast(&storage); - return *ptr; + return *reinterpret_cast(&storage); } template From 80e109a604650acde168a81e052781c9c0984b2b Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Thu, 23 Dec 2021 20:34:52 +0800 Subject: [PATCH 0261/1260] Update disks.md --- docs/zh/operations/system-tables/disks.md | 40 ++++++++++------------- 1 file changed, 18 insertions(+), 22 deletions(-) diff --git a/docs/zh/operations/system-tables/disks.md b/docs/zh/operations/system-tables/disks.md index 8cd24d24550..7100bac8dd6 100644 --- a/docs/zh/operations/system-tables/disks.md +++ b/docs/zh/operations/system-tables/disks.md @@ -1,31 +1,27 @@ ---- -machine_translated: true -machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 ---- +# system.disks {#system_tables-disks} -# 系统。磁盘 {#system_tables-disks} - -包含有关在定义的磁盘信息 [服务器配置](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure). +包含在 [服务器配置](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure) 中定义的磁盘信息. 列: -- `name` ([字符串](../../sql-reference/data-types/string.md)) — Name of a disk in the server configuration. -- `path` ([字符串](../../sql-reference/data-types/string.md)) — Path to the mount point in the file system. -- `free_space` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Free space on disk in bytes. -- `total_space` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Disk volume in bytes. -- `keep_free_space` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Amount of disk space that should stay free on disk in bytes. Defined in the `keep_free_space_bytes` 磁盘配置参数。 +- `name` ([字符串](../../sql-reference/data-types/string.md)) — 服务器配置中的磁盘名称. +- `path` ([字符串](../../sql-reference/data-types/string.md)) — 文件系统中挂载点的路径Path to the mount point in the file system. +- `free_space` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 磁盘上的可用空间,以字节为单位. +- `total_space` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 磁盘容量,以字节为单位。 +- `keep_free_space` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 在磁盘上应保持空闲的磁盘空间的数量,以字节为单位。在磁盘配置的 `keep_free_space_bytes` 参数中定义。 -## 系统。storage_policies {#system_tables-storage_policies} +**示例** -包含有关存储策略和卷中定义的信息 [服务器配置](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure). +```sql +:) SELECT * FROM system.disks; +``` -列: +```text +┌─name────┬─path─────────────────┬───free_space─┬──total_space─┬─keep_free_space─┐ +│ default │ /var/lib/clickhouse/ │ 276392587264 │ 490652508160 │ 0 │ +└─────────┴──────────────────────┴──────────────┴──────────────┴─────────────────┘ -- `policy_name` ([字符串](../../sql-reference/data-types/string.md)) — Name of the storage policy. -- `volume_name` ([字符串](../../sql-reference/data-types/string.md)) — Volume name defined in the storage policy. -- `volume_priority` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Volume order number in the configuration. -- `disks` ([数组(字符串)](../../sql-reference/data-types/array.md)) — Disk names, defined in the storage policy. -- `max_data_part_size` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Maximum size of a data part that can be stored on volume disks (0 — no limit). -- `move_factor` ([Float64](../../sql-reference/data-types/float.md)) — Ratio of free disk space. When the ratio exceeds the value of configuration parameter, ClickHouse start to move data to the next volume in order. +1 rows in set. Elapsed: 0.001 sec. +``` -如果存储策略包含多个卷,则每个卷的信息将存储在表的单独行中。 +[原文](https://clickhouse.com/docs/zh/operations/system-tables/disks) From 98c620cfac2683a9a3dde3c133a73165be48c12c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 22 Dec 2021 11:19:21 +0800 Subject: [PATCH 0262/1260] inject git info for clickhouse-client clickhouse-server --- programs/client/Client.cpp | 16 ++++++++++++++++ programs/server/Server.cpp | 13 +++++++++++++ src/CMakeLists.txt | 28 ++++++++++++++++++++++++++++ src/Common/git_info.h.in | 19 +++++++++++++++++++ 4 files changed, 76 insertions(+) create mode 100644 src/Common/git_info.h.in diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index e01677aaac6..013870907db 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -21,6 +21,7 @@ #include #include +#include #include #include #include @@ -76,6 +77,13 @@ namespace ErrorCodes extern const int AUTHENTICATION_FAILED; } +namespace GitInfo +{ + extern const std::string GIT_SHA1; + extern const std::string GIT_BRANCH; + extern const std::string GIT_DATE; + extern const std::string GIT_COMMIT_SUBJECT; +} void Client::processError(const String & query) const { @@ -1143,6 +1151,14 @@ void Client::processOptions(const OptionsDescription & options_description, global_context->getClientInfo().client_trace_context.tracestate = options["opentelemetry-tracestate"].as(); } +static void showClientVersion() +{ + std::cout << DBMS_NAME << " client version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl; + std::cout << "GIT SHA1: " << GitInfo::GIT_SHA1 << std::endl; + std::cout << "GIT BRANCH: " << GitInfo::GIT_BRANCH << std::endl; + std::cout << "GIT DATE: " << GitInfo::GIT_DATE << std::endl; + std::cout << "GIT COMMIT SUBJECT: " << GitInfo::GIT_COMMIT_SUBJECT << std::endl; +} void Client::processConfig() { diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 14075f9fbf2..560ecb5d249 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -84,6 +84,7 @@ #include "config_core.h" #include "Common/config_version.h" +#include "Common/git_info.h" #if defined(OS_LINUX) # include @@ -127,6 +128,14 @@ namespace CurrentMetrics extern const Metric MaxPushedDDLEntryID; } +namespace GitInfo +{ + extern const std::string GIT_SHA1; + extern const std::string GIT_BRANCH; + extern const std::string GIT_DATE; + extern const std::string GIT_COMMIT_SUBJECT; +} + namespace fs = std::filesystem; #if USE_JEMALLOC @@ -396,6 +405,10 @@ int Server::run() if (config().hasOption("version")) { std::cout << DBMS_NAME << " server version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl; + std::cout << "GIT SHA1: " << GitInfo::GIT_SHA1 << std::endl; + std::cout << "GIT BRANCH: " << GitInfo::GIT_BRANCH << std::endl; + std::cout << "GIT DATE: " << GitInfo::GIT_DATE << std::endl; + std::cout << "GIT COMMIT SUBJECT: " << GitInfo::GIT_COMMIT_SUBJECT << std::endl; return 0; } return Application::run(); // NOLINT diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 7124961821e..2d097e47ef9 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -20,6 +20,7 @@ include(../cmake/limit_jobs.cmake) set (CONFIG_VERSION "${CMAKE_CURRENT_BINARY_DIR}/Common/config_version.h") set (CONFIG_COMMON "${CMAKE_CURRENT_BINARY_DIR}/Common/config.h") +set (GIT_INFO "${CMAKE_CURRENT_BINARY_DIR}/Common/git_info.h") include (../cmake/version.cmake) message (STATUS "Will build ${VERSION_FULL} revision ${VERSION_REVISION} ${VERSION_OFFICIAL}") @@ -27,6 +28,33 @@ configure_file (Common/config.h.in ${CONFIG_COMMON}) configure_file (Common/config_version.h.in ${CONFIG_VERSION}) configure_file (Core/config_core.h.in "${CMAKE_CURRENT_BINARY_DIR}/Core/include/config_core.h") +set(GIT_EXECUTABLE /usr/bin/git) +# the commit's SHA1, and whether the building workspace was dirty or not +execute_process(COMMAND + "${GIT_EXECUTABLE}" rev-parse HEAD + WORKING_DIRECTORY "${CMAKE_SOURCE_DIR}" + OUTPUT_VARIABLE GIT_SHA1 + ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) +# git branch name +execute_process(COMMAND + "${GIT_EXECUTABLE}" rev-parse --abbrev-ref HEAD + WORKING_DIRECTORY "${CMAKE_SOURCE_DIR}" + OUTPUT_VARIABLE GIT_BRANCH + ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) +# the date of the commit +execute_process(COMMAND + "${GIT_EXECUTABLE}" log -1 --format=%ad --date=local + WORKING_DIRECTORY "${CMAKE_SOURCE_DIR}" + OUTPUT_VARIABLE GIT_DATE + ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) +# the subject of the commit +execute_process(COMMAND + "${GIT_EXECUTABLE}" log -1 --format=%s + WORKING_DIRECTORY "${CMAKE_SOURCE_DIR}" + OUTPUT_VARIABLE GIT_COMMIT_SUBJECT + ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) +configure_file (Common/git_info.h.in ${GIT_INFO}) + if (USE_DEBUG_HELPERS) get_target_property(MAGIC_ENUM_INCLUDE_DIR magic_enum INTERFACE_INCLUDE_DIRECTORIES) set (INCLUDE_DEBUG_HELPERS "-I\"${ClickHouse_SOURCE_DIR}/base\" -I\"${MAGIC_ENUM_INCLUDE_DIR}\" -include \"${ClickHouse_SOURCE_DIR}/src/Core/iostream_debug_helpers.h\"") diff --git a/src/Common/git_info.h.in b/src/Common/git_info.h.in new file mode 100644 index 00000000000..d338ae2e1e5 --- /dev/null +++ b/src/Common/git_info.h.in @@ -0,0 +1,19 @@ +#pragma once + +// .h autogenerated by cmake! + +#include + + +namespace DB +{ + +namespace GitInfo +{ + const std::string GIT_SHA1 = "@GIT_SHA1@"; + const std::string GIT_BRANCH = "@GIT_BRANCH@"; + const std::string GIT_DATE = "@GIT_DATE@"; + const std::string GIT_COMMIT_SUBJECT = "@GIT_COMMIT_SUBJECT@"; +} + +} From 81e2e72dd07120708ce3b4fdd3886e7884fe84ed Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 23 Dec 2021 20:52:36 +0800 Subject: [PATCH 0263/1260] fix bug --- programs/client/Client.cpp | 16 ---------------- src/Client/ClientBase.cpp | 12 ++++++++++++ 2 files changed, 12 insertions(+), 16 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 013870907db..e01677aaac6 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -21,7 +21,6 @@ #include #include -#include #include #include #include @@ -77,13 +76,6 @@ namespace ErrorCodes extern const int AUTHENTICATION_FAILED; } -namespace GitInfo -{ - extern const std::string GIT_SHA1; - extern const std::string GIT_BRANCH; - extern const std::string GIT_DATE; - extern const std::string GIT_COMMIT_SUBJECT; -} void Client::processError(const String & query) const { @@ -1151,14 +1143,6 @@ void Client::processOptions(const OptionsDescription & options_description, global_context->getClientInfo().client_trace_context.tracestate = options["opentelemetry-tracestate"].as(); } -static void showClientVersion() -{ - std::cout << DBMS_NAME << " client version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl; - std::cout << "GIT SHA1: " << GitInfo::GIT_SHA1 << std::endl; - std::cout << "GIT BRANCH: " << GitInfo::GIT_BRANCH << std::endl; - std::cout << "GIT DATE: " << GitInfo::GIT_DATE << std::endl; - std::cout << "GIT COMMIT SUBJECT: " << GitInfo::GIT_COMMIT_SUBJECT << std::endl; -} void Client::processConfig() { diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 8fd95bb706c..4ac31ca65bb 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -23,6 +23,7 @@ #include "Formats/FormatFactory.h" #include +#include #include #include #include @@ -107,6 +108,13 @@ namespace ProfileEvents namespace DB { +namespace GitInfo +{ + extern const std::string GIT_SHA1; + extern const std::string GIT_BRANCH; + extern const std::string GIT_DATE; + extern const std::string GIT_COMMIT_SUBJECT; +} static void incrementProfileEventsBlock(Block & dst, const Block & src) { @@ -1641,6 +1649,10 @@ void ClientBase::clearTerminal() void ClientBase::showClientVersion() { std::cout << DBMS_NAME << " " + getName() + " version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl; + std::cout << "GIT SHA1: " << GitInfo::GIT_SHA1 << std::endl; + std::cout << "GIT BRANCH: " << GitInfo::GIT_BRANCH << std::endl; + std::cout << "GIT DATE: " << GitInfo::GIT_DATE << std::endl; + std::cout << "GIT COMMIT SUBJECT: " << GitInfo::GIT_COMMIT_SUBJECT << std::endl; } From d9bcb633de95762cbbacffe00aaa93baea7ae1b1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 23 Dec 2021 13:53:36 +0100 Subject: [PATCH 0264/1260] Change perf set_index test to check build index --- tests/performance/set_index.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/performance/set_index.xml b/tests/performance/set_index.xml index 2a8bb925c7d..631cad9986e 100644 --- a/tests/performance/set_index.xml +++ b/tests/performance/set_index.xml @@ -3,14 +3,14 @@ INSERT INTO test_in SELECT number FROM numbers(500000000) - SELECT count() FROM test_in WHERE a IN (SELECT rand(1) FROM numbers(200000)) SETTINGS max_rows_to_read = 1, read_overflow_mode = 'break' + SELECT count() FROM test_in WHERE a IN (SELECT rand(1) FROM numbers(200000)) SETTINGS max_rows_to_read = 200001, read_overflow_mode = 'break' - SELECT count() FROM test_in WHERE toInt64(a) IN (SELECT toInt64(rand(1)) FROM numbers(200000)) settings max_rows_to_read=1, read_overflow_mode='break' + SELECT count() FROM test_in WHERE toInt64(a) IN (SELECT toInt64(rand(1)) FROM numbers(200000)) settings max_rows_to_read=200001, read_overflow_mode='break' - SELECT count() FROM test_in WHERE -toInt64(a) IN (SELECT toInt64(rand(1)) FROM numbers(200000)) settings max_rows_to_read=1, read_overflow_mode='break' + SELECT count() FROM test_in WHERE -toInt64(a) IN (SELECT toInt64(rand(1)) FROM numbers(200000)) settings max_rows_to_read=200001, read_overflow_mode='break' - SELECT count() FROM test_in WHERE -toInt64(a) NOT IN (SELECT toInt64(rand(1)) FROM numbers(200000)) settings max_rows_to_read=1, read_overflow_mode='break' + SELECT count() FROM test_in WHERE -toInt64(a) NOT IN (SELECT toInt64(rand(1)) FROM numbers(200000)) settings max_rows_to_read=200001, read_overflow_mode='break' From b71a71d7f57c75a5f8e63be85516f96a76b15ab5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 23 Dec 2021 16:01:47 +0100 Subject: [PATCH 0266/1260] Test that JSON output statistics include the rows read in scalar subqueries --- .../02136_scalar_read_rows_json.reference | 50 +++++++++++++++++++ .../02136_scalar_read_rows_json.sh | 10 ++++ 2 files changed, 60 insertions(+) create mode 100644 tests/queries/0_stateless/02136_scalar_read_rows_json.reference create mode 100755 tests/queries/0_stateless/02136_scalar_read_rows_json.sh diff --git a/tests/queries/0_stateless/02136_scalar_read_rows_json.reference b/tests/queries/0_stateless/02136_scalar_read_rows_json.reference new file mode 100644 index 00000000000..49020a4432f --- /dev/null +++ b/tests/queries/0_stateless/02136_scalar_read_rows_json.reference @@ -0,0 +1,50 @@ +#1 +{ + "meta": + [ + { + "name": "count()", + "type": "UInt64" + } + ], + + "data": + [ + { + "count()": "100" + } + ], + + "rows": 1, + + "rows_before_limit_at_least": 100, + + "statistics": + { + "rows_read": 100, + "bytes_read": 800 + } +} +#2 +{ + "meta": + [ + { + "type": "Tuple(UInt64, UInt64)" + } + ], + + "data": + [ + { + } + ], + + "rows": 1, + + "statistics": + { + "rows_read": 131011, + "bytes_read": 1048081 + } +} diff --git a/tests/queries/0_stateless/02136_scalar_read_rows_json.sh b/tests/queries/0_stateless/02136_scalar_read_rows_json.sh new file mode 100755 index 00000000000..d589cb60086 --- /dev/null +++ b/tests/queries/0_stateless/02136_scalar_read_rows_json.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash +# Ref: https://github.com/ClickHouse/ClickHouse/issues/1576 +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo "#1" +${CLICKHOUSE_CLIENT} --query='SELECT count() FROM numbers(100) FORMAT JSON;' | grep -a -v "elapsed" +echo "#2" +${CLICKHOUSE_CLIENT} --query='SELECT (SELECT max(number), count(number) FROM numbers(100000) as n) FORMAT JSON;' | grep -a -v "elapsed" | grep -v "_subquery" From 66c0735a3877d0766cdeb4f933cbaac92d4cdc55 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 23 Dec 2021 16:11:23 +0100 Subject: [PATCH 0267/1260] Check that progress is sent while processing scalar subqueries --- tests/queries/0_stateless/02136_scalar_progress.reference | 6 ++++++ tests/queries/0_stateless/02136_scalar_progress.sh | 7 +++++++ 2 files changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/02136_scalar_progress.reference create mode 100755 tests/queries/0_stateless/02136_scalar_progress.sh diff --git a/tests/queries/0_stateless/02136_scalar_progress.reference b/tests/queries/0_stateless/02136_scalar_progress.reference new file mode 100644 index 00000000000..21f6d3e0043 --- /dev/null +++ b/tests/queries/0_stateless/02136_scalar_progress.reference @@ -0,0 +1,6 @@ +< X-ClickHouse-Progress: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000"} +< X-ClickHouse-Progress: {"read_rows":"65505","read_bytes":"524040","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000"} +< X-ClickHouse-Progress: {"read_rows":"131010","read_bytes":"1048080","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000"} +< X-ClickHouse-Progress: {"read_rows":"131011","read_bytes":"1048081","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000"} +< X-ClickHouse-Progress: {"read_rows":"131011","read_bytes":"1048081","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000"} +< X-ClickHouse-Summary: {"read_rows":"131011","read_bytes":"1048081","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000"} diff --git a/tests/queries/0_stateless/02136_scalar_progress.sh b/tests/queries/0_stateless/02136_scalar_progress.sh new file mode 100755 index 00000000000..4608031f83d --- /dev/null +++ b/tests/queries/0_stateless/02136_scalar_progress.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash +# Ref: https://github.com/ClickHouse/ClickHouse/issues/1576 +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CURL -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d "SELECT (SELECT max(number), count(number) FROM numbers(100000));" -v 2>&1 | grep -E "X-ClickHouse-Summary|X-ClickHouse-Progress" From 547c0f26d5d036666a80db27a30f914917c2bc49 Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Thu, 23 Dec 2021 23:21:05 +0800 Subject: [PATCH 0268/1260] Update databases.md --- docs/zh/operations/system-tables/databases.md | 43 +++++++++++++++---- 1 file changed, 35 insertions(+), 8 deletions(-) diff --git a/docs/zh/operations/system-tables/databases.md b/docs/zh/operations/system-tables/databases.md index 134b8ebc7ab..3fadb02446d 100644 --- a/docs/zh/operations/system-tables/databases.md +++ b/docs/zh/operations/system-tables/databases.md @@ -1,12 +1,39 @@ ---- -machine_translated: true -machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 ---- +# system.databases {#system-databases} -# 系统。数据库 {#system-databases} +包含当前用户可用的数据库的相关信息。 -此表包含一个名为"字符串"的列 ‘name’ – the name of a database. +列: -服务器知道的每个数据库在表中都有相应的条目。 +- `name` ([String](../../sql-reference/data-types/string.md)) — 数据库的名称。 +- `engine` ([String](../../sql-reference/data-types/string.md)) — [数据库的引擎](../../engines/database-engines/index.md)。 +- `data_path` ([String](../../sql-reference/data-types/string.md)) — 数据的路径。 +- `metadata_path` ([String](../../sql-reference/data-types/enum.md)) — 元数据的路径。 +- `uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — 数据库的 UUID。 +- `comment` ([String](../../sql-reference/data-types/enum.md)) — 数据库的注释。 -该系统表用于实现 `SHOW DATABASES` 查询。 +这个系统表的 `name` 列被用于实现 `SHOW DATABASES` 查询。 + +**示例** + +创建一个数据库。 + +``` sql +CREATE DATABASE test; +``` + +查询此用户所有可用的数据库。 + +``` sql +SELECT * FROM system.databases; +``` + +``` text +┌─name───────────────┬─engine─┬─data_path──────────────────┬─metadata_path───────────────────────────────────────────────────────┬─uuid─────────────────────────────────┬─comment─┐ +│ INFORMATION_SCHEMA │ Memory │ /var/lib/clickhouse/ │ │ 00000000-0000-0000-0000-000000000000 │ │ +│ default │ Atomic │ /var/lib/clickhouse/store/ │ /var/lib/clickhouse/store/d31/d317b4bd-3595-4386-81ee-c2334694128a/ │ 24363899-31d7-42a0-a436-389931d752a0 │ │ +│ information_schema │ Memory │ /var/lib/clickhouse/ │ │ 00000000-0000-0000-0000-000000000000 │ │ +│ system │ Atomic │ /var/lib/clickhouse/store/ │ /var/lib/clickhouse/store/1d1/1d1c869d-e465-4b1b-a51f-be033436ebf9/ │ 03e9f3d1-cc88-4a49-83e9-f3d1cc881a49 │ │ +└────────────────────┴────────┴────────────────────────────┴─────────────────────────────────────────────────────────────────────┴──────────────────────────────────────┴─────────┘ +``` + +[原文](https://clickhouse.com/docs/zh/operations/system-tables/databases) From a97633f0688484ccac2fa580d7563975c7a02bdd Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Thu, 23 Dec 2021 23:25:31 +0800 Subject: [PATCH 0269/1260] Update disks.md Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/zh/operations/system-tables/disks.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/operations/system-tables/disks.md b/docs/zh/operations/system-tables/disks.md index 7100bac8dd6..2a6dcc9ae45 100644 --- a/docs/zh/operations/system-tables/disks.md +++ b/docs/zh/operations/system-tables/disks.md @@ -5,7 +5,7 @@ 列: - `name` ([字符串](../../sql-reference/data-types/string.md)) — 服务器配置中的磁盘名称. -- `path` ([字符串](../../sql-reference/data-types/string.md)) — 文件系统中挂载点的路径Path to the mount point in the file system. +- `path` ([字符串](../../sql-reference/data-types/string.md)) — 文件系统中挂载点的路径. - `free_space` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 磁盘上的可用空间,以字节为单位. - `total_space` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 磁盘容量,以字节为单位。 - `keep_free_space` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 在磁盘上应保持空闲的磁盘空间的数量,以字节为单位。在磁盘配置的 `keep_free_space_bytes` 参数中定义。 From c79344373e5270b680b872fe72d06f511611798e Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Thu, 23 Dec 2021 23:40:01 +0800 Subject: [PATCH 0270/1260] Fix some words --- .../system-tables/storage_policies.md | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/docs/zh/operations/system-tables/storage_policies.md b/docs/zh/operations/system-tables/storage_policies.md index ce2adb74ee0..e2531649493 100644 --- a/docs/zh/operations/system-tables/storage_policies.md +++ b/docs/zh/operations/system-tables/storage_policies.md @@ -1,17 +1,17 @@ # system.storage_policies {#system_tables-storage_policies} -包含有关 [服务器配置](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure) 中定义的存储策略和卷信息. +包含有关 [服务器配置](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure) 中定义的存储策略和卷信息。 列: -- `policy_name` ([String](../../sql-reference/data-types/string.md)) — 存储策略名称. -- `volume_name` ([String](../../sql-reference/data-types/string.md)) — 存储策略中定义的卷名称. -- `volume_priority` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 配置中的卷顺序号,数据根据这个优先级填充卷,比如插入和合并期间的数据将被写入优先级较低的卷。 (考虑到其他规则: TTL, `max_data_part_size`, `move_factor`). -- `disks` ([Array(String)](../../sql-reference/data-types/array.md)) — 存储策略中定义的磁盘名. -- `max_data_part_size` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 可以存储在卷磁盘上数据部分的最大大小 (0 — 不限制). -- `move_factor` ([Float64](../../sql-reference/data-types/float.md)) — 磁盘空闲的比率。当比率超过配置的值,ClickHouse 将把数据向下一个卷移动. -- `prefer_not_to_merge` ([UInt8](../../sql-reference/data-types/int-uint.md)) — 设置中 `prefer_not_to_merge` 的值. 当这个设置启用时,不允许在此卷上合并数据。这将允许控制ClickHouse 如何与运行速度较慢的磁盘一起工作. +- `policy_name` ([String](../../sql-reference/data-types/string.md)) — 存储策略的名称。 +- `volume_name` ([String](../../sql-reference/data-types/string.md)) — 存储策略中定义的卷名称。 +- `volume_priority` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 配置中的卷顺序号,数据根据这个优先级填充卷,比如插入和合并期间的数据将被写入优先级较低的卷 (还需考虑其他规则: TTL, `max_data_part_size`, `move_factor`)。 +- `disks` ([Array(String)](../../sql-reference/data-types/array.md)) — 存储策略中定义的磁盘名。 +- `max_data_part_size` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 可以存储在卷磁盘上数据部分的最大大小 (0 - 不限制)。 +- `move_factor` ([Float64](../../sql-reference/data-types/float.md)) — 磁盘空闲的比率。当比率超过配置的值,ClickHouse 将把数据向下一个卷移动。 +- `prefer_not_to_merge` ([UInt8](../../sql-reference/data-types/int-uint.md)) — 设置中 `prefer_not_to_merge` 的值. 当这个设置启用时,不允许在此卷上合并数据。这将允许控制 ClickHouse 如何与运行速度较慢的磁盘一起工作。 -如果存储策略包含多个卷,则每个卷的信息将在表中存储为单独行。 +如果存储策略包含多个卷,则每个卷的信息将在表中作为单独一行存储。 -[原文](https://clickhouse.com/docs/en/operations/system-tables/storage_policies) +[原文](https://clickhouse.com/docs/zh/operations/system-tables/storage_policies) From 0d463ffc2da6b8a20ae0aa62e5b7bb3fd29c4e2d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 23 Dec 2021 15:59:00 +0000 Subject: [PATCH 0271/1260] Fix build --- src/AggregateFunctions/AggregateFunctionNothing.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionNothing.h b/src/AggregateFunctions/AggregateFunctionNothing.h index d1dd321f0df..22e296a96a4 100644 --- a/src/AggregateFunctions/AggregateFunctionNothing.h +++ b/src/AggregateFunctions/AggregateFunctionNothing.h @@ -48,7 +48,7 @@ public: size_t sizeOfData() const override { - return 1; + return 0; } size_t alignOfData() const override @@ -65,13 +65,13 @@ public: } - void serialize(ConstAggregateDataPtr __restrict, WriteBuffer &, std::optional) const override + void serialize(ConstAggregateDataPtr __restrict, WriteBuffer & buf, std::optional) const override { writeChar('\0', buf); } - void deserialize(AggregateDataPtr, ReadBuffer &, std::optional, Arena *) const override + void deserialize(AggregateDataPtr, ReadBuffer & buf, std::optional, Arena *) const override { [[maybe_unused]] char symbol; readChar(symbol, buf); From 3d2bacc56d321c8082f618bd65ff5f151179740f Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Fri, 24 Dec 2021 00:38:50 +0800 Subject: [PATCH 0272/1260] Update one.md --- docs/zh/operations/system-tables/one.md | 27 +++++++++++++++++-------- 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/docs/zh/operations/system-tables/one.md b/docs/zh/operations/system-tables/one.md index a8dc64c18c7..79d2c0199d8 100644 --- a/docs/zh/operations/system-tables/one.md +++ b/docs/zh/operations/system-tables/one.md @@ -1,12 +1,23 @@ ---- -machine_translated: true -machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 ---- +# system.one {#system-one} -# 系统。一 {#system-one} +此表包含一行只有一个值为 0 的 `dummy` UInt8 列的数据。 -此表包含一行,其中包含一行 `dummy` UInt8列包含值0。 +如果 `SELECT` 查询没有指定 `FROM` 子句,就会使用这个表来查询。 -如果使用此表 `SELECT` 查询不指定 `FROM` 条款 +这个表类似于其他数据库管理系统(DMBS)中的 `DUAL` 表。 -这类似于 `DUAL` 表在其他Dbms中找到。 +**示例** + +```sql +:) SELECT * FROM system.one LIMIT 10; +``` + +```text +┌─dummy─┐ +│ 0 │ +└───────┘ + +1 rows in set. Elapsed: 0.001 sec. +``` + +[原文](https://clickhouse.com/docs/zh/operations/system-tables/one) From 19faf46b683c734d6250f3a8c0ffda7fa7f2c50a Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 20 Dec 2021 23:24:16 +0300 Subject: [PATCH 0273/1260] Improve Bool data type --- .../Serializations/SerializationBool.cpp | 177 ++++++++++++------ .../Serializations/SerializationBool.h | 19 +- src/Functions/FunctionsConversion.h | 23 ++- .../0_stateless/02152_bool_type.reference | 36 ++++ tests/queries/0_stateless/02152_bool_type.sql | 37 ++++ .../02152_bool_type_parsing.reference | 42 +++++ .../0_stateless/02152_bool_type_parsing.sh | 16 ++ 7 files changed, 285 insertions(+), 65 deletions(-) create mode 100644 tests/queries/0_stateless/02152_bool_type.reference create mode 100644 tests/queries/0_stateless/02152_bool_type.sql create mode 100644 tests/queries/0_stateless/02152_bool_type_parsing.reference create mode 100755 tests/queries/0_stateless/02152_bool_type_parsing.sh diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp index def2b565afc..d1c9e7f475f 100644 --- a/src/DataTypes/Serializations/SerializationBool.cpp +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -7,6 +7,9 @@ #include #include +#include +#include + namespace DB { @@ -16,67 +19,57 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } -SerializationBool::SerializationBool(const SerializationPtr &nested_) - : SerializationCustomSimpleText(nested_) +static const std::unordered_set true_values = { -} + "true", + "yes", + "on", + "t", + "y", + "1", +}; -void SerializationBool::serializeText(const IColumn &column, size_t row_num, WriteBuffer &ostr, const FormatSettings &) const +static const std::unordered_set false_values = { - const auto *col = checkAndGetColumn(&column); - if (!col) + "false", + "no", + "off", + "f", + "n", + "0", +}; + +static const ColumnUInt8 * checkAndGetSerializeColumnType(const IColumn & column) +{ + const auto * col = checkAndGetColumn(&column); + if (!checkAndGetColumn(&column)) throw Exception("Bool type can only serialize columns of type UInt8." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); - - if (col->getData()[row_num]) - ostr.write(str_true, sizeof(str_true) - 1); - else - ostr.write(str_false, sizeof(str_false) - 1); + return col; } -void SerializationBool::deserializeText(IColumn &column, ReadBuffer &istr, const FormatSettings & settings, bool whole) const +static ColumnUInt8 * checkAndGetDeserializeColumnType(IColumn & column) { - ColumnUInt8 *col = typeid_cast(&column); - if (!col) - { + auto * col = typeid_cast(&column); + if (!checkAndGetColumn(&column)) throw Exception("Bool type can only deserialize columns of type UInt8." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); - } + return col; +} - if (!istr.eof()) - { - bool value = false; +SerializationBool::SerializationBool(const SerializationPtr &nested_) + : SerializationWrapper(nested_) +{ +} - if (*istr.position() == 't' || *istr.position() == 'f' || *istr.position() == 'T' || *istr.position() == 'F') - readBoolTextWord(value, istr, true); - else if (*istr.position() == '1' || *istr.position() == '0') - readBoolText(value, istr); - else - throw Exception("Invalid boolean value, should be true/false, TRUE/FALSE, 1/0.", - ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); - col->insert(value); - } - else - throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); - - if (whole && !istr.eof()) - throwUnexpectedDataAfterParsedValue(column, istr, settings, "Bool"); +void SerializationBool::serializeText(const IColumn & column, size_t row_num, WriteBuffer &ostr, const FormatSettings & settings) const +{ + serializeCustom(column, row_num, ostr, settings); } void SerializationBool::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - const auto *col = checkAndGetColumn(&column); - if (!col) - throw Exception("Bool type can only serialize columns of type UInt8." + column.getName(), - ErrorCodes::ILLEGAL_COLUMN); - if (col->getData()[row_num]) - { - writeString(settings.bool_true_representation, ostr); - } - else - { - writeString(settings.bool_false_representation, ostr); - } + serializeCustom(column, row_num, ostr, settings); } void SerializationBool::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -90,17 +83,12 @@ void SerializationBool::deserializeTextEscaped(IColumn & column, ReadBuffer & is void SerializationBool::serializeTextJSON(const IColumn &column, size_t row_num, WriteBuffer &ostr, const FormatSettings &settings) const { - serializeText(column, row_num, ostr, settings); + serializeSimple(column, row_num, ostr, settings); } void SerializationBool::deserializeTextJSON(IColumn &column, ReadBuffer &istr, const FormatSettings &) const { - ColumnUInt8 *col = typeid_cast(&column); - if (!col) - { - throw Exception("Bool type can only deserialize columns of type UInt8." + column.getName(), - ErrorCodes::ILLEGAL_COLUMN); - } + ColumnUInt8 * col = checkAndGetDeserializeColumnType(column); if (!istr.eof()) { @@ -121,7 +109,7 @@ void SerializationBool::deserializeTextJSON(IColumn &column, ReadBuffer &istr, c void SerializationBool::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - serializeTextEscaped(column, row_num, ostr, settings); + serializeCustom(column, row_num, ostr, settings); } void SerializationBool::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -135,7 +123,7 @@ void SerializationBool::deserializeTextCSV(IColumn & column, ReadBuffer & istr, void SerializationBool::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - serializeTextEscaped(column, row_num, ostr, settings); + serializeCustom(column, row_num, ostr, settings); } void SerializationBool::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -147,13 +135,68 @@ void SerializationBool::deserializeTextRaw(IColumn & column, ReadBuffer & istr, deserializeFromString(column, input, settings); } -void SerializationBool::deserializeFromString(IColumn & column, String & input, const FormatSettings & settings) +void SerializationBool::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - ColumnUInt8 * col = typeid_cast(&column); - if (!col) + serializeSimple(column, row_num, ostr, settings); +} + +void SerializationBool::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + if (!istr.eof()) + throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); + + auto * col = checkAndGetDeserializeColumnType(column); + bool value = false; + + if (*istr.position() == 't' || *istr.position() == 'f' || *istr.position() == 'T' || *istr.position() == 'F') + readBoolTextWord(value, istr, true); + else if (*istr.position() == '1' || *istr.position() == '0') + readBoolText(value, istr); + else + throw Exception("Invalid boolean value, should be true/false, TRUE/FALSE, 1/0.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); + col->insert(value); +} + +void SerializationBool::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String input; + readStringUntilEOF(input, istr); + deserializeFromString(column, input, settings); + assert(istr.eof()); +} + +void SerializationBool::serializeCustom(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const auto * col = checkAndGetSerializeColumnType(column); + + if (col->getData()[row_num]) { - throw Exception("Bool type can only deserialize columns of type UInt8." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); + writeString(settings.bool_true_representation, ostr); } + else + { + writeString(settings.bool_false_representation, ostr); + } +} + +void SerializationBool::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + serializeSimple(column, row_num, ostr, settings); +} + +void SerializationBool::serializeSimple(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const +{ + const auto * col = checkAndGetSerializeColumnType(column); + + if (col->getData()[row_num]) + ostr.write(str_true, sizeof(str_true) - 1); + else + ostr.write(str_false, sizeof(str_false) - 1); +} + +void SerializationBool::deserializeFromString(IColumn & column, String & input, const FormatSettings & settings) const +{ + ColumnUInt8 * col = checkAndGetDeserializeColumnType(column); if (settings.bool_true_representation == input) { @@ -164,6 +207,22 @@ void SerializationBool::deserializeFromString(IColumn & column, String & input, col->insert(false); } else - throw Exception("Invalid boolean value, should be " + settings.bool_true_representation + " or " + settings.bool_false_representation + " controlled by setting bool_true_representation and bool_false_representation.", ErrorCodes::ILLEGAL_COLUMN); + { + String input_lower = boost::algorithm::to_lower_copy(input); + if (true_values.contains(input_lower)) + { + col->insert(true); + } + else if (false_values.contains(input_lower)) + { + col->insert(false); + } + else + throw Exception( + "Invalid boolean value '" + input + "', should be " + settings.bool_true_representation + " or " + settings.bool_false_representation + + " controlled by setting bool_true_representation and bool_false_representation or one of " + "True/False/T/F/Y/N/Yes/No/On/Off", + ErrorCodes::ILLEGAL_COLUMN); + } } } diff --git a/src/DataTypes/Serializations/SerializationBool.h b/src/DataTypes/Serializations/SerializationBool.h index eda37864db5..dc7abbbb088 100644 --- a/src/DataTypes/Serializations/SerializationBool.h +++ b/src/DataTypes/Serializations/SerializationBool.h @@ -1,11 +1,12 @@ #pragma once -#include +#include +#include namespace DB { -class SerializationBool final : public SerializationCustomSimpleText +class SerializationBool final : public SerializationWrapper { private: static constexpr char str_true[5] = "true"; @@ -15,12 +16,11 @@ public: SerializationBool(const SerializationPtr & nested_); void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; - void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings,bool whole) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; - void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; @@ -29,8 +29,17 @@ public: void serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + + void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + protected: - static void deserializeFromString(IColumn & column, String & input, const FormatSettings & settings); + void serializeCustom(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; + void serializeSimple(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; + void deserializeFromString(IColumn & column, String & input, const FormatSettings & settings) const; }; } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index ac360834c94..895e5699d19 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2610,6 +2610,20 @@ private: }; } + template + WrapperType createBoolWrapper(const DataTypePtr & from_type, const ToDataType * const to_type, bool requested_result_is_nullable) const + { + if (checkAndGetDataType(from_type.get())) + { + return [] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t /*input_rows_count*/) + { + return ConvertImplGenericFromString::execute(arguments, result_type); + }; + } + + return createWrapper(from_type, to_type, requested_result_is_nullable); + } + static WrapperType createStringWrapper(const DataTypePtr & from_type) { FunctionPtr function = FunctionToString::create(); @@ -3278,7 +3292,6 @@ private: using ToDataType = typename Types::LeftType; if constexpr ( - std::is_same_v || std::is_same_v || std::is_same_v || std::is_same_v || @@ -3300,6 +3313,14 @@ private: ret = createWrapper(from_type, checkAndGetDataType(to_type.get()), requested_result_is_nullable); return true; } + if constexpr (std::is_same_v) + { + if (to_type->getName() == "Bool") + ret = createBoolWrapper(from_type, checkAndGetDataType(to_type.get()), requested_result_is_nullable); + else + ret = createWrapper(from_type, checkAndGetDataType(to_type.get()), requested_result_is_nullable); + return true; + } if constexpr ( std::is_same_v || std::is_same_v) diff --git a/tests/queries/0_stateless/02152_bool_type.reference b/tests/queries/0_stateless/02152_bool_type.reference new file mode 100644 index 00000000000..98eca499889 --- /dev/null +++ b/tests/queries/0_stateless/02152_bool_type.reference @@ -0,0 +1,36 @@ +true +true +true +true +true +true +true +true +true +true +true +false +false +false +false +false +false +false +false +false +false +false +false +false +Custom true +Custom true +(true) +Row 1: +────── +CAST('true', 'Bool'): Custom true +┏━━━━━━━━━━━━━━━━━━━━━━┓ +┃ CAST('true', 'Bool') ┃ +┡━━━━━━━━━━━━━━━━━━━━━━┩ +│ Custom true │ +└──────────────────────┘ +{"CAST('true', 'Bool')":true} diff --git a/tests/queries/0_stateless/02152_bool_type.sql b/tests/queries/0_stateless/02152_bool_type.sql new file mode 100644 index 00000000000..b7e3e19a42d --- /dev/null +++ b/tests/queries/0_stateless/02152_bool_type.sql @@ -0,0 +1,37 @@ +SELECT CAST('True', 'Bool'); +SELECT CAST('TrUe', 'Bool'); +SELECT CAST('true', 'Bool'); +SELECT CAST('On', 'Bool'); +SELECT CAST('on', 'Bool'); +SELECT CAST('Yes', 'Bool'); +SELECT CAST('yes', 'Bool'); +SELECT CAST('T', 'Bool'); +SELECT CAST('t', 'Bool'); +SELECT CAST('Y', 'Bool'); +SELECT CAST('y', 'Bool'); +SELECT CAST('0', 'Bool'); + +SELECT CAST('False', 'Bool'); +SELECT CAST('FaLse', 'Bool'); +SELECT CAST('false', 'Bool'); +SELECT CAST('Off', 'Bool'); +SELECT CAST('off', 'Bool'); +SELECT CAST('No', 'Bool'); +SELECT CAST('no', 'Bool'); +SELECT CAST('N', 'Bool'); +SELECT CAST('n', 'Bool'); +SELECT CAST('F', 'Bool'); +SELECT CAST('f', 'Bool'); +SELECT CAST('0', 'Bool'); + +SET bool_true_representation = 'Custom true'; +SET bool_false_representation = 'Custom false'; + +SELECT CAST('true', 'Bool') format CSV; +SELECT CAST('true', 'Bool') format TSV; +SELECT CAST('true', 'Bool') format Values; +SELECT ''; +SELECT CAST('true', 'Bool') format Vertical; +SELECT CAST('true', 'Bool') format Pretty; +SELECT CAST('true', 'Bool') format JSONEachRow; + diff --git a/tests/queries/0_stateless/02152_bool_type_parsing.reference b/tests/queries/0_stateless/02152_bool_type_parsing.reference new file mode 100644 index 00000000000..b75df1faa65 --- /dev/null +++ b/tests/queries/0_stateless/02152_bool_type_parsing.reference @@ -0,0 +1,42 @@ +true +false +true +false +true +false +true +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +true +false +true +false +true +false +true +false +true +false +true +false +true +false diff --git a/tests/queries/0_stateless/02152_bool_type_parsing.sh b/tests/queries/0_stateless/02152_bool_type_parsing.sh new file mode 100755 index 00000000000..8a956bfc15b --- /dev/null +++ b/tests/queries/0_stateless/02152_bool_type_parsing.sh @@ -0,0 +1,16 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +FILE_NAME=test_02152.data +DATA_FILE=$USER_FILES_PATH/$FILE_NAME + +echo -e "Custom true\nCustom false\nYes\nNo\nyes\nno\ny\nY\nN\nTrue\nFalse\ntrue\nfalse\nt\nf\nT\nF\nOn\nOff\non\noff" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'TSV', 'bool Bool') settings bool_true_representation='Custom true', bool_false_representation='Custom false'" +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'CSV', 'bool Bool') settings bool_true_representation='Custom true', bool_false_representation='Custom false'" + +rm $DATA_FILE + From 5e5d6609ed13489ae19eacc4adfae2e6e3b0e2d9 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 21 Dec 2021 02:59:08 +0300 Subject: [PATCH 0274/1260] Proper cast to bool --- src/DataTypes/IDataType.h | 5 ++ .../Serializations/SerializationBool.cpp | 8 ++- src/Functions/FunctionsConversion.h | 51 ++++++++++++++++++- .../0_stateless/02152_bool_type.reference | 10 ++++ tests/queries/0_stateless/02152_bool_type.sql | 13 ++++- 5 files changed, 82 insertions(+), 5 deletions(-) diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 864dde0e86e..e74df5c327a 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -512,6 +512,11 @@ inline bool isCompilableType(const DataTypePtr & data_type) return data_type->isValueRepresentedByNumber() && !isDecimal(data_type); } +inline bool isBool(const DataTypePtr & data_type) +{ + return data_type->getName() == "Bool"; +} + template constexpr bool IsDataTypeDecimal = false; template constexpr bool IsDataTypeNumber = false; template constexpr bool IsDataTypeDateOrDateTime = false; diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp index d1c9e7f475f..7d38db66538 100644 --- a/src/DataTypes/Serializations/SerializationBool.cpp +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -19,8 +19,10 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } -static const std::unordered_set true_values = +static const std::unordered_set true_values = { + "enable", + "enabled", "true", "yes", "on", @@ -29,8 +31,10 @@ static const std::unordered_set true_values = "1", }; -static const std::unordered_set false_values = +static const std::unordered_set false_values = { + "disable", + "disabled", "false", "no", "off", diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 895e5699d19..e36111231f1 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -268,6 +268,12 @@ struct ConvertImpl vec_to[i] = static_cast(vec_from[i]); } } + + if constexpr (std::is_same_v) + { + if (result_type->getName() == "Bool") + vec_to[i] = static_cast(vec_to[i]); + } } } @@ -1352,6 +1358,18 @@ struct ConvertImpl, T, Name, ConvertDefau } }; +template +struct ConvertImpl +{ + template + static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/, + Additions additions [[maybe_unused]] = Additions()) + { + + return arguments[0].column; + } +}; + /** Conversion from FixedString to String. * Cutting sequences of zero bytes from end of strings. @@ -1721,7 +1739,10 @@ private: throw Exception("Wrong UUID conversion", ErrorCodes::CANNOT_CONVERT_TYPE); } else - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count); + { + result_column + = ConvertImpl::execute(arguments, result_type, input_rows_count); + } } else { @@ -2551,6 +2572,7 @@ private: { /// In case when converting to Nullable type, we apply different parsing rule, /// that will not throw an exception but return NULL in case of malformed input. + FunctionPtr function = FunctionConvertFromString::create(); return createFunctionAdaptor(function, from_type); } @@ -2624,6 +2646,26 @@ private: return createWrapper(from_type, to_type, requested_result_is_nullable); } + WrapperType createUInt8ToUInt8Wrapper(const DataTypePtr from_type, const DataTypePtr to_type) const + { + return [from_type, to_type] (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t /*input_rows_count*/) -> ColumnPtr + { + if (isBool(from_type) || !isBool(to_type)) + return arguments.front().column; + + /// Special case when we convert UInt8 column to Bool column. + /// both columns have type UInt8, but we shouldn't use identity wrapper, + /// because Bool column can contain only 0 and 1. + auto res_column = to_type->createColumn(); + const auto & data_from = checkAndGetColumn(arguments[0].column.get())->getData(); + auto & data_to = assert_cast(res_column.get())->getData(); + data_to.resize(data_from.size()); + for (size_t i = 0; i != data_from.size(); ++i) + data_to[i] = static_cast(data_from[i]); + return res_column; + }; + } + static WrapperType createStringWrapper(const DataTypePtr & from_type) { FunctionPtr function = FunctionToString::create(); @@ -3280,7 +3322,12 @@ private: WrapperType prepareImpl(const DataTypePtr & from_type, const DataTypePtr & to_type, bool requested_result_is_nullable) const { if (from_type->equals(*to_type)) + { + if (isUInt8(from_type)) + return createUInt8ToUInt8Wrapper(from_type, to_type); + return createIdentityWrapper(from_type); + } else if (WhichDataType(from_type).isNothing()) return createNothingWrapper(to_type.get()); @@ -3315,7 +3362,7 @@ private: } if constexpr (std::is_same_v) { - if (to_type->getName() == "Bool") + if (isBool(to_type)) ret = createBoolWrapper(from_type, checkAndGetDataType(to_type.get()), requested_result_is_nullable); else ret = createWrapper(from_type, checkAndGetDataType(to_type.get()), requested_result_is_nullable); diff --git a/tests/queries/0_stateless/02152_bool_type.reference b/tests/queries/0_stateless/02152_bool_type.reference index 98eca499889..a8c04f651e9 100644 --- a/tests/queries/0_stateless/02152_bool_type.reference +++ b/tests/queries/0_stateless/02152_bool_type.reference @@ -9,6 +9,10 @@ true true true true +true +true +true +false false false false @@ -34,3 +38,9 @@ CAST('true', 'Bool'): Custom true │ Custom true │ └──────────────────────┘ {"CAST('true', 'Bool')":true} +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02152_bool_type.sql b/tests/queries/0_stateless/02152_bool_type.sql index b7e3e19a42d..e9efde0795f 100644 --- a/tests/queries/0_stateless/02152_bool_type.sql +++ b/tests/queries/0_stateless/02152_bool_type.sql @@ -9,7 +9,9 @@ SELECT CAST('T', 'Bool'); SELECT CAST('t', 'Bool'); SELECT CAST('Y', 'Bool'); SELECT CAST('y', 'Bool'); -SELECT CAST('0', 'Bool'); +SELECT CAST('1', 'Bool'); +SELECT CAST('enabled', 'Bool'); +SELECT CAST('enable', 'Bool'); SELECT CAST('False', 'Bool'); SELECT CAST('FaLse', 'Bool'); @@ -23,6 +25,8 @@ SELECT CAST('n', 'Bool'); SELECT CAST('F', 'Bool'); SELECT CAST('f', 'Bool'); SELECT CAST('0', 'Bool'); +SELECT CAST('disabled', 'Bool'); +SELECT CAST('disable', 'Bool'); SET bool_true_representation = 'Custom true'; SET bool_false_representation = 'Custom false'; @@ -35,3 +39,10 @@ SELECT CAST('true', 'Bool') format Vertical; SELECT CAST('true', 'Bool') format Pretty; SELECT CAST('true', 'Bool') format JSONEachRow; +SELECT CAST(CAST(2, 'Bool'), 'UInt8'); +SELECT CAST(CAST(toUInt32(2), 'Bool'), 'UInt8'); +SELECT CAST(CAST(toInt8(2), 'Bool'), 'UInt8'); +SELECT CAST(CAST(toFloat32(2), 'Bool'), 'UInt8'); +SELECT CAST(CAST(toDecimal32(2, 2), 'Bool'), 'UInt8'); +SELECT CAST(CAST(materialize(2), 'Bool'), 'UInt8'); + From ce68c54b0321733505184f912d88ec9ab35afacf Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 23 Dec 2021 16:46:16 +0300 Subject: [PATCH 0275/1260] Fix comments, reduce extra copying --- .../Serializations/SerializationBool.cpp | 213 ++++++++++++------ .../Serializations/SerializationBool.h | 4 +- src/Functions/FunctionsConversion.h | 3 +- .../02152_bool_type_parsing.reference | 104 +++++++++ .../0_stateless/02152_bool_type_parsing.sh | 11 +- 5 files changed, 259 insertions(+), 76 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp index 7d38db66538..e663bc87ea3 100644 --- a/src/DataTypes/Serializations/SerializationBool.cpp +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -6,43 +6,19 @@ #include #include #include +#include #include -#include namespace DB { namespace ErrorCodes { - extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING; extern const int ILLEGAL_COLUMN; + extern const int CANNOT_PARSE_BOOL; } -static const std::unordered_set true_values = -{ - "enable", - "enabled", - "true", - "yes", - "on", - "t", - "y", - "1", -}; - -static const std::unordered_set false_values = -{ - "disable", - "disabled", - "false", - "no", - "off", - "f", - "n", - "0", -}; - static const ColumnUInt8 * checkAndGetSerializeColumnType(const IColumn & column) { const auto * col = checkAndGetColumn(&column); @@ -79,10 +55,9 @@ void SerializationBool::serializeTextEscaped(const IColumn & column, size_t row_ void SerializationBool::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { if (istr.eof()) - throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); - String input; - readEscapedString(input, istr); - deserializeFromString(column, input, settings); + throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL); + + deserializeWithCustom(column, istr, settings, [](ReadBuffer & buf){ return buf.eof() || *buf.position() == '\t' || *buf.position() == '\n'; }); } void SerializationBool::serializeTextJSON(const IColumn &column, size_t row_num, WriteBuffer &ostr, const FormatSettings &settings) const @@ -104,11 +79,11 @@ void SerializationBool::deserializeTextJSON(IColumn &column, ReadBuffer &istr, c readBoolText(value, istr); else throw Exception("Invalid boolean value, should be true/false, 1/0.", - ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); + ErrorCodes::CANNOT_PARSE_BOOL); col->insert(value); } else - throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); + throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL); } void SerializationBool::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -119,10 +94,9 @@ void SerializationBool::serializeTextCSV(const IColumn & column, size_t row_num, void SerializationBool::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { if (istr.eof()) - throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); - String input; - readCSVString(input, istr, settings.csv); - deserializeFromString(column, input, settings); + throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL); + + deserializeWithCustom(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof() || *buf.position() == settings.csv.delimiter || *buf.position() == '\n'; }); } void SerializationBool::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -133,10 +107,9 @@ void SerializationBool::serializeTextRaw(const IColumn & column, size_t row_num, void SerializationBool::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { if (istr.eof()) - throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); - String input; - readString(input, istr); - deserializeFromString(column, input, settings); + throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL); + + deserializeWithCustom(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof() || *buf.position() == '\t' || *buf.position() == '\n'; }); } void SerializationBool::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -146,27 +119,22 @@ void SerializationBool::serializeTextQuoted(const IColumn & column, size_t row_n void SerializationBool::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { - if (!istr.eof()) - throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); + if (istr.eof()) + throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL); auto * col = checkAndGetDeserializeColumnType(column); - bool value = false; - - if (*istr.position() == 't' || *istr.position() == 'f' || *istr.position() == 'T' || *istr.position() == 'F') - readBoolTextWord(value, istr, true); - else if (*istr.position() == '1' || *istr.position() == '0') - readBoolText(value, istr); - else - throw Exception("Invalid boolean value, should be true/false, TRUE/FALSE, 1/0.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); - col->insert(value); + if (!deserializeImpl(col, istr)) + throw Exception( + ErrorCodes::CANNOT_PARSE_BOOL, + "Invalid boolean value, should be one of True/False/T/F/Y/N/Yes/No/On/Off/Enable/Disable/Enabled/Disabled/1/0"); } void SerializationBool::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - String input; - readStringUntilEOF(input, istr); - deserializeFromString(column, input, settings); - assert(istr.eof()); + if (istr.eof()) + throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL); + + deserializeWithCustom(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof(); }); } void SerializationBool::serializeCustom(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -198,35 +166,134 @@ void SerializationBool::serializeSimple(const IColumn & column, size_t row_num, ostr.write(str_false, sizeof(str_false) - 1); } -void SerializationBool::deserializeFromString(IColumn & column, String & input, const FormatSettings & settings) const +void SerializationBool::deserializeWithCustom( + IColumn & column, ReadBuffer & istr, const FormatSettings & settings, std::function check_end_of_value) const { ColumnUInt8 * col = checkAndGetDeserializeColumnType(column); - if (settings.bool_true_representation == input) + PeekableReadBuffer buf(istr); + buf.setCheckpoint(); + if (checkString(settings.bool_true_representation, buf) && check_end_of_value(buf)) { col->insert(true); + return; } - else if (settings.bool_false_representation == input) + + buf.rollbackToCheckpoint(); + if (checkString(settings.bool_false_representation, buf) && check_end_of_value(buf)) { col->insert(false); + buf.dropCheckpoint(); + if (buf.hasUnreadData()) + throw Exception( + ErrorCodes::CANNOT_PARSE_BOOL, + "Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if " + "bool_true_representation or bool_false_representation contains some delimiters of input format"); + return; + } + + buf.rollbackToCheckpoint(); + if (deserializeImpl(col, buf) && check_end_of_value(buf)) + { + buf.dropCheckpoint(); + if (buf.hasUnreadData()) + throw Exception( + ErrorCodes::CANNOT_PARSE_BOOL, + "Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if " + "bool_true_representation or bool_false_representation contains some delimiters of input format"); + return; + } + + throw Exception( + ErrorCodes::CANNOT_PARSE_BOOL, + "Invalid boolean value, should be '{}' or '{}' controlled by setting bool_true_representation and " + "bool_false_representation or one of " + "True/False/T/F/Y/N/Yes/No/On/Off/Enable/Disable/Enabled/Disabled/1/0", + settings.bool_true_representation, settings.bool_false_representation); +} + +bool SerializationBool::deserializeImpl(ColumnUInt8 * column, ReadBuffer & istr) const +{ + if (checkCharCaseInsensitive('1', istr)) + { + column->insert(true); + } + else if (checkCharCaseInsensitive('0', istr)) + { + column->insert(false); + } + /// 'True' and 'T' + else if (checkCharCaseInsensitive('t', istr)) + { + /// Check if it's just short form `T` or full form `True` + if (checkCharCaseInsensitive('r', istr)) + { + if (!checkStringCaseInsensitive("ue", istr)) + return false; + } + column->insert(true); + } + /// 'False' and 'F' + else if (checkCharCaseInsensitive('f', istr)) + { + /// Check if it's just short form `F` or full form `False` + if (checkCharCaseInsensitive('a', istr)) + { + if (!checkStringCaseInsensitive("lse", istr)) + return false; + } + column->insert(false); + } + /// 'Yes' and 'Y' + else if (checkCharCaseInsensitive('y', istr)) + { + /// Check if it's just short form `Y` or full form `Yes` + if (checkCharCaseInsensitive('e', istr)) + { + if (!checkCharCaseInsensitive('s', istr)) + return false; + } + column->insert(true); + } + /// 'No' and 'N' + else if (checkCharCaseInsensitive('n', istr)) + { + /// Check if it's just short form `N` or full form `No` + checkCharCaseInsensitive('o', istr); + column->insert(false); + } + /// 'On' and 'Off' + else if (checkCharCaseInsensitive('o', istr)) + { + if (checkCharCaseInsensitive('n', istr)) + column->insert(true); + else if (checkStringCaseInsensitive("ff", istr)) + { + column->insert(false); + } + else + return false; + } + /// 'Enable' and 'Enabled' + else if (checkStringCaseInsensitive("enable", istr)) + { + /// Check if it's 'enable' or 'enabled' + checkCharCaseInsensitive('d', istr); + column->insert(true); + } + /// 'Disable' and 'Disabled' + else if (checkStringCaseInsensitive("disable", istr)) + { + /// Check if it's 'disable' or 'disabled' + checkCharCaseInsensitive('d', istr); + column->insert(false); } else { - String input_lower = boost::algorithm::to_lower_copy(input); - if (true_values.contains(input_lower)) - { - col->insert(true); - } - else if (false_values.contains(input_lower)) - { - col->insert(false); - } - else - throw Exception( - "Invalid boolean value '" + input + "', should be " + settings.bool_true_representation + " or " + settings.bool_false_representation - + " controlled by setting bool_true_representation and bool_false_representation or one of " - "True/False/T/F/Y/N/Yes/No/On/Off", - ErrorCodes::ILLEGAL_COLUMN); + return false; } + + return true; } + } diff --git a/src/DataTypes/Serializations/SerializationBool.h b/src/DataTypes/Serializations/SerializationBool.h index dc7abbbb088..7999f5388d7 100644 --- a/src/DataTypes/Serializations/SerializationBool.h +++ b/src/DataTypes/Serializations/SerializationBool.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include namespace DB @@ -39,7 +40,8 @@ public: protected: void serializeCustom(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; void serializeSimple(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; - void deserializeFromString(IColumn & column, String & input, const FormatSettings & settings) const; + void deserializeWithCustom(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, std::function check_end_of_value) const; + bool deserializeImpl(ColumnUInt8 * column, ReadBuffer & istr) const; }; } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index e36111231f1..74ff4e002ba 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -182,6 +182,7 @@ struct ConvertImpl vec_null_map_to = &col_null_map_to->getData(); } + bool result_is_bool = isBool(result_type); for (size_t i = 0; i < input_rows_count; ++i) { if constexpr (std::is_same_v != std::is_same_v) @@ -271,7 +272,7 @@ struct ConvertImpl if constexpr (std::is_same_v) { - if (result_type->getName() == "Bool") + if (result_is_bool) vec_to[i] = static_cast(vec_to[i]); } } diff --git a/tests/queries/0_stateless/02152_bool_type_parsing.reference b/tests/queries/0_stateless/02152_bool_type_parsing.reference index b75df1faa65..f9fcd324dbc 100644 --- a/tests/queries/0_stateless/02152_bool_type_parsing.reference +++ b/tests/queries/0_stateless/02152_bool_type_parsing.reference @@ -26,6 +26,110 @@ false true false true +false +true +false +true +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +false +true +true +false +true +false true false true diff --git a/tests/queries/0_stateless/02152_bool_type_parsing.sh b/tests/queries/0_stateless/02152_bool_type_parsing.sh index 8a956bfc15b..65c173e700b 100755 --- a/tests/queries/0_stateless/02152_bool_type_parsing.sh +++ b/tests/queries/0_stateless/02152_bool_type_parsing.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -8,9 +9,17 @@ USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonex FILE_NAME=test_02152.data DATA_FILE=$USER_FILES_PATH/$FILE_NAME -echo -e "Custom true\nCustom false\nYes\nNo\nyes\nno\ny\nY\nN\nTrue\nFalse\ntrue\nfalse\nt\nf\nT\nF\nOn\nOff\non\noff" > $DATA_FILE +echo -e "Custom true\nCustom false\nYes\nNo\nyes\nno\ny\nY\nN\nTrue\nFalse\ntrue\nfalse\nt\nf\nT\nF\nOn\nOff\non\noff\nenable\ndisable\nenabled\ndisabled" > $DATA_FILE + $CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'TSV', 'bool Bool') settings bool_true_representation='Custom true', bool_false_representation='Custom false'" +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'TSV', 'bool Bool') settings bool_true_representation='Custom true', bool_false_representation='Custom false', input_format_parallel_parsing=0, max_read_buffer_size=2" + $CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'CSV', 'bool Bool') settings bool_true_representation='Custom true', bool_false_representation='Custom false'" +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'CSV', 'bool Bool') settings bool_true_representation='Custom true', bool_false_representation='Custom false', input_format_parallel_parsing=0, max_read_buffer_size=2" + +echo -e "Yes\nNo\nyes\nno\ny\nY\nN\nTrue\nFalse\ntrue\nfalse\nt\nf\nT\nF\nOn\nOff\non\noff\nenable\ndisable\nenabled\ndisabled" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'CustomSeparated', 'bool Bool') settings format_custom_escaping_rule='Quoted'" +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'CustomSeparated', 'bool Bool') settings format_custom_escaping_rule='Quoted', input_format_parallel_parsing=0, max_read_buffer_size=2" rm $DATA_FILE From 9333265c1a8a83970e813ccd03b7148af1dff833 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 23 Dec 2021 16:48:53 +0300 Subject: [PATCH 0276/1260] Minor change --- .../Serializations/SerializationBool.cpp | 29 +++++++++---------- 1 file changed, 13 insertions(+), 16 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp index e663bc87ea3..0542e050a25 100644 --- a/src/DataTypes/Serializations/SerializationBool.cpp +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -67,23 +67,20 @@ void SerializationBool::serializeTextJSON(const IColumn &column, size_t row_num, void SerializationBool::deserializeTextJSON(IColumn &column, ReadBuffer &istr, const FormatSettings &) const { - ColumnUInt8 * col = checkAndGetDeserializeColumnType(column); - - if (!istr.eof()) - { - bool value = false; - - if (*istr.position() == 't' || *istr.position() == 'f') - readBoolTextWord(value, istr); - else if (*istr.position() == '1' || *istr.position() == '0') - readBoolText(value, istr); - else - throw Exception("Invalid boolean value, should be true/false, 1/0.", - ErrorCodes::CANNOT_PARSE_BOOL); - col->insert(value); - } - else + if (istr.eof()) throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL); + + ColumnUInt8 * col = checkAndGetDeserializeColumnType(column); + bool value = false; + + if (*istr.position() == 't' || *istr.position() == 'f') + readBoolTextWord(value, istr); + else if (*istr.position() == '1' || *istr.position() == '0') + readBoolText(value, istr); + else + throw Exception("Invalid boolean value, should be true/false, 1/0.", + ErrorCodes::CANNOT_PARSE_BOOL); + col->insert(value); } void SerializationBool::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const From 2e02a6bccab717f8779fa16b79ad70a04ba513d8 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 23 Dec 2021 17:49:29 +0300 Subject: [PATCH 0277/1260] Try to fix build --- src/Functions/FunctionsConversion.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 74ff4e002ba..a908a74dad3 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2638,7 +2638,7 @@ private: { if (checkAndGetDataType(from_type.get())) { - return [] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t /*input_rows_count*/) + return [](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t /*input_rows_count*/) -> ColumnPtr { return ConvertImplGenericFromString::execute(arguments, result_type); }; From 85d214d5604c00dd88a4183a8f8534be08f181b7 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 23 Dec 2021 20:14:54 +0300 Subject: [PATCH 0278/1260] Fix build --- src/Functions/FunctionsConversion.h | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index a908a74dad3..8018fa8e726 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2638,10 +2638,7 @@ private: { if (checkAndGetDataType(from_type.get())) { - return [](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t /*input_rows_count*/) -> ColumnPtr - { - return ConvertImplGenericFromString::execute(arguments, result_type); - }; + return &ConvertImplGenericFromString::execute; } return createWrapper(from_type, to_type, requested_result_is_nullable); From 47ad4200c9d1b1a1d17c5dbdf2cfd2fbcefe8016 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 23 Dec 2021 21:00:22 +0300 Subject: [PATCH 0279/1260] Fix test --- tests/queries/0_stateless/02114_bool_type.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02114_bool_type.sql b/tests/queries/0_stateless/02114_bool_type.sql index 4542cc68a3a..d4ea4e54028 100644 --- a/tests/queries/0_stateless/02114_bool_type.sql +++ b/tests/queries/0_stateless/02114_bool_type.sql @@ -5,7 +5,7 @@ CREATE TABLE bool_test (value Bool,f String) ENGINE = Memory; -- value column shoud have type 'Bool' SHOW CREATE TABLE bool_test; -INSERT INTO bool_test (value,f) VALUES ('false', 'test'), ('true' , 'test'), (0, 'test'), (1, 'test'), ('FALSE', 'test'), ('TRUE', 'test'); +INSERT INTO bool_test (value,f) VALUES (false, 'test'), (true , 'test'), (0, 'test'), (1, 'test'), (FALSE, 'test'), (TRUE, 'test'); INSERT INTO bool_test (value,f) FORMAT JSONEachRow {"value":false,"f":"test"}{"value":true,"f":"test"}{"value":0,"f":"test"}{"value":1,"f":"test"} SELECT value,f FROM bool_test; From 26dc1bc4e1b7f94c8b6da2b46f1162b5a8d32b7a Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 23 Dec 2021 21:03:26 +0300 Subject: [PATCH 0280/1260] Fix style --- src/DataTypes/Serializations/SerializationBool.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp index 0542e050a25..d157ce66826 100644 --- a/src/DataTypes/Serializations/SerializationBool.cpp +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -225,7 +225,7 @@ bool SerializationBool::deserializeImpl(ColumnUInt8 * column, ReadBuffer & istr) /// Check if it's just short form `T` or full form `True` if (checkCharCaseInsensitive('r', istr)) { - if (!checkStringCaseInsensitive("ue", istr)) + if (!checkStringCaseInsensitive("ue", istr)) // NOLINT return false; } column->insert(true); From e38cb0389c2c835fe7e40ab2d3b3af0f804512e9 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 23 Dec 2021 21:22:51 +0300 Subject: [PATCH 0281/1260] Remove NOLINT --- src/DataTypes/Serializations/SerializationBool.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp index d157ce66826..0542e050a25 100644 --- a/src/DataTypes/Serializations/SerializationBool.cpp +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -225,7 +225,7 @@ bool SerializationBool::deserializeImpl(ColumnUInt8 * column, ReadBuffer & istr) /// Check if it's just short form `T` or full form `True` if (checkCharCaseInsensitive('r', istr)) { - if (!checkStringCaseInsensitive("ue", istr)) // NOLINT + if (!checkStringCaseInsensitive("ue", istr)) return false; } column->insert(true); From c6f66306272a43ef69a29759f0bcda3d1eae7481 Mon Sep 17 00:00:00 2001 From: karnevil13 Date: Fri, 24 Dec 2021 02:28:39 +0300 Subject: [PATCH 0282/1260] insert_quorum_parallel-EdTranRus --- .../table-engines/integrations/hdfs.md | 38 ++++++++++--------- .../table-engines/integrations/kafka.md | 2 +- docs/en/operations/settings/settings.md | 32 +++++++++++++--- .../table-engines/integrations/hdfs.md | 34 +++++++++-------- .../table-engines/integrations/kafka.md | 2 +- docs/ru/operations/settings/settings.md | 32 +++++++++++++--- 6 files changed, 94 insertions(+), 46 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index 0fcf7a63dd8..95f1a029d13 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -5,8 +5,7 @@ toc_title: HDFS # HDFS {#table_engines-hdfs} -This engine provides integration with [Apache Hadoop](https://en.wikipedia.org/wiki/Apache_Hadoop) ecosystem by allowing to manage data on [HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html) via ClickHouse. This engine is similar -to the [File](../../../engines/table-engines/special/file.md#table_engines-file) and [URL](../../../engines/table-engines/special/url.md#table_engines-url) engines, but provides Hadoop-specific features. +This engine provides integration with [Apache Hadoop](https://en.wikipedia.org/wiki/Apache_Hadoop) ecosystem by allowing to manage data on [HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html) via ClickHouse. This engine is similar to the [File](../../../engines/table-engines/special/file.md#table_engines-file) and [URL](../../../engines/table-engines/special/url.md#table_engines-url) engines, but provides Hadoop-specific features. ## Usage {#usage} @@ -14,12 +13,13 @@ to the [File](../../../engines/table-engines/special/file.md#table_engines-file) ENGINE = HDFS(URI, format) ``` -The `URI` parameter is the whole file URI in HDFS. -The `format` parameter specifies one of the available file formats. To perform +**Engine Parameters** + +- `URI` - whole file URI in HDFS. The path part of `URI` may contain globs. In this case the table would be readonly. +- `format` - specifies one of the available file formats. To perform `SELECT` queries, the format must be supported for input, and to perform `INSERT` queries – for output. The available formats are listed in the [Formats](../../../interfaces/formats.md#formats) section. -The path part of `URI` may contain globs. In this case the table would be readonly. **Example:** @@ -71,12 +71,12 @@ Constructions with `{}` are similar to the [remote](../../../sql-reference/table 1. Suppose we have several files in TSV format with the following URIs on HDFS: -- 'hdfs://hdfs1:9000/some_dir/some_file_1' -- 'hdfs://hdfs1:9000/some_dir/some_file_2' -- 'hdfs://hdfs1:9000/some_dir/some_file_3' -- 'hdfs://hdfs1:9000/another_dir/some_file_1' -- 'hdfs://hdfs1:9000/another_dir/some_file_2' -- 'hdfs://hdfs1:9000/another_dir/some_file_3' + - 'hdfs://hdfs1:9000/some_dir/some_file_1' + - 'hdfs://hdfs1:9000/some_dir/some_file_2' + - 'hdfs://hdfs1:9000/some_dir/some_file_3' + - 'hdfs://hdfs1:9000/another_dir/some_file_1' + - 'hdfs://hdfs1:9000/another_dir/some_file_2' + - 'hdfs://hdfs1:9000/another_dir/some_file_3' 1. There are several ways to make a table consisting of all six files: @@ -132,6 +132,7 @@ Similar to GraphiteMergeTree, the HDFS engine supports extended configuration us | **parameter** | **default value** | +| - | - | | rpc\_client\_connect\_tcpnodelay | true | | dfs\_client\_read\_shortcircuit | true | | output\_replace-datanode-on-failure | true | @@ -181,25 +182,26 @@ Similar to GraphiteMergeTree, the HDFS engine supports extended configuration us #### ClickHouse extras {#clickhouse-extras} | **parameter** | **default value** | +| - | - | |hadoop\_kerberos\_keytab | "" | |hadoop\_kerberos\_principal | "" | |hadoop\_kerberos\_kinit\_command | kinit | |libhdfs3\_conf | "" | ### Limitations {#limitations} - * hadoop\_security\_kerberos\_ticket\_cache\_path and libhdfs3\_conf can be global only, not user specific +* `hadoop_security_kerberos_ticket_cache_path` and `libhdfs3_conf` can be global only, not user specific ## Kerberos support {#kerberos-support} -If hadoop\_security\_authentication parameter has value 'kerberos', ClickHouse authentifies via Kerberos facility. -Parameters [here](#clickhouse-extras) and hadoop\_security\_kerberos\_ticket\_cache\_path may be of help. +If the `hadoop_security_authentication` parameter has the value `kerberos`, ClickHouse authenticates via Kerberos. +Parameters are [here](#clickhouse-extras) and `hadoop_security_kerberos_ticket_cache_path` may be of help. Note that due to libhdfs3 limitations only old-fashioned approach is supported, -datanode communications are not secured by SASL (HADOOP\_SECURE\_DN\_USER is a reliable indicator of such -security approach). Use tests/integration/test\_storage\_kerberized\_hdfs/hdfs_configs/bootstrap.sh for reference. +datanode communications are not secured by SASL (`HADOOP_SECURE_DN_USER` is a reliable indicator of such +security approach). Use `tests/integration/test_storage_kerberized_hdfs/hdfs_configs/bootstrap.sh` for reference. -If hadoop\_kerberos\_keytab, hadoop\_kerberos\_principal or hadoop\_kerberos\_kinit\_command is specified, kinit will be invoked. hadoop\_kerberos\_keytab and hadoop\_kerberos\_principal are mandatory in this case. kinit tool and krb5 configuration files are required. +If `hadoop_kerberos_keytab`, `hadoop_kerberos_principal` or `hadoop_kerberos_kinit_command` is specified, `kinit` will be invoked. `hadoop_kerberos_keytab` and `hadoop_kerberos_principal` are mandatory in this case. `kinit` tool and krb5 configuration files are required. -## HDFS Namenode HA support{#namenode-ha} +## HDFS Namenode HA support {#namenode-ha} libhdfs3 support HDFS namenode HA. diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index 879e919e823..b39de0a5958 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -192,6 +192,6 @@ Example: **See Also** - [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns) -- [background_schedule_pool_size](../../../operations/settings/settings.md#background_schedule_pool_size) +- [background_message_broker_schedule_pool_size](../../../operations/settings/settings.md#background_message_broker_schedule_pool_size) [Original article](https://clickhouse.com/docs/en/engines/table-engines/integrations/kafka/) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f78fbc8a2bc..85d8cefe49a 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1581,18 +1581,17 @@ Quorum writes `INSERT` succeeds only when ClickHouse manages to correctly write data to the `insert_quorum` of replicas during the `insert_quorum_timeout`. If for any reason the number of replicas with successful writes does not reach the `insert_quorum`, the write is considered failed and ClickHouse will delete the inserted block from all the replicas where data has already been written. -All the replicas in the quorum are consistent, i.e., they contain data from all previous `INSERT` queries. The `INSERT` sequence is linearized. +When `insert_quorum_parallel` is disabled, all replicas in the quorum are consistent, i.e. they contain data from all previous `INSERT` queries (the `INSERT` sequence is linearized). When reading data written using `insert_quorum` and `insert_quorum_parallel` is disabled, you can turn on sequential consistency for `SELECT` queries using [select_sequential_consistency](#settings-select_sequential_consistency). -When reading the data written from the `insert_quorum`, you can use the [select_sequential_consistency](#settings-select_sequential_consistency) option. - -ClickHouse generates an exception +ClickHouse generates an exception: - If the number of available replicas at the time of the query is less than the `insert_quorum`. -- At an attempt to write data when the previous block has not yet been inserted in the `insert_quorum` of replicas. This situation may occur if the user tries to perform an `INSERT` before the previous one with the `insert_quorum` is completed. +- When `insert_quorum_parallel` is disabled and an attempt to write data is made when the previous block has not yet been inserted in `insert_quorum` of replicas. This situation may occur if the user tries to perform another `INSERT` query to the same table before the previous one with `insert_quorum` is completed. See also: - [insert_quorum_timeout](#settings-insert_quorum_timeout) +- [insert_quorum_parallel](#settings-insert_quorum_parallel) - [select_sequential_consistency](#settings-select_sequential_consistency) ## insert_quorum_timeout {#settings-insert_quorum_timeout} @@ -1604,11 +1603,29 @@ Default value: 600 000 milliseconds (ten minutes). See also: - [insert_quorum](#settings-insert_quorum) +- [insert_quorum_parallel](#settings-insert_quorum_parallel) +- [select_sequential_consistency](#settings-select_sequential_consistency) + +## insert_quorum_parallel {#settings-insert_quorum_parallel} + +Enables or disables parallelism for quorum `INSERT` queries. If enabled, additional `INSERT` queries can be sent while previous queries have not yet finished. If disabled, additional writes to the same table will be rejected. + +Possible values: + +- 0 — Disabled. +- 1 — Enabled. + +Default value: 1. + +See also: + +- [insert_quorum](#settings-insert_quorum) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) - [select_sequential_consistency](#settings-select_sequential_consistency) ## select_sequential_consistency {#settings-select_sequential_consistency} -Enables or disables sequential consistency for `SELECT` queries: +Enables or disables sequential consistency for `SELECT` queries. Requires `insert_quorum_parallel` to be disabled (enabled by default). Possible values: @@ -1621,10 +1638,13 @@ Usage When sequential consistency is enabled, ClickHouse allows the client to execute the `SELECT` query only for those replicas that contain data from all previous `INSERT` queries executed with `insert_quorum`. If the client refers to a partial replica, ClickHouse will generate an exception. The SELECT query will not include data that has not yet been written to the quorum of replicas. +When `insert_quorum_parallel` is enabled (the default), then `select_sequential_consistency` does not work. This is because parallel `INSERT` queries can be written to different sets of quorum replicas so there is no guarantee a single replica will have received all writes. + See also: - [insert_quorum](#settings-insert_quorum) - [insert_quorum_timeout](#settings-insert_quorum_timeout) +- [insert_quorum_parallel](#settings-insert_quorum_parallel) ## insert_deduplicate {#settings-insert-deduplicate} diff --git a/docs/ru/engines/table-engines/integrations/hdfs.md b/docs/ru/engines/table-engines/integrations/hdfs.md index 5949cc8a0d7..78a82955cd2 100644 --- a/docs/ru/engines/table-engines/integrations/hdfs.md +++ b/docs/ru/engines/table-engines/integrations/hdfs.md @@ -5,7 +5,7 @@ toc_title: HDFS # HDFS {#table_engines-hdfs} -Управляет данными в HDFS. Данный движок похож на движки [File](../special/file.md#table_engines-file) и [URL](../special/url.md#table_engines-url). +Этот движок обеспечивает интеграцию с экосистемой [Apache Hadoop](https://ru.wikipedia.org/wiki/Hadoop), позволяя управлять данными в HDFS посредством ClickHouse. Данный движок похож на движки [File](../special/file.md#table_engines-file) и [URL](../special/url.md#table_engines-url), но предоставляет возможности, характерные для Hadoop. ## Использование движка {#usage} @@ -13,9 +13,11 @@ toc_title: HDFS ENGINE = HDFS(URI, format) ``` -В параметр `URI` нужно передавать полный URI файла в HDFS. +**Параметры движка** + +В параметр `URI` нужно передавать полный URI файла в HDFS. Часть URI с путем файла может содержать шаблоны. В этом случае таблица может использоваться только для чтения. Параметр `format` должен быть таким, который ClickHouse может использовать и в запросах `INSERT`, и в запросах `SELECT`. Полный список поддерживаемых форматов смотрите в разделе [Форматы](../../../interfaces/formats.md#formats). -Часть URI с путем файла может содержать шаблоны. В этом случае таблица может использоваться только для чтения. + **Пример:** @@ -67,12 +69,12 @@ SELECT * FROM hdfs_engine_table LIMIT 2 1. Предположим, у нас есть несколько файлов со следующими URI в HDFS: -- 'hdfs://hdfs1:9000/some_dir/some_file_1' -- 'hdfs://hdfs1:9000/some_dir/some_file_2' -- 'hdfs://hdfs1:9000/some_dir/some_file_3' -- 'hdfs://hdfs1:9000/another_dir/some_file_1' -- 'hdfs://hdfs1:9000/another_dir/some_file_2' -- 'hdfs://hdfs1:9000/another_dir/some_file_3' + - 'hdfs://hdfs1:9000/some_dir/some_file_1' + - 'hdfs://hdfs1:9000/some_dir/some_file_2' + - 'hdfs://hdfs1:9000/some_dir/some_file_3' + - 'hdfs://hdfs1:9000/another_dir/some_file_1' + - 'hdfs://hdfs1:9000/another_dir/some_file_2' + - 'hdfs://hdfs1:9000/another_dir/some_file_3' 1. Есть несколько возможностей создать таблицу, состояющую из этих шести файлов: @@ -128,6 +130,7 @@ CREATE TABLE big_table (name String, value UInt32) ENGINE = HDFS('hdfs://hdfs1:9 | **параметр** | **по умолчанию** | +| - | - | | rpc\_client\_connect\_tcpnodelay | true | | dfs\_client\_read\_shortcircuit | true | | output\_replace-datanode-on-failure | true | @@ -177,22 +180,23 @@ CREATE TABLE big_table (name String, value UInt32) ENGINE = HDFS('hdfs://hdfs1:9 #### Расширенные параметры для ClickHouse {#clickhouse-extras} | **параметр** | **по умолчанию** | +| - | - | |hadoop\_kerberos\_keytab | "" | |hadoop\_kerberos\_principal | "" | |hadoop\_kerberos\_kinit\_command | kinit | ### Ограничения {#limitations} - * hadoop\_security\_kerberos\_ticket\_cache\_path могут быть определены только на глобальном уровне + * `hadoop_security_kerberos_ticket_cache_path` и `libhdfs3_conf` могут быть определены только на глобальном, а не на пользовательском уровне ## Поддержка Kerberos {#kerberos-support} -Если hadoop\_security\_authentication параметр имеет значение 'kerberos', ClickHouse аутентифицируется с помощью Kerberos. -[Расширенные параметры](#clickhouse-extras) и hadoop\_security\_kerberos\_ticket\_cache\_path помогают сделать это. +Если параметр `hadoop_security_authentication` имеет значение `kerberos`, ClickHouse аутентифицируется с помощью Kerberos. +[Расширенные параметры](#clickhouse-extras) и `hadoop_security_kerberos_ticket_cache_path` помогают сделать это. Обратите внимание что из-за ограничений libhdfs3 поддерживается только устаревший метод аутентификации, -коммуникация с узлами данных не защищена SASL (HADOOP\_SECURE\_DN\_USER надежный показатель такого -подхода к безопасности). Используйте tests/integration/test\_storage\_kerberized\_hdfs/hdfs_configs/bootstrap.sh для примера настроек. +коммуникация с узлами данных не защищена SASL (`HADOOP_SECURE_DN_USER` надежный показатель такого +подхода к безопасности). Используйте `tests/integration/test_storage_kerberized_hdfs/hdfs_configs/bootstrap.sh` для примера настроек. -Если hadoop\_kerberos\_keytab, hadoop\_kerberos\_principal или hadoop\_kerberos\_kinit\_command указаны в настройках, kinit будет вызван. hadoop\_kerberos\_keytab и hadoop\_kerberos\_principal обязательны в этом случае. Необходимо также будет установить kinit и файлы конфигурации krb5. +Если `hadoop_kerberos_keytab`, `hadoop_kerberos_principal` или `hadoop_kerberos_kinit_command` указаны в настройках, `kinit` будет вызван. `hadoop_kerberos_keytab` и `hadoop_kerberos_principal` обязательны в этом случае. Необходимо также будет установить `kinit` и файлы конфигурации krb5. ## Виртуальные столбцы {#virtual-columns} diff --git a/docs/ru/engines/table-engines/integrations/kafka.md b/docs/ru/engines/table-engines/integrations/kafka.md index 19e2850dd51..7ea3d124ab3 100644 --- a/docs/ru/engines/table-engines/integrations/kafka.md +++ b/docs/ru/engines/table-engines/integrations/kafka.md @@ -191,5 +191,5 @@ ClickHouse может поддерживать учетные данные Kerbe **Смотрите также** - [Виртуальные столбцы](index.md#table_engines-virtual_columns) -- [background_schedule_pool_size](../../../operations/settings/settings.md#background_schedule_pool_size) +- [background_message_broker_schedule_pool_size](../../../operations/settings/settings.md#background_message_broker_schedule_pool_size) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 500485aea2f..32bd96cf063 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1544,18 +1544,19 @@ INSERT INTO table_with_enum_column_for_csv_insert FORMAT CSV 102,2; `INSERT` завершается успешно только в том случае, когда ClickHouse смог без ошибки записать данные в `insert_quorum` реплик за время `insert_quorum_timeout`. Если по любой причине количество реплик с успешной записью не достигнет `insert_quorum`, то запись считается не состоявшейся и ClickHouse удалит вставленный блок из всех реплик, куда уже успел записать данные. -Все реплики в кворуме консистентны, т.е. содержат данные всех более ранних запросов `INSERT`. Последовательность `INSERT` линеаризуется. +Когда `insert_quorum_parallel` выключена, все реплики кворума консистентны, то есть содержат данные всех предыдущих запросов `INSERT` (последовательность `INSERT` линеаризуется). При чтении с диска данных, записанных с помощью `insert_quorum` и при выключенной `insert_quorum_parallel`, можно включить последовательную консистентность для запросов `SELECT` с помощью [select_sequential_consistency](#settings-select_sequential_consistency). -При чтении данных, записанных с `insert_quorum` можно использовать настройку [select_sequential_consistency](#settings-select_sequential_consistency). - -ClickHouse генерирует исключение +ClickHouse генерирует исключение: - Если количество доступных реплик на момент запроса меньше `insert_quorum`. - При попытке записать данные в момент, когда предыдущий блок ещё не вставлен в `insert_quorum` реплик. Эта ситуация может возникнуть, если пользователь вызвал `INSERT` прежде, чем завершился предыдущий с `insert_quorum`. +- При выключенной `insert_quorum_parallel` и при попытке записать данные в момент, когда предыдущий блок еще не вставлен в `insert_quorum` реплик. Эта ситуация может возникнуть при попытке пользователя выполнить очередной запрос `INSERT` к той же таблице, прежде чем завершится предыдущий с `insert_quorum`. + См. также: - [insert_quorum_timeout](#settings-insert_quorum_timeout) +- [insert_quorum_parallel](#settings-insert_quorum_parallel) - [select_sequential_consistency](#settings-select_sequential_consistency) ## insert_quorum_timeout {#settings-insert_quorum_timeout} @@ -1567,11 +1568,29 @@ ClickHouse генерирует исключение См. также: - [insert_quorum](#settings-insert_quorum) +- [insert_quorum_parallel](#settings-insert_quorum_parallel) +- [select_sequential_consistency](#settings-select_sequential_consistency) + +## insert_quorum_parallel {#settings-insert_quorum_parallel} + +Включает и выключает параллелизм для кворума запросов `INSERT`. Когда включена, можно выполнить дополнительные запросы `INSERT` в то время, пока предыдущие запросы еще не завершены. Когда выключена, допольнительные записи в ту же таблицу будут отклонены. + +Возможные значения: + +- 0 — Выключена. +- 1 — Включена. + +Значение по умолчанию: 1. + +См. также: + +- [insert_quorum](#settings-insert_quorum) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) - [select_sequential_consistency](#settings-select_sequential_consistency) ## select_sequential_consistency {#settings-select_sequential_consistency} -Включает или выключает последовательную консистентность для запросов `SELECT`. +Включает или выключает последовательную консистентность для запросов `SELECT`. Необходимо, чтобы `insert_quorum_parallel` была выключена (по умолчанию включена). Возможные значения: @@ -1584,10 +1603,13 @@ ClickHouse генерирует исключение Когда последовательная консистентность включена, то ClickHouse позволит клиенту выполнить запрос `SELECT` только к тем репликам, которые содержат данные всех предыдущих запросов `INSERT`, выполненных с `insert_quorum`. Если клиент обратится к неполной реплике, то ClickHouse сгенерирует исключение. В запросе SELECT не будут участвовать данные, которые ещё не были записаны на кворум реплик. +Если `insert_quorum_parallel` включена (по умолчанию это так), тогда `select_sequential_consistency` не будет работать. Причина в том, что параллельные запросы `INSERT` можно записать в разные наборы реплик кворума, поэтому нет гарантии того, что в отдельно взятую реплику будут сделаны все записи. + См. также: - [insert_quorum](#settings-insert_quorum) - [insert_quorum_timeout](#settings-insert_quorum_timeout) +- [insert_quorum_parallel](#settings-insert_quorum_parallel) ## insert_deduplicate {#settings-insert-deduplicate} From dea2543177fd28480f9eb0a6bf29506137e40f47 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 24 Dec 2021 03:26:31 +0300 Subject: [PATCH 0283/1260] Update Field.h --- src/Core/Field.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Field.h b/src/Core/Field.h index 7c09ec71749..19573ed9831 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -823,7 +823,7 @@ inline char & Field::reinterpret() { // For String we want to return a pointer to the data, not the start of the class // as the layout of std::string depends on the STD version and options - char * MAY_ALIAS ptr = reinterpret_cast(&storage)->data(); + char * ptr = reinterpret_cast(&storage)->data(); return *ptr; } return *reinterpret_cast(&storage); From df8689ce22328d9fa850c27fb9e76ca89577f01c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Dec 2021 03:57:50 +0300 Subject: [PATCH 0284/1260] Fix test that was dependent on time zone --- tests/queries/0_stateless/01821_to_date_time_ubsan.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01821_to_date_time_ubsan.sql b/tests/queries/0_stateless/01821_to_date_time_ubsan.sql index 843f271d22b..377291e015f 100644 --- a/tests/queries/0_stateless/01821_to_date_time_ubsan.sql +++ b/tests/queries/0_stateless/01821_to_date_time_ubsan.sql @@ -1,2 +1,2 @@ -SELECT toDateTime('9223372036854775806', 7); -SELECT toDateTime('9223372036854775806', 8); +SELECT toDateTime('9223372036854775806', 7, 'Europe/Moscow'); +SELECT toDateTime('9223372036854775806', 8, 'Europe/Moscow'); From 26e10c1fbb3f578836e09ae9267c6389ecb16be9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Dec 2021 04:02:26 +0300 Subject: [PATCH 0285/1260] Remove unused header --- src/Interpreters/FillingRow.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index df99c0d11ed..94f185a44cc 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -1,5 +1,4 @@ #include -#include #include From 004180c92f8be2bd44367c1fcb597b3a8a6c67df Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 24 Dec 2021 10:27:44 +0800 Subject: [PATCH 0286/1260] fix sytle --- programs/server/Server.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 560ecb5d249..3943071b8d1 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -128,7 +128,7 @@ namespace CurrentMetrics extern const Metric MaxPushedDDLEntryID; } -namespace GitInfo +namespace GitInfo { extern const std::string GIT_SHA1; extern const std::string GIT_BRANCH; From 82465fd22e8589119b4fb8fa9022795bda86a0f0 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 24 Dec 2021 10:49:12 +0800 Subject: [PATCH 0287/1260] fix style --- src/CMakeLists.txt | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 2d097e47ef9..11932d75fdb 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -29,25 +29,25 @@ configure_file (Common/config_version.h.in ${CONFIG_VERSION}) configure_file (Core/config_core.h.in "${CMAKE_CURRENT_BINARY_DIR}/Core/include/config_core.h") set(GIT_EXECUTABLE /usr/bin/git) -# the commit's SHA1, and whether the building workspace was dirty or not +# The commit's SHA1, and whether the building workspace was dirty or not execute_process(COMMAND "${GIT_EXECUTABLE}" rev-parse HEAD WORKING_DIRECTORY "${CMAKE_SOURCE_DIR}" OUTPUT_VARIABLE GIT_SHA1 ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) -# git branch name +# Git branch name execute_process(COMMAND "${GIT_EXECUTABLE}" rev-parse --abbrev-ref HEAD WORKING_DIRECTORY "${CMAKE_SOURCE_DIR}" OUTPUT_VARIABLE GIT_BRANCH ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) -# the date of the commit +# The date of the commit execute_process(COMMAND "${GIT_EXECUTABLE}" log -1 --format=%ad --date=local WORKING_DIRECTORY "${CMAKE_SOURCE_DIR}" OUTPUT_VARIABLE GIT_DATE ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) -# the subject of the commit +# The subject of the commit execute_process(COMMAND "${GIT_EXECUTABLE}" log -1 --format=%s WORKING_DIRECTORY "${CMAKE_SOURCE_DIR}" From 80c5bfcbcb215cd5f576f4471d815446dd97eca7 Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 24 Dec 2021 05:08:54 +0000 Subject: [PATCH 0288/1260] fix fix --- src/Core/DecimalComparison.h | 193 +++++------------- src/Core/DecimalFloatComparison.h | 58 ------ src/Functions/FunctionsComparison.h | 29 ++- src/Functions/IsOperation.h | 4 - .../00700_decimal_compare.reference | 2 +- ...4_comparison_betwwen_decimal_and_float.sql | 2 +- 6 files changed, 71 insertions(+), 217 deletions(-) delete mode 100644 src/Core/DecimalFloatComparison.h diff --git a/src/Core/DecimalComparison.h b/src/Core/DecimalComparison.h index 62d7d72a924..b8498b6c84b 100644 --- a/src/Core/DecimalComparison.h +++ b/src/Core/DecimalComparison.h @@ -1,21 +1,17 @@ #pragma once -#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include -#include -#include /// TODO Core should not depend on Functions -#include -#include +#include +#include /// TODO Core should not depend on Functions -#include - namespace DB { @@ -56,14 +52,8 @@ struct DecCompareInt using TypeB = Type; }; -template < - typename A, - typename B, - template - typename Operation, - bool _check_overflow = true, - bool _actual = is_decimal || is_decimal, - bool _has_float = std::is_floating_point_v || std::is_floating_point_v> +template typename Operation, bool _check_overflow = true, + bool _actual = is_decimal || is_decimal> class DecimalComparison { public: @@ -230,135 +220,48 @@ private: template static NO_INLINE UInt8 apply(A a, B b, CompareInt scale [[maybe_unused]]) { - /// Decimal compares with Float - if constexpr (_has_float) - { - if constexpr (IsOperation::equals) - { - if constexpr (std::is_floating_point_v && is_decimal) - { - CompareInt decimal_value = b.value; - return DecimalFloatComparison::equals(a, decimal_value, scale); - } - if constexpr (std::is_floating_point_v && is_decimal) - { - CompareInt decimal_value = a.value; - return DecimalFloatComparison::equals(b, decimal_value, scale); - } - } - if constexpr (IsOperation::not_equals) - { - if constexpr (std::is_floating_point_v && is_decimal) - { - CompareInt decimal_value = b.value; - return DecimalFloatComparison::notEquals(a, decimal_value, scale); - } - if constexpr (std::is_floating_point_v && is_decimal) - { - CompareInt decimal_value = a.value; - return DecimalFloatComparison::notEquals(b, decimal_value, scale); - } - } - if constexpr (IsOperation::less) - { - if constexpr (std::is_floating_point_v && is_decimal) - { - CompareInt decimal_value = b.value; - return DecimalFloatComparison::less(a, decimal_value, scale); - } - if constexpr (std::is_floating_point_v && is_decimal) - { - CompareInt decimal_value = a.value; - return DecimalFloatComparison::greater(b, decimal_value, scale); - } - } - if constexpr (IsOperation::less_or_equals) - { - if constexpr (std::is_floating_point_v && is_decimal) - { - CompareInt decimal_value = b.value; - return DecimalFloatComparison::lessOrEquals(a, decimal_value, scale); - } - if constexpr (std::is_floating_point_v && is_decimal) - { - CompareInt decimal_value = a.value; - return DecimalFloatComparison::greaterOrEquals(b, decimal_value, scale); - } - } - if constexpr (IsOperation::greater) - { - if constexpr (std::is_floating_point_v && is_decimal) - { - CompareInt decimal_value = b.value; - return DecimalFloatComparison::greater(a, decimal_value, scale); - } - if constexpr (std::is_floating_point_v && is_decimal) - { - CompareInt decimal_value = a.value; - return DecimalFloatComparison::less(b, decimal_value, scale); - } - } - if constexpr (IsOperation::greater_or_equals) - { - if constexpr (std::is_floating_point_v && is_decimal) - { - CompareInt decimal_value = b.value; - return DecimalFloatComparison::greaterOrEquals(a, decimal_value, scale); - } - if constexpr (std::is_floating_point_v && is_decimal) - { - CompareInt decimal_value = a.value; - return DecimalFloatComparison::lessOrEquals(b, decimal_value, scale); - } - } - } + CompareInt x; + if constexpr (is_decimal) + x = a.value; + else + x = a; - /// Decimal compares with Int + CompareInt y; + if constexpr (is_decimal) + y = b.value; + else + y = b; + + if constexpr (_check_overflow) + { + bool overflow = false; + + if constexpr (sizeof(A) > sizeof(CompareInt)) + overflow |= (static_cast(x) != a); + if constexpr (sizeof(B) > sizeof(CompareInt)) + overflow |= (static_cast(y) != b); + if constexpr (is_unsigned_v) + overflow |= (x < 0); + if constexpr (is_unsigned_v) + overflow |= (y < 0); + + if constexpr (scale_left) + overflow |= common::mulOverflow(x, scale, x); + if constexpr (scale_right) + overflow |= common::mulOverflow(y, scale, y); + + if (overflow) + throw Exception("Can't compare decimal number due to overflow", ErrorCodes::DECIMAL_OVERFLOW); + } else { - CompareInt x; - if constexpr (is_decimal) - x = a.value; - else - x = a; - - CompareInt y; - if constexpr (is_decimal) - y = b.value; - else - y = b; - - if constexpr (_check_overflow) - { - bool overflow = false; - - if constexpr (sizeof(A) > sizeof(CompareInt)) - overflow |= (static_cast(x) != a); - if constexpr (sizeof(B) > sizeof(CompareInt)) - overflow |= (static_cast(y) != b); - if constexpr (is_unsigned_v) - overflow |= (x < 0); - if constexpr (is_unsigned_v) - overflow |= (y < 0); - - if constexpr (scale_left) - overflow |= common::mulOverflow(x, scale, x); - if constexpr (scale_right) - overflow |= common::mulOverflow(y, scale, y); - - if (overflow) - throw Exception("Can't compare decimal number due to overflow", ErrorCodes::DECIMAL_OVERFLOW); - } - else - { - if constexpr (scale_left) - x = common::mulIgnoreOverflow(x, scale); - if constexpr (scale_right) - y = common::mulIgnoreOverflow(y, scale); - } - - return Op::apply(x, y); + if constexpr (scale_left) + x = common::mulIgnoreOverflow(x, scale); + if constexpr (scale_right) + y = common::mulIgnoreOverflow(y, scale); } + + return Op::apply(x, y); } template diff --git a/src/Core/DecimalFloatComparison.h b/src/Core/DecimalFloatComparison.h deleted file mode 100644 index 8524478e312..00000000000 --- a/src/Core/DecimalFloatComparison.h +++ /dev/null @@ -1,58 +0,0 @@ -#pragma once - -#include - -namespace DB -{ -struct DecimalFloatComparison -{ - template - static int compare(Float a, Int b, Int scale) - { - /// TODO need to implement comparison - if (a) - return -1; - if (b) - return 0; - if (scale) - return 1; - return 0; - } - - template - static bool equals(Float a, Int b, Int scale) - { - return compare(a, b, scale) == 0; - } - - template - static bool notEquals(Float a, Int b, Int scale) - { - return compare(a, b, scale) != 0; - } - - template - static bool less(Float a, Int b, Int scale) - { - return compare(a, b, scale) < 0; - } - - template - static bool greater(Float a, Int b, Int scale) - { - return compare(a, b, scale) > 0; - } - - template - static bool lessOrEquals(Float a, Int b, Int scale) - { - return compare(a, b, scale) <= 0; - } - - template - static bool greaterOrEquals(Float a, Int b, Int scale) - { - return compare(a, b, scale) >= 0; - } -}; -} diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 28ab01b77a4..ba0d7ea83b5 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -11,17 +11,18 @@ #include #include -#include -#include -#include +#include #include #include -#include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include +#include +#include +#include #include #include @@ -1178,6 +1179,9 @@ public: const bool left_is_string = which_left.isStringOrFixedString(); const bool right_is_string = which_right.isStringOrFixedString(); + const bool left_is_float = which_left.isFloat(); + const bool right_is_float = which_right.isFloat(); + bool date_and_datetime = (which_left.idx != which_right.idx) && (which_left.isDate() || which_left.isDate32() || which_left.isDateTime() || which_left.isDateTime64()) && (which_right.isDate() || which_right.isDate32() || which_right.isDateTime() || which_right.isDateTime64()); @@ -1237,6 +1241,15 @@ public: throw Exception( "No operation " + getName() + " between " + left_type->getName() + " and " + right_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + /// When Decimal comparing to Float32/64, We convert both of them into Float64. Other systems like MySQL + /// also do as this. + if (left_is_float || right_is_float) + { + const auto converted_type = DataTypeFactory::instance().get("Float64"); + ColumnPtr c0_converted = castColumn(col_with_type_and_name_left, converted_type); + ColumnPtr c1_converted = castColumn(col_with_type_and_name_right, converted_type); + return executeGenericIdenticalTypes(c0_converted.get(), c1_converted.get()); + } return executeDecimal(col_with_type_and_name_left, col_with_type_and_name_right); } diff --git a/src/Functions/IsOperation.h b/src/Functions/IsOperation.h index 0e4feeaf314..369978fe271 100644 --- a/src/Functions/IsOperation.h +++ b/src/Functions/IsOperation.h @@ -17,9 +17,7 @@ template struct GreatestBaseImpl; template struct ModuloImpl; template struct EqualsOp; template struct NotEqualsOp; -template struct LessOp; template struct LessOrEqualsOp; -template struct GreaterOp; template struct GreaterOrEqualsOp; template @@ -44,9 +42,7 @@ struct IsOperation { static constexpr bool equals = IsSameOperation::value; static constexpr bool not_equals = IsSameOperation::value; - static constexpr bool less = IsSameOperation::value; static constexpr bool less_or_equals = IsSameOperation::value; - static constexpr bool greater = IsSameOperation::value; static constexpr bool greater_or_equals = IsSameOperation::value; static constexpr bool plus = IsSameOperation::value; diff --git a/tests/queries/0_stateless/00700_decimal_compare.reference b/tests/queries/0_stateless/00700_decimal_compare.reference index 81cc789eaf1..623c3161b24 100644 --- a/tests/queries/0_stateless/00700_decimal_compare.reference +++ b/tests/queries/0_stateless/00700_decimal_compare.reference @@ -1,7 +1,7 @@ -1 0 1 0 +1 0 1 -42 -42 1 0 0 0 1 1 diff --git a/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.sql b/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.sql index 19808d6fbd9..954637eec9b 100644 --- a/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.sql +++ b/tests/queries/0_stateless/02124_comparison_betwwen_decimal_and_float.sql @@ -17,4 +17,4 @@ select CAST(1.0, 'Decimal(15,2)') != CAST(-1, 'Float32'); SELECT toDecimal32('11.00', 2) > 1.; -SELECT 0.1 > 0.1::Decimal256(70); +SELECT 0.1000000000000000055511151231257827021181583404541015625::Decimal256(70) = 0.1; From 148fbb82ad43bc051c7db8c985ead5e58505f872 Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 24 Dec 2021 05:19:51 +0000 Subject: [PATCH 0289/1260] fix fix comment --- src/Functions/FunctionsComparison.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index ba0d7ea83b5..4c21346e338 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -688,7 +688,7 @@ private: return (res = DecimalComparison::apply(col_left, col_right)) != nullptr; }; - if (!callOnBasicTypes(left_number, right_number, call)) + if (!callOnBasicTypes(left_number, right_number, call)) throw Exception("Wrong call for " + getName() + " with " + col_left.type->getName() + " and " + col_right.type->getName(), ErrorCodes::LOGICAL_ERROR); @@ -1236,13 +1236,13 @@ public: } else { - // compare + /// Check does another data type is comparable to Decimal, includes Int and Float. if (!allowDecimalComparison(left_type, right_type) && !date_and_datetime) throw Exception( "No operation " + getName() + " between " + left_type->getName() + " and " + right_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - /// When Decimal comparing to Float32/64, We convert both of them into Float64. Other systems like MySQL - /// also do as this. + /// When Decimal comparing to Float32/64, we convert both of them into Float64. + /// Other systems like MySQL and Spark also do as this. if (left_is_float || right_is_float) { const auto converted_type = DataTypeFactory::instance().get("Float64"); From f67e60a55d830984145f8564ef44bee948f6175a Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 24 Dec 2021 06:17:30 +0000 Subject: [PATCH 0290/1260] fix test --- tests/queries/0_stateless/00700_decimal_compare.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00700_decimal_compare.sql b/tests/queries/0_stateless/00700_decimal_compare.sql index 2ffd931820b..41ff8b38102 100644 --- a/tests/queries/0_stateless/00700_decimal_compare.sql +++ b/tests/queries/0_stateless/00700_decimal_compare.sql @@ -17,8 +17,8 @@ CREATE TABLE IF NOT EXISTS decimal INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (42, 42, 42, 0.42, 0.42, 0.42, 42.42, 42.42, 42.42, 42.42); INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-42, -42, -42, -0.42, -0.42, -0.42, -42.42, -42.42, -42.42, -42.42); -SELECT a > toFloat64(0) FROM decimal; -SELECT g > toFloat32(0) FROM decimal; +SELECT a > toFloat64(0) FROM decimal ORDER BY a; +SELECT g > toFloat32(0) FROM decimal ORDER BY g; SELECT a > '0.0' FROM decimal ORDER BY a; SELECT a, b, a = b, a < b, a > b, a != b, a <= b, a >= b FROM decimal ORDER BY a; From fbddbebca8a8a5dfc6b4c978bdbeae1b3b422333 Mon Sep 17 00:00:00 2001 From: freedomDR <1640145602@qq.com> Date: Fri, 24 Dec 2021 06:21:13 +0000 Subject: [PATCH 0291/1260] fix bitmapData rb_contain rb_remove --- src/AggregateFunctions/AggregateFunctionGroupBitmapData.h | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index 878cbc3219f..eee91904b9b 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -421,6 +421,9 @@ public: */ UInt8 rb_contains(UInt64 x) const { + if (!std::is_same_v && x > rb_max()) + return 0; + if (isSmall()) return small.find(x) != small.end(); else @@ -432,6 +435,9 @@ public: */ void rb_remove(UInt64 x) { + if (!std::is_same_v && x > rb_max()) + return; + if (isSmall()) toLarge(); From ffc1fca29609251157a6d64e1801f7a34ba9e70b Mon Sep 17 00:00:00 2001 From: Kevin Michel Date: Fri, 22 Oct 2021 09:15:34 +0200 Subject: [PATCH 0292/1260] Start/stop servers when listen_host/*_port changes This allows starting and stopping separately each protocol server without restarting ClickHouse. This also allows adding or removing `listen_host` entries, which start and stops servers for all enabled ports. When stopping a server, the listening socket is immediately closed (and available for another server). Protocols with persistent connections try to wait for any currently running query to finish before closing the connection, but idle connection are closed quickly (depending on how often the protocol is polled). An extra ProfileEvent is added, `MainConfigLoads`, it is incremented every time the configuration is reloaded. This helps when trying to assess whether the new configuration was applied. --- programs/keeper/Keeper.cpp | 19 +- programs/server/Server.cpp | 681 +++++++++++------- programs/server/Server.h | 28 +- src/Common/ProfileEvents.cpp | 3 +- src/Interpreters/AsynchronousMetrics.cpp | 35 +- src/Interpreters/AsynchronousMetrics.h | 14 +- src/Server/GRPCServer.h | 4 + src/Server/HTTP/HTTPServer.cpp | 22 +- src/Server/HTTP/HTTPServer.h | 18 +- src/Server/HTTP/HTTPServerConnection.cpp | 13 +- src/Server/HTTP/HTTPServerConnection.h | 3 + .../HTTP/HTTPServerConnectionFactory.cpp | 4 +- src/Server/HTTP/HTTPServerConnectionFactory.h | 6 +- src/Server/KeeperTCPHandlerFactory.h | 6 +- src/Server/MySQLHandler.cpp | 15 +- src/Server/MySQLHandler.h | 10 +- src/Server/MySQLHandlerFactory.cpp | 6 +- src/Server/MySQLHandlerFactory.h | 7 +- src/Server/PostgreSQLHandler.cpp | 12 +- src/Server/PostgreSQLHandler.h | 7 +- src/Server/PostgreSQLHandlerFactory.cpp | 5 +- src/Server/PostgreSQLHandlerFactory.h | 6 +- src/Server/ProtocolServerAdapter.cpp | 39 +- src/Server/ProtocolServerAdapter.h | 21 +- src/Server/TCPHandler.cpp | 10 +- src/Server/TCPHandler.h | 4 +- src/Server/TCPHandlerFactory.h | 8 +- src/Server/TCPServer.cpp | 36 + src/Server/TCPServer.h | 47 ++ src/Server/TCPServerConnectionFactory.h | 27 + tests/integration/helpers/cluster.py | 18 +- .../integration/test_server_reload/.gitignore | 1 + .../test_server_reload/__init__.py | 0 .../configs/default_passwd.xml | 13 + .../test_server_reload/configs/dhparam.pem | 8 + .../configs/ports_from_zk.xml | 9 + .../test_server_reload/configs/server.crt | 18 + .../test_server_reload/configs/server.key | 28 + .../test_server_reload/configs/ssl_conf.xml | 18 + .../protos/clickhouse_grpc.proto | 174 +++++ tests/integration/test_server_reload/test.py | 284 ++++++++ 41 files changed, 1287 insertions(+), 400 deletions(-) create mode 100644 src/Server/TCPServer.cpp create mode 100644 src/Server/TCPServer.h create mode 100644 src/Server/TCPServerConnectionFactory.h create mode 100644 tests/integration/test_server_reload/.gitignore create mode 100644 tests/integration/test_server_reload/__init__.py create mode 100644 tests/integration/test_server_reload/configs/default_passwd.xml create mode 100644 tests/integration/test_server_reload/configs/dhparam.pem create mode 100644 tests/integration/test_server_reload/configs/ports_from_zk.xml create mode 100644 tests/integration/test_server_reload/configs/server.crt create mode 100644 tests/integration/test_server_reload/configs/server.key create mode 100644 tests/integration/test_server_reload/configs/ssl_conf.xml create mode 100644 tests/integration/test_server_reload/protos/clickhouse_grpc.proto create mode 100644 tests/integration/test_server_reload/test.py diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index afd6a36ea15..d144b4d332e 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -379,11 +380,11 @@ int Keeper::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.receive_timeout); socket.setSendTimeout(settings.send_timeout); servers->emplace_back( + listen_host, port_name, - std::make_unique( - new KeeperTCPHandlerFactory(*this, false), server_pool, socket, new Poco::Net::TCPServerParams)); - - LOG_INFO(log, "Listening for connections to Keeper (tcp): {}", address.toString()); + "Keeper (tcp): " + address.toString(), + std::make_unique( + new KeeperTCPHandlerFactory(*this, false), server_pool, socket)); }); const char * secure_port_name = "keeper_server.tcp_port_secure"; @@ -395,10 +396,11 @@ int Keeper::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.receive_timeout); socket.setSendTimeout(settings.send_timeout); servers->emplace_back( + listen_host, secure_port_name, - std::make_unique( - new KeeperTCPHandlerFactory(*this, true), server_pool, socket, new Poco::Net::TCPServerParams)); - LOG_INFO(log, "Listening for connections to Keeper with secure protocol (tcp_secure): {}", address.toString()); + "Keeper with secure protocol (tcp_secure): " + address.toString(), + std::make_unique( + new KeeperTCPHandlerFactory(*this, true), server_pool, socket)); #else UNUSED(port); throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.", @@ -408,7 +410,10 @@ int Keeper::main(const std::vector & /*args*/) } for (auto & server : *servers) + { server.start(); + LOG_INFO(log, "Listening for {}", server.getDescription()); + } zkutil::EventPtr unused_event = std::make_shared(); zkutil::ZooKeeperNodeCache unused_cache([] { return nullptr; }); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 14075f9fbf2..43d2b64c4f2 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -34,6 +34,7 @@ #include #include #include +#include #include #include #include @@ -70,6 +71,7 @@ #include "MetricsTransmitter.h" #include #include +#include #include #include #include @@ -127,6 +129,11 @@ namespace CurrentMetrics extern const Metric MaxPushedDDLEntryID; } +namespace ProfileEvents +{ + extern const Event MainConfigLoads; +} + namespace fs = std::filesystem; #if USE_JEMALLOC @@ -344,16 +351,53 @@ Poco::Net::SocketAddress Server::socketBindListen(Poco::Net::ServerSocket & sock return address; } -void Server::createServer(const std::string & listen_host, const char * port_name, bool listen_try, CreateServerFunc && func) const +std::vector getListenHosts(const Poco::Util::AbstractConfiguration & config) +{ + auto listen_hosts = DB::getMultipleValuesFromConfig(config, "", "listen_host"); + if (listen_hosts.empty()) + { + listen_hosts.emplace_back("::1"); + listen_hosts.emplace_back("127.0.0.1"); + } + return listen_hosts; +} + +bool getListenTry(const Poco::Util::AbstractConfiguration & config) +{ + bool listen_try = config.getBool("listen_try", false); + if (!listen_try) + listen_try = DB::getMultipleValuesFromConfig(config, "", "listen_host").empty(); + return listen_try; +} + + +void Server::createServer( + Poco::Util::AbstractConfiguration & config, + const std::string & listen_host, + const char * port_name, + bool listen_try, + bool start_server, + std::vector & servers, + CreateServerFunc && func) const { /// For testing purposes, user may omit tcp_port or http_port or https_port in configuration file. - if (!config().has(port_name)) + if (config.getString(port_name, "").empty()) return; - auto port = config().getInt(port_name); + /// If we already have an active server for this listen_host/port_name, don't create it again + for (const auto & server : servers) + if (!server.isStopping() && server.getListenHost() == listen_host && server.getPortName() == port_name) + return; + + auto port = config.getInt(port_name); try { - func(port); + servers.push_back(func(port)); + if (start_server) + { + servers.back().start(); + LOG_INFO(&logger(), "Listening for {}", servers.back().getDescription()); + } global_context->registerServerPort(port_name, port); } catch (const Poco::Exception &) @@ -515,6 +559,25 @@ if (ThreadFuzzer::instance().isEffective()) config().getUInt("thread_pool_queue_size", 10000) ); + Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024)); + std::mutex servers_lock; + std::vector servers; + std::vector servers_to_start_before_tables; + /// This object will periodically calculate some metrics. + AsynchronousMetrics async_metrics( + global_context, config().getUInt("asynchronous_metrics_update_period_s", 1), + [&]() -> std::vector + { + std::vector metrics; + for (const auto & server : servers_to_start_before_tables) + metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); + std::lock_guard lock(servers_lock); + for (const auto & server : servers) + metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); + return metrics; + } + ); + ConnectionCollector::init(global_context, config().getUInt("max_threads_for_connection_collector", 10)); bool has_zookeeper = config().has("zookeeper"); @@ -870,12 +933,17 @@ if (ThreadFuzzer::instance().isEffective()) global_context->reloadZooKeeperIfChanged(config); global_context->reloadAuxiliaryZooKeepersConfigIfChanged(config); + + std::lock_guard lock(servers_lock); + updateServers(*config, server_pool, async_metrics, servers); } global_context->updateStorageConfiguration(*config); global_context->updateInterserverCredentials(*config); CompressionCodecEncrypted::Configuration::instance().tryLoad(*config, "encryption_codecs"); + + ProfileEvents::increment(ProfileEvents::MainConfigLoads); }, /* already_loaded = */ false); /// Reload it right now (initial loading) @@ -987,24 +1055,8 @@ if (ThreadFuzzer::instance().isEffective()) /// try set up encryption. There are some errors in config, error will be printed and server wouldn't start. CompressionCodecEncrypted::Configuration::instance().load(config(), "encryption_codecs"); - Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0); - - Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024)); - Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; - http_params->setTimeout(settings.http_receive_timeout); - http_params->setKeepAliveTimeout(keep_alive_timeout); - - auto servers_to_start_before_tables = std::make_shared>(); - - std::vector listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host"); - - bool listen_try = config().getBool("listen_try", false); - if (listen_hosts.empty()) - { - listen_hosts.emplace_back("::1"); - listen_hosts.emplace_back("127.0.0.1"); - listen_try = true; - } + const auto listen_hosts = getListenHosts(config()); + const auto listen_try = getListenTry(config()); if (config().has("keeper_server")) { @@ -1027,39 +1079,46 @@ if (ThreadFuzzer::instance().isEffective()) { /// TCP Keeper const char * port_name = "keeper_server.tcp_port"; - createServer(listen_host, port_name, listen_try, [&](UInt16 port) - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(socket, listen_host, port); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - servers_to_start_before_tables->emplace_back( - port_name, - std::make_unique( - new KeeperTCPHandlerFactory(*this, false), server_pool, socket, new Poco::Net::TCPServerParams)); - - LOG_INFO(log, "Listening for connections to Keeper (tcp): {}", address.toString()); - }); + createServer( + config(), listen_host, port_name, listen_try, /* start_server: */ false, + servers_to_start_before_tables, + [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(socket, listen_host, port); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "Keeper (tcp): " + address.toString(), + std::make_unique( + new KeeperTCPHandlerFactory(*this, false), server_pool, socket)); + }); const char * secure_port_name = "keeper_server.tcp_port_secure"; - createServer(listen_host, secure_port_name, listen_try, [&](UInt16 port) - { + createServer( + config(), listen_host, secure_port_name, listen_try, /* start_server: */ false, + servers_to_start_before_tables, + [&](UInt16 port) -> ProtocolServerAdapter + { #if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - servers_to_start_before_tables->emplace_back( - secure_port_name, - std::make_unique( - new KeeperTCPHandlerFactory(*this, true), server_pool, socket, new Poco::Net::TCPServerParams)); - LOG_INFO(log, "Listening for connections to Keeper with secure protocol (tcp_secure): {}", address.toString()); + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + secure_port_name, + "Keeper with secure protocol (tcp_secure): " + address.toString(), + std::make_unique( + new KeeperTCPHandlerFactory(*this, true), server_pool, socket)); #else - UNUSED(port); - throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.", - ErrorCodes::SUPPORT_IS_DISABLED}; + UNUSED(port); + throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.", + ErrorCodes::SUPPORT_IS_DISABLED}; #endif - }); + }); } #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "ClickHouse server built without NuRaft library. Cannot use internal coordination."); @@ -1067,14 +1126,19 @@ if (ThreadFuzzer::instance().isEffective()) } - for (auto & server : *servers_to_start_before_tables) + for (auto & server : servers_to_start_before_tables) + { server.start(); + LOG_INFO(log, "Listening for {}", server.getDescription()); + } SCOPE_EXIT({ /// Stop reloading of the main config. This must be done before `global_context->shutdown()` because /// otherwise the reloading may pass a changed config to some destroyed parts of ContextSharedPart. main_config_reloader.reset(); + async_metrics.stop(); + /** Ask to cancel background jobs all table engines, * and also query_log. * It is important to do early, not in destructor of Context, because @@ -1086,11 +1150,11 @@ if (ThreadFuzzer::instance().isEffective()) LOG_DEBUG(log, "Shut down storages."); - if (!servers_to_start_before_tables->empty()) + if (!servers_to_start_before_tables.empty()) { LOG_DEBUG(log, "Waiting for current connections to servers for tables to finish."); int current_connections = 0; - for (auto & server : *servers_to_start_before_tables) + for (auto & server : servers_to_start_before_tables) { server.stop(); current_connections += server.currentConnections(); @@ -1102,7 +1166,7 @@ if (ThreadFuzzer::instance().isEffective()) LOG_INFO(log, "Closed all listening sockets."); if (current_connections > 0) - current_connections = waitServersToFinish(*servers_to_start_before_tables, config().getInt("shutdown_wait_unfinished", 5)); + current_connections = waitServersToFinish(servers_to_start_before_tables, config().getInt("shutdown_wait_unfinished", 5)); if (current_connections) LOG_INFO(log, "Closed connections to servers for tables. But {} remain. Probably some tables of other users cannot finish their connections after context shutdown.", current_connections); @@ -1256,223 +1320,18 @@ if (ThreadFuzzer::instance().isEffective()) LOG_INFO(log, "TaskStats is not implemented for this OS. IO accounting will be disabled."); #endif - auto servers = std::make_shared>(); { - /// This object will periodically calculate some metrics. - AsynchronousMetrics async_metrics( - global_context, config().getUInt("asynchronous_metrics_update_period_s", 1), servers_to_start_before_tables, servers); attachSystemTablesAsync(global_context, *DatabaseCatalog::instance().getSystemDatabase(), async_metrics); - for (const auto & listen_host : listen_hosts) { - /// HTTP - const char * port_name = "http_port"; - createServer(listen_host, port_name, listen_try, [&](UInt16 port) - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(socket, listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - - servers->emplace_back( - port_name, - std::make_unique( - context(), createHandlerFactory(*this, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params)); - - LOG_INFO(log, "Listening for http://{}", address.toString()); - }); - - /// HTTPS - port_name = "https_port"; - createServer(listen_host, port_name, listen_try, [&](UInt16 port) - { -#if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - servers->emplace_back( - port_name, - std::make_unique( - context(), createHandlerFactory(*this, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params)); - - LOG_INFO(log, "Listening for https://{}", address.toString()); -#else - UNUSED(port); - throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.", - ErrorCodes::SUPPORT_IS_DISABLED}; -#endif - }); - - /// TCP - port_name = "tcp_port"; - createServer(listen_host, port_name, listen_try, [&](UInt16 port) - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(socket, listen_host, port); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - servers->emplace_back(port_name, std::make_unique( - new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ false), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - - LOG_INFO(log, "Listening for connections with native protocol (tcp): {}", address.toString()); - }); - - /// TCP with PROXY protocol, see https://github.com/wolfeidau/proxyv2/blob/master/docs/proxy-protocol.txt - port_name = "tcp_with_proxy_port"; - createServer(listen_host, port_name, listen_try, [&](UInt16 port) - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(socket, listen_host, port); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - servers->emplace_back(port_name, std::make_unique( - new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ true), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - - LOG_INFO(log, "Listening for connections with native protocol (tcp) with PROXY: {}", address.toString()); - }); - - /// TCP with SSL - port_name = "tcp_port_secure"; - createServer(listen_host, port_name, listen_try, [&](UInt16 port) - { -#if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - servers->emplace_back(port_name, std::make_unique( - new TCPHandlerFactory(*this, /* secure */ true, /* proxy protocol */ false), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - LOG_INFO(log, "Listening for connections with secure native protocol (tcp_secure): {}", address.toString()); -#else - UNUSED(port); - throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.", - ErrorCodes::SUPPORT_IS_DISABLED}; -#endif - }); - - /// Interserver IO HTTP - port_name = "interserver_http_port"; - createServer(listen_host, port_name, listen_try, [&](UInt16 port) - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(socket, listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - servers->emplace_back( - port_name, - std::make_unique( - context(), - createHandlerFactory(*this, async_metrics, "InterserverIOHTTPHandler-factory"), - server_pool, - socket, - http_params)); - - LOG_INFO(log, "Listening for replica communication (interserver): http://{}", address.toString()); - }); - - port_name = "interserver_https_port"; - createServer(listen_host, port_name, listen_try, [&](UInt16 port) - { -#if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - servers->emplace_back( - port_name, - std::make_unique( - context(), - createHandlerFactory(*this, async_metrics, "InterserverIOHTTPSHandler-factory"), - server_pool, - socket, - http_params)); - - LOG_INFO(log, "Listening for secure replica communication (interserver): https://{}", address.toString()); -#else - UNUSED(port); - throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.", - ErrorCodes::SUPPORT_IS_DISABLED}; -#endif - }); - - port_name = "mysql_port"; - createServer(listen_host, port_name, listen_try, [&](UInt16 port) - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(Poco::Timespan()); - socket.setSendTimeout(settings.send_timeout); - servers->emplace_back(port_name, std::make_unique( - new MySQLHandlerFactory(*this), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - - LOG_INFO(log, "Listening for MySQL compatibility protocol: {}", address.toString()); - }); - - port_name = "postgresql_port"; - createServer(listen_host, port_name, listen_try, [&](UInt16 port) - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(Poco::Timespan()); - socket.setSendTimeout(settings.send_timeout); - servers->emplace_back(port_name, std::make_unique( - new PostgreSQLHandlerFactory(*this), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - - LOG_INFO(log, "Listening for PostgreSQL compatibility protocol: " + address.toString()); - }); - -#if USE_GRPC - port_name = "grpc_port"; - createServer(listen_host, port_name, listen_try, [&](UInt16 port) - { - Poco::Net::SocketAddress server_address(listen_host, port); - servers->emplace_back(port_name, std::make_unique(*this, makeSocketAddress(listen_host, port, log))); - LOG_INFO(log, "Listening for gRPC protocol: " + server_address.toString()); - }); -#endif - - /// Prometheus (if defined and not setup yet with http_port) - port_name = "prometheus.port"; - createServer(listen_host, port_name, listen_try, [&](UInt16 port) - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(socket, listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - servers->emplace_back( - port_name, - std::make_unique( - context(), - createHandlerFactory(*this, async_metrics, "PrometheusHandler-factory"), - server_pool, - socket, - http_params)); - - LOG_INFO(log, "Listening for Prometheus: http://{}", address.toString()); - }); + std::lock_guard lock(servers_lock); + createServers(config(), listen_hosts, listen_try, server_pool, async_metrics, servers); + if (servers.empty()) + throw Exception( + "No servers started (add valid listen_host and 'tcp_port' or 'http_port' to configuration file.)", + ErrorCodes::NO_ELEMENTS_IN_CONFIG); } - if (servers->empty()) - throw Exception("No servers started (add valid listen_host and 'tcp_port' or 'http_port' to configuration file.)", - ErrorCodes::NO_ELEMENTS_IN_CONFIG); - - /// Must be done after initialization of `servers`, because async_metrics will access `servers` variable from its thread. async_metrics.start(); { @@ -1551,9 +1410,15 @@ if (ThreadFuzzer::instance().isEffective()) &CurrentMetrics::MaxDDLEntryID, &CurrentMetrics::MaxPushedDDLEntryID)); } - for (auto & server : *servers) - server.start(); - LOG_INFO(log, "Ready for connections."); + { + std::lock_guard lock(servers_lock); + for (auto & server : servers) + { + server.start(); + LOG_INFO(log, "Listening for {}", server.getDescription()); + } + LOG_INFO(log, "Ready for connections."); + } SCOPE_EXIT_SAFE({ LOG_DEBUG(log, "Received termination signal."); @@ -1562,10 +1427,13 @@ if (ThreadFuzzer::instance().isEffective()) is_cancelled = true; int current_connections = 0; - for (auto & server : *servers) { - server.stop(); - current_connections += server.currentConnections(); + std::lock_guard lock(servers_lock); + for (auto & server : servers) + { + server.stop(); + current_connections += server.currentConnections(); + } } if (current_connections) @@ -1578,7 +1446,7 @@ if (ThreadFuzzer::instance().isEffective()) global_context->getProcessList().killAllQueries(); if (current_connections) - current_connections = waitServersToFinish(*servers, config().getInt("shutdown_wait_unfinished", 5)); + current_connections = waitServersToFinish(servers, config().getInt("shutdown_wait_unfinished", 5)); if (current_connections) LOG_INFO(log, "Closed connections. But {} remain." @@ -1614,4 +1482,273 @@ if (ThreadFuzzer::instance().isEffective()) return Application::EXIT_OK; } + +void Server::createServers( + Poco::Util::AbstractConfiguration & config, + const std::vector & listen_hosts, + bool listen_try, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + std::vector & servers, + bool start_servers) +{ + const Settings & settings = global_context->getSettingsRef(); + + Poco::Timespan keep_alive_timeout(config.getUInt("keep_alive_timeout", 10), 0); + Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; + http_params->setTimeout(settings.http_receive_timeout); + http_params->setKeepAliveTimeout(keep_alive_timeout); + + for (const auto & listen_host : listen_hosts) + { + /// HTTP + const char * port_name = "http_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(socket, listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + + return ProtocolServerAdapter( + listen_host, + port_name, + "http://" + address.toString(), + std::make_unique( + context(), createHandlerFactory(*this, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params)); + }); + + /// HTTPS + port_name = "https_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { +#if USE_SSL + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "https://" + address.toString(), + std::make_unique( + context(), createHandlerFactory(*this, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params)); +#else + UNUSED(port); + throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.", + ErrorCodes::SUPPORT_IS_DISABLED}; +#endif + }); + + /// TCP + port_name = "tcp_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(socket, listen_host, port); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "native protocol (tcp): " + address.toString(), + std::make_unique( + new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ false), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + }); + + /// TCP with PROXY protocol, see https://github.com/wolfeidau/proxyv2/blob/master/docs/proxy-protocol.txt + port_name = "tcp_with_proxy_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(socket, listen_host, port); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "native protocol (tcp) with PROXY: " + address.toString(), + std::make_unique( + new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ true), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + }); + + /// TCP with SSL + port_name = "tcp_port_secure"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { +#if USE_SSL + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "secure native protocol (tcp_secure): " + address.toString(), + std::make_unique( + new TCPHandlerFactory(*this, /* secure */ true, /* proxy protocol */ false), + server_pool, + socket, + new Poco::Net::TCPServerParams)); +#else + UNUSED(port); + throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.", + ErrorCodes::SUPPORT_IS_DISABLED}; +#endif + }); + + /// Interserver IO HTTP + port_name = "interserver_http_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(socket, listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "replica communication (interserver): http://" + address.toString(), + std::make_unique( + context(), + createHandlerFactory(*this, async_metrics, "InterserverIOHTTPHandler-factory"), + server_pool, + socket, + http_params)); + }); + + port_name = "interserver_https_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { +#if USE_SSL + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "secure replica communication (interserver): https://" + address.toString(), + std::make_unique( + context(), + createHandlerFactory(*this, async_metrics, "InterserverIOHTTPSHandler-factory"), + server_pool, + socket, + http_params)); +#else + UNUSED(port); + throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.", + ErrorCodes::SUPPORT_IS_DISABLED}; +#endif + }); + + port_name = "mysql_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(Poco::Timespan()); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "MySQL compatibility protocol: " + address.toString(), + std::make_unique(new MySQLHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams)); + }); + + port_name = "postgresql_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(Poco::Timespan()); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "PostgreSQL compatibility protocol: " + address.toString(), + std::make_unique(new PostgreSQLHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams)); + }); + +#if USE_GRPC + port_name = "grpc_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::SocketAddress server_address(listen_host, port); + return ProtocolServerAdapter( + listen_host, + port_name, + "gRPC protocol: " + server_address.toString(), + std::make_unique(*this, makeSocketAddress(listen_host, port, &logger()))); + }); +#endif + + /// Prometheus (if defined and not setup yet with http_port) + port_name = "prometheus.port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(socket, listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "Prometheus: http://" + address.toString(), + std::make_unique( + context(), createHandlerFactory(*this, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); + }); + } + +} + +void Server::updateServers( + Poco::Util::AbstractConfiguration & config, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + std::vector & servers) +{ + Poco::Logger * log = &logger(); + /// Gracefully shutdown servers when their port is removed from config + const auto listen_hosts = getListenHosts(config); + const auto listen_try = getListenTry(config); + + for (auto & server : servers) + if (!server.isStopping()) + { + bool has_host = std::find(listen_hosts.begin(), listen_hosts.end(), server.getListenHost()) != listen_hosts.end(); + bool has_port = !config.getString(server.getPortName(), "").empty(); + if (!has_host || !has_port || config.getInt(server.getPortName()) != server.portNumber()) + { + server.stop(); + LOG_INFO(log, "Stopped listening for {}", server.getDescription()); + } + } + + createServers(config, listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers: */ true); + + /// Remove servers once all their connections are closed + while (std::any_of(servers.begin(), servers.end(), [](const auto & server) { return server.isStopping(); })) + { + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + std::erase_if(servers, [&log](auto & server) + { + if (!server.isStopping()) + return false; + auto is_finished = server.currentConnections() == 0; + if (is_finished) + LOG_DEBUG(log, "Server finished: {}", server.getDescription()); + else + LOG_TRACE(log, "Waiting server to finish: {}", server.getDescription()); + return is_finished; + }); + } +} + } diff --git a/programs/server/Server.h b/programs/server/Server.h index 45e5fccd51d..b4f2ea3bb79 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -24,6 +24,8 @@ namespace Poco namespace DB { +class AsynchronousMetrics; +class ProtocolServerAdapter; class Server : public BaseDaemon, public IServer { @@ -67,8 +69,30 @@ private: ContextMutablePtr global_context; Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const; - using CreateServerFunc = std::function; - void createServer(const std::string & listen_host, const char * port_name, bool listen_try, CreateServerFunc && func) const; + using CreateServerFunc = std::function; + void createServer( + Poco::Util::AbstractConfiguration & config, + const std::string & listen_host, + const char * port_name, + bool listen_try, + bool start_server, + std::vector & servers, + CreateServerFunc && func) const; + + void createServers( + Poco::Util::AbstractConfiguration & config, + const std::vector & listen_hosts, + bool listen_try, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + std::vector & servers, + bool start_servers = false); + + void updateServers( + Poco::Util::AbstractConfiguration & config, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + std::vector & servers); }; } diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 982523a3ef2..ce321d17b48 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -274,7 +274,8 @@ M(ThreadPoolReaderPageCacheMissElapsedMicroseconds, "Time spent reading data inside the asynchronous job in ThreadPoolReader - when read was not done from page cache.") \ \ M(AsynchronousReadWaitMicroseconds, "Time spent in waiting for asynchronous reads.") \ - + \ + M(MainConfigLoads, "Number of times the main configuration was reloaded.") \ namespace ProfileEvents { diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 121f7c4153f..d1c5fbebbc7 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -69,12 +69,10 @@ static std::unique_ptr openFileIfExists(const std::stri AsynchronousMetrics::AsynchronousMetrics( ContextPtr global_context_, int update_period_seconds, - std::shared_ptr> servers_to_start_before_tables_, - std::shared_ptr> servers_) + const ProtocolServerMetricsFunc & protocol_server_metrics_func_) : WithContext(global_context_) , update_period(update_period_seconds) - , servers_to_start_before_tables(servers_to_start_before_tables_) - , servers(servers_) + , protocol_server_metrics_func(protocol_server_metrics_func_) , log(&Poco::Logger::get("AsynchronousMetrics")) { #if defined(OS_LINUX) @@ -238,7 +236,7 @@ void AsynchronousMetrics::start() thread = std::make_unique([this] { run(); }); } -AsynchronousMetrics::~AsynchronousMetrics() +void AsynchronousMetrics::stop() { try { @@ -249,7 +247,10 @@ AsynchronousMetrics::~AsynchronousMetrics() wait_cond.notify_one(); if (thread) + { thread->join(); + thread.reset(); + } } catch (...) { @@ -257,6 +258,11 @@ AsynchronousMetrics::~AsynchronousMetrics() } } +AsynchronousMetrics::~AsynchronousMetrics() +{ + stop(); +} + AsynchronousMetricValues AsynchronousMetrics::getValues() const { @@ -1381,22 +1387,11 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti return it->second; }; - if (servers_to_start_before_tables) + const auto server_metrics = protocol_server_metrics_func(); + for (const auto & server_metric : server_metrics) { - for (const auto & server : *servers_to_start_before_tables) - { - if (const auto * name = get_metric_name(server.getPortName())) - new_values[name] = server.currentThreads(); - } - } - - if (servers) - { - for (const auto & server : *servers) - { - if (const auto * name = get_metric_name(server.getPortName())) - new_values[name] = server.currentThreads(); - } + if (const auto * name = get_metric_name(server_metric.port_name)) + new_values[name] = server_metric.current_threads; } } diff --git a/src/Interpreters/AsynchronousMetrics.h b/src/Interpreters/AsynchronousMetrics.h index 7a5c2d638d7..3c7581ce1a3 100644 --- a/src/Interpreters/AsynchronousMetrics.h +++ b/src/Interpreters/AsynchronousMetrics.h @@ -30,6 +30,11 @@ class ReadBuffer; using AsynchronousMetricValue = double; using AsynchronousMetricValues = std::unordered_map; +struct ProtocolServerMetrics +{ + String port_name; + size_t current_threads; +}; /** Periodically (by default, each minute, starting at 30 seconds offset) * calculates and updates some metrics, @@ -41,24 +46,25 @@ using AsynchronousMetricValues = std::unordered_map()>; AsynchronousMetrics( ContextPtr global_context_, int update_period_seconds, - std::shared_ptr> servers_to_start_before_tables_, - std::shared_ptr> servers_); + const ProtocolServerMetricsFunc & protocol_server_metrics_func_); ~AsynchronousMetrics(); /// Separate method allows to initialize the `servers` variable beforehand. void start(); + void stop(); + /// Returns copy of all values. AsynchronousMetricValues getValues() const; private: const std::chrono::seconds update_period; - std::shared_ptr> servers_to_start_before_tables{nullptr}; - std::shared_ptr> servers{nullptr}; + ProtocolServerMetricsFunc protocol_server_metrics_func; mutable std::mutex mutex; std::condition_variable wait_cond; diff --git a/src/Server/GRPCServer.h b/src/Server/GRPCServer.h index 25c3813c11d..e2b48f1c16b 100644 --- a/src/Server/GRPCServer.h +++ b/src/Server/GRPCServer.h @@ -4,6 +4,7 @@ #if USE_GRPC #include +#include #include "clickhouse_grpc.grpc.pb.h" namespace Poco { class Logger; } @@ -30,6 +31,9 @@ public: /// Stops the server. No new connections will be accepted. void stop(); + /// Returns the port this server is listening to. + UInt16 portNumber() const { return address_to_listen.port(); } + /// Returns the number of currently handled connections. size_t currentConnections() const; diff --git a/src/Server/HTTP/HTTPServer.cpp b/src/Server/HTTP/HTTPServer.cpp index 42e6467d0af..2e91fad1c0f 100644 --- a/src/Server/HTTP/HTTPServer.cpp +++ b/src/Server/HTTP/HTTPServer.cpp @@ -5,31 +5,13 @@ namespace DB { -HTTPServer::HTTPServer( - ContextPtr context, - HTTPRequestHandlerFactoryPtr factory_, - UInt16 port_number, - Poco::Net::HTTPServerParams::Ptr params) - : TCPServer(new HTTPServerConnectionFactory(context, params, factory_), port_number, params), factory(factory_) -{ -} - -HTTPServer::HTTPServer( - ContextPtr context, - HTTPRequestHandlerFactoryPtr factory_, - const Poco::Net::ServerSocket & socket, - Poco::Net::HTTPServerParams::Ptr params) - : TCPServer(new HTTPServerConnectionFactory(context, params, factory_), socket, params), factory(factory_) -{ -} - HTTPServer::HTTPServer( ContextPtr context, HTTPRequestHandlerFactoryPtr factory_, Poco::ThreadPool & thread_pool, - const Poco::Net::ServerSocket & socket, + Poco::Net::ServerSocket & socket_, Poco::Net::HTTPServerParams::Ptr params) - : TCPServer(new HTTPServerConnectionFactory(context, params, factory_), thread_pool, socket, params), factory(factory_) + : TCPServer(new HTTPServerConnectionFactory(context, params, factory_), thread_pool, socket_, params), factory(factory_) { } diff --git a/src/Server/HTTP/HTTPServer.h b/src/Server/HTTP/HTTPServer.h index 3518fd66d20..07ad54d267f 100644 --- a/src/Server/HTTP/HTTPServer.h +++ b/src/Server/HTTP/HTTPServer.h @@ -1,9 +1,9 @@ #pragma once #include +#include #include -#include #include @@ -13,26 +13,14 @@ namespace DB class Context; -class HTTPServer : public Poco::Net::TCPServer +class HTTPServer : public TCPServer { public: explicit HTTPServer( - ContextPtr context, - HTTPRequestHandlerFactoryPtr factory, - UInt16 port_number = 80, - Poco::Net::HTTPServerParams::Ptr params = new Poco::Net::HTTPServerParams); - - HTTPServer( - ContextPtr context, - HTTPRequestHandlerFactoryPtr factory, - const Poco::Net::ServerSocket & socket, - Poco::Net::HTTPServerParams::Ptr params); - - HTTPServer( ContextPtr context, HTTPRequestHandlerFactoryPtr factory, Poco::ThreadPool & thread_pool, - const Poco::Net::ServerSocket & socket, + Poco::Net::ServerSocket & socket, Poco::Net::HTTPServerParams::Ptr params); ~HTTPServer() override; diff --git a/src/Server/HTTP/HTTPServerConnection.cpp b/src/Server/HTTP/HTTPServerConnection.cpp index de81da20ead..7020b8e9a23 100644 --- a/src/Server/HTTP/HTTPServerConnection.cpp +++ b/src/Server/HTTP/HTTPServerConnection.cpp @@ -1,4 +1,5 @@ #include +#include #include @@ -7,10 +8,11 @@ namespace DB HTTPServerConnection::HTTPServerConnection( ContextPtr context_, + TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket, Poco::Net::HTTPServerParams::Ptr params_, HTTPRequestHandlerFactoryPtr factory_) - : TCPServerConnection(socket), context(Context::createCopy(context_)), params(params_), factory(factory_), stopped(false) + : TCPServerConnection(socket), context(Context::createCopy(context_)), tcp_server(tcp_server_), params(params_), factory(factory_), stopped(false) { poco_check_ptr(factory); } @@ -20,12 +22,12 @@ void HTTPServerConnection::run() std::string server = params->getSoftwareVersion(); Poco::Net::HTTPServerSession session(socket(), params); - while (!stopped && session.hasMoreRequests()) + while (!stopped && tcp_server.isOpen() && session.hasMoreRequests()) { try { std::unique_lock lock(mutex); - if (!stopped) + if (!stopped && tcp_server.isOpen()) { HTTPServerResponse response(session); HTTPServerRequest request(context, response, session); @@ -48,6 +50,11 @@ void HTTPServerConnection::run() response.set("Server", server); try { + if (!tcp_server.isOpen()) + { + sendErrorResponse(session, Poco::Net::HTTPResponse::HTTP_SERVICE_UNAVAILABLE); + break; + } std::unique_ptr handler(factory->createRequestHandler(request)); if (handler) diff --git a/src/Server/HTTP/HTTPServerConnection.h b/src/Server/HTTP/HTTPServerConnection.h index 1c7ae6cd2b7..db3969f6ffb 100644 --- a/src/Server/HTTP/HTTPServerConnection.h +++ b/src/Server/HTTP/HTTPServerConnection.h @@ -9,12 +9,14 @@ namespace DB { +class TCPServer; class HTTPServerConnection : public Poco::Net::TCPServerConnection { public: HTTPServerConnection( ContextPtr context, + TCPServer & tcp_server, const Poco::Net::StreamSocket & socket, Poco::Net::HTTPServerParams::Ptr params, HTTPRequestHandlerFactoryPtr factory); @@ -26,6 +28,7 @@ protected: private: ContextPtr context; + TCPServer & tcp_server; Poco::Net::HTTPServerParams::Ptr params; HTTPRequestHandlerFactoryPtr factory; bool stopped; diff --git a/src/Server/HTTP/HTTPServerConnectionFactory.cpp b/src/Server/HTTP/HTTPServerConnectionFactory.cpp index 0e4fb6cfcec..008da222c79 100644 --- a/src/Server/HTTP/HTTPServerConnectionFactory.cpp +++ b/src/Server/HTTP/HTTPServerConnectionFactory.cpp @@ -11,9 +11,9 @@ HTTPServerConnectionFactory::HTTPServerConnectionFactory( poco_check_ptr(factory); } -Poco::Net::TCPServerConnection * HTTPServerConnectionFactory::createConnection(const Poco::Net::StreamSocket & socket) +Poco::Net::TCPServerConnection * HTTPServerConnectionFactory::createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server) { - return new HTTPServerConnection(context, socket, params, factory); + return new HTTPServerConnection(context, tcp_server, socket, params, factory); } } diff --git a/src/Server/HTTP/HTTPServerConnectionFactory.h b/src/Server/HTTP/HTTPServerConnectionFactory.h index 3f11eca0f69..a19dc6d4d5c 100644 --- a/src/Server/HTTP/HTTPServerConnectionFactory.h +++ b/src/Server/HTTP/HTTPServerConnectionFactory.h @@ -2,19 +2,19 @@ #include #include +#include #include -#include namespace DB { -class HTTPServerConnectionFactory : public Poco::Net::TCPServerConnectionFactory +class HTTPServerConnectionFactory : public TCPServerConnectionFactory { public: HTTPServerConnectionFactory(ContextPtr context, Poco::Net::HTTPServerParams::Ptr params, HTTPRequestHandlerFactoryPtr factory); - Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket) override; + Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server) override; private: ContextPtr context; diff --git a/src/Server/KeeperTCPHandlerFactory.h b/src/Server/KeeperTCPHandlerFactory.h index 67bb3dab268..58dc73d7c27 100644 --- a/src/Server/KeeperTCPHandlerFactory.h +++ b/src/Server/KeeperTCPHandlerFactory.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include @@ -10,7 +10,7 @@ namespace DB { -class KeeperTCPHandlerFactory : public Poco::Net::TCPServerConnectionFactory +class KeeperTCPHandlerFactory : public TCPServerConnectionFactory { private: IServer & server; @@ -29,7 +29,7 @@ public: { } - Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket) override + Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer &) override { try { diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index deebc073ad5..2836ee05c30 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -62,10 +63,11 @@ static String showTableStatusReplacementQuery(const String & query); static String killConnectionIdReplacementQuery(const String & query); static String selectLimitReplacementQuery(const String & query); -MySQLHandler::MySQLHandler(IServer & server_, const Poco::Net::StreamSocket & socket_, +MySQLHandler::MySQLHandler(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_) : Poco::Net::TCPServerConnection(socket_) , server(server_) + , tcp_server(tcp_server_) , log(&Poco::Logger::get("MySQLHandler")) , connection_id(connection_id_) , auth_plugin(new MySQLProtocol::Authentication::Native41()) @@ -138,11 +140,14 @@ void MySQLHandler::run() OKPacket ok_packet(0, handshake_response.capability_flags, 0, 0, 0); packet_endpoint->sendPacket(ok_packet, true); - while (true) + while (tcp_server.isOpen()) { packet_endpoint->resetSequenceId(); MySQLPacketPayloadReadBuffer payload = packet_endpoint->getPayload(); + while (!in->poll(1000000)) + if (!tcp_server.isOpen()) + return; char command = 0; payload.readStrict(command); @@ -152,6 +157,8 @@ void MySQLHandler::run() LOG_DEBUG(log, "Received command: {}. Connection id: {}.", static_cast(static_cast(command)), connection_id); + if (!tcp_server.isOpen()) + return; try { switch (command) @@ -369,8 +376,8 @@ void MySQLHandler::finishHandshakeSSL( } #if USE_SSL -MySQLHandlerSSL::MySQLHandlerSSL(IServer & server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_, RSA & public_key_, RSA & private_key_) - : MySQLHandler(server_, socket_, ssl_enabled, connection_id_) +MySQLHandlerSSL::MySQLHandlerSSL(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_, RSA & public_key_, RSA & private_key_) + : MySQLHandler(server_, tcp_server_, socket_, ssl_enabled, connection_id_) , public_key(public_key_) , private_key(private_key_) {} diff --git a/src/Server/MySQLHandler.h b/src/Server/MySQLHandler.h index 7ef212bf36e..3af5f7a0eb2 100644 --- a/src/Server/MySQLHandler.h +++ b/src/Server/MySQLHandler.h @@ -24,11 +24,14 @@ namespace CurrentMetrics namespace DB { +class ReadBufferFromPocoSocket; +class TCPServer; + /// Handler for MySQL wire protocol connections. Allows to connect to ClickHouse using MySQL client. class MySQLHandler : public Poco::Net::TCPServerConnection { public: - MySQLHandler(IServer & server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_); + MySQLHandler(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_); void run() final; @@ -52,6 +55,7 @@ protected: virtual void finishHandshakeSSL(size_t packet_size, char * buf, size_t pos, std::function read_bytes, MySQLProtocol::ConnectionPhase::HandshakeResponse & packet); IServer & server; + TCPServer & tcp_server; Poco::Logger * log; UInt64 connection_id = 0; @@ -68,7 +72,7 @@ protected: Replacements replacements; std::unique_ptr auth_plugin; - std::shared_ptr in; + std::shared_ptr in; std::shared_ptr out; bool secure_connection = false; }; @@ -77,7 +81,7 @@ protected: class MySQLHandlerSSL : public MySQLHandler { public: - MySQLHandlerSSL(IServer & server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_, RSA & public_key_, RSA & private_key_); + MySQLHandlerSSL(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_, RSA & public_key_, RSA & private_key_); private: void authPluginSSL() override; diff --git a/src/Server/MySQLHandlerFactory.cpp b/src/Server/MySQLHandlerFactory.cpp index 7a0bfd8ab09..f7bb073e275 100644 --- a/src/Server/MySQLHandlerFactory.cpp +++ b/src/Server/MySQLHandlerFactory.cpp @@ -118,14 +118,14 @@ void MySQLHandlerFactory::generateRSAKeys() } #endif -Poco::Net::TCPServerConnection * MySQLHandlerFactory::createConnection(const Poco::Net::StreamSocket & socket) +Poco::Net::TCPServerConnection * MySQLHandlerFactory::createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server) { size_t connection_id = last_connection_id++; LOG_TRACE(log, "MySQL connection. Id: {}. Address: {}", connection_id, socket.peerAddress().toString()); #if USE_SSL - return new MySQLHandlerSSL(server, socket, ssl_enabled, connection_id, *public_key, *private_key); + return new MySQLHandlerSSL(server, tcp_server, socket, ssl_enabled, connection_id, *public_key, *private_key); #else - return new MySQLHandler(server, socket, ssl_enabled, connection_id); + return new MySQLHandler(server, tcp_server, socket, ssl_enabled, connection_id); #endif } diff --git a/src/Server/MySQLHandlerFactory.h b/src/Server/MySQLHandlerFactory.h index 106fdfdf341..25f1af85273 100644 --- a/src/Server/MySQLHandlerFactory.h +++ b/src/Server/MySQLHandlerFactory.h @@ -1,9 +1,9 @@ #pragma once -#include #include #include #include +#include #include @@ -13,8 +13,9 @@ namespace DB { +class TCPServer; -class MySQLHandlerFactory : public Poco::Net::TCPServerConnectionFactory +class MySQLHandlerFactory : public TCPServerConnectionFactory { private: IServer & server; @@ -43,7 +44,7 @@ public: void generateRSAKeys(); - Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket) override; + Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server) override; }; } diff --git a/src/Server/PostgreSQLHandler.cpp b/src/Server/PostgreSQLHandler.cpp index fee4ace3452..9808b538280 100644 --- a/src/Server/PostgreSQLHandler.cpp +++ b/src/Server/PostgreSQLHandler.cpp @@ -6,6 +6,7 @@ #include #include "PostgreSQLHandler.h" #include +#include #include #include #include @@ -28,11 +29,13 @@ namespace ErrorCodes PostgreSQLHandler::PostgreSQLHandler( const Poco::Net::StreamSocket & socket_, IServer & server_, + TCPServer & tcp_server_, bool ssl_enabled_, Int32 connection_id_, std::vector> & auth_methods_) : Poco::Net::TCPServerConnection(socket_) , server(server_) + , tcp_server(tcp_server_) , ssl_enabled(ssl_enabled_) , connection_id(connection_id_) , authentication_manager(auth_methods_) @@ -60,11 +63,18 @@ void PostgreSQLHandler::run() if (!startup()) return; - while (true) + while (tcp_server.isOpen()) { message_transport->send(PostgreSQLProtocol::Messaging::ReadyForQuery(), true); + + constexpr size_t connection_check_timeout = 1; // 1 second + while (!in->poll(1000000 * connection_check_timeout)) + if (!tcp_server.isOpen()) + return; PostgreSQLProtocol::Messaging::FrontMessageType message_type = message_transport->receiveMessageType(); + if (!tcp_server.isOpen()) + return; switch (message_type) { case PostgreSQLProtocol::Messaging::FrontMessageType::QUERY: diff --git a/src/Server/PostgreSQLHandler.h b/src/Server/PostgreSQLHandler.h index 1d33f41f255..4fd08cc2606 100644 --- a/src/Server/PostgreSQLHandler.h +++ b/src/Server/PostgreSQLHandler.h @@ -18,8 +18,9 @@ namespace CurrentMetrics namespace DB { - +class ReadBufferFromPocoSocket; class Session; +class TCPServer; /** PostgreSQL wire protocol implementation. * For more info see https://www.postgresql.org/docs/current/protocol.html @@ -30,6 +31,7 @@ public: PostgreSQLHandler( const Poco::Net::StreamSocket & socket_, IServer & server_, + TCPServer & tcp_server_, bool ssl_enabled_, Int32 connection_id_, std::vector> & auth_methods_); @@ -40,12 +42,13 @@ private: Poco::Logger * log = &Poco::Logger::get("PostgreSQLHandler"); IServer & server; + TCPServer & tcp_server; std::unique_ptr session; bool ssl_enabled = false; Int32 connection_id = 0; Int32 secret_key = 0; - std::shared_ptr in; + std::shared_ptr in; std::shared_ptr out; std::shared_ptr message_transport; diff --git a/src/Server/PostgreSQLHandlerFactory.cpp b/src/Server/PostgreSQLHandlerFactory.cpp index 1158cf5835e..6f2124861e7 100644 --- a/src/Server/PostgreSQLHandlerFactory.cpp +++ b/src/Server/PostgreSQLHandlerFactory.cpp @@ -1,5 +1,4 @@ #include "PostgreSQLHandlerFactory.h" -#include #include #include @@ -17,11 +16,11 @@ PostgreSQLHandlerFactory::PostgreSQLHandlerFactory(IServer & server_) }; } -Poco::Net::TCPServerConnection * PostgreSQLHandlerFactory::createConnection(const Poco::Net::StreamSocket & socket) +Poco::Net::TCPServerConnection * PostgreSQLHandlerFactory::createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server) { Int32 connection_id = last_connection_id++; LOG_TRACE(log, "PostgreSQL connection. Id: {}. Address: {}", connection_id, socket.peerAddress().toString()); - return new PostgreSQLHandler(socket, server, ssl_enabled, connection_id, auth_methods); + return new PostgreSQLHandler(socket, server, tcp_server, ssl_enabled, connection_id, auth_methods); } } diff --git a/src/Server/PostgreSQLHandlerFactory.h b/src/Server/PostgreSQLHandlerFactory.h index dc3d4047d2a..e9241da6f0e 100644 --- a/src/Server/PostgreSQLHandlerFactory.h +++ b/src/Server/PostgreSQLHandlerFactory.h @@ -1,16 +1,16 @@ #pragma once -#include #include #include #include +#include #include #include namespace DB { -class PostgreSQLHandlerFactory : public Poco::Net::TCPServerConnectionFactory +class PostgreSQLHandlerFactory : public TCPServerConnectionFactory { private: IServer & server; @@ -28,6 +28,6 @@ private: public: explicit PostgreSQLHandlerFactory(IServer & server_); - Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket) override; + Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer & server) override; }; } diff --git a/src/Server/ProtocolServerAdapter.cpp b/src/Server/ProtocolServerAdapter.cpp index 6ec1ec572f7..b41ad2376f1 100644 --- a/src/Server/ProtocolServerAdapter.cpp +++ b/src/Server/ProtocolServerAdapter.cpp @@ -1,5 +1,5 @@ #include -#include +#include #if USE_GRPC #include @@ -11,20 +11,29 @@ namespace DB class ProtocolServerAdapter::TCPServerAdapterImpl : public Impl { public: - explicit TCPServerAdapterImpl(std::unique_ptr tcp_server_) : tcp_server(std::move(tcp_server_)) {} + explicit TCPServerAdapterImpl(std::unique_ptr tcp_server_) : tcp_server(std::move(tcp_server_)) {} ~TCPServerAdapterImpl() override = default; void start() override { tcp_server->start(); } void stop() override { tcp_server->stop(); } + bool isStopping() const override { return !tcp_server->isOpen(); } + UInt16 portNumber() const override { return tcp_server->portNumber(); } size_t currentConnections() const override { return tcp_server->currentConnections(); } size_t currentThreads() const override { return tcp_server->currentThreads(); } private: - std::unique_ptr tcp_server; + std::unique_ptr tcp_server; }; -ProtocolServerAdapter::ProtocolServerAdapter(const char * port_name_, std::unique_ptr tcp_server_) - : port_name(port_name_), impl(std::make_unique(std::move(tcp_server_))) +ProtocolServerAdapter::ProtocolServerAdapter( + const std::string & listen_host_, + const char * port_name_, + const std::string & description_, + std::unique_ptr tcp_server_) + : listen_host(listen_host_) + , port_name(port_name_) + , description(description_) + , impl(std::make_unique(std::move(tcp_server_))) { } @@ -36,16 +45,30 @@ public: ~GRPCServerAdapterImpl() override = default; void start() override { grpc_server->start(); } - void stop() override { grpc_server->stop(); } + void stop() override + { + is_stopping = true; + grpc_server->stop(); + } + bool isStopping() const override { return is_stopping; } + UInt16 portNumber() const override { return grpc_server->portNumber(); } size_t currentConnections() const override { return grpc_server->currentConnections(); } size_t currentThreads() const override { return grpc_server->currentThreads(); } private: std::unique_ptr grpc_server; + bool is_stopping = false; }; -ProtocolServerAdapter::ProtocolServerAdapter(const char * port_name_, std::unique_ptr grpc_server_) - : port_name(port_name_), impl(std::make_unique(std::move(grpc_server_))) +ProtocolServerAdapter::ProtocolServerAdapter( + const std::string & listen_host_, + const char * port_name_, + const std::string & description_, + std::unique_ptr grpc_server_) + : listen_host(listen_host_) + , port_name(port_name_) + , description(description_) + , impl(std::make_unique(std::move(grpc_server_))) { } #endif diff --git a/src/Server/ProtocolServerAdapter.h b/src/Server/ProtocolServerAdapter.h index 04c46b53356..9b3b1af0301 100644 --- a/src/Server/ProtocolServerAdapter.h +++ b/src/Server/ProtocolServerAdapter.h @@ -2,14 +2,14 @@ #include +#include #include #include -namespace Poco::Net { class TCPServer; } - namespace DB { class GRPCServer; +class TCPServer; /// Provides an unified interface to access a protocol implementing server /// no matter what type it has (HTTPServer, TCPServer, MySQLServer, GRPCServer, ...). @@ -19,10 +19,10 @@ class ProtocolServerAdapter public: ProtocolServerAdapter(ProtocolServerAdapter && src) = default; ProtocolServerAdapter & operator =(ProtocolServerAdapter && src) = default; - ProtocolServerAdapter(const char * port_name_, std::unique_ptr tcp_server_); + ProtocolServerAdapter(const std::string & listen_host_, const char * port_name_, const std::string & description_, std::unique_ptr tcp_server_); #if USE_GRPC - ProtocolServerAdapter(const char * port_name_, std::unique_ptr grpc_server_); + ProtocolServerAdapter(const std::string & listen_host_, const char * port_name_, const std::string & description_, std::unique_ptr grpc_server_); #endif /// Starts the server. A new thread will be created that waits for and accepts incoming connections. @@ -31,14 +31,23 @@ public: /// Stops the server. No new connections will be accepted. void stop() { impl->stop(); } + bool isStopping() const { return impl->isStopping(); } + /// Returns the number of currently handled connections. size_t currentConnections() const { return impl->currentConnections(); } /// Returns the number of current threads. size_t currentThreads() const { return impl->currentThreads(); } + /// Returns the port this server is listening to. + UInt16 portNumber() const { return impl->portNumber(); } + + const std::string & getListenHost() const { return listen_host; } + const std::string & getPortName() const { return port_name; } + const std::string & getDescription() const { return description; } + private: class Impl { @@ -46,13 +55,17 @@ private: virtual ~Impl() {} virtual void start() = 0; virtual void stop() = 0; + virtual bool isStopping() const = 0; + virtual UInt16 portNumber() const = 0; virtual size_t currentConnections() const = 0; virtual size_t currentThreads() const = 0; }; class TCPServerAdapterImpl; class GRPCServerAdapterImpl; + std::string listen_host; std::string port_name; + std::string description; std::unique_ptr impl; }; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index cdf1838c06b..c2dcd5d7222 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -81,9 +82,10 @@ namespace ErrorCodes extern const int UNKNOWN_PROTOCOL; } -TCPHandler::TCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_, bool parse_proxy_protocol_, std::string server_display_name_) +TCPHandler::TCPHandler(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool parse_proxy_protocol_, std::string server_display_name_) : Poco::Net::TCPServerConnection(socket_) , server(server_) + , tcp_server(tcp_server_) , parse_proxy_protocol(parse_proxy_protocol_) , log(&Poco::Logger::get("TCPHandler")) , server_display_name(std::move(server_display_name_)) @@ -172,13 +174,13 @@ void TCPHandler::runImpl() throw; } - while (true) + while (tcp_server.isOpen()) { /// We are waiting for a packet from the client. Thus, every `poll_interval` seconds check whether we need to shut down. { Stopwatch idle_time; UInt64 timeout_ms = std::min(poll_interval, idle_connection_timeout) * 1000000; - while (!server.isCancelled() && !static_cast(*in).poll(timeout_ms)) + while (tcp_server.isOpen() && !server.isCancelled() && !static_cast(*in).poll(timeout_ms)) { if (idle_time.elapsedSeconds() > idle_connection_timeout) { @@ -189,7 +191,7 @@ void TCPHandler::runImpl() } /// If we need to shut down, or client disconnects. - if (server.isCancelled() || in->eof()) + if (!tcp_server.isOpen() || server.isCancelled() || in->eof()) break; Stopwatch watch; diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 4a340e328ed..791222dd0dc 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -35,6 +35,7 @@ class Session; struct Settings; class ColumnsDescription; struct ProfileInfo; +class TCPServer; /// State of query processing. struct QueryState @@ -127,7 +128,7 @@ public: * because it allows to check the IP ranges of the trusted proxy. * Proxy-forwarded (original client) IP address is used for quota accounting if quota is keyed by forwarded IP. */ - TCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_, bool parse_proxy_protocol_, std::string server_display_name_); + TCPHandler(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool parse_proxy_protocol_, std::string server_display_name_); ~TCPHandler() override; void run() override; @@ -137,6 +138,7 @@ public: private: IServer & server; + TCPServer & tcp_server; bool parse_proxy_protocol = false; Poco::Logger * log; diff --git a/src/Server/TCPHandlerFactory.h b/src/Server/TCPHandlerFactory.h index e610bea330c..03b2592198d 100644 --- a/src/Server/TCPHandlerFactory.h +++ b/src/Server/TCPHandlerFactory.h @@ -1,17 +1,17 @@ #pragma once -#include #include #include #include #include +#include namespace Poco { class Logger; } namespace DB { -class TCPHandlerFactory : public Poco::Net::TCPServerConnectionFactory +class TCPHandlerFactory : public TCPServerConnectionFactory { private: IServer & server; @@ -38,13 +38,13 @@ public: server_display_name = server.config().getString("display_name", getFQDNOrHostName()); } - Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket) override + Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server) override { try { LOG_TRACE(log, "TCP Request. Address: {}", socket.peerAddress().toString()); - return new TCPHandler(server, socket, parse_proxy_protocol, server_display_name); + return new TCPHandler(server, tcp_server, socket, parse_proxy_protocol, server_display_name); } catch (const Poco::Net::NetException &) { diff --git a/src/Server/TCPServer.cpp b/src/Server/TCPServer.cpp new file mode 100644 index 00000000000..380c4ef9924 --- /dev/null +++ b/src/Server/TCPServer.cpp @@ -0,0 +1,36 @@ +#include +#include + +namespace DB +{ + +class TCPServerConnectionFactoryImpl : public Poco::Net::TCPServerConnectionFactory +{ +public: + TCPServerConnectionFactoryImpl(TCPServer & tcp_server_, DB::TCPServerConnectionFactory::Ptr factory_) + : tcp_server(tcp_server_) + , factory(factory_) + {} + + Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket) override + { + return factory->createConnection(socket, tcp_server); + } +private: + TCPServer & tcp_server; + DB::TCPServerConnectionFactory::Ptr factory; +}; + +TCPServer::TCPServer( + TCPServerConnectionFactory::Ptr factory_, + Poco::ThreadPool & thread_pool, + Poco::Net::ServerSocket & socket_, + Poco::Net::TCPServerParams::Ptr params) + : Poco::Net::TCPServer(new TCPServerConnectionFactoryImpl(*this, factory_), thread_pool, socket_, params) + , factory(factory_) + , socket(socket_) + , is_open(true) + , port_number(socket.address().port()) +{} + +} diff --git a/src/Server/TCPServer.h b/src/Server/TCPServer.h new file mode 100644 index 00000000000..219fed5342b --- /dev/null +++ b/src/Server/TCPServer.h @@ -0,0 +1,47 @@ +#pragma once + +#include + +#include +#include + + +namespace DB +{ +class Context; + +class TCPServer : public Poco::Net::TCPServer +{ +public: + explicit TCPServer( + TCPServerConnectionFactory::Ptr factory, + Poco::ThreadPool & thread_pool, + Poco::Net::ServerSocket & socket, + Poco::Net::TCPServerParams::Ptr params = new Poco::Net::TCPServerParams); + + /// Close the socket and ask existing connections to stop serving queries + void stop() + { + Poco::Net::TCPServer::stop(); + // This notifies already established connections that they should stop serving + // queries and close their socket as soon as they can. + is_open = false; + // Poco's stop() stops listening on the socket but leaves it open. + // To be able to hand over control of the listening port to a new server, and + // to get fast connection refusal instead of timeouts, we also need to close + // the listening socket. + socket.close(); + } + + bool isOpen() const { return is_open; } + + UInt16 portNumber() const { return port_number; } + +private: + TCPServerConnectionFactory::Ptr factory; + Poco::Net::ServerSocket socket; + std::atomic is_open; + UInt16 port_number; +}; + +} diff --git a/src/Server/TCPServerConnectionFactory.h b/src/Server/TCPServerConnectionFactory.h new file mode 100644 index 00000000000..613f98352bd --- /dev/null +++ b/src/Server/TCPServerConnectionFactory.h @@ -0,0 +1,27 @@ +#pragma once + +#include + +namespace Poco +{ +namespace Net +{ + class StreamSocket; + class TCPServerConnection; +} +} +namespace DB +{ +class TCPServer; + +class TCPServerConnectionFactory +{ +public: + using Ptr = Poco::SharedPtr; + + virtual ~TCPServerConnectionFactory() = default; + + /// Same as Poco::Net::TCPServerConnectionFactory except we can pass the TCPServer + virtual Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server) = 0; +}; +} diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index ff945068732..d113c825205 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2045,7 +2045,8 @@ class ClickHouseInstance: user=user, password=password, database=database) # Connects to the instance via HTTP interface, sends a query and returns the answer - def http_query(self, sql, data=None, params=None, user=None, password=None, expect_fail_and_get_error=False): + def http_query(self, sql, data=None, params=None, user=None, password=None, expect_fail_and_get_error=False, + port=8123, timeout=None, retry_strategy=None): logging.debug(f"Executing query {sql} on {self.name} via HTTP interface") if params is None: params = {} @@ -2059,12 +2060,19 @@ class ClickHouseInstance: auth = requests.auth.HTTPBasicAuth(user, password) elif user: auth = requests.auth.HTTPBasicAuth(user, '') - url = "http://" + self.ip_address + ":8123/?" + urllib.parse.urlencode(params) + url = f"http://{self.ip_address}:{port}/?" + urllib.parse.urlencode(params) - if data: - r = requests.post(url, data, auth=auth) + if retry_strategy is None: + requester = requests else: - r = requests.get(url, auth=auth) + adapter = requests.adapters.HTTPAdapter(max_retries=retry_strategy) + requester = requests.Session() + requester.mount("https://", adapter) + requester.mount("http://", adapter) + if data: + r = requester.post(url, data, auth=auth, timeout=timeout) + else: + r = requester.get(url, auth=auth, timeout=timeout) def http_code_and_message(): code = r.status_code diff --git a/tests/integration/test_server_reload/.gitignore b/tests/integration/test_server_reload/.gitignore new file mode 100644 index 00000000000..edf565ec632 --- /dev/null +++ b/tests/integration/test_server_reload/.gitignore @@ -0,0 +1 @@ +_gen diff --git a/tests/integration/test_server_reload/__init__.py b/tests/integration/test_server_reload/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_server_reload/configs/default_passwd.xml b/tests/integration/test_server_reload/configs/default_passwd.xml new file mode 100644 index 00000000000..5c23be0dcb0 --- /dev/null +++ b/tests/integration/test_server_reload/configs/default_passwd.xml @@ -0,0 +1,13 @@ + + + + + + + + + + 123 + + + diff --git a/tests/integration/test_server_reload/configs/dhparam.pem b/tests/integration/test_server_reload/configs/dhparam.pem new file mode 100644 index 00000000000..fb935b9c898 --- /dev/null +++ b/tests/integration/test_server_reload/configs/dhparam.pem @@ -0,0 +1,8 @@ +-----BEGIN DH PARAMETERS----- +MIIBCAKCAQEAkPGhfLY5nppeQkFBKYRpiisxzrRQfyyTUu6aabZP2CbAMAuoYzaC +Z+iqeWSQZKRYeA21SZXkC9xE1e5FJsc5IWzCRiMNZeLuj4ApUNysMu89DpX8/b91 ++Ka6wRJnaO43ZqHj/9FpU4JiYtxoIpXDC9HeiSAnwLwJc3L+nkYfnSGgvzWIxhGV +gCoVmVBoTe7wrqCyVlM5nrNZSjhlSugvXmu2bSK3MwYF08QLKvlF68eedbs0PMWh +WC0bFM/X7gMBEqL4DiINufAShbZPKxD6eL2APiHPUo6xun3ed/Po/5j8QBmiku0c +5Jb12ZhOTRTQjaRg2aFF8LPdW2tDE7HmewIBAg== +-----END DH PARAMETERS----- diff --git a/tests/integration/test_server_reload/configs/ports_from_zk.xml b/tests/integration/test_server_reload/configs/ports_from_zk.xml new file mode 100644 index 00000000000..ae3435a3d3c --- /dev/null +++ b/tests/integration/test_server_reload/configs/ports_from_zk.xml @@ -0,0 +1,9 @@ + + + + + + + + + \ No newline at end of file diff --git a/tests/integration/test_server_reload/configs/server.crt b/tests/integration/test_server_reload/configs/server.crt new file mode 100644 index 00000000000..6f4deca038f --- /dev/null +++ b/tests/integration/test_server_reload/configs/server.crt @@ -0,0 +1,18 @@ +-----BEGIN CERTIFICATE----- +MIIC+zCCAeOgAwIBAgIJAIhI9ozZJ+TWMA0GCSqGSIb3DQEBCwUAMBQxEjAQBgNV +BAMMCWxvY2FsaG9zdDAeFw0xOTA0MjIwNDMyNTJaFw0yMDA0MjEwNDMyNTJaMBQx +EjAQBgNVBAMMCWxvY2FsaG9zdDCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoC +ggEBAK+wVUEdqF2uXvN0MJBgnAHyXi6JTi4p/F6igsrCjSNjJWzHH0vQmK8ujfcF +CkifW88i+W5eHctuEtQqNHK+t9x9YiZtXrj6m/XkOXs20mYgENSmbbbHbriTPnZB +zZrq6UqMlwIHNNAa+I3NMORQxVRaI0ybXnGVO5elr70xHpk03xL0JWKHpEqYp4db +2aBQgF6y3Ww4khxjIYqpUYXWXGFnVIRU7FKVEAM1xyKqvQzXjQ5sVM/wyHknveEF +3b/X4ggN+KNl5KOc0cWDh1/XaatJAPaUUPqZcq76tynLbP64Xm3dxHcj+gtRkO67 +ef6MSg6l63m3XQP6Qb+MIkd06OsCAwEAAaNQME4wHQYDVR0OBBYEFDmODTO8QLDN +ykR3x0LIOnjNhrKhMB8GA1UdIwQYMBaAFDmODTO8QLDNykR3x0LIOnjNhrKhMAwG +A1UdEwQFMAMBAf8wDQYJKoZIhvcNAQELBQADggEBAAwaiJc7uqEpnH3aukbftDwX +m8GfEnj1HVdgg+9GGNq+9rvUYBF6gdPmjRCX9dO0cclLFx8jc2org0rTSq9WoOhX +E6qL4Eqrmc5SE3Y9jZM0h6GRD4oXK014FmtZ3T6ddZU3dQLj3BS2r1XrvmubTvGN +ZuTJNY8nx8Hh6H5XINmsEjUF9E5hog+PwCE03xt2adIdYL+gsbxASeNYyeUFpZv5 +zcXR3VoakBWnAaOVgCHq2qh96QAnL7ZKzFkGf/MdwV10KU3dmb+ICbQUUdf9Gc17 +aaDCIRws312F433FdXBkGs2UkB7ZZme9dfn6O1QbeTNvex2VLMqYx/CTkfFbOQA= +-----END CERTIFICATE----- diff --git a/tests/integration/test_server_reload/configs/server.key b/tests/integration/test_server_reload/configs/server.key new file mode 100644 index 00000000000..6eddb3295db --- /dev/null +++ b/tests/integration/test_server_reload/configs/server.key @@ -0,0 +1,28 @@ +-----BEGIN PRIVATE KEY----- +MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQCvsFVBHahdrl7z +dDCQYJwB8l4uiU4uKfxeooLKwo0jYyVsxx9L0JivLo33BQpIn1vPIvluXh3LbhLU +KjRyvrfcfWImbV64+pv15Dl7NtJmIBDUpm22x264kz52Qc2a6ulKjJcCBzTQGviN +zTDkUMVUWiNMm15xlTuXpa+9MR6ZNN8S9CVih6RKmKeHW9mgUIBest1sOJIcYyGK +qVGF1lxhZ1SEVOxSlRADNcciqr0M140ObFTP8Mh5J73hBd2/1+IIDfijZeSjnNHF +g4df12mrSQD2lFD6mXKu+rcpy2z+uF5t3cR3I/oLUZDuu3n+jEoOpet5t10D+kG/ +jCJHdOjrAgMBAAECggEARF66zrxb6RkSmmt8+rKeA6PuQu3sHsr4C1vyyjUr97l9 +tvdGlpp20LWtSZQMjHZ3pARYTTsTHTeY3DgQcRcHNicVKx8k3ZepWeeW9vw+pL+V +zSt3RsoVrH6gsCSrfr4sS3aqzX9AbjwQvh48CJ3mLQ1m70kHV+xbZIh1+4pB/hyP +1wKyUE18ZkOptXvO/TtoHzLQCecpkXtWzmry1Eh2isvXA+NMrAtLibGsyM1mtm7i +5ozevzHabvvCDBEe+KgZdONgVhhhvm2eOd+/s4w3rw4ETud4fI/ZAJyWXhiIKFnA +VJbElWruSAoVBW7p2bsF5PbmVzvo8vXL+VylxYD+AQKBgQDhLoRKTVhNkn/QjKxq +sdOh+QZra0LzjVpAmkQzu7wZMSHEz9qePQciDQQrYKrmRF1vNcIRCVUTqWYheJ/1 +lKRrCGa0ab6k96zkWMqLHD5u+UeJV7r1dJIx08ME9kNJ+x/XtB8klRIji16NiQUS +qc6p8z0M2AnbJzsRfWZRH8FeYwKBgQDHu8dzdtVGI7MtxfPOE/bfajiopDg8BdTC +pdug2T8XofRHRq7Q+0vYjTAZFT/slib91Pk6VvvPdo9VBZiL4omv4dAq6mOOdX/c +U14mJe1X5GCrr8ExZ8BfNJ3t/6sV1fcxyJwAw7iBguqxA2JqdM/wFk10K8XqvzVn +CD6O9yGt2QKBgFX1BMi8N538809vs41S7l9hCQNOQZNo/O+2M5yv6ECRkbtoQKKw +1x03bMUGNJaLuELweXE5Z8GGo5bZTe5X3F+DKHlr+DtO1C+ieUaa9HY2MAmMdLCn +2/qrREGLo+oEs4YKmuzC/taUp/ZNPKOAMISNdluFyFVg51pozPrgrVbTAoGBAKkE +LBl3O67o0t0vH8sJdeVFG8EJhlS0koBMnfgVHqC++dm+5HwPyvTrNQJkyv1HaqNt +r6FArkG3ED9gRuBIyT6+lctbIPgSUip9mbQqcBfqOCvQxGksZMur2ODncz09HLtS +CUFUXjOqNzOnq4ZuZu/Bz7U4vXiSaXxQq6+LTUKxAoGAFZU/qrI06XxnrE9A1X0W +l7DSkpZaDcu11NrZ473yONih/xOZNh4SSBpX8a7F6Pmh9BdtGqphML8NFPvQKcfP +b9H2iid2tc292uyrUEb5uTMmv61zoTwtitqLzO0+tS6PT3fXobX+eyeEWKzPBljL +HFtxG5CCXpkdnWRmaJnhTzA= +-----END PRIVATE KEY----- diff --git a/tests/integration/test_server_reload/configs/ssl_conf.xml b/tests/integration/test_server_reload/configs/ssl_conf.xml new file mode 100644 index 00000000000..43b25032059 --- /dev/null +++ b/tests/integration/test_server_reload/configs/ssl_conf.xml @@ -0,0 +1,18 @@ + + + + + + + /etc/clickhouse-server/config.d/server.crt + /etc/clickhouse-server/config.d/server.key + + /etc/clickhouse-server/config.d/dhparam.pem + none + true + true + sslv2,sslv3 + true + + + diff --git a/tests/integration/test_server_reload/protos/clickhouse_grpc.proto b/tests/integration/test_server_reload/protos/clickhouse_grpc.proto new file mode 100644 index 00000000000..c6cafaf6e40 --- /dev/null +++ b/tests/integration/test_server_reload/protos/clickhouse_grpc.proto @@ -0,0 +1,174 @@ +/* This file describes gRPC protocol supported in ClickHouse. + * + * To use this protocol a client should send one or more messages of the QueryInfo type + * and then receive one or more messages of the Result type. + * According to that the service provides four methods for that: + * ExecuteQuery(QueryInfo) returns (Result) + * ExecuteQueryWithStreamInput(stream QueryInfo) returns (Result) + * ExecuteQueryWithStreamOutput(QueryInfo) returns (stream Result) + * ExecuteQueryWithStreamIO(stream QueryInfo) returns (stream Result) + * It's up to the client to choose which method to use. + * For example, ExecuteQueryWithStreamInput() allows the client to add data multiple times + * while executing a query, which is suitable for inserting many rows. + */ + +syntax = "proto3"; + +package clickhouse.grpc; + +message NameAndType { + string name = 1; + string type = 2; +} + +// Describes an external table - a table which will exists only while a query is executing. +message ExternalTable { + // Name of the table. If omitted, "_data" is used. + string name = 1; + + // Columns of the table. Types are required, names can be omitted. If the names are omitted, "_1", "_2", ... is used. + repeated NameAndType columns = 2; + + // Data to insert to the external table. + // If a method with streaming input (i.e. ExecuteQueryWithStreamInput() or ExecuteQueryWithStreamIO()) is used, + // then data for insertion to the same external table can be split between multiple QueryInfos. + bytes data = 3; + + // Format of the data to insert to the external table. + string format = 4; + + // Settings for executing that insertion, applied after QueryInfo.settings. + map settings = 5; +} + +enum CompressionAlgorithm { + NO_COMPRESSION = 0; + DEFLATE = 1; + GZIP = 2; + STREAM_GZIP = 3; +} + +enum CompressionLevel { + COMPRESSION_NONE = 0; + COMPRESSION_LOW = 1; + COMPRESSION_MEDIUM = 2; + COMPRESSION_HIGH = 3; +} + +message Compression { + CompressionAlgorithm algorithm = 1; + CompressionLevel level = 2; +} + +// Information about a query which a client sends to a ClickHouse server. +// The first QueryInfo can set any of the following fields. Extra QueryInfos only add extra data. +// In extra QueryInfos only `input_data`, `external_tables`, `next_query_info` and `cancel` fields can be set. +message QueryInfo { + string query = 1; + string query_id = 2; + map settings = 3; + + // Default database. + string database = 4; + + // Input data, used both as data for INSERT query and as data for the input() function. + bytes input_data = 5; + + // Delimiter for input_data, inserted between input_data from adjacent QueryInfos. + bytes input_data_delimiter = 6; + + // Default output format. If not specified, 'TabSeparated' is used. + string output_format = 7; + + repeated ExternalTable external_tables = 8; + + string user_name = 9; + string password = 10; + string quota = 11; + + // Works exactly like sessions in the HTTP protocol. + string session_id = 12; + bool session_check = 13; + uint32 session_timeout = 14; + + // Set `cancel` to true to stop executing the query. + bool cancel = 15; + + // If true there will be at least one more QueryInfo in the input stream. + // `next_query_info` is allowed to be set only if a method with streaming input (i.e. ExecuteQueryWithStreamInput() or ExecuteQueryWithStreamIO()) is used. + bool next_query_info = 16; + + /// Controls how a ClickHouse server will compress query execution results before sending back to the client. + /// If not set the compression settings from the configuration file will be used. + Compression result_compression = 17; +} + +enum LogsLevel { + LOG_NONE = 0; + LOG_FATAL = 1; + LOG_CRITICAL = 2; + LOG_ERROR = 3; + LOG_WARNING = 4; + LOG_NOTICE = 5; + LOG_INFORMATION = 6; + LOG_DEBUG = 7; + LOG_TRACE = 8; +} + +message LogEntry { + uint32 time = 1; + uint32 time_microseconds = 2; + uint64 thread_id = 3; + string query_id = 4; + LogsLevel level = 5; + string source = 6; + string text = 7; +} + +message Progress { + uint64 read_rows = 1; + uint64 read_bytes = 2; + uint64 total_rows_to_read = 3; + uint64 written_rows = 4; + uint64 written_bytes = 5; +} + +message Stats { + uint64 rows = 1; + uint64 blocks = 2; + uint64 allocated_bytes = 3; + bool applied_limit = 4; + uint64 rows_before_limit = 5; +} + +message Exception { + int32 code = 1; + string name = 2; + string display_text = 3; + string stack_trace = 4; +} + +// Result of execution of a query which is sent back by the ClickHouse server to the client. +message Result { + // Output of the query, represented in the `output_format` or in a format specified in `query`. + bytes output = 1; + bytes totals = 2; + bytes extremes = 3; + + repeated LogEntry logs = 4; + Progress progress = 5; + Stats stats = 6; + + // Set by the ClickHouse server if there was an exception thrown while executing. + Exception exception = 7; + + // Set by the ClickHouse server if executing was cancelled by the `cancel` field in QueryInfo. + bool cancelled = 8; +} + +service ClickHouse { + rpc ExecuteQuery(QueryInfo) returns (Result) {} + rpc ExecuteQueryWithStreamInput(stream QueryInfo) returns (Result) {} + rpc ExecuteQueryWithStreamOutput(QueryInfo) returns (stream Result) {} + rpc ExecuteQueryWithStreamIO(stream QueryInfo) returns (stream Result) {} +} diff --git a/tests/integration/test_server_reload/test.py b/tests/integration/test_server_reload/test.py new file mode 100644 index 00000000000..3c22b476f64 --- /dev/null +++ b/tests/integration/test_server_reload/test.py @@ -0,0 +1,284 @@ +import contextlib +import grpc +import psycopg2 +import pymysql.connections +import pymysql.err +import pytest +import sys +import time +from helpers.cluster import ClickHouseCluster, run_and_check +from helpers.client import Client, QueryRuntimeException +from kazoo.exceptions import NodeExistsError +from pathlib import Path +from requests.exceptions import ConnectionError +from urllib3.util.retry import Retry + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "instance", + main_configs=[ + "configs/ports_from_zk.xml", "configs/ssl_conf.xml", "configs/dhparam.pem", "configs/server.crt", "configs/server.key" + ], + user_configs=["configs/default_passwd.xml"], + with_zookeeper=True) + + +LOADS_QUERY = "SELECT value FROM system.events WHERE event = 'MainConfigLoads'" + + +# Use grpcio-tools to generate *pb2.py files from *.proto. + +proto_dir = Path(__file__).parent / "protos" +gen_dir = Path(__file__).parent / "_gen" +gen_dir.mkdir(exist_ok=True) +run_and_check( + f"python3 -m grpc_tools.protoc -I{proto_dir!s} --python_out={gen_dir!s} --grpc_python_out={gen_dir!s} \ + {proto_dir!s}/clickhouse_grpc.proto", shell=True) + +sys.path.append(str(gen_dir)) +import clickhouse_grpc_pb2 +import clickhouse_grpc_pb2_grpc + + +@pytest.fixture(name="cluster", scope="module") +def fixture_cluster(): + try: + cluster.add_zookeeper_startup_command(configure_ports_from_zk) + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(name="zk", scope="module") +def fixture_zk(cluster): + return cluster.get_kazoo_client("zoo1") + + +def get_client(cluster, port): + return Client(host=cluster.get_instance_ip("instance"), port=port, command=cluster.client_bin_path) + + +def get_mysql_client(cluster, port): + start_time = time.monotonic() + while True: + try: + return pymysql.connections.Connection( + host=cluster.get_instance_ip("instance"), user="default", password="", database="default", port=port) + except pymysql.err.OperationalError: + if time.monotonic() - start_time > 10: + raise + time.sleep(0.1) + + +def get_pgsql_client(cluster, port): + start_time = time.monotonic() + while True: + try: + return psycopg2.connect( + host=cluster.get_instance_ip("instance"), user="postgresql", password="123", database="default", port=port) + except psycopg2.OperationalError: + if time.monotonic() - start_time > 10: + raise + time.sleep(0.1) + + +def get_grpc_channel(cluster, port): + host_port = cluster.get_instance_ip("instance") + f":{port}" + channel = grpc.insecure_channel(host_port) + grpc.channel_ready_future(channel).result(timeout=10) + return channel + + +def grpc_query(channel, query_text): + query_info = clickhouse_grpc_pb2.QueryInfo(query=query_text) + stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel) + result = stub.ExecuteQuery(query_info) + if result and result.HasField("exception"): + raise Exception(result.exception.display_text) + return result.output.decode() + + +def configure_ports_from_zk(zk, querier=None): + default_config = [ + ("/clickhouse/listen_hosts", b"0.0.0.0"), + ("/clickhouse/ports/tcp", b"9000"), + ("/clickhouse/ports/http", b"8123"), + ("/clickhouse/ports/mysql", b"9004"), + ("/clickhouse/ports/postgresql", b"9005"), + ("/clickhouse/ports/grpc", b"9100"), + ] + for path, value in default_config: + if querier is not None: + loads_before = querier(LOADS_QUERY) + has_changed = False + try: + zk.create(path=path, value=value, makepath=True) + has_changed = True + except NodeExistsError: + if zk.get(path) != value: + zk.set(path=path, value=value) + has_changed = True + if has_changed and querier is not None: + wait_loaded_config_changed(loads_before, querier) + + +@contextlib.contextmanager +def sync_loaded_config(querier): + # Depending on whether we test a change on tcp or http + # we monitor canges using the other, untouched, protocol + loads_before = querier(LOADS_QUERY) + yield + wait_loaded_config_changed(loads_before, querier) + + +def wait_loaded_config_changed(loads_before, querier): + loads_after = None + start_time = time.monotonic() + while time.monotonic() - start_time < 10: + try: + loads_after = querier(LOADS_QUERY) + if loads_after != loads_before: + return + except (QueryRuntimeException, ConnectionError): + pass + time.sleep(0.1) + assert loads_after is not None and loads_after != loads_before + + +@contextlib.contextmanager +def default_client(cluster, zk, restore_via_http=False): + client = get_client(cluster, port=9000) + try: + yield client + finally: + querier = instance.http_query if restore_via_http else client.query + configure_ports_from_zk(zk, querier) + + +def test_change_tcp_port(cluster, zk): + with default_client(cluster, zk, restore_via_http=True) as client: + assert client.query("SELECT 1") == "1\n" + with sync_loaded_config(instance.http_query): + zk.set("/clickhouse/ports/tcp", b"9090") + with pytest.raises(QueryRuntimeException, match="Connection refused"): + client.query("SELECT 1") + client_on_new_port = get_client(cluster, port=9090) + assert client_on_new_port.query("SELECT 1") == "1\n" + + +def test_change_http_port(cluster, zk): + with default_client(cluster, zk) as client: + retry_strategy = Retry(total=10, backoff_factor=0.1) + assert instance.http_query("SELECT 1", retry_strategy=retry_strategy) == "1\n" + with sync_loaded_config(client.query): + zk.set("/clickhouse/ports/http", b"9090") + with pytest.raises(ConnectionError, match="Connection refused"): + instance.http_query("SELECT 1") + instance.http_query("SELECT 1", port=9090) == "1\n" + + +def test_change_mysql_port(cluster, zk): + with default_client(cluster, zk) as client: + mysql_client = get_mysql_client(cluster, port=9004) + assert mysql_client.query("SELECT 1") == 1 + with sync_loaded_config(client.query): + zk.set("/clickhouse/ports/mysql", b"9090") + with pytest.raises(pymysql.err.OperationalError, match="Lost connection"): + mysql_client.query("SELECT 1") + mysql_client_on_new_port = get_mysql_client(cluster, port=9090) + assert mysql_client_on_new_port.query("SELECT 1") == 1 + + +def test_change_postgresql_port(cluster, zk): + with default_client(cluster, zk) as client: + pgsql_client = get_pgsql_client(cluster, port=9005) + cursor = pgsql_client.cursor() + cursor.execute("SELECT 1") + assert cursor.fetchall() == [(1,)] + with sync_loaded_config(client.query): + zk.set("/clickhouse/ports/postgresql", b"9090") + with pytest.raises(psycopg2.OperationalError, match="closed"): + cursor.execute("SELECT 1") + pgsql_client_on_new_port = get_pgsql_client(cluster, port=9090) + cursor = pgsql_client_on_new_port.cursor() + cursor.execute("SELECT 1") + cursor.fetchall() == [(1,)] + + +def test_change_grpc_port(cluster, zk): + with default_client(cluster, zk) as client: + grpc_channel = get_grpc_channel(cluster, port=9100) + assert grpc_query(grpc_channel, "SELECT 1") == "1\n" + with sync_loaded_config(client.query): + zk.set("/clickhouse/ports/grpc", b"9090") + with pytest.raises(grpc._channel._InactiveRpcError, match="StatusCode.UNAVAILABLE"): + grpc_query(grpc_channel, "SELECT 1") + grpc_channel_on_new_port = get_grpc_channel(cluster, port=9090) + assert grpc_query(grpc_channel_on_new_port, "SELECT 1") == "1\n" + + +def test_remove_tcp_port(cluster, zk): + with default_client(cluster, zk, restore_via_http=True) as client: + assert client.query("SELECT 1") == "1\n" + with sync_loaded_config(instance.http_query): + zk.delete("/clickhouse/ports/tcp") + with pytest.raises(QueryRuntimeException, match="Connection refused"): + client.query("SELECT 1") + + +def test_remove_http_port(cluster, zk): + with default_client(cluster, zk) as client: + assert instance.http_query("SELECT 1") == "1\n" + with sync_loaded_config(client.query): + zk.delete("/clickhouse/ports/http") + with pytest.raises(ConnectionError, match="Connection refused"): + instance.http_query("SELECT 1") + + +def test_remove_mysql_port(cluster, zk): + with default_client(cluster, zk) as client: + mysql_client = get_mysql_client(cluster, port=9004) + assert mysql_client.query("SELECT 1") == 1 + with sync_loaded_config(client.query): + zk.delete("/clickhouse/ports/mysql") + with pytest.raises(pymysql.err.OperationalError, match="Lost connection"): + mysql_client.query("SELECT 1") + + +def test_remove_postgresql_port(cluster, zk): + with default_client(cluster, zk) as client: + pgsql_client = get_pgsql_client(cluster, port=9005) + cursor = pgsql_client.cursor() + cursor.execute("SELECT 1") + assert cursor.fetchall() == [(1,)] + with sync_loaded_config(client.query): + zk.delete("/clickhouse/ports/postgresql") + with pytest.raises(psycopg2.OperationalError, match="closed"): + cursor.execute("SELECT 1") + + +def test_remove_grpc_port(cluster, zk): + with default_client(cluster, zk) as client: + grpc_channel = get_grpc_channel(cluster, port=9100) + assert grpc_query(grpc_channel, "SELECT 1") == "1\n" + with sync_loaded_config(client.query): + zk.delete("/clickhouse/ports/grpc") + with pytest.raises(grpc._channel._InactiveRpcError, match="StatusCode.UNAVAILABLE"): + grpc_query(grpc_channel, "SELECT 1") + + +def test_change_listen_host(cluster, zk): + localhost_client = Client(host="127.0.0.1", port=9000, command="/usr/bin/clickhouse") + localhost_client.command = ["docker", "exec", "-i", instance.docker_id] + localhost_client.command + try: + client = get_client(cluster, port=9000) + with sync_loaded_config(localhost_client.query): + zk.set("/clickhouse/listen_hosts", b"127.0.0.1") + with pytest.raises(QueryRuntimeException, match="Connection refused"): + client.query("SELECT 1") + assert localhost_client.query("SELECT 1") == "1\n" + finally: + with sync_loaded_config(localhost_client.query): + configure_ports_from_zk(zk) + From df0af7586422600a7ebc9053301e8303bcaeccbe Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 24 Dec 2021 10:36:39 +0300 Subject: [PATCH 0293/1260] Fix clang tidy --- src/Access/IAccessStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 3633a060044..0a7fc630d97 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -169,7 +169,7 @@ std::vector IAccessStorage::insert(const std::vector & mu if (!successfully_inserted.empty()) { String successfully_inserted_str; - for (auto entity : successfully_inserted) + for (const auto & entity : successfully_inserted) { if (!successfully_inserted_str.empty()) successfully_inserted_str += ", "; From 204cc11e293ebfc5b02230029bc8b4b31ab848ef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 26 Nov 2021 00:33:49 +0300 Subject: [PATCH 0294/1260] Fix incorrect include --- src/Storages/FileLog/DirectoryWatcherBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index f2737219fd8..005e1e5fd1b 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace DB { From 1fcb0006f3f5affde4722ee055f261c58e5d4882 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Nov 2021 00:06:54 +0300 Subject: [PATCH 0295/1260] Disable some libraries --- cmake/target.cmake | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/cmake/target.cmake b/cmake/target.cmake index 3c02c4313f1..4b109d165e7 100644 --- a/cmake/target.cmake +++ b/cmake/target.cmake @@ -42,6 +42,14 @@ if (CMAKE_CROSSCOMPILING) message (FATAL_ERROR "Trying to cross-compile to unsupported system: ${CMAKE_SYSTEM_NAME}!") endif () + if (USE_MUSL) + set (USE_SENTRY OFF CACHE INTERNAL "") + set (ENABLE_ODBC OFF CACHE INTERNAL "") + set (ENABLE_GRPC OFF CACHE INTERNAL "") + set (ENABLE_HDFS OFF CACHE INTERNAL "") + set (ENABLE_EMBEDDED_COMPILER OFF CACHE INTERNAL "") + endif () + # Don't know why but CXX_STANDARD doesn't work for cross-compilation set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++20") From 6472e8691d4a56df1b30e6b19a40e680df4bdc40 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Nov 2021 04:29:26 +0300 Subject: [PATCH 0296/1260] Fix build --- contrib/arrow-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 231185462dc..56b3e1b4f81 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -446,7 +446,7 @@ set (HAVE_STRERROR_R 1) set (HAVE_SCHED_GET_PRIORITY_MAX 1) set (HAVE_SCHED_GET_PRIORITY_MIN 1) -if (OS_LINUX) +if (OS_LINUX AND NOT USE_MUSL) set (STRERROR_R_CHAR_P 1) endif () From ad05c90c00dd98c959c23696e64b3a0a420103ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 29 Nov 2021 05:56:05 +0300 Subject: [PATCH 0297/1260] Fix build --- contrib/libuv-cmake/CMakeLists.txt | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/contrib/libuv-cmake/CMakeLists.txt b/contrib/libuv-cmake/CMakeLists.txt index 4fbd0575b55..dc47b0bf496 100644 --- a/contrib/libuv-cmake/CMakeLists.txt +++ b/contrib/libuv-cmake/CMakeLists.txt @@ -1,17 +1,8 @@ # This file is a modified version of contrib/libuv/CMakeLists.txt -include(CMakeDependentOption) - set (SOURCE_DIR "${CMAKE_SOURCE_DIR}/contrib/libuv") set (BINARY_DIR "${CMAKE_BINARY_DIR}/contrib/libuv") - -if(CMAKE_C_COMPILER_ID MATCHES "AppleClang|Clang|GNU") - list(APPEND uv_cflags -fvisibility=hidden --std=gnu89) - list(APPEND uv_cflags -Wall -Wextra -Wstrict-prototypes) - list(APPEND uv_cflags -Wno-unused-parameter) -endif() - set(uv_sources src/fs-poll.c src/idna.c @@ -76,7 +67,7 @@ endif() if(CMAKE_SYSTEM_NAME STREQUAL "Linux") list(APPEND uv_defines _GNU_SOURCE _POSIX_C_SOURCE=200112) - list(APPEND uv_libraries dl rt) + list(APPEND uv_libraries rt) list(APPEND uv_sources src/unix/linux-core.c src/unix/linux-inotify.c From 7897d93b25715b31152950050b9b0ed3b2dbbab0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Dec 2021 07:20:17 +0300 Subject: [PATCH 0298/1260] Increase stack size on Musl --- CMakeLists.txt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index d59dbbfc8b3..215e811a80c 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -424,6 +424,11 @@ if (OS_LINUX AND NOT SANITIZE) set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--no-undefined") endif () +# Increase stack size on Musl. We need big stack for our recursive-descend parser. +if (USE_MUSL) + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,-z,stack-size=2097152") +endif () + include(cmake/dbms_glob_sources.cmake) if (OS_LINUX OR OS_ANDROID) From fb5b1c3d6a19ccac0f0a2677acc7669ce1756ad7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Dec 2021 08:48:16 +0300 Subject: [PATCH 0299/1260] Make tests not depend on subtle changes in math functions --- .../01305_polygons_union.reference | 8 +- .../0_stateless/01305_polygons_union.sql | 13 +- .../01306_polygons_intersection.reference | 4 +- .../01306_polygons_intersection.sql | 13 +- ...01720_country_perimeter_and_area.reference | 420 +++++++++--------- .../01720_country_perimeter_and_area.sh | 8 +- 6 files changed, 236 insertions(+), 230 deletions(-) diff --git a/tests/queries/0_stateless/01305_polygons_union.reference b/tests/queries/0_stateless/01305_polygons_union.reference index f87d03c151c..6deb8dce7ce 100644 --- a/tests/queries/0_stateless/01305_polygons_union.reference +++ b/tests/queries/0_stateless/01305_polygons_union.reference @@ -1,8 +1,8 @@ [[[(1,2.9),(1,4),(4,4),(4,1),(2.9,1),(3,0),(0,0),(0,3),(1,2.9)]]] -[[[(4.3666052904432435,50.84337386140151),(4.366227,50.840809),(4.344961,50.833264),(4.338074,50.848677),(4.346693,50.858306),(4.3526804582393535,50.856658100365976),(4.3613577,50.8651821),(4.3613148,50.8651279),(4.3904543,50.8564867),(4.3830299,50.8428851),(4.3666052904432435,50.84337386140151)]]] +[[[(4.366605,50.843374),(4.366227,50.840809),(4.344961,50.833264),(4.338074,50.848677),(4.346693,50.858306),(4.35268,50.856658),(4.361358,50.865182),(4.361315,50.865128),(4.390454,50.856487),(4.38303,50.842885),(4.366605,50.843374)]]] -------- MultiPolygon with Polygon -MULTIPOLYGON(((35.5408 58.9593,37.2817 59.9768,38.7325 59.9465,36.9725 59.0149,37.3119 59.0258,37.8553 58.9075,39.7299 59.9314,44.4751 59.81,44.4146 55.3097,40.0925 52.1652,38.3395 52.1652,39.1456 52.7573,38.0214 52.8989,37.1608 52.2393,35.4682 52.2022,36.5022 53.0008,35.3776 53.0462,35.3645 53.076,34.2895 52.2208,32.5969 52.2208,33.4048 52.8423,33.1712 52.8276,32.5275 53.1741,31.2368 52.1652,29.7861 52.1466,32.2523 53.964,31.8748 54.1736,29.3931 52.2763,29.4536 59.7796,30.5719 59.9919,30.4812 58.8542,32.3249 59.9465,33.6548 59.9465,30.179 57.9196,30.179 56.9764,32.2175 58.3664,32.2342 58.4928,32.5691 58.5924,34.8637 59.9768,36.2843 59.9616,34.0496 58.6717,34.9952 58.6226,35.3712 58.8556,34.6522 58.9167,35.5408 58.9593),(36.4989 58.7512,36.1498 58.553,36.3447 58.5402,36.0877 58.5174,35.4314 58.1349,36.403 58.0507,36.5949 58.1673,36.0123 58.2869,37.191 58.6819,36.4989 58.7512),(34.4816 56.8232,34.8098 57.0409,33.5602 56.9781,33.3418 56.8364,33.8361 56.6953,34.1885 56.6259,34.3867 56.7596,34.229 56.7948,34.4816 56.8232),(35.9179 57.7512,35.7402 57.7909,36.0848 57.855,36.3932 58.0447,35.1134 57.9454,34.6332 57.6538,35.613 57.5595,35.9179 57.7512),(36.8709 53.2765,37.135 53.4711,36.9794 53.5878,37.3119 53.9273,37.0035 54.2999,36.6985 54.0791,36.919 53.8561,36.3552 53.8269,36.1528 53.6763,36.8709 53.2765),(38.1601 55.1091,38.3093 55.1546,38.2609 55.1775,38.1601 55.1091),(38.1688 56.0758,38.4339 56.2361,37.5054 56.5484,37.2281 56.3799,38.1688 56.0758),(38.1319 56.0534,36.647 55.9411,37.6238 55.7402,38.1319 56.0534),(37.2824 55.5258,36.8283 55.4471,37.06 55.3843,37.2824 55.5258),(36.151 54.791,36.0123 54.7554,36.0472 54.7217,36.151 54.791),(34.9611 53.9765,34.894 54.1226,35.6193 54.4929,34.9706 54.9262,34.7231 54.7576,35.0753 54.5981,34.1081 54.1757,34.7279 53.8116,34.9611 53.9765),(38.2312 56.9795,37.565 56.5843,38.9742 56.8774,38.4915 57.1308,38.2699 57.0021,38.3093 56.9929,38.2312 56.9795),(36.5334 56.6753,36.375 56.6455,36.4446 56.6242,36.5334 56.6753),(36.1999 57.0022,36.9794 57.0751,36.4587 57.1544,36.1999 57.0022),(34.6028 58.3749,33.6245 58.271,34.3593 58.2189,34.6028 58.3749),(33.7581 57.8255,33.2316 57.7748,33.6325 57.7419,33.7581 57.8255),(31.6069 56.3194,31.7506 56.8609,31.6514 57.1258,30.3301 56.1942,30.2394 55.2753,31.6069 56.3194),(34.2274 57.4023,34.0208 57.2724,35.0338 57.1875,35.4682 57.4674,34.2274 57.4023),(31.7782 55.7778,30.2092 54.6331,30.2394 53.6774,31.7439 54.8677,31.8413 54.9989,32.204 55.5156,31.7782 55.7778),(33.7222 56.3063,32.8387 56.3117,33.5244 56.1686,33.7222 56.3063),(33.1204 55.8832,32.748 55.9072,32.9547 55.7645,33.1204 55.8832),(35.2275 55.0993,36.4354 55.3441,35.7505 55.4454,35.2275 55.0993),(35.9817 55.5958,36.5563 55.6352,36.193 55.7319,35.9817 55.5958),(35.0954 55.822,35.3188 55.9582,34.7331 56.1049,34.4996 55.9565,35.0954 55.822),(34.9721 55.7463,34.2598 55.8023,33.6125 55.3778,34.3709 55.3709,34.9721 55.7463),(35.6571 56.1619,36.0233 56.3789,35.4083 56.5254,35.2273 56.414,35.71 56.3117,35.0485 56.303,34.744 56.1118,35.6571 56.1619),(40.2143 54.467,40.3948 54.4403,40.6064 54.034,39.9716 53.9807,40.2437 53.5878,39.5485 53.5878,39.9942 53.358,43.0243 55.3269,43.0243 56.2614,40.2143 54.467),(38.5511 53.2922,38.4609 53.226,39.2704 52.8471,39.9877 53.3534,38.5511 53.2922),(40.5716 55.8007,43.0243 57.2554,43.0243 58.0797,40.4543 56.5923,40.4855 56.4957,40.2529 56.4682,39.8102 56.1914,39.8205 56.0763,40.425 56.1942,40.5716 55.8007),(40.5504 55.7875,39.7601 55.7544,39.8151 55.3187,40.5504 55.7875),(39.7863 57.025,42.5105 58.477,41.6944 58.8542,40.1389 58.048,40.2437 58.0478,40.3343 57.4673,39.7299 57.4673,39.7863 57.025),(38.0744 57.5312,38.3737 57.6908,38.3395 57.7103,38.8533 58.0638,38.432 58.2584,38.0535 58.0542,38.3395 57.9356,37.4328 57.7103,38.0744 57.5312),(37.9669 57.4734,37.1608 57.2554,37.4489 57.1909,37.9669 57.4734),(40.4136 58.7241,41.2108 59.1035,40.6366 59.3817,39.8163 58.9766,40.4552 58.9011,40.4136 58.7241),(39.7184 58.3823,39.6392 58.3821,39.6392 58.3427,39.7184 58.3823),(38.7465 58.4255,39.5485 58.7133,39.4085 58.7696,38.7465 58.4255))) +MULTIPOLYGON(((35.5408 58.9593,37.2817 59.9768,38.7325 59.9465,36.9725 59.0149,37.3119 59.0258,37.8553 58.9075,39.7299 59.9314,44.4751 59.81,44.4146 55.3097,40.0925 52.1652,38.3395 52.1652,39.1456 52.7572,38.0214 52.8989,37.1608 52.2393,35.4682 52.2022,36.5022 53.0008,35.3776 53.0462,35.3645 53.076,34.2895 52.2208,32.5969 52.2208,33.4048 52.8423,33.1712 52.8276,32.5275 53.1741,31.2368 52.1652,29.7861 52.1466,32.2523 53.964,31.8748 54.1736,29.3931 52.2763,29.4536 59.7796,30.5719 59.9919,30.4812 58.8542,32.3249 59.9465,33.6548 59.9465,30.179 57.9196,30.179 56.9764,32.2175 58.3664,32.2342 58.4928,32.5691 58.5924,34.8637 59.9768,36.2843 59.9616,34.0496 58.6717,34.9952 58.6226,35.3712 58.8556,34.6522 58.9167,35.5408 58.9593),(36.4989 58.7512,36.1498 58.553,36.3447 58.5402,36.0877 58.5174,35.4314 58.1349,36.403 58.0507,36.5949 58.1673,36.0123 58.2869,37.191 58.6819,36.4989 58.7512),(34.4816 56.8232,34.8098 57.0409,33.5602 56.9781,33.3418 56.8364,33.8361 56.6953,34.1885 56.6259,34.3867 56.7596,34.229 56.7948,34.4816 56.8232),(35.9179 57.7512,35.7402 57.7909,36.0848 57.855,36.3932 58.0447,35.1134 57.9454,34.6332 57.6538,35.613 57.5595,35.9179 57.7512),(36.8709 53.2765,37.135 53.4711,36.9794 53.5878,37.3119 53.9273,37.0035 54.2999,36.6985 54.0791,36.919 53.8561,36.3552 53.8269,36.1528 53.6763,36.8709 53.2765),(38.1601 55.1091,38.3093 55.1546,38.2609 55.1775,38.1601 55.1091),(38.1688 56.0758,38.4339 56.2361,37.5054 56.5484,37.2281 56.3799,38.1688 56.0758),(38.1319 56.0534,36.647 55.9411,37.6238 55.7402,38.1319 56.0534),(37.2824 55.5258,36.8283 55.4471,37.06 55.3843,37.2824 55.5258),(36.151 54.791,36.0123 54.7554,36.0472 54.7217,36.151 54.791),(34.9611 53.9765,34.894 54.1226,35.6193 54.4929,34.9706 54.9262,34.7231 54.7576,35.0753 54.5981,34.1081 54.1757,34.7279 53.8116,34.9611 53.9765),(38.2312 56.9795,37.565 56.5843,38.9742 56.8774,38.4915 57.1308,38.2699 57.0021,38.3093 56.9929,38.2312 56.9795),(36.5334 56.6753,36.375 56.6455,36.4446 56.6242,36.5334 56.6753),(36.1999 57.0022,36.9794 57.0751,36.4587 57.1544,36.1999 57.0022),(34.6028 58.3749,33.6245 58.271,34.3593 58.2189,34.6028 58.3749),(33.7581 57.8255,33.2316 57.7748,33.6325 57.7419,33.7581 57.8255),(31.6069 56.3194,31.7506 56.8609,31.6514 57.1258,30.3301 56.1942,30.2394 55.2753,31.6069 56.3194),(34.2274 57.4023,34.0208 57.2724,35.0338 57.1875,35.4682 57.4674,34.2274 57.4023),(31.7782 55.7778,30.2092 54.6331,30.2394 53.6774,31.7439 54.8677,31.8413 54.9989,32.204 55.5156,31.7782 55.7778),(33.7222 56.3063,32.8387 56.3117,33.5244 56.1686,33.7222 56.3063),(33.1204 55.8832,32.748 55.9072,32.9547 55.7645,33.1204 55.8832),(35.2275 55.0993,36.4354 55.3441,35.7505 55.4454,35.2275 55.0993),(35.9817 55.5958,36.5563 55.6352,36.193 55.7319,35.9817 55.5958),(35.0954 55.822,35.3188 55.9582,34.7331 56.1049,34.4996 55.9565,35.0954 55.822),(34.9721 55.7463,34.2598 55.8023,33.6125 55.3778,34.3709 55.3709,34.9721 55.7463),(35.6571 56.1619,36.0233 56.3789,35.4083 56.5254,35.2273 56.414,35.71 56.3117,35.0485 56.303,34.744 56.1118,35.6571 56.1619),(40.2143 54.467,40.3948 54.4403,40.6064 54.034,39.9716 53.9807,40.2437 53.5878,39.5485 53.5878,39.9942 53.358,43.0243 55.3269,43.0243 56.2614,40.2143 54.467),(38.5511 53.2922,38.4609 53.226,39.2704 52.8471,39.9877 53.3534,38.5511 53.2922),(40.5716 55.8007,43.0243 57.2554,43.0243 58.0797,40.4543 56.5923,40.4855 56.4957,40.2529 56.4682,39.8102 56.1914,39.8205 56.0763,40.425 56.1942,40.5716 55.8007),(40.5504 55.7875,39.7601 55.7544,39.8151 55.3187,40.5504 55.7875),(39.7863 57.025,42.5105 58.477,41.6944 58.8542,40.1389 58.048,40.2437 58.0478,40.3343 57.4673,39.7299 57.4673,39.7863 57.025),(38.0744 57.5312,38.3737 57.6908,38.3395 57.7103,38.8533 58.0638,38.432 58.2584,38.0535 58.0542,38.3395 57.9356,37.4328 57.7103,38.0744 57.5312),(37.9669 57.4734,37.1608 57.2554,37.4489 57.1909,37.9669 57.4734),(40.4136 58.7241,41.2108 59.1035,40.6366 59.3817,39.8163 58.9766,40.4552 58.9011,40.4136 58.7241),(39.7184 58.3823,39.6392 58.3821,39.6392 58.3427,39.7184 58.3823),(38.7465 58.4255,39.5485 58.7133,39.4085 58.7696,38.7465 58.4255))) -------- MultiPolygon with Polygon with Holes -MULTIPOLYGON(((24.3677 61.4598,26.6528 61.1008,26.8726 61.7107,30.564 61.0583,31.3989 62.0215,36.0132 61.1432,36.8921 62.0009,42.6489 60.6301,43.5718 61.3757,47.0435 59.8889,49.5923 60.0868,49.1528 58.1707,51.9214 57.9148,50.2515 56.1455,52.6685 55.826,51.6577 54.2909,52.8882 53.9302,50.647 53.0148,51.394 52.4828,48.0542 51.1793,49.2847 50.5414,47.1753 49.153,43.9233 49.8096,42.561 48.7779,36.936 49.6676,35.2661 48.7489,32.8052 49.5252,27.2241 48.9802,26.1255 50.4015,21.2036 50.205,20.0171 51.5634,17.4683 53.0148,19.4458 54.0852,19.4458 55.8753,19.5776 57.4922,19.5776 58.6769,24.3677 61.4598),(24.4556 59.4227,21.2036 58.4937,21.3354 56.897,21.5991 55.9246,25.2026 55.9984,28.8501 57.0646,27.0923 57.8448,28.8062 59.1759,26.2573 59.1759,24.4556 59.4227),(33.1079 56.9523,33.1392 56.8934,33.7182 56.7292,35.1489 56.5859,34.229 56.7948,36.9794 57.0751,35.7705 57.2554,37.0097 57.4998,35.7402 57.7909,37.1608 58.0478,36.0123 58.2869,37.191 58.6819,34.6522 58.9167,37.2327 59.0233,37.1118 59.6677,35.1343 59.8448,31.9702 58.9727,32.25 58.4976,33.4734 58.8542,34.7428 59.5659,33.8361 58.6819,36.3447 58.5402,33.6245 58.271,36.4354 58.0478,33.2316 57.7748,36.1936 57.4998,33.1712 57.337,36.0727 57.0915,33.1079 56.9523),(37.0604 52.9744,34.9585 51.4814,36.5405 50.4015,39.6606 50.2893,39.7925 52.1335,41.77 50.6808,44.4946 51.9713,47.3071 52.5095,44.0552 53.5403,46.604 53.6967,47.6147 55.4041,45.3735 55.4041,42.8247 56.5837,40.4412 56.1511,40.5761 55.7884,39.7601 55.7544,39.8205 55.2753,40.3948 55.2408,40.3948 54.8773,39.5485 54.8773,39.5485 54.5631,40.3948 54.4403,40.6064 54.034,39.9716 53.9807,40.2437 53.5878,39.5485 53.5878,40.0019 53.354,38.3395 53.2817,39.5787 52.6996,37.8559 52.9188,37.4471 53.2343,37.2165 53.0798,37.4328 52.9552,37.0604 52.9744),(31.627 54.7093,29.5972 55.5037,29.1577 55.7518,22.5659 55.1286,22.5659 53.5403,22.0386 51.4814,26.2573 51.4266,30.1245 50.5414,32.1899 51.1793,30.1245 53.1731,32.4808 53.1989,32.0831 53.408,32.476 53.8383,31.4182 54.4227,31.627 54.7093),(34.7731 53.3243,34.7731 53.1793,35.0903 53.1731,34.7731 53.3243),(36.9508 55.414,37.7653 55.1891,36.8822 54.975,37.0572 54.7635,38.3093 55.1546,37.7955 55.3956,38.4907 55.5327,38.3184 55.7179,38.0262 55.6546,38.0373 55.6523,37.9482 55.6376,36.9508 55.414),(38.3092 56.9929,38.5798 57.0849,38.2186 57.2717,38.7325 57.4835,38.3395 57.7103,38.8533 58.0638,38.3698 58.2869,39.5485 58.7133,38.8838 58.9777,38.0944 58.8545,38.5813 58.7446,37.4026 58.3187,38.3395 57.9356,37.4328 57.7103,38.128 57.516,37.1608 57.2554,38.3092 56.9929),(38.309 56.9928,36.375 56.6455,36.8799 56.4895,38.309 56.9928),(40.3237 57.5365,42.6929 58.0314,40.8911 59.2659,39.2792 59.0373,40.4552 58.9011,40.3343 58.3821,39.6392 58.3821,39.6392 58.0478,40.2437 58.0478,40.3237 57.5365),(40.0149 57.4677,39.7299 57.4673,39.7379 57.4051,40.0149 57.4677))) +MULTIPOLYGON(((24.3677 61.4598,26.6528 61.1008,26.8726 61.7107,30.564 61.0583,31.3989 62.0215,36.0132 61.1432,36.8921 62.0009,42.6489 60.6301,43.5718 61.3757,47.0435 59.8889,49.5923 60.0868,49.1528 58.1707,51.9214 57.9148,50.2515 56.1456,52.6685 55.826,51.6577 54.2909,52.8882 53.9302,50.647 53.0148,51.394 52.4828,48.0542 51.1793,49.2847 50.5414,47.1753 49.153,43.9233 49.8096,42.561 48.7779,36.936 49.6676,35.2661 48.7489,32.8052 49.5252,27.2241 48.9802,26.1255 50.4015,21.2036 50.205,20.0171 51.5634,17.4683 53.0148,19.4458 54.0852,19.4458 55.8753,19.5776 57.4922,19.5776 58.6769,24.3677 61.4598),(24.4556 59.4227,21.2036 58.4937,21.3354 56.897,21.5991 55.9246,25.2026 55.9984,28.8501 57.0646,27.0923 57.8448,28.8062 59.1759,26.2573 59.1759,24.4556 59.4227),(33.1079 56.9523,33.1392 56.8934,33.7182 56.7292,35.1489 56.5859,34.229 56.7948,36.9794 57.0751,35.7705 57.2554,37.0097 57.4998,35.7402 57.7909,37.1608 58.0478,36.0123 58.2869,37.191 58.6819,34.6522 58.9167,37.2327 59.0233,37.1118 59.6677,35.1343 59.8448,31.9702 58.9727,32.25 58.4976,33.4734 58.8542,34.7428 59.5659,33.8361 58.6819,36.3447 58.5402,33.6245 58.271,36.4354 58.0478,33.2316 57.7748,36.1936 57.4998,33.1712 57.337,36.0727 57.0915,33.1079 56.9523),(37.0604 52.9744,34.9585 51.4814,36.5405 50.4015,39.6606 50.2893,39.7925 52.1335,41.77 50.6808,44.4946 51.9713,47.3071 52.5095,44.0552 53.5403,46.604 53.6967,47.6147 55.4041,45.3735 55.4041,42.8247 56.5837,40.4412 56.1511,40.5761 55.7884,39.7601 55.7544,39.8205 55.2753,40.3948 55.2408,40.3948 54.8773,39.5485 54.8773,39.5485 54.5631,40.3948 54.4403,40.6064 54.034,39.9716 53.9807,40.2437 53.5878,39.5485 53.5878,40.0019 53.354,38.3395 53.2817,39.5787 52.6996,37.8559 52.9188,37.4471 53.2343,37.2165 53.0798,37.4328 52.9552,37.0604 52.9744),(31.627 54.7093,29.5972 55.5037,29.1577 55.7518,22.5659 55.1286,22.5659 53.5403,22.0386 51.4814,26.2573 51.4266,30.1245 50.5414,32.1899 51.1793,30.1245 53.1731,32.4808 53.1989,32.0831 53.408,32.476 53.8383,31.4182 54.4227,31.627 54.7093),(34.7731 53.3243,34.7731 53.1793,35.0903 53.1731,34.7731 53.3243),(36.9508 55.414,37.7653 55.1891,36.8822 54.975,37.0572 54.7635,38.3093 55.1546,37.7955 55.3956,38.4907 55.5327,38.3184 55.7179,38.0262 55.6546,38.0373 55.6523,37.9482 55.6376,36.9508 55.414),(38.3092 56.9929,38.5798 57.0849,38.2186 57.2717,38.7325 57.4835,38.3395 57.7103,38.8533 58.0638,38.3698 58.2869,39.5485 58.7133,38.8838 58.9777,38.0944 58.8545,38.5813 58.7446,37.4026 58.3187,38.3395 57.9356,37.4328 57.7103,38.128 57.516,37.1608 57.2554,38.3092 56.9929),(38.309 56.9928,36.375 56.6455,36.8799 56.4895,38.309 56.9928),(40.3237 57.5365,42.6929 58.0314,40.8911 59.2659,39.2792 59.0373,40.4552 58.9011,40.3343 58.3821,39.6392 58.3821,39.6392 58.0478,40.2437 58.0478,40.3237 57.5365),(40.0149 57.4677,39.7299 57.4673,39.7379 57.4051,40.0149 57.4677))) -------- Polygon with Polygon with Holes -MULTIPOLYGON(((24.3677 61.4598,26.6528 61.1008,26.8726 61.7107,30.564 61.0583,31.3989 62.0215,36.0132 61.1432,36.8921 62.0009,42.6489 60.6301,43.5718 61.3757,47.0435 59.8889,49.5923 60.0868,49.1528 58.1707,51.9214 57.9148,50.2515 56.1455,52.6685 55.826,51.6577 54.2909,52.8882 53.9302,50.647 53.0148,51.394 52.4828,48.0542 51.1793,49.2847 50.5414,47.1753 49.153,43.9233 49.8096,42.561 48.7779,36.936 49.6676,35.2661 48.7489,32.8052 49.5252,27.2241 48.9802,26.1255 50.4015,21.2036 50.205,20.0171 51.5634,17.4683 53.0148,19.4458 54.0852,19.4458 55.8753,19.5776 57.4922,19.5776 58.6769,24.3677 61.4598),(24.4556 59.4227,21.2036 58.4937,21.3354 56.897,21.5991 55.9246,25.2026 55.9984,28.8501 57.0646,27.0923 57.8448,28.8062 59.1759,26.2573 59.1759,24.4556 59.4227),(32.6512 57.792,32.9378 57.2699,36.7912 59.6986,35.9475 59.7758,32.6512 57.792),(33.2446 56.7729,34.2635 56.6767,37.6322 58.7797,37.2876 58.7226,37.2102 59.1452,33.2446 56.7729),(36.1815 56.4715,41.168 59.0834,40.9299 59.2404,40.8804 59.2644,40.2079 59.1718,35.4536 56.5531,36.1815 56.4715),(30.7705 55.0525,30.2092 54.6331,30.2394 53.6774,31.5682 54.7333,30.7705 55.0525),(33.8733 53.1922,34.3351 53.53,33.5144 53.9057,32.5603 53.1989,33.8733 53.1922),(31.1968 52.1649,29.7861 52.1466,30.5785 52.7531,30.3098 53.0028,29.3931 52.2763,29.4171 55.606,29.1577 55.7518,22.5659 55.1286,22.5659 53.5403,22.0386 51.4814,26.2573 51.4266,30.1245 50.5414,32.1899 51.1793,31.1968 52.1649),(31.1682 53.1903,32.6907 54.2663,32.2591 54.4483,30.5408 53.1811,31.1682 53.1903),(39.4328 55.9511,37.2766 54.4948,37.7431 53.9104,41.4519 56.3413,39.4328 55.9511),(40.9691 57.677,42.2498 58.3455,41.5887 58.8012,38.1759 56.9472,39.0894 57.2553,40.9691 57.677),(37.1934 55.4694,36.5845 55.3291,36.7219 55.1665,37.1934 55.4694),(32.2964 58.4175,34.2247 59.6064,31.9702 58.9727,32.2964 58.4175),(35.9681 52.2157,34.9585 51.4814,36.5405 50.4015,39.6606 50.2893,39.7925 52.1335,41.77 50.6808,44.4946 51.9713,47.3071 52.5095,44.0552 53.5403,46.604 53.6967,47.6147 55.4041,45.3735 55.4041,44.4212 55.8594,44.4146 55.3097,40.0925 52.1652,38.3395 52.1652,43.0243 55.3269,43.0243 56.2614,37.1608 52.2393,35.9681 52.2157))) +MULTIPOLYGON(((24.3677 61.4598,26.6528 61.1008,26.8726 61.7107,30.564 61.0583,31.3989 62.0215,36.0132 61.1432,36.8921 62.0009,42.6489 60.6301,43.5718 61.3757,47.0435 59.8889,49.5923 60.0868,49.1528 58.1707,51.9214 57.9148,50.2515 56.1456,52.6685 55.826,51.6577 54.2909,52.8882 53.9302,50.647 53.0148,51.394 52.4828,48.0542 51.1793,49.2847 50.5414,47.1753 49.153,43.9233 49.8096,42.561 48.7779,36.936 49.6676,35.2661 48.7489,32.8052 49.5252,27.2241 48.9802,26.1255 50.4015,21.2036 50.205,20.0171 51.5634,17.4683 53.0148,19.4458 54.0852,19.4458 55.8753,19.5776 57.4922,19.5776 58.6769,24.3677 61.4598),(24.4556 59.4227,21.2036 58.4937,21.3354 56.897,21.5991 55.9246,25.2026 55.9984,28.8501 57.0646,27.0923 57.8448,28.8062 59.1759,26.2573 59.1759,24.4556 59.4227),(32.6512 57.792,32.9378 57.2699,36.7912 59.6986,35.9475 59.7758,32.6512 57.792),(33.2446 56.7729,34.2635 56.6767,37.6322 58.7797,37.2876 58.7226,37.2102 59.1452,33.2446 56.7729),(36.1815 56.4715,41.168 59.0834,40.9299 59.2404,40.8804 59.2644,40.2079 59.1718,35.4536 56.5531,36.1815 56.4715),(30.7705 55.0525,30.2092 54.6331,30.2394 53.6774,31.5682 54.7333,30.7705 55.0525),(33.8733 53.1922,34.3351 53.53,33.5144 53.9057,32.5603 53.1989,33.8733 53.1922),(31.1968 52.1649,29.7861 52.1466,30.5785 52.7531,30.3098 53.0028,29.3931 52.2763,29.4171 55.606,29.1577 55.7518,22.5659 55.1286,22.5659 53.5403,22.0386 51.4814,26.2573 51.4266,30.1245 50.5414,32.1899 51.1793,31.1968 52.1649),(31.1682 53.1903,32.6907 54.2663,32.2591 54.4483,30.5408 53.1811,31.1682 53.1903),(39.4328 55.9511,37.2766 54.4948,37.7431 53.9104,41.4519 56.3413,39.4328 55.9511),(40.9691 57.677,42.2498 58.3455,41.5887 58.8012,38.1759 56.9472,39.0894 57.2553,40.9691 57.677),(37.1934 55.4694,36.5845 55.3291,36.7219 55.1665,37.1934 55.4694),(32.2964 58.4175,34.2247 59.6064,31.9702 58.9727,32.2964 58.4175),(35.9681 52.2157,34.9585 51.4814,36.5405 50.4015,39.6606 50.2893,39.7925 52.1335,41.77 50.6808,44.4946 51.9713,47.3071 52.5095,44.0552 53.5403,46.604 53.6967,47.6147 55.4041,45.3735 55.4041,44.4212 55.8594,44.4146 55.3097,40.0925 52.1652,38.3395 52.1652,43.0243 55.3269,43.0243 56.2614,37.1608 52.2393,35.9681 52.2157))) diff --git a/tests/queries/0_stateless/01305_polygons_union.sql b/tests/queries/0_stateless/01305_polygons_union.sql index 01982c21e6e..23ea0d050c3 100644 --- a/tests/queries/0_stateless/01305_polygons_union.sql +++ b/tests/queries/0_stateless/01305_polygons_union.sql @@ -1,15 +1,18 @@ select polygonsUnionCartesian([[[(0., 0.),(0., 3.),(1., 2.9),(2., 2.6),(2.6, 2.),(2.9, 1),(3., 0.),(0., 0.)]]], [[[(1., 1.),(1., 4.),(4., 4.),(4., 1.),(1., 1.)]]]); -SELECT polygonsUnionCartesian([[[(2., 100.0000991821289), (0., 3.), (1., 2.9), (2., 2.6), (2.6, 2.), (2.9, 1), (3., 0.), (100.0000991821289, 2.)]]], [[[(1., 1.), (1000.0001220703125, nan), (4., 4.), (4., 1.), (1., 1.)]]]); -- { serverError 43 } +SELECT arrayMap(a -> arrayMap(b -> arrayMap(c -> (round(c.1, 6), round(c.2, 6)), b), a), polygonsUnionCartesian([[[(2., 100.0000991821289), (0., 3.), (1., 2.9), (2., 2.6), (2.6, 2.), (2.9, 1), (3., 0.), (100.0000991821289, 2.)]]], [[[(1., 1.), (1000.0001220703125, nan), (4., 4.), (4., 1.), (1., 1.)]]])); -- { serverError 43 } -select polygonsUnionSpherical([[[(4.3613577, 50.8651821), (4.349556, 50.8535879), (4.3602419, 50.8435626), (4.3830299, 50.8428851), (4.3904543, 50.8564867), (4.3613148, 50.8651279)]]], [[[(4.346693, 50.858306), (4.367945, 50.852455), (4.366227, 50.840809), (4.344961, 50.833264), (4.338074, 50.848677), (4.346693, 50.858306)]]]); +select arrayMap(a -> arrayMap(b -> arrayMap(c -> (round(c.1, 6), round(c.2, 6)), b), a), polygonsUnionSpherical([[[(4.3613577, 50.8651821), (4.349556, 50.8535879), (4.3602419, 50.8435626), (4.3830299, 50.8428851), (4.3904543, 50.8564867), (4.3613148, 50.8651279)]]], [[[(4.346693, 50.858306), (4.367945, 50.852455), (4.366227, 50.840809), (4.344961, 50.833264), (4.338074, 50.848677), (4.346693, 50.858306)]]])); select '-------- MultiPolygon with Polygon'; -select wkt(polygonsUnionSpherical([[(29.453587685533865,59.779570356240356),(29.393139070478895,52.276266797422124),(40.636581470703206,59.38168915000267),(41.21084331372543,59.103467777099866),(29.786055068336193,52.146627480315004),(31.23682182965546,52.16517054781818),(41.69443223416517,58.85424941916091),(42.51048853740727,58.47703162291134),(32.59691566839227,52.22075341251539),(34.289476889931414,52.22075341251539),(43.02430176537451,58.07974369546071),(43.02430176537451,57.25537683364851),(35.468224883503325,52.2022335126388),(37.16078610504247,52.23926559241349),(43.02430176537451,56.26136189644947),(43.02430176537451,55.326904361850836),(38.33953409861437,52.16517054781818),(40.09254393520848,52.16517054781818),(44.4146199116388,55.3097062225408),(44.47506852669377,59.80998197603594),(39.72985224487867,59.931351417569715),(30.23941968124846,53.67744677450975),(30.20919537372098,54.63314259659509),(38.73245009647167,59.94649146557819),(37.2816833351524,59.97675082987618),(30.23941968124846,55.2752875586599),(30.33009260383092,56.19415599955667),(36.28428118674541,59.96162460231375),(34.863738732953635,59.97675082987618),(30.178971066193498,56.97640788219866),(30.178971066193498,57.91957806959033),(33.65476643185424,59.94649146557819),(32.32489690064491,59.94649146557819),(30.481214141468342,58.85424941916091),(30.571887064050795,59.99187015036608),(29.453587685533865,59.779570356240356)]], [[[(33.473420586689336,58.85424941916091),(32.23422397806246,58.492830557036),(32.173775363007486,58.03176922751564),(31.508840597402823,57.499784781503735),(31.750635057622702,56.86092686957355),(31.508840597402823,55.941082594334574),(32.20399967053497,55.515591939372456),(31.84130798020516,54.998862226280465),(31.418167674820367,54.422670886434275),(32.47601843828233,53.83826377018255),(32.08310244042503,53.408048308050866),(33.171177511414484,52.82758702113742),(34.77306581037117,52.91880107773494),(34.77306581037117,53.784726518357985),(34.108131044766516,54.17574726780569),(35.07530888564602,54.59813930694554),(34.25925258240394,54.96417435716029),(35.01486027059106,55.361278263643584),(33.50364489421682,55.37845402950552),(32.7480372060297,55.90721384574556),(35.67979503619571,55.68634475630185),(32.83871012861215,56.311688992608396),(34.591719965206266,56.29492065473883),(35.7100193437232,56.311688992608396),(33.83611227701915,56.695333481003644),(32.95960735872209,56.9434497616887),(36.072711034053015,57.091531913901434),(33.171177511414484,57.33702717078384),(36.193608264162954,57.499784781503735),(33.23162612646945,57.77481561306047),(36.43540272438284,58.04776787540811),(33.62454212432676,58.27099811968307),(36.344729801800376,58.54018474404165),(33.83611227701915,58.68186423448108),(34.74284150284369,59.565911441555244),(33.473420586689336,58.85424941916091)]], [[(34.65216858026123,58.91672306881671),(37.19101041256995,58.68186423448108),(36.01226241899805,58.28688958537609),(37.16078610504247,58.04776787540811),(35.74024365125068,57.79092907387934),(37.009664567405046,57.499784781503735),(35.77046795877817,57.25537683364851),(36.979440259877556,57.07510745541089),(34.22902827487645,56.794777197297435),(36.7074214921302,56.210968525786996),(34.712617195316206,56.10998276812964),(36.55629995449277,55.63519693782703),(35.13575750070099,55.53270067649592),(36.43540272438284,55.34409504165558),(34.83351442542614,55.01619492319591),(35.61934642114075,54.49294870011772),(34.89396304048112,54.12264226523038),(35.37755196092087,53.046178687628185),(37.43280487278982,52.95523300597458),(35.92158949641559,53.80257986695776),(36.91899164482259,53.856094327816805),(36.01226241899805,54.75541714463799),(37.765272255592166,55.189110239786885),(36.828318722240134,55.44708256557195),(38.03729102333953,55.652253637168315),(36.64697287707522,55.941082594334574),(38.21863686850443,56.05939028508024),(36.37495410932787,56.64551287174558),(38.30930979108689,56.992876013526654),(37.16078610504247,57.25537683364851),(38.127963945921984,57.516020773674256),(37.43280487278982,57.710289827306724),(38.33953409861437,57.935626886818994),(37.40258056526235,58.31865112960426),(38.58132855883426,58.744648733419496),(37.31190764267989,59.02578062465136),(34.65216858026123,58.91672306881671)]], [[(38.52087994377928,59.11898412389468),(39.54850639971376,58.713270635642914),(38.369758406141855,58.28688958537609),(38.85334732658162,58.06375936407028),(38.33953409861437,57.710289827306724),(38.73245009647167,57.48354156434209),(38.21863686850443,57.271721400459285),(38.97424455669155,56.87744603722649),(37.463029180317314,56.5623320541159),(38.94402024916407,56.05939028508024),(38.18841256097694,55.856355210835915),(38.490655636251795,55.53270067649592),(37.795496563119656,55.39562234093384),(38.30930979108689,55.154587013355666),(36.7074214921302,54.65063295250911),(37.31190764267989,53.92734063371401),(36.979440259877556,53.58783775557231),(37.855945178174615,52.91880107773497),(39.57873070724124,52.69956490610895),(38.33953409861437,53.281741738901104),(40.00187101262603,53.35396273604752),(39.54850639971376,53.58783775557231),(40.24366547284591,53.58783775557231),(39.97164670509855,53.98069568468355),(40.60635716317572,54.03398248547225),(40.39478701048334,54.44025165268903),(39.54850639971376,54.56310590284329),(39.54850639971376,54.87732350170489),(40.39478701048334,54.87732350170489),(40.39478701048334,55.24083903654295),(39.82052516746112,55.2752875586599),(39.760076552406154,55.75443792473942),(40.57613285564824,55.78844000174894),(40.425011318010824,56.19415599955667),(39.82052516746112,56.07626182891758),(39.79030085993364,56.41214455508424),(40.48545993306579,56.495655446714636),(40.33433839542836,56.95993246553937),(39.79030085993364,56.992876013526654),(39.72985224487867,57.46729112028032),(40.33433839542836,57.46729112028032),(40.24366547284591,58.04776787540811),(39.63917932229622,58.04776787540811),(39.63917932229622,58.382088724871295),(40.33433839542836,58.382088724871295),(40.45523562553831,58.9011152358548),(38.52087994377928,59.11898412389468)]]])) format TSV; +select wkt(arrayMap(a -> arrayMap(b -> arrayMap(c -> (round(c.1, 6), round(c.2, 6)), b), a), +polygonsUnionSpherical([[(29.453587685533865,59.779570356240356),(29.393139070478895,52.276266797422124),(40.636581470703206,59.38168915000267),(41.21084331372543,59.103467777099866),(29.786055068336193,52.146627480315004),(31.23682182965546,52.16517054781818),(41.69443223416517,58.85424941916091),(42.51048853740727,58.47703162291134),(32.59691566839227,52.22075341251539),(34.289476889931414,52.22075341251539),(43.02430176537451,58.07974369546071),(43.02430176537451,57.25537683364851),(35.468224883503325,52.2022335126388),(37.16078610504247,52.23926559241349),(43.02430176537451,56.26136189644947),(43.02430176537451,55.326904361850836),(38.33953409861437,52.16517054781818),(40.09254393520848,52.16517054781818),(44.4146199116388,55.3097062225408),(44.47506852669377,59.80998197603594),(39.72985224487867,59.931351417569715),(30.23941968124846,53.67744677450975),(30.20919537372098,54.63314259659509),(38.73245009647167,59.94649146557819),(37.2816833351524,59.97675082987618),(30.23941968124846,55.2752875586599),(30.33009260383092,56.19415599955667),(36.28428118674541,59.96162460231375),(34.863738732953635,59.97675082987618),(30.178971066193498,56.97640788219866),(30.178971066193498,57.91957806959033),(33.65476643185424,59.94649146557819),(32.32489690064491,59.94649146557819),(30.481214141468342,58.85424941916091),(30.571887064050795,59.99187015036608),(29.453587685533865,59.779570356240356)]], [[[(33.473420586689336,58.85424941916091),(32.23422397806246,58.492830557036),(32.173775363007486,58.03176922751564),(31.508840597402823,57.499784781503735),(31.750635057622702,56.86092686957355),(31.508840597402823,55.941082594334574),(32.20399967053497,55.515591939372456),(31.84130798020516,54.998862226280465),(31.418167674820367,54.422670886434275),(32.47601843828233,53.83826377018255),(32.08310244042503,53.408048308050866),(33.171177511414484,52.82758702113742),(34.77306581037117,52.91880107773494),(34.77306581037117,53.784726518357985),(34.108131044766516,54.17574726780569),(35.07530888564602,54.59813930694554),(34.25925258240394,54.96417435716029),(35.01486027059106,55.361278263643584),(33.50364489421682,55.37845402950552),(32.7480372060297,55.90721384574556),(35.67979503619571,55.68634475630185),(32.83871012861215,56.311688992608396),(34.591719965206266,56.29492065473883),(35.7100193437232,56.311688992608396),(33.83611227701915,56.695333481003644),(32.95960735872209,56.9434497616887),(36.072711034053015,57.091531913901434),(33.171177511414484,57.33702717078384),(36.193608264162954,57.499784781503735),(33.23162612646945,57.77481561306047),(36.43540272438284,58.04776787540811),(33.62454212432676,58.27099811968307),(36.344729801800376,58.54018474404165),(33.83611227701915,58.68186423448108),(34.74284150284369,59.565911441555244),(33.473420586689336,58.85424941916091)]], [[(34.65216858026123,58.91672306881671),(37.19101041256995,58.68186423448108),(36.01226241899805,58.28688958537609),(37.16078610504247,58.04776787540811),(35.74024365125068,57.79092907387934),(37.009664567405046,57.499784781503735),(35.77046795877817,57.25537683364851),(36.979440259877556,57.07510745541089),(34.22902827487645,56.794777197297435),(36.7074214921302,56.210968525786996),(34.712617195316206,56.10998276812964),(36.55629995449277,55.63519693782703),(35.13575750070099,55.53270067649592),(36.43540272438284,55.34409504165558),(34.83351442542614,55.01619492319591),(35.61934642114075,54.49294870011772),(34.89396304048112,54.12264226523038),(35.37755196092087,53.046178687628185),(37.43280487278982,52.95523300597458),(35.92158949641559,53.80257986695776),(36.91899164482259,53.856094327816805),(36.01226241899805,54.75541714463799),(37.765272255592166,55.189110239786885),(36.828318722240134,55.44708256557195),(38.03729102333953,55.652253637168315),(36.64697287707522,55.941082594334574),(38.21863686850443,56.05939028508024),(36.37495410932787,56.64551287174558),(38.30930979108689,56.992876013526654),(37.16078610504247,57.25537683364851),(38.127963945921984,57.516020773674256),(37.43280487278982,57.710289827306724),(38.33953409861437,57.935626886818994),(37.40258056526235,58.31865112960426),(38.58132855883426,58.744648733419496),(37.31190764267989,59.02578062465136),(34.65216858026123,58.91672306881671)]], [[(38.52087994377928,59.11898412389468),(39.54850639971376,58.713270635642914),(38.369758406141855,58.28688958537609),(38.85334732658162,58.06375936407028),(38.33953409861437,57.710289827306724),(38.73245009647167,57.48354156434209),(38.21863686850443,57.271721400459285),(38.97424455669155,56.87744603722649),(37.463029180317314,56.5623320541159),(38.94402024916407,56.05939028508024),(38.18841256097694,55.856355210835915),(38.490655636251795,55.53270067649592),(37.795496563119656,55.39562234093384),(38.30930979108689,55.154587013355666),(36.7074214921302,54.65063295250911),(37.31190764267989,53.92734063371401),(36.979440259877556,53.58783775557231),(37.855945178174615,52.91880107773497),(39.57873070724124,52.69956490610895),(38.33953409861437,53.281741738901104),(40.00187101262603,53.35396273604752),(39.54850639971376,53.58783775557231),(40.24366547284591,53.58783775557231),(39.97164670509855,53.98069568468355),(40.60635716317572,54.03398248547225),(40.39478701048334,54.44025165268903),(39.54850639971376,54.56310590284329),(39.54850639971376,54.87732350170489),(40.39478701048334,54.87732350170489),(40.39478701048334,55.24083903654295),(39.82052516746112,55.2752875586599),(39.760076552406154,55.75443792473942),(40.57613285564824,55.78844000174894),(40.425011318010824,56.19415599955667),(39.82052516746112,56.07626182891758),(39.79030085993364,56.41214455508424),(40.48545993306579,56.495655446714636),(40.33433839542836,56.95993246553937),(39.79030085993364,56.992876013526654),(39.72985224487867,57.46729112028032),(40.33433839542836,57.46729112028032),(40.24366547284591,58.04776787540811),(39.63917932229622,58.04776787540811),(39.63917932229622,58.382088724871295),(40.33433839542836,58.382088724871295),(40.45523562553831,58.9011152358548),(38.52087994377928,59.11898412389468)]]]))) format TSV; select '-------- MultiPolygon with Polygon with Holes'; -select wkt(polygonsUnionSpherical([[[(33.473420586689336,58.85424941916091),(32.23422397806246,58.492830557036),(32.173775363007486,58.03176922751564),(31.508840597402823,57.499784781503735),(31.750635057622702,56.86092686957355),(31.508840597402823,55.941082594334574),(32.20399967053497,55.515591939372456),(31.84130798020516,54.998862226280465),(31.418167674820367,54.422670886434275),(32.47601843828233,53.83826377018255),(32.08310244042503,53.408048308050866),(33.171177511414484,52.82758702113742),(34.77306581037117,52.91880107773494),(34.77306581037117,53.784726518357985),(34.108131044766516,54.17574726780569),(35.07530888564602,54.59813930694554),(34.25925258240394,54.96417435716029),(35.01486027059106,55.361278263643584),(33.50364489421682,55.37845402950552),(32.7480372060297,55.90721384574556),(35.67979503619571,55.68634475630185),(32.83871012861215,56.311688992608396),(34.591719965206266,56.29492065473883),(35.7100193437232,56.311688992608396),(33.83611227701915,56.695333481003644),(32.95960735872209,56.9434497616887),(36.072711034053015,57.091531913901434),(33.171177511414484,57.33702717078384),(36.193608264162954,57.499784781503735),(33.23162612646945,57.77481561306047),(36.43540272438284,58.04776787540811),(33.62454212432676,58.27099811968307),(36.344729801800376,58.54018474404165),(33.83611227701915,58.68186423448108),(34.74284150284369,59.565911441555244),(33.473420586689336,58.85424941916091)]], [[(34.65216858026123,58.91672306881671),(37.19101041256995,58.68186423448108),(36.01226241899805,58.28688958537609),(37.16078610504247,58.04776787540811),(35.74024365125068,57.79092907387934),(37.009664567405046,57.499784781503735),(35.77046795877817,57.25537683364851),(36.979440259877556,57.07510745541089),(34.22902827487645,56.794777197297435),(36.7074214921302,56.210968525786996),(34.712617195316206,56.10998276812964),(36.55629995449277,55.63519693782703),(35.13575750070099,55.53270067649592),(36.43540272438284,55.34409504165558),(34.83351442542614,55.01619492319591),(35.61934642114075,54.49294870011772),(34.89396304048112,54.12264226523038),(35.37755196092087,53.046178687628185),(37.43280487278982,52.95523300597458),(35.92158949641559,53.80257986695776),(36.91899164482259,53.856094327816805),(36.01226241899805,54.75541714463799),(37.765272255592166,55.189110239786885),(36.828318722240134,55.44708256557195),(38.03729102333953,55.652253637168315),(36.64697287707522,55.941082594334574),(38.21863686850443,56.05939028508024),(36.37495410932787,56.64551287174558),(38.30930979108689,56.992876013526654),(37.16078610504247,57.25537683364851),(38.127963945921984,57.516020773674256),(37.43280487278982,57.710289827306724),(38.33953409861437,57.935626886818994),(37.40258056526235,58.31865112960426),(38.58132855883426,58.744648733419496),(37.31190764267989,59.02578062465136),(34.65216858026123,58.91672306881671)]], [[(38.52087994377928,59.11898412389468),(39.54850639971376,58.713270635642914),(38.369758406141855,58.28688958537609),(38.85334732658162,58.06375936407028),(38.33953409861437,57.710289827306724),(38.73245009647167,57.48354156434209),(38.21863686850443,57.271721400459285),(38.97424455669155,56.87744603722649),(37.463029180317314,56.5623320541159),(38.94402024916407,56.05939028508024),(38.18841256097694,55.856355210835915),(38.490655636251795,55.53270067649592),(37.795496563119656,55.39562234093384),(38.30930979108689,55.154587013355666),(36.7074214921302,54.65063295250911),(37.31190764267989,53.92734063371401),(36.979440259877556,53.58783775557231),(37.855945178174615,52.91880107773497),(39.57873070724124,52.69956490610895),(38.33953409861437,53.281741738901104),(40.00187101262603,53.35396273604752),(39.54850639971376,53.58783775557231),(40.24366547284591,53.58783775557231),(39.97164670509855,53.98069568468355),(40.60635716317572,54.03398248547225),(40.39478701048334,54.44025165268903),(39.54850639971376,54.56310590284329),(39.54850639971376,54.87732350170489),(40.39478701048334,54.87732350170489),(40.39478701048334,55.24083903654295),(39.82052516746112,55.2752875586599),(39.760076552406154,55.75443792473942),(40.57613285564824,55.78844000174894),(40.425011318010824,56.19415599955667),(39.82052516746112,56.07626182891758),(39.79030085993364,56.41214455508424),(40.48545993306579,56.495655446714636),(40.33433839542836,56.95993246553937),(39.79030085993364,56.992876013526654),(39.72985224487867,57.46729112028032),(40.33433839542836,57.46729112028032),(40.24366547284591,58.04776787540811),(39.63917932229622,58.04776787540811),(39.63917932229622,58.382088724871295),(40.33433839542836,58.382088724871295),(40.45523562553831,58.9011152358548),(38.52087994377928,59.11898412389468)]]], [[(24.367675781249993,61.45977057029751),(19.577636718749993,58.67693767258692),(19.577636718749993,57.492213666700735),(19.445800781249996,55.87531083569678),(19.445800781249996,54.085173420886775),(17.468261718749996,53.014783245859235),(20.017089843749993,51.563412328675895),(21.203613281249993,50.205033264943324),(26.125488281249993,50.40151532278236),(27.22412109374999,48.980216985374994),(32.80517578124999,49.525208341974405),(35.26611328124999,48.74894534343292),(36.93603515624999,49.66762782262194),(42.56103515625,48.77791275550183),(43.92333984374999,49.8096315635631),(47.17529296875,49.152969656170455),(49.28466796875,50.54136296522162),(48.05419921875,51.17934297928929),(51.39404296875,52.48278022207825),(50.64697265625,53.014783245859235),(52.88818359375,53.93021986394004),(51.65771484374999,54.29088164657006),(52.66845703125,55.825973254619015),(50.25146484375,56.145549500679095),(51.92138671875,57.914847767009206),(49.15283203125,58.17070248348605),(49.59228515625,60.086762746260064),(47.043457031249986,59.88893689676584),(43.57177734375,61.37567331572748),(42.64892578125,60.630101766266705),(36.89208984374999,62.000904713685856),(36.01318359374999,61.143235250840576),(31.398925781249993,62.02152819100766),(30.563964843749996,61.05828537037917),(26.872558593749993,61.71070595883174),(26.652832031249993,61.10078883158897),(24.367675781249993,61.45977057029751)], [(24.455566406249993,59.42272750081452),(21.203613281249993,58.49369382056807),(21.335449218749993,56.89700392127261),(21.599121093749993,55.92458580482949),(25.202636718749993,55.998380955359636),(28.850097656249993,57.06463027327854),(27.09228515625,57.844750992890994),(28.806152343749996,59.17592824927138),(26.257324218749993,59.17592824927138),(24.455566406249993,59.42272750081452)], [(35.13427734375,59.84481485969107),(31.970214843749993,58.97266715450152),(33.20068359374999,56.776808316568406),(36.67236328125,56.41390137600675),(39.08935546874999,57.25528054528888),(42.69287109374999,58.03137242177638),(40.89111328124999,59.26588062825809),(37.28759765625,58.722598828043374),(37.11181640624999,59.66774058164964),(35.13427734375,59.84481485969107)], [(29.157714843749993,55.75184939173528),(22.565917968749993,55.128649068488784),(22.565917968749993,53.54030739150019),(22.038574218749996,51.48138289610097),(26.257324218749993,51.42661449707484),(30.124511718749993,50.54136296522162),(32.18994140624999,51.17934297928929),(30.124511718749993,53.173119202640635),(35.09033203124999,53.173119202640635),(33.11279296875,54.085173420886775),(29.597167968749993,55.50374985927513),(29.157714843749993,55.75184939173528)], [(42.82470703125,56.58369172128337),(36.584472656249986,55.329144408405085),(37.99072265625,53.592504809039355),(34.95849609374999,51.48138289610097),(36.54052734374999,50.40151532278236),(39.66064453124999,50.289339253291786),(39.79248046875,52.13348804077148),(41.77001953125,50.68079714532166),(44.49462890624999,51.97134580885171),(47.30712890624999,52.509534770327264),(44.05517578125,53.54030739150019),(46.60400390625,53.696706475303245),(47.61474609375,55.40406982700608),(45.37353515625,55.40406982700608),(42.82470703125,56.58369172128337)]])) format TSV; +select wkt(arrayMap(a -> arrayMap(b -> arrayMap(c -> (round(c.1, 6), round(c.2, 6)), b), a), +polygonsUnionSpherical([[[(33.473420586689336,58.85424941916091),(32.23422397806246,58.492830557036),(32.173775363007486,58.03176922751564),(31.508840597402823,57.499784781503735),(31.750635057622702,56.86092686957355),(31.508840597402823,55.941082594334574),(32.20399967053497,55.515591939372456),(31.84130798020516,54.998862226280465),(31.418167674820367,54.422670886434275),(32.47601843828233,53.83826377018255),(32.08310244042503,53.408048308050866),(33.171177511414484,52.82758702113742),(34.77306581037117,52.91880107773494),(34.77306581037117,53.784726518357985),(34.108131044766516,54.17574726780569),(35.07530888564602,54.59813930694554),(34.25925258240394,54.96417435716029),(35.01486027059106,55.361278263643584),(33.50364489421682,55.37845402950552),(32.7480372060297,55.90721384574556),(35.67979503619571,55.68634475630185),(32.83871012861215,56.311688992608396),(34.591719965206266,56.29492065473883),(35.7100193437232,56.311688992608396),(33.83611227701915,56.695333481003644),(32.95960735872209,56.9434497616887),(36.072711034053015,57.091531913901434),(33.171177511414484,57.33702717078384),(36.193608264162954,57.499784781503735),(33.23162612646945,57.77481561306047),(36.43540272438284,58.04776787540811),(33.62454212432676,58.27099811968307),(36.344729801800376,58.54018474404165),(33.83611227701915,58.68186423448108),(34.74284150284369,59.565911441555244),(33.473420586689336,58.85424941916091)]], [[(34.65216858026123,58.91672306881671),(37.19101041256995,58.68186423448108),(36.01226241899805,58.28688958537609),(37.16078610504247,58.04776787540811),(35.74024365125068,57.79092907387934),(37.009664567405046,57.499784781503735),(35.77046795877817,57.25537683364851),(36.979440259877556,57.07510745541089),(34.22902827487645,56.794777197297435),(36.7074214921302,56.210968525786996),(34.712617195316206,56.10998276812964),(36.55629995449277,55.63519693782703),(35.13575750070099,55.53270067649592),(36.43540272438284,55.34409504165558),(34.83351442542614,55.01619492319591),(35.61934642114075,54.49294870011772),(34.89396304048112,54.12264226523038),(35.37755196092087,53.046178687628185),(37.43280487278982,52.95523300597458),(35.92158949641559,53.80257986695776),(36.91899164482259,53.856094327816805),(36.01226241899805,54.75541714463799),(37.765272255592166,55.189110239786885),(36.828318722240134,55.44708256557195),(38.03729102333953,55.652253637168315),(36.64697287707522,55.941082594334574),(38.21863686850443,56.05939028508024),(36.37495410932787,56.64551287174558),(38.30930979108689,56.992876013526654),(37.16078610504247,57.25537683364851),(38.127963945921984,57.516020773674256),(37.43280487278982,57.710289827306724),(38.33953409861437,57.935626886818994),(37.40258056526235,58.31865112960426),(38.58132855883426,58.744648733419496),(37.31190764267989,59.02578062465136),(34.65216858026123,58.91672306881671)]], [[(38.52087994377928,59.11898412389468),(39.54850639971376,58.713270635642914),(38.369758406141855,58.28688958537609),(38.85334732658162,58.06375936407028),(38.33953409861437,57.710289827306724),(38.73245009647167,57.48354156434209),(38.21863686850443,57.271721400459285),(38.97424455669155,56.87744603722649),(37.463029180317314,56.5623320541159),(38.94402024916407,56.05939028508024),(38.18841256097694,55.856355210835915),(38.490655636251795,55.53270067649592),(37.795496563119656,55.39562234093384),(38.30930979108689,55.154587013355666),(36.7074214921302,54.65063295250911),(37.31190764267989,53.92734063371401),(36.979440259877556,53.58783775557231),(37.855945178174615,52.91880107773497),(39.57873070724124,52.69956490610895),(38.33953409861437,53.281741738901104),(40.00187101262603,53.35396273604752),(39.54850639971376,53.58783775557231),(40.24366547284591,53.58783775557231),(39.97164670509855,53.98069568468355),(40.60635716317572,54.03398248547225),(40.39478701048334,54.44025165268903),(39.54850639971376,54.56310590284329),(39.54850639971376,54.87732350170489),(40.39478701048334,54.87732350170489),(40.39478701048334,55.24083903654295),(39.82052516746112,55.2752875586599),(39.760076552406154,55.75443792473942),(40.57613285564824,55.78844000174894),(40.425011318010824,56.19415599955667),(39.82052516746112,56.07626182891758),(39.79030085993364,56.41214455508424),(40.48545993306579,56.495655446714636),(40.33433839542836,56.95993246553937),(39.79030085993364,56.992876013526654),(39.72985224487867,57.46729112028032),(40.33433839542836,57.46729112028032),(40.24366547284591,58.04776787540811),(39.63917932229622,58.04776787540811),(39.63917932229622,58.382088724871295),(40.33433839542836,58.382088724871295),(40.45523562553831,58.9011152358548),(38.52087994377928,59.11898412389468)]]], [[(24.367675781249993,61.45977057029751),(19.577636718749993,58.67693767258692),(19.577636718749993,57.492213666700735),(19.445800781249996,55.87531083569678),(19.445800781249996,54.085173420886775),(17.468261718749996,53.014783245859235),(20.017089843749993,51.563412328675895),(21.203613281249993,50.205033264943324),(26.125488281249993,50.40151532278236),(27.22412109374999,48.980216985374994),(32.80517578124999,49.525208341974405),(35.26611328124999,48.74894534343292),(36.93603515624999,49.66762782262194),(42.56103515625,48.77791275550183),(43.92333984374999,49.8096315635631),(47.17529296875,49.152969656170455),(49.28466796875,50.54136296522162),(48.05419921875,51.17934297928929),(51.39404296875,52.48278022207825),(50.64697265625,53.014783245859235),(52.88818359375,53.93021986394004),(51.65771484374999,54.29088164657006),(52.66845703125,55.825973254619015),(50.25146484375,56.145549500679095),(51.92138671875,57.914847767009206),(49.15283203125,58.17070248348605),(49.59228515625,60.086762746260064),(47.043457031249986,59.88893689676584),(43.57177734375,61.37567331572748),(42.64892578125,60.630101766266705),(36.89208984374999,62.000904713685856),(36.01318359374999,61.143235250840576),(31.398925781249993,62.02152819100766),(30.563964843749996,61.05828537037917),(26.872558593749993,61.71070595883174),(26.652832031249993,61.10078883158897),(24.367675781249993,61.45977057029751)], [(24.455566406249993,59.42272750081452),(21.203613281249993,58.49369382056807),(21.335449218749993,56.89700392127261),(21.599121093749993,55.92458580482949),(25.202636718749993,55.998380955359636),(28.850097656249993,57.06463027327854),(27.09228515625,57.844750992890994),(28.806152343749996,59.17592824927138),(26.257324218749993,59.17592824927138),(24.455566406249993,59.42272750081452)], [(35.13427734375,59.84481485969107),(31.970214843749993,58.97266715450152),(33.20068359374999,56.776808316568406),(36.67236328125,56.41390137600675),(39.08935546874999,57.25528054528888),(42.69287109374999,58.03137242177638),(40.89111328124999,59.26588062825809),(37.28759765625,58.722598828043374),(37.11181640624999,59.66774058164964),(35.13427734375,59.84481485969107)], [(29.157714843749993,55.75184939173528),(22.565917968749993,55.128649068488784),(22.565917968749993,53.54030739150019),(22.038574218749996,51.48138289610097),(26.257324218749993,51.42661449707484),(30.124511718749993,50.54136296522162),(32.18994140624999,51.17934297928929),(30.124511718749993,53.173119202640635),(35.09033203124999,53.173119202640635),(33.11279296875,54.085173420886775),(29.597167968749993,55.50374985927513),(29.157714843749993,55.75184939173528)], [(42.82470703125,56.58369172128337),(36.584472656249986,55.329144408405085),(37.99072265625,53.592504809039355),(34.95849609374999,51.48138289610097),(36.54052734374999,50.40151532278236),(39.66064453124999,50.289339253291786),(39.79248046875,52.13348804077148),(41.77001953125,50.68079714532166),(44.49462890624999,51.97134580885171),(47.30712890624999,52.509534770327264),(44.05517578125,53.54030739150019),(46.60400390625,53.696706475303245),(47.61474609375,55.40406982700608),(45.37353515625,55.40406982700608),(42.82470703125,56.58369172128337)]]))) format TSV; select '-------- Polygon with Polygon with Holes'; -select wkt(polygonsUnionSpherical([[(29.453587685533865,59.779570356240356),(29.393139070478895,52.276266797422124),(40.636581470703206,59.38168915000267),(41.21084331372543,59.103467777099866),(29.786055068336193,52.146627480315004),(31.23682182965546,52.16517054781818),(41.69443223416517,58.85424941916091),(42.51048853740727,58.47703162291134),(32.59691566839227,52.22075341251539),(34.289476889931414,52.22075341251539),(43.02430176537451,58.07974369546071),(43.02430176537451,57.25537683364851),(35.468224883503325,52.2022335126388),(37.16078610504247,52.23926559241349),(43.02430176537451,56.26136189644947),(43.02430176537451,55.326904361850836),(38.33953409861437,52.16517054781818),(40.09254393520848,52.16517054781818),(44.4146199116388,55.3097062225408),(44.47506852669377,59.80998197603594),(39.72985224487867,59.931351417569715),(30.23941968124846,53.67744677450975),(30.20919537372098,54.63314259659509),(38.73245009647167,59.94649146557819),(37.2816833351524,59.97675082987618),(30.23941968124846,55.2752875586599),(30.33009260383092,56.19415599955667),(36.28428118674541,59.96162460231375),(34.863738732953635,59.97675082987618),(30.178971066193498,56.97640788219866),(30.178971066193498,57.91957806959033),(33.65476643185424,59.94649146557819),(32.32489690064491,59.94649146557819),(30.481214141468342,58.85424941916091),(30.571887064050795,59.99187015036608),(29.453587685533865,59.779570356240356)]], [[(24.367675781249993,61.45977057029751),(19.577636718749993,58.67693767258692),(19.577636718749993,57.492213666700735),(19.445800781249996,55.87531083569678),(19.445800781249996,54.085173420886775),(17.468261718749996,53.014783245859235),(20.017089843749993,51.563412328675895),(21.203613281249993,50.205033264943324),(26.125488281249993,50.40151532278236),(27.22412109374999,48.980216985374994),(32.80517578124999,49.525208341974405),(35.26611328124999,48.74894534343292),(36.93603515624999,49.66762782262194),(42.56103515625,48.77791275550183),(43.92333984374999,49.8096315635631),(47.17529296875,49.152969656170455),(49.28466796875,50.54136296522162),(48.05419921875,51.17934297928929),(51.39404296875,52.48278022207825),(50.64697265625,53.014783245859235),(52.88818359375,53.93021986394004),(51.65771484374999,54.29088164657006),(52.66845703125,55.825973254619015),(50.25146484375,56.145549500679095),(51.92138671875,57.914847767009206),(49.15283203125,58.17070248348605),(49.59228515625,60.086762746260064),(47.043457031249986,59.88893689676584),(43.57177734375,61.37567331572748),(42.64892578125,60.630101766266705),(36.89208984374999,62.000904713685856),(36.01318359374999,61.143235250840576),(31.398925781249993,62.02152819100766),(30.563964843749996,61.05828537037917),(26.872558593749993,61.71070595883174),(26.652832031249993,61.10078883158897),(24.367675781249993,61.45977057029751)], [(24.455566406249993,59.42272750081452),(21.203613281249993,58.49369382056807),(21.335449218749993,56.89700392127261),(21.599121093749993,55.92458580482949),(25.202636718749993,55.998380955359636),(28.850097656249993,57.06463027327854),(27.09228515625,57.844750992890994),(28.806152343749996,59.17592824927138),(26.257324218749993,59.17592824927138),(24.455566406249993,59.42272750081452)], [(35.13427734375,59.84481485969107),(31.970214843749993,58.97266715450152),(33.20068359374999,56.776808316568406),(36.67236328125,56.41390137600675),(39.08935546874999,57.25528054528888),(42.69287109374999,58.03137242177638),(40.89111328124999,59.26588062825809),(37.28759765625,58.722598828043374),(37.11181640624999,59.66774058164964),(35.13427734375,59.84481485969107)], [(29.157714843749993,55.75184939173528),(22.565917968749993,55.128649068488784),(22.565917968749993,53.54030739150019),(22.038574218749996,51.48138289610097),(26.257324218749993,51.42661449707484),(30.124511718749993,50.54136296522162),(32.18994140624999,51.17934297928929),(30.124511718749993,53.173119202640635),(35.09033203124999,53.173119202640635),(33.11279296875,54.085173420886775),(29.597167968749993,55.50374985927513),(29.157714843749993,55.75184939173528)], [(42.82470703125,56.58369172128337),(36.584472656249986,55.329144408405085),(37.99072265625,53.592504809039355),(34.95849609374999,51.48138289610097),(36.54052734374999,50.40151532278236),(39.66064453124999,50.289339253291786),(39.79248046875,52.13348804077148),(41.77001953125,50.68079714532166),(44.49462890624999,51.97134580885171),(47.30712890624999,52.509534770327264),(44.05517578125,53.54030739150019),(46.60400390625,53.696706475303245),(47.61474609375,55.40406982700608),(45.37353515625,55.40406982700608),(42.82470703125,56.58369172128337)]])) format TSV; +select wkt(arrayMap(a -> arrayMap(b -> arrayMap(c -> (round(c.1, 6), round(c.2, 6)), b), a), +polygonsUnionSpherical([[(29.453587685533865,59.779570356240356),(29.393139070478895,52.276266797422124),(40.636581470703206,59.38168915000267),(41.21084331372543,59.103467777099866),(29.786055068336193,52.146627480315004),(31.23682182965546,52.16517054781818),(41.69443223416517,58.85424941916091),(42.51048853740727,58.47703162291134),(32.59691566839227,52.22075341251539),(34.289476889931414,52.22075341251539),(43.02430176537451,58.07974369546071),(43.02430176537451,57.25537683364851),(35.468224883503325,52.2022335126388),(37.16078610504247,52.23926559241349),(43.02430176537451,56.26136189644947),(43.02430176537451,55.326904361850836),(38.33953409861437,52.16517054781818),(40.09254393520848,52.16517054781818),(44.4146199116388,55.3097062225408),(44.47506852669377,59.80998197603594),(39.72985224487867,59.931351417569715),(30.23941968124846,53.67744677450975),(30.20919537372098,54.63314259659509),(38.73245009647167,59.94649146557819),(37.2816833351524,59.97675082987618),(30.23941968124846,55.2752875586599),(30.33009260383092,56.19415599955667),(36.28428118674541,59.96162460231375),(34.863738732953635,59.97675082987618),(30.178971066193498,56.97640788219866),(30.178971066193498,57.91957806959033),(33.65476643185424,59.94649146557819),(32.32489690064491,59.94649146557819),(30.481214141468342,58.85424941916091),(30.571887064050795,59.99187015036608),(29.453587685533865,59.779570356240356)]], [[(24.367675781249993,61.45977057029751),(19.577636718749993,58.67693767258692),(19.577636718749993,57.492213666700735),(19.445800781249996,55.87531083569678),(19.445800781249996,54.085173420886775),(17.468261718749996,53.014783245859235),(20.017089843749993,51.563412328675895),(21.203613281249993,50.205033264943324),(26.125488281249993,50.40151532278236),(27.22412109374999,48.980216985374994),(32.80517578124999,49.525208341974405),(35.26611328124999,48.74894534343292),(36.93603515624999,49.66762782262194),(42.56103515625,48.77791275550183),(43.92333984374999,49.8096315635631),(47.17529296875,49.152969656170455),(49.28466796875,50.54136296522162),(48.05419921875,51.17934297928929),(51.39404296875,52.48278022207825),(50.64697265625,53.014783245859235),(52.88818359375,53.93021986394004),(51.65771484374999,54.29088164657006),(52.66845703125,55.825973254619015),(50.25146484375,56.145549500679095),(51.92138671875,57.914847767009206),(49.15283203125,58.17070248348605),(49.59228515625,60.086762746260064),(47.043457031249986,59.88893689676584),(43.57177734375,61.37567331572748),(42.64892578125,60.630101766266705),(36.89208984374999,62.000904713685856),(36.01318359374999,61.143235250840576),(31.398925781249993,62.02152819100766),(30.563964843749996,61.05828537037917),(26.872558593749993,61.71070595883174),(26.652832031249993,61.10078883158897),(24.367675781249993,61.45977057029751)], [(24.455566406249993,59.42272750081452),(21.203613281249993,58.49369382056807),(21.335449218749993,56.89700392127261),(21.599121093749993,55.92458580482949),(25.202636718749993,55.998380955359636),(28.850097656249993,57.06463027327854),(27.09228515625,57.844750992890994),(28.806152343749996,59.17592824927138),(26.257324218749993,59.17592824927138),(24.455566406249993,59.42272750081452)], [(35.13427734375,59.84481485969107),(31.970214843749993,58.97266715450152),(33.20068359374999,56.776808316568406),(36.67236328125,56.41390137600675),(39.08935546874999,57.25528054528888),(42.69287109374999,58.03137242177638),(40.89111328124999,59.26588062825809),(37.28759765625,58.722598828043374),(37.11181640624999,59.66774058164964),(35.13427734375,59.84481485969107)], [(29.157714843749993,55.75184939173528),(22.565917968749993,55.128649068488784),(22.565917968749993,53.54030739150019),(22.038574218749996,51.48138289610097),(26.257324218749993,51.42661449707484),(30.124511718749993,50.54136296522162),(32.18994140624999,51.17934297928929),(30.124511718749993,53.173119202640635),(35.09033203124999,53.173119202640635),(33.11279296875,54.085173420886775),(29.597167968749993,55.50374985927513),(29.157714843749993,55.75184939173528)], [(42.82470703125,56.58369172128337),(36.584472656249986,55.329144408405085),(37.99072265625,53.592504809039355),(34.95849609374999,51.48138289610097),(36.54052734374999,50.40151532278236),(39.66064453124999,50.289339253291786),(39.79248046875,52.13348804077148),(41.77001953125,50.68079714532166),(44.49462890624999,51.97134580885171),(47.30712890624999,52.509534770327264),(44.05517578125,53.54030739150019),(46.60400390625,53.696706475303245),(47.61474609375,55.40406982700608),(45.37353515625,55.40406982700608),(42.82470703125,56.58369172128337)]]))) format TSV; diff --git a/tests/queries/0_stateless/01306_polygons_intersection.reference b/tests/queries/0_stateless/01306_polygons_intersection.reference index 43ee975913e..99b26f7acc7 100644 --- a/tests/queries/0_stateless/01306_polygons_intersection.reference +++ b/tests/queries/0_stateless/01306_polygons_intersection.reference @@ -1,9 +1,9 @@ [[[(1,2.9),(2,2.6),(2.6,2),(2.9,1),(1,1),(1,2.9)]]] [] [] -[[[(4.3666052904432435,50.84337386140151),(4.3602419,50.8435626),(4.349556,50.8535879),(4.3526804582393535,50.856658100365976),(4.367945,50.852455),(4.3666052904432435,50.84337386140151)]]] +[[[(4.366605,50.843374),(4.360242,50.843563),(4.349556,50.853588),(4.35268,50.856658),(4.367945,50.852455),(4.366605,50.843374)]]] -------- MultiPolygon with Polygon -MULTIPOLYGON(((35.5408 58.9593,36.9725 59.0149,36.4989 58.7512,35.3712 58.8556,35.5408 58.9593)),((34.4816 56.8232,36.1999 57.0022,35.4083 56.5254,34.3867 56.7596,34.4816 56.8232)),((35.9179 57.7512,36.0848 57.855,37.1608 58.0478,36.5949 58.1673,37.8553 58.9075,38.5813 58.7446,37.4026 58.3187,38.0535 58.0542,36.4587 57.1544,35.7705 57.2554,37.0097 57.4998,35.9179 57.7512)),((36.8709 53.2765,37.4328 52.9552,36.5022 53.0008,36.8709 53.2765)),((36.1528 53.6763,35.3645 53.076,34.9611 53.9765,36.0472 54.7217,36.6985 54.0791,36.3552 53.8269,35.9216 53.8026,36.1528 53.6763)),((37.0035 54.2999,36.7074 54.6506,38.1601 55.1091,37.0035 54.2999)),((38.1688 56.0758,38.2186 56.0594,38.1319 56.0534,38.1688 56.0758)),((37.6238 55.7402,38.0373 55.6523,37.2824 55.5258,37.6238 55.7402)),((37.06 55.3843,37.7653 55.1891,36.151 54.791,37.06 55.3843)),((38.2312 56.9795,36.5334 56.6753,37.4489 57.1909,38.2699 57.0021,38.2312 56.9795)),((37.2281 56.3799,36.193 55.7319,35.3188 55.9582,35.6571 56.1619,36.7074 56.211,36.0233 56.3789,36.4446 56.6242,37.2281 56.3799)),((34.9952 58.6226,36.1498 58.553,36.0877 58.5174,34.6028 58.3749,34.9952 58.6226)),((34.3593 58.2189,35.4314 58.1349,35.1134 57.9454,33.7581 57.8255,34.3593 58.2189)),((33.6325 57.7419,34.6332 57.6538,34.2274 57.4023,33.1712 57.337,34.0208 57.2724,33.5602 56.9781,32.9596 56.9434,33.3418 56.8364,31.7782 55.7778,31.5088 55.9411,31.6069 56.3194,33.6325 57.7419)),((36.403 58.0507,36.4354 58.0478,36.3932 58.0447,36.403 58.0507)),((35.613 57.5595,36.1936 57.4998,35.4682 57.4674,35.613 57.5595)),((35.0338 57.1875,36.0727 57.0915,34.8098 57.0409,35.0338 57.1875)),((34.1885 56.6259,35.2273 56.414,35.0485 56.303,34.5917 56.2949,33.7222 56.3063,34.1885 56.6259)),((33.5244 56.1686,34.4996 55.9565,34.2598 55.8023,33.1204 55.8832,33.5244 56.1686)),((32.9547 55.7645,33.5036 55.3785,33.6125 55.3778,31.8748 54.1736,31.4182 54.4227,31.7439 54.8677,32.9547 55.7645)),((34.7279 53.8116,34.7731 53.7847,34.7731 52.9188,33.4048 52.8423,34.7279 53.8116)),((34.7231 54.7576,32.5275 53.1741,32.0831 53.408,32.476 53.8383,32.2523 53.964,34.3709 55.3709,35.0149 55.3613,34.2593 54.9642,34.7231 54.7576)),((34.9706 54.9262,34.8335 55.0162,35.2275 55.0993,34.9706 54.9262)),((35.7505 55.4454,35.1358 55.5327,35.9817 55.5958,35.7505 55.4454)),((35.0954 55.822,35.6798 55.6863,34.9721 55.7463,35.0954 55.822)),((34.7331 56.1049,34.7126 56.11,34.744 56.1118,34.7331 56.1049)),((40.2143 54.467,38.5511 53.2922,38.3395 53.2817,38.4609 53.226,38.0214 52.8989,37.8559 52.9188,37.135 53.4711,39.8151 55.3187,39.8205 55.2753,40.3948 55.2408,40.3948 54.8773,39.5485 54.8773,39.5485 54.5631,40.2143 54.467)),((40.5716 55.8007,40.5761 55.7884,40.5504 55.7875,40.5716 55.8007)),((40.4543 56.5923,40.2529 56.4682,39.7903 56.4121,39.8102 56.1914,38.2609 55.1775,37.7955 55.3956,38.4907 55.5327,38.1884 55.8564,38.944 56.0594,38.4339 56.2361,39.7863 57.025,39.7903 56.9929,40.3343 56.9599,40.4543 56.5923)),((40.1389 58.048,38.4915 57.1308,38.2186 57.2717,38.7325 57.4835,38.3737 57.6908,39.6392 58.3427,39.6392 58.0478,40.1389 58.048)),((37.5054 56.5484,37.463 56.5623,37.565 56.5843,37.5054 56.5484)),((38.0744 57.5312,38.128 57.516,37.9669 57.4734,38.0744 57.5312)),((40.4136 58.7241,40.3343 58.3821,39.7184 58.3823,40.4136 58.7241)),((39.8163 58.9766,39.4085 58.7696,38.5209 59.119,39.8163 58.9766)),((38.432 58.2584,38.3698 58.2869,38.7465 58.4255,38.432 58.2584)),((32.2175 58.3664,32.5691 58.5924,33.4734 58.8542,34.7428 59.5659,33.8361 58.6819,34.0496 58.6717,31.6514 57.1258,31.5088 57.4998,32.1738 58.0318,32.2175 58.3664)),((39.9942 53.358,40.0019 53.354,39.9877 53.3534,39.9942 53.358)),((39.2704 52.8471,39.5787 52.6996,39.1456 52.7573,39.2704 52.8471))) +MULTIPOLYGON(((35.5408 58.9593,36.9725 59.0149,36.4989 58.7512,35.3712 58.8556,35.5408 58.9593)),((34.4816 56.8232,36.1999 57.0022,35.4083 56.5254,34.3867 56.7596,34.4816 56.8232)),((35.9179 57.7512,36.0848 57.855,37.1608 58.0478,36.5949 58.1673,37.8553 58.9075,38.5813 58.7446,37.4026 58.3187,38.0535 58.0542,36.4587 57.1544,35.7705 57.2554,37.0097 57.4998,35.9179 57.7512)),((36.8709 53.2765,37.4328 52.9552,36.5022 53.0008,36.8709 53.2765)),((36.1528 53.6763,35.3645 53.076,34.9611 53.9765,36.0472 54.7217,36.6985 54.0791,36.3552 53.8269,35.9216 53.8026,36.1528 53.6763)),((37.0035 54.2999,36.7074 54.6506,38.1601 55.1091,37.0035 54.2999)),((38.1688 56.0758,38.2186 56.0594,38.1319 56.0534,38.1688 56.0758)),((37.6238 55.7402,38.0373 55.6523,37.2824 55.5258,37.6238 55.7402)),((37.06 55.3843,37.7653 55.1891,36.151 54.791,37.06 55.3843)),((38.2312 56.9795,36.5334 56.6753,37.4489 57.1909,38.2699 57.0021,38.2312 56.9795)),((37.2281 56.3799,36.193 55.7319,35.3188 55.9582,35.6571 56.1619,36.7074 56.211,36.0233 56.3789,36.4446 56.6242,37.2281 56.3799)),((34.9952 58.6226,36.1498 58.553,36.0877 58.5174,34.6028 58.3749,34.9952 58.6226)),((34.3593 58.2189,35.4314 58.1349,35.1134 57.9454,33.7581 57.8255,34.3593 58.2189)),((33.6325 57.7419,34.6332 57.6538,34.2274 57.4023,33.1712 57.337,34.0208 57.2724,33.5602 56.9781,32.9596 56.9434,33.3418 56.8364,31.7782 55.7778,31.5088 55.9411,31.6069 56.3194,33.6325 57.7419)),((36.403 58.0507,36.4354 58.0478,36.3932 58.0447,36.403 58.0507)),((35.613 57.5595,36.1936 57.4998,35.4682 57.4674,35.613 57.5595)),((35.0338 57.1875,36.0727 57.0915,34.8098 57.0409,35.0338 57.1875)),((34.1885 56.6259,35.2273 56.414,35.0485 56.303,34.5917 56.2949,33.7222 56.3063,34.1885 56.6259)),((33.5244 56.1686,34.4996 55.9565,34.2598 55.8023,33.1204 55.8832,33.5244 56.1686)),((32.9547 55.7645,33.5036 55.3785,33.6125 55.3778,31.8748 54.1736,31.4182 54.4227,31.7439 54.8677,32.9547 55.7645)),((34.7279 53.8116,34.7731 53.7847,34.7731 52.9188,33.4048 52.8423,34.7279 53.8116)),((34.7231 54.7576,32.5275 53.1741,32.0831 53.408,32.476 53.8383,32.2523 53.964,34.3709 55.3709,35.0149 55.3613,34.2593 54.9642,34.7231 54.7576)),((34.9706 54.9262,34.8335 55.0162,35.2275 55.0993,34.9706 54.9262)),((35.7505 55.4454,35.1358 55.5327,35.9817 55.5958,35.7505 55.4454)),((35.0954 55.822,35.6798 55.6863,34.9721 55.7463,35.0954 55.822)),((34.7331 56.1049,34.7126 56.11,34.744 56.1118,34.7331 56.1049)),((40.2143 54.467,38.5511 53.2922,38.3395 53.2817,38.4609 53.226,38.0214 52.8989,37.8559 52.9188,37.135 53.4711,39.8151 55.3187,39.8205 55.2753,40.3948 55.2408,40.3948 54.8773,39.5485 54.8773,39.5485 54.5631,40.2143 54.467)),((40.5716 55.8007,40.5761 55.7884,40.5504 55.7875,40.5716 55.8007)),((40.4543 56.5923,40.2529 56.4682,39.7903 56.4121,39.8102 56.1914,38.2609 55.1775,37.7955 55.3956,38.4907 55.5327,38.1884 55.8564,38.944 56.0594,38.4339 56.2361,39.7863 57.025,39.7903 56.9929,40.3343 56.9599,40.4543 56.5923)),((40.1389 58.048,38.4915 57.1308,38.2186 57.2717,38.7325 57.4835,38.3737 57.6908,39.6392 58.3427,39.6392 58.0478,40.1389 58.048)),((37.5054 56.5484,37.463 56.5623,37.565 56.5843,37.5054 56.5484)),((38.0744 57.5312,38.128 57.516,37.9669 57.4734,38.0744 57.5312)),((40.4136 58.7241,40.3343 58.3821,39.7184 58.3823,40.4136 58.7241)),((39.8163 58.9766,39.4085 58.7696,38.5209 59.119,39.8163 58.9766)),((38.432 58.2584,38.3698 58.2869,38.7465 58.4255,38.432 58.2584)),((32.2175 58.3664,32.5691 58.5924,33.4734 58.8542,34.7428 59.5659,33.8361 58.6819,34.0496 58.6717,31.6514 57.1258,31.5088 57.4998,32.1738 58.0318,32.2175 58.3664)),((39.9942 53.358,40.0019 53.354,39.9877 53.3534,39.9942 53.358)),((39.2704 52.8471,39.5787 52.6996,39.1456 52.7572,39.2704 52.8471))) -------- MultiPolygon with Polygon with Holes MULTIPOLYGON(((33.1079 56.9523,32.9596 56.9434,33.1392 56.8934,33.2007 56.7768,33.7182 56.7292,33.8361 56.6953,35.71 56.3117,34.5917 56.2949,32.8387 56.3117,35.6798 55.6863,32.748 55.9072,33.5036 55.3785,35.0149 55.3613,34.2593 54.9642,35.0753 54.5981,34.1081 54.1757,34.7731 53.7847,34.7731 53.3243,33.1128 54.0852,31.627 54.7093,31.8413 54.9989,32.204 55.5156,31.5088 55.9411,31.7506 56.8609,31.5088 57.4998,32.1738 58.0318,32.2342 58.4928,32.25 58.4976,33.1079 56.9523)),((35.1489 56.5859,36.6724 56.4139,36.8799 56.4895,38.2186 56.0594,36.647 55.9411,38.0262 55.6546,37.9482 55.6376,36.8283 55.4471,36.9508 55.414,36.5845 55.3291,36.8822 54.975,36.0123 54.7554,36.919 53.8561,35.9216 53.8026,37.2165 53.0798,37.0604 52.9744,35.3776 53.0462,34.894 54.1226,35.6193 54.4929,34.8335 55.0162,36.4354 55.3441,35.1358 55.5327,36.5563 55.6352,34.7126 56.11,36.7074 56.211,35.1489 56.5859)),((37.2327 59.0233,37.3119 59.0258,38.0944 58.8545,37.2876 58.7226,37.2327 59.0233)),((37.4471 53.2343,36.9794 53.5878,37.3119 53.9273,36.7074 54.6506,37.0572 54.7635,37.9907 53.5925,37.4471 53.2343)),((34.7731 53.1793,34.7731 52.9188,33.1712 52.8276,32.4808 53.1989,34.7731 53.1793)),((40.4412 56.1511,38.3184 55.7179,38.1884 55.8564,38.944 56.0594,37.463 56.5623,38.9742 56.8774,38.5798 57.0849,39.0894 57.2553,39.7379 57.4051,39.7903 56.9929,40.3343 56.9599,40.4855 56.4957,39.7903 56.4121,39.8205 56.0763,40.425 56.1942,40.4412 56.1511)),((38.3092 56.9929,38.3093 56.9929,38.309 56.9928,38.3092 56.9929)),((40.3237 57.5365,40.3343 57.4673,40.0149 57.4677,40.3237 57.5365)),((39.2792 59.0373,38.8838 58.9777,38.5209 59.119,39.2792 59.0373))) -------- Polygon with Polygon with Holes diff --git a/tests/queries/0_stateless/01306_polygons_intersection.sql b/tests/queries/0_stateless/01306_polygons_intersection.sql index 144408ca0ae..5bfba6124cd 100644 --- a/tests/queries/0_stateless/01306_polygons_intersection.sql +++ b/tests/queries/0_stateless/01306_polygons_intersection.sql @@ -1,14 +1,17 @@ select polygonsIntersectionCartesian([[[(0., 0.),(0., 3.),(1., 2.9),(2., 2.6),(2.6, 2.),(2.9, 1.),(3., 0.),(0., 0.)]]], [[[(1., 1.),(1., 4.),(4., 4.),(4., 1.),(1., 1.)]]]); select polygonsIntersectionCartesian([[[(0., 0.),(0., 3.),(1., 2.9),(2., 2.6),(2.6, 2.),(2.9, 1.),(3., 0.),(0., 0.)]]], [[[(3., 3.),(3., 4.),(4., 4.),(4., 3.),(3., 3.)]]]); -select polygonsIntersectionSpherical([[[(4.346693, 50.858306), (4.367945, 50.852455), (4.366227, 50.840809), (4.344961, 50.833264), (4.338074, 50.848677), (4.346693, 50.858306)]]], [[[(25.0010, 136.9987), (17.7500, 142.5000), (11.3733, 142.5917)]]]); -select polygonsIntersectionSpherical([[[(4.3613577, 50.8651821), (4.349556, 50.8535879), (4.3602419, 50.8435626), (4.3830299, 50.8428851), (4.3904543, 50.8564867), (4.3613148, 50.8651279)]]], [[[(4.346693, 50.858306), (4.367945, 50.852455), (4.366227, 50.840809), (4.344961, 50.833264), (4.338074, 50.848677), (4.346693, 50.858306)]]]); +select arrayMap(a -> arrayMap(b -> arrayMap(c -> (round(c.1, 6), round(c.2, 6)), b), a), polygonsIntersectionSpherical([[[(4.346693, 50.858306), (4.367945, 50.852455), (4.366227, 50.840809), (4.344961, 50.833264), (4.338074, 50.848677), (4.346693, 50.858306)]]], [[[(25.0010, 136.9987), (17.7500, 142.5000), (11.3733, 142.5917)]]])); +select arrayMap(a -> arrayMap(b -> arrayMap(c -> (round(c.1, 6), round(c.2, 6)), b), a),polygonsIntersectionSpherical([[[(4.3613577, 50.8651821), (4.349556, 50.8535879), (4.3602419, 50.8435626), (4.3830299, 50.8428851), (4.3904543, 50.8564867), (4.3613148, 50.8651279)]]], [[[(4.346693, 50.858306), (4.367945, 50.852455), (4.366227, 50.840809), (4.344961, 50.833264), (4.338074, 50.848677), (4.346693, 50.858306)]]])); select '-------- MultiPolygon with Polygon'; -select wkt(polygonsIntersectionSpherical([[(29.453587685533865,59.779570356240356),(29.393139070478895,52.276266797422124),(40.636581470703206,59.38168915000267),(41.21084331372543,59.103467777099866),(29.786055068336193,52.146627480315004),(31.23682182965546,52.16517054781818),(41.69443223416517,58.85424941916091),(42.51048853740727,58.47703162291134),(32.59691566839227,52.22075341251539),(34.289476889931414,52.22075341251539),(43.02430176537451,58.07974369546071),(43.02430176537451,57.25537683364851),(35.468224883503325,52.2022335126388),(37.16078610504247,52.23926559241349),(43.02430176537451,56.26136189644947),(43.02430176537451,55.326904361850836),(38.33953409861437,52.16517054781818),(40.09254393520848,52.16517054781818),(44.4146199116388,55.3097062225408),(44.47506852669377,59.80998197603594),(39.72985224487867,59.931351417569715),(30.23941968124846,53.67744677450975),(30.20919537372098,54.63314259659509),(38.73245009647167,59.94649146557819),(37.2816833351524,59.97675082987618),(30.23941968124846,55.2752875586599),(30.33009260383092,56.19415599955667),(36.28428118674541,59.96162460231375),(34.863738732953635,59.97675082987618),(30.178971066193498,56.97640788219866),(30.178971066193498,57.91957806959033),(33.65476643185424,59.94649146557819),(32.32489690064491,59.94649146557819),(30.481214141468342,58.85424941916091),(30.571887064050795,59.99187015036608),(29.453587685533865,59.779570356240356)]], [[[(33.473420586689336,58.85424941916091),(32.23422397806246,58.492830557036),(32.173775363007486,58.03176922751564),(31.508840597402823,57.499784781503735),(31.750635057622702,56.86092686957355),(31.508840597402823,55.941082594334574),(32.20399967053497,55.515591939372456),(31.84130798020516,54.998862226280465),(31.418167674820367,54.422670886434275),(32.47601843828233,53.83826377018255),(32.08310244042503,53.408048308050866),(33.171177511414484,52.82758702113742),(34.77306581037117,52.91880107773494),(34.77306581037117,53.784726518357985),(34.108131044766516,54.17574726780569),(35.07530888564602,54.59813930694554),(34.25925258240394,54.96417435716029),(35.01486027059106,55.361278263643584),(33.50364489421682,55.37845402950552),(32.7480372060297,55.90721384574556),(35.67979503619571,55.68634475630185),(32.83871012861215,56.311688992608396),(34.591719965206266,56.29492065473883),(35.7100193437232,56.311688992608396),(33.83611227701915,56.695333481003644),(32.95960735872209,56.9434497616887),(36.072711034053015,57.091531913901434),(33.171177511414484,57.33702717078384),(36.193608264162954,57.499784781503735),(33.23162612646945,57.77481561306047),(36.43540272438284,58.04776787540811),(33.62454212432676,58.27099811968307),(36.344729801800376,58.54018474404165),(33.83611227701915,58.68186423448108),(34.74284150284369,59.565911441555244),(33.473420586689336,58.85424941916091)]], [[(34.65216858026123,58.91672306881671),(37.19101041256995,58.68186423448108),(36.01226241899805,58.28688958537609),(37.16078610504247,58.04776787540811),(35.74024365125068,57.79092907387934),(37.009664567405046,57.499784781503735),(35.77046795877817,57.25537683364851),(36.979440259877556,57.07510745541089),(34.22902827487645,56.794777197297435),(36.7074214921302,56.210968525786996),(34.712617195316206,56.10998276812964),(36.55629995449277,55.63519693782703),(35.13575750070099,55.53270067649592),(36.43540272438284,55.34409504165558),(34.83351442542614,55.01619492319591),(35.61934642114075,54.49294870011772),(34.89396304048112,54.12264226523038),(35.37755196092087,53.046178687628185),(37.43280487278982,52.95523300597458),(35.92158949641559,53.80257986695776),(36.91899164482259,53.856094327816805),(36.01226241899805,54.75541714463799),(37.765272255592166,55.189110239786885),(36.828318722240134,55.44708256557195),(38.03729102333953,55.652253637168315),(36.64697287707522,55.941082594334574),(38.21863686850443,56.05939028508024),(36.37495410932787,56.64551287174558),(38.30930979108689,56.992876013526654),(37.16078610504247,57.25537683364851),(38.127963945921984,57.516020773674256),(37.43280487278982,57.710289827306724),(38.33953409861437,57.935626886818994),(37.40258056526235,58.31865112960426),(38.58132855883426,58.744648733419496),(37.31190764267989,59.02578062465136),(34.65216858026123,58.91672306881671)]], [[(38.52087994377928,59.11898412389468),(39.54850639971376,58.713270635642914),(38.369758406141855,58.28688958537609),(38.85334732658162,58.06375936407028),(38.33953409861437,57.710289827306724),(38.73245009647167,57.48354156434209),(38.21863686850443,57.271721400459285),(38.97424455669155,56.87744603722649),(37.463029180317314,56.5623320541159),(38.94402024916407,56.05939028508024),(38.18841256097694,55.856355210835915),(38.490655636251795,55.53270067649592),(37.795496563119656,55.39562234093384),(38.30930979108689,55.154587013355666),(36.7074214921302,54.65063295250911),(37.31190764267989,53.92734063371401),(36.979440259877556,53.58783775557231),(37.855945178174615,52.91880107773497),(39.57873070724124,52.69956490610895),(38.33953409861437,53.281741738901104),(40.00187101262603,53.35396273604752),(39.54850639971376,53.58783775557231),(40.24366547284591,53.58783775557231),(39.97164670509855,53.98069568468355),(40.60635716317572,54.03398248547225),(40.39478701048334,54.44025165268903),(39.54850639971376,54.56310590284329),(39.54850639971376,54.87732350170489),(40.39478701048334,54.87732350170489),(40.39478701048334,55.24083903654295),(39.82052516746112,55.2752875586599),(39.760076552406154,55.75443792473942),(40.57613285564824,55.78844000174894),(40.425011318010824,56.19415599955667),(39.82052516746112,56.07626182891758),(39.79030085993364,56.41214455508424),(40.48545993306579,56.495655446714636),(40.33433839542836,56.95993246553937),(39.79030085993364,56.992876013526654),(39.72985224487867,57.46729112028032),(40.33433839542836,57.46729112028032),(40.24366547284591,58.04776787540811),(39.63917932229622,58.04776787540811),(39.63917932229622,58.382088724871295),(40.33433839542836,58.382088724871295),(40.45523562553831,58.9011152358548),(38.52087994377928,59.11898412389468)]]])) format TSV; +select wkt(arrayMap(a -> arrayMap(b -> arrayMap(c -> (round(c.1, 6), round(c.2, 6)), b), a), +polygonsIntersectionSpherical([[(29.453587685533865,59.779570356240356),(29.393139070478895,52.276266797422124),(40.636581470703206,59.38168915000267),(41.21084331372543,59.103467777099866),(29.786055068336193,52.146627480315004),(31.23682182965546,52.16517054781818),(41.69443223416517,58.85424941916091),(42.51048853740727,58.47703162291134),(32.59691566839227,52.22075341251539),(34.289476889931414,52.22075341251539),(43.02430176537451,58.07974369546071),(43.02430176537451,57.25537683364851),(35.468224883503325,52.2022335126388),(37.16078610504247,52.23926559241349),(43.02430176537451,56.26136189644947),(43.02430176537451,55.326904361850836),(38.33953409861437,52.16517054781818),(40.09254393520848,52.16517054781818),(44.4146199116388,55.3097062225408),(44.47506852669377,59.80998197603594),(39.72985224487867,59.931351417569715),(30.23941968124846,53.67744677450975),(30.20919537372098,54.63314259659509),(38.73245009647167,59.94649146557819),(37.2816833351524,59.97675082987618),(30.23941968124846,55.2752875586599),(30.33009260383092,56.19415599955667),(36.28428118674541,59.96162460231375),(34.863738732953635,59.97675082987618),(30.178971066193498,56.97640788219866),(30.178971066193498,57.91957806959033),(33.65476643185424,59.94649146557819),(32.32489690064491,59.94649146557819),(30.481214141468342,58.85424941916091),(30.571887064050795,59.99187015036608),(29.453587685533865,59.779570356240356)]], [[[(33.473420586689336,58.85424941916091),(32.23422397806246,58.492830557036),(32.173775363007486,58.03176922751564),(31.508840597402823,57.499784781503735),(31.750635057622702,56.86092686957355),(31.508840597402823,55.941082594334574),(32.20399967053497,55.515591939372456),(31.84130798020516,54.998862226280465),(31.418167674820367,54.422670886434275),(32.47601843828233,53.83826377018255),(32.08310244042503,53.408048308050866),(33.171177511414484,52.82758702113742),(34.77306581037117,52.91880107773494),(34.77306581037117,53.784726518357985),(34.108131044766516,54.17574726780569),(35.07530888564602,54.59813930694554),(34.25925258240394,54.96417435716029),(35.01486027059106,55.361278263643584),(33.50364489421682,55.37845402950552),(32.7480372060297,55.90721384574556),(35.67979503619571,55.68634475630185),(32.83871012861215,56.311688992608396),(34.591719965206266,56.29492065473883),(35.7100193437232,56.311688992608396),(33.83611227701915,56.695333481003644),(32.95960735872209,56.9434497616887),(36.072711034053015,57.091531913901434),(33.171177511414484,57.33702717078384),(36.193608264162954,57.499784781503735),(33.23162612646945,57.77481561306047),(36.43540272438284,58.04776787540811),(33.62454212432676,58.27099811968307),(36.344729801800376,58.54018474404165),(33.83611227701915,58.68186423448108),(34.74284150284369,59.565911441555244),(33.473420586689336,58.85424941916091)]], [[(34.65216858026123,58.91672306881671),(37.19101041256995,58.68186423448108),(36.01226241899805,58.28688958537609),(37.16078610504247,58.04776787540811),(35.74024365125068,57.79092907387934),(37.009664567405046,57.499784781503735),(35.77046795877817,57.25537683364851),(36.979440259877556,57.07510745541089),(34.22902827487645,56.794777197297435),(36.7074214921302,56.210968525786996),(34.712617195316206,56.10998276812964),(36.55629995449277,55.63519693782703),(35.13575750070099,55.53270067649592),(36.43540272438284,55.34409504165558),(34.83351442542614,55.01619492319591),(35.61934642114075,54.49294870011772),(34.89396304048112,54.12264226523038),(35.37755196092087,53.046178687628185),(37.43280487278982,52.95523300597458),(35.92158949641559,53.80257986695776),(36.91899164482259,53.856094327816805),(36.01226241899805,54.75541714463799),(37.765272255592166,55.189110239786885),(36.828318722240134,55.44708256557195),(38.03729102333953,55.652253637168315),(36.64697287707522,55.941082594334574),(38.21863686850443,56.05939028508024),(36.37495410932787,56.64551287174558),(38.30930979108689,56.992876013526654),(37.16078610504247,57.25537683364851),(38.127963945921984,57.516020773674256),(37.43280487278982,57.710289827306724),(38.33953409861437,57.935626886818994),(37.40258056526235,58.31865112960426),(38.58132855883426,58.744648733419496),(37.31190764267989,59.02578062465136),(34.65216858026123,58.91672306881671)]], [[(38.52087994377928,59.11898412389468),(39.54850639971376,58.713270635642914),(38.369758406141855,58.28688958537609),(38.85334732658162,58.06375936407028),(38.33953409861437,57.710289827306724),(38.73245009647167,57.48354156434209),(38.21863686850443,57.271721400459285),(38.97424455669155,56.87744603722649),(37.463029180317314,56.5623320541159),(38.94402024916407,56.05939028508024),(38.18841256097694,55.856355210835915),(38.490655636251795,55.53270067649592),(37.795496563119656,55.39562234093384),(38.30930979108689,55.154587013355666),(36.7074214921302,54.65063295250911),(37.31190764267989,53.92734063371401),(36.979440259877556,53.58783775557231),(37.855945178174615,52.91880107773497),(39.57873070724124,52.69956490610895),(38.33953409861437,53.281741738901104),(40.00187101262603,53.35396273604752),(39.54850639971376,53.58783775557231),(40.24366547284591,53.58783775557231),(39.97164670509855,53.98069568468355),(40.60635716317572,54.03398248547225),(40.39478701048334,54.44025165268903),(39.54850639971376,54.56310590284329),(39.54850639971376,54.87732350170489),(40.39478701048334,54.87732350170489),(40.39478701048334,55.24083903654295),(39.82052516746112,55.2752875586599),(39.760076552406154,55.75443792473942),(40.57613285564824,55.78844000174894),(40.425011318010824,56.19415599955667),(39.82052516746112,56.07626182891758),(39.79030085993364,56.41214455508424),(40.48545993306579,56.495655446714636),(40.33433839542836,56.95993246553937),(39.79030085993364,56.992876013526654),(39.72985224487867,57.46729112028032),(40.33433839542836,57.46729112028032),(40.24366547284591,58.04776787540811),(39.63917932229622,58.04776787540811),(39.63917932229622,58.382088724871295),(40.33433839542836,58.382088724871295),(40.45523562553831,58.9011152358548),(38.52087994377928,59.11898412389468)]]]))) format TSV; select '-------- MultiPolygon with Polygon with Holes'; -select wkt(polygonsIntersectionSpherical([[[(33.473420586689336,58.85424941916091),(32.23422397806246,58.492830557036),(32.173775363007486,58.03176922751564),(31.508840597402823,57.499784781503735),(31.750635057622702,56.86092686957355),(31.508840597402823,55.941082594334574),(32.20399967053497,55.515591939372456),(31.84130798020516,54.998862226280465),(31.418167674820367,54.422670886434275),(32.47601843828233,53.83826377018255),(32.08310244042503,53.408048308050866),(33.171177511414484,52.82758702113742),(34.77306581037117,52.91880107773494),(34.77306581037117,53.784726518357985),(34.108131044766516,54.17574726780569),(35.07530888564602,54.59813930694554),(34.25925258240394,54.96417435716029),(35.01486027059106,55.361278263643584),(33.50364489421682,55.37845402950552),(32.7480372060297,55.90721384574556),(35.67979503619571,55.68634475630185),(32.83871012861215,56.311688992608396),(34.591719965206266,56.29492065473883),(35.7100193437232,56.311688992608396),(33.83611227701915,56.695333481003644),(32.95960735872209,56.9434497616887),(36.072711034053015,57.091531913901434),(33.171177511414484,57.33702717078384),(36.193608264162954,57.499784781503735),(33.23162612646945,57.77481561306047),(36.43540272438284,58.04776787540811),(33.62454212432676,58.27099811968307),(36.344729801800376,58.54018474404165),(33.83611227701915,58.68186423448108),(34.74284150284369,59.565911441555244),(33.473420586689336,58.85424941916091)]], [[(34.65216858026123,58.91672306881671),(37.19101041256995,58.68186423448108),(36.01226241899805,58.28688958537609),(37.16078610504247,58.04776787540811),(35.74024365125068,57.79092907387934),(37.009664567405046,57.499784781503735),(35.77046795877817,57.25537683364851),(36.979440259877556,57.07510745541089),(34.22902827487645,56.794777197297435),(36.7074214921302,56.210968525786996),(34.712617195316206,56.10998276812964),(36.55629995449277,55.63519693782703),(35.13575750070099,55.53270067649592),(36.43540272438284,55.34409504165558),(34.83351442542614,55.01619492319591),(35.61934642114075,54.49294870011772),(34.89396304048112,54.12264226523038),(35.37755196092087,53.046178687628185),(37.43280487278982,52.95523300597458),(35.92158949641559,53.80257986695776),(36.91899164482259,53.856094327816805),(36.01226241899805,54.75541714463799),(37.765272255592166,55.189110239786885),(36.828318722240134,55.44708256557195),(38.03729102333953,55.652253637168315),(36.64697287707522,55.941082594334574),(38.21863686850443,56.05939028508024),(36.37495410932787,56.64551287174558),(38.30930979108689,56.992876013526654),(37.16078610504247,57.25537683364851),(38.127963945921984,57.516020773674256),(37.43280487278982,57.710289827306724),(38.33953409861437,57.935626886818994),(37.40258056526235,58.31865112960426),(38.58132855883426,58.744648733419496),(37.31190764267989,59.02578062465136),(34.65216858026123,58.91672306881671)]], [[(38.52087994377928,59.11898412389468),(39.54850639971376,58.713270635642914),(38.369758406141855,58.28688958537609),(38.85334732658162,58.06375936407028),(38.33953409861437,57.710289827306724),(38.73245009647167,57.48354156434209),(38.21863686850443,57.271721400459285),(38.97424455669155,56.87744603722649),(37.463029180317314,56.5623320541159),(38.94402024916407,56.05939028508024),(38.18841256097694,55.856355210835915),(38.490655636251795,55.53270067649592),(37.795496563119656,55.39562234093384),(38.30930979108689,55.154587013355666),(36.7074214921302,54.65063295250911),(37.31190764267989,53.92734063371401),(36.979440259877556,53.58783775557231),(37.855945178174615,52.91880107773497),(39.57873070724124,52.69956490610895),(38.33953409861437,53.281741738901104),(40.00187101262603,53.35396273604752),(39.54850639971376,53.58783775557231),(40.24366547284591,53.58783775557231),(39.97164670509855,53.98069568468355),(40.60635716317572,54.03398248547225),(40.39478701048334,54.44025165268903),(39.54850639971376,54.56310590284329),(39.54850639971376,54.87732350170489),(40.39478701048334,54.87732350170489),(40.39478701048334,55.24083903654295),(39.82052516746112,55.2752875586599),(39.760076552406154,55.75443792473942),(40.57613285564824,55.78844000174894),(40.425011318010824,56.19415599955667),(39.82052516746112,56.07626182891758),(39.79030085993364,56.41214455508424),(40.48545993306579,56.495655446714636),(40.33433839542836,56.95993246553937),(39.79030085993364,56.992876013526654),(39.72985224487867,57.46729112028032),(40.33433839542836,57.46729112028032),(40.24366547284591,58.04776787540811),(39.63917932229622,58.04776787540811),(39.63917932229622,58.382088724871295),(40.33433839542836,58.382088724871295),(40.45523562553831,58.9011152358548),(38.52087994377928,59.11898412389468)]]], [[(24.367675781249993,61.45977057029751),(19.577636718749993,58.67693767258692),(19.577636718749993,57.492213666700735),(19.445800781249996,55.87531083569678),(19.445800781249996,54.085173420886775),(17.468261718749996,53.014783245859235),(20.017089843749993,51.563412328675895),(21.203613281249993,50.205033264943324),(26.125488281249993,50.40151532278236),(27.22412109374999,48.980216985374994),(32.80517578124999,49.525208341974405),(35.26611328124999,48.74894534343292),(36.93603515624999,49.66762782262194),(42.56103515625,48.77791275550183),(43.92333984374999,49.8096315635631),(47.17529296875,49.152969656170455),(49.28466796875,50.54136296522162),(48.05419921875,51.17934297928929),(51.39404296875,52.48278022207825),(50.64697265625,53.014783245859235),(52.88818359375,53.93021986394004),(51.65771484374999,54.29088164657006),(52.66845703125,55.825973254619015),(50.25146484375,56.145549500679095),(51.92138671875,57.914847767009206),(49.15283203125,58.17070248348605),(49.59228515625,60.086762746260064),(47.043457031249986,59.88893689676584),(43.57177734375,61.37567331572748),(42.64892578125,60.630101766266705),(36.89208984374999,62.000904713685856),(36.01318359374999,61.143235250840576),(31.398925781249993,62.02152819100766),(30.563964843749996,61.05828537037917),(26.872558593749993,61.71070595883174),(26.652832031249993,61.10078883158897),(24.367675781249993,61.45977057029751)], [(24.455566406249993,59.42272750081452),(21.203613281249993,58.49369382056807),(21.335449218749993,56.89700392127261),(21.599121093749993,55.92458580482949),(25.202636718749993,55.998380955359636),(28.850097656249993,57.06463027327854),(27.09228515625,57.844750992890994),(28.806152343749996,59.17592824927138),(26.257324218749993,59.17592824927138),(24.455566406249993,59.42272750081452)], [(35.13427734375,59.84481485969107),(31.970214843749993,58.97266715450152),(33.20068359374999,56.776808316568406),(36.67236328125,56.41390137600675),(39.08935546874999,57.25528054528888),(42.69287109374999,58.03137242177638),(40.89111328124999,59.26588062825809),(37.28759765625,58.722598828043374),(37.11181640624999,59.66774058164964),(35.13427734375,59.84481485969107)], [(29.157714843749993,55.75184939173528),(22.565917968749993,55.128649068488784),(22.565917968749993,53.54030739150019),(22.038574218749996,51.48138289610097),(26.257324218749993,51.42661449707484),(30.124511718749993,50.54136296522162),(32.18994140624999,51.17934297928929),(30.124511718749993,53.173119202640635),(35.09033203124999,53.173119202640635),(33.11279296875,54.085173420886775),(29.597167968749993,55.50374985927513),(29.157714843749993,55.75184939173528)], [(42.82470703125,56.58369172128337),(36.584472656249986,55.329144408405085),(37.99072265625,53.592504809039355),(34.95849609374999,51.48138289610097),(36.54052734374999,50.40151532278236),(39.66064453124999,50.289339253291786),(39.79248046875,52.13348804077148),(41.77001953125,50.68079714532166),(44.49462890624999,51.97134580885171),(47.30712890624999,52.509534770327264),(44.05517578125,53.54030739150019),(46.60400390625,53.696706475303245),(47.61474609375,55.40406982700608),(45.37353515625,55.40406982700608),(42.82470703125,56.58369172128337)]])) format TSV; +select wkt(arrayMap(a -> arrayMap(b -> arrayMap(c -> (round(c.1, 6), round(c.2, 6)), b), a), +polygonsIntersectionSpherical([[[(33.473420586689336,58.85424941916091),(32.23422397806246,58.492830557036),(32.173775363007486,58.03176922751564),(31.508840597402823,57.499784781503735),(31.750635057622702,56.86092686957355),(31.508840597402823,55.941082594334574),(32.20399967053497,55.515591939372456),(31.84130798020516,54.998862226280465),(31.418167674820367,54.422670886434275),(32.47601843828233,53.83826377018255),(32.08310244042503,53.408048308050866),(33.171177511414484,52.82758702113742),(34.77306581037117,52.91880107773494),(34.77306581037117,53.784726518357985),(34.108131044766516,54.17574726780569),(35.07530888564602,54.59813930694554),(34.25925258240394,54.96417435716029),(35.01486027059106,55.361278263643584),(33.50364489421682,55.37845402950552),(32.7480372060297,55.90721384574556),(35.67979503619571,55.68634475630185),(32.83871012861215,56.311688992608396),(34.591719965206266,56.29492065473883),(35.7100193437232,56.311688992608396),(33.83611227701915,56.695333481003644),(32.95960735872209,56.9434497616887),(36.072711034053015,57.091531913901434),(33.171177511414484,57.33702717078384),(36.193608264162954,57.499784781503735),(33.23162612646945,57.77481561306047),(36.43540272438284,58.04776787540811),(33.62454212432676,58.27099811968307),(36.344729801800376,58.54018474404165),(33.83611227701915,58.68186423448108),(34.74284150284369,59.565911441555244),(33.473420586689336,58.85424941916091)]], [[(34.65216858026123,58.91672306881671),(37.19101041256995,58.68186423448108),(36.01226241899805,58.28688958537609),(37.16078610504247,58.04776787540811),(35.74024365125068,57.79092907387934),(37.009664567405046,57.499784781503735),(35.77046795877817,57.25537683364851),(36.979440259877556,57.07510745541089),(34.22902827487645,56.794777197297435),(36.7074214921302,56.210968525786996),(34.712617195316206,56.10998276812964),(36.55629995449277,55.63519693782703),(35.13575750070099,55.53270067649592),(36.43540272438284,55.34409504165558),(34.83351442542614,55.01619492319591),(35.61934642114075,54.49294870011772),(34.89396304048112,54.12264226523038),(35.37755196092087,53.046178687628185),(37.43280487278982,52.95523300597458),(35.92158949641559,53.80257986695776),(36.91899164482259,53.856094327816805),(36.01226241899805,54.75541714463799),(37.765272255592166,55.189110239786885),(36.828318722240134,55.44708256557195),(38.03729102333953,55.652253637168315),(36.64697287707522,55.941082594334574),(38.21863686850443,56.05939028508024),(36.37495410932787,56.64551287174558),(38.30930979108689,56.992876013526654),(37.16078610504247,57.25537683364851),(38.127963945921984,57.516020773674256),(37.43280487278982,57.710289827306724),(38.33953409861437,57.935626886818994),(37.40258056526235,58.31865112960426),(38.58132855883426,58.744648733419496),(37.31190764267989,59.02578062465136),(34.65216858026123,58.91672306881671)]], [[(38.52087994377928,59.11898412389468),(39.54850639971376,58.713270635642914),(38.369758406141855,58.28688958537609),(38.85334732658162,58.06375936407028),(38.33953409861437,57.710289827306724),(38.73245009647167,57.48354156434209),(38.21863686850443,57.271721400459285),(38.97424455669155,56.87744603722649),(37.463029180317314,56.5623320541159),(38.94402024916407,56.05939028508024),(38.18841256097694,55.856355210835915),(38.490655636251795,55.53270067649592),(37.795496563119656,55.39562234093384),(38.30930979108689,55.154587013355666),(36.7074214921302,54.65063295250911),(37.31190764267989,53.92734063371401),(36.979440259877556,53.58783775557231),(37.855945178174615,52.91880107773497),(39.57873070724124,52.69956490610895),(38.33953409861437,53.281741738901104),(40.00187101262603,53.35396273604752),(39.54850639971376,53.58783775557231),(40.24366547284591,53.58783775557231),(39.97164670509855,53.98069568468355),(40.60635716317572,54.03398248547225),(40.39478701048334,54.44025165268903),(39.54850639971376,54.56310590284329),(39.54850639971376,54.87732350170489),(40.39478701048334,54.87732350170489),(40.39478701048334,55.24083903654295),(39.82052516746112,55.2752875586599),(39.760076552406154,55.75443792473942),(40.57613285564824,55.78844000174894),(40.425011318010824,56.19415599955667),(39.82052516746112,56.07626182891758),(39.79030085993364,56.41214455508424),(40.48545993306579,56.495655446714636),(40.33433839542836,56.95993246553937),(39.79030085993364,56.992876013526654),(39.72985224487867,57.46729112028032),(40.33433839542836,57.46729112028032),(40.24366547284591,58.04776787540811),(39.63917932229622,58.04776787540811),(39.63917932229622,58.382088724871295),(40.33433839542836,58.382088724871295),(40.45523562553831,58.9011152358548),(38.52087994377928,59.11898412389468)]]], [[(24.367675781249993,61.45977057029751),(19.577636718749993,58.67693767258692),(19.577636718749993,57.492213666700735),(19.445800781249996,55.87531083569678),(19.445800781249996,54.085173420886775),(17.468261718749996,53.014783245859235),(20.017089843749993,51.563412328675895),(21.203613281249993,50.205033264943324),(26.125488281249993,50.40151532278236),(27.22412109374999,48.980216985374994),(32.80517578124999,49.525208341974405),(35.26611328124999,48.74894534343292),(36.93603515624999,49.66762782262194),(42.56103515625,48.77791275550183),(43.92333984374999,49.8096315635631),(47.17529296875,49.152969656170455),(49.28466796875,50.54136296522162),(48.05419921875,51.17934297928929),(51.39404296875,52.48278022207825),(50.64697265625,53.014783245859235),(52.88818359375,53.93021986394004),(51.65771484374999,54.29088164657006),(52.66845703125,55.825973254619015),(50.25146484375,56.145549500679095),(51.92138671875,57.914847767009206),(49.15283203125,58.17070248348605),(49.59228515625,60.086762746260064),(47.043457031249986,59.88893689676584),(43.57177734375,61.37567331572748),(42.64892578125,60.630101766266705),(36.89208984374999,62.000904713685856),(36.01318359374999,61.143235250840576),(31.398925781249993,62.02152819100766),(30.563964843749996,61.05828537037917),(26.872558593749993,61.71070595883174),(26.652832031249993,61.10078883158897),(24.367675781249993,61.45977057029751)], [(24.455566406249993,59.42272750081452),(21.203613281249993,58.49369382056807),(21.335449218749993,56.89700392127261),(21.599121093749993,55.92458580482949),(25.202636718749993,55.998380955359636),(28.850097656249993,57.06463027327854),(27.09228515625,57.844750992890994),(28.806152343749996,59.17592824927138),(26.257324218749993,59.17592824927138),(24.455566406249993,59.42272750081452)], [(35.13427734375,59.84481485969107),(31.970214843749993,58.97266715450152),(33.20068359374999,56.776808316568406),(36.67236328125,56.41390137600675),(39.08935546874999,57.25528054528888),(42.69287109374999,58.03137242177638),(40.89111328124999,59.26588062825809),(37.28759765625,58.722598828043374),(37.11181640624999,59.66774058164964),(35.13427734375,59.84481485969107)], [(29.157714843749993,55.75184939173528),(22.565917968749993,55.128649068488784),(22.565917968749993,53.54030739150019),(22.038574218749996,51.48138289610097),(26.257324218749993,51.42661449707484),(30.124511718749993,50.54136296522162),(32.18994140624999,51.17934297928929),(30.124511718749993,53.173119202640635),(35.09033203124999,53.173119202640635),(33.11279296875,54.085173420886775),(29.597167968749993,55.50374985927513),(29.157714843749993,55.75184939173528)], [(42.82470703125,56.58369172128337),(36.584472656249986,55.329144408405085),(37.99072265625,53.592504809039355),(34.95849609374999,51.48138289610097),(36.54052734374999,50.40151532278236),(39.66064453124999,50.289339253291786),(39.79248046875,52.13348804077148),(41.77001953125,50.68079714532166),(44.49462890624999,51.97134580885171),(47.30712890624999,52.509534770327264),(44.05517578125,53.54030739150019),(46.60400390625,53.696706475303245),(47.61474609375,55.40406982700608),(45.37353515625,55.40406982700608),(42.82470703125,56.58369172128337)]]))) format TSV; select '-------- Polygon with Polygon with Holes'; -select wkt(polygonsIntersectionSpherical([[(29.453587685533865,59.779570356240356),(29.393139070478895,52.276266797422124),(40.636581470703206,59.38168915000267),(41.21084331372543,59.103467777099866),(29.786055068336193,52.146627480315004),(31.23682182965546,52.16517054781818),(41.69443223416517,58.85424941916091),(42.51048853740727,58.47703162291134),(32.59691566839227,52.22075341251539),(34.289476889931414,52.22075341251539),(43.02430176537451,58.07974369546071),(43.02430176537451,57.25537683364851),(35.468224883503325,52.2022335126388),(37.16078610504247,52.23926559241349),(43.02430176537451,56.26136189644947),(43.02430176537451,55.326904361850836),(38.33953409861437,52.16517054781818),(40.09254393520848,52.16517054781818),(44.4146199116388,55.3097062225408),(44.47506852669377,59.80998197603594),(39.72985224487867,59.931351417569715),(30.23941968124846,53.67744677450975),(30.20919537372098,54.63314259659509),(38.73245009647167,59.94649146557819),(37.2816833351524,59.97675082987618),(30.23941968124846,55.2752875586599),(30.33009260383092,56.19415599955667),(36.28428118674541,59.96162460231375),(34.863738732953635,59.97675082987618),(30.178971066193498,56.97640788219866),(30.178971066193498,57.91957806959033),(33.65476643185424,59.94649146557819),(32.32489690064491,59.94649146557819),(30.481214141468342,58.85424941916091),(30.571887064050795,59.99187015036608),(29.453587685533865,59.779570356240356)]], [[(24.367675781249993,61.45977057029751),(19.577636718749993,58.67693767258692),(19.577636718749993,57.492213666700735),(19.445800781249996,55.87531083569678),(19.445800781249996,54.085173420886775),(17.468261718749996,53.014783245859235),(20.017089843749993,51.563412328675895),(21.203613281249993,50.205033264943324),(26.125488281249993,50.40151532278236),(27.22412109374999,48.980216985374994),(32.80517578124999,49.525208341974405),(35.26611328124999,48.74894534343292),(36.93603515624999,49.66762782262194),(42.56103515625,48.77791275550183),(43.92333984374999,49.8096315635631),(47.17529296875,49.152969656170455),(49.28466796875,50.54136296522162),(48.05419921875,51.17934297928929),(51.39404296875,52.48278022207825),(50.64697265625,53.014783245859235),(52.88818359375,53.93021986394004),(51.65771484374999,54.29088164657006),(52.66845703125,55.825973254619015),(50.25146484375,56.145549500679095),(51.92138671875,57.914847767009206),(49.15283203125,58.17070248348605),(49.59228515625,60.086762746260064),(47.043457031249986,59.88893689676584),(43.57177734375,61.37567331572748),(42.64892578125,60.630101766266705),(36.89208984374999,62.000904713685856),(36.01318359374999,61.143235250840576),(31.398925781249993,62.02152819100766),(30.563964843749996,61.05828537037917),(26.872558593749993,61.71070595883174),(26.652832031249993,61.10078883158897),(24.367675781249993,61.45977057029751)], [(24.455566406249993,59.42272750081452),(21.203613281249993,58.49369382056807),(21.335449218749993,56.89700392127261),(21.599121093749993,55.92458580482949),(25.202636718749993,55.998380955359636),(28.850097656249993,57.06463027327854),(27.09228515625,57.844750992890994),(28.806152343749996,59.17592824927138),(26.257324218749993,59.17592824927138),(24.455566406249993,59.42272750081452)], [(35.13427734375,59.84481485969107),(31.970214843749993,58.97266715450152),(33.20068359374999,56.776808316568406),(36.67236328125,56.41390137600675),(39.08935546874999,57.25528054528888),(42.69287109374999,58.03137242177638),(40.89111328124999,59.26588062825809),(37.28759765625,58.722598828043374),(37.11181640624999,59.66774058164964),(35.13427734375,59.84481485969107)], [(29.157714843749993,55.75184939173528),(22.565917968749993,55.128649068488784),(22.565917968749993,53.54030739150019),(22.038574218749996,51.48138289610097),(26.257324218749993,51.42661449707484),(30.124511718749993,50.54136296522162),(32.18994140624999,51.17934297928929),(30.124511718749993,53.173119202640635),(35.09033203124999,53.173119202640635),(33.11279296875,54.085173420886775),(29.597167968749993,55.50374985927513),(29.157714843749993,55.75184939173528)], [(42.82470703125,56.58369172128337),(36.584472656249986,55.329144408405085),(37.99072265625,53.592504809039355),(34.95849609374999,51.48138289610097),(36.54052734374999,50.40151532278236),(39.66064453124999,50.289339253291786),(39.79248046875,52.13348804077148),(41.77001953125,50.68079714532166),(44.49462890624999,51.97134580885171),(47.30712890624999,52.509534770327264),(44.05517578125,53.54030739150019),(46.60400390625,53.696706475303245),(47.61474609375,55.40406982700608),(45.37353515625,55.40406982700608),(42.82470703125,56.58369172128337)]])) format TSV; +select wkt(arrayMap(a -> arrayMap(b -> arrayMap(c -> (round(c.1, 6), round(c.2, 6)), b), a), +polygonsIntersectionSpherical([[(29.453587685533865,59.779570356240356),(29.393139070478895,52.276266797422124),(40.636581470703206,59.38168915000267),(41.21084331372543,59.103467777099866),(29.786055068336193,52.146627480315004),(31.23682182965546,52.16517054781818),(41.69443223416517,58.85424941916091),(42.51048853740727,58.47703162291134),(32.59691566839227,52.22075341251539),(34.289476889931414,52.22075341251539),(43.02430176537451,58.07974369546071),(43.02430176537451,57.25537683364851),(35.468224883503325,52.2022335126388),(37.16078610504247,52.23926559241349),(43.02430176537451,56.26136189644947),(43.02430176537451,55.326904361850836),(38.33953409861437,52.16517054781818),(40.09254393520848,52.16517054781818),(44.4146199116388,55.3097062225408),(44.47506852669377,59.80998197603594),(39.72985224487867,59.931351417569715),(30.23941968124846,53.67744677450975),(30.20919537372098,54.63314259659509),(38.73245009647167,59.94649146557819),(37.2816833351524,59.97675082987618),(30.23941968124846,55.2752875586599),(30.33009260383092,56.19415599955667),(36.28428118674541,59.96162460231375),(34.863738732953635,59.97675082987618),(30.178971066193498,56.97640788219866),(30.178971066193498,57.91957806959033),(33.65476643185424,59.94649146557819),(32.32489690064491,59.94649146557819),(30.481214141468342,58.85424941916091),(30.571887064050795,59.99187015036608),(29.453587685533865,59.779570356240356)]], [[(24.367675781249993,61.45977057029751),(19.577636718749993,58.67693767258692),(19.577636718749993,57.492213666700735),(19.445800781249996,55.87531083569678),(19.445800781249996,54.085173420886775),(17.468261718749996,53.014783245859235),(20.017089843749993,51.563412328675895),(21.203613281249993,50.205033264943324),(26.125488281249993,50.40151532278236),(27.22412109374999,48.980216985374994),(32.80517578124999,49.525208341974405),(35.26611328124999,48.74894534343292),(36.93603515624999,49.66762782262194),(42.56103515625,48.77791275550183),(43.92333984374999,49.8096315635631),(47.17529296875,49.152969656170455),(49.28466796875,50.54136296522162),(48.05419921875,51.17934297928929),(51.39404296875,52.48278022207825),(50.64697265625,53.014783245859235),(52.88818359375,53.93021986394004),(51.65771484374999,54.29088164657006),(52.66845703125,55.825973254619015),(50.25146484375,56.145549500679095),(51.92138671875,57.914847767009206),(49.15283203125,58.17070248348605),(49.59228515625,60.086762746260064),(47.043457031249986,59.88893689676584),(43.57177734375,61.37567331572748),(42.64892578125,60.630101766266705),(36.89208984374999,62.000904713685856),(36.01318359374999,61.143235250840576),(31.398925781249993,62.02152819100766),(30.563964843749996,61.05828537037917),(26.872558593749993,61.71070595883174),(26.652832031249993,61.10078883158897),(24.367675781249993,61.45977057029751)], [(24.455566406249993,59.42272750081452),(21.203613281249993,58.49369382056807),(21.335449218749993,56.89700392127261),(21.599121093749993,55.92458580482949),(25.202636718749993,55.998380955359636),(28.850097656249993,57.06463027327854),(27.09228515625,57.844750992890994),(28.806152343749996,59.17592824927138),(26.257324218749993,59.17592824927138),(24.455566406249993,59.42272750081452)], [(35.13427734375,59.84481485969107),(31.970214843749993,58.97266715450152),(33.20068359374999,56.776808316568406),(36.67236328125,56.41390137600675),(39.08935546874999,57.25528054528888),(42.69287109374999,58.03137242177638),(40.89111328124999,59.26588062825809),(37.28759765625,58.722598828043374),(37.11181640624999,59.66774058164964),(35.13427734375,59.84481485969107)], [(29.157714843749993,55.75184939173528),(22.565917968749993,55.128649068488784),(22.565917968749993,53.54030739150019),(22.038574218749996,51.48138289610097),(26.257324218749993,51.42661449707484),(30.124511718749993,50.54136296522162),(32.18994140624999,51.17934297928929),(30.124511718749993,53.173119202640635),(35.09033203124999,53.173119202640635),(33.11279296875,54.085173420886775),(29.597167968749993,55.50374985927513),(29.157714843749993,55.75184939173528)], [(42.82470703125,56.58369172128337),(36.584472656249986,55.329144408405085),(37.99072265625,53.592504809039355),(34.95849609374999,51.48138289610097),(36.54052734374999,50.40151532278236),(39.66064453124999,50.289339253291786),(39.79248046875,52.13348804077148),(41.77001953125,50.68079714532166),(44.49462890624999,51.97134580885171),(47.30712890624999,52.509534770327264),(44.05517578125,53.54030739150019),(46.60400390625,53.696706475303245),(47.61474609375,55.40406982700608),(45.37353515625,55.40406982700608),(42.82470703125,56.58369172128337)]]))) format TSV; diff --git a/tests/queries/0_stateless/01720_country_perimeter_and_area.reference b/tests/queries/0_stateless/01720_country_perimeter_and_area.reference index 8a9690791c6..461aea090e7 100644 --- a/tests/queries/0_stateless/01720_country_perimeter_and_area.reference +++ b/tests/queries/0_stateless/01720_country_perimeter_and_area.reference @@ -1,214 +1,214 @@ -Dhekelia Sovereign Base Area 0.0186259930051051 -Kyrgyzstan 0.5868323961091907 +Dhekelia Sovereign Base Area 0.018626 +Kyrgyzstan 0.586832 ------------------------------------- -Dhekelia Sovereign Base Area 0.000003139488070896512 -Kyrgyzstan 0.004895645023822883 +Dhekelia Sovereign Base Area 0.000003 +Kyrgyzstan 0.004896 ------------------------------------- -Aruba 0.011249330810410983 -Afghanistan 0.8199216326776404 -Albania 0.17108622597702605 -Andorra 0.015145740647213184 -Ashmore and Cartier Islands 0.001111472909012953 -Austria 0.3258464621357028 -Burundi 0.1409500621452211 -Belgium 0.1794463601873955 -Benin 0.31426073515874664 -Burkina Faso 0.5144381682226761 -Bulgaria 0.3083164214454252 -Bahrain 0.02137170357214413 -Bosnia and Herzegovina 0.20611959113245232 -Bajo Nuevo Bank (Petrel Is.) 0.0001254597070361587 -Saint Barthelemy 0.0032990108720812672 -Belarus 0.42899119772830474 -Bolivia 0.9279328001326348 -Barbados 0.014116142490651021 -Bhutan 0.1601735058766338 -Botswana 0.5896697538755427 -Central African Republic 0.7760222837198817 -Switzerland 0.2318851512510408 -Clipperton Island 0.0014072924221565273 -Cameroon 0.8001045813665599 -Republic of Congo 0.6904316055863188 -Coral Sea Islands 0.00011634674137689659 -Curaçao 0.02078862020307983 -Czech Republic 0.2708588915805718 -Djibouti 0.12937731543684822 -Dominica 0.020094439807419574 -Algeria 1.1549683948032776 -Ethiopia 0.8210654364815099 -Georgia 0.26823008017781313 -Ghana 0.4056578143818251 -Gibraltar 0.0014059440610631154 -Guinea 0.6350853755877334 -Gambia 0.19279774895359095 -Guatemala 0.3030953561509038 -Guam 0.020321390076536976 -Heard Island and McDonald Islands 0.017334896920453105 -Hungary 0.2617732480910806 -Isle of Man 0.01875803631141408 -Iraq 0.5469861219502402 -Israel 0.19353851895699914 -Jamaica 0.10055860979159512 -Jersey 0.008427337812134537 -Jordan 0.2642243503964102 -Baykonur Cosmodrome 0.04482995477542441 -Siachen Glacier 0.03872116827341272 -Kosovo 0.08773172991408161 -Laos 0.6899867972760174 -Lebanon 0.09676977254650951 -Liberia 0.2961649538030388 -Libya 0.9538430912224716 -Saint Lucia 0.016786201647759867 -Liechtenstein 0.009288582116863231 -Lesotho 0.12315874900320756 -Luxembourg 0.04125996057810259 -Latvia 0.24488610945731157 -Saint Martin 0.006547834154217771 -Morocco 0.8817924249630141 -Monaco 0.0026049777439637527 -Moldova 0.20765701819586885 -Macedonia 0.1128831074330059 -Mali 1.1385970015559317 -Montenegro 0.11756794062084858 -Mongolia 1.142306166871007 -Montserrat 0.006620100691409788 -Namibia 0.843464957679987 -Niger 0.8780744302377772 -Norfolk Island 0.004912027225339993 -Niue 0.009881892958363517 -Nepal 0.4076113675280835 -Nauru 0.0031205159769295255 -Poland 0.48922069488271314 -Paraguay 0.5475256537493991 -Qatar 0.09362771431858698 -Romania 0.44095021664473105 -Rwanda 0.1293663890297039 -Western Sahara 0.4691920993279596 -Scarborough Reef 0.00019842225207367386 -South Sudan 0.7584190842556537 -Senegal 0.5883247226863264 -Serranilla Bank 0.0002389083935906293 -Singapore 0.015233384733369614 -San Marino 0.004596873449598911 -Somaliland 0.3096791489207226 -Somalia 0.6879915318072617 -Republic of Serbia 0.29677234233404165 -Suriname 0.32255243342976203 -Slovakia 0.19843599488831584 -Slovenia 0.14713148471782736 -Swaziland 0.08434161089555517 -Sint Maarten 0.0037955305365309296 -Syria 0.35675522352394456 -Chad 0.9102578296637189 -Togo 0.2600585482954555 -Uganda 0.38301730108810556 -Uruguay 0.3083564407046887 -Vatican 0.00006702452496391445 -Akrotiri Sovereign Base Area 0.013376747415600219 -Zambia 0.8807923488623808 -Zimbabwe 0.4553903789902945 +Aruba 0.011249 +Afghanistan 0.819922 +Albania 0.171086 +Andorra 0.015146 +Ashmore and Cartier Islands 0.001111 +Austria 0.325846 +Burundi 0.14095 +Belgium 0.179446 +Benin 0.314261 +Burkina Faso 0.514438 +Bulgaria 0.308316 +Bahrain 0.021372 +Bosnia and Herzegovina 0.20612 +Bajo Nuevo Bank (Petrel Is.) 0.000125 +Saint Barthelemy 0.003299 +Belarus 0.428991 +Bolivia 0.927933 +Barbados 0.014116 +Bhutan 0.160174 +Botswana 0.58967 +Central African Republic 0.776022 +Switzerland 0.231885 +Clipperton Island 0.001407 +Cameroon 0.800105 +Republic of Congo 0.690432 +Coral Sea Islands 0.000116 +Curaçao 0.020789 +Czech Republic 0.270859 +Djibouti 0.129377 +Dominica 0.020094 +Algeria 1.154968 +Ethiopia 0.821065 +Georgia 0.26823 +Ghana 0.405658 +Gibraltar 0.001406 +Guinea 0.635085 +Gambia 0.192798 +Guatemala 0.303095 +Guam 0.020321 +Heard Island and McDonald Islands 0.017335 +Hungary 0.261773 +Isle of Man 0.018758 +Iraq 0.546986 +Israel 0.193539 +Jamaica 0.100559 +Jersey 0.008427 +Jordan 0.264224 +Baykonur Cosmodrome 0.04483 +Siachen Glacier 0.038721 +Kosovo 0.087732 +Laos 0.689987 +Lebanon 0.09677 +Liberia 0.296165 +Libya 0.953843 +Saint Lucia 0.016786 +Liechtenstein 0.009289 +Lesotho 0.123159 +Luxembourg 0.04126 +Latvia 0.244886 +Saint Martin 0.006548 +Morocco 0.881792 +Monaco 0.002605 +Moldova 0.207657 +Macedonia 0.112883 +Mali 1.138597 +Montenegro 0.117568 +Mongolia 1.142306 +Montserrat 0.00662 +Namibia 0.843465 +Niger 0.878074 +Norfolk Island 0.004912 +Niue 0.009882 +Nepal 0.407611 +Nauru 0.003121 +Poland 0.489221 +Paraguay 0.547526 +Qatar 0.093628 +Romania 0.44095 +Rwanda 0.129366 +Western Sahara 0.469192 +Scarborough Reef 0.000198 +South Sudan 0.758419 +Senegal 0.588325 +Serranilla Bank 0.000239 +Singapore 0.015233 +San Marino 0.004597 +Somaliland 0.309679 +Somalia 0.687992 +Republic of Serbia 0.296772 +Suriname 0.322552 +Slovakia 0.198436 +Slovenia 0.147131 +Swaziland 0.084342 +Sint Maarten 0.003796 +Syria 0.356755 +Chad 0.910258 +Togo 0.260059 +Uganda 0.383017 +Uruguay 0.308356 +Vatican 0.000067 +Akrotiri Sovereign Base Area 0.013377 +Zambia 0.880792 +Zimbabwe 0.45539 ------------------------------------- -Aruba 0.0000041986375296795025 -Afghanistan 0.015826481758320493 -Albania 0.0006971811189621746 -Andorra 0.00001112355564980348 -Ashmore and Cartier Islands 6.66668338977609e-8 -Austria 0.0020634744883290235 -Burundi 0.000669169243101558 -Belgium 0.0007529367590741593 -Benin 0.00287239734953164 -Burkina Faso 0.006746218025419332 -Bulgaria 0.0027733372191197786 -Bahrain 0.00001443842547561405 -Bosnia and Herzegovina 0.0012742491201009779 -Bajo Nuevo Bank (Petrel Is.) 8.864825701897049e-10 -Saint Barthelemy 6.036607210116289e-7 -Belarus 0.005090738074359067 -Bolivia 0.026865324735758436 -Barbados 0.0000109856680212211 -Bhutan 0.0009961026696220909 -Botswana 0.01430200501713062 -Central African Republic 0.015290667187215962 -Switzerland 0.0010181463734151514 -Clipperton Island 1.2373029819547803e-7 -Cameroon 0.011488908713113137 -Republic of Congo 0.008534881807187833 -Coral Sea Islands 5.121674593493771e-10 -Curaçao 0.000011457378136273848 -Czech Republic 0.0019339153549488386 -Djibouti 0.000540370985929321 -Dominica 0.000018056168258583246 -Algeria 0.05696762706232162 -Ethiopia 0.02789047634482515 -Georgia 0.0017113229913929072 -Ghana 0.0059048504621945965 -Gibraltar 9.095456688875715e-8 -Guinea 0.006043151808047173 -Gambia 0.0002596816395280707 -Guatemala 0.0026901925526205263 -Guam 0.000013952443476670549 -Heard Island and McDonald Islands 0.000009688375334192321 -Hungary 0.0022899094702118978 -Isle of Man 0.00001410012284549863 -Iraq 0.010780689598789812 -Israel 0.0005400181032289429 -Jamaica 0.00027268062650994383 -Jersey 0.0000029236161155167853 -Jordan 0.002191215069390572 -Baykonur Cosmodrome 0.00015978303781425133 -Siachen Glacier 0.0000513879615262916 -Kosovo 0.0002684178325412152 -Laos 0.005637555524983489 -Lebanon 0.0002464436461544738 -Liberia 0.002357973807538481 -Libya 0.040072512808839354 -Saint Lucia 0.000014963842166249258 -Liechtenstein 0.0000033722024322722466 -Lesotho 0.0007426290112070925 -Luxembourg 0.00006405006804909529 -Latvia 0.00158313668683266 -Saint Martin 0.00000168759530251474 -Morocco 0.014595589778269167 -Monaco 4.6325700981005285e-7 -Moldova 0.0008158639460823913 -Macedonia 0.0006245180554490506 -Mali 0.03096381132470007 -Montenegro 0.00033762445623993013 -Mongolia 0.038446609480001344 -Montserrat 0.0000024620326175206004 -Namibia 0.020320978539029165 -Niger 0.02919849042641136 -Norfolk Island 0.0000010150641235563077 -Niue 0.000005450796200539049 -Nepal 0.003629565673884544 -Nauru 7.119067469952887e-7 -Poland 0.0076921097527402876 -Paraguay 0.009875843128670564 -Qatar 0.0002752610716836153 -Romania 0.005809479702080411 -Rwanda 0.0006262235765421803 -Western Sahara 0.0022344529652030694 -Scarborough Reef 2.4176335726807567e-9 -South Sudan 0.015509656314462458 -Senegal 0.00485201810074574 -Serranilla Bank 2.6035559945372385e-9 -Singapore 0.000012633505579848072 -San Marino 0.0000014830814619737624 -Somaliland 0.0041412916217828406 -Somalia 0.011674654119996183 -Republic of Serbia 0.001907268740192651 -Suriname 0.0035911641359236534 -Slovakia 0.0011901587428922095 -Slovenia 0.0004995546076509384 -Swaziland 0.00042234053226485263 -Sint Maarten 5.772865969377286e-7 -Syria 0.004581243750467663 -Chad 0.0313064894302088 -Togo 0.0014067991034602252 -Uganda 0.005985159048654327 -Uruguay 0.0043716082436750115 -Vatican 3.002600504657064e-10 -Akrotiri Sovereign Base Area 0.0000024314362587592923 -Zambia 0.018594119224502336 -Zimbabwe 0.009621356779606268 +Aruba 0.000004 +Afghanistan 0.015826 +Albania 0.000697 +Andorra 0.000011 +Ashmore and Cartier Islands 0 +Austria 0.002063 +Burundi 0.000669 +Belgium 0.000753 +Benin 0.002872 +Burkina Faso 0.006746 +Bulgaria 0.002773 +Bahrain 0.000014 +Bosnia and Herzegovina 0.001274 +Bajo Nuevo Bank (Petrel Is.) 0 +Saint Barthelemy 0.000001 +Belarus 0.005091 +Bolivia 0.026865 +Barbados 0.000011 +Bhutan 0.000996 +Botswana 0.014302 +Central African Republic 0.015291 +Switzerland 0.001018 +Clipperton Island 0 +Cameroon 0.011489 +Republic of Congo 0.008535 +Coral Sea Islands 0 +Curaçao 0.000011 +Czech Republic 0.001934 +Djibouti 0.00054 +Dominica 0.000018 +Algeria 0.056968 +Ethiopia 0.02789 +Georgia 0.001711 +Ghana 0.005905 +Gibraltar 0 +Guinea 0.006043 +Gambia 0.00026 +Guatemala 0.00269 +Guam 0.000014 +Heard Island and McDonald Islands 0.00001 +Hungary 0.00229 +Isle of Man 0.000014 +Iraq 0.010781 +Israel 0.00054 +Jamaica 0.000273 +Jersey 0.000003 +Jordan 0.002191 +Baykonur Cosmodrome 0.00016 +Siachen Glacier 0.000051 +Kosovo 0.000268 +Laos 0.005638 +Lebanon 0.000246 +Liberia 0.002358 +Libya 0.040073 +Saint Lucia 0.000015 +Liechtenstein 0.000003 +Lesotho 0.000743 +Luxembourg 0.000064 +Latvia 0.001583 +Saint Martin 0.000002 +Morocco 0.014596 +Monaco 0 +Moldova 0.000816 +Macedonia 0.000625 +Mali 0.030964 +Montenegro 0.000338 +Mongolia 0.038447 +Montserrat 0.000002 +Namibia 0.020321 +Niger 0.029198 +Norfolk Island 0.000001 +Niue 0.000005 +Nepal 0.00363 +Nauru 0.000001 +Poland 0.007692 +Paraguay 0.009876 +Qatar 0.000275 +Romania 0.005809 +Rwanda 0.000626 +Western Sahara 0.002234 +Scarborough Reef 0 +South Sudan 0.01551 +Senegal 0.004852 +Serranilla Bank 0 +Singapore 0.000013 +San Marino 0.000001 +Somaliland 0.004141 +Somalia 0.011675 +Republic of Serbia 0.001907 +Suriname 0.003591 +Slovakia 0.00119 +Slovenia 0.0005 +Swaziland 0.000422 +Sint Maarten 0.000001 +Syria 0.004581 +Chad 0.031306 +Togo 0.001407 +Uganda 0.005985 +Uruguay 0.004372 +Vatican 0 +Akrotiri Sovereign Base Area 0.000002 +Zambia 0.018594 +Zimbabwe 0.009621 ------------------------------------- diff --git a/tests/queries/0_stateless/01720_country_perimeter_and_area.sh b/tests/queries/0_stateless/01720_country_perimeter_and_area.sh index 75016ee1d1f..0080c9a1a1b 100755 --- a/tests/queries/0_stateless/01720_country_perimeter_and_area.sh +++ b/tests/queries/0_stateless/01720_country_perimeter_and_area.sh @@ -8,9 +8,9 @@ ${CLICKHOUSE_CLIENT} -q "drop table if exists country_polygons;" ${CLICKHOUSE_CLIENT} -q "create table country_polygons(name String, p Array(Array(Tuple(Float64, Float64)))) engine=MergeTree() order by tuple();" cat ${CURDIR}/country_polygons.tsv | ${CLICKHOUSE_CLIENT} -q "insert into country_polygons format TSV" -${CLICKHOUSE_CLIENT} -q "SELECT name, polygonPerimeterSpherical(p) from country_polygons" +${CLICKHOUSE_CLIENT} -q "SELECT name, round(polygonPerimeterSpherical(p), 6) from country_polygons" ${CLICKHOUSE_CLIENT} -q "SELECT '-------------------------------------'" -${CLICKHOUSE_CLIENT} -q "SELECT name, polygonAreaSpherical(p) from country_polygons" +${CLICKHOUSE_CLIENT} -q "SELECT name, round(polygonAreaSpherical(p), 6) from country_polygons" ${CLICKHOUSE_CLIENT} -q "SELECT '-------------------------------------'" ${CLICKHOUSE_CLIENT} -q "drop table if exists country_rings;" @@ -18,9 +18,9 @@ ${CLICKHOUSE_CLIENT} -q "drop table if exists country_rings;" ${CLICKHOUSE_CLIENT} -q "create table country_rings(name String, p Array(Tuple(Float64, Float64))) engine=MergeTree() order by tuple();" cat ${CURDIR}/country_rings.tsv | ${CLICKHOUSE_CLIENT} -q "insert into country_rings format TSV" -${CLICKHOUSE_CLIENT} -q "SELECT name, polygonPerimeterSpherical(p) from country_rings" +${CLICKHOUSE_CLIENT} -q "SELECT name, round(polygonPerimeterSpherical(p), 6) from country_rings" ${CLICKHOUSE_CLIENT} -q "SELECT '-------------------------------------'" -${CLICKHOUSE_CLIENT} -q "SELECT name, polygonAreaSpherical(p) from country_rings" +${CLICKHOUSE_CLIENT} -q "SELECT name, round(polygonAreaSpherical(p), 6) from country_rings" ${CLICKHOUSE_CLIENT} -q "SELECT '-------------------------------------'" ${CLICKHOUSE_CLIENT} -q "drop table if exists country_rings;" From d6ece47b7f89f3495bc75d02c291e3745592d946 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Dec 2021 08:53:35 +0300 Subject: [PATCH 0300/1260] Allow vfork --- src/Common/ShellCommand.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index f238e976f8a..99461862ef9 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -155,12 +155,17 @@ std::unique_ptr ShellCommand::executeImpl( { logCommand(filename, argv); +#if !defined(USE_MUSL) /** Here it is written that with a normal call `vfork`, there is a chance of deadlock in multithreaded programs, * because of the resolving of symbols in the shared library * http://www.oracle.com/technetwork/server-storage/solaris10/subprocess-136439.html * Therefore, separate the resolving of the symbol from the call. */ static void * real_vfork = dlsym(RTLD_DEFAULT, "vfork"); +#else + /// If we use Musl with static linking, there is no dlsym and no issue with vfork. + static void * real_vfork = reinterpret_cast(&vfork); +#endif if (!real_vfork) throwFromErrno("Cannot find symbol vfork in myself", ErrorCodes::CANNOT_DLSYM); From f6dc07b4750c60b708370a5e9c869da4247e2783 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Dec 2021 08:57:48 +0300 Subject: [PATCH 0301/1260] Update test --- tests/queries/0_stateless/00700_decimal_math.reference | 6 +++--- tests/queries/0_stateless/00700_decimal_math.sql | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/00700_decimal_math.reference b/tests/queries/0_stateless/00700_decimal_math.reference index eb556ac49b8..389b428e27b 100644 --- a/tests/queries/0_stateless/00700_decimal_math.reference +++ b/tests/queries/0_stateless/00700_decimal_math.reference @@ -4,7 +4,7 @@ 42.42 6.513 42.419169 42.42 3.4875 42.417263671875 1 0.8427007929497149 0.15729920705028513 -42.42 115.60113124678627 1.6029995567009473e50 +42.42 115.601131 1.603 0 0 1 0 3.14159265 0 -1 -0 1 1.5707963267948966 0 0.7853981633974483 @@ -14,7 +14,7 @@ 42.42 6.513 42.419169 42.42 3.4875 42.417263671875 1 0.8427007929497149 0.15729920705028513 -42.42 115.60113124678627 1.6029995567009473e50 +42.42 115.601131 1.603 0 0 1 0 3.14159265358979328 0 -1 -0 1 1.5707963267948966 0 0.7853981633974483 @@ -24,7 +24,7 @@ 42.42 6.513 42.419169 42.42 3.4875 42.417263671875 1 0.8427007929497149 0.15729920705028513 -42.42 115.60113124678627 1.6029995567009473e50 +42.42 115.601131 1.603 0 0 1 0 3.14159265358979 0 -1 -0 1 1.5707963267948966 0 0.7853981633974483 diff --git a/tests/queries/0_stateless/00700_decimal_math.sql b/tests/queries/0_stateless/00700_decimal_math.sql index 237bee1c691..cefbf2fd604 100644 --- a/tests/queries/0_stateless/00700_decimal_math.sql +++ b/tests/queries/0_stateless/00700_decimal_math.sql @@ -5,7 +5,7 @@ SELECT toDecimal32('42.42', 4) AS x, toDecimal32(log10(x), 4) AS y, round(exp10( SELECT toDecimal32('42.42', 4) AS x, toDecimal32(sqrt(x), 3) AS y, y * y; SELECT toDecimal32('42.42', 4) AS x, toDecimal32(cbrt(x), 4) AS y, toDecimal64(y, 4) * y * y; SELECT toDecimal32('1.0', 5) AS x, erf(x), erfc(x); -SELECT toDecimal32('42.42', 4) AS x, lgamma(x), tgamma(x); +SELECT toDecimal32('42.42', 4) AS x, round(lgamma(x), 6), round(tgamma(x) / 1e50, 6); SELECT toDecimal32('0.0', 2) AS x, round(sin(x), 8), round(cos(x), 8), round(tan(x), 8); SELECT toDecimal32(pi(), 8) AS x, round(sin(x), 8), round(cos(x), 8), round(tan(x), 8); @@ -19,7 +19,7 @@ SELECT toDecimal64('42.42', 4) AS x, toDecimal32(log10(x), 4) AS y, round(exp10( SELECT toDecimal64('42.42', 4) AS x, toDecimal32(sqrt(x), 3) AS y, y * y; SELECT toDecimal64('42.42', 4) AS x, toDecimal32(cbrt(x), 4) AS y, toDecimal64(y, 4) * y * y; SELECT toDecimal64('1.0', 5) AS x, erf(x), erfc(x); -SELECT toDecimal64('42.42', 4) AS x, lgamma(x), tgamma(x); +SELECT toDecimal64('42.42', 4) AS x, round(lgamma(x), 6), round(tgamma(x) / 1e50, 6); SELECT toDecimal64('0.0', 2) AS x, round(sin(x), 8), round(cos(x), 8), round(tan(x), 8); SELECT toDecimal64(pi(), 17) AS x, round(sin(x), 8), round(cos(x), 8), round(tan(x), 8); @@ -33,7 +33,7 @@ SELECT toDecimal128('42.42', 4) AS x, toDecimal32(log10(x), 4) AS y, round(exp10 SELECT toDecimal128('42.42', 4) AS x, toDecimal32(sqrt(x), 3) AS y, y * y; SELECT toDecimal128('42.42', 4) AS x, toDecimal32(cbrt(x), 4) AS y, toDecimal64(y, 4) * y * y; SELECT toDecimal128('1.0', 5) AS x, erf(x), erfc(x); -SELECT toDecimal128('42.42', 4) AS x, lgamma(x), tgamma(x); +SELECT toDecimal128('42.42', 4) AS x, round(lgamma(x), 6), round(tgamma(x) / 1e50, 6); SELECT toDecimal128('0.0', 2) AS x, round(sin(x), 8), round(cos(x), 8), round(tan(x), 8); SELECT toDecimal128(pi(), 14) AS x, round(sin(x), 8), round(cos(x), 8), round(tan(x), 8); From f9d3e45e49ac32a52e48a8b01f27972ae8e0ac1e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Dec 2021 09:32:48 +0300 Subject: [PATCH 0302/1260] Remove strange whitespace --- .../1_stateful/00168_parallel_processing_on_replicas_part_1.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh b/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh index ba1245d9679..d025dae5b2e 100755 --- a/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh +++ b/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh @@ -72,7 +72,7 @@ do if [[ "$expected" != "$actual" ]]; then FAILED+=("$TESTNAME") - echo "Failed! ❌ " + echo "Failed! ❌" echo "Plain:" cat $TESTNAME_RESULT echo "Distributed:" From 29d28c531f5670bd957713ef4713c3ba149bee30 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 21 Dec 2021 16:41:53 +0300 Subject: [PATCH 0303/1260] Move code around to avoid dlsym on Musl --- base/CMakeLists.txt | 4 -- base/base/CMakeLists.txt | 3 - base/mysqlxx/CMakeLists.txt | 61 ---------------- programs/install/Install.cpp | 2 +- programs/obfuscator/Obfuscator.cpp | 2 +- src/Client/ClientBase.cpp | 4 +- src/Client/ClientBaseHelpers.cpp | 4 +- src/Common/CMakeLists.txt | 7 +- src/Common/Config/ConfigProcessor.cpp | 2 +- {base/base => src/Common}/DateLUT.cpp | 0 {base/base => src/Common}/DateLUT.h | 2 +- {base/base => src/Common}/DateLUTImpl.cpp | 2 +- {base/base => src/Common}/DateLUTImpl.h | 6 +- {base/base => src/Common}/LocalDate.h | 2 +- {base/base => src/Common}/LocalDateTime.h | 4 +- src/Common/StatusFile.cpp | 2 +- src/Common/SymbolIndex.cpp | 69 +++++++++++++++---- src/Common/SymbolIndex.h | 13 ++++ {base/base => src/Common}/getResource.cpp | 13 +++- {base/base => src/Common}/getResource.h | 0 src/Common/mysqlxx/CMakeLists.txt | 24 +++++++ {base => src/Common}/mysqlxx/Connection.cpp | 0 {base => src/Common}/mysqlxx/Exception.cpp | 0 {base => src/Common}/mysqlxx/Pool.cpp | 0 {base => src/Common}/mysqlxx/PoolFactory.cpp | 0 .../Common}/mysqlxx/PoolWithFailover.cpp | 0 {base => src/Common}/mysqlxx/Query.cpp | 0 {base => src/Common}/mysqlxx/ResultBase.cpp | 0 {base => src/Common}/mysqlxx/Row.cpp | 0 .../Common}/mysqlxx/UseQueryResult.cpp | 0 {base => src/Common}/mysqlxx/Value.cpp | 0 .../Common/mysqlxx}/mysqlxx/Connection.h | 0 .../Common/mysqlxx}/mysqlxx/Exception.h | 0 {base => src/Common/mysqlxx}/mysqlxx/Null.h | 0 {base => src/Common/mysqlxx}/mysqlxx/Pool.h | 0 .../Common/mysqlxx}/mysqlxx/PoolFactory.h | 0 .../mysqlxx}/mysqlxx/PoolWithFailover.h | 0 {base => src/Common/mysqlxx}/mysqlxx/Query.h | 0 .../Common/mysqlxx}/mysqlxx/ResultBase.h | 0 {base => src/Common/mysqlxx}/mysqlxx/Row.h | 0 .../Common/mysqlxx}/mysqlxx/Transaction.h | 0 {base => src/Common/mysqlxx}/mysqlxx/Types.h | 0 .../Common/mysqlxx}/mysqlxx/UseQueryResult.h | 0 {base => src/Common/mysqlxx}/mysqlxx/Value.h | 4 +- .../Common/mysqlxx}/mysqlxx/mysqlxx.h | 4 +- .../Common}/mysqlxx/tests/CMakeLists.txt | 0 .../mysqlxx/tests/mysqlxx_pool_test.cpp | 0 src/Common/tests/gtest_DateLUTImpl.cpp | 4 +- .../gtest_local_date_time_comparison.cpp | 2 +- src/Core/MySQL/MySQLReplication.cpp | 2 +- src/DataTypes/DataTypeDate32.h | 2 +- .../Serializations/SerializationDateTime.cpp | 2 +- .../SerializationDateTime64.cpp | 2 +- src/DataTypes/TimezoneMixin.h | 2 +- .../ExecutableDictionarySource.cpp | 2 +- .../ExecutablePoolDictionarySource.cpp | 2 +- src/Dictionaries/HTTPDictionarySource.h | 2 +- src/Dictionaries/LibraryDictionarySource.h | 2 +- src/Dictionaries/MySQLDictionarySource.cpp | 2 +- src/Dictionaries/MySQLDictionarySource.h | 2 +- src/Dictionaries/PostgreSQLDictionarySource.h | 2 +- src/Dictionaries/XDBCDictionarySource.cpp | 2 +- src/Functions/CustomWeekTransforms.h | 2 +- src/Functions/DateTimeTransforms.h | 4 +- .../FunctionDateOrDateTimeAddInterval.h | 2 +- src/Functions/FunctionsTimeWindow.h | 2 +- src/Functions/dateName.cpp | 2 +- .../extractTimeZoneFromFunctionArguments.cpp | 2 +- src/Functions/formatDateTime.cpp | 2 +- src/Functions/serverConstants.cpp | 2 +- src/Functions/timezoneOf.cpp | 2 +- src/Functions/toStartOfInterval.cpp | 2 +- src/Functions/today.cpp | 2 +- src/Functions/yesterday.cpp | 2 +- src/IO/ReadHelpers.h | 6 +- src/IO/WriteHelpers.h | 6 +- src/IO/parseDateTimeBestEffort.cpp | 2 +- src/IO/tests/gtest_DateTimeToString.cpp | 2 +- src/Interpreters/QueryViewsLog.cpp | 2 +- src/Interpreters/convertFieldToType.cpp | 2 +- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 2 +- .../GraphiteRollupSortedAlgorithm.cpp | 4 +- src/Processors/TTL/ITTLAlgorithm.h | 2 +- src/Processors/Transforms/TTLCalcTransform.h | 2 +- src/Processors/Transforms/TTLTransform.h | 2 +- src/Server/WebUIRequestHandler.cpp | 2 +- .../System/attachInformationSchemaTables.cpp | 2 +- 87 files changed, 175 insertions(+), 153 deletions(-) delete mode 100644 base/mysqlxx/CMakeLists.txt rename {base/base => src/Common}/DateLUT.cpp (100%) rename {base/base => src/Common}/DateLUT.h (98%) rename {base/base => src/Common}/DateLUTImpl.cpp (99%) rename {base/base => src/Common}/DateLUTImpl.h (99%) rename {base/base => src/Common}/LocalDate.h (99%) rename {base/base => src/Common}/LocalDateTime.h (98%) rename {base/base => src/Common}/getResource.cpp (80%) rename {base/base => src/Common}/getResource.h (100%) create mode 100644 src/Common/mysqlxx/CMakeLists.txt rename {base => src/Common}/mysqlxx/Connection.cpp (100%) rename {base => src/Common}/mysqlxx/Exception.cpp (100%) rename {base => src/Common}/mysqlxx/Pool.cpp (100%) rename {base => src/Common}/mysqlxx/PoolFactory.cpp (100%) rename {base => src/Common}/mysqlxx/PoolWithFailover.cpp (100%) rename {base => src/Common}/mysqlxx/Query.cpp (100%) rename {base => src/Common}/mysqlxx/ResultBase.cpp (100%) rename {base => src/Common}/mysqlxx/Row.cpp (100%) rename {base => src/Common}/mysqlxx/UseQueryResult.cpp (100%) rename {base => src/Common}/mysqlxx/Value.cpp (100%) rename {base => src/Common/mysqlxx}/mysqlxx/Connection.h (100%) rename {base => src/Common/mysqlxx}/mysqlxx/Exception.h (100%) rename {base => src/Common/mysqlxx}/mysqlxx/Null.h (100%) rename {base => src/Common/mysqlxx}/mysqlxx/Pool.h (100%) rename {base => src/Common/mysqlxx}/mysqlxx/PoolFactory.h (100%) rename {base => src/Common/mysqlxx}/mysqlxx/PoolWithFailover.h (100%) rename {base => src/Common/mysqlxx}/mysqlxx/Query.h (100%) rename {base => src/Common/mysqlxx}/mysqlxx/ResultBase.h (100%) rename {base => src/Common/mysqlxx}/mysqlxx/Row.h (100%) rename {base => src/Common/mysqlxx}/mysqlxx/Transaction.h (100%) rename {base => src/Common/mysqlxx}/mysqlxx/Types.h (100%) rename {base => src/Common/mysqlxx}/mysqlxx/UseQueryResult.h (100%) rename {base => src/Common/mysqlxx}/mysqlxx/Value.h (99%) rename {base => src/Common/mysqlxx}/mysqlxx/mysqlxx.h (97%) rename {base => src/Common}/mysqlxx/tests/CMakeLists.txt (100%) rename {base => src/Common}/mysqlxx/tests/mysqlxx_pool_test.cpp (100%) diff --git a/base/CMakeLists.txt b/base/CMakeLists.txt index 452b483fb6a..be1a0fb2af1 100644 --- a/base/CMakeLists.txt +++ b/base/CMakeLists.txt @@ -9,7 +9,3 @@ add_subdirectory (pcg-random) add_subdirectory (widechar_width) add_subdirectory (readpassphrase) add_subdirectory (bridge) - -if (USE_MYSQL) - add_subdirectory (mysqlxx) -endif () diff --git a/base/base/CMakeLists.txt b/base/base/CMakeLists.txt index c0b0801bd2e..e62299f3d06 100644 --- a/base/base/CMakeLists.txt +++ b/base/base/CMakeLists.txt @@ -1,8 +1,6 @@ set (SRCS argsToConfig.cpp coverage.cpp - DateLUT.cpp - DateLUTImpl.cpp demangle.cpp getFQDNOrHostName.cpp getMemoryAmount.cpp @@ -18,7 +16,6 @@ set (SRCS sleep.cpp terminalColors.cpp errnoToString.cpp - getResource.cpp StringRef.cpp ) diff --git a/base/mysqlxx/CMakeLists.txt b/base/mysqlxx/CMakeLists.txt deleted file mode 100644 index 80db50c2593..00000000000 --- a/base/mysqlxx/CMakeLists.txt +++ /dev/null @@ -1,61 +0,0 @@ -add_library (mysqlxx - Connection.cpp - Exception.cpp - Query.cpp - ResultBase.cpp - UseQueryResult.cpp - Row.cpp - Value.cpp - Pool.cpp - PoolFactory.cpp - PoolWithFailover.cpp -) - -target_include_directories (mysqlxx PUBLIC ..) - -if (NOT USE_INTERNAL_MYSQL_LIBRARY) - set(PLATFORM_LIBRARIES ${CMAKE_DL_LIBS}) - - if (USE_MYSQL) - target_include_directories (mysqlxx SYSTEM PRIVATE ${MYSQL_INCLUDE_DIR}) - endif () - - if (APPLE) - find_library (ICONV_LIBRARY iconv) - set (MYSQLCLIENT_LIBRARIES ${MYSQLCLIENT_LIBRARIES} ${STATIC_MYSQLCLIENT_LIB} ${ICONV_LIBRARY}) - elseif (USE_STATIC_LIBRARIES AND STATIC_MYSQLCLIENT_LIB) - set (MYSQLCLIENT_LIBRARIES ${STATIC_MYSQLCLIENT_LIB}) - endif () -endif () - -target_link_libraries (mysqlxx - PUBLIC - common - PRIVATE - ${MYSQLCLIENT_LIBRARIES} - ${ZLIB_LIBRARIES} -) - -if(OPENSSL_LIBRARIES) - target_link_libraries(mysqlxx PRIVATE ${OPENSSL_LIBRARIES}) -endif() - -target_link_libraries(mysqlxx PRIVATE ${PLATFORM_LIBRARIES}) - -if (NOT USE_INTERNAL_MYSQL_LIBRARY AND OPENSSL_INCLUDE_DIR) - target_include_directories (mysqlxx SYSTEM PRIVATE ${OPENSSL_INCLUDE_DIR}) -endif () - -target_no_warning(mysqlxx reserved-macro-identifier) - -if (NOT USE_INTERNAL_MYSQL_LIBRARY AND USE_STATIC_LIBRARIES) - message(WARNING "Statically linking with system mysql/mariadb only works " - "if mysql client libraries are built with same openssl version as " - "we are going to use now. It wouldn't work if GnuTLS is used. " - "Try -D\"USE_INTERNAL_MYSQL_LIBRARY\"=ON or -D\"ENABLE_MYSQL\"=OFF or " - "-D\"USE_STATIC_LIBRARIES\"=OFF") -endif () - -if (ENABLE_TESTS) - add_subdirectory (tests) -endif () diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index a017355cfdd..706e273e2b4 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -20,7 +20,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index caccc726923..947e7ab1768 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -30,7 +30,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 8fd95bb706c..75e0588f786 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -8,8 +8,8 @@ #include #include -#include -#include +#include +#include #include #include #include "Common/Exception.h" diff --git a/src/Client/ClientBaseHelpers.cpp b/src/Client/ClientBaseHelpers.cpp index e1c1481c5b4..3a5d4f4cf33 100644 --- a/src/Client/ClientBaseHelpers.cpp +++ b/src/Client/ClientBaseHelpers.cpp @@ -1,8 +1,8 @@ #include "ClientBaseHelpers.h" -#include -#include +#include +#include #include #include diff --git a/src/Common/CMakeLists.txt b/src/Common/CMakeLists.txt index 1935fe4fed1..1e7d3591a48 100644 --- a/src/Common/CMakeLists.txt +++ b/src/Common/CMakeLists.txt @@ -1,8 +1,9 @@ add_subdirectory(StringUtils) -# after common_io -#add_subdirectory(ZooKeeper) -#add_subdirectory(ConfigProcessor) if (ENABLE_EXAMPLES) add_subdirectory(examples) endif() + +if (USE_MYSQL) + add_subdirectory (mysqlxx) +endif () diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 0bb25df5d4e..41e9a53e50f 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -18,7 +18,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/base/base/DateLUT.cpp b/src/Common/DateLUT.cpp similarity index 100% rename from base/base/DateLUT.cpp rename to src/Common/DateLUT.cpp diff --git a/base/base/DateLUT.h b/src/Common/DateLUT.h similarity index 98% rename from base/base/DateLUT.h rename to src/Common/DateLUT.h index 31fc6b1e24b..edf09250e6a 100644 --- a/base/base/DateLUT.h +++ b/src/Common/DateLUT.h @@ -2,7 +2,7 @@ #include "DateLUTImpl.h" -#include "defines.h" +#include #include diff --git a/base/base/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp similarity index 99% rename from base/base/DateLUTImpl.cpp rename to src/Common/DateLUTImpl.cpp index bbce3b111d3..ebf32c4dbd9 100644 --- a/base/base/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include diff --git a/base/base/DateLUTImpl.h b/src/Common/DateLUTImpl.h similarity index 99% rename from base/base/DateLUTImpl.h rename to src/Common/DateLUTImpl.h index 012d2cefe84..e52e6547fa2 100644 --- a/base/base/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -1,8 +1,8 @@ #pragma once -#include "DayNum.h" -#include "defines.h" -#include "types.h" +#include +#include +#include #include #include diff --git a/base/base/LocalDate.h b/src/Common/LocalDate.h similarity index 99% rename from base/base/LocalDate.h rename to src/Common/LocalDate.h index d199b9dbb4b..5ca0696c371 100644 --- a/base/base/LocalDate.h +++ b/src/Common/LocalDate.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include /** Stores a calendar date in broken-down form (year, month, day-in-month). diff --git a/base/base/LocalDateTime.h b/src/Common/LocalDateTime.h similarity index 98% rename from base/base/LocalDateTime.h rename to src/Common/LocalDateTime.h index 282a56ac640..fa0a680d274 100644 --- a/base/base/LocalDateTime.h +++ b/src/Common/LocalDateTime.h @@ -3,8 +3,8 @@ #include #include #include -#include -#include +#include +#include /** Stores calendar date and time in broken-down form. diff --git a/src/Common/StatusFile.cpp b/src/Common/StatusFile.cpp index 5e4c31b149e..cfeab149d30 100644 --- a/src/Common/StatusFile.cpp +++ b/src/Common/StatusFile.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index 2d875b7042d..46b3ffdd45d 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -5,6 +5,7 @@ #include #include +#include #include @@ -85,12 +86,45 @@ namespace /// https://stackoverflow.com/questions/32088140/multiple-string-tables-in-elf-object +void updateResources(std::string_view name, const void * address, SymbolIndex::Resources & resources) +{ + const char * char_address = static_cast(address); + + if (name.starts_with("_binary_") || name.starts_with("binary_")) + { + std::cerr << name << "\n"; + + if (name.ends_with("_start")) + { + name = name.substr((name[0] == '_') + strlen("binary_")); + name = name.substr(0, name.size() - strlen("_start")); + + resources[name] = std::string_view{char_address, 0}; + } + else if (name.ends_with("_end")) + { + name = name.substr((name[0] == '_') + strlen("binary_")); + name = name.substr(0, name.size() - strlen("_end")); + + if (auto it = resources.find(name); it != resources.end()) + { + auto start = it->second.data(); + assert(char_address >= start); + it->second = std::string_view{start, static_cast(char_address - start)}; + } + } + } +} + + /// Based on the code of musl-libc and the answer of Kanalpiroge on /// https://stackoverflow.com/questions/15779185/list-all-the-functions-symbols-on-the-fly-in-c-code-on-a-linux-architecture /// It does not extract all the symbols (but only public - exported and used for dynamic linking), /// but will work if we cannot find or parse ELF files. -void collectSymbolsFromProgramHeaders(dl_phdr_info * info, - std::vector & symbols) +void collectSymbolsFromProgramHeaders( + dl_phdr_info * info, + std::vector & symbols, + SymbolIndex::Resources & resources) { /* Iterate over all headers of the current shared lib * (first call is for the executable itself) @@ -201,6 +235,8 @@ void collectSymbolsFromProgramHeaders(dl_phdr_info * info, symbol.address_end = reinterpret_cast(info->dlpi_addr + elf_sym[sym_index].st_value + elf_sym[sym_index].st_size); symbol.name = sym_name; symbols.push_back(symbol); + + updateResources(symbol.name, symbol.address_begin, resources); } break; @@ -229,7 +265,8 @@ void collectSymbolsFromELFSymbolTable( const Elf & elf, const Elf::Section & symbol_table, const Elf::Section & string_table, - std::vector & symbols) + std::vector & symbols, + SymbolIndex::Resources & resources) { /// Iterate symbol table. const ElfSym * symbol_table_entry = reinterpret_cast(symbol_table.begin()); @@ -256,6 +293,8 @@ void collectSymbolsFromELFSymbolTable( symbol.address_end = reinterpret_cast(info->dlpi_addr + symbol_table_entry->st_value + symbol_table_entry->st_size); symbol.name = symbol_name; symbols.push_back(symbol); + + updateResources(symbol.name, symbol.address_begin, resources); } } @@ -265,7 +304,8 @@ bool searchAndCollectSymbolsFromELFSymbolTable( const Elf & elf, unsigned section_header_type, const char * string_table_name, - std::vector & symbols) + std::vector & symbols, + SymbolIndex::Resources & resources) { std::optional symbol_table; std::optional string_table; @@ -280,17 +320,20 @@ bool searchAndCollectSymbolsFromELFSymbolTable( return (symbol_table && string_table); })) { + std::cerr << "!\n"; return false; } - collectSymbolsFromELFSymbolTable(info, elf, *symbol_table, *string_table, symbols); + collectSymbolsFromELFSymbolTable(info, elf, *symbol_table, *string_table, symbols, resources); return true; } -void collectSymbolsFromELF(dl_phdr_info * info, +void collectSymbolsFromELF( + dl_phdr_info * info, std::vector & symbols, std::vector & objects, + SymbolIndex::Resources & resources, String & build_id) { /// MSan does not know that the program segments in memory are initialized. @@ -377,10 +420,12 @@ void collectSymbolsFromELF(dl_phdr_info * info, object.name = object_name; objects.push_back(std::move(object)); - searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_SYMTAB, ".strtab", symbols); + searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_SYMTAB, ".strtab", symbols, resources); - /// Unneeded because they were parsed from "program headers" of loaded objects. - //searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_DYNSYM, ".dynstr", symbols); + /// Unneeded if they were parsed from "program headers" of loaded objects. +#if defined USE_MUSL + searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_DYNSYM, ".dynstr", symbols, resources); +#endif } @@ -392,8 +437,8 @@ int collectSymbols(dl_phdr_info * info, size_t, void * data_ptr) { SymbolIndex::Data & data = *reinterpret_cast(data_ptr); - collectSymbolsFromProgramHeaders(info, data.symbols); - collectSymbolsFromELF(info, data.symbols, data.objects, data.build_id); + collectSymbolsFromProgramHeaders(info, data.symbols, data.resources); + collectSymbolsFromELF(info, data.symbols, data.objects, data.resources, data.build_id); /* Continue iterations */ return 0; @@ -424,7 +469,7 @@ const T * find(const void * address, const std::vector & vec) void SymbolIndex::update() { - dl_iterate_phdr(collectSymbols, &data.symbols); + dl_iterate_phdr(collectSymbols, &data); std::sort(data.objects.begin(), data.objects.end(), [](const Object & a, const Object & b) { return a.address_begin < b.address_begin; }); std::sort(data.symbols.begin(), data.symbols.end(), [](const Symbol & a, const Symbol & b) { return a.address_begin < b.address_begin; }); diff --git a/src/Common/SymbolIndex.h b/src/Common/SymbolIndex.h index 37862987bd2..7c542980099 100644 --- a/src/Common/SymbolIndex.h +++ b/src/Common/SymbolIndex.h @@ -4,6 +4,7 @@ #include #include +#include #include #include @@ -47,15 +48,27 @@ public: const std::vector & symbols() const { return data.symbols; } const std::vector & objects() const { return data.objects; } + std::string_view getResource(String name) const + { + if (auto it = data.resources.find(name); it != data.resources.end()) + return it->second; + return {}; + } + /// The BuildID that is generated by compiler. String getBuildID() const { return data.build_id; } String getBuildIDHex() const; + using Resources = std::unordered_map; + struct Data { std::vector symbols; std::vector objects; String build_id; + + /// Resources (embedded binary data) are located by symbols in form of _binary_name_start and _binary_name_end. + Resources resources; }; private: Data data; diff --git a/base/base/getResource.cpp b/src/Common/getResource.cpp similarity index 80% rename from base/base/getResource.cpp rename to src/Common/getResource.cpp index 6682ae0a01f..fe603fcc550 100644 --- a/base/base/getResource.cpp +++ b/src/Common/getResource.cpp @@ -1,8 +1,9 @@ #include "getResource.h" -#include "unaligned.h" #include #include #include +#include + std::string_view getResource(std::string_view name) { @@ -13,6 +14,11 @@ std::string_view getResource(std::string_view name) std::replace(name_replaced.begin(), name_replaced.end(), '.', '_'); boost::replace_all(name_replaced, "+", "_PLUS_"); +#if defined USE_MUSL + /// If static linking is used, we cannot use dlsym and have to parse ELF symbol table by ourself. + return DB::SymbolIndex::instance()->getResource(name_replaced); + +#else // In most `dlsym(3)` APIs, one passes the symbol name as it appears via // something like `nm` or `objdump -t`. For example, a symbol `_foo` would be // looked up with the string `"_foo"`. @@ -33,8 +39,8 @@ std::string_view getResource(std::string_view name) std::string symbol_name_start = prefix + name_replaced + "_start"; std::string symbol_name_end = prefix + name_replaced + "_end"; - const char* sym_start = reinterpret_cast(dlsym(RTLD_DEFAULT, symbol_name_start.c_str())); - const char* sym_end = reinterpret_cast(dlsym(RTLD_DEFAULT, symbol_name_end.c_str())); + const char * sym_start = reinterpret_cast(dlsym(RTLD_DEFAULT, symbol_name_start.c_str())); + const char * sym_end = reinterpret_cast(dlsym(RTLD_DEFAULT, symbol_name_end.c_str())); if (sym_start && sym_end) { @@ -42,4 +48,5 @@ std::string_view getResource(std::string_view name) return { sym_start, resource_size }; } return {}; +#endif } diff --git a/base/base/getResource.h b/src/Common/getResource.h similarity index 100% rename from base/base/getResource.h rename to src/Common/getResource.h diff --git a/src/Common/mysqlxx/CMakeLists.txt b/src/Common/mysqlxx/CMakeLists.txt new file mode 100644 index 00000000000..76005651e61 --- /dev/null +++ b/src/Common/mysqlxx/CMakeLists.txt @@ -0,0 +1,24 @@ +add_library (mysqlxx + Connection.cpp + Exception.cpp + Query.cpp + ResultBase.cpp + UseQueryResult.cpp + Row.cpp + Value.cpp + Pool.cpp + PoolFactory.cpp + PoolWithFailover.cpp +) + +target_include_directories (mysqlxx PUBLIC .) + +target_link_libraries (mysqlxx + clickhouse_common_io + ${MYSQLCLIENT_LIBRARIES} + ${ZLIB_LIBRARIES} +) + +if (ENABLE_TESTS) + add_subdirectory (tests) +endif () diff --git a/base/mysqlxx/Connection.cpp b/src/Common/mysqlxx/Connection.cpp similarity index 100% rename from base/mysqlxx/Connection.cpp rename to src/Common/mysqlxx/Connection.cpp diff --git a/base/mysqlxx/Exception.cpp b/src/Common/mysqlxx/Exception.cpp similarity index 100% rename from base/mysqlxx/Exception.cpp rename to src/Common/mysqlxx/Exception.cpp diff --git a/base/mysqlxx/Pool.cpp b/src/Common/mysqlxx/Pool.cpp similarity index 100% rename from base/mysqlxx/Pool.cpp rename to src/Common/mysqlxx/Pool.cpp diff --git a/base/mysqlxx/PoolFactory.cpp b/src/Common/mysqlxx/PoolFactory.cpp similarity index 100% rename from base/mysqlxx/PoolFactory.cpp rename to src/Common/mysqlxx/PoolFactory.cpp diff --git a/base/mysqlxx/PoolWithFailover.cpp b/src/Common/mysqlxx/PoolWithFailover.cpp similarity index 100% rename from base/mysqlxx/PoolWithFailover.cpp rename to src/Common/mysqlxx/PoolWithFailover.cpp diff --git a/base/mysqlxx/Query.cpp b/src/Common/mysqlxx/Query.cpp similarity index 100% rename from base/mysqlxx/Query.cpp rename to src/Common/mysqlxx/Query.cpp diff --git a/base/mysqlxx/ResultBase.cpp b/src/Common/mysqlxx/ResultBase.cpp similarity index 100% rename from base/mysqlxx/ResultBase.cpp rename to src/Common/mysqlxx/ResultBase.cpp diff --git a/base/mysqlxx/Row.cpp b/src/Common/mysqlxx/Row.cpp similarity index 100% rename from base/mysqlxx/Row.cpp rename to src/Common/mysqlxx/Row.cpp diff --git a/base/mysqlxx/UseQueryResult.cpp b/src/Common/mysqlxx/UseQueryResult.cpp similarity index 100% rename from base/mysqlxx/UseQueryResult.cpp rename to src/Common/mysqlxx/UseQueryResult.cpp diff --git a/base/mysqlxx/Value.cpp b/src/Common/mysqlxx/Value.cpp similarity index 100% rename from base/mysqlxx/Value.cpp rename to src/Common/mysqlxx/Value.cpp diff --git a/base/mysqlxx/Connection.h b/src/Common/mysqlxx/mysqlxx/Connection.h similarity index 100% rename from base/mysqlxx/Connection.h rename to src/Common/mysqlxx/mysqlxx/Connection.h diff --git a/base/mysqlxx/Exception.h b/src/Common/mysqlxx/mysqlxx/Exception.h similarity index 100% rename from base/mysqlxx/Exception.h rename to src/Common/mysqlxx/mysqlxx/Exception.h diff --git a/base/mysqlxx/Null.h b/src/Common/mysqlxx/mysqlxx/Null.h similarity index 100% rename from base/mysqlxx/Null.h rename to src/Common/mysqlxx/mysqlxx/Null.h diff --git a/base/mysqlxx/Pool.h b/src/Common/mysqlxx/mysqlxx/Pool.h similarity index 100% rename from base/mysqlxx/Pool.h rename to src/Common/mysqlxx/mysqlxx/Pool.h diff --git a/base/mysqlxx/PoolFactory.h b/src/Common/mysqlxx/mysqlxx/PoolFactory.h similarity index 100% rename from base/mysqlxx/PoolFactory.h rename to src/Common/mysqlxx/mysqlxx/PoolFactory.h diff --git a/base/mysqlxx/PoolWithFailover.h b/src/Common/mysqlxx/mysqlxx/PoolWithFailover.h similarity index 100% rename from base/mysqlxx/PoolWithFailover.h rename to src/Common/mysqlxx/mysqlxx/PoolWithFailover.h diff --git a/base/mysqlxx/Query.h b/src/Common/mysqlxx/mysqlxx/Query.h similarity index 100% rename from base/mysqlxx/Query.h rename to src/Common/mysqlxx/mysqlxx/Query.h diff --git a/base/mysqlxx/ResultBase.h b/src/Common/mysqlxx/mysqlxx/ResultBase.h similarity index 100% rename from base/mysqlxx/ResultBase.h rename to src/Common/mysqlxx/mysqlxx/ResultBase.h diff --git a/base/mysqlxx/Row.h b/src/Common/mysqlxx/mysqlxx/Row.h similarity index 100% rename from base/mysqlxx/Row.h rename to src/Common/mysqlxx/mysqlxx/Row.h diff --git a/base/mysqlxx/Transaction.h b/src/Common/mysqlxx/mysqlxx/Transaction.h similarity index 100% rename from base/mysqlxx/Transaction.h rename to src/Common/mysqlxx/mysqlxx/Transaction.h diff --git a/base/mysqlxx/Types.h b/src/Common/mysqlxx/mysqlxx/Types.h similarity index 100% rename from base/mysqlxx/Types.h rename to src/Common/mysqlxx/mysqlxx/Types.h diff --git a/base/mysqlxx/UseQueryResult.h b/src/Common/mysqlxx/mysqlxx/UseQueryResult.h similarity index 100% rename from base/mysqlxx/UseQueryResult.h rename to src/Common/mysqlxx/mysqlxx/UseQueryResult.h diff --git a/base/mysqlxx/Value.h b/src/Common/mysqlxx/mysqlxx/Value.h similarity index 99% rename from base/mysqlxx/Value.h rename to src/Common/mysqlxx/mysqlxx/Value.h index 6d3b2e96ebd..797a65a63f9 100644 --- a/base/mysqlxx/Value.h +++ b/src/Common/mysqlxx/mysqlxx/Value.h @@ -10,10 +10,10 @@ #include #include -#include +#include #include -#include +#include namespace mysqlxx diff --git a/base/mysqlxx/mysqlxx.h b/src/Common/mysqlxx/mysqlxx/mysqlxx.h similarity index 97% rename from base/mysqlxx/mysqlxx.h rename to src/Common/mysqlxx/mysqlxx/mysqlxx.h index 0caadcbb720..c117d43fd93 100644 --- a/base/mysqlxx/mysqlxx.h +++ b/src/Common/mysqlxx/mysqlxx/mysqlxx.h @@ -3,8 +3,8 @@ #include #include #include -#include -#include +#include +#include #include diff --git a/base/mysqlxx/tests/CMakeLists.txt b/src/Common/mysqlxx/tests/CMakeLists.txt similarity index 100% rename from base/mysqlxx/tests/CMakeLists.txt rename to src/Common/mysqlxx/tests/CMakeLists.txt diff --git a/base/mysqlxx/tests/mysqlxx_pool_test.cpp b/src/Common/mysqlxx/tests/mysqlxx_pool_test.cpp similarity index 100% rename from base/mysqlxx/tests/mysqlxx_pool_test.cpp rename to src/Common/mysqlxx/tests/mysqlxx_pool_test.cpp diff --git a/src/Common/tests/gtest_DateLUTImpl.cpp b/src/Common/tests/gtest_DateLUTImpl.cpp index a621f05c517..1220c50b409 100644 --- a/src/Common/tests/gtest_DateLUTImpl.cpp +++ b/src/Common/tests/gtest_DateLUTImpl.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include #include diff --git a/src/Common/tests/gtest_local_date_time_comparison.cpp b/src/Common/tests/gtest_local_date_time_comparison.cpp index 9f66da51c94..8aea710ea55 100644 --- a/src/Common/tests/gtest_local_date_time_comparison.cpp +++ b/src/Common/tests/gtest_local_date_time_comparison.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include void fillStackWithGarbage() diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 78554a08d07..fb230f412f0 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/DataTypes/DataTypeDate32.h b/src/DataTypes/DataTypeDate32.h index 9b93e5feb16..f0f78849e06 100644 --- a/src/DataTypes/DataTypeDate32.h +++ b/src/DataTypes/DataTypeDate32.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/DataTypes/Serializations/SerializationDateTime.cpp b/src/DataTypes/Serializations/SerializationDateTime.cpp index ce64bfd785a..b4269fb0f8c 100644 --- a/src/DataTypes/Serializations/SerializationDateTime.cpp +++ b/src/DataTypes/Serializations/SerializationDateTime.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/DataTypes/Serializations/SerializationDateTime64.cpp b/src/DataTypes/Serializations/SerializationDateTime64.cpp index aeba7e7e341..b9ed5bd4a02 100644 --- a/src/DataTypes/Serializations/SerializationDateTime64.cpp +++ b/src/DataTypes/Serializations/SerializationDateTime64.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/DataTypes/TimezoneMixin.h b/src/DataTypes/TimezoneMixin.h index e6e9f7a7989..03ecde5dd0a 100644 --- a/src/DataTypes/TimezoneMixin.h +++ b/src/DataTypes/TimezoneMixin.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include class DateLUTImpl; diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 5816b942d36..8d10a6665cf 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -1,7 +1,7 @@ #include "ExecutableDictionarySource.h" #include -#include +#include #include #include diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 01be40412b6..a0eb3435a11 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -1,7 +1,7 @@ #include "ExecutablePoolDictionarySource.h" #include -#include +#include #include #include diff --git a/src/Dictionaries/HTTPDictionarySource.h b/src/Dictionaries/HTTPDictionarySource.h index 35fbabecf2a..ce357814982 100644 --- a/src/Dictionaries/HTTPDictionarySource.h +++ b/src/Dictionaries/HTTPDictionarySource.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include "DictionaryStructure.h" #include "IDictionarySource.h" #include diff --git a/src/Dictionaries/LibraryDictionarySource.h b/src/Dictionaries/LibraryDictionarySource.h index e1cb01c0a14..9d7590c20ad 100644 --- a/src/Dictionaries/LibraryDictionarySource.h +++ b/src/Dictionaries/LibraryDictionarySource.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include "DictionaryStructure.h" #include diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 1ac2652d090..0bf5cc3cae0 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -101,7 +101,7 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) # include # include # include -# include +# include # include # include "readInvalidateQuery.h" # include diff --git a/src/Dictionaries/MySQLDictionarySource.h b/src/Dictionaries/MySQLDictionarySource.h index 37743ca2f62..90506ad1726 100644 --- a/src/Dictionaries/MySQLDictionarySource.h +++ b/src/Dictionaries/MySQLDictionarySource.h @@ -5,7 +5,7 @@ #include "config_core.h" #if USE_MYSQL -# include +# include # include # include "DictionaryStructure.h" # include "ExternalQueryBuilder.h" diff --git a/src/Dictionaries/PostgreSQLDictionarySource.h b/src/Dictionaries/PostgreSQLDictionarySource.h index 1cde2958107..87a87eac363 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.h +++ b/src/Dictionaries/PostgreSQLDictionarySource.h @@ -7,7 +7,7 @@ #if USE_LIBPQXX #include "ExternalQueryBuilder.h" #include -#include +#include #include #include diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 89cf18d530b..ab7cf65eb8b 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include "DictionarySourceFactory.h" #include "DictionaryStructure.h" diff --git a/src/Functions/CustomWeekTransforms.h b/src/Functions/CustomWeekTransforms.h index 218dcd083eb..5ccb2e06c44 100644 --- a/src/Functions/CustomWeekTransforms.h +++ b/src/Functions/CustomWeekTransforms.h @@ -9,7 +9,7 @@ #include #include #include -#include +#include /// The default mode value to use for the WEEK() function #define DEFAULT_WEEK_MODE 0 diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 15a08c4e76d..08dac9c2ba0 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -3,8 +3,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include diff --git a/src/Functions/FunctionDateOrDateTimeAddInterval.h b/src/Functions/FunctionDateOrDateTimeAddInterval.h index af34f27d6b8..4224a74ae8e 100644 --- a/src/Functions/FunctionDateOrDateTimeAddInterval.h +++ b/src/Functions/FunctionDateOrDateTimeAddInterval.h @@ -1,5 +1,5 @@ #pragma once -#include +#include #include #include diff --git a/src/Functions/FunctionsTimeWindow.h b/src/Functions/FunctionsTimeWindow.h index 6e3b5da9971..6e5d79fd062 100644 --- a/src/Functions/FunctionsTimeWindow.h +++ b/src/Functions/FunctionsTimeWindow.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/src/Functions/dateName.cpp b/src/Functions/dateName.cpp index c8c86060265..c89a7f80dfd 100644 --- a/src/Functions/dateName.cpp +++ b/src/Functions/dateName.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Functions/extractTimeZoneFromFunctionArguments.cpp b/src/Functions/extractTimeZoneFromFunctionArguments.cpp index 50254606510..88e1d664bf0 100644 --- a/src/Functions/extractTimeZoneFromFunctionArguments.cpp +++ b/src/Functions/extractTimeZoneFromFunctionArguments.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index 9bb2abcb2c7..9f303b86ad3 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -14,7 +14,7 @@ #include -#include +#include #include #include diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index 87d5f955e88..e277c906c1b 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #if defined(OS_LINUX) # include diff --git a/src/Functions/timezoneOf.cpp b/src/Functions/timezoneOf.cpp index a6556bdb800..03c9e27a3a8 100644 --- a/src/Functions/timezoneOf.cpp +++ b/src/Functions/timezoneOf.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index ecc3b80f088..f8ea44851b6 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Functions/today.cpp b/src/Functions/today.cpp index fb9fd945239..fe63197d127 100644 --- a/src/Functions/today.cpp +++ b/src/Functions/today.cpp @@ -1,4 +1,4 @@ -#include +#include #include diff --git a/src/Functions/yesterday.cpp b/src/Functions/yesterday.cpp index f792f885472..364d4721b34 100644 --- a/src/Functions/yesterday.cpp +++ b/src/Functions/yesterday.cpp @@ -1,4 +1,4 @@ -#include +#include #include diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 9d2d7fefd60..b2ad4035cdc 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -8,9 +8,9 @@ #include -#include -#include -#include +#include +#include +#include #include #include #include diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 5498e1c90f3..ca2c202014c 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -9,9 +9,9 @@ #include -#include -#include -#include +#include +#include +#include #include #include #include diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index 274a4ecc2f2..4f00ec5f96d 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/IO/tests/gtest_DateTimeToString.cpp b/src/IO/tests/gtest_DateTimeToString.cpp index c30c8943944..2d878fdd548 100644 --- a/src/IO/tests/gtest_DateTimeToString.cpp +++ b/src/IO/tests/gtest_DateTimeToString.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include diff --git a/src/Interpreters/QueryViewsLog.cpp b/src/Interpreters/QueryViewsLog.cpp index 2c0f1ecd878..c0703d77691 100644 --- a/src/Interpreters/QueryViewsLog.cpp +++ b/src/Interpreters/QueryViewsLog.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index fa8e77e4f4c..d0f15a4c595 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -22,7 +22,7 @@ #include #include -#include +#include #include diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index fee5c77f9ec..272907022a1 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp index 328c34823a0..d5a35fef7bd 100644 --- a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp @@ -1,7 +1,7 @@ #include #include -#include -#include +#include +#include #include diff --git a/src/Processors/TTL/ITTLAlgorithm.h b/src/Processors/TTL/ITTLAlgorithm.h index d219f9f7ad3..49cd2c46d9d 100644 --- a/src/Processors/TTL/ITTLAlgorithm.h +++ b/src/Processors/TTL/ITTLAlgorithm.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Processors/Transforms/TTLCalcTransform.h b/src/Processors/Transforms/TTLCalcTransform.h index 14592c07155..495879400dc 100644 --- a/src/Processors/Transforms/TTLCalcTransform.h +++ b/src/Processors/Transforms/TTLCalcTransform.h @@ -6,7 +6,7 @@ #include #include -#include +#include namespace DB { diff --git a/src/Processors/Transforms/TTLTransform.h b/src/Processors/Transforms/TTLTransform.h index 9207c68448b..3f0dffd1998 100644 --- a/src/Processors/Transforms/TTLTransform.h +++ b/src/Processors/Transforms/TTLTransform.h @@ -7,7 +7,7 @@ #include #include -#include +#include namespace DB { diff --git a/src/Server/WebUIRequestHandler.cpp b/src/Server/WebUIRequestHandler.cpp index 1dcad049f49..50aa0be4778 100644 --- a/src/Server/WebUIRequestHandler.cpp +++ b/src/Server/WebUIRequestHandler.cpp @@ -6,7 +6,7 @@ #include #include -#include +#include namespace DB diff --git a/src/Storages/System/attachInformationSchemaTables.cpp b/src/Storages/System/attachInformationSchemaTables.cpp index 803e9d55dac..68a1eac305e 100644 --- a/src/Storages/System/attachInformationSchemaTables.cpp +++ b/src/Storages/System/attachInformationSchemaTables.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB { From ae61cf66b54b5ec7af2689cda926aa604114c4b2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 22 Dec 2021 01:11:27 +0300 Subject: [PATCH 0304/1260] Fix resources --- src/Common/SymbolIndex.cpp | 30 ++++++++++++++++-------------- 1 file changed, 16 insertions(+), 14 deletions(-) diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index 46b3ffdd45d..104c0b7d779 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -92,8 +92,6 @@ void updateResources(std::string_view name, const void * address, SymbolIndex::R if (name.starts_with("_binary_") || name.starts_with("binary_")) { - std::cerr << name << "\n"; - if (name.ends_with("_start")) { name = name.substr((name[0] == '_') + strlen("binary_")); @@ -218,10 +216,6 @@ void collectSymbolsFromProgramHeaders( /* Iterate over the symbol table */ for (ElfW(Word) sym_index = 0; sym_index < ElfW(Word)(sym_cnt); ++sym_index) { - /// We are not interested in empty symbols. - if (!elf_sym[sym_index].st_size) - continue; - /* Get the name of the sym_index-th symbol. * This is located at the address of st_name relative to the beginning of the string table. */ @@ -231,11 +225,17 @@ void collectSymbolsFromProgramHeaders( continue; SymbolIndex::Symbol symbol; - symbol.address_begin = reinterpret_cast(info->dlpi_addr + elf_sym[sym_index].st_value); - symbol.address_end = reinterpret_cast(info->dlpi_addr + elf_sym[sym_index].st_value + elf_sym[sym_index].st_size); + symbol.address_begin = reinterpret_cast( + info->dlpi_addr + elf_sym[sym_index].st_value); + symbol.address_end = reinterpret_cast( + info->dlpi_addr + elf_sym[sym_index].st_value + elf_sym[sym_index].st_size); symbol.name = sym_name; - symbols.push_back(symbol); + /// We are not interested in empty symbols. + if (elf_sym[sym_index].st_size) + symbols.push_back(symbol); + + /// But resources can be represented by a pair of empty symbols (indicating their boundaries). updateResources(symbol.name, symbol.address_begin, resources); } @@ -278,7 +278,6 @@ void collectSymbolsFromELFSymbolTable( { if (!symbol_table_entry->st_name || !symbol_table_entry->st_value - || !symbol_table_entry->st_size || strings + symbol_table_entry->st_name >= elf.end()) continue; @@ -289,10 +288,14 @@ void collectSymbolsFromELFSymbolTable( continue; SymbolIndex::Symbol symbol; - symbol.address_begin = reinterpret_cast(info->dlpi_addr + symbol_table_entry->st_value); - symbol.address_end = reinterpret_cast(info->dlpi_addr + symbol_table_entry->st_value + symbol_table_entry->st_size); + symbol.address_begin = reinterpret_cast( + info->dlpi_addr + symbol_table_entry->st_value); + symbol.address_end = reinterpret_cast( + info->dlpi_addr + symbol_table_entry->st_value + symbol_table_entry->st_size); symbol.name = symbol_name; - symbols.push_back(symbol); + + if (symbol_table_entry->st_size) + symbols.push_back(symbol); updateResources(symbol.name, symbol.address_begin, resources); } @@ -320,7 +323,6 @@ bool searchAndCollectSymbolsFromELFSymbolTable( return (symbol_table && string_table); })) { - std::cerr << "!\n"; return false; } From 003ee51c6deb57713efd369603cdf5ffd0848bd9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 22 Dec 2021 01:40:14 +0300 Subject: [PATCH 0305/1260] Fix buildId --- src/Common/SymbolIndex.cpp | 20 +++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index 104c0b7d779..8bf01e4356c 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -246,6 +246,7 @@ void collectSymbolsFromProgramHeaders( } +#if !defined USE_MUSL String getBuildIDFromProgramHeaders(dl_phdr_info * info) { for (size_t header_index = 0; header_index < info->dlpi_phnum; ++header_index) @@ -258,6 +259,7 @@ String getBuildIDFromProgramHeaders(dl_phdr_info * info) } return {}; } +#endif void collectSymbolsFromELFSymbolTable( @@ -338,21 +340,33 @@ void collectSymbolsFromELF( SymbolIndex::Resources & resources, String & build_id) { + String object_name; + String our_build_id; + +#if defined (USE_MUSL) + object_name = "/proc/self/exe"; + our_build_id = Elf(object_name).getBuildID(); + build_id = our_build_id; +#else /// MSan does not know that the program segments in memory are initialized. __msan_unpoison_string(info->dlpi_name); - std::string object_name = info->dlpi_name; - - String our_build_id = getBuildIDFromProgramHeaders(info); + object_name = info->dlpi_name; + our_build_id = getBuildIDFromProgramHeaders(info); /// If the name is empty and there is a non-empty build-id - it's main executable. /// Find a elf file for the main executable and set the build-id. if (object_name.empty()) { object_name = "/proc/self/exe"; + + if (our_build_id.empty()) + our_build_id = Elf(object_name).getBuildID(); + if (build_id.empty()) build_id = our_build_id; } +#endif std::error_code ec; std::filesystem::path canonical_path = std::filesystem::canonical(object_name, ec); From 82c188e665e41c741fd3ac9b17c2b07bf6c7d40b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Dec 2021 06:47:19 +0300 Subject: [PATCH 0306/1260] Fix build --- contrib/mariadb-connector-c-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/mariadb-connector-c-cmake/CMakeLists.txt b/contrib/mariadb-connector-c-cmake/CMakeLists.txt index ea74e13b7f0..ebc4e657da8 100644 --- a/contrib/mariadb-connector-c-cmake/CMakeLists.txt +++ b/contrib/mariadb-connector-c-cmake/CMakeLists.txt @@ -238,6 +238,6 @@ target_link_libraries(mariadbclient ${SYSTEM_LIBS}) target_include_directories(mariadbclient PRIVATE ${CC_BINARY_DIR}/include-private - PUBLIC ${CC_BINARY_DIR}/include-public ${CC_SOURCE_DIR}/include ${CC_SOURCE_DIR}/libmariadb) + PUBLIC SYSTEM ${CC_BINARY_DIR}/include-public ${CC_SOURCE_DIR}/include ${CC_SOURCE_DIR}/libmariadb) set_target_properties(mariadbclient PROPERTIES IMPORTED_INTERFACE_LINK_LIBRARIES "${SYSTEM_LIBS}") From 6e09a2a642a7c4436b5bc809bc257d90567ff379 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Dec 2021 06:52:08 +0300 Subject: [PATCH 0307/1260] Fix build --- contrib/mariadb-connector-c-cmake/CMakeLists.txt | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/contrib/mariadb-connector-c-cmake/CMakeLists.txt b/contrib/mariadb-connector-c-cmake/CMakeLists.txt index ebc4e657da8..7c3f25cdf87 100644 --- a/contrib/mariadb-connector-c-cmake/CMakeLists.txt +++ b/contrib/mariadb-connector-c-cmake/CMakeLists.txt @@ -236,8 +236,7 @@ set(LIBMARIADB_SOURCES ${LIBMARIADB_SOURCES} ${CC_SOURCE_DIR}/libmariadb/mariadb add_library(mariadbclient STATIC ${LIBMARIADB_SOURCES}) target_link_libraries(mariadbclient ${SYSTEM_LIBS}) -target_include_directories(mariadbclient - PRIVATE ${CC_BINARY_DIR}/include-private - PUBLIC SYSTEM ${CC_BINARY_DIR}/include-public ${CC_SOURCE_DIR}/include ${CC_SOURCE_DIR}/libmariadb) +target_include_directories(mariadbclient PRIVATE ${CC_BINARY_DIR}/include-private) +target_include_directories(mariadbclient SYSTEM PUBLIC ${CC_BINARY_DIR}/include-public ${CC_SOURCE_DIR}/include ${CC_SOURCE_DIR}/libmariadb) set_target_properties(mariadbclient PROPERTIES IMPORTED_INTERFACE_LINK_LIBRARIES "${SYSTEM_LIBS}") From a96b4d50d693e4b29decf35544fe29c5976343e6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Dec 2021 07:28:51 +0300 Subject: [PATCH 0308/1260] Fix error --- src/Common/SymbolIndex.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index 8bf01e4356c..ce811055dde 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -97,14 +97,14 @@ void updateResources(std::string_view name, const void * address, SymbolIndex::R name = name.substr((name[0] == '_') + strlen("binary_")); name = name.substr(0, name.size() - strlen("_start")); - resources[name] = std::string_view{char_address, 0}; + resources.emplace(name, std::string_view{char_address, 0}); } else if (name.ends_with("_end")) { name = name.substr((name[0] == '_') + strlen("binary_")); name = name.substr(0, name.size() - strlen("_end")); - if (auto it = resources.find(name); it != resources.end()) + if (auto it = resources.find(name); it != resources.end() && it->second.size() == 0) { auto start = it->second.data(); assert(char_address >= start); From 8b91bdf9f20293e13dda8121d69a06e6280eae28 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Dec 2021 07:36:23 +0300 Subject: [PATCH 0309/1260] Enable query profiler --- base/base/phdr_cache.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/base/base/phdr_cache.cpp b/base/base/phdr_cache.cpp index 8ae10f6bf83..20a755ed7a4 100644 --- a/base/base/phdr_cache.cpp +++ b/base/base/phdr_cache.cpp @@ -123,6 +123,12 @@ bool hasPHDRCache() #else void updatePHDRCache() {} -bool hasPHDRCache() { return false; } + +#if defined(USE_MUSL) + /// With statically linked with musl, dl_iterate_phdr is immutable. + bool hasPHDRCache() { return true; } +#else + bool hasPHDRCache() { return false; } +#endif #endif From d1df2af2dfba31443e601a8d2d52c19c1950aaa3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Dec 2021 09:12:41 +0300 Subject: [PATCH 0310/1260] Move two tests out of fast test --- tests/queries/0_stateless/01502_jemalloc_percpu_arena.sh | 2 +- tests/queries/0_stateless/02003_memory_limit_in_client.expect | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01502_jemalloc_percpu_arena.sh b/tests/queries/0_stateless/01502_jemalloc_percpu_arena.sh index 7d03d1cbf5a..265ca4a6763 100755 --- a/tests/queries/0_stateless/01502_jemalloc_percpu_arena.sh +++ b/tests/queries/0_stateless/01502_jemalloc_percpu_arena.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan, no-asan, no-msan, no-ubsan +# Tags: no-tsan, no-asan, no-msan, no-ubsan, no-fasttest # ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ # NOTE: jemalloc is disabled under sanitizers diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.expect b/tests/queries/0_stateless/02003_memory_limit_in_client.expect index 19e3d4019ac..47ac4926537 100755 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.expect +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.expect @@ -1,5 +1,5 @@ #!/usr/bin/expect -f -# Tags: no-parallel +# Tags: no-parallel, no-fasttest # This is a test for system.warnings. Testing in interactive mode is necessary, # as we want to see certain warnings from client From edb50b5844b35a373152ce824469c9eabc3e51c9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Dec 2021 09:29:29 +0300 Subject: [PATCH 0311/1260] Remove unused code --- src/Common/LocalDate.h | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/src/Common/LocalDate.h b/src/Common/LocalDate.h index 5ca0696c371..6c847ceff0e 100644 --- a/src/Common/LocalDate.h +++ b/src/Common/LocalDate.h @@ -2,7 +2,6 @@ #include #include -#include #include #include @@ -154,19 +153,6 @@ public: { return !(*this == other); } - - /// NOTE Inefficient. - std::string toString(char separator = '-') const - { - std::stringstream ss; - if (separator) - ss << year() << separator << (month() / 10) << (month() % 10) - << separator << (day() / 10) << (day() % 10); - else - ss << year() << (month() / 10) << (month() % 10) - << (day() / 10) << (day() % 10); - return ss.str(); - } }; static_assert(sizeof(LocalDate) == 4); From ce5d63510f290dca6eb369ef2d177ff5fa6501e7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Dec 2021 09:47:17 +0300 Subject: [PATCH 0312/1260] Remove even more useless code --- src/Common/mysqlxx/Query.cpp | 21 +++------------ src/Common/mysqlxx/Value.cpp | 14 +++++----- src/Common/mysqlxx/mysqlxx/Connection.h | 2 +- src/Common/mysqlxx/mysqlxx/Query.h | 27 ++++--------------- src/Common/mysqlxx/mysqlxx/mysqlxx.h | 2 +- .../mysqlxx/tests/mysqlxx_pool_test.cpp | 9 ++----- 6 files changed, 21 insertions(+), 54 deletions(-) diff --git a/src/Common/mysqlxx/Query.cpp b/src/Common/mysqlxx/Query.cpp index d4514c3e988..11e72e14f9d 100644 --- a/src/Common/mysqlxx/Query.cpp +++ b/src/Common/mysqlxx/Query.cpp @@ -21,10 +21,7 @@ Query::Query(Connection * conn_, const std::string & query_string) : conn(conn_) /// Важно в случае, если Query используется не из того же потока, что Connection. mysql_thread_init(); - if (!query_string.empty()) - query_buf << query_string; - - query_buf.imbue(std::locale::classic()); + query = query_string; } Query::Query(const Query & other) : conn(other.conn) @@ -32,9 +29,7 @@ Query::Query(const Query & other) : conn(other.conn) /// Важно в случае, если Query используется не из того же потока, что Connection. mysql_thread_init(); - query_buf.imbue(std::locale::classic()); - - *this << other.str(); + query = other.query; } Query & Query::operator= (const Query & other) @@ -43,8 +38,7 @@ Query & Query::operator= (const Query & other) return *this; conn = other.conn; - - query_buf.str(other.str()); + query = other.query; return *this; } @@ -54,20 +48,13 @@ Query::~Query() mysql_thread_end(); } -void Query::reset() -{ - query_buf.str({}); -} - void Query::executeImpl() { - std::string query_string = query_buf.str(); - MYSQL* mysql_driver = conn->getDriver(); auto & logger = Poco::Logger::get("mysqlxx::Query"); logger.trace("Running MySQL query using connection %lu", mysql_thread_id(mysql_driver)); - if (mysql_real_query(mysql_driver, query_string.data(), query_string.size())) + if (mysql_real_query(mysql_driver, query.data(), query.size())) { const auto err_no = mysql_errno(mysql_driver); switch (err_no) diff --git a/src/Common/mysqlxx/Value.cpp b/src/Common/mysqlxx/Value.cpp index ed66167e8ea..85b63b722a2 100644 --- a/src/Common/mysqlxx/Value.cpp +++ b/src/Common/mysqlxx/Value.cpp @@ -156,19 +156,21 @@ void Value::throwException(const char * text) const { static constexpr size_t preview_length = 1000; - std::stringstream info; - info << text; + std::string info(text); if (!isNull()) { - info << ": "; - info.write(m_data, m_length); + info.append(": "); + info.append(m_data, m_length); } if (res && res->getQuery()) - info << ", query: " << res->getQuery()->str().substr(0, preview_length); + { + info.append(", query: "); + info.append(res->getQuery()->str().substr(0, preview_length)); + } - throw CannotParseValue(info.str()); + throw CannotParseValue(info); } } diff --git a/src/Common/mysqlxx/mysqlxx/Connection.h b/src/Common/mysqlxx/mysqlxx/Connection.h index 65955136eb1..548e75a1fef 100644 --- a/src/Common/mysqlxx/mysqlxx/Connection.h +++ b/src/Common/mysqlxx/mysqlxx/Connection.h @@ -154,7 +154,7 @@ public: bool ping(); /// Creates query. It can be set with query string or later. - Query query(const std::string & str = ""); + Query query(const std::string & str); /// Get MySQL C API MYSQL object. MYSQL * getDriver(); diff --git a/src/Common/mysqlxx/mysqlxx/Query.h b/src/Common/mysqlxx/mysqlxx/Query.h index 036e8952bc3..49aa3f223e7 100644 --- a/src/Common/mysqlxx/mysqlxx/Query.h +++ b/src/Common/mysqlxx/mysqlxx/Query.h @@ -13,9 +13,7 @@ namespace mysqlxx * Ссылается на Connection. Если уничтожить Connection, то Query станет некорректным и пользоваться им будет нельзя. * * Пример использования: - * mysqlxx::Query query = connection.query(); - * query << "SELECT 1 AS x, 2 AS y, 3 AS z"; - * query << " LIMIT 1"; + * mysqlxx::Query query = connection.query("SELECT 1 AS x, 2 AS y, 3 AS z LIMIT 1"); * mysqlxx::UseQueryResult result = query.use(); * * while (mysqlxx::Row row = result.fetch()) @@ -29,14 +27,11 @@ namespace mysqlxx class Query { public: - Query(Connection * conn_, const std::string & query_string = ""); + Query(Connection * conn_, const std::string & query_string); Query(const Query & other); Query & operator= (const Query & other); ~Query(); - /** Сбросить текст запроса. Это используется, если нужно написать новый запрос в том же объекте. */ - void reset(); - /** Выполнить запрос, результат которого не имеет значения (почти всё кроме SELECT). */ void execute(); @@ -54,24 +49,12 @@ public: /// Получить текст запроса (например, для вывода его в лог). См. ещё operator<< ниже. std::string str() const { - return query_buf.str(); - } - - auto rdbuf() const - { - return query_buf.rdbuf(); - } - - template - inline Query & operator<< (T && x) - { - query_buf << std::forward(x); - return *this; + return query; } private: Connection * conn; - std::ostringstream query_buf; + std::string query; void executeImpl(); }; @@ -80,7 +63,7 @@ private: /// Вывести текст запроса в ostream. inline std::ostream & operator<< (std::ostream & ostr, const Query & query) { - return ostr << query.rdbuf(); + return ostr << query.str(); } diff --git a/src/Common/mysqlxx/mysqlxx/mysqlxx.h b/src/Common/mysqlxx/mysqlxx/mysqlxx.h index c117d43fd93..785d4361fd7 100644 --- a/src/Common/mysqlxx/mysqlxx/mysqlxx.h +++ b/src/Common/mysqlxx/mysqlxx/mysqlxx.h @@ -23,7 +23,7 @@ * where values are stored consecutively as (non-zero-terminated) strings. * * 2. Too slow methods for converting values to numbers. - * In mysql++, it is done through std::stringstream. + * In mysql++, it is done through std::s*****stream (it is banned in our codebase). * This is slower than POSIX functions (strtoul, etc). * In turn, this is slower than simple hand-coded functions, * that doesn't respect locales and unused by MySQL number representations. diff --git a/src/Common/mysqlxx/tests/mysqlxx_pool_test.cpp b/src/Common/mysqlxx/tests/mysqlxx_pool_test.cpp index 3dc23e4da85..61d6a117285 100644 --- a/src/Common/mysqlxx/tests/mysqlxx_pool_test.cpp +++ b/src/Common/mysqlxx/tests/mysqlxx_pool_test.cpp @@ -2,7 +2,6 @@ #include #include -#include #include @@ -41,10 +40,7 @@ mysqlxx::Pool::Entry getWithFailover(mysqlxx::Pool & connections_pool) std::this_thread::sleep_for(1s); } - std::stringstream message; - message << "Connections to all replicas failed: " << connections_pool.getDescription(); - - throw Poco::Exception(message.str()); + throw Poco::Exception("Connections to all replicas failed: " + connections_pool.getDescription()); } } @@ -69,8 +65,7 @@ int main(int, char **) std::clog << "Preparing query (5s sleep) ..."; std::this_thread::sleep_for(5s); - mysqlxx::Query query = worker->query(); - query << test_query; + mysqlxx::Query query = worker->query(test_query); std::clog << "ok" << std::endl; std::clog << "Querying result (5s sleep) ..."; From b7799dcc456be2a07c602ef68b8355309f547773 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Dec 2021 09:51:05 +0300 Subject: [PATCH 0313/1260] Remove even more useless code --- src/Common/mysqlxx/Exception.cpp | 7 ++----- src/Common/mysqlxx/PoolWithFailover.cpp | 13 ++++--------- 2 files changed, 6 insertions(+), 14 deletions(-) diff --git a/src/Common/mysqlxx/Exception.cpp b/src/Common/mysqlxx/Exception.cpp index 188e7bd740d..0f5320da754 100644 --- a/src/Common/mysqlxx/Exception.cpp +++ b/src/Common/mysqlxx/Exception.cpp @@ -4,6 +4,7 @@ #include #endif #include +#include namespace mysqlxx @@ -11,11 +12,7 @@ namespace mysqlxx std::string errorMessage(MYSQL * driver) { - std::stringstream res; - res << mysql_error(driver) - << " (" << (driver->host ? driver->host : "(nullptr)") - << ":" << driver->port << ")"; - return res.str(); + return fmt::format("{} ({}:{})", mysql_error(driver), driver->host ? driver->host : "(nullptr)", driver->port); } void checkError(MYSQL * driver) diff --git a/src/Common/mysqlxx/PoolWithFailover.cpp b/src/Common/mysqlxx/PoolWithFailover.cpp index 9a5214454f5..e4da07c7a79 100644 --- a/src/Common/mysqlxx/PoolWithFailover.cpp +++ b/src/Common/mysqlxx/PoolWithFailover.cpp @@ -3,13 +3,8 @@ #include #include #include - - -/// Duplicate of code from StringUtils.h. Copied here for less dependencies. -static bool startsWith(const std::string & s, const char * prefix) -{ - return s.size() >= strlen(prefix) && 0 == memcmp(s.data(), prefix, strlen(prefix)); -} +#include +#include using namespace mysqlxx; @@ -31,7 +26,7 @@ PoolWithFailover::PoolWithFailover( for (const auto & replica_config_key : replica_keys) { /// There could be another elements in the same level in configuration file, like "password", "port"... - if (startsWith(replica_config_key, "replica")) + if (replica_config_key.starts_with("replica")) { std::string replica_name = config_name_ + "." + replica_config_key; @@ -181,7 +176,7 @@ PoolWithFailover::Entry PoolWithFailover::get() return (*full_pool)->get(wait_timeout); } - std::stringstream message; + DB::WriteBufferFromOwnString message; message << "Connections to all replicas failed: "; for (auto it = replicas_by_priority.begin(); it != replicas_by_priority.end(); ++it) for (auto jt = it->second.begin(); jt != it->second.end(); ++jt) From e3d8457a6fde69c6245d8b245dc77316d475c3a9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 24 Dec 2021 12:41:01 +0300 Subject: [PATCH 0314/1260] Fix PVS --- tests/ci/pvs_check.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index 1866d523ffd..af543211c16 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -87,7 +87,7 @@ if __name__ == "__main__": logging.info("Run Ok") if retcode != 0: - commit.create_status(context=NAME, description='PVS report failed to build', state='failure', + commit.create_status(context=NAME, description='PVS report failed to build', state='error', target_url=f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/actions/runs/{GITHUB_RUN_ID}") sys.exit(1) @@ -102,7 +102,7 @@ if __name__ == "__main__": break if not index_html: - commit.create_status(context=NAME, description='PVS report failed to build', state='failure', + commit.create_status(context=NAME, description='PVS report failed to build', state='error', target_url=f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/actions/runs/{GITHUB_RUN_ID}") sys.exit(1) From 1a3cdc3388a24ea161fd3ac1c739a4bf7e0d462a Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 24 Dec 2021 12:56:09 +0300 Subject: [PATCH 0315/1260] Upload build artifact in case of build failures --- .github/workflows/backport_branches.yml | 4 ++++ .github/workflows/main.yml | 16 ++++++++++++++++ .github/workflows/master.yml | 15 +++++++++++++++ .github/workflows/release_branches.yml | 6 ++++++ tests/ci/build_check.py | 6 +++++- 5 files changed, 46 insertions(+), 1 deletion(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index a9b3931fbaf..a9e472ab09c 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -95,6 +95,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -139,6 +140,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -183,6 +185,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -227,6 +230,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 78f559c8eb8..c42513ff9a8 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -236,6 +236,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -280,6 +281,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -324,6 +326,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -365,6 +368,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -409,6 +413,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -453,6 +458,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -497,6 +503,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -541,6 +548,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -585,6 +593,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -632,6 +641,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -676,6 +686,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -720,6 +731,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -764,6 +776,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -808,6 +821,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -852,6 +866,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -896,6 +911,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 19adf028084..c2ed39224aa 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -157,6 +157,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -201,6 +202,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -246,6 +248,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -290,6 +293,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -334,6 +338,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -378,6 +383,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -422,6 +428,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -466,6 +473,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -514,6 +522,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -559,6 +568,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -604,6 +614,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -649,6 +660,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -694,6 +706,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -739,6 +752,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -784,6 +798,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 7a898c79b1b..68a3554741d 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -98,6 +98,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -142,6 +143,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -186,6 +188,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -230,6 +233,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -274,6 +278,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} @@ -318,6 +323,7 @@ jobs: cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 0cf55e9cf7b..f37ea49e387 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -116,7 +116,11 @@ def create_json_artifact(temp_path, build_name, log_url, build_urls, build_confi "status": success, } - with open(os.path.join(temp_path, "build_urls_" + build_name + '.json'), 'w') as build_links: + json_name = "build_urls_" + build_name + '.json' + + print ("Dump json report", result, "to", json_name, "with env", "build_urls_{build_name}") + + with open(os.path.join(temp_path, json_name), 'w') as build_links: json.dump(result, build_links) From d68d26e4c1b7789ef3210e1ef5399daa435525db Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 24 Dec 2021 13:05:09 +0300 Subject: [PATCH 0316/1260] Fix woboq codebrowser --- tests/ci/codebrowser_check.py | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/tests/ci/codebrowser_check.py b/tests/ci/codebrowser_check.py index eeb06d2c684..97fd58c3235 100644 --- a/tests/ci/codebrowser_check.py +++ b/tests/ci/codebrowser_check.py @@ -12,7 +12,6 @@ from stopwatch import Stopwatch from upload_result_helper import upload_results from s3_helper import S3Helper from get_robot_token import get_best_robot_token -from pr_info import PRInfo from commit_status_helper import post_commit_status from docker_pull_helper import get_image_with_version from tee_popen import TeePopen @@ -33,8 +32,6 @@ if __name__ == "__main__": temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) - pr_info = PRInfo() - gh = Github(get_best_robot_token()) if not os.path.exists(temp_path): @@ -71,8 +68,8 @@ if __name__ == "__main__": test_results = [(index_html, "Look at the report")] - report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, [], NAME) + report_url = upload_results(s3_helper, 0, os.getenv("GITHUB_SHA"), test_results, [], NAME) print(f"::notice ::Report url: {report_url}") - post_commit_status(gh, pr_info.sha, NAME, "Report built", "success", report_url) + post_commit_status(gh, os.getenv("GITHUB_SHA"), NAME, "Report built", "success", report_url) From 602578deabf2931faa0d70946623ce40b8c59d1c Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 24 Dec 2021 13:20:58 +0300 Subject: [PATCH 0317/1260] Update bool serialization/deserialization for Values format --- .../Serializations/SerializationBool.cpp | 55 ++++++++++++++----- .../Serializations/SerializationBool.h | 4 +- .../0_stateless/02152_bool_type_parsing.sh | 2 +- utils/check-style/codespell-ignore-words.list | 2 +- 4 files changed, 46 insertions(+), 17 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp index 0542e050a25..1709e3321d6 100644 --- a/src/DataTypes/Serializations/SerializationBool.cpp +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -57,7 +57,7 @@ void SerializationBool::deserializeTextEscaped(IColumn & column, ReadBuffer & is if (istr.eof()) throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL); - deserializeWithCustom(column, istr, settings, [](ReadBuffer & buf){ return buf.eof() || *buf.position() == '\t' || *buf.position() == '\n'; }); + deserializeImpl(column, istr, settings, [](ReadBuffer & buf){ return buf.eof() || *buf.position() == '\t' || *buf.position() == '\n'; }); } void SerializationBool::serializeTextJSON(const IColumn &column, size_t row_num, WriteBuffer &ostr, const FormatSettings &settings) const @@ -93,7 +93,7 @@ void SerializationBool::deserializeTextCSV(IColumn & column, ReadBuffer & istr, if (istr.eof()) throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL); - deserializeWithCustom(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof() || *buf.position() == settings.csv.delimiter || *buf.position() == '\n'; }); + deserializeImpl(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof() || *buf.position() == settings.csv.delimiter || *buf.position() == '\n'; }); } void SerializationBool::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -106,7 +106,7 @@ void SerializationBool::deserializeTextRaw(IColumn & column, ReadBuffer & istr, if (istr.eof()) throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL); - deserializeWithCustom(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof() || *buf.position() == '\t' || *buf.position() == '\n'; }); + deserializeImpl(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof() || *buf.position() == '\t' || *buf.position() == '\n'; }); } void SerializationBool::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -114,16 +114,42 @@ void SerializationBool::serializeTextQuoted(const IColumn & column, size_t row_n serializeSimple(column, row_num, ostr, settings); } -void SerializationBool::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +void SerializationBool::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { if (istr.eof()) throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL); auto * col = checkAndGetDeserializeColumnType(column); - if (!deserializeImpl(col, istr)) - throw Exception( - ErrorCodes::CANNOT_PARSE_BOOL, - "Invalid boolean value, should be one of True/False/T/F/Y/N/Yes/No/On/Off/Enable/Disable/Enabled/Disabled/1/0"); + + char symbol = toLowerIfAlphaASCII(*istr.position()); + switch (symbol) + { + case 't': + assertStringCaseInsensitive("true", istr); + col->insert(true); + break; + case 'f': + assertStringCaseInsensitive("false", istr); + col->insert(false); + break; + case '1': + col->insert(true); + break; + case '0': + col->insert(false); + break; + case '\'': + ++istr.position(); + deserializeImpl(column, istr, settings, [](ReadBuffer & buf){ return !buf.eof() && *buf.position() == '\''; }); + assertChar('\'', istr); + break; + default: + throw Exception( + ErrorCodes::CANNOT_PARSE_BOOL, + "Cannot parse boolean value here: '{}', should be true/false, 1/0 or on of " + "True/False/T/F/Y/N/Yes/No/On/Off/Enable/Disable/Enabled/Disabled/1/0 in quotes", + String(istr.position(), std::min(10ul, istr.available()))); + } } void SerializationBool::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -131,7 +157,7 @@ void SerializationBool::deserializeWholeText(IColumn & column, ReadBuffer & istr if (istr.eof()) throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL); - deserializeWithCustom(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof(); }); + deserializeImpl(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof(); }); } void SerializationBool::serializeCustom(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -163,7 +189,7 @@ void SerializationBool::serializeSimple(const IColumn & column, size_t row_num, ostr.write(str_false, sizeof(str_false) - 1); } -void SerializationBool::deserializeWithCustom( +void SerializationBool::deserializeImpl( IColumn & column, ReadBuffer & istr, const FormatSettings & settings, std::function check_end_of_value) const { ColumnUInt8 * col = checkAndGetDeserializeColumnType(column); @@ -190,7 +216,7 @@ void SerializationBool::deserializeWithCustom( } buf.rollbackToCheckpoint(); - if (deserializeImpl(col, buf) && check_end_of_value(buf)) + if (tryDeserializeAllVariants(col, buf) && check_end_of_value(buf)) { buf.dropCheckpoint(); if (buf.hasUnreadData()) @@ -201,15 +227,18 @@ void SerializationBool::deserializeWithCustom( return; } + buf.makeContinuousMemoryFromCheckpointToPos(); + buf.rollbackToCheckpoint(); throw Exception( ErrorCodes::CANNOT_PARSE_BOOL, - "Invalid boolean value, should be '{}' or '{}' controlled by setting bool_true_representation and " + "Cannot parse boolean value here: '{}', should be '{}' or '{}' controlled by setting bool_true_representation and " "bool_false_representation or one of " "True/False/T/F/Y/N/Yes/No/On/Off/Enable/Disable/Enabled/Disabled/1/0", + String(buf.position(), std::min(10lu, buf.available())), settings.bool_true_representation, settings.bool_false_representation); } -bool SerializationBool::deserializeImpl(ColumnUInt8 * column, ReadBuffer & istr) const +bool SerializationBool::tryDeserializeAllVariants(ColumnUInt8 * column, ReadBuffer & istr) const { if (checkCharCaseInsensitive('1', istr)) { diff --git a/src/DataTypes/Serializations/SerializationBool.h b/src/DataTypes/Serializations/SerializationBool.h index 7999f5388d7..48c8b679694 100644 --- a/src/DataTypes/Serializations/SerializationBool.h +++ b/src/DataTypes/Serializations/SerializationBool.h @@ -40,8 +40,8 @@ public: protected: void serializeCustom(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; void serializeSimple(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; - void deserializeWithCustom(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, std::function check_end_of_value) const; - bool deserializeImpl(ColumnUInt8 * column, ReadBuffer & istr) const; + void deserializeImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, std::function check_end_of_value) const; + bool tryDeserializeAllVariants(ColumnUInt8 * column, ReadBuffer & istr) const; }; } diff --git a/tests/queries/0_stateless/02152_bool_type_parsing.sh b/tests/queries/0_stateless/02152_bool_type_parsing.sh index 65c173e700b..9e9db499cf5 100755 --- a/tests/queries/0_stateless/02152_bool_type_parsing.sh +++ b/tests/queries/0_stateless/02152_bool_type_parsing.sh @@ -17,7 +17,7 @@ $CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'TSV', 'bool Bool') sett $CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'CSV', 'bool Bool') settings bool_true_representation='Custom true', bool_false_representation='Custom false'" $CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'CSV', 'bool Bool') settings bool_true_representation='Custom true', bool_false_representation='Custom false', input_format_parallel_parsing=0, max_read_buffer_size=2" -echo -e "Yes\nNo\nyes\nno\ny\nY\nN\nTrue\nFalse\ntrue\nfalse\nt\nf\nT\nF\nOn\nOff\non\noff\nenable\ndisable\nenabled\ndisabled" > $DATA_FILE +echo -e "'Yes'\n'No'\n'yes'\n'no'\n'y'\n'Y'\n'N'\nTrue\nFalse\ntrue\nfalse\n't'\n'f'\n'T'\n'F'\n'On'\n'Off'\n'on'\n'off'\n'enable'\n'disable'\n'enabled'\n'disabled'" > $DATA_FILE $CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'CustomSeparated', 'bool Bool') settings format_custom_escaping_rule='Quoted'" $CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'CustomSeparated', 'bool Bool') settings format_custom_escaping_rule='Quoted', input_format_parallel_parsing=0, max_read_buffer_size=2" diff --git a/utils/check-style/codespell-ignore-words.list b/utils/check-style/codespell-ignore-words.list index 200b55d112d..d3a7586647c 100644 --- a/utils/check-style/codespell-ignore-words.list +++ b/utils/check-style/codespell-ignore-words.list @@ -6,7 +6,7 @@ nd ect pullrequest pullrequests -thenn ths offsett numer +ue From 9461b7dba2e8c5e991aca0a00c4e6ecc2c3fc82f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 24 Dec 2021 13:31:34 +0300 Subject: [PATCH 0318/1260] Finally fix jepsen --- tests/ci/keeper_jepsen_check.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/ci/keeper_jepsen_check.py b/tests/ci/keeper_jepsen_check.py index 6f049eb7a57..1897d1628cb 100644 --- a/tests/ci/keeper_jepsen_check.py +++ b/tests/ci/keeper_jepsen_check.py @@ -3,6 +3,7 @@ import time import logging import os +import sys import boto3 from github import Github @@ -20,6 +21,7 @@ from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickh from tee_popen import TeePopen from ssh import SSHKey from build_download_helper import get_build_name_for_check, get_build_urls +from rerun_helper import RerunHelper JEPSEN_GROUP_NAME = 'jepsen_group' DESIRED_INSTANCE_COUNT = 3 @@ -86,7 +88,7 @@ def prepare_autoscaling_group_and_get_hostnames(): def clear_autoscaling_group(): - asg_client = boto3.client('autoscaling') + asg_client = boto3.client('autoscaling', region_name='us-east-1') asg_client.set_desired_capacity(AutoScalingGroupName=JEPSEN_GROUP_NAME, DesiredCapacity=0) instances = get_autoscaling_group_instances_ids(asg_client, JEPSEN_GROUP_NAME) counter = 0 @@ -119,6 +121,11 @@ if __name__ == "__main__": gh = Github(get_best_robot_token()) + rerun_helper = RerunHelper(gh, pr_info, CHECK_NAME) + if rerun_helper.is_already_finished_by_status(): + logging.info("Check is already finished according to github status, exiting") + sys.exit(0) + if not os.path.exists(TEMP_PATH): os.makedirs(TEMP_PATH) From 3dbf7a9da9c0a703a9bef2faca9a636d1c6d93f8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 24 Dec 2021 13:35:40 +0300 Subject: [PATCH 0319/1260] More clear exception --- tests/ci/keeper_jepsen_check.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/keeper_jepsen_check.py b/tests/ci/keeper_jepsen_check.py index 1897d1628cb..03fd2f159bc 100644 --- a/tests/ci/keeper_jepsen_check.py +++ b/tests/ci/keeper_jepsen_check.py @@ -177,7 +177,8 @@ if __name__ == "__main__": compress_fast(os.path.join(result_path, 'store'), os.path.join(result_path, 'jepsen_store.tar.gz')) additional_data.append(os.path.join(result_path, 'jepsen_store.tar.gz')) - except: + except Exception as ex: + print("Exception", ex) status = 'failure' description = 'No Jepsen output log' test_result = [('No Jepsen output log', 'FAIL')] From 118004015aeccee8e81048c3bbf4d7a4ab80a950 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 24 Dec 2021 14:56:51 +0300 Subject: [PATCH 0320/1260] Longer timeout for server start in fuzzer --- docker/test/fuzzer/run-fuzzer.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 8b9be4077ae..844a7396134 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -194,8 +194,8 @@ quit time clickhouse-client --query "SELECT 'Connected to clickhouse-server after attaching gdb'" ||: # Check connectivity after we attach gdb, because it might cause the server - # to freeze and the fuzzer will fail. - for _ in {1..60} + # to freeze and the fuzzer will fail. In debug build it can take a lot of time. + for _ in {1..180} do sleep 1 if clickhouse-client --query "select 1" From e8cc6df7bbe6a3f24048bf0cbe14ede54b9db392 Mon Sep 17 00:00:00 2001 From: Andrii Buriachevskyi Date: Fri, 24 Dec 2021 13:10:19 +0100 Subject: [PATCH 0321/1260] Add support of DEFAULT keyword for INSERT --- .../Formats/Impl/ValuesBlockInputFormat.cpp | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index adf6d2e8a25..afe2d6a9f30 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -176,12 +176,20 @@ bool ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx) try { bool read = true; - const auto & type = types[column_idx]; - const auto & serialization = serializations[column_idx]; - if (format_settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable()) - read = SerializationNullable::deserializeTextQuotedImpl(column, *buf, format_settings, serialization); + if (bool default_value = checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("DEFAULT", *buf); default_value) + { + column.insertDefault(); + read = false; + } else - serialization->deserializeTextQuoted(column, *buf, format_settings); + { + const auto & type = types[column_idx]; + const auto & serialization = serializations[column_idx]; + if (format_settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable()) + read = SerializationNullable::deserializeTextQuotedImpl(column, *buf, format_settings, serialization); + else + serialization->deserializeTextQuoted(column, *buf, format_settings); + } rollback_on_exception = true; From d4debd115d3a40c0f82ac6d5c08acd4fe6edc5d5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Dec 2021 15:37:40 +0300 Subject: [PATCH 0322/1260] Remove unused code --- programs/odbc-bridge/ODBCBlockOutputStream.cpp | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/programs/odbc-bridge/ODBCBlockOutputStream.cpp b/programs/odbc-bridge/ODBCBlockOutputStream.cpp index 710614130c3..1c28da2a072 100644 --- a/programs/odbc-bridge/ODBCBlockOutputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockOutputStream.cpp @@ -1,13 +1,8 @@ #include "ODBCBlockOutputStream.h" -#include #include -#include -#include -#include -#include "getIdentifierQuote.h" -#include -#include +#include +#include #include #include @@ -45,7 +40,7 @@ void ODBCSink::consume(Chunk chunk) std::string query = getInsertQuery(db_name, table_name, block.getColumnsWithTypeAndName(), quoting) + values_buf.str(); execute(connection_holder, - [&](nanodbc::connection & connection) { execute(connection, query); }); + [&](nanodbc::connection & connection) { execute(connection, query); }); } } From 6a0a0efd29fb36e0abb7cc33f381420b5a6292e5 Mon Sep 17 00:00:00 2001 From: Andrii Buriachevskyi Date: Fri, 24 Dec 2021 13:45:43 +0100 Subject: [PATCH 0323/1260] Update docs with DEFAULT keyword for INSERT statement --- docs/en/sql-reference/statements/insert-into.md | 6 +++++- docs/ru/sql-reference/statements/insert-into.md | 7 ++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index c36f1b1c9cf..f8eefad7051 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -55,7 +55,11 @@ SELECT * FROM insert_select_testtable; └───┴───┴───┘ ``` -In this example, we see that the second inserted row has `a` and `c` columns filled by the passed values, and `b` filled with value by default. +In this example, we see that the second inserted row has `a` and `c` columns filled by the passed values, and `b` filled with value by default. It is also possible to use `DEFAULT` keyword to insert default values: + +``` sql +INSERT INTO insert_select_testtable VALUES (1, DEFAULT, 1) ; +``` If a list of columns does not include all existing columns, the rest of the columns are filled with: diff --git a/docs/ru/sql-reference/statements/insert-into.md b/docs/ru/sql-reference/statements/insert-into.md index 21fef0e0286..a2d9ebaf68e 100644 --- a/docs/ru/sql-reference/statements/insert-into.md +++ b/docs/ru/sql-reference/statements/insert-into.md @@ -55,7 +55,12 @@ SELECT * FROM insert_select_testtable └───┴───┴───┘ ``` -В этом примере мы видим, что вторая строка содержит столбцы `a` и `c`, заполненные переданными значениями и `b`, заполненный значением по умолчанию. +В этом примере мы видим, что вторая строка содержит столбцы `a` и `c`, заполненные переданными значениями и `b`, заполненный значением по умолчанию. Также можно использовать ключевое слово `DEFAULT` для вставки значений по умолчанию: + +``` sql +INSERT INTO insert_select_testtable VALUES (1, DEFAULT, 1) ; +``` + Если список столбцов не включает все существующие столбцы, то все остальные столбцы заполняются следующим образом: - Значения, вычисляемые из `DEFAULT` выражений, указанных в определении таблицы. From 42461f453efa13b7566734b3512e4bb793eb32aa Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 24 Dec 2021 15:56:03 +0300 Subject: [PATCH 0324/1260] Add a separate workflow --- .github/workflows/jepsen.yml | 42 +++++++++++++++++++++++++++++++++ .github/workflows/main.yml | 33 -------------------------- tests/ci/keeper_jepsen_check.py | 42 +++++++++++++++++++++++---------- 3 files changed, 72 insertions(+), 45 deletions(-) create mode 100644 .github/workflows/jepsen.yml diff --git a/.github/workflows/jepsen.yml b/.github/workflows/jepsen.yml new file mode 100644 index 00000000000..eb079d50d1d --- /dev/null +++ b/.github/workflows/jepsen.yml @@ -0,0 +1,42 @@ +name: JepsenWorkflow +env: + # Force the stdout and stderr streams to be unbuffered + PYTHONUNBUFFERED: 1 +concurrency: + group: jepsen +on: # yamllint disable-line rule:truthy + schedule: + - cron: '0 */6 * * *' + workflow_run: + workflows: ["CIGithubActions"] + types: + - completed + workflow_dispatch: +jobs: + KeeperJepsenRelease: + runs-on: [self-hosted, style-checker] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/keeper_jepsen + REPO_COPY=${{runner.temp}}/keeper_jepsen/ClickHouse + EOF + - name: Clear repository + run: | + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE + - name: Check out repository code + uses: actions/checkout@v2 + - name: Jepsen Test + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 keeper_jepsen_check.py + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 0adbf1b784e..a88fee52b3c 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -199,39 +199,6 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH - KeeperJepsenRelease: - needs: [BuilderBinRelease] - runs-on: [self-hosted, style-checker] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/keeper_jepsen - REPORTS_PATH=${{runner.temp}}/reports_dir - REPO_COPY=${{runner.temp}}/keeper_jepsen/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - - name: Check out repository code - uses: actions/checkout@v2 - - name: Jepsen Test - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 keeper_jepsen_check.py - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH ######################################################################################### #################################### ORDINARY BUILDS #################################### ######################################################################################### diff --git a/tests/ci/keeper_jepsen_check.py b/tests/ci/keeper_jepsen_check.py index 03fd2f159bc..cebe5f88d08 100644 --- a/tests/ci/keeper_jepsen_check.py +++ b/tests/ci/keeper_jepsen_check.py @@ -7,8 +7,9 @@ import sys import boto3 from github import Github +import requests -from env_helper import REPORTS_PATH, REPO_COPY, TEMP_PATH +from env_helper import REPO_COPY, TEMP_PATH from stopwatch import Stopwatch from upload_result_helper import upload_results from s3_helper import S3Helper @@ -16,11 +17,11 @@ from get_robot_token import get_best_robot_token, get_parameter_from_ssm from pr_info import PRInfo from compress_files import compress_fast from commit_status_helper import post_commit_status -from docker_pull_helper import get_image_with_version from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse +from version_helper import get_version_from_repo from tee_popen import TeePopen from ssh import SSHKey -from build_download_helper import get_build_name_for_check, get_build_urls +from build_download_helper import get_build_name_for_check from rerun_helper import RerunHelper JEPSEN_GROUP_NAME = 'jepsen_group' @@ -119,6 +120,10 @@ if __name__ == "__main__": pr_info = PRInfo() + if pr_info.number != 0 and 'jepsen test' not in pr_info.labels(): + logging.info("Not jepsen test label in labels list, skipping") + sys.exit(0) + gh = Github(get_best_robot_token()) rerun_helper = RerunHelper(gh, pr_info, CHECK_NAME) @@ -136,19 +141,32 @@ if __name__ == "__main__": instances = prepare_autoscaling_group_and_get_hostnames() nodes_path = save_nodes_to_file(instances, TEMP_PATH) - docker_image = get_image_with_version(REPORTS_PATH, IMAGE_NAME) + # always use latest + docker_image = IMAGE_NAME build_name = get_build_name_for_check(CHECK_NAME) - urls = get_build_urls(build_name, REPORTS_PATH) - if not urls: - raise Exception("No build URLs found") - for url in urls: - if url.endswith('/clickhouse'): - build_url = url - break + if pr_info.number == 0: + version = get_version_from_repo(REPO_COPY) + release_or_pr = ".".join(version.as_tuple()[:2]) else: - raise Exception("Cannot binary clickhouse among build results") + # PR number for anything else + release_or_pr = str(pr_info.number) + + # This check run separately from other checks because it requires exclusive + # run (see .github/workflows/jepsen.yml) So we cannot add explicit + # dependency on a build job and using busy loop on it's results. For the + # same reason we are using latest docker image. + build_url = f"https://s3.amazonaws.com/clickhouse-builds/{release_or_pr}/{pr_info.sha}/{build_name}/clickhouse" + head = requests.head(build_url) + counter = 0 + while head.status_code != 200: + time.sleep(10) + head = requests.head(build_url) + counter += 1 + if counter >= 180: + post_commit_status(gh, pr_info.sha, CHECK_NAME, "Cannot fetch build to run", "error", "") + raise Exception("Cannot fetch build") with SSHKey(key_value=get_parameter_from_ssm("jepsen_ssh_key") + '\n'): ssh_auth_sock = os.environ['SSH_AUTH_SOCK'] From 05648434b9892dc613184a73f69611515f86b8ef Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 24 Dec 2021 15:56:38 +0300 Subject: [PATCH 0325/1260] Fix label --- tests/ci/keeper_jepsen_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/keeper_jepsen_check.py b/tests/ci/keeper_jepsen_check.py index cebe5f88d08..5c7582242a9 100644 --- a/tests/ci/keeper_jepsen_check.py +++ b/tests/ci/keeper_jepsen_check.py @@ -120,7 +120,7 @@ if __name__ == "__main__": pr_info = PRInfo() - if pr_info.number != 0 and 'jepsen test' not in pr_info.labels(): + if pr_info.number != 0 and 'jepsen-test' not in pr_info.labels(): logging.info("Not jepsen test label in labels list, skipping") sys.exit(0) From 413e9fa48f770785dd65b44948d1a0a02bbda6f2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 24 Dec 2021 15:57:08 +0300 Subject: [PATCH 0326/1260] Fix --- .github/workflows/main.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index a88fee52b3c..78f559c8eb8 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -2919,7 +2919,6 @@ jobs: - SplitBuildSmokeTest - CompatibilityCheck - IntegrationTestsFlakyCheck - - KeeperJepsenRelease runs-on: [self-hosted, style-checker] steps: - name: Clear repository From 6d0881ead67dc08790e10edfb6acf3e819c3d832 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 24 Dec 2021 14:00:36 +0000 Subject: [PATCH 0327/1260] Fix --- src/AggregateFunctions/AggregateFunctionNothing.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionNothing.h b/src/AggregateFunctions/AggregateFunctionNothing.h index 22e296a96a4..34d83ecfff6 100644 --- a/src/AggregateFunctions/AggregateFunctionNothing.h +++ b/src/AggregateFunctions/AggregateFunctionNothing.h @@ -28,7 +28,7 @@ public: DataTypePtr getReturnType() const override { - return argument_types.front(); + return std::make_shared(std::make_shared()); } bool allocatesMemoryInArena() const override { return false; } From 6a0834499f1af59dcf8dfca1ed135a67e313fd58 Mon Sep 17 00:00:00 2001 From: Andrii Buriachevskyi Date: Fri, 24 Dec 2021 15:09:32 +0100 Subject: [PATCH 0328/1260] Add tests for DEFAULT in INSERT --- .../0_stateless/02154_default_keyword_insert.reference | 3 +++ tests/queries/0_stateless/02154_default_keyword_insert.sql | 7 +++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/02154_default_keyword_insert.reference create mode 100644 tests/queries/0_stateless/02154_default_keyword_insert.sql diff --git a/tests/queries/0_stateless/02154_default_keyword_insert.reference b/tests/queries/0_stateless/02154_default_keyword_insert.reference new file mode 100644 index 00000000000..158abdfd686 --- /dev/null +++ b/tests/queries/0_stateless/02154_default_keyword_insert.reference @@ -0,0 +1,3 @@ +0 42 33 +1 42 33 +2 33 33 diff --git a/tests/queries/0_stateless/02154_default_keyword_insert.sql b/tests/queries/0_stateless/02154_default_keyword_insert.sql new file mode 100644 index 00000000000..efccc35e6a8 --- /dev/null +++ b/tests/queries/0_stateless/02154_default_keyword_insert.sql @@ -0,0 +1,7 @@ +CREATE TEMPORARY TABLE IF NOT EXISTS default_table (x UInt32, y UInt32 DEFAULT 42, z UInt32 DEFAULT 33) ENGINE = Memory; + +INSERT INTO default_table(x) values (DEFAULT); +INSERT INTO default_table(x, z) values (1, DEFAULT); +INSERT INTO default_table values (2, 33, DEFAULT); + +SELECT * FROM default_table ORDER BY x; From 58966e90e941ba32c5351a6d6f72ad44f2d6a641 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 24 Dec 2021 17:13:35 +0300 Subject: [PATCH 0329/1260] Don't allow to write into S3 if path contains globs --- src/Storages/StorageS3.cpp | 9 ++++----- tests/integration/test_storage_s3/test.py | 7 +++++++ 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 3f08dee62b6..552aa1caa6b 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -10,7 +10,6 @@ #include -#include #include #include #include @@ -25,7 +24,6 @@ #include #include -#include #include #include @@ -54,13 +52,10 @@ #include #include #include -#include #include namespace fs = std::filesystem; -#include - static const String PARTITION_ID_WILDCARD = "{_partition_id}"; @@ -74,6 +69,7 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int S3_ERROR; extern const int UNEXPECTED_EXPRESSION; + extern const int CANNOT_OPEN_FILE; } class IOutputFormat; @@ -312,6 +308,9 @@ public: , sample_block(sample_block_) , format_settings(format_settings_) { + if (key.find_first_of("*?{") != std::string::npos) + throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "S3 key '{}' contains globs, so the table is in readonly mode", key); + write_buf = wrapWriteBufferWithCompressionMethod( std::make_unique(client, bucket, key, min_upload_part_size, max_single_part_upload_size), compression_method, 3); writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, {}, format_settings); diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index b4f54aa1e10..f3c4b1dd0cf 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -858,3 +858,10 @@ def test_empty_file(started_cluster): table_function = f"s3('{url}', 'CSV', 'id Int32')" result = instance.query(f"SELECT count() FROM {table_function}") assert(int(result) == 0) + + +def test_insert_with_path_with_globs(started_cluster): + instance = started_cluster.instances["dummy"] + + table_function_3 = f"s3('http://minio1:9001/root/test_parquet*', 'minio', 'minio123', 'Parquet', 'a Int32, b String')" + instance.query_and_get_error(f"insert into table function {table_function_3} SELECT number, randomString(100) FROM numbers(500)") From c8b8cc74fcda9cbbd565983f122300d4501ad60b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Dec 2021 18:15:04 +0300 Subject: [PATCH 0330/1260] Fix clang-tidy --- src/Common/SymbolIndex.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index ce811055dde..c117ea8c391 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -97,14 +97,14 @@ void updateResources(std::string_view name, const void * address, SymbolIndex::R name = name.substr((name[0] == '_') + strlen("binary_")); name = name.substr(0, name.size() - strlen("_start")); - resources.emplace(name, std::string_view{char_address, 0}); + resources.emplace(name, std::string_view{char_address, 0}); // NOLINT } else if (name.ends_with("_end")) { name = name.substr((name[0] == '_') + strlen("binary_")); name = name.substr(0, name.size() - strlen("_end")); - if (auto it = resources.find(name); it != resources.end() && it->second.size() == 0) + if (auto it = resources.find(name); it != resources.end() && it->second.empty()) { auto start = it->second.data(); assert(char_address >= start); From bbd8e055a14a6e5ab3fcba1b07b33961d09ac688 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Dec 2021 18:25:59 +0300 Subject: [PATCH 0331/1260] Fix clang-tidy --- src/Common/SymbolIndex.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index c117ea8c391..568f633975b 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -106,7 +106,7 @@ void updateResources(std::string_view name, const void * address, SymbolIndex::R if (auto it = resources.find(name); it != resources.end() && it->second.empty()) { - auto start = it->second.data(); + const char * start = it->second.data(); assert(char_address >= start); it->second = std::string_view{start, static_cast(char_address - start)}; } From 1772795d435eb59bb4451757fab0310e5da9beba Mon Sep 17 00:00:00 2001 From: Vitaly Artemyev Date: Fri, 24 Dec 2021 18:28:29 +0300 Subject: [PATCH 0332/1260] Fix for example request with settings --- docs/en/engines/database-engines/materialized-postgresql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index d2c4dbf1f3c..a8fdcb4949f 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -89,7 +89,7 @@ Tables are accessed via schema name and table name at the same time: ``` sql CREATE DATABASE database1 ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password') -SETTINGS materialized_postgresql_tables_list = 'schema1.table1,schema2.table2,schema1.table3'; +SETTINGS materialized_postgresql_tables_list = 'schema1.table1,schema2.table2,schema1.table3', materialized_postgresql_tables_list_with_schema = 1; SELECT * FROM database1.`schema1.table1`; From 534dda2be899b411ed5a465abf015f4fd6e1cf9b Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 24 Dec 2021 12:34:18 -0400 Subject: [PATCH 0333/1260] refresh zh on_time doc --- docs/zh/getting-started/example-datasets/ontime.md | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/docs/zh/getting-started/example-datasets/ontime.md b/docs/zh/getting-started/example-datasets/ontime.md index 03a9a8c4278..907f63634cc 100644 --- a/docs/zh/getting-started/example-datasets/ontime.md +++ b/docs/zh/getting-started/example-datasets/ontime.md @@ -15,17 +15,9 @@ toc_title: OnTime 下载数据: ``` bash -for s in `seq 1987 2018` -do -for m in `seq 1 12` -do -wget https://transtats.bts.gov/PREZIP/On_Time_Reporting_Carrier_On_Time_Performance_1987_present_${s}_${m}.zip -done -done +wget --no-check-certificate --continue https://transtats.bts.gov/PREZIP/On_Time_Reporting_Carrier_On_Time_Performance_1987_present_{1987..2021}_{1..12}.zip ``` -(参考 https://github.com/Percona-Lab/ontime-airline-performance/blob/master/download.sh ) - 创建表结构: ``` sql @@ -40,7 +32,7 @@ CREATE TABLE `ontime` `Reporting_Airline` String, `DOT_ID_Reporting_Airline` Int32, `IATA_CODE_Reporting_Airline` String, - `Tail_Number` Int32, + `Tail_Number` String, `Flight_Number_Reporting_Airline` String, `OriginAirportID` Int32, `OriginAirportSeqID` Int32, From 37294b4cbe0730e95a0dc19ed430c253fbce7c9f Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 24 Dec 2021 12:38:05 -0400 Subject: [PATCH 0334/1260] Update ontime.md --- docs/ru/getting-started/example-datasets/ontime.md | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/docs/ru/getting-started/example-datasets/ontime.md b/docs/ru/getting-started/example-datasets/ontime.md index e1d47a5a9e7..2ee4315c76f 100644 --- a/docs/ru/getting-started/example-datasets/ontime.md +++ b/docs/ru/getting-started/example-datasets/ontime.md @@ -15,13 +15,7 @@ toc_title: OnTime Скачивание данных (из `https://github.com/Percona-Lab/ontime-airline-performance/blob/master/download.sh`): ``` bash -for s in `seq 1987 2018` -do -for m in `seq 1 12` -do -wget https://transtats.bts.gov/PREZIP/On_Time_Reporting_Carrier_On_Time_Performance_1987_present_${s}_${m}.zip -done -done +wget --no-check-certificate --continue https://transtats.bts.gov/PREZIP/On_Time_Reporting_Carrier_On_Time_Performance_1987_present_{1987..2021}_{1..12}.zip ``` Создание таблицы: @@ -38,7 +32,7 @@ CREATE TABLE `ontime` `Reporting_Airline` String, `DOT_ID_Reporting_Airline` Int32, `IATA_CODE_Reporting_Airline` String, - `Tail_Number` Int32, + `Tail_Number` String, `Flight_Number_Reporting_Airline` String, `OriginAirportID` Int32, `OriginAirportSeqID` Int32, From 91bf7b8898200fa9dcba83eea6ef6d7338f7a116 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 24 Dec 2021 12:39:37 -0400 Subject: [PATCH 0335/1260] Update ontime.md --- docs/ja/getting-started/example-datasets/ontime.md | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/docs/ja/getting-started/example-datasets/ontime.md b/docs/ja/getting-started/example-datasets/ontime.md index 2a951af6026..33314faa53d 100644 --- a/docs/ja/getting-started/example-datasets/ontime.md +++ b/docs/ja/getting-started/example-datasets/ontime.md @@ -15,13 +15,7 @@ toc_title: OnTime データのダウンロード: ``` bash -for s in `seq 1987 2018` -do -for m in `seq 1 12` -do -wget https://transtats.bts.gov/PREZIP/On_Time_Reporting_Carrier_On_Time_Performance_1987_present_${s}_${m}.zip -done -done +wget --no-check-certificate --continue https://transtats.bts.gov/PREZIP/On_Time_Reporting_Carrier_On_Time_Performance_1987_present_{1987..2021}_{1..12}.zip ``` (https://github.com/Percona-Lab/ontime-airline-performance/blob/master/download.sh より) @@ -40,7 +34,7 @@ CREATE TABLE `ontime` `Reporting_Airline` String, `DOT_ID_Reporting_Airline` Int32, `IATA_CODE_Reporting_Airline` String, - `Tail_Number` Int32, + `Tail_Number` String, `Flight_Number_Reporting_Airline` String, `OriginAirportID` Int32, `OriginAirportSeqID` Int32, From c37f1c40be13690f69e9a68a3c03ec8fc029143f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 24 Dec 2021 19:39:50 +0300 Subject: [PATCH 0336/1260] fix functions with sparse arguments --- src/Functions/IFunction.cpp | 2 +- .../0_stateless/01780_column_sparse_pk.reference | 6 ++++++ tests/queries/0_stateless/01780_column_sparse_pk.sql | 10 +++++++++- 3 files changed, 16 insertions(+), 2 deletions(-) diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 222e91aac63..cfb4e12a025 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -321,7 +321,7 @@ ColumnPtr IExecutableFunction::execute(const ColumnsWithTypeAndName & arguments, return res->cloneResized(input_rows_count); /// If default of sparse column is changed after execution of function, convert to full column. - if (!res->isDefaultAt(0)) + if (!result_type->supportsSparseSerialization() || !res->isDefaultAt(0)) { const auto & offsets_data = assert_cast &>(*sparse_offsets).getData(); return res->createWithOffsets(offsets_data, (*res)[0], input_rows_count, /*shift=*/ 1); diff --git a/tests/queries/0_stateless/01780_column_sparse_pk.reference b/tests/queries/0_stateless/01780_column_sparse_pk.reference index 56131dcfaed..11bb0471689 100644 --- a/tests/queries/0_stateless/01780_column_sparse_pk.reference +++ b/tests/queries/0_stateless/01780_column_sparse_pk.reference @@ -7,3 +7,9 @@ 200 84 800 167 800 167 +\N +\N +\N +[] +[] +[] diff --git a/tests/queries/0_stateless/01780_column_sparse_pk.sql b/tests/queries/0_stateless/01780_column_sparse_pk.sql index a162f7bdfce..63ed9e99a87 100644 --- a/tests/queries/0_stateless/01780_column_sparse_pk.sql +++ b/tests/queries/0_stateless/01780_column_sparse_pk.sql @@ -32,4 +32,12 @@ SELECT count(v), sum(v) FROM t_sparse_pk WHERE k = 0; SELECT count(v), sum(v) FROM t_full_pk WHERE k = 0; SELECT count(v), sum(v) FROM t_sparse_pk WHERE k = 0 OR k = 3 OR k = 7 OR k = 8; -SELECT count(v), sum(v) FROM t_full_pk WHERE k = 0 OR k = 3 OR k = 7 OR k = 8; \ No newline at end of file +SELECT count(v), sum(v) FROM t_full_pk WHERE k = 0 OR k = 3 OR k = 7 OR k = 8; + +SET force_primary_key = 0; + +SELECT (k = NULL) OR (k = 1000) FROM t_sparse_pk LIMIT 3; +SELECT range(k) FROM t_sparse_pk LIMIT 3; + +DROP TABLE IF EXISTS t_sparse_pk; +DROP TABLE IF EXISTS t_full_pk; From 9d83b904de56d299c1c2c670b48c9fb294cd70b9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 24 Dec 2021 20:57:32 +0300 Subject: [PATCH 0337/1260] Add pretty print unknow event --- tests/ci/pr_info.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 795fe9aaad3..f4a8ba7e1ec 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -111,6 +111,7 @@ class PRInfo: else: self.diff_url = pull_request['diff_url'] else: + print(json.dumps(github_event, sort_keys=True, indent=4)) raise Exception("Cannot detect type of event") if need_changed_files: self.fetch_changed_files() From 46569e7a5bbeadc3e26d0f42197becc8903c3862 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 24 Dec 2021 21:02:40 +0300 Subject: [PATCH 0338/1260] Better --- tests/ci/pr_info.py | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index f4a8ba7e1ec..e389b75beae 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -112,11 +112,26 @@ class PRInfo: self.diff_url = pull_request['diff_url'] else: print(json.dumps(github_event, sort_keys=True, indent=4)) - raise Exception("Cannot detect type of event") + self.sha = os.getenv("GITHUB_SHA") + self.number = 0 + self.labels = {} + repo_prefix = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}" + self.task_url = f"{repo_prefix}/actions/runs/{GITHUB_RUN_ID or '0'}" + self.commit_html_url = f"{repo_prefix}/commits/{self.sha}" + self.repo_full_name = GITHUB_REPOSITORY + self.pr_html_url = f"{repo_prefix}/commits/master" + self.base_ref = "master" + self.base_name = self.repo_full_name + self.head_ref = "master" + self.head_name = self.repo_full_name + if need_changed_files: self.fetch_changed_files() def fetch_changed_files(self): + if not self.diff_url: + raise Exception("Diff URL cannot be find for event") + if 'commits' in self.event and self.number == 0: response = requests.get(self.diff_url) response.raise_for_status() From bd7f5ea50f839832d0bc973aca382cad7b2a4a73 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 24 Dec 2021 21:22:29 +0300 Subject: [PATCH 0339/1260] Better --- tests/ci/pr_info.py | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index e389b75beae..d72c8324b02 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -42,6 +42,14 @@ class PRInfo: github_event = {'commits': 1, 'after': 'HEAD', 'ref': None} self.event = github_event self.changed_files = set([]) + + # workflow completed event, used for PRs only + if github_event['action'] == 'completed': + self.sha = github_event['workflow_run']['head_sha'] + prs_for_sha = requests.get(f"https://api.github.com/repos/{GITHUB_REPOSITORY}/commits/{self.sha}/pulls").json() + if len(prs_for_sha) != 0: + github_event['pull_request'] = prs_for_sha[0] + if 'pull_request' in github_event: # pull request and other similar events self.number = github_event['number'] if 'after' in github_event: From a7752eb14488591861cef7e0a7bda910754684bf Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 24 Dec 2021 21:26:06 +0300 Subject: [PATCH 0340/1260] Better jepsen workflow --- .github/workflows/jepsen.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/jepsen.yml b/.github/workflows/jepsen.yml index eb079d50d1d..1b01b4d5074 100644 --- a/.github/workflows/jepsen.yml +++ b/.github/workflows/jepsen.yml @@ -27,6 +27,8 @@ jobs: sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 + with: + fetch-depth: 0 - name: Jepsen Test run: | sudo rm -fr $TEMP_PATH From 9687f1603a7828f03dab3e59485a7eed54dbf4cc Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 24 Dec 2021 21:34:24 +0300 Subject: [PATCH 0341/1260] Better --- tests/ci/pr_info.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index d72c8324b02..812834824b7 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -44,14 +44,14 @@ class PRInfo: self.changed_files = set([]) # workflow completed event, used for PRs only - if github_event['action'] == 'completed': + if 'action' in github_event and github_event['action'] == 'completed': self.sha = github_event['workflow_run']['head_sha'] prs_for_sha = requests.get(f"https://api.github.com/repos/{GITHUB_REPOSITORY}/commits/{self.sha}/pulls").json() if len(prs_for_sha) != 0: github_event['pull_request'] = prs_for_sha[0] if 'pull_request' in github_event: # pull request and other similar events - self.number = github_event['number'] + self.number = github_event['pull_request']['number'] if 'after' in github_event: self.sha = github_event['after'] else: From b4420cfa3e834ab9026914978697ded3180122a0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Dec 2021 15:40:54 +0300 Subject: [PATCH 0342/1260] Fix --- src/IO/ReadBufferFromAzureBlobStorage.cpp | 4 ++-- src/IO/WriteBufferFromAzureBlobStorage.cpp | 20 ++++++++++++++++++++ src/IO/WriteBufferFromAzureBlobStorage.h | 2 ++ 3 files changed, 24 insertions(+), 2 deletions(-) diff --git a/src/IO/ReadBufferFromAzureBlobStorage.cpp b/src/IO/ReadBufferFromAzureBlobStorage.cpp index d4762ecc044..4e1efefd83e 100644 --- a/src/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/IO/ReadBufferFromAzureBlobStorage.cpp @@ -149,9 +149,9 @@ void ReadBufferFromAzureBlobStorage::initialize() data_stream = std::move(download_response.Value.BodyStream); break; } - catch (const Azure::Storage::StorageException & e) + catch (const Azure::Core::RequestFailedException & e) { - LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {} : {}", path, offset, i, e.Message); + LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {} : {}", path, offset, i + 1, e.Message); if (i + 1 == max_single_download_retries) throw e; diff --git a/src/IO/WriteBufferFromAzureBlobStorage.cpp b/src/IO/WriteBufferFromAzureBlobStorage.cpp index 3e79b4c0660..c80d577c946 100644 --- a/src/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/IO/WriteBufferFromAzureBlobStorage.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -28,6 +29,25 @@ WriteBufferFromAzureBlobStorage::~WriteBufferFromAzureBlobStorage() finalize(); } +void WriteBufferFromAzureBlobStorage::finalizeImpl() +{ + const size_t max_tries = 3; + for (size_t i = 0; i < max_tries; ++i) + { + try + { + next(); + break; + } + catch (const Azure::Core::RequestFailedException & e) + { + if (i == max_tries - 1) + throw; + LOG_INFO(&Poco::Logger::get("WriteBufferFromAzureBlobStorage"), + "Exception caught during finalizing azure storage write at attempt {} : {}", i + 1, e.Message); + } + } +} void WriteBufferFromAzureBlobStorage::nextImpl() { diff --git a/src/IO/WriteBufferFromAzureBlobStorage.h b/src/IO/WriteBufferFromAzureBlobStorage.h index 133803aad27..cbbfb577a91 100644 --- a/src/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/IO/WriteBufferFromAzureBlobStorage.h @@ -32,6 +32,8 @@ public: void nextImpl() override; private: + void finalizeImpl() override; + std::shared_ptr blob_container_client; size_t max_single_part_upload_size; const String blob_path; From 6651bc7dc84b365cca475ef2b4bae8d6b9835bfa Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 24 Dec 2021 23:35:09 +0300 Subject: [PATCH 0343/1260] Fix shared hermetic builds on Arch linux On Arch linux clang uses PIE by default, and so it requires Scrt1.o: $ /usr/bin/clang++ --target=x86_64-linux-gnu --sysroot=/src/ch/clickhouse/cmake/linux/../../contrib/sysroot/linux-x86_64/x86_64-linux-gnu/libc --gcc-toolchain=/src/ch/clickhouse/cmake/linux/../../contrib/sysroot/linux-x86_64 --gcc-toolchain=/src/ch/clickhouse/cmake/linux/../../contrib/sysroot/linux-x86_64 -std=c++20 ... -o base/base/tests/dump_variable ... clang version 13.0.0 Target: x86_64-unknown-linux-gnu Thread model: posix InstalledDir: /usr/bin Found candidate GCC installation: /src/ch/clickhouse/cmake/linux/../../contrib/sysroot/linux-x86_64/lib/gcc/x86_64-linux-gnu/9 Selected GCC installation: /src/ch/clickhouse/cmake/linux/../../contrib/sysroot/linux-x86_64/lib/gcc/x86_64-linux-gnu/9 Candidate multilib: .;@m64 Selected multilib: .;@m64 "/usr/bin/ld.lld" --sysroot=/src/ch/clickhouse/cmake/linux/../../contrib/sysroot/linux-x86_64/x86_64-linux-gnu/libc -pie -export-dynamic --eh-frame-hdr -m elf_x86_64 -export-dynamic -dynamic-linker /lib64/ld-linux-x86-64.so.2 -o base/base/tests/dump_variable Scrt1.o ... ^^^^^^^ --- CMakeLists.txt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index 215e811a80c..fdc9cfcd303 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -456,6 +456,11 @@ if (MAKE_STATIC_LIBRARIES) endif () else () set (CMAKE_POSITION_INDEPENDENT_CODE ON) + # This is required for clang on Arch linux, that uses PIE by default. + # See enable-SSP-and-PIE-by-default.patch [1]. + # + # [1]: https://github.com/archlinux/svntogit-packages/blob/6e681aa860e65ad46a1387081482eb875c2200f2/trunk/enable-SSP-and-PIE-by-default.patch + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -no-pie") endif () if (ENABLE_TESTS) From 83074c06bf69b00d320c74f1ebf17e45ce6ca887 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 24 Dec 2021 23:45:47 +0300 Subject: [PATCH 0344/1260] Fix ccache with ENABLE_CHECK_HEAVY_BUILDS (ccache 4.0 and 4.1 only affected) Before this patch ccache 4.0/4.1 has not been working correctly with ENABLE_CHECK_HEAVY_BUILDS, this was because of mixing CMAKE_CXX_COMPILER_LAUNCHER/CMAKE_C_COMPILER_LAUNCHER and RULE_LAUNCH_LINK/RULE_LAUNCH_COMPILE. Fixes: #28845 --- cmake/find/ccache.cmake | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/cmake/find/ccache.cmake b/cmake/find/ccache.cmake index 43c2de0c921..95ec3d8a034 100644 --- a/cmake/find/ccache.cmake +++ b/cmake/find/ccache.cmake @@ -32,11 +32,6 @@ if (CCACHE_FOUND AND NOT COMPILER_MATCHES_CCACHE) if (CCACHE_VERSION VERSION_GREATER "3.2.0" OR NOT CMAKE_CXX_COMPILER_ID STREQUAL "Clang") message(STATUS "Using ${CCACHE_FOUND} ${CCACHE_VERSION}") - set (CMAKE_CXX_COMPILER_LAUNCHER ${CCACHE_FOUND} ${CMAKE_CXX_COMPILER_LAUNCHER}) - set (CMAKE_C_COMPILER_LAUNCHER ${CCACHE_FOUND} ${CMAKE_C_COMPILER_LAUNCHER}) - - set_property (GLOBAL PROPERTY RULE_LAUNCH_LINK ${CCACHE_FOUND}) - # debian (debhelpers) set SOURCE_DATE_EPOCH environment variable, that is # filled from the debian/changelog or current time. # @@ -49,11 +44,14 @@ if (CCACHE_FOUND AND NOT COMPILER_MATCHES_CCACHE) # - 4.0+ will ignore SOURCE_DATE_EPOCH environment variable. if (CCACHE_VERSION VERSION_GREATER_EQUAL "4.2") message(STATUS "ccache is 4.2+ no quirks for SOURCE_DATE_EPOCH required") + set(LAUNCHER ${CCACHE_FOUND}) elseif (CCACHE_VERSION VERSION_GREATER_EQUAL "4.0") message(STATUS "Ignore SOURCE_DATE_EPOCH for ccache") - set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE "env -u SOURCE_DATE_EPOCH") - set_property (GLOBAL PROPERTY RULE_LAUNCH_LINK "env -u SOURCE_DATE_EPOCH") + set(LAUNCHER env -u SOURCE_DATE_EPOCH ${CCACHE_FOUND}) endif() + + set (CMAKE_CXX_COMPILER_LAUNCHER ${LAUNCHER} ${CMAKE_CXX_COMPILER_LAUNCHER}) + set (CMAKE_C_COMPILER_LAUNCHER ${LAUNCHER} ${CMAKE_C_COMPILER_LAUNCHER}) else () message(${RECONFIGURE_MESSAGE_LEVEL} "Not using ${CCACHE_FOUND} ${CCACHE_VERSION} bug: https://bugzilla.samba.org/show_bug.cgi?id=8118") endif () From 565a38a2e3c1ea4c5a6d143fb6d5012e1dbd2327 Mon Sep 17 00:00:00 2001 From: Yatian Xu Date: Fri, 24 Dec 2021 17:49:59 -0800 Subject: [PATCH 0345/1260] fix incorrect metric: StorageBufferBytes --- src/Storages/StorageBuffer.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index f5526781f41..87a8ea2315d 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -455,10 +455,8 @@ static void appendBlock(const Block & from, Block & to) size_t rows = from.rows(); size_t bytes = from.bytes(); - CurrentMetrics::add(CurrentMetrics::StorageBufferRows, rows); - CurrentMetrics::add(CurrentMetrics::StorageBufferBytes, bytes); - size_t old_rows = to.rows(); + size_t old_bytes = to.bytes(); MutableColumnPtr last_col; try @@ -468,6 +466,8 @@ static void appendBlock(const Block & from, Block & to) if (to.rows() == 0) { to = from; + CurrentMetrics::add(CurrentMetrics::StorageBufferRows, rows); + CurrentMetrics::add(CurrentMetrics::StorageBufferBytes, bytes); } else { @@ -480,6 +480,8 @@ static void appendBlock(const Block & from, Block & to) to.getByPosition(column_no).column = std::move(last_col); } + CurrentMetrics::add(CurrentMetrics::StorageBufferRows, rows); + CurrentMetrics::add(CurrentMetrics::StorageBufferBytes, to.bytes() - old_bytes); } } catch (...) From f2075fac6482b19aee101653b833f7c0de413a10 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 25 Dec 2021 05:55:49 +0300 Subject: [PATCH 0346/1260] Update ReadBufferFromAzureBlobStorage.cpp --- src/IO/ReadBufferFromAzureBlobStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadBufferFromAzureBlobStorage.cpp b/src/IO/ReadBufferFromAzureBlobStorage.cpp index 4e1efefd83e..258ad532315 100644 --- a/src/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/IO/ReadBufferFromAzureBlobStorage.cpp @@ -81,7 +81,7 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() } catch (const Azure::Storage::StorageException & e) { - LOG_INFO(log, "Exception caught during Azure Read for file {} at attempt {} : {}", path, i, e.Message); + LOG_INFO(log, "Exception caught during Azure Read for file {} at attempt {}: {}", path, i, e.Message); if (i + 1 == max_single_read_retries) throw e; From bcd4df789df52eb85025bcb966ab282ca6efc428 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 25 Dec 2021 05:56:15 +0300 Subject: [PATCH 0347/1260] Update WriteBufferFromAzureBlobStorage.cpp --- src/IO/WriteBufferFromAzureBlobStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromAzureBlobStorage.cpp b/src/IO/WriteBufferFromAzureBlobStorage.cpp index c80d577c946..88882fcef65 100644 --- a/src/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/IO/WriteBufferFromAzureBlobStorage.cpp @@ -44,7 +44,7 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() if (i == max_tries - 1) throw; LOG_INFO(&Poco::Logger::get("WriteBufferFromAzureBlobStorage"), - "Exception caught during finalizing azure storage write at attempt {} : {}", i + 1, e.Message); + "Exception caught during finalizing azure storage write at attempt {}: {}", i + 1, e.Message); } } } From d15da636f670e576b597b51ddcc66516e48073c1 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 25 Dec 2021 05:57:15 +0300 Subject: [PATCH 0348/1260] Update ReadBufferFromAzureBlobStorage.cpp --- src/IO/ReadBufferFromAzureBlobStorage.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/ReadBufferFromAzureBlobStorage.cpp b/src/IO/ReadBufferFromAzureBlobStorage.cpp index 258ad532315..0ce6db97437 100644 --- a/src/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/IO/ReadBufferFromAzureBlobStorage.cpp @@ -83,7 +83,7 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() { LOG_INFO(log, "Exception caught during Azure Read for file {} at attempt {}: {}", path, i, e.Message); if (i + 1 == max_single_read_retries) - throw e; + throw; sleepForMilliseconds(sleep_time_with_backoff_milliseconds); sleep_time_with_backoff_milliseconds *= 2; @@ -153,7 +153,7 @@ void ReadBufferFromAzureBlobStorage::initialize() { LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {} : {}", path, offset, i + 1, e.Message); if (i + 1 == max_single_download_retries) - throw e; + throw; sleepForMilliseconds(sleep_time_with_backoff_milliseconds); sleep_time_with_backoff_milliseconds *= 2; From f03cb4c762648c14a4e8b7991449b6f18159827d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Dec 2021 06:10:59 +0300 Subject: [PATCH 0349/1260] Cleanup trash from Kafka and HDFS --- src/Coordination/Changelog.cpp | 2 +- .../AzureBlobStorage/DiskAzureBlobStorage.cpp | 4 ++-- src/IO/ReadBufferFromAzureBlobStorage.cpp | 4 ++-- src/Storages/HDFS/StorageHDFS.cpp | 15 ++++++++------- src/Storages/HDFS/StorageHDFS.h | 6 +++++- src/Storages/Kafka/WriteBufferToKafkaProducer.cpp | 4 ++-- 6 files changed, 20 insertions(+), 15 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 2c730ee16ed..74e093284a8 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -252,7 +252,7 @@ public: catch (const Exception & ex) { if (ex.code() == ErrorCodes::UNKNOWN_FORMAT_VERSION) - throw ex; + throw; result.error = true; LOG_WARNING(log, "Cannot completely read changelog on path {}, error: {}", filepath, ex.message()); diff --git a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp index 0b65f09338c..e2ee6ee0153 100644 --- a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp +++ b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp @@ -160,10 +160,10 @@ void DiskAzureBlobStorage::removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_kee if (!delete_info.Value.Deleted) throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Failed to delete file in AzureBlob Storage: {}", path); } - catch (const Azure::Storage::StorageException& e) + catch (const Azure::Storage::StorageException & e) { LOG_INFO(log, "Caught an error while deleting file {} : {}", path, e.Message); - throw e; + throw; } } } diff --git a/src/IO/ReadBufferFromAzureBlobStorage.cpp b/src/IO/ReadBufferFromAzureBlobStorage.cpp index d4762ecc044..ae09aad8b63 100644 --- a/src/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/IO/ReadBufferFromAzureBlobStorage.cpp @@ -83,7 +83,7 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() { LOG_INFO(log, "Exception caught during Azure Read for file {} at attempt {} : {}", path, i, e.Message); if (i + 1 == max_single_read_retries) - throw e; + throw; sleepForMilliseconds(sleep_time_with_backoff_milliseconds); sleep_time_with_backoff_milliseconds *= 2; @@ -153,7 +153,7 @@ void ReadBufferFromAzureBlobStorage::initialize() { LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {} : {}", path, offset, i, e.Message); if (i + 1 == max_single_download_retries) - throw e; + throw; sleepForMilliseconds(sleep_time_with_backoff_milliseconds); sleep_time_with_backoff_milliseconds *= 2; diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index ce9fc5c8129..306c010d2cd 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -53,7 +53,8 @@ namespace ErrorCodes extern const int ACCESS_DENIED; } -Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match); +static Strings listFilesWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match); + StorageHDFS::StorageHDFS( const String & uri_, @@ -95,10 +96,10 @@ public: HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context_->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); - std::lock_guard lock(mutex); - uris = LSWithRegexpMatching("/", fs, path_from_uri); - for (size_t i=0; ipoll(timeout); continue; } - throw e; + throw; } break; @@ -126,7 +126,7 @@ void WriteBufferToKafkaProducer::flush() { if (e.get_error() == RD_KAFKA_RESP_ERR__TIMED_OUT) continue; - throw e; + throw; } break; From 555ae50c4e9fc0524383ce866207e48ff7ef5217 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Dec 2021 10:11:31 +0300 Subject: [PATCH 0350/1260] Fix Date32 test --- .../0_stateless/02098_date32_comparison.sql | 38 +++++++++---------- 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/tests/queries/0_stateless/02098_date32_comparison.sql b/tests/queries/0_stateless/02098_date32_comparison.sql index 5fd7172e0bb..b35191e58ed 100644 --- a/tests/queries/0_stateless/02098_date32_comparison.sql +++ b/tests/queries/0_stateless/02098_date32_comparison.sql @@ -1,19 +1,19 @@ -select toDate32('1990-01-01') = toDate('1990-01-01'); -select toDate('1991-01-02') > toDate32('1990-01-01'); -select toDate32('1925-01-01') <= toDate('1990-01-01'); -select toDate('1991-01-01') < toDate32('2283-11-11'); -select toDate32('1990-01-01') = toDateTime('1990-01-01'); -select toDateTime('1991-01-02') > toDate32('1990-01-01'); -select toDate32('1925-01-01') <= toDateTime('1990-01-01'); -select toDateTime('1991-01-01') < toDate32('2283-11-11'); -select toDate32('1990-01-01') = toDateTime64('1990-01-01',2); -select toDateTime64('1991-01-02',2) > toDate32('1990-01-01'); -select toDate32('1925-01-01') = toDateTime64('1925-01-01',2); -select toDateTime64('1925-01-02',2) > toDate32('1925-01-01'); -select toDate32('2283-11-11') = toDateTime64('2283-11-11',2); -select toDateTime64('2283-11-11',2) > toDate32('1925-01-01'); -select toDate32('1990-01-01') = '1990-01-01'; -select '1991-01-02' > toDate32('1990-01-01'); -select toDate32('1925-01-01') = '1925-01-01'; -select '2283-11-11' >= toDate32('2283-11-10'); -select '2283-11-11' > toDate32('1925-01-01'); \ No newline at end of file +select toDate32('1990-02-01') = toDate('1990-02-01'); +select toDate('1991-01-02') > toDate32('1990-02-01'); +select toDate32('1925-02-01') <= toDate('1990-02-01'); +select toDate('1991-02-01') < toDate32('2283-11-11'); +select toDate32('1990-02-01') = toDateTime('1990-02-01'); +select toDateTime('1991-01-02') > toDate32('1990-02-01'); +select toDate32('1925-02-01') <= toDateTime('1990-02-01'); +select toDateTime('1991-02-01') < toDate32('2283-11-11'); +select toDate32('1990-02-01') = toDateTime64('1990-02-01',2); +select toDateTime64('1991-01-02',2) > toDate32('1990-02-01'); +select toDate32('1925-02-01') = toDateTime64('1925-02-01',2); +select toDateTime64('1925-02-02',2) > toDate32('1925-02-01'); +select toDate32('2283-11-11') = toDateTime64('2283-11-11',2); +select toDateTime64('2283-11-11',2) > toDate32('1925-02-01'); +select toDate32('1990-02-01') = '1990-02-01'; +select '1991-01-02' > toDate32('1990-02-01'); +select toDate32('1925-02-01') = '1925-02-01'; +select '2283-11-11' >= toDate32('2283-11-10'); +select '2283-11-11' > toDate32('1925-02-01'); From a88b2975c0589e24b1fa705de2d0818cf3ddad5d Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 25 Dec 2021 15:45:50 +0800 Subject: [PATCH 0351/1260] inject into system.build_options --- programs/server/Server.cpp | 13 --------- src/CMakeLists.txt | 28 ------------------ src/Client/ClientBase.cpp | 12 -------- src/Common/git_info.h.in | 19 ------------ src/Storages/System/CMakeLists.txt | 29 +++++++++++++++++++ ...StorageSystemBuildOptions.generated.cpp.in | 4 +++ 6 files changed, 33 insertions(+), 72 deletions(-) delete mode 100644 src/Common/git_info.h.in diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 3943071b8d1..14075f9fbf2 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -84,7 +84,6 @@ #include "config_core.h" #include "Common/config_version.h" -#include "Common/git_info.h" #if defined(OS_LINUX) # include @@ -128,14 +127,6 @@ namespace CurrentMetrics extern const Metric MaxPushedDDLEntryID; } -namespace GitInfo -{ - extern const std::string GIT_SHA1; - extern const std::string GIT_BRANCH; - extern const std::string GIT_DATE; - extern const std::string GIT_COMMIT_SUBJECT; -} - namespace fs = std::filesystem; #if USE_JEMALLOC @@ -405,10 +396,6 @@ int Server::run() if (config().hasOption("version")) { std::cout << DBMS_NAME << " server version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl; - std::cout << "GIT SHA1: " << GitInfo::GIT_SHA1 << std::endl; - std::cout << "GIT BRANCH: " << GitInfo::GIT_BRANCH << std::endl; - std::cout << "GIT DATE: " << GitInfo::GIT_DATE << std::endl; - std::cout << "GIT COMMIT SUBJECT: " << GitInfo::GIT_COMMIT_SUBJECT << std::endl; return 0; } return Application::run(); // NOLINT diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 11932d75fdb..7124961821e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -20,7 +20,6 @@ include(../cmake/limit_jobs.cmake) set (CONFIG_VERSION "${CMAKE_CURRENT_BINARY_DIR}/Common/config_version.h") set (CONFIG_COMMON "${CMAKE_CURRENT_BINARY_DIR}/Common/config.h") -set (GIT_INFO "${CMAKE_CURRENT_BINARY_DIR}/Common/git_info.h") include (../cmake/version.cmake) message (STATUS "Will build ${VERSION_FULL} revision ${VERSION_REVISION} ${VERSION_OFFICIAL}") @@ -28,33 +27,6 @@ configure_file (Common/config.h.in ${CONFIG_COMMON}) configure_file (Common/config_version.h.in ${CONFIG_VERSION}) configure_file (Core/config_core.h.in "${CMAKE_CURRENT_BINARY_DIR}/Core/include/config_core.h") -set(GIT_EXECUTABLE /usr/bin/git) -# The commit's SHA1, and whether the building workspace was dirty or not -execute_process(COMMAND - "${GIT_EXECUTABLE}" rev-parse HEAD - WORKING_DIRECTORY "${CMAKE_SOURCE_DIR}" - OUTPUT_VARIABLE GIT_SHA1 - ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) -# Git branch name -execute_process(COMMAND - "${GIT_EXECUTABLE}" rev-parse --abbrev-ref HEAD - WORKING_DIRECTORY "${CMAKE_SOURCE_DIR}" - OUTPUT_VARIABLE GIT_BRANCH - ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) -# The date of the commit -execute_process(COMMAND - "${GIT_EXECUTABLE}" log -1 --format=%ad --date=local - WORKING_DIRECTORY "${CMAKE_SOURCE_DIR}" - OUTPUT_VARIABLE GIT_DATE - ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) -# The subject of the commit -execute_process(COMMAND - "${GIT_EXECUTABLE}" log -1 --format=%s - WORKING_DIRECTORY "${CMAKE_SOURCE_DIR}" - OUTPUT_VARIABLE GIT_COMMIT_SUBJECT - ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) -configure_file (Common/git_info.h.in ${GIT_INFO}) - if (USE_DEBUG_HELPERS) get_target_property(MAGIC_ENUM_INCLUDE_DIR magic_enum INTERFACE_INCLUDE_DIRECTORIES) set (INCLUDE_DEBUG_HELPERS "-I\"${ClickHouse_SOURCE_DIR}/base\" -I\"${MAGIC_ENUM_INCLUDE_DIR}\" -include \"${ClickHouse_SOURCE_DIR}/src/Core/iostream_debug_helpers.h\"") diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 4ac31ca65bb..8fd95bb706c 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -23,7 +23,6 @@ #include "Formats/FormatFactory.h" #include -#include #include #include #include @@ -108,13 +107,6 @@ namespace ProfileEvents namespace DB { -namespace GitInfo -{ - extern const std::string GIT_SHA1; - extern const std::string GIT_BRANCH; - extern const std::string GIT_DATE; - extern const std::string GIT_COMMIT_SUBJECT; -} static void incrementProfileEventsBlock(Block & dst, const Block & src) { @@ -1649,10 +1641,6 @@ void ClientBase::clearTerminal() void ClientBase::showClientVersion() { std::cout << DBMS_NAME << " " + getName() + " version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl; - std::cout << "GIT SHA1: " << GitInfo::GIT_SHA1 << std::endl; - std::cout << "GIT BRANCH: " << GitInfo::GIT_BRANCH << std::endl; - std::cout << "GIT DATE: " << GitInfo::GIT_DATE << std::endl; - std::cout << "GIT COMMIT SUBJECT: " << GitInfo::GIT_COMMIT_SUBJECT << std::endl; } diff --git a/src/Common/git_info.h.in b/src/Common/git_info.h.in deleted file mode 100644 index d338ae2e1e5..00000000000 --- a/src/Common/git_info.h.in +++ /dev/null @@ -1,19 +0,0 @@ -#pragma once - -// .h autogenerated by cmake! - -#include - - -namespace DB -{ - -namespace GitInfo -{ - const std::string GIT_SHA1 = "@GIT_SHA1@"; - const std::string GIT_BRANCH = "@GIT_BRANCH@"; - const std::string GIT_DATE = "@GIT_DATE@"; - const std::string GIT_COMMIT_SUBJECT = "@GIT_COMMIT_SUBJECT@"; -} - -} diff --git a/src/Storages/System/CMakeLists.txt b/src/Storages/System/CMakeLists.txt index 96c05a59173..6689deac84b 100644 --- a/src/Storages/System/CMakeLists.txt +++ b/src/Storages/System/CMakeLists.txt @@ -9,6 +9,35 @@ get_property (BUILD_COMPILE_DEFINITIONS DIRECTORY ${ClickHouse_SOURCE_DIR} PROPE get_property(TZDATA_VERSION GLOBAL PROPERTY TZDATA_VERSION_PROP) + +find_package(Git) +if(Git_FOUND) + # The commit's git hash, and whether the building workspace was dirty or not + execute_process(COMMAND + "${GIT_EXECUTABLE}" rev-parse HEAD + WORKING_DIRECTORY "${ClickHouse_SOURCE_DIR}" + OUTPUT_VARIABLE GIT_HASH + ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) + # Git branch name + execute_process(COMMAND + "${GIT_EXECUTABLE}" rev-parse --abbrev-ref HEAD + WORKING_DIRECTORY "${ClickHouse_SOURCE_DIR}" + OUTPUT_VARIABLE GIT_BRANCH + ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) + # The date of the commit + execute_process(COMMAND + "${GIT_EXECUTABLE}" log -1 --format=%ad --date=local + WORKING_DIRECTORY "${ClickHouse_SOURCE_DIR}" + OUTPUT_VARIABLE GIT_DATE + ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) + # The subject of the commit + execute_process(COMMAND + "${GIT_EXECUTABLE}" log -1 --format=%s + WORKING_DIRECTORY "${ClickHouse_SOURCE_DIR}" + OUTPUT_VARIABLE GIT_COMMIT_SUBJECT + ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) +endif() + configure_file (StorageSystemBuildOptions.generated.cpp.in ${CONFIG_BUILD}) include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") diff --git a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index da563cc245b..8a19d7649aa 100644 --- a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -50,6 +50,10 @@ const char * auto_config_build[] "USE_KRB5", "@USE_KRB5@", "USE_FILELOG", "@USE_FILELOG@", "USE_BZIP2", "@USE_BZIP2@", + "GIT_HASH", "@GIT_HASH@", + "GIT_BRANCH", "@GIT_BRANCH@", + "GIT_DATE", "@GIT_DATE@", + "GIT_COMMIT_SUBJECT", "@GIT_COMMIT_SUBJECT@", nullptr, nullptr }; From 02362a35b4ab1626cb808a0ec546043c51b8544f Mon Sep 17 00:00:00 2001 From: Yatian Xu Date: Sat, 25 Dec 2021 00:02:02 -0800 Subject: [PATCH 0352/1260] add test case for metrics: StorageBufferBytes --- tests/integration/test_system_metrics/test.py | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/tests/integration/test_system_metrics/test.py b/tests/integration/test_system_metrics/test.py index 9e8eac162f6..29361d755e2 100644 --- a/tests/integration/test_system_metrics/test.py +++ b/tests/integration/test_system_metrics/test.py @@ -59,3 +59,25 @@ def test_readonly_metrics(start_cluster): node1.query("ATTACH TABLE test.test_table") assert_eq_with_retry(node1, "SELECT value FROM system.metrics WHERE metric = 'ReadonlyReplica'", "0\n", retry_count=300, sleep_time=1) +def test_metrics_storage_buffer_size(start_cluster): + node1.query(''' + CREATE TABLE test.test_mem_table + ( + `str` LowCardinality(String) + ) + ENGINE = Memory; + + CREATE TABLE test.buffer_table + ( + `str` LowCardinality(String) + ) + ENGINE = Buffer('test', 'test_table', 1, 5, 5, 1000, 100000, 100000, 10000000); + ''') + node1.query("INSERT INTO test.buffer_table VALUES('hello');") + node1.query("INSERT INTO test.buffer_table VALUES('hello');") + #before flush + assert node1.query("SELECT value FROM system.metrics WHERE metric = 'StorageBufferRows'") == "2\n" + time.sleep(6) + #after flush + assert node1.query("SELECT value FROM system.metrics WHERE metric = 'StorageBufferRows'") == "0\n" + assert node1.query("SELECT value FROM system.metrics WHERE metric = 'StorageBufferBytes'") == "0\n" From 2e151a278dbd1dcf15c27c32b2a58777b9b8c603 Mon Sep 17 00:00:00 2001 From: Yatian Xu Date: Sat, 25 Dec 2021 00:03:23 -0800 Subject: [PATCH 0353/1260] correct table name --- tests/integration/test_system_metrics/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_system_metrics/test.py b/tests/integration/test_system_metrics/test.py index 29361d755e2..ce6eb68109c 100644 --- a/tests/integration/test_system_metrics/test.py +++ b/tests/integration/test_system_metrics/test.py @@ -71,7 +71,7 @@ def test_metrics_storage_buffer_size(start_cluster): ( `str` LowCardinality(String) ) - ENGINE = Buffer('test', 'test_table', 1, 5, 5, 1000, 100000, 100000, 10000000); + ENGINE = Buffer('test', 'test_mem_table', 1, 5, 5, 1000, 100000, 100000, 10000000); ''') node1.query("INSERT INTO test.buffer_table VALUES('hello');") node1.query("INSERT INTO test.buffer_table VALUES('hello');") From 3fa78c564274026eb7a0b41b2b70aed80da90d8e Mon Sep 17 00:00:00 2001 From: Yatian Xu Date: Sat, 25 Dec 2021 00:43:45 -0800 Subject: [PATCH 0354/1260] test case for metrics: StorageBufferBytes --- tests/integration/test_system_metrics/test.py | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_system_metrics/test.py b/tests/integration/test_system_metrics/test.py index ce6eb68109c..02d9a676efb 100644 --- a/tests/integration/test_system_metrics/test.py +++ b/tests/integration/test_system_metrics/test.py @@ -71,13 +71,19 @@ def test_metrics_storage_buffer_size(start_cluster): ( `str` LowCardinality(String) ) - ENGINE = Buffer('test', 'test_mem_table', 1, 5, 5, 1000, 100000, 100000, 10000000); + ENGINE = Buffer('test', 'test_mem_table', 1, 600, 600, 1000, 100000, 100000, 10000000); ''') - node1.query("INSERT INTO test.buffer_table VALUES('hello');") - node1.query("INSERT INTO test.buffer_table VALUES('hello');") + #before flush + node1.query("INSERT INTO test.buffer_table VALUES('hello');") + assert node1.query("SELECT value FROM system.metrics WHERE metric = 'StorageBufferRows'") == "1\n" + assert node1.query("SELECT value FROM system.metrics WHERE metric = 'StorageBufferBytes'") == "24\n" + + node1.query("INSERT INTO test.buffer_table VALUES('hello');") assert node1.query("SELECT value FROM system.metrics WHERE metric = 'StorageBufferRows'") == "2\n" - time.sleep(6) - #after flush + assert node1.query("SELECT value FROM system.metrics WHERE metric = 'StorageBufferBytes'") == "25\n" + + #flush + node1.query("OPTIMIZE TABLE test.buffer_table") assert node1.query("SELECT value FROM system.metrics WHERE metric = 'StorageBufferRows'") == "0\n" assert node1.query("SELECT value FROM system.metrics WHERE metric = 'StorageBufferBytes'") == "0\n" From da211909e91b3bdcef096626ddcec39267efc43b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 25 Dec 2021 11:52:39 +0300 Subject: [PATCH 0355/1260] tests/integration: refactor test_async_drain_connection --- .../test_async_drain_connection/test.py | 33 +++++++++---------- 1 file changed, 15 insertions(+), 18 deletions(-) diff --git a/tests/integration/test_async_drain_connection/test.py b/tests/integration/test_async_drain_connection/test.py index 21f9b142e7a..5eaac2b28b7 100644 --- a/tests/integration/test_async_drain_connection/test.py +++ b/tests/integration/test_async_drain_connection/test.py @@ -1,21 +1,21 @@ -import os -import sys -import time -from multiprocessing.dummy import Pool +# pylint: disable=redefined-outer-name +# pylint: disable=unused-argument + import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node", main_configs=["configs/config.xml"]) +node = cluster.add_instance('node', main_configs=['configs/config.xml']) -@pytest.fixture(scope="module") +@pytest.fixture(scope='module') def started_cluster(): try: cluster.start() - node.query( - 'create table t (number UInt64) engine = Distributed(test_cluster_two_shards, system, numbers);' - ) + node.query(""" + create table t (number UInt64) + engine = Distributed(test_cluster_two_shards, system, numbers) + """) yield cluster finally: @@ -23,14 +23,11 @@ def started_cluster(): def test_filled_async_drain_connection_pool(started_cluster): - busy_pool = Pool(10) - - def execute_query(i): + def execute_queries(_): for _ in range(100): - node.query('select * from t where number = 0 limit 2;', - settings={ - "sleep_in_receive_cancel_ms": 10000000, - "max_execution_time": 5 - }) + node.query('select * from t where number = 0 limit 2', settings={ + 'sleep_in_receive_cancel_ms': int(10e6), + 'max_execution_time': 5, + }) - p = busy_pool.map(execute_query, range(10)) + any(map(execute_queries, range(10))) From 42ab02a81034d7c88a9a4426b470d7422857d4c5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 25 Dec 2021 11:52:57 +0300 Subject: [PATCH 0356/1260] Fix test_async_drain_connection flakiness Sometimes [1] 5 seconds is not enough, since drain_timeout is 3 seconds, and 2 seconds sometimes is not enough to do other things, especially under sanitizers: E Code: 159. DB::Exception: Received from 172.16.1.2:9000. DB::Exception: Timeout exceeded: elapsed 5.019254094 seconds, maximum: 5. Stack trace: [1]: https://s3.amazonaws.com/clickhouse-test-reports/0/826f7cb0f53e20e67ef52800cb735bb88a6de658/integration_tests__thread__actions__[4/4].html --- tests/integration/test_async_drain_connection/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_async_drain_connection/test.py b/tests/integration/test_async_drain_connection/test.py index 5eaac2b28b7..40d78ebbe7c 100644 --- a/tests/integration/test_async_drain_connection/test.py +++ b/tests/integration/test_async_drain_connection/test.py @@ -28,6 +28,9 @@ def test_filled_async_drain_connection_pool(started_cluster): node.query('select * from t where number = 0 limit 2', settings={ 'sleep_in_receive_cancel_ms': int(10e6), 'max_execution_time': 5, + # decrease drain_timeout to make test more stable + # (another way is to increase max_execution_time, but this will make test slower) + 'drain_timeout': 1, }) any(map(execute_queries, range(10))) From 19038b0afd2b638bf8bfec9dac9a617abac06a07 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 25 Dec 2021 12:18:17 +0300 Subject: [PATCH 0357/1260] Update docs/en/operations/settings/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index a3f326d2e50..107d367bf43 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -823,7 +823,7 @@ Default value: `163840`. ## merge_tree_min_rows_for_concurrent_read_for_remote_filesystem {#merge-tree-min-rows-for-concurrent-read-for-remote-filesystem} -If at least as many lines are read from one file, the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) engine can reads parallelized, when reading from remote filesystem. +The minimum number of lines to read from one file before [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) engine can parallelize reading, when reading from remote filesystem. Possible values: From c43443287860f51917a212e3b8a34442479fe3e0 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 25 Dec 2021 12:18:22 +0300 Subject: [PATCH 0358/1260] Update docs/en/operations/settings/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 107d367bf43..1e08cb727a9 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -843,7 +843,7 @@ Default value: `251658240`. ## merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem {#merge-tree-min-bytes-for-concurrent-read-for-remote-filesystem} -If at least as many bytes are read from one file, the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) engine can reads parallelized, when reading from remote filesystem. +The minimum number of bytes to read from one file before [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) engine can parallelize reading, when reading from remote filesystem. Possible values: From 59dd5ba063af7118001c8b687c0849b6548ab66b Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Dec 2021 10:20:31 +0300 Subject: [PATCH 0359/1260] Validate config keys for external dictionaries --- .../ExternalDataSourceConfiguration.cpp | 28 +++++++++++++++++-- .../test_dictionaries_postgresql/test.py | 23 +++++++++++++++ 2 files changed, 48 insertions(+), 3 deletions(-) diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 42b3b148551..13e6f292602 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -24,6 +24,12 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +static const std::unordered_set dictionary_allowed_keys = { + "host", "port", "user", "password", "db", + "database", "table", "schema", "replica", + "update_field", "update_tag", "invalidate_query", "query", + "where", "name", "secure", "uri"}; + String ExternalDataSourceConfiguration::toString() const { WriteBufferFromOwnString configuration_info; @@ -159,10 +165,22 @@ std::optional getExternalDataSourceConfiguration(const return std::nullopt; } +static void validateConfigKeys( + const Poco::Util::AbstractConfiguration & dict_config, const String & config_prefix, const std::unordered_set & allowed_keys) +{ + Poco::Util::AbstractConfiguration::Keys config_keys; + dict_config.keys(config_prefix, config_keys); + for (const auto & config_key : config_keys) + { + if (!allowed_keys.contains(config_key)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected key `{}` in dictionary source configuration", config_key); + } +} std::optional getExternalDataSourceConfiguration( const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context) { + validateConfigKeys(dict_config, dict_config_prefix, dictionary_allowed_keys); ExternalDataSourceConfiguration configuration; auto collection_name = dict_config.getString(dict_config_prefix + ".name", ""); @@ -170,6 +188,7 @@ std::optional getExternalDataSourceConfiguratio { const auto & config = context->getConfigRef(); const auto & collection_prefix = fmt::format("named_collections.{}", collection_name); + validateConfigKeys(dict_config, collection_prefix, dictionary_allowed_keys); if (!config.has(collection_prefix)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", collection_name); @@ -178,7 +197,8 @@ std::optional getExternalDataSourceConfiguratio configuration.port = dict_config.getInt(dict_config_prefix + ".port", config.getUInt(collection_prefix + ".port", 0)); configuration.username = dict_config.getString(dict_config_prefix + ".user", config.getString(collection_prefix + ".user", "")); configuration.password = dict_config.getString(dict_config_prefix + ".password", config.getString(collection_prefix + ".password", "")); - configuration.database = dict_config.getString(dict_config_prefix + ".db", config.getString(collection_prefix + ".database", "")); + configuration.database = dict_config.getString(dict_config_prefix + ".db", config.getString(dict_config_prefix + ".database", + config.getString(collection_prefix + ".db", config.getString(collection_prefix + ".database", "")))); configuration.table = dict_config.getString(dict_config_prefix + ".table", config.getString(collection_prefix + ".table", "")); configuration.schema = dict_config.getString(dict_config_prefix + ".schema", config.getString(collection_prefix + ".schema", "")); @@ -196,6 +216,7 @@ std::optional getExternalDataSourceConfiguratio ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority( const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context) { + validateConfigKeys(dict_config, dict_config_prefix, dictionary_allowed_keys); ExternalDataSourceConfiguration common_configuration; auto named_collection = getExternalDataSourceConfiguration(dict_config, dict_config_prefix, context); @@ -209,7 +230,7 @@ ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority( common_configuration.port = dict_config.getUInt(dict_config_prefix + ".port", 0); common_configuration.username = dict_config.getString(dict_config_prefix + ".user", ""); common_configuration.password = dict_config.getString(dict_config_prefix + ".password", ""); - common_configuration.database = dict_config.getString(dict_config_prefix + ".db", ""); + common_configuration.database = dict_config.getString(dict_config_prefix + ".db", dict_config.getString(dict_config_prefix + ".database", "")); common_configuration.table = dict_config.getString(fmt::format("{}.table", dict_config_prefix), ""); common_configuration.schema = dict_config.getString(fmt::format("{}.schema", dict_config_prefix), ""); } @@ -233,8 +254,9 @@ ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority( { ExternalDataSourceConfiguration replica_configuration(common_configuration); String replica_name = dict_config_prefix + "." + config_key; - size_t priority = dict_config.getInt(replica_name + ".priority", 0); + validateConfigKeys(dict_config, replica_name, {"host", "port", "user", "password", "priority"}); + size_t priority = dict_config.getInt(replica_name + ".priority", 0); replica_configuration.host = dict_config.getString(replica_name + ".host", common_configuration.host); replica_configuration.port = dict_config.getUInt(replica_name + ".port", common_configuration.port); replica_configuration.username = dict_config.getString(replica_name + ".user", common_configuration.username); diff --git a/tests/integration/test_dictionaries_postgresql/test.py b/tests/integration/test_dictionaries_postgresql/test.py index 8869e9112d1..b3914ea3a38 100644 --- a/tests/integration/test_dictionaries_postgresql/test.py +++ b/tests/integration/test_dictionaries_postgresql/test.py @@ -369,6 +369,29 @@ def test_predefined_connection_configuration(started_cluster): assert(int(result.strip()) == 99) +def test_bad_configuration(started_cluster): + conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) + cursor = conn.cursor() + + node1.query(''' + DROP DICTIONARY IF EXISTS postgres_dict; + CREATE DICTIONARY postgres_dict (id UInt32, value UInt32) + PRIMARY KEY id + SOURCE(POSTGRESQL( + port 5432 + host 'postgres1' + user 'postgres' + password 'mysecretpassword' + dbbb 'clickhouse' + table 'test_schema.test_table')) + LIFETIME(MIN 1 MAX 2) + LAYOUT(HASHED()); + ''') + + result = node1.query_and_get_error("SELECT dictGetUInt32(postgres_dict, 'value', toUInt64(1))") + assert 'Unexpected key in config: dbbb' in result + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From 2ce3c6c91dbe376e904d317fc37c476e29d2205a Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 25 Dec 2021 14:52:15 +0300 Subject: [PATCH 0360/1260] Update RU translation. --- docs/ru/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 03d0aee4fef..700ee6df887 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -746,7 +746,7 @@ ClickHouse может парсить только базовый формат `Y ## merge_tree_min_rows_for_concurrent_read_for_remote_filesystem {#merge-tree-min-rows-for-concurrent-read-for-remote-filesystem} -Если из одного файла считывается указанное количество строк, движок [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) может выполнять параллельное чтение из удаленной файловой системы. +Минимальное количество строк для чтения из одного файла, прежде чем движок [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) может выполнять параллельное чтение из удаленной файловой системы. Возможные значения: @@ -766,7 +766,7 @@ ClickHouse может парсить только базовый формат `Y ## merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem {#merge-tree-min-bytes-for-concurrent-read-for-remote-filesystem} -Если из одного файла считывается указанное число байтов, движок [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) может выполнять параллельное чтение из удаленной файловой системы. +Минимальное количество байтов для чтения из одного файла, прежде чем движок [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) может выполнять параллельное чтение из удаленной файловой системы. Возможное значение: From fa25a95273892438d9e40bb238d6984acdd322d9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Dec 2021 15:41:07 +0300 Subject: [PATCH 0361/1260] Fix clang-tidy --- .../Serializations/SerializationBool.cpp | 327 +++++++++--------- .../Serializations/SerializationBool.h | 10 - 2 files changed, 168 insertions(+), 169 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp index 1709e3321d6..1efacaaecc5 100644 --- a/src/DataTypes/Serializations/SerializationBool.cpp +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -19,7 +19,13 @@ namespace ErrorCodes extern const int CANNOT_PARSE_BOOL; } -static const ColumnUInt8 * checkAndGetSerializeColumnType(const IColumn & column) +namespace +{ + +constexpr char str_true[5] = "true"; +constexpr char str_false[6] = "false"; + +const ColumnUInt8 * checkAndGetSerializeColumnType(const IColumn & column) { const auto * col = checkAndGetColumn(&column); if (!checkAndGetColumn(&column)) @@ -28,7 +34,7 @@ static const ColumnUInt8 * checkAndGetSerializeColumnType(const IColumn & column return col; } -static ColumnUInt8 * checkAndGetDeserializeColumnType(IColumn & column) +ColumnUInt8 * checkAndGetDeserializeColumnType(IColumn & column) { auto * col = typeid_cast(&column); if (!checkAndGetColumn(&column)) @@ -37,6 +43,166 @@ static ColumnUInt8 * checkAndGetDeserializeColumnType(IColumn & column) return col; } +void serializeCustom(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) +{ + const auto * col = checkAndGetSerializeColumnType(column); + + if (col->getData()[row_num]) + { + writeString(settings.bool_true_representation, ostr); + } + else + { + writeString(settings.bool_false_representation, ostr); + } +} + +void serializeSimple(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) +{ + const auto * col = checkAndGetSerializeColumnType(column); + + if (col->getData()[row_num]) + ostr.write(str_true, sizeof(str_true) - 1); + else + ostr.write(str_false, sizeof(str_false) - 1); +} + +bool tryDeserializeAllVariants(ColumnUInt8 * column, ReadBuffer & istr) +{ + if (checkCharCaseInsensitive('1', istr)) + { + column->insert(true); + } + else if (checkCharCaseInsensitive('0', istr)) + { + column->insert(false); + } + /// 'True' and 'T' + else if (checkCharCaseInsensitive('t', istr)) + { + /// Check if it's just short form `T` or full form `True` + if (checkCharCaseInsensitive('r', istr)) + { + if (!checkStringCaseInsensitive("ue", istr)) + return false; + } + column->insert(true); + } + /// 'False' and 'F' + else if (checkCharCaseInsensitive('f', istr)) + { + /// Check if it's just short form `F` or full form `False` + if (checkCharCaseInsensitive('a', istr)) + { + if (!checkStringCaseInsensitive("lse", istr)) + return false; + } + column->insert(false); + } + /// 'Yes' and 'Y' + else if (checkCharCaseInsensitive('y', istr)) + { + /// Check if it's just short form `Y` or full form `Yes` + if (checkCharCaseInsensitive('e', istr)) + { + if (!checkCharCaseInsensitive('s', istr)) + return false; + } + column->insert(true); + } + /// 'No' and 'N' + else if (checkCharCaseInsensitive('n', istr)) + { + /// Check if it's just short form `N` or full form `No` + checkCharCaseInsensitive('o', istr); + column->insert(false); + } + /// 'On' and 'Off' + else if (checkCharCaseInsensitive('o', istr)) + { + if (checkCharCaseInsensitive('n', istr)) + column->insert(true); + else if (checkStringCaseInsensitive("ff", istr)) + { + column->insert(false); + } + else + return false; + } + /// 'Enable' and 'Enabled' + else if (checkStringCaseInsensitive("enable", istr)) + { + /// Check if it's 'enable' or 'enabled' + checkCharCaseInsensitive('d', istr); + column->insert(true); + } + /// 'Disable' and 'Disabled' + else if (checkStringCaseInsensitive("disable", istr)) + { + /// Check if it's 'disable' or 'disabled' + checkCharCaseInsensitive('d', istr); + column->insert(false); + } + else + { + return false; + } + + return true; +} + +void deserializeImpl( + IColumn & column, ReadBuffer & istr, const FormatSettings & settings, std::function check_end_of_value) +{ + ColumnUInt8 * col = checkAndGetDeserializeColumnType(column); + + PeekableReadBuffer buf(istr); + buf.setCheckpoint(); + if (checkString(settings.bool_true_representation, buf) && check_end_of_value(buf)) + { + col->insert(true); + return; + } + + buf.rollbackToCheckpoint(); + if (checkString(settings.bool_false_representation, buf) && check_end_of_value(buf)) + { + col->insert(false); + buf.dropCheckpoint(); + if (buf.hasUnreadData()) + throw Exception( + ErrorCodes::CANNOT_PARSE_BOOL, + "Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if " + "bool_true_representation or bool_false_representation contains some delimiters of input format"); + return; + } + + buf.rollbackToCheckpoint(); + if (tryDeserializeAllVariants(col, buf) && check_end_of_value(buf)) + { + buf.dropCheckpoint(); + if (buf.hasUnreadData()) + throw Exception( + ErrorCodes::CANNOT_PARSE_BOOL, + "Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if " + "bool_true_representation or bool_false_representation contains some delimiters of input format"); + return; + } + + buf.makeContinuousMemoryFromCheckpointToPos(); + buf.rollbackToCheckpoint(); + throw Exception( + ErrorCodes::CANNOT_PARSE_BOOL, + "Cannot parse boolean value here: '{}', should be '{}' or '{}' controlled by setting bool_true_representation and " + "bool_false_representation or one of " + "True/False/T/F/Y/N/Yes/No/On/Off/Enable/Disable/Enabled/Disabled/1/0", + String(buf.position(), std::min(10lu, buf.available())), + settings.bool_true_representation, settings.bool_false_representation); +} + +} + + SerializationBool::SerializationBool(const SerializationPtr &nested_) : SerializationWrapper(nested_) { @@ -160,166 +326,9 @@ void SerializationBool::deserializeWholeText(IColumn & column, ReadBuffer & istr deserializeImpl(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof(); }); } -void SerializationBool::serializeCustom(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -{ - const auto * col = checkAndGetSerializeColumnType(column); - - if (col->getData()[row_num]) - { - writeString(settings.bool_true_representation, ostr); - } - else - { - writeString(settings.bool_false_representation, ostr); - } -} - void SerializationBool::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { serializeSimple(column, row_num, ostr, settings); } -void SerializationBool::serializeSimple(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const -{ - const auto * col = checkAndGetSerializeColumnType(column); - - if (col->getData()[row_num]) - ostr.write(str_true, sizeof(str_true) - 1); - else - ostr.write(str_false, sizeof(str_false) - 1); -} - -void SerializationBool::deserializeImpl( - IColumn & column, ReadBuffer & istr, const FormatSettings & settings, std::function check_end_of_value) const -{ - ColumnUInt8 * col = checkAndGetDeserializeColumnType(column); - - PeekableReadBuffer buf(istr); - buf.setCheckpoint(); - if (checkString(settings.bool_true_representation, buf) && check_end_of_value(buf)) - { - col->insert(true); - return; - } - - buf.rollbackToCheckpoint(); - if (checkString(settings.bool_false_representation, buf) && check_end_of_value(buf)) - { - col->insert(false); - buf.dropCheckpoint(); - if (buf.hasUnreadData()) - throw Exception( - ErrorCodes::CANNOT_PARSE_BOOL, - "Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if " - "bool_true_representation or bool_false_representation contains some delimiters of input format"); - return; - } - - buf.rollbackToCheckpoint(); - if (tryDeserializeAllVariants(col, buf) && check_end_of_value(buf)) - { - buf.dropCheckpoint(); - if (buf.hasUnreadData()) - throw Exception( - ErrorCodes::CANNOT_PARSE_BOOL, - "Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if " - "bool_true_representation or bool_false_representation contains some delimiters of input format"); - return; - } - - buf.makeContinuousMemoryFromCheckpointToPos(); - buf.rollbackToCheckpoint(); - throw Exception( - ErrorCodes::CANNOT_PARSE_BOOL, - "Cannot parse boolean value here: '{}', should be '{}' or '{}' controlled by setting bool_true_representation and " - "bool_false_representation or one of " - "True/False/T/F/Y/N/Yes/No/On/Off/Enable/Disable/Enabled/Disabled/1/0", - String(buf.position(), std::min(10lu, buf.available())), - settings.bool_true_representation, settings.bool_false_representation); -} - -bool SerializationBool::tryDeserializeAllVariants(ColumnUInt8 * column, ReadBuffer & istr) const -{ - if (checkCharCaseInsensitive('1', istr)) - { - column->insert(true); - } - else if (checkCharCaseInsensitive('0', istr)) - { - column->insert(false); - } - /// 'True' and 'T' - else if (checkCharCaseInsensitive('t', istr)) - { - /// Check if it's just short form `T` or full form `True` - if (checkCharCaseInsensitive('r', istr)) - { - if (!checkStringCaseInsensitive("ue", istr)) - return false; - } - column->insert(true); - } - /// 'False' and 'F' - else if (checkCharCaseInsensitive('f', istr)) - { - /// Check if it's just short form `F` or full form `False` - if (checkCharCaseInsensitive('a', istr)) - { - if (!checkStringCaseInsensitive("lse", istr)) - return false; - } - column->insert(false); - } - /// 'Yes' and 'Y' - else if (checkCharCaseInsensitive('y', istr)) - { - /// Check if it's just short form `Y` or full form `Yes` - if (checkCharCaseInsensitive('e', istr)) - { - if (!checkCharCaseInsensitive('s', istr)) - return false; - } - column->insert(true); - } - /// 'No' and 'N' - else if (checkCharCaseInsensitive('n', istr)) - { - /// Check if it's just short form `N` or full form `No` - checkCharCaseInsensitive('o', istr); - column->insert(false); - } - /// 'On' and 'Off' - else if (checkCharCaseInsensitive('o', istr)) - { - if (checkCharCaseInsensitive('n', istr)) - column->insert(true); - else if (checkStringCaseInsensitive("ff", istr)) - { - column->insert(false); - } - else - return false; - } - /// 'Enable' and 'Enabled' - else if (checkStringCaseInsensitive("enable", istr)) - { - /// Check if it's 'enable' or 'enabled' - checkCharCaseInsensitive('d', istr); - column->insert(true); - } - /// 'Disable' and 'Disabled' - else if (checkStringCaseInsensitive("disable", istr)) - { - /// Check if it's 'disable' or 'disabled' - checkCharCaseInsensitive('d', istr); - column->insert(false); - } - else - { - return false; - } - - return true; -} - } diff --git a/src/DataTypes/Serializations/SerializationBool.h b/src/DataTypes/Serializations/SerializationBool.h index 48c8b679694..a9f4c6404b3 100644 --- a/src/DataTypes/Serializations/SerializationBool.h +++ b/src/DataTypes/Serializations/SerializationBool.h @@ -9,10 +9,6 @@ namespace DB class SerializationBool final : public SerializationWrapper { -private: - static constexpr char str_true[5] = "true"; - static constexpr char str_false[6] = "false"; - public: SerializationBool(const SerializationPtr & nested_); @@ -36,12 +32,6 @@ public: void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; - -protected: - void serializeCustom(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; - void serializeSimple(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; - void deserializeImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, std::function check_end_of_value) const; - bool tryDeserializeAllVariants(ColumnUInt8 * column, ReadBuffer & istr) const; }; } From 465a9bf615e1b233606460f956c09f71931c99a2 Mon Sep 17 00:00:00 2001 From: Alexander Burmak Date: Sat, 25 Dec 2021 15:45:08 +0300 Subject: [PATCH 0362/1260] ClickHouse diagnostics tool --- utils/clickhouse-diagnostics/README.md | 2657 +++++++++++++++++ .../clickhouse-diagnostics | 960 ++++++ utils/clickhouse-diagnostics/requirements.txt | 6 + 3 files changed, 3623 insertions(+) create mode 100644 utils/clickhouse-diagnostics/README.md create mode 100644 utils/clickhouse-diagnostics/clickhouse-diagnostics create mode 100644 utils/clickhouse-diagnostics/requirements.txt diff --git a/utils/clickhouse-diagnostics/README.md b/utils/clickhouse-diagnostics/README.md new file mode 100644 index 00000000000..991efefdf5a --- /dev/null +++ b/utils/clickhouse-diagnostics/README.md @@ -0,0 +1,2657 @@ +## Installation + +``` +python3 -m pip install -r requirements.txt +``` + +## Usage + +``` +./clickhouse-diagnostics +``` + +Example output: + +### Diagnostics data for host clickhouse01.test_net_3697 +Version: **21.11.8.4** +Timestamp: **2021-12-25 15:34:02** +Uptime: **13 minutes and 51 seconds** +#### ClickHouse configuration +**result** +```XML + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + 1 + + 8123 + 9000 + 9004 + 9005 + 9009 + 4096 + 3 + + false + /path/to/ssl_cert_file + /path/to/ssl_key_file + false + /path/to/ssl_ca_cert_file + deflate + medium + -1 + -1 + false + + + + /etc/clickhouse-server/server.crt + /etc/clickhouse-server/server.key + + none + true + true + sslv2,sslv3 + true + + + true + true + sslv2,sslv3,tlsv1,tlsv1_1 + true + + RejectCertificateHandler + + /etc/clickhouse-server/server.crt + /etc/clickhouse-server/server.key + /etc/clickhouse-server/allCAs.pem + + + 100 + 0 + 10000 + 0.9 + 4194304 + 0 + 8589934592 + 5368709120 + 1000 + 134217728 + 10000 + /var/lib/clickhouse/ + /var/lib/clickhouse/tmp/ + /var/lib/clickhouse/user_files/ + + + + users.xml + + + /var/lib/clickhouse/access/ + + + default + + default + true + false + + + + + localhost + 9000 + + + + + + + localhost + 9000 + + + + + localhost + 9000 + + + + + + + 127.0.0.1 + 9000 + + + + + 127.0.0.2 + 9000 + + + + + + true + + 127.0.0.1 + 9000 + + + + true + + 127.0.0.2 + 9000 + + + + + + + localhost + 9440 + 1 + + + + + + + localhost + 9000 + + + + + localhost + 1 + + + + + + + clickhouse01.test_net_3697 + 9000 + + + + + 3600 + 3600 + 60 + + system + query_log
+ toYYYYMM(event_date) + 7500 +
+ + system + trace_log
+ toYYYYMM(event_date) + 7500 +
+ + system + query_thread_log
+ toYYYYMM(event_date) + 7500 +
+ + system + query_views_log
+ toYYYYMM(event_date) + 7500 +
+ + system + part_log
+ toYYYYMM(event_date) + 7500 +
+ + system + metric_log
+ 7500 + 1000 +
+ + system + asynchronous_metric_log
+ 7000 +
+ + engine MergeTree + partition by toYYYYMM(finish_date) + order by (finish_date, finish_time_us, trace_id) + system + opentelemetry_span_log
+ 7500 +
+ + system + crash_log
+ + 1000 +
+ + system + session_log
+ toYYYYMM(event_date) + 7500 +
+ + *_dictionary.xml + *_function.xml + + + /clickhouse/task_queue/ddl + + + + click_cost + any + + 0 + 3600 + + + 86400 + 60 + + + + max + + 0 + 60 + + + 3600 + 300 + + + 86400 + 3600 + + + + /var/lib/clickhouse/format_schemas/ + + + hide encrypt/decrypt arguments + ((?:aes_)?(?:encrypt|decrypt)(?:_mysql)?)\s*\(\s*(?:'(?:\\'|.)+'|.*?)\s*\) + \1(???) + + + + false + false + https://6f33034cfe684dd7a3ab9875e57b1c8d@o388870.ingest.sentry.io/5226277 + + 0.0.0.0 + 8443 + 9440 + + + zookeeper01.test_net_3697 + 2281 + 1 + + 3000 + /clickhouse01 + ***** + + + clickhouse01 + shard1 + + 0 + + + + /hdd1/ + + + /hdd2/ + + + s3 + http://minio01:9000/cloud-storage-01/data/ + bB5vT2M8yaRv9J14SnAP + ***** + true + + + + + +
+ default +
+ + hdd1 + + + hdd2 + +
+ 0.0 +
+ + +
+ s3 +
+ + default + +
+ 0.0 +
+ + +
+ default +
+ + s3 + +
+ 0.0 +
+
+
+``` +#### Access configuration +**query** +```sql +SHOW ACCESS +``` +**result** +``` +CREATE USER default IDENTIFIED WITH plaintext_password SETTINGS PROFILE default +CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = 'random' +CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1 +CREATE QUOTA default KEYED BY user_name FOR INTERVAL 1 hour TRACKING ONLY TO default +GRANT ALL ON *.* TO default WITH GRANT OPTION +``` +#### Quotas +**query** +```sql +SHOW QUOTA +``` +**result** +``` +Row 1: +────── +quota_name: default +quota_key: default +start_time: 2021-12-25 15:00:00 +end_time: 2021-12-25 16:00:00 +duration: 3600 +queries: 49 +max_queries: ᴺᵁᴸᴸ +query_selects: 49 +max_query_selects: ᴺᵁᴸᴸ +query_inserts: 0 +max_query_inserts: ᴺᵁᴸᴸ +errors: 6 +max_errors: ᴺᵁᴸᴸ +result_rows: 607 +max_result_rows: ᴺᵁᴸᴸ +result_bytes: 237632 +max_result_bytes: ᴺᵁᴸᴸ +read_rows: 1256 +max_read_rows: ᴺᵁᴸᴸ +read_bytes: 778936 +max_read_bytes: ᴺᵁᴸᴸ +execution_time: 0 +max_execution_time: ᴺᵁᴸᴸ +``` +#### Schema +##### Database engines +**query** +```sql +SELECT + engine, + count() "count" +FROM system.databases +GROUP BY engine +``` +**result** +``` +┌─engine─┬─count─┐ +│ Memory │ 2 │ +│ Atomic │ 2 │ +└────────┴───────┘ +``` +##### Databases (top 10 by size) +**query** +```sql +SELECT + name, + engine, + tables, + partitions, + parts, + formatReadableSize(bytes_on_disk) "disk_size" +FROM system.databases db +LEFT JOIN +( + SELECT + database, + uniq(table) "tables", + uniq(table, partition) "partitions", + count() AS parts, + sum(bytes_on_disk) "bytes_on_disk" + FROM system.parts + WHERE active + GROUP BY database +) AS db_stats ON db.name = db_stats.database +ORDER BY bytes_on_disk DESC +LIMIT 10 +``` +**result** +``` +┌─name───────────────┬─engine─┬─tables─┬─partitions─┬─parts─┬─disk_size──┐ +│ system │ Atomic │ 6 │ 6 │ 22 │ 716.29 KiB │ +│ INFORMATION_SCHEMA │ Memory │ 0 │ 0 │ 0 │ 0.00 B │ +│ default │ Atomic │ 0 │ 0 │ 0 │ 0.00 B │ +│ information_schema │ Memory │ 0 │ 0 │ 0 │ 0.00 B │ +└────────────────────┴────────┴────────┴────────────┴───────┴────────────┘ +``` +##### Table engines +**query** +```sql +SELECT + engine, + count() "count" +FROM system.tables +WHERE database != 'system' +GROUP BY engine +``` +**result** +``` +┌─engine─┬─count─┐ +│ View │ 8 │ +└────────┴───────┘ +``` +##### Dictionaries +**query** +```sql +SELECT + source, + type, + status, + count() "count" +FROM system.dictionaries +GROUP BY source, type, status +ORDER BY status DESC, source +``` +**result** +``` + +``` +#### Replication +##### Replicated tables (top 10 by absolute delay) +**query** +```sql +SELECT + database, + table, + is_leader, + is_readonly, + absolute_delay, + queue_size, + inserts_in_queue, + merges_in_queue +FROM system.replicas +ORDER BY absolute_delay DESC +LIMIT 10 +``` +**result** +``` + +``` +##### Replication queue (top 20 oldest tasks) +**query** +```sql +SELECT + database, + table, + replica_name, + position, + node_name, + type, + source_replica, + parts_to_merge, + new_part_name, + create_time, + required_quorum, + is_detach, + is_currently_executing, + num_tries, + last_attempt_time, + last_exception, + concat('time: ', toString(last_postpone_time), ', number: ', toString(num_postponed), ', reason: ', postpone_reason) postpone +FROM system.replication_queue +ORDER BY create_time ASC +LIMIT 20 +``` +**result** +``` + +``` +##### Replicated fetches +**query** +```sql +SELECT + database, + table, + round(elapsed, 1) "elapsed", + round(100 * progress, 1) "progress", + partition_id, + result_part_name, + result_part_path, + total_size_bytes_compressed, + bytes_read_compressed, + source_replica_path, + source_replica_hostname, + source_replica_port, + interserver_scheme, + to_detached, + thread_id +FROM system.replicated_fetches +``` +**result** +``` + +``` +#### Top 10 tables by max parts per partition +**query** +```sql +SELECT + database, + table, + count() "partitions", + sum(part_count) "parts", + max(part_count) "max_parts_per_partition" +FROM +( + SELECT + database, + table, + partition, + count() "part_count" + FROM system.parts + WHERE active + GROUP BY database, table, partition +) partitions +GROUP BY database, table +ORDER BY max_parts_per_partition DESC +LIMIT 10 +``` +**result** +``` +┌─database─┬─table───────────────────┬─partitions─┬─parts─┬─max_parts_per_partition─┐ +│ system │ metric_log │ 1 │ 5 │ 5 │ +│ system │ trace_log │ 1 │ 5 │ 5 │ +│ system │ query_thread_log │ 1 │ 3 │ 3 │ +│ system │ query_log │ 1 │ 3 │ 3 │ +│ system │ asynchronous_metric_log │ 1 │ 3 │ 3 │ +│ system │ session_log │ 1 │ 3 │ 3 │ +└──────────┴─────────────────────────┴────────────┴───────┴─────────────────────────┘ +``` +#### Merges in progress +**query** +```sql +SELECT + database, + table, + round(elapsed, 1) "elapsed", + round(100 * progress, 1) "progress", + is_mutation, + partition_id, +result_part_path, + source_part_paths, +num_parts, + formatReadableSize(total_size_bytes_compressed) "total_size_compressed", + formatReadableSize(bytes_read_uncompressed) "read_uncompressed", + formatReadableSize(bytes_written_uncompressed) "written_uncompressed", + columns_written, +formatReadableSize(memory_usage) "memory_usage", + thread_id +FROM system.merges +``` +**result** +``` + +``` +#### Mutations in progress +**query** +```sql +SELECT + database, + table, + mutation_id, + command, + create_time, +parts_to_do_names, +parts_to_do, + is_done, + latest_failed_part, + latest_fail_time, + latest_fail_reason +FROM system.mutations +WHERE NOT is_done +ORDER BY create_time DESC +``` +**result** +``` + +``` +#### Recent data parts (modification time within last 3 minutes) +**query** +```sql +SELECT + database, + table, + engine, + partition_id, + name, +part_type, +active, + level, +disk_name, +path, + marks, + rows, + bytes_on_disk, + data_compressed_bytes, + data_uncompressed_bytes, + marks_bytes, + modification_time, + remove_time, + refcount, + is_frozen, + min_date, + max_date, + min_time, + max_time, + min_block_number, + max_block_number +FROM system.parts +WHERE modification_time > now() - INTERVAL 3 MINUTE +ORDER BY modification_time DESC +``` +**result** +``` +Row 1: +────── +database: system +table: metric_log +engine: MergeTree +partition_id: 202112 +name: 202112_110_110_0 +part_type: Compact +active: 1 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/9a2/9a2fb3b4-8ced-4c0b-9a2f-b3b48ced4c0b/202112_110_110_0/ +marks: 2 +rows: 8 +bytes_on_disk: 21752 +data_compressed_bytes: 11699 +data_uncompressed_bytes: 19952 +marks_bytes: 10032 +modification_time: 2021-12-25 15:33:59 +remove_time: 1970-01-01 03:00:00 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 110 +max_block_number: 110 + +Row 2: +────── +database: system +table: asynchronous_metric_log +engine: MergeTree +partition_id: 202112 +name: 202112_118_118_0 +part_type: Compact +active: 1 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/78e/78e6eec8-3f71-4724-b8e6-eec83f71a724/202112_118_118_0/ +marks: 2 +rows: 4767 +bytes_on_disk: 10856 +data_compressed_bytes: 10656 +data_uncompressed_bytes: 128675 +marks_bytes: 176 +modification_time: 2021-12-25 15:33:58 +remove_time: 1970-01-01 03:00:00 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 118 +max_block_number: 118 + +Row 3: +────── +database: system +table: asynchronous_metric_log +engine: MergeTree +partition_id: 202112 +name: 202112_117_117_0 +part_type: Compact +active: 1 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/78e/78e6eec8-3f71-4724-b8e6-eec83f71a724/202112_117_117_0/ +marks: 2 +rows: 4767 +bytes_on_disk: 11028 +data_compressed_bytes: 10828 +data_uncompressed_bytes: 128675 +marks_bytes: 176 +modification_time: 2021-12-25 15:33:51 +remove_time: 1970-01-01 03:00:00 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 117 +max_block_number: 117 + +Row 4: +────── +database: system +table: metric_log +engine: MergeTree +partition_id: 202112 +name: 202112_109_109_0 +part_type: Compact +active: 1 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/9a2/9a2fb3b4-8ced-4c0b-9a2f-b3b48ced4c0b/202112_109_109_0/ +marks: 2 +rows: 7 +bytes_on_disk: 21802 +data_compressed_bytes: 11749 +data_uncompressed_bytes: 17458 +marks_bytes: 10032 +modification_time: 2021-12-25 15:33:51 +remove_time: 1970-01-01 03:00:00 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 109 +max_block_number: 109 + +Row 5: +────── +database: system +table: trace_log +engine: MergeTree +partition_id: 202112 +name: 202112_53_53_0 +part_type: Compact +active: 1 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/c0b/c0bc3be3-22d7-45a3-80bc-3be322d7b5a3/202112_53_53_0/ +marks: 2 +rows: 6 +bytes_on_disk: 1057 +data_compressed_bytes: 700 +data_uncompressed_bytes: 1894 +marks_bytes: 336 +modification_time: 2021-12-25 15:33:49 +remove_time: 1970-01-01 03:00:00 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 53 +max_block_number: 53 + +Row 6: +────── +database: system +table: asynchronous_metric_log +engine: MergeTree +partition_id: 202112 +name: 202112_116_116_0 +part_type: Compact +active: 0 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/78e/78e6eec8-3f71-4724-b8e6-eec83f71a724/202112_116_116_0/ +marks: 2 +rows: 4767 +bytes_on_disk: 10911 +data_compressed_bytes: 10711 +data_uncompressed_bytes: 128675 +marks_bytes: 176 +modification_time: 2021-12-25 15:33:44 +remove_time: 2021-12-25 15:33:44 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 116 +max_block_number: 116 + +Row 7: +────── +database: system +table: asynchronous_metric_log +engine: MergeTree +partition_id: 202112 +name: 202112_1_116_23 +part_type: Wide +active: 1 +level: 23 +disk_name: default +path: /var/lib/clickhouse/store/78e/78e6eec8-3f71-4724-b8e6-eec83f71a724/202112_1_116_23/ +marks: 69 +rows: 553071 +bytes_on_disk: 435279 +data_compressed_bytes: 424915 +data_uncompressed_bytes: 13289123 +marks_bytes: 9936 +modification_time: 2021-12-25 15:33:44 +remove_time: 1970-01-01 03:00:00 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 1 +max_block_number: 116 + +Row 8: +────── +database: system +table: metric_log +engine: MergeTree +partition_id: 202112 +name: 202112_108_108_0 +part_type: Compact +active: 1 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/9a2/9a2fb3b4-8ced-4c0b-9a2f-b3b48ced4c0b/202112_108_108_0/ +marks: 2 +rows: 8 +bytes_on_disk: 21833 +data_compressed_bytes: 11780 +data_uncompressed_bytes: 19952 +marks_bytes: 10032 +modification_time: 2021-12-25 15:33:44 +remove_time: 1970-01-01 03:00:00 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 108 +max_block_number: 108 + +Row 9: +─────── +database: system +table: asynchronous_metric_log +engine: MergeTree +partition_id: 202112 +name: 202112_115_115_0 +part_type: Compact +active: 0 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/78e/78e6eec8-3f71-4724-b8e6-eec83f71a724/202112_115_115_0/ +marks: 2 +rows: 4767 +bytes_on_disk: 11146 +data_compressed_bytes: 10946 +data_uncompressed_bytes: 128675 +marks_bytes: 176 +modification_time: 2021-12-25 15:33:37 +remove_time: 2021-12-25 15:33:44 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 115 +max_block_number: 115 + +Row 10: +─────── +database: system +table: metric_log +engine: MergeTree +partition_id: 202112 +name: 202112_107_107_0 +part_type: Compact +active: 1 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/9a2/9a2fb3b4-8ced-4c0b-9a2f-b3b48ced4c0b/202112_107_107_0/ +marks: 2 +rows: 7 +bytes_on_disk: 21996 +data_compressed_bytes: 11943 +data_uncompressed_bytes: 17458 +marks_bytes: 10032 +modification_time: 2021-12-25 15:33:36 +remove_time: 1970-01-01 03:00:00 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 107 +max_block_number: 107 + +Row 11: +─────── +database: system +table: session_log +engine: MergeTree +partition_id: 202112 +name: 202112_3_3_0 +part_type: Compact +active: 1 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/9f3/9f3dd592-781c-48d8-9f3d-d592781c48d8/202112_3_3_0/ +marks: 2 +rows: 44 +bytes_on_disk: 2208 +data_compressed_bytes: 1498 +data_uncompressed_bytes: 5130 +marks_bytes: 688 +modification_time: 2021-12-25 15:33:34 +remove_time: 1970-01-01 03:00:00 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 3 +max_block_number: 3 + +Row 12: +─────── +database: system +table: query_log +engine: MergeTree +partition_id: 202112 +name: 202112_3_3_0 +part_type: Compact +active: 1 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/1a3/1a3ec308-d42e-4f3c-9a3e-c308d42e2f3c/202112_3_3_0/ +marks: 2 +rows: 43 +bytes_on_disk: 17843 +data_compressed_bytes: 15725 +data_uncompressed_bytes: 61869 +marks_bytes: 2096 +modification_time: 2021-12-25 15:33:34 +remove_time: 1970-01-01 03:00:00 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 3 +max_block_number: 3 + +Row 13: +─────── +database: system +table: query_thread_log +engine: MergeTree +partition_id: 202112 +name: 202112_3_3_0 +part_type: Compact +active: 1 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/afa/afa652ef-f91d-4a48-afa6-52eff91daa48/202112_3_3_0/ +marks: 2 +rows: 43 +bytes_on_disk: 11878 +data_compressed_bytes: 10432 +data_uncompressed_bytes: 52339 +marks_bytes: 1424 +modification_time: 2021-12-25 15:33:34 +remove_time: 1970-01-01 03:00:00 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 3 +max_block_number: 3 + +Row 14: +─────── +database: system +table: trace_log +engine: MergeTree +partition_id: 202112 +name: 202112_52_52_0 +part_type: Compact +active: 1 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/c0b/c0bc3be3-22d7-45a3-80bc-3be322d7b5a3/202112_52_52_0/ +marks: 2 +rows: 4 +bytes_on_disk: 1078 +data_compressed_bytes: 721 +data_uncompressed_bytes: 1252 +marks_bytes: 336 +modification_time: 2021-12-25 15:33:34 +remove_time: 1970-01-01 03:00:00 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 52 +max_block_number: 52 + +Row 15: +─────── +database: system +table: asynchronous_metric_log +engine: MergeTree +partition_id: 202112 +name: 202112_114_114_0 +part_type: Compact +active: 0 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/78e/78e6eec8-3f71-4724-b8e6-eec83f71a724/202112_114_114_0/ +marks: 2 +rows: 4767 +bytes_on_disk: 11447 +data_compressed_bytes: 11247 +data_uncompressed_bytes: 128675 +marks_bytes: 176 +modification_time: 2021-12-25 15:33:30 +remove_time: 2021-12-25 15:33:44 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 114 +max_block_number: 114 + +Row 16: +─────── +database: system +table: metric_log +engine: MergeTree +partition_id: 202112 +name: 202112_1_106_21 +part_type: Compact +active: 1 +level: 21 +disk_name: default +path: /var/lib/clickhouse/store/9a2/9a2fb3b4-8ced-4c0b-9a2f-b3b48ced4c0b/202112_1_106_21/ +marks: 2 +rows: 798 +bytes_on_disk: 84853 +data_compressed_bytes: 74798 +data_uncompressed_bytes: 1990212 +marks_bytes: 10032 +modification_time: 2021-12-25 15:33:29 +remove_time: 1970-01-01 03:00:00 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 1 +max_block_number: 106 + +Row 17: +─────── +database: system +table: metric_log +engine: MergeTree +partition_id: 202112 +name: 202112_106_106_0 +part_type: Compact +active: 0 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/9a2/9a2fb3b4-8ced-4c0b-9a2f-b3b48ced4c0b/202112_106_106_0/ +marks: 2 +rows: 8 +bytes_on_disk: 21863 +data_compressed_bytes: 11810 +data_uncompressed_bytes: 19952 +marks_bytes: 10032 +modification_time: 2021-12-25 15:33:28 +remove_time: 2021-12-25 15:33:29 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 106 +max_block_number: 106 + +Row 18: +─────── +database: system +table: asynchronous_metric_log +engine: MergeTree +partition_id: 202112 +name: 202112_113_113_0 +part_type: Compact +active: 0 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/78e/78e6eec8-3f71-4724-b8e6-eec83f71a724/202112_113_113_0/ +marks: 2 +rows: 4767 +bytes_on_disk: 11191 +data_compressed_bytes: 10991 +data_uncompressed_bytes: 128675 +marks_bytes: 176 +modification_time: 2021-12-25 15:33:23 +remove_time: 2021-12-25 15:33:44 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 113 +max_block_number: 113 + +Row 19: +─────── +database: system +table: metric_log +engine: MergeTree +partition_id: 202112 +name: 202112_105_105_0 +part_type: Compact +active: 0 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/9a2/9a2fb3b4-8ced-4c0b-9a2f-b3b48ced4c0b/202112_105_105_0/ +marks: 2 +rows: 7 +bytes_on_disk: 21786 +data_compressed_bytes: 11733 +data_uncompressed_bytes: 17458 +marks_bytes: 10032 +modification_time: 2021-12-25 15:33:21 +remove_time: 2021-12-25 15:33:29 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 105 +max_block_number: 105 + +Row 20: +─────── +database: system +table: asynchronous_metric_log +engine: MergeTree +partition_id: 202112 +name: 202112_112_112_0 +part_type: Compact +active: 0 +level: 0 +disk_name: default +path: /var/lib/clickhouse/store/78e/78e6eec8-3f71-4724-b8e6-eec83f71a724/202112_112_112_0/ +marks: 2 +rows: 4767 +bytes_on_disk: 11281 +data_compressed_bytes: 11081 +data_uncompressed_bytes: 128675 +marks_bytes: 176 +modification_time: 2021-12-25 15:33:16 +remove_time: 2021-12-25 15:33:44 +refcount: 1 +is_frozen: 0 +min_date: 2021-12-25 +max_date: 2021-12-25 +min_time: 1970-01-01 03:00:00 +max_time: 1970-01-01 03:00:00 +min_block_number: 112 +max_block_number: 112 +``` +#### Detached data +##### system.detached_parts +**query** +```sql +SELECT + database, + table, + partition_id, + name, + disk, + reason, + min_block_number, + max_block_number, + level +FROM system.detached_parts +``` +**result** +``` +┌─database─┬─table─┬─partition_id─┬─name─┬─disk─┬─reason─┬─min_block_number─┬─max_block_number─┬─level─┐ +└──────────┴───────┴──────────────┴──────┴──────┴────────┴──────────────────┴──────────────────┴───────┘ +``` +##### Disk space usage +**command** +``` +du -sh -L -c /var/lib/clickhouse/data/*/*/detached/* | sort -rsh +``` +**result** +``` +0 total + +``` +#### Queries +##### Queries in progress (process list) +**query** +```sql +SELECT + elapsed, + query_id, + query, + is_cancelled, + concat(toString(read_rows), ' rows / ', formatReadableSize(read_bytes)) AS read, + concat(toString(written_rows), ' rows / ', formatReadableSize(written_bytes)) AS written, + formatReadableSize(memory_usage) AS "memory usage", + user, + multiIf(empty(client_name), http_user_agent, concat(client_name, ' ', toString(client_version_major), '.', toString(client_version_minor), '.', toString(client_version_patch))) AS client, + thread_ids, + ProfileEvents, + Settings + FROM system.processes +ORDER BY elapsed DESC +``` +**result** +``` +Row 1: +────── +elapsed: 0.000785246 +query_id: b51cbc7a-2260-4c9b-b26c-6307b10ad948 +query: SELECT + elapsed, + query_id, + query, + is_cancelled, + concat(toString(read_rows), ' rows / ', formatReadableSize(read_bytes)) AS read, + concat(toString(written_rows), ' rows / ', formatReadableSize(written_bytes)) AS written, + formatReadableSize(memory_usage) AS "memory usage", + user, + multiIf(empty(client_name), http_user_agent, concat(client_name, ' ', toString(client_version_major), '.', toString(client_version_minor), '.', toString(client_version_patch))) AS client, + thread_ids, + ProfileEvents, + Settings + FROM system.processes +ORDER BY elapsed DESC FORMAT Vertical + +is_cancelled: 0 +read: 0 rows / 0.00 B +written: 0 rows / 0.00 B +memory usage: 0.00 B +user: default +client: python-requests/2.26.0 +thread_ids: [66] +ProfileEvents: {'Query':1,'SelectQuery':1,'ContextLock':38,'RWLockAcquiredReadLocks':1} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} +``` +##### Top 10 queries by duration +**query** +```sql +SELECT + type, + query_start_time, + query_duration_ms, + query_id, + query_kind, + is_initial_query, + query, + concat(toString(read_rows), ' rows / ', formatReadableSize(read_bytes)) AS read, + concat(toString(written_rows), ' rows / ', formatReadableSize(written_bytes)) AS written, + concat(toString(result_rows), ' rows / ', formatReadableSize(result_bytes)) AS result, + formatReadableSize(memory_usage) AS "memory usage", + exception, + '\n' || stack_trace AS stack_trace, + user, + initial_user, + multiIf(empty(client_name), http_user_agent, concat(client_name, ' ', toString(client_version_major), '.', toString(client_version_minor), '.', toString(client_version_patch))) AS client, + client_hostname, + databases, + tables, + columns, + used_aggregate_functions, + used_aggregate_function_combinators, + used_database_engines, + used_data_type_families, + used_dictionaries, + used_formats, + used_functions, + used_storages, + used_table_functions, + thread_ids, + ProfileEvents, + Settings + FROM system.query_log +WHERE type != 'QueryStart' + AND event_date >= today() - 1 + AND event_time >= now() - INTERVAL 1 DAY +ORDER BY query_duration_ms DESC +LIMIT 10 +``` +**result** +``` +Row 1: +────── +type: QueryFinish +query_start_time: 2021-12-25 15:25:01 +query_duration_ms: 60 +query_id: f72e1120-cc66-434c-9809-3a99077ed842 +query_kind: Select +is_initial_query: 1 +query: SELECT + database, + table, + count() "partitions", + sum(part_count) "parts", + max(part_count) "max_parts_per_partition" +FROM +( + SELECT + database, + table, + partition, + count() "part_count" + FROM system.parts + WHERE active + GROUP BY database, table, partition +) partitions +GROUP BY database, table +ORDER BY max_parts_per_partition DESC +LIMIT 10 FORMAT PrettyCompactNoEscapes + +read: 5 rows / 262.00 B +written: 0 rows / 0.00 B +result: 3 rows / 488.00 B +memory usage: 0.00 B +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: ['system'] +tables: ['system.parts'] +columns: ['system.parts.active','system.parts.database','system.parts.partition','system.parts.table'] +used_aggregate_functions: ['count','max','sum'] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['PrettyCompactNoEscapes'] +used_functions: [] +used_storages: [] +used_table_functions: [] +thread_ids: [66] +ProfileEvents: {'Query':1,'SelectQuery':1,'ArenaAllocChunks':2,'ArenaAllocBytes':8192,'CompileFunction':1,'CompileExpressionsMicroseconds':52574,'CompileExpressionsBytes':8192,'SelectedRows':5,'SelectedBytes':262,'ContextLock':58,'RWLockAcquiredReadLocks':6,'RealTimeMicroseconds':61493,'UserTimeMicroseconds':34154,'SystemTimeMicroseconds':9874,'SoftPageFaults':170,'HardPageFaults':33,'OSIOWaitMicroseconds':10000,'OSCPUWaitMicroseconds':2433,'OSCPUVirtualTimeMicroseconds':43706,'OSReadBytes':3080192,'OSWriteBytes':4096,'OSReadChars':863,'OSWriteChars':5334} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 2: +────── +type: QueryFinish +query_start_time: 2021-12-25 15:26:26 +query_duration_ms: 12 +query_id: eabd7483-70df-4d60-a668-d8961416e3fb +query_kind: Select +is_initial_query: 1 +query: SELECT + type, + query_start_time, + query_duration_ms, + query_id, + query_kind, + is_initial_query, + query, + concat(toString(read_rows), ' rows / ', formatReadableSize(read_bytes)) AS read, + concat(toString(written_rows), ' rows / ', formatReadableSize(written_bytes)) AS written, + concat(toString(result_rows), ' rows / ', formatReadableSize(result_bytes)) AS result, + formatReadableSize(memory_usage) AS "memory usage", + exception, + '\n' || stack_trace AS stack_trace, + user, + initial_user, + multiIf(empty(client_name), http_user_agent, concat(client_name, ' ', toString(client_version_major), '.', toString(client_version_minor), '.', toString(client_version_patch))) AS client, + client_hostname, + databases, + tables, + columns, + used_aggregate_functions, + used_aggregate_function_combinators, + used_database_engines, + used_data_type_families, + used_dictionaries, + used_formats, + used_functions, + used_storages, + used_table_functions, + thread_ids, + ProfileEvents, + Settings + FROM system.query_log +WHERE type != 'QueryStart' + AND event_date >= today() - 1 + AND event_time >= now() - INTERVAL 1 DAY +ORDER BY query_duration_ms DESC +LIMIT 10 FORMAT Vertical + +read: 40 rows / 67.42 KiB +written: 0 rows / 0.00 B +result: 10 rows / 41.23 KiB +memory usage: 0.00 B +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: ['system'] +tables: ['system.query_log'] +columns: ['system.query_log.ProfileEvents','system.query_log.Settings','system.query_log.client_hostname','system.query_log.client_name','system.query_log.client_version_major','system.query_log.client_version_minor','system.query_log.client_version_patch','system.query_log.columns','system.query_log.databases','system.query_log.event_date','system.query_log.event_time','system.query_log.exception','system.query_log.http_user_agent','system.query_log.initial_user','system.query_log.is_initial_query','system.query_log.memory_usage','system.query_log.query','system.query_log.query_duration_ms','system.query_log.query_id','system.query_log.query_kind','system.query_log.query_start_time','system.query_log.read_bytes','system.query_log.read_rows','system.query_log.result_bytes','system.query_log.result_rows','system.query_log.stack_trace','system.query_log.tables','system.query_log.thread_ids','system.query_log.type','system.query_log.used_aggregate_function_combinators','system.query_log.used_aggregate_functions','system.query_log.used_data_type_families','system.query_log.used_database_engines','system.query_log.used_dictionaries','system.query_log.used_formats','system.query_log.used_functions','system.query_log.used_storages','system.query_log.used_table_functions','system.query_log.user','system.query_log.written_bytes','system.query_log.written_rows'] +used_aggregate_functions: [] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['Vertical'] +used_functions: ['empty','and','now','concat','today','toIntervalDay','formatReadableSize','minus','greaterOrEquals','multiIf','toString','subtractDays','notEquals'] +used_storages: [] +used_table_functions: [] +thread_ids: [66] +ProfileEvents: {'Query':1,'SelectQuery':1,'FileOpen':2,'Seek':3,'ReadBufferFromFileDescriptorRead':10,'ReadBufferFromFileDescriptorReadBytes':16873,'ReadCompressedBytes':12855,'CompressedReadBufferBlocks':41,'CompressedReadBufferBytes':61376,'IOBufferAllocs':5,'IOBufferAllocBytes':26594,'FunctionExecute':28,'MarkCacheHits':1,'MarkCacheMisses':1,'CreatedReadBufferOrdinary':3,'DiskReadElapsedMicroseconds':30,'SelectedParts':1,'SelectedRanges':1,'SelectedMarks':1,'SelectedRows':40,'SelectedBytes':69039,'ContextLock':342,'RWLockAcquiredReadLocks':1,'RealTimeMicroseconds':14451,'UserTimeMicroseconds':10009,'SystemTimeMicroseconds':1515,'SoftPageFaults':44,'OSCPUWaitMicroseconds':3050,'OSCPUVirtualTimeMicroseconds':11523,'OSReadChars':17311,'OSWriteChars':7288} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 3: +────── +type: QueryFinish +query_start_time: 2021-12-25 15:33:29 +query_duration_ms: 12 +query_id: d9557845-5b5e-44ef-befa-55f837065d00 +query_kind: Select +is_initial_query: 1 +query: SELECT + type, + query_start_time, + query_duration_ms, + query_id, + query_kind, + is_initial_query, + query, + concat(toString(read_rows), ' rows / ', formatReadableSize(read_bytes)) AS read, + concat(toString(written_rows), ' rows / ', formatReadableSize(written_bytes)) AS written, + concat(toString(result_rows), ' rows / ', formatReadableSize(result_bytes)) AS result, + formatReadableSize(memory_usage) AS "memory usage", + exception, + '\n' || stack_trace AS stack_trace, + user, + initial_user, + multiIf(empty(client_name), http_user_agent, concat(client_name, ' ', toString(client_version_major), '.', toString(client_version_minor), '.', toString(client_version_patch))) AS client, + client_hostname, + databases, + tables, + columns, + used_aggregate_functions, + used_aggregate_function_combinators, + used_database_engines, + used_data_type_families, + used_dictionaries, + used_formats, + used_functions, + used_storages, + used_table_functions, + thread_ids, + ProfileEvents, + Settings + FROM system.query_log +WHERE type != 'QueryStart' + AND event_date >= today() - 1 + AND event_time >= now() - INTERVAL 1 DAY +ORDER BY query_duration_ms DESC +LIMIT 10 FORMAT Vertical + +read: 83 rows / 130.00 KiB +written: 0 rows / 0.00 B +result: 10 rows / 183.10 KiB +memory usage: 0.00 B +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: ['system'] +tables: ['system.query_log'] +columns: ['system.query_log.ProfileEvents','system.query_log.Settings','system.query_log.client_hostname','system.query_log.client_name','system.query_log.client_version_major','system.query_log.client_version_minor','system.query_log.client_version_patch','system.query_log.columns','system.query_log.databases','system.query_log.event_date','system.query_log.event_time','system.query_log.exception','system.query_log.http_user_agent','system.query_log.initial_user','system.query_log.is_initial_query','system.query_log.memory_usage','system.query_log.query','system.query_log.query_duration_ms','system.query_log.query_id','system.query_log.query_kind','system.query_log.query_start_time','system.query_log.read_bytes','system.query_log.read_rows','system.query_log.result_bytes','system.query_log.result_rows','system.query_log.stack_trace','system.query_log.tables','system.query_log.thread_ids','system.query_log.type','system.query_log.used_aggregate_function_combinators','system.query_log.used_aggregate_functions','system.query_log.used_data_type_families','system.query_log.used_database_engines','system.query_log.used_dictionaries','system.query_log.used_formats','system.query_log.used_functions','system.query_log.used_storages','system.query_log.used_table_functions','system.query_log.user','system.query_log.written_bytes','system.query_log.written_rows'] +used_aggregate_functions: [] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['Vertical'] +used_functions: ['empty','and','now','concat','today','toIntervalDay','formatReadableSize','minus','greaterOrEquals','multiIf','toString','subtractDays','notEquals'] +used_storages: [] +used_table_functions: [] +thread_ids: [66,283,225,281,282] +ProfileEvents: {'Query':1,'SelectQuery':1,'FileOpen':3,'Seek':6,'ReadBufferFromFileDescriptorRead':18,'ReadBufferFromFileDescriptorReadBytes':32140,'ReadCompressedBytes':25892,'CompressedReadBufferBlocks':82,'CompressedReadBufferBytes':116215,'IOBufferAllocs':9,'IOBufferAllocBytes':47368,'FunctionExecute':51,'MarkCacheHits':3,'MarkCacheMisses':1,'CreatedReadBufferOrdinary':5,'DiskReadElapsedMicroseconds':13,'SelectedParts':2,'SelectedRanges':2,'SelectedMarks':2,'SelectedRows':83,'SelectedBytes':133125,'ContextLock':351,'RWLockAcquiredReadLocks':1,'RealTimeMicroseconds':19368,'UserTimeMicroseconds':12036,'SystemTimeMicroseconds':2047,'SoftPageFaults':42,'OSCPUWaitMicroseconds':710,'OSCPUVirtualTimeMicroseconds':13623,'OSWriteBytes':4096,'OSReadChars':34225,'OSWriteChars':8142} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 4: +────── +type: QueryFinish +query_start_time: 2021-12-25 15:33:29 +query_duration_ms: 11 +query_id: bae8a338-eee9-406b-80d2-4596af2ba31f +query_kind: Select +is_initial_query: 1 +query: SELECT + name, + engine, + tables, + partitions, + parts, + formatReadableSize(bytes_on_disk) "disk_size" +FROM system.databases db +LEFT JOIN +( + SELECT + database, + uniq(table) "tables", + uniq(table, partition) "partitions", + count() AS parts, + sum(bytes_on_disk) "bytes_on_disk" + FROM system.parts + WHERE active + GROUP BY database +) AS db_stats ON db.name = db_stats.database +ORDER BY bytes_on_disk DESC +LIMIT 10 FORMAT PrettyCompactNoEscapes + +read: 17 rows / 1.31 KiB +written: 0 rows / 0.00 B +result: 4 rows / 640.00 B +memory usage: 0.00 B +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: ['system'] +tables: ['system.databases','system.parts'] +columns: ['system.databases.engine','system.databases.name','system.parts.active','system.parts.bytes_on_disk','system.parts.database','system.parts.partition','system.parts.table'] +used_aggregate_functions: ['count','sum','uniq'] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['PrettyCompactNoEscapes'] +used_functions: ['formatReadableSize'] +used_storages: [] +used_table_functions: [] +thread_ids: [66] +ProfileEvents: {'Query':1,'SelectQuery':1,'ArenaAllocChunks':5,'ArenaAllocBytes':20480,'FunctionExecute':1,'SelectedRows':17,'SelectedBytes':1345,'ContextLock':69,'RWLockAcquiredReadLocks':9,'RealTimeMicroseconds':12225,'UserTimeMicroseconds':10731,'SystemTimeMicroseconds':1146,'SoftPageFaults':2,'OSCPUWaitMicroseconds':720,'OSCPUVirtualTimeMicroseconds':11876,'OSWriteBytes':4096,'OSReadChars':438,'OSWriteChars':8938} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 5: +────── +type: QueryFinish +query_start_time: 2021-12-25 15:26:26 +query_duration_ms: 9 +query_id: f0c62bc7-36da-4542-a3d5-68a40c1c4b48 +query_kind: Select +is_initial_query: 1 +query: SELECT + type, + query_start_time, + query_duration_ms, + query_id, + query_kind, + is_initial_query, + query, + concat(toString(read_rows), ' rows / ', formatReadableSize(read_bytes)) AS read, + concat(toString(written_rows), ' rows / ', formatReadableSize(written_bytes)) AS written, + concat(toString(result_rows), ' rows / ', formatReadableSize(result_bytes)) AS result, + formatReadableSize(memory_usage) AS "memory usage", + exception, + '\n' || stack_trace AS stack_trace, + user, + initial_user, + multiIf(empty(client_name), http_user_agent, concat(client_name, ' ', toString(client_version_major), '.', toString(client_version_minor), '.', toString(client_version_patch))) AS client, + client_hostname, + databases, + tables, + columns, + used_aggregate_functions, + used_aggregate_function_combinators, + used_database_engines, + used_data_type_families, + used_dictionaries, + used_formats, + used_functions, + used_storages, + used_table_functions, + thread_ids, + ProfileEvents, + Settings + FROM system.query_log +WHERE type != 'QueryStart' + AND event_date >= today() - 1 + AND event_time >= now() - INTERVAL 1 DAY + AND exception != '' +ORDER BY query_start_time DESC +LIMIT 10 FORMAT Vertical + +read: 40 rows / 67.42 KiB +written: 0 rows / 0.00 B +result: 4 rows / 43.13 KiB +memory usage: 0.00 B +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: ['system'] +tables: ['system.query_log'] +columns: ['system.query_log.ProfileEvents','system.query_log.Settings','system.query_log.client_hostname','system.query_log.client_name','system.query_log.client_version_major','system.query_log.client_version_minor','system.query_log.client_version_patch','system.query_log.columns','system.query_log.databases','system.query_log.event_date','system.query_log.event_time','system.query_log.exception','system.query_log.http_user_agent','system.query_log.initial_user','system.query_log.is_initial_query','system.query_log.memory_usage','system.query_log.query','system.query_log.query_duration_ms','system.query_log.query_id','system.query_log.query_kind','system.query_log.query_start_time','system.query_log.read_bytes','system.query_log.read_rows','system.query_log.result_bytes','system.query_log.result_rows','system.query_log.stack_trace','system.query_log.tables','system.query_log.thread_ids','system.query_log.type','system.query_log.used_aggregate_function_combinators','system.query_log.used_aggregate_functions','system.query_log.used_data_type_families','system.query_log.used_database_engines','system.query_log.used_dictionaries','system.query_log.used_formats','system.query_log.used_functions','system.query_log.used_storages','system.query_log.used_table_functions','system.query_log.user','system.query_log.written_bytes','system.query_log.written_rows'] +used_aggregate_functions: [] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['Vertical'] +used_functions: ['empty','and','now','concat','today','toIntervalDay','formatReadableSize','minus','greaterOrEquals','multiIf','toString','subtractDays','notEquals'] +used_storages: [] +used_table_functions: [] +thread_ids: [66] +ProfileEvents: {'Query':1,'SelectQuery':1,'FileOpen':1,'Seek':3,'ReadBufferFromFileDescriptorRead':8,'ReadBufferFromFileDescriptorReadBytes':15561,'ReadCompressedBytes':12855,'CompressedReadBufferBlocks':41,'CompressedReadBufferBytes':61376,'IOBufferAllocs':4,'IOBufferAllocBytes':25506,'FunctionExecute':31,'MarkCacheHits':2,'CreatedReadBufferOrdinary':2,'DiskReadElapsedMicroseconds':16,'SelectedParts':1,'SelectedRanges':1,'SelectedMarks':1,'SelectedRows':40,'SelectedBytes':69039,'ContextLock':361,'RWLockAcquiredReadLocks':1,'RealTimeMicroseconds':11353,'UserTimeMicroseconds':8910,'SystemTimeMicroseconds':533,'SoftPageFaults':7,'HardPageFaults':2,'OSCPUWaitMicroseconds':1117,'OSCPUVirtualTimeMicroseconds':9443,'OSReadBytes':16384,'OSWriteBytes':4096,'OSReadChars':15999,'OSWriteChars':7714,'QueryProfilerRuns':1} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 6: +────── +type: QueryFinish +query_start_time: 2021-12-25 15:33:29 +query_duration_ms: 8 +query_id: 72f3f9de-d17c-456b-8316-d494bea2096a +query_kind: Select +is_initial_query: 1 +query: SELECT name FROM system.tables WHERE database = 'system' FORMAT JSONCompact + +read: 74 rows / 2.61 KiB +written: 0 rows / 0.00 B +result: 74 rows / 2.00 KiB +memory usage: 0.00 B +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: ['system'] +tables: ['system.tables'] +columns: ['system.tables.database','system.tables.name'] +used_aggregate_functions: [] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['JSONCompact'] +used_functions: ['equals'] +used_storages: [] +used_table_functions: [] +thread_ids: [66] +ProfileEvents: {'Query':1,'SelectQuery':1,'IOBufferAllocs':2,'IOBufferAllocBytes':8192,'FunctionExecute':4,'SelectedRows':74,'SelectedBytes':2675,'ContextLock':23,'RWLockAcquiredReadLocks':75,'RealTimeMicroseconds':9190,'UserTimeMicroseconds':6468,'SystemTimeMicroseconds':517,'OSCPUWaitMicroseconds':2237,'OSCPUVirtualTimeMicroseconds':6984,'OSReadChars':438,'OSWriteChars':1270} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 7: +────── +type: QueryFinish +query_start_time: 2021-12-25 15:33:29 +query_duration_ms: 8 +query_id: d55da87f-b030-4b5d-95fc-f9103ce58601 +query_kind: Select +is_initial_query: 1 +query: SELECT + type, + query_start_time, + query_duration_ms, + query_id, + query_kind, + is_initial_query, + query, + concat(toString(read_rows), ' rows / ', formatReadableSize(read_bytes)) AS read, + concat(toString(written_rows), ' rows / ', formatReadableSize(written_bytes)) AS written, + concat(toString(result_rows), ' rows / ', formatReadableSize(result_bytes)) AS result, + formatReadableSize(memory_usage) AS "memory usage", + exception, + '\n' || stack_trace AS stack_trace, + user, + initial_user, + multiIf(empty(client_name), http_user_agent, concat(client_name, ' ', toString(client_version_major), '.', toString(client_version_minor), '.', toString(client_version_patch))) AS client, + client_hostname, + databases, + tables, + columns, + used_aggregate_functions, + used_aggregate_function_combinators, + used_database_engines, + used_data_type_families, + used_dictionaries, + used_formats, + used_functions, + used_storages, + used_table_functions, + thread_ids, + ProfileEvents, + Settings + FROM system.query_log +WHERE type != 'QueryStart' + AND event_date >= today() - 1 + AND event_time >= now() - INTERVAL 1 DAY +ORDER BY memory_usage DESC +LIMIT 10 FORMAT Vertical + +read: 83 rows / 130.00 KiB +written: 0 rows / 0.00 B +result: 10 rows / 178.41 KiB +memory usage: 0.00 B +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: ['system'] +tables: ['system.query_log'] +columns: ['system.query_log.ProfileEvents','system.query_log.Settings','system.query_log.client_hostname','system.query_log.client_name','system.query_log.client_version_major','system.query_log.client_version_minor','system.query_log.client_version_patch','system.query_log.columns','system.query_log.databases','system.query_log.event_date','system.query_log.event_time','system.query_log.exception','system.query_log.http_user_agent','system.query_log.initial_user','system.query_log.is_initial_query','system.query_log.memory_usage','system.query_log.query','system.query_log.query_duration_ms','system.query_log.query_id','system.query_log.query_kind','system.query_log.query_start_time','system.query_log.read_bytes','system.query_log.read_rows','system.query_log.result_bytes','system.query_log.result_rows','system.query_log.stack_trace','system.query_log.tables','system.query_log.thread_ids','system.query_log.type','system.query_log.used_aggregate_function_combinators','system.query_log.used_aggregate_functions','system.query_log.used_data_type_families','system.query_log.used_database_engines','system.query_log.used_dictionaries','system.query_log.used_formats','system.query_log.used_functions','system.query_log.used_storages','system.query_log.used_table_functions','system.query_log.user','system.query_log.written_bytes','system.query_log.written_rows'] +used_aggregate_functions: [] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['Vertical'] +used_functions: ['empty','and','now','concat','today','toIntervalDay','formatReadableSize','minus','greaterOrEquals','multiIf','toString','subtractDays','notEquals'] +used_storages: [] +used_table_functions: [] +thread_ids: [66,283,283,225,282] +ProfileEvents: {'Query':1,'SelectQuery':1,'FileOpen':2,'Seek':6,'ReadBufferFromFileDescriptorRead':16,'ReadBufferFromFileDescriptorReadBytes':30044,'ReadCompressedBytes':25892,'CompressedReadBufferBlocks':82,'CompressedReadBufferBytes':116215,'IOBufferAllocs':8,'IOBufferAllocBytes':45272,'FunctionExecute':51,'MarkCacheHits':4,'CreatedReadBufferOrdinary':4,'SelectedParts':2,'SelectedRanges':2,'SelectedMarks':2,'SelectedRows':83,'SelectedBytes':133125,'ContextLock':351,'RWLockAcquiredReadLocks':1,'RealTimeMicroseconds':12416,'UserTimeMicroseconds':7727,'SystemTimeMicroseconds':1247,'SoftPageFaults':41,'OSCPUWaitMicroseconds':1058,'OSCPUVirtualTimeMicroseconds':9018,'OSWriteBytes':4096,'OSReadChars':32137,'OSWriteChars':8108} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 8: +────── +type: QueryFinish +query_start_time: 2021-12-25 15:33:29 +query_duration_ms: 8 +query_id: cc2a0e7a-3b9b-47d2-9255-009c62584bc4 +query_kind: Select +is_initial_query: 1 +query: SELECT + type, + query_start_time, + query_duration_ms, + query_id, + query_kind, + is_initial_query, + query, + concat(toString(read_rows), ' rows / ', formatReadableSize(read_bytes)) AS read, + concat(toString(written_rows), ' rows / ', formatReadableSize(written_bytes)) AS written, + concat(toString(result_rows), ' rows / ', formatReadableSize(result_bytes)) AS result, + formatReadableSize(memory_usage) AS "memory usage", + exception, + '\n' || stack_trace AS stack_trace, + user, + initial_user, + multiIf(empty(client_name), http_user_agent, concat(client_name, ' ', toString(client_version_major), '.', toString(client_version_minor), '.', toString(client_version_patch))) AS client, + client_hostname, + databases, + tables, + columns, + used_aggregate_functions, + used_aggregate_function_combinators, + used_database_engines, + used_data_type_families, + used_dictionaries, + used_formats, + used_functions, + used_storages, + used_table_functions, + thread_ids, + ProfileEvents, + Settings + FROM system.query_log +WHERE type != 'QueryStart' + AND event_date >= today() - 1 + AND event_time >= now() - INTERVAL 1 DAY + AND exception != '' +ORDER BY query_start_time DESC +LIMIT 10 FORMAT Vertical + +read: 83 rows / 130.00 KiB +written: 0 rows / 0.00 B +result: 5 rows / 57.80 KiB +memory usage: 0.00 B +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: ['system'] +tables: ['system.query_log'] +columns: ['system.query_log.ProfileEvents','system.query_log.Settings','system.query_log.client_hostname','system.query_log.client_name','system.query_log.client_version_major','system.query_log.client_version_minor','system.query_log.client_version_patch','system.query_log.columns','system.query_log.databases','system.query_log.event_date','system.query_log.event_time','system.query_log.exception','system.query_log.http_user_agent','system.query_log.initial_user','system.query_log.is_initial_query','system.query_log.memory_usage','system.query_log.query','system.query_log.query_duration_ms','system.query_log.query_id','system.query_log.query_kind','system.query_log.query_start_time','system.query_log.read_bytes','system.query_log.read_rows','system.query_log.result_bytes','system.query_log.result_rows','system.query_log.stack_trace','system.query_log.tables','system.query_log.thread_ids','system.query_log.type','system.query_log.used_aggregate_function_combinators','system.query_log.used_aggregate_functions','system.query_log.used_data_type_families','system.query_log.used_database_engines','system.query_log.used_dictionaries','system.query_log.used_formats','system.query_log.used_functions','system.query_log.used_storages','system.query_log.used_table_functions','system.query_log.user','system.query_log.written_bytes','system.query_log.written_rows'] +used_aggregate_functions: [] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['Vertical'] +used_functions: ['empty','and','now','concat','today','toIntervalDay','formatReadableSize','minus','greaterOrEquals','multiIf','toString','subtractDays','notEquals'] +used_storages: [] +used_table_functions: [] +thread_ids: [66,281,283,282,225] +ProfileEvents: {'Query':1,'SelectQuery':1,'FileOpen':2,'Seek':6,'ReadBufferFromFileDescriptorRead':16,'ReadBufferFromFileDescriptorReadBytes':31464,'ReadCompressedBytes':25892,'CompressedReadBufferBlocks':82,'CompressedReadBufferBytes':116215,'IOBufferAllocs':8,'IOBufferAllocBytes':46860,'FunctionExecute':56,'MarkCacheHits':4,'CreatedReadBufferOrdinary':4,'SelectedParts':2,'SelectedRanges':2,'SelectedMarks':2,'SelectedRows':83,'SelectedBytes':133125,'ContextLock':370,'RWLockAcquiredReadLocks':1,'RealTimeMicroseconds':13096,'UserTimeMicroseconds':9503,'SystemTimeMicroseconds':195,'SoftPageFaults':23,'OSCPUWaitMicroseconds':1380,'OSCPUVirtualTimeMicroseconds':9661,'OSWriteBytes':4096,'OSReadChars':33567,'OSWriteChars':8310} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 9: +─────── +type: QueryFinish +query_start_time: 2021-12-25 15:25:01 +query_duration_ms: 8 +query_id: a3d717fd-c43f-4723-a18d-557c733299f6 +query_kind: Select +is_initial_query: 1 +query: SELECT + name, + engine, + tables, + partitions, + parts, + formatReadableSize(bytes_on_disk) "disk_size" +FROM system.databases db +LEFT JOIN +( + SELECT + database, + uniq(table) "tables", + uniq(table, partition) "partitions", + count() AS parts, + sum(bytes_on_disk) "bytes_on_disk" + FROM system.parts + WHERE active + GROUP BY database +) AS db_stats ON db.name = db_stats.database +ORDER BY bytes_on_disk DESC +LIMIT 10 FORMAT PrettyCompactNoEscapes + +read: 9 rows / 845.00 B +written: 0 rows / 0.00 B +result: 4 rows / 640.00 B +memory usage: 0.00 B +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: ['system'] +tables: ['system.databases','system.parts'] +columns: ['system.databases.engine','system.databases.name','system.parts.active','system.parts.bytes_on_disk','system.parts.database','system.parts.partition','system.parts.table'] +used_aggregate_functions: ['count','sum','uniq'] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['PrettyCompactNoEscapes'] +used_functions: ['formatReadableSize'] +used_storages: [] +used_table_functions: [] +thread_ids: [66] +ProfileEvents: {'Query':1,'SelectQuery':1,'ArenaAllocChunks':5,'ArenaAllocBytes':20480,'FunctionExecute':1,'SelectedRows':9,'SelectedBytes':845,'ContextLock':69,'RWLockAcquiredReadLocks':6,'RealTimeMicroseconds':9090,'UserTimeMicroseconds':4654,'SystemTimeMicroseconds':1171,'SoftPageFaults':8,'HardPageFaults':2,'OSCPUWaitMicroseconds':2126,'OSCPUVirtualTimeMicroseconds':5824,'OSReadBytes':212992,'OSWriteBytes':4096,'OSReadChars':427,'OSWriteChars':8936} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 10: +─────── +type: QueryFinish +query_start_time: 2021-12-25 15:26:26 +query_duration_ms: 7 +query_id: 49305759-0f08-4d5a-81d8-c1a11cfc0eb4 +query_kind: Select +is_initial_query: 1 +query: SELECT + type, + query_start_time, + query_duration_ms, + query_id, + query_kind, + is_initial_query, + query, + concat(toString(read_rows), ' rows / ', formatReadableSize(read_bytes)) AS read, + concat(toString(written_rows), ' rows / ', formatReadableSize(written_bytes)) AS written, + concat(toString(result_rows), ' rows / ', formatReadableSize(result_bytes)) AS result, + formatReadableSize(memory_usage) AS "memory usage", + exception, + '\n' || stack_trace AS stack_trace, + user, + initial_user, + multiIf(empty(client_name), http_user_agent, concat(client_name, ' ', toString(client_version_major), '.', toString(client_version_minor), '.', toString(client_version_patch))) AS client, + client_hostname, + databases, + tables, + columns, + used_aggregate_functions, + used_aggregate_function_combinators, + used_database_engines, + used_data_type_families, + used_dictionaries, + used_formats, + used_functions, + used_storages, + used_table_functions, + thread_ids, + ProfileEvents, + Settings + FROM system.query_log +WHERE type != 'QueryStart' + AND event_date >= today() - 1 + AND event_time >= now() - INTERVAL 1 DAY +ORDER BY memory_usage DESC +LIMIT 10 FORMAT Vertical + +read: 40 rows / 67.42 KiB +written: 0 rows / 0.00 B +result: 10 rows / 57.95 KiB +memory usage: 0.00 B +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: ['system'] +tables: ['system.query_log'] +columns: ['system.query_log.ProfileEvents','system.query_log.Settings','system.query_log.client_hostname','system.query_log.client_name','system.query_log.client_version_major','system.query_log.client_version_minor','system.query_log.client_version_patch','system.query_log.columns','system.query_log.databases','system.query_log.event_date','system.query_log.event_time','system.query_log.exception','system.query_log.http_user_agent','system.query_log.initial_user','system.query_log.is_initial_query','system.query_log.memory_usage','system.query_log.query','system.query_log.query_duration_ms','system.query_log.query_id','system.query_log.query_kind','system.query_log.query_start_time','system.query_log.read_bytes','system.query_log.read_rows','system.query_log.result_bytes','system.query_log.result_rows','system.query_log.stack_trace','system.query_log.tables','system.query_log.thread_ids','system.query_log.type','system.query_log.used_aggregate_function_combinators','system.query_log.used_aggregate_functions','system.query_log.used_data_type_families','system.query_log.used_database_engines','system.query_log.used_dictionaries','system.query_log.used_formats','system.query_log.used_functions','system.query_log.used_storages','system.query_log.used_table_functions','system.query_log.user','system.query_log.written_bytes','system.query_log.written_rows'] +used_aggregate_functions: [] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['Vertical'] +used_functions: ['empty','and','now','concat','today','toIntervalDay','formatReadableSize','minus','greaterOrEquals','multiIf','toString','subtractDays','notEquals'] +used_storages: [] +used_table_functions: [] +thread_ids: [66] +ProfileEvents: {'Query':1,'SelectQuery':1,'FileOpen':1,'Seek':3,'ReadBufferFromFileDescriptorRead':8,'ReadBufferFromFileDescriptorReadBytes':14777,'ReadCompressedBytes':12855,'CompressedReadBufferBlocks':41,'CompressedReadBufferBytes':61376,'IOBufferAllocs':4,'IOBufferAllocBytes':24498,'FunctionExecute':28,'MarkCacheHits':2,'CreatedReadBufferOrdinary':2,'DiskReadElapsedMicroseconds':16,'SelectedParts':1,'SelectedRanges':1,'SelectedMarks':1,'SelectedRows':40,'SelectedBytes':69039,'ContextLock':342,'RWLockAcquiredReadLocks':1,'RealTimeMicroseconds':9159,'UserTimeMicroseconds':4713,'SystemTimeMicroseconds':1942,'SoftPageFaults':19,'OSCPUWaitMicroseconds':2421,'OSCPUVirtualTimeMicroseconds':6655,'OSWriteBytes':4096,'OSReadChars':15215,'OSWriteChars':7278} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} +``` +##### Top 10 queries by memory usage +**query** +```sql +SELECT + type, + query_start_time, + query_duration_ms, + query_id, + query_kind, + is_initial_query, + query, + concat(toString(read_rows), ' rows / ', formatReadableSize(read_bytes)) AS read, + concat(toString(written_rows), ' rows / ', formatReadableSize(written_bytes)) AS written, + concat(toString(result_rows), ' rows / ', formatReadableSize(result_bytes)) AS result, + formatReadableSize(memory_usage) AS "memory usage", + exception, + '\n' || stack_trace AS stack_trace, + user, + initial_user, + multiIf(empty(client_name), http_user_agent, concat(client_name, ' ', toString(client_version_major), '.', toString(client_version_minor), '.', toString(client_version_patch))) AS client, + client_hostname, + databases, + tables, + columns, + used_aggregate_functions, + used_aggregate_function_combinators, + used_database_engines, + used_data_type_families, + used_dictionaries, + used_formats, + used_functions, + used_storages, + used_table_functions, + thread_ids, + ProfileEvents, + Settings + FROM system.query_log +WHERE type != 'QueryStart' + AND event_date >= today() - 1 + AND event_time >= now() - INTERVAL 1 DAY +ORDER BY memory_usage DESC +LIMIT 10 +``` +**result** +``` +Row 1: +────── +type: QueryFinish +query_start_time: 2021-12-25 15:26:25 +query_duration_ms: 0 +query_id: c6b6a96c-d5c5-4406-98cd-80857a8412d4 +query_kind: +is_initial_query: 1 +query: SHOW ACCESS FORMAT TSVRaw + +read: 5 rows / 405.00 B +written: 0 rows / 0.00 B +result: 5 rows / 4.50 KiB +memory usage: 1.82 KiB +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: [] +tables: [] +columns: [] +used_aggregate_functions: [] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['TSVRaw'] +used_functions: [] +used_storages: [] +used_table_functions: [] +thread_ids: [66,283,225,281] +ProfileEvents: {'Query':1,'IOBufferAllocs':3,'IOBufferAllocBytes':3145728,'SelectedRows':5,'SelectedBytes':405,'ContextLock':8,'RealTimeMicroseconds':959,'UserTimeMicroseconds':452,'SystemTimeMicroseconds':238,'OSCPUWaitMicroseconds':90,'OSCPUVirtualTimeMicroseconds':690,'OSWriteBytes':4096,'OSReadChars':846,'OSWriteChars':880} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 2: +────── +type: QueryFinish +query_start_time: 2021-12-25 15:33:29 +query_duration_ms: 2 +query_id: 253362ba-40a1-4593-a4cc-30d3dfdfe0ab +query_kind: +is_initial_query: 1 +query: SHOW ACCESS FORMAT TSVRaw + +read: 5 rows / 405.00 B +written: 0 rows / 0.00 B +result: 5 rows / 4.50 KiB +memory usage: 1.82 KiB +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: [] +tables: [] +columns: [] +used_aggregate_functions: [] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['TSVRaw'] +used_functions: [] +used_storages: [] +used_table_functions: [] +thread_ids: [66,225,283,282] +ProfileEvents: {'Query':1,'IOBufferAllocs':3,'IOBufferAllocBytes':3145728,'SelectedRows':5,'SelectedBytes':405,'ContextLock':8,'RealTimeMicroseconds':4687,'UserTimeMicroseconds':2171,'SystemTimeMicroseconds':1264,'OSCPUWaitMicroseconds':513,'OSCPUVirtualTimeMicroseconds':3335,'OSReadChars':848,'OSWriteChars':880} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 3: +────── +type: QueryFinish +query_start_time: 2021-12-25 15:25:01 +query_duration_ms: 1 +query_id: 61b20c8c-ca63-4384-adb4-ce7765d77389 +query_kind: +is_initial_query: 1 +query: SHOW ACCESS FORMAT TSVRaw + +read: 5 rows / 405.00 B +written: 0 rows / 0.00 B +result: 5 rows / 4.50 KiB +memory usage: 1.82 KiB +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: [] +tables: [] +columns: [] +used_aggregate_functions: [] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['TSVRaw'] +used_functions: [] +used_storages: [] +used_table_functions: [] +thread_ids: [66,225,281,283] +ProfileEvents: {'Query':1,'IOBufferAllocs':3,'IOBufferAllocBytes':3145728,'SelectedRows':5,'SelectedBytes':405,'ContextLock':8,'RealTimeMicroseconds':3442,'UserTimeMicroseconds':715,'SystemTimeMicroseconds':485,'SoftPageFaults':1,'OSCPUWaitMicroseconds':443,'OSCPUVirtualTimeMicroseconds':1170,'OSReadChars':833,'OSWriteChars':880} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 4: +────── +type: QueryFinish +query_start_time: 2021-12-25 15:26:25 +query_duration_ms: 1 +query_id: 13ebdab7-e368-4f9f-b47e-023dbd9e91ce +query_kind: Select +is_initial_query: 1 +query: +SELECT formatReadableTimeDelta(uptime()) + + +read: 1 rows / 1.00 B +written: 0 rows / 0.00 B +result: 1 rows / 128.00 B +memory usage: 1.49 KiB +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: ['system'] +tables: ['system.one'] +columns: ['system.one.dummy'] +used_aggregate_functions: [] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['TabSeparated'] +used_functions: ['uptime','formatReadableTimeDelta'] +used_storages: [] +used_table_functions: [] +thread_ids: [66,283,282,225,281] +ProfileEvents: {'Query':1,'SelectQuery':1,'IOBufferAllocs':3,'IOBufferAllocBytes':3145728,'SelectedRows':1,'SelectedBytes':1,'ContextLock':17,'RWLockAcquiredReadLocks':1,'RealTimeMicroseconds':1613,'UserTimeMicroseconds':708,'SystemTimeMicroseconds':274,'SoftPageFaults':3,'OSCPUWaitMicroseconds':2,'OSCPUVirtualTimeMicroseconds':980,'OSReadChars':846,'OSWriteChars':1190} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 5: +────── +type: QueryFinish +query_start_time: 2021-12-25 15:33:29 +query_duration_ms: 2 +query_id: ff330183-854b-46bc-a548-30e12a7bee9c +query_kind: Select +is_initial_query: 1 +query: +SELECT formatReadableTimeDelta(uptime()) + + +read: 1 rows / 1.00 B +written: 0 rows / 0.00 B +result: 1 rows / 128.00 B +memory usage: 1.49 KiB +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: ['system'] +tables: ['system.one'] +columns: ['system.one.dummy'] +used_aggregate_functions: [] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['TabSeparated'] +used_functions: ['formatReadableTimeDelta','uptime'] +used_storages: [] +used_table_functions: [] +thread_ids: [66,225,283,281,282] +ProfileEvents: {'Query':1,'SelectQuery':1,'IOBufferAllocs':3,'IOBufferAllocBytes':3145728,'SelectedRows':1,'SelectedBytes':1,'ContextLock':17,'RWLockAcquiredReadLocks':1,'RealTimeMicroseconds':4372,'UserTimeMicroseconds':1022,'SystemTimeMicroseconds':177,'OSCPUWaitMicroseconds':2070,'OSCPUVirtualTimeMicroseconds':1198,'OSWriteBytes':4096,'OSReadChars':848,'OSWriteChars':1190} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 6: +────── +type: QueryFinish +query_start_time: 2021-12-25 15:25:01 +query_duration_ms: 3 +query_id: b763c2f9-6234-47f7-8b30-43d619909289 +query_kind: Select +is_initial_query: 1 +query: +SELECT formatReadableTimeDelta(uptime()) + + +read: 1 rows / 1.00 B +written: 0 rows / 0.00 B +result: 1 rows / 128.00 B +memory usage: 1.49 KiB +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: ['system'] +tables: ['system.one'] +columns: ['system.one.dummy'] +used_aggregate_functions: [] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['TabSeparated'] +used_functions: ['uptime','formatReadableTimeDelta'] +used_storages: [] +used_table_functions: [] +thread_ids: [66,225,281,283,282] +ProfileEvents: {'Query':1,'SelectQuery':1,'IOBufferAllocs':3,'IOBufferAllocBytes':3145728,'SelectedRows':1,'SelectedBytes':1,'ContextLock':17,'RWLockAcquiredReadLocks':1,'RealTimeMicroseconds':6367,'UserTimeMicroseconds':3329,'SystemTimeMicroseconds':531,'SoftPageFaults':6,'HardPageFaults':1,'OSCPUWaitMicroseconds':1090,'OSCPUVirtualTimeMicroseconds':3859,'OSReadBytes':102400,'OSReadChars':830,'OSWriteChars':1190} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 7: +────── +type: QueryFinish +query_start_time: 2021-12-25 15:26:25 +query_duration_ms: 1 +query_id: e9c25bd1-00d3-4239-9611-1c3d391178da +query_kind: Select +is_initial_query: 1 +query: SELECT version() + +read: 1 rows / 1.00 B +written: 0 rows / 0.00 B +result: 1 rows / 128.00 B +memory usage: 1.45 KiB +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: ['system'] +tables: ['system.one'] +columns: ['system.one.dummy'] +used_aggregate_functions: [] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['TabSeparated'] +used_functions: ['version'] +used_storages: [] +used_table_functions: [] +thread_ids: [66,283,225,282] +ProfileEvents: {'Query':1,'SelectQuery':1,'IOBufferAllocs':3,'IOBufferAllocBytes':3145728,'SelectedRows':1,'SelectedBytes':1,'ContextLock':15,'RWLockAcquiredReadLocks':1,'RealTimeMicroseconds':2720,'UserTimeMicroseconds':648,'SystemTimeMicroseconds':1144,'OSCPUWaitMicroseconds':110,'OSCPUVirtualTimeMicroseconds':1790,'OSReadChars':845,'OSWriteChars':1140} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 8: +────── +type: QueryFinish +query_start_time: 2021-12-25 15:33:29 +query_duration_ms: 4 +query_id: 69762642-8a75-4149-aaf5-bc1969558747 +query_kind: Select +is_initial_query: 1 +query: SELECT version() + +read: 1 rows / 1.00 B +written: 0 rows / 0.00 B +result: 1 rows / 128.00 B +memory usage: 1.45 KiB +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: ['system'] +tables: ['system.one'] +columns: ['system.one.dummy'] +used_aggregate_functions: [] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['TabSeparated'] +used_functions: ['version'] +used_storages: [] +used_table_functions: [] +thread_ids: [66,282,283] +ProfileEvents: {'Query':1,'SelectQuery':1,'IOBufferAllocs':3,'IOBufferAllocBytes':3145728,'SelectedRows':1,'SelectedBytes':1,'ContextLock':15,'RWLockAcquiredReadLocks':1,'RealTimeMicroseconds':10137,'UserTimeMicroseconds':6289,'SystemTimeMicroseconds':47,'SoftPageFaults':2,'HardPageFaults':1,'OSCPUWaitMicroseconds':859,'OSCPUVirtualTimeMicroseconds':6336,'OSReadBytes':12288,'OSReadChars':845,'OSWriteChars':1140} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 9: +─────── +type: QueryFinish +query_start_time: 2021-12-25 15:25:01 +query_duration_ms: 4 +query_id: 9e31242c-62c5-4bb1-9a3e-f96e99f3bddf +query_kind: Select +is_initial_query: 1 +query: SELECT version() + +read: 1 rows / 1.00 B +written: 0 rows / 0.00 B +result: 1 rows / 128.00 B +memory usage: 1.45 KiB +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: ['system'] +tables: ['system.one'] +columns: ['system.one.dummy'] +used_aggregate_functions: [] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['TabSeparated'] +used_functions: ['version'] +used_storages: [] +used_table_functions: [] +thread_ids: [66,225,282,281,283] +ProfileEvents: {'Query':1,'SelectQuery':1,'IOBufferAllocs':3,'IOBufferAllocBytes':3145728,'SelectedRows':1,'SelectedBytes':1,'ContextLock':15,'RWLockAcquiredReadLocks':1,'RealTimeMicroseconds':8688,'UserTimeMicroseconds':3598,'SystemTimeMicroseconds':1288,'SoftPageFaults':42,'HardPageFaults':1,'OSCPUWaitMicroseconds':214,'OSCPUVirtualTimeMicroseconds':4885,'OSReadBytes':98304,'OSReadChars':818,'OSWriteChars':1140} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} + +Row 10: +─────── +type: QueryFinish +query_start_time: 2021-12-25 15:26:26 +query_duration_ms: 2 +query_id: de1fc64c-09c3-420a-8801-a2f9f04407cd +query_kind: Select +is_initial_query: 1 +query: SELECT + database, + table, + count() "partitions", + sum(part_count) "parts", + max(part_count) "max_parts_per_partition" +FROM +( + SELECT + database, + table, + partition, + count() "part_count" + FROM system.parts + WHERE active + GROUP BY database, table, partition +) partitions +GROUP BY database, table +ORDER BY max_parts_per_partition DESC +LIMIT 10 FORMAT PrettyCompactNoEscapes + +read: 12 rows / 643.00 B +written: 0 rows / 0.00 B +result: 6 rows / 752.00 B +memory usage: 0.00 B +exception: +stack_trace: + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: ['system'] +tables: ['system.parts'] +columns: ['system.parts.active','system.parts.database','system.parts.partition','system.parts.table'] +used_aggregate_functions: ['count','max','sum'] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: ['PrettyCompactNoEscapes'] +used_functions: [] +used_storages: [] +used_table_functions: [] +thread_ids: [66] +ProfileEvents: {'Query':1,'SelectQuery':1,'ArenaAllocChunks':2,'ArenaAllocBytes':8192,'SelectedRows':12,'SelectedBytes':643,'ContextLock':58,'RWLockAcquiredReadLocks':9,'RWLockReadersWaitMilliseconds':1,'RealTimeMicroseconds':2924,'UserTimeMicroseconds':1583,'SystemTimeMicroseconds':892,'SoftPageFaults':6,'OSCPUVirtualTimeMicroseconds':3423,'OSReadChars':438,'OSWriteChars':5086} +Settings: {'load_balancing':'random','max_memory_usage':'10000000000'} +``` +##### Last 10 failed queries +**query** +```sql +SELECT + type, + query_start_time, + query_duration_ms, + query_id, + query_kind, + is_initial_query, + query, + concat(toString(read_rows), ' rows / ', formatReadableSize(read_bytes)) AS read, + concat(toString(written_rows), ' rows / ', formatReadableSize(written_bytes)) AS written, + concat(toString(result_rows), ' rows / ', formatReadableSize(result_bytes)) AS result, + formatReadableSize(memory_usage) AS "memory usage", + exception, + '\n' || stack_trace AS stack_trace, + user, + initial_user, + multiIf(empty(client_name), http_user_agent, concat(client_name, ' ', toString(client_version_major), '.', toString(client_version_minor), '.', toString(client_version_patch))) AS client, + client_hostname, + databases, + tables, + columns, + used_aggregate_functions, + used_aggregate_function_combinators, + used_database_engines, + used_data_type_families, + used_dictionaries, + used_formats, + used_functions, + used_storages, + used_table_functions, + thread_ids, + ProfileEvents, + Settings + FROM system.query_log +WHERE type != 'QueryStart' + AND event_date >= today() - 1 + AND event_time >= now() - INTERVAL 1 DAY + AND exception != '' +ORDER BY query_start_time DESC +LIMIT 10 +``` +**result** +``` +Row 1: +────── +type: ExceptionBeforeStart +query_start_time: 2021-12-25 15:33:29 +query_duration_ms: 0 +query_id: 323743ef-4dff-4ed3-9559-f405c64fbd4a +query_kind: Select +is_initial_query: 1 +query: SELECT + '\n' || arrayStringConcat( + arrayMap( + x, + y -> concat(x, ': ', y), + arrayMap(x -> addressToLine(x), trace), + arrayMap(x -> demangle(addressToSymbol(x)), trace)), + '\n') AS trace +FROM system.stack_trace FORMAT Vertical + +read: 0 rows / 0.00 B +written: 0 rows / 0.00 B +result: 0 rows / 0.00 B +memory usage: 0.00 B +exception: Code: 446. DB::Exception: default: Introspection functions are disabled, because setting 'allow_introspection_functions' is set to 0: While processing concat('\n', arrayStringConcat(arrayMap((x, y) -> concat(x, ': ', y), arrayMap(x -> addressToLine(x), trace), arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n')) AS trace. (FUNCTION_NOT_ALLOWED) (version 21.11.8.4 (official build)) +stack_trace: +0. DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int, bool) @ 0x9b682d4 in /usr/bin/clickhouse +1. bool DB::ContextAccess::checkAccessImplHelper(DB::AccessFlags const&) const::'lambda'(std::__1::basic_string, std::__1::allocator > const&, int)::operator()(std::__1::basic_string, std::__1::allocator > const&, int) const @ 0x119786bc in /usr/bin/clickhouse +2. bool DB::ContextAccess::checkAccessImplHelper(DB::AccessFlags const&) const @ 0x11977416 in /usr/bin/clickhouse +3. DB::Context::checkAccess(DB::AccessFlags const&) const @ 0x11eb2f08 in /usr/bin/clickhouse +4. ? @ 0xf96aefb in /usr/bin/clickhouse +5. DB::FunctionFactory::tryGetImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::shared_ptr) const @ 0x118f74b4 in /usr/bin/clickhouse +6. DB::FunctionFactory::getImpl(std::__1::basic_string, std::__1::allocator > const&, std::__1::shared_ptr) const @ 0x118f71fc in /usr/bin/clickhouse +7. DB::ActionsMatcher::visit(DB::ASTFunction const&, std::__1::shared_ptr const&, DB::ActionsMatcher::Data&) @ 0x120c3abf in /usr/bin/clickhouse +8. DB::ActionsMatcher::visit(DB::ASTFunction const&, std::__1::shared_ptr const&, DB::ActionsMatcher::Data&) @ 0x120c6b9f in /usr/bin/clickhouse +9. DB::ActionsMatcher::visit(DB::ASTFunction const&, std::__1::shared_ptr const&, DB::ActionsMatcher::Data&) @ 0x120c41ed in /usr/bin/clickhouse +10. DB::ActionsMatcher::visit(DB::ASTFunction const&, std::__1::shared_ptr const&, DB::ActionsMatcher::Data&) @ 0x120c41ed in /usr/bin/clickhouse +11. DB::ActionsMatcher::visit(DB::ASTFunction const&, std::__1::shared_ptr const&, DB::ActionsMatcher::Data&) @ 0x120c41ed in /usr/bin/clickhouse +12. DB::ActionsMatcher::visit(DB::ASTExpressionList&, std::__1::shared_ptr const&, DB::ActionsMatcher::Data&) @ 0x120ca818 in /usr/bin/clickhouse +13. DB::InDepthNodeVisitor const>::visit(std::__1::shared_ptr const&) @ 0x12099bb7 in /usr/bin/clickhouse +14. DB::ExpressionAnalyzer::getRootActions(std::__1::shared_ptr const&, bool, std::__1::shared_ptr&, bool) @ 0x120999cb in /usr/bin/clickhouse +15. DB::SelectQueryExpressionAnalyzer::appendSelect(DB::ExpressionActionsChain&, bool) @ 0x120a4409 in /usr/bin/clickhouse +16. DB::ExpressionAnalysisResult::ExpressionAnalysisResult(DB::SelectQueryExpressionAnalyzer&, std::__1::shared_ptr const&, bool, bool, bool, std::__1::shared_ptr const&, DB::Block const&) @ 0x120a9070 in /usr/bin/clickhouse +17. DB::InterpreterSelectQuery::getSampleBlockImpl() @ 0x1232fd0d in /usr/bin/clickhouse +18. ? @ 0x12328864 in /usr/bin/clickhouse +19. DB::InterpreterSelectQuery::InterpreterSelectQuery(std::__1::shared_ptr const&, std::__1::shared_ptr, std::__1::optional, std::__1::shared_ptr const&, DB::SelectQueryOptions const&, std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&, std::__1::shared_ptr const&, std::__1::unordered_map, DB::PreparedSetKey::Hash, std::__1::equal_to, std::__1::allocator > > >) @ 0x123232c7 in /usr/bin/clickhouse +20. DB::InterpreterSelectQuery::InterpreterSelectQuery(std::__1::shared_ptr const&, std::__1::shared_ptr, DB::SelectQueryOptions const&, std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&) @ 0x12321c54 in /usr/bin/clickhouse +21. DB::InterpreterSelectWithUnionQuery::buildCurrentChildInterpreter(std::__1::shared_ptr const&, std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&) @ 0x12547fa2 in /usr/bin/clickhouse +22. DB::InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery(std::__1::shared_ptr const&, std::__1::shared_ptr, DB::SelectQueryOptions const&, std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&) @ 0x12546680 in /usr/bin/clickhouse +23. DB::InterpreterFactory::get(std::__1::shared_ptr&, std::__1::shared_ptr, DB::SelectQueryOptions const&) @ 0x122c6216 in /usr/bin/clickhouse +24. ? @ 0x1277dd26 in /usr/bin/clickhouse +25. DB::executeQuery(DB::ReadBuffer&, DB::WriteBuffer&, bool, std::__1::shared_ptr, std::__1::function, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&)>, std::__1::optional const&) @ 0x12781319 in /usr/bin/clickhouse +26. DB::HTTPHandler::processQuery(DB::HTTPServerRequest&, DB::HTMLForm&, DB::HTTPServerResponse&, DB::HTTPHandler::Output&, std::__1::optional&) @ 0x130c20fa in /usr/bin/clickhouse +27. DB::HTTPHandler::handleRequest(DB::HTTPServerRequest&, DB::HTTPServerResponse&) @ 0x130c6760 in /usr/bin/clickhouse +28. DB::HTTPServerConnection::run() @ 0x1312b5e8 in /usr/bin/clickhouse +29. Poco::Net::TCPServerConnection::start() @ 0x15d682cf in /usr/bin/clickhouse +30. Poco::Net::TCPServerDispatcher::run() @ 0x15d6a6c1 in /usr/bin/clickhouse +31. Poco::PooledThread::run() @ 0x15e7f069 in /usr/bin/clickhouse + +user: default +initial_user: default +client: python-requests/2.26.0 +client_hostname: +databases: [] +tables: [] +columns: [] +used_aggregate_functions: [] +used_aggregate_function_combinators: [] +used_database_engines: [] +used_data_type_families: [] +used_dictionaries: [] +used_formats: [] +used_functions: [] +used_storages: [] +used_table_functions: [] +thread_ids: [] +ProfileEvents: {} +Settings: {} + +``` +#### Stack traces +**query** +```sql +SELECT + '\n' || arrayStringConcat( + arrayMap( + x, + y -> concat(x, ': ', y), + arrayMap(x -> addressToLine(x), trace), + arrayMap(x -> demangle(addressToSymbol(x)), trace)), + '\n') AS trace +FROM system.stack_trace +``` +**result** +``` +ClickhouseError("Code: 446. DB::Exception: default: Introspection functions are disabled, because setting 'allow_introspection_functions' is set to 0: While processing concat('\\n', arrayStringConcat(arrayMap((x, y) -> concat(x, ': ', y), arrayMap(x -> addressToLine(x), trace), arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\\n')) AS trace. (FUNCTION_NOT_ALLOWED) (version 21.11.8.4 (official build))",) +``` +#### uname +**command** +``` +uname -a +``` +**result** +``` +Linux clickhouse01 5.10.76-linuxkit #1 SMP Mon Nov 8 10:21:19 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux +``` diff --git a/utils/clickhouse-diagnostics/clickhouse-diagnostics b/utils/clickhouse-diagnostics/clickhouse-diagnostics new file mode 100644 index 00000000000..ffddee0bdc4 --- /dev/null +++ b/utils/clickhouse-diagnostics/clickhouse-diagnostics @@ -0,0 +1,960 @@ +#!/usr/bin/env python3 + +import argparse +import gzip +import io +import json +import socket +import subprocess +import sys +from copy import deepcopy +from datetime import datetime +from typing import MutableMapping + +import jinja2 +import requests +import sqlparse +import tenacity +import xmltodict +import yaml + +SELECT_VERSION = r'SELECT version()' + +SELECT_UPTIME = r''' +{% if version_ge('21.3') -%} +SELECT formatReadableTimeDelta(uptime()) +{% else -%} +SELECT + toString(floor(uptime() / 3600 / 24)) || ' days ' || + toString(floor(uptime() % (24 * 3600) / 3600, 1)) || ' hours' +{% endif -%} +''' + +SELECT_SYSTEM_TABLES = "SELECT name FROM system.tables WHERE database = 'system'" + +SELECT_DATABASE_ENGINES = r'''SELECT + engine, + count() "count" +FROM system.databases +GROUP BY engine +''' + +SELECT_DATABASES = r'''SELECT + name, + engine, + tables, + partitions, + parts, + formatReadableSize(bytes_on_disk) "disk_size" +FROM system.databases db +LEFT JOIN +( + SELECT + database, + uniq(table) "tables", + uniq(table, partition) "partitions", + count() AS parts, + sum(bytes_on_disk) "bytes_on_disk" + FROM system.parts + WHERE active + GROUP BY database +) AS db_stats ON db.name = db_stats.database +ORDER BY bytes_on_disk DESC +LIMIT 10 +''' + +SELECT_TABLE_ENGINES = r'''SELECT + engine, + count() "count" +FROM system.tables +WHERE database != 'system' +GROUP BY engine +''' + +SELECT_DICTIONARIES = r'''SELECT + source, + type, + status, + count() "count" +FROM system.dictionaries +GROUP BY source, type, status +ORDER BY status DESC, source +''' + +SELECT_ACCESS = "SHOW ACCESS" + +SELECT_QUOTA_USAGE = "SHOW QUOTA" + +SELECT_REPLICAS = r'''SELECT + database, + table, + is_leader, + is_readonly, + absolute_delay, + queue_size, + inserts_in_queue, + merges_in_queue +FROM system.replicas +ORDER BY absolute_delay DESC +LIMIT 10 +''' + +SELECT_REPLICATION_QUEUE = r'''SELECT + database, + table, + replica_name, + position, + node_name, + type, + source_replica, + parts_to_merge, + new_part_name, + create_time, + required_quorum, + is_detach, + is_currently_executing, + num_tries, + last_attempt_time, + last_exception, + concat('time: ', toString(last_postpone_time), ', number: ', toString(num_postponed), ', reason: ', postpone_reason) postpone +FROM system.replication_queue +ORDER BY create_time ASC +LIMIT 20 +''' + +SELECT_REPLICATED_FETCHES = r'''SELECT + database, + table, + round(elapsed, 1) "elapsed", + round(100 * progress, 1) "progress", + partition_id, + result_part_name, + result_part_path, + total_size_bytes_compressed, + bytes_read_compressed, + source_replica_path, + source_replica_hostname, + source_replica_port, + interserver_scheme, + to_detached, + thread_id +FROM system.replicated_fetches +''' + +SELECT_PARTS_PER_TABLE = r'''SELECT + database, + table, + count() "partitions", + sum(part_count) "parts", + max(part_count) "max_parts_per_partition" +FROM +( + SELECT + database, + table, + partition, + count() "part_count" + FROM system.parts + WHERE active + GROUP BY database, table, partition +) partitions +GROUP BY database, table +ORDER BY max_parts_per_partition DESC +LIMIT 10 +''' + +SELECT_MERGES = r'''SELECT + database, + table, + round(elapsed, 1) "elapsed", + round(100 * progress, 1) "progress", + is_mutation, + partition_id, +{% if version_ge('20.3') -%} + result_part_path, + source_part_paths, +{% endif -%} + num_parts, + formatReadableSize(total_size_bytes_compressed) "total_size_compressed", + formatReadableSize(bytes_read_uncompressed) "read_uncompressed", + formatReadableSize(bytes_written_uncompressed) "written_uncompressed", + columns_written, +{% if version_ge('20.3') -%} + formatReadableSize(memory_usage) "memory_usage", + thread_id +{% else -%} + formatReadableSize(memory_usage) "memory_usage" +{% endif -%} +FROM system.merges +''' + +SELECT_MUTATIONS = r'''SELECT + database, + table, + mutation_id, + command, + create_time, +{% if version_ge('20.3') -%} + parts_to_do_names, +{% endif -%} + parts_to_do, + is_done, + latest_failed_part, + latest_fail_time, + latest_fail_reason +FROM system.mutations +WHERE NOT is_done +ORDER BY create_time DESC +''' + +SELECT_RECENT_DATA_PARTS = r'''SELECT + database, + table, + engine, + partition_id, + name, +{% if version_ge('20.3') -%} + part_type, +{% endif -%} + active, + level, +{% if version_ge('20.3') -%} + disk_name, +{% endif -%} + path, + marks, + rows, + bytes_on_disk, + data_compressed_bytes, + data_uncompressed_bytes, + marks_bytes, + modification_time, + remove_time, + refcount, + is_frozen, + min_date, + max_date, + min_time, + max_time, + min_block_number, + max_block_number +FROM system.parts +WHERE modification_time > now() - INTERVAL 3 MINUTE +ORDER BY modification_time DESC +''' + +SELECT_DETACHED_DATA_PARTS = r'''SELECT + database, + table, + partition_id, + name, + disk, + reason, + min_block_number, + max_block_number, + level +FROM system.detached_parts +''' + +SELECT_PROCESSES = r'''SELECT + elapsed, + query_id, + {% if normalize_queries -%} + normalizeQuery(query) AS normalized_query, + {% else -%} + query, + {% endif -%} + is_cancelled, + concat(toString(read_rows), ' rows / ', formatReadableSize(read_bytes)) AS read, + concat(toString(written_rows), ' rows / ', formatReadableSize(written_bytes)) AS written, + formatReadableSize(memory_usage) AS "memory usage", + user, + multiIf(empty(client_name), http_user_agent, concat(client_name, ' ', toString(client_version_major), '.', toString(client_version_minor), '.', toString(client_version_patch))) AS client, + {% if version_ge('21.3') -%} + thread_ids, + {% endif -%} + {% if version_ge('21.8') -%} + ProfileEvents, + Settings + {% else -%} + ProfileEvents.Names, + ProfileEvents.Values, + Settings.Names, + Settings.Values + {% endif -%} +FROM system.processes +ORDER BY elapsed DESC +''' + +SELECT_TOP_QUERIES_BY_DURATION = r'''SELECT + type, + query_start_time, + query_duration_ms, + query_id, + query_kind, + is_initial_query, + {% if normalize_queries -%} + normalizeQuery(query) AS normalized_query, + {% else -%} + query, + {% endif -%} + concat(toString(read_rows), ' rows / ', formatReadableSize(read_bytes)) AS read, + concat(toString(written_rows), ' rows / ', formatReadableSize(written_bytes)) AS written, + concat(toString(result_rows), ' rows / ', formatReadableSize(result_bytes)) AS result, + formatReadableSize(memory_usage) AS "memory usage", + exception, + '\n' || stack_trace AS stack_trace, + user, + initial_user, + multiIf(empty(client_name), http_user_agent, concat(client_name, ' ', toString(client_version_major), '.', toString(client_version_minor), '.', toString(client_version_patch))) AS client, + client_hostname, + {% if version_ge('21.3') -%} + databases, + tables, + columns, + used_aggregate_functions, + used_aggregate_function_combinators, + used_database_engines, + used_data_type_families, + used_dictionaries, + used_formats, + used_functions, + used_storages, + used_table_functions, + thread_ids, + {% endif -%} + {% if version_ge('21.8') -%} + ProfileEvents, + Settings + {% else -%} + ProfileEvents.Names, + ProfileEvents.Values, + Settings.Names, + Settings.Values + {% endif -%} +FROM system.query_log +WHERE type != 'QueryStart' + AND event_date >= today() - 1 + AND event_time >= now() - INTERVAL 1 DAY +ORDER BY query_duration_ms DESC +LIMIT 10 +''' + +SELECT_TOP_QUERIES_BY_MEMORY_USAGE = r'''SELECT + type, + query_start_time, + query_duration_ms, + query_id, + query_kind, + is_initial_query, + {% if normalize_queries -%} + normalizeQuery(query) AS normalized_query, + {% else -%} + query, + {% endif -%} + concat(toString(read_rows), ' rows / ', formatReadableSize(read_bytes)) AS read, + concat(toString(written_rows), ' rows / ', formatReadableSize(written_bytes)) AS written, + concat(toString(result_rows), ' rows / ', formatReadableSize(result_bytes)) AS result, + formatReadableSize(memory_usage) AS "memory usage", + exception, + '\n' || stack_trace AS stack_trace, + user, + initial_user, + multiIf(empty(client_name), http_user_agent, concat(client_name, ' ', toString(client_version_major), '.', toString(client_version_minor), '.', toString(client_version_patch))) AS client, + client_hostname, + {% if version_ge('21.3') -%} + databases, + tables, + columns, + used_aggregate_functions, + used_aggregate_function_combinators, + used_database_engines, + used_data_type_families, + used_dictionaries, + used_formats, + used_functions, + used_storages, + used_table_functions, + thread_ids, + {% endif -%} + {% if version_ge('21.8') -%} + ProfileEvents, + Settings + {% else -%} + ProfileEvents.Names, + ProfileEvents.Values, + Settings.Names, + Settings.Values + {% endif -%} +FROM system.query_log +WHERE type != 'QueryStart' + AND event_date >= today() - 1 + AND event_time >= now() - INTERVAL 1 DAY +ORDER BY memory_usage DESC +LIMIT 10 +''' + +SELECT_FAILED_QUERIES = r'''SELECT + type, + query_start_time, + query_duration_ms, + query_id, + query_kind, + is_initial_query, + {% if normalize_queries -%} + normalizeQuery(query) AS normalized_query, + {% else -%} + query, + {% endif -%} + concat(toString(read_rows), ' rows / ', formatReadableSize(read_bytes)) AS read, + concat(toString(written_rows), ' rows / ', formatReadableSize(written_bytes)) AS written, + concat(toString(result_rows), ' rows / ', formatReadableSize(result_bytes)) AS result, + formatReadableSize(memory_usage) AS "memory usage", + exception, + '\n' || stack_trace AS stack_trace, + user, + initial_user, + multiIf(empty(client_name), http_user_agent, concat(client_name, ' ', toString(client_version_major), '.', toString(client_version_minor), '.', toString(client_version_patch))) AS client, + client_hostname, + {% if version_ge('21.3') -%} + databases, + tables, + columns, + used_aggregate_functions, + used_aggregate_function_combinators, + used_database_engines, + used_data_type_families, + used_dictionaries, + used_formats, + used_functions, + used_storages, + used_table_functions, + thread_ids, + {% endif -%} + {% if version_ge('21.8') -%} + ProfileEvents, + Settings + {% else -%} + ProfileEvents.Names, + ProfileEvents.Values, + Settings.Names, + Settings.Values + {% endif -%} +FROM system.query_log +WHERE type != 'QueryStart' + AND event_date >= today() - 1 + AND event_time >= now() - INTERVAL 1 DAY + AND exception != '' +ORDER BY query_start_time DESC +LIMIT 10 +''' + +SELECT_STACK_TRACES = r'''SELECT + '\n' || arrayStringConcat( + arrayMap( + x, + y -> concat(x, ': ', y), + arrayMap(x -> addressToLine(x), trace), + arrayMap(x -> demangle(addressToSymbol(x)), trace)), + '\n') AS trace +FROM system.stack_trace +''' + +SELECT_CRASH_LOG = r'''SELECT + event_time, + signal, + thread_id, + query_id, + '\n' || arrayStringConcat(trace_full, '\n') AS trace, + version +FROM system.crash_log +ORDER BY event_time DESC +''' + + +def retry(exception_types, max_attempts=5, max_interval=5): + """ + Function decorator that retries wrapped function on failures. + """ + return tenacity.retry( + retry=tenacity.retry_if_exception_type(exception_types), + wait=tenacity.wait_random_exponential(multiplier=0.5, max=max_interval), + stop=tenacity.stop_after_attempt(max_attempts), + reraise=True) + + +class ClickhouseError(Exception): + """ + ClickHouse interaction error. + """ + + def __init__(self, response): + self.response = response + super().__init__(self.response.text.strip()) + + +class ClickhouseClient: + """ + ClickHouse client. + """ + + def __init__(self, *, host, port=8123, user=None): + self._session = requests.Session() + if user: + self._session.headers['X-ClickHouse-User'] = user + self._url = f'http://{host}:{port}' + self._timeout = 60 + self._ch_version = None + + @property + def clickhouse_version(self): + if self._ch_version is None: + self._ch_version = self.query(SELECT_VERSION) + + return self._ch_version + + @retry(requests.exceptions.ConnectionError) + def query(self, query, query_args=None, format=None, post_data=None, timeout=None, echo=False, dry_run=False): + """ + Execute query. + """ + if query_args: + query = self.render_query(query, **query_args) + + if format: + query += f' FORMAT {format}' + + if timeout is None: + timeout = self._timeout + + if echo: + print(sqlparse.format(query, reindent=True), '\n') + + if dry_run: + return None + + try: + response = self._session.post(self._url, + params={ + 'query': query, + }, + json=post_data, + timeout=timeout) + + response.raise_for_status() + + if format in ('JSON', 'JSONCompact'): + return response.json() + + return response.text.strip() + except requests.exceptions.HTTPError as e: + raise ClickhouseError(e.response) from None + + def render_query(self, query, **kwargs): + env = jinja2.Environment() + + env.globals['version_ge'] = lambda version: version_ge(self.clickhouse_version, version) + + template = env.from_string(query) + return template.render(kwargs) + + +class ClickhouseConfig: + """ + ClickHouse server configuration. + """ + + def __init__(self, config): + self._config = config + + def dump(self, mask_secrets=True): + config = deepcopy(self._config) + if mask_secrets: + self._mask_secrets(config) + + return xmltodict.unparse(config, pretty=True) + + @classmethod + def load(cls): + return ClickhouseConfig(cls._load_config('/var/lib/clickhouse/preprocessed_configs/config.xml')) + + @staticmethod + def _load_config(config_path): + with open(config_path, 'r') as file: + return xmltodict.parse(file.read()) + + @classmethod + def _mask_secrets(cls, config): + if isinstance(config, MutableMapping): + for key, value in list(config.items()): + if isinstance(value, MutableMapping): + cls._mask_secrets(config[key]) + elif key in ('password', 'secret_access_key', 'header', 'identity'): + config[key] = '*****' + + +class DiagnosticsData: + """ + Diagnostics data. + """ + + def __init__(self, args, host): + self.args = args + self.host = host + self._sections = [{'section': None, 'data': {}}] + + def add_string(self, name, value, section=None): + self._section(section)[name] = { + 'type': 'string', + 'value': value, + } + + def add_xml_document(self, name, document, section=None): + self._section(section)[name] = { + 'type': 'xml', + 'value': document, + } + + def add_query(self, name, query, result, section=None): + self._section(section)[name] = { + 'type': 'query', + 'query': query, + 'result': result, + } + + def add_command(self, name, command, result, section=None): + self._section(section)[name] = { + 'type': 'command', + 'command': command, + 'result': result, + } + + def dump(self, format): + if format.startswith('json'): + result = self._dump_json() + elif format.startswith('yaml'): + result = self._dump_yaml() + else: + result = self._dump_wiki() + + if format.endswith('.gz'): + compressor = gzip.GzipFile(mode='wb', fileobj=sys.stdout.buffer) + compressor.write(result.encode()) + else: + print(result) + + def _section(self, name=None): + if self._sections[-1]['section'] != name: + self._sections.append({'section': name, 'data': {}}) + + return self._sections[-1]['data'] + + def _dump_json(self): + """ + Dump diagnostic data in JSON format. + """ + return json.dumps(self._sections, indent=2, ensure_ascii=False) + + def _dump_yaml(self): + """ + Dump diagnostic data in YAML format. + """ + return yaml.dump(self._sections, default_flow_style=False, allow_unicode=True) + + def _dump_wiki(self): + """ + Dump diagnostic data in Yandex wiki format. + """ + + def _write_title(buffer, value): + buffer.write(f'### {value}\n') + + def _write_subtitle(buffer, value): + buffer.write(f'#### {value}\n') + + def _write_string_item(buffer, name, item): + value = item['value'] + if value != '': + value = f'**{value}**' + buffer.write(f'{name}: {value}\n') + + def _write_xml_item(buffer, section_name, name, item): + if section_name: + buffer.write(f'##### {name}\n') + else: + _write_subtitle(buffer, name) + + _write_result(buffer, item['value'], format='XML') + + def _write_query_item(buffer, section_name, name, item): + if section_name: + buffer.write(f'##### {name}\n') + else: + _write_subtitle(buffer, name) + + _write_query(buffer, item['query']) + _write_result(buffer, item['result']) + + def _write_command_item(buffer, section_name, name, item): + if section_name: + buffer.write(f'##### {name}\n') + else: + _write_subtitle(buffer, name) + + _write_command(buffer, item['command']) + _write_result(buffer, item['result']) + + def _write_unknown_item(buffer, section_name, name, item): + if section_name: + buffer.write(f'**{name}**\n') + else: + _write_subtitle(buffer, name) + + json.dump(item, buffer, indent=2) + + def _write_query(buffer, query): + buffer.write('**query**\n') + buffer.write('```sql\n') + buffer.write(query) + buffer.write('\n```\n') + + def _write_command(buffer, command): + buffer.write('**command**\n') + buffer.write('```\n') + buffer.write(command) + buffer.write('\n```\n') + + def _write_result(buffer, result, format=None): + buffer.write('**result**\n') + buffer.write(f'```{format}\n' if format else '```\n') + buffer.write(result) + buffer.write('\n```\n') + + buffer = io.StringIO() + + _write_title(buffer, f'Diagnostics data for host {self.host}') + for section in self._sections: + section_name = section['section'] + if section_name: + _write_subtitle(buffer, section_name) + + for name, item in section['data'].items(): + if item['type'] == 'string': + _write_string_item(buffer, name, item) + elif item['type'] == 'query': + _write_query_item(buffer, section_name, name, item) + elif item['type'] == 'command': + _write_command_item(buffer, section_name, name, item) + elif item['type'] == 'xml': + _write_xml_item(buffer, section_name, name, item) + else: + _write_unknown_item(buffer, section_name, name, item) + + return buffer.getvalue() + + +def main(): + """ + Program entry point. + """ + args = parse_args() + + host = socket.getfqdn() + timestamp = datetime.strftime(datetime.now(), '%Y-%m-%d %H:%M:%S') + client = ClickhouseClient(host=host) + ch_config = ClickhouseConfig.load() + version = client.clickhouse_version + system_tables = [row[0] for row in execute_query(client, SELECT_SYSTEM_TABLES, format='JSONCompact')['data']] + + diagnostics = DiagnosticsData(args, host) + diagnostics.add_string('Version', version) + diagnostics.add_string('Timestamp', timestamp) + diagnostics.add_string('Uptime', execute_query(client, SELECT_UPTIME)) + + diagnostics.add_xml_document('ClickHouse configuration', ch_config.dump()) + + if version_ge(version, '20.8'): + add_query(diagnostics, 'Access configuration', + client=client, + query=SELECT_ACCESS, + format='TSVRaw') + add_query(diagnostics, 'Quotas', + client=client, + query=SELECT_QUOTA_USAGE, + format='Vertical') + + add_query(diagnostics, 'Database engines', + client=client, + query=SELECT_DATABASE_ENGINES, + format='PrettyCompactNoEscapes', + section='Schema') + add_query(diagnostics, 'Databases (top 10 by size)', + client=client, + query=SELECT_DATABASES, + format='PrettyCompactNoEscapes', + section='Schema') + add_query(diagnostics, 'Table engines', + client=client, + query=SELECT_TABLE_ENGINES, + format='PrettyCompactNoEscapes', + section='Schema') + add_query(diagnostics, 'Dictionaries', + client=client, + query=SELECT_DICTIONARIES, + format='PrettyCompactNoEscapes', + section='Schema') + + add_query(diagnostics, 'Replicated tables (top 10 by absolute delay)', + client=client, + query=SELECT_REPLICAS, + format='PrettyCompactNoEscapes', + section='Replication') + add_query(diagnostics, 'Replication queue (top 20 oldest tasks)', + client=client, + query=SELECT_REPLICATION_QUEUE, + format='Vertical', + section='Replication') + if version_ge(version, '21.3'): + add_query(diagnostics, 'Replicated fetches', + client=client, + query=SELECT_REPLICATED_FETCHES, + format='Vertical', + section='Replication') + + add_query(diagnostics, 'Top 10 tables by max parts per partition', + client=client, + query=SELECT_PARTS_PER_TABLE, + format='PrettyCompactNoEscapes') + add_query(diagnostics, 'Merges in progress', + client=client, + query=SELECT_MERGES, + format='Vertical') + add_query(diagnostics, 'Mutations in progress', + client=client, + query=SELECT_MUTATIONS, + format='Vertical') + add_query(diagnostics, 'Recent data parts (modification time within last 3 minutes)', + client=client, + query=SELECT_RECENT_DATA_PARTS, + format='Vertical') + + add_query(diagnostics, 'system.detached_parts', + client=client, + query=SELECT_DETACHED_DATA_PARTS, + format='PrettyCompactNoEscapes', + section='Detached data') + add_command(diagnostics, 'Disk space usage', + command='du -sh -L -c /var/lib/clickhouse/data/*/*/detached/* | sort -rsh', + section='Detached data') + + add_query(diagnostics, 'Queries in progress (process list)', + client=client, + query=SELECT_PROCESSES, + format='Vertical', + section='Queries') + add_query(diagnostics, 'Top 10 queries by duration', + client=client, + query=SELECT_TOP_QUERIES_BY_DURATION, + format='Vertical', + section='Queries') + add_query(diagnostics, 'Top 10 queries by memory usage', + client=client, + query=SELECT_TOP_QUERIES_BY_MEMORY_USAGE, + format='Vertical', + section='Queries') + add_query(diagnostics, 'Last 10 failed queries', + client=client, + query=SELECT_FAILED_QUERIES, + format='Vertical', + section='Queries') + + add_query(diagnostics, 'Stack traces', + client=client, + query=SELECT_STACK_TRACES, + format='Vertical') + + if 'crash_log' in system_tables: + add_query(diagnostics, 'Crash log', + client=client, + query=SELECT_CRASH_LOG, + format='Vertical') + + add_command(diagnostics, 'uname', 'uname -a') + + diagnostics.dump(args.format) + + +def parse_args(): + """ + Parse command-line arguments. + """ + parser = argparse.ArgumentParser() + parser.add_argument('--format', + choices=['json', 'yaml', 'json.gz', 'yaml.gz', 'wiki', 'wiki.gz'], + default='wiki') + parser.add_argument('--normalize-queries', + action='store_true', + default=False) + return parser.parse_args() + + +def add_query(diagnostics, name, client, query, format, section=None): + query_args = { + 'normalize_queries': diagnostics.args.normalize_queries, + } + query = client.render_query(query, **query_args) + diagnostics.add_query( + name=name, + query=query, + result=execute_query(client, query, render_query=False, format=format), + section=section) + + +def execute_query(client, query, render_query=True, format=None): + if render_query: + query = client.render_query(query) + + try: + return client.query(query, format=format) + except Exception as e: + return repr(e) + + +def add_command(diagnostics, name, command, section=None): + diagnostics.add_command( + name=name, + command=command, + result=execute_command(command), + section=section) + + +def execute_command(command, input=None): + proc = subprocess.Popen(command, shell=True, stdin=subprocess.PIPE, stdout=subprocess.PIPE, stderr=subprocess.PIPE) + + if isinstance(input, str): + input = input.encode() + + stdout, stderr = proc.communicate(input=input) + + if proc.returncode: + return f'failed with exit code {proc.returncode}\n{stderr.decode()}' + + return stdout.decode() + + +def version_ge(version1, version2): + """ + Return True if version1 is greater or equal than version2. + """ + return parse_version(version1) >= parse_version(version2) + + +def parse_version(version): + """ + Parse version string. + """ + return [int(x) for x in version.strip().split('.')] + + +if __name__ == '__main__': + main() diff --git a/utils/clickhouse-diagnostics/requirements.txt b/utils/clickhouse-diagnostics/requirements.txt new file mode 100644 index 00000000000..1d2b6ef3916 --- /dev/null +++ b/utils/clickhouse-diagnostics/requirements.txt @@ -0,0 +1,6 @@ +Jinja2 +PyYAML +requests +sqlparse +tenacity +xmltodict From f78678dc3f90fcedfc865abba014b50913b1b507 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Dec 2021 16:01:35 +0300 Subject: [PATCH 0363/1260] Great simplification of test 01622_defaults_for_url_engine --- .../01622_defaults_for_url_engine.reference | 2 +- .../01622_defaults_for_url_engine.sh | 36 ++++--------------- 2 files changed, 7 insertions(+), 31 deletions(-) diff --git a/tests/queries/0_stateless/01622_defaults_for_url_engine.reference b/tests/queries/0_stateless/01622_defaults_for_url_engine.reference index 7326d960397..c0a49154cd9 100644 --- a/tests/queries/0_stateless/01622_defaults_for_url_engine.reference +++ b/tests/queries/0_stateless/01622_defaults_for_url_engine.reference @@ -1 +1 @@ -Ok +(1,7,8) \ No newline at end of file diff --git a/tests/queries/0_stateless/01622_defaults_for_url_engine.sh b/tests/queries/0_stateless/01622_defaults_for_url_engine.sh index 0a93a3ef479..cf89a624bc9 100755 --- a/tests/queries/0_stateless/01622_defaults_for_url_engine.sh +++ b/tests/queries/0_stateless/01622_defaults_for_url_engine.sh @@ -1,37 +1,13 @@ #!/usr/bin/env bash # Tags: no-fasttest -# Tag no-fasttest: nc - command not found CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh - -PORT="$(($RANDOM%63000+2001))" - -function thread1 -{ - while true; do - echo -e "HTTP/1.1 200 OK\n\n{\"a\": 1}" | nc -l -p $1 -q 1; - done -} - -function thread2 -{ - while true; do - $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 -q "SELECT * FROM url('http://127.0.0.1:$1/', JSONEachRow, 'a int, b int default 7, c default a + b') format Values" | grep -F '(1,7,8)' && break - done -} - -# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout -export -f thread1; -export -f thread2; - -TIMEOUT=60 - -timeout $TIMEOUT bash -c "thread1 $PORT" > /dev/null 2>&1 & -PID=$! - -bash -c "thread2 $PORT" 2> /dev/null | grep -q -F '(1,7,8)' && echo "Ok" && kill -9 $PID - -wait >/dev/null 2>&1 +${CLICKHOUSE_CLIENT} --query " +SELECT * FROM url( + \$\$http://127.0.0.1:${CLICKHOUSE_PORT_HTTP}/?query=SELECT+'{\"a\":1}'\$\$, + JSONEachRow, + 'a int, b int default 7, c default a + b') +FORMAT Values" From 688cffb8becadd5f85a3d9bc2d7786ebb2751589 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 25 Dec 2021 16:37:44 +0300 Subject: [PATCH 0364/1260] Update postgres doc --- .../materialized-postgresql.md | 118 ++++++++++++------ .../integrations/materialized-postgresql.md | 4 +- .../table-engines/integrations/postgresql.md | 25 ++++ docs/en/operations/settings/settings.md | 53 +------- 4 files changed, 109 insertions(+), 91 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index d2c4dbf1f3c..329034b70bb 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -5,15 +5,15 @@ toc_title: MaterializedPostgreSQL # [experimental] MaterializedPostgreSQL {#materialize-postgresql} -Creates ClickHouse database with an initial data dump of PostgreSQL database tables and starts replication process, i.e. executes background job to apply new changes as they happen on PostgreSQL database tables in the remote PostgreSQL database. +Creates a ClickHouse database with tables from PostgreSQL database. Firstly, database with engine `MaterializedPostgreSQL` creates a snapshot of PostgreSQL database and loads required tables. Required tables can include any subset of tables from any subset of schemas from specified database. Along with the snapshot database engine acquires LSN and once initial dump of tables is performed - it starts pulling updates from WAL. After database is created, newly added tables to PostgreSQL database are not automatically added to replication. They have to be added manually with `ATTACH TABLE db.table` query. -ClickHouse server works as PostgreSQL replica. It reads WAL and performs DML queries. DDL is not replicated, but can be handled (described below). +Replication is implemented with PostgreSQL Logical Replication Protocol, which does not allow to replicate DDL, but allows to know whether replication breaking changes happened (column type changes, adding/removing columns). Such changes are detected and according tables stop receiving updates. Such tables can be automatically reloaded in the background in case required setting is turned on. Safest way for now is to use `ATTACH`/ `DETACH` queries to reload table completely. If DDL does not break replication (for example, renaming a column) table will still receive updates (insertion is done by position). ## Creating a Database {#creating-a-database} ``` sql CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] -ENGINE = MaterializedPostgreSQL('host:port', ['database' | database], 'user', 'password') [SETTINGS ...] +ENGINE = MaterializedPostgreSQL('host:port', 'database', 'user', 'password') [SETTINGS ...] ``` **Engine Parameters** @@ -23,51 +23,37 @@ ENGINE = MaterializedPostgreSQL('host:port', ['database' | database], 'user', 'p - `user` — PostgreSQL user. - `password` — User password. +## Example of Use {#example-of-use} + +``` sql +CREATE DATABASE postgresql; +ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password'); + +SHOW TABLES FROM postgres_db; + +┌─name───┐ +│ table1 │ +└────────┘ + +SELECT * FROM postgresql_db.postgres_table; +``` + ## Dynamically adding new tables to replication {#dynamically-adding-table-to-replication} +After `MaterializedPostgreSQL` database is created, it does not automatically detect new tables in according PostgreSQL database. Such tables can be added manually: + ``` sql ATTACH TABLE postgres_database.new_table; ``` -When specifying a specific list of tables in the database using the setting [materialized_postgresql_tables_list](../../operations/settings/settings.md#materialized-postgresql-tables-list), it will be updated to the current state, taking into account the tables which were added by the `ATTACH TABLE` query. - ## Dynamically removing tables from replication {#dynamically-removing-table-from-replication} +It is possible to remove specific tables from replication: + ``` sql DETACH TABLE postgres_database.table_to_remove; ``` -## Settings {#settings} - -- [materialized_postgresql_tables_list](../../operations/settings/settings.md#materialized-postgresql-tables-list) - -- [materialized_postgresql_schema](../../operations/settings/settings.md#materialized-postgresql-schema) - -- [materialized_postgresql_schema_list](../../operations/settings/settings.md#materialized-postgresql-schema-list) - -- [materialized_postgresql_allow_automatic_update](../../operations/settings/settings.md#materialized-postgresql-allow-automatic-update) - -- [materialized_postgresql_max_block_size](../../operations/settings/settings.md#materialized-postgresql-max-block-size) - -- [materialized_postgresql_replication_slot](../../operations/settings/settings.md#materialized-postgresql-replication-slot) - -- [materialized_postgresql_snapshot](../../operations/settings/settings.md#materialized-postgresql-snapshot) - -``` sql -CREATE DATABASE database1 -ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password') -SETTINGS materialized_postgresql_tables_list = 'table1,table2,table3'; - -SELECT * FROM database1.table1; -``` - -The settings can be changed, if necessary, using a DDL query. But it is impossible to change the setting `materialized_postgresql_tables_list`. To update the list of tables in this setting use the `ATTACH TABLE` query. - -``` sql -ALTER DATABASE postgres_database MODIFY SETTING materialized_postgresql_max_block_size = ; -``` - - ## PostgreSQL schema {#schema} PostgreSQL [schema](https://www.postgresql.org/docs/9.1/ddl-schemas.html) can be configured in 3 ways (starting from version 21.12). @@ -150,13 +136,63 @@ WHERE oid = 'postgres_table'::regclass; !!! warning "Warning" Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.html) values is not supported. The default value for the data type will be used. -## Example of Use {#example-of-use} +## Settings {#settings} + +1. materialized_postgresql_tables_list {#materialized-postgresql-tables-list} + +Sets a comma-separated list of PostgreSQL database tables, which will be replicated via [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) database engine. + +Default value: empty list — means whole PostgreSQL database will be replicated. + +2. materialized_postgresql_schema {#materialized-postgresql-schema} + +Default value: empty string. (Default schema is used) + +3. materialized_postgresql_schema_list {#materialized-postgresql-schema-list} + +Default value: empty list. (Default schema is used) + +4. materialized_postgresql_allow_automatic_update {#materialized-postgresql-allow-automatic-update} + +Allows reloading table in the background, when schema changes are detected. DDL queries on the PostgreSQL side are not replicated via ClickHouse [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) engine, because it is not allowed with PostgreSQL logical replication protocol, but the fact of DDL changes is detected transactionally. In this case, the default behaviour is to stop replicating those tables once DDL is detected. However, if this setting is enabled, then, instead of stopping the replication of those tables, they will be reloaded in the background via database snapshot without data losses and replication will continue for them. + +Possible values: + +- 0 — The table is not automatically updated in the background, when schema changes are detected. +- 1 — The table is automatically updated in the background, when schema changes are detected. + +Default value: `0`. + +5. materialized_postgresql_max_block_size {#materialized-postgresql-max-block-size} + +Sets the number of rows collected in memory before flushing data into PostgreSQL database table. + +Possible values: + +- Positive integer. + +Default value: `65536`. + +6. materialized_postgresql_replication_slot {#materialized-postgresql-replication-slot} + +A user-created replication slot. Must be used together with [materialized_postgresql_snapshot](#materialized-postgresql-snapshot). + +7. materialized_postgresql_snapshot {#materialized-postgresql-snapshot} + +A text string identifying a snapshot, from which [initial dump of PostgreSQL tables](../../engines/database-engines/materialized-postgresql.md) will be performed. Must be used together with [materialized_postgresql_replication_slot](#materialized-postgresql-replication-slot). ``` sql -CREATE DATABASE postgresql_db -ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password'); +CREATE DATABASE database1 +ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password') +SETTINGS materialized_postgresql_tables_list = 'table1,table2,table3'; -SELECT * FROM postgresql_db.postgres_table; +SELECT * FROM database1.table1; +``` + +The settings can be changed, if necessary, using a DDL query. But it is impossible to change the setting `materialized_postgresql_tables_list`. To update the list of tables in this setting use the `ATTACH TABLE` query. + +``` sql +ALTER DATABASE postgres_database MODIFY SETTING materialized_postgresql_max_block_size = ; ``` ## Notes {#notes} @@ -169,7 +205,7 @@ A solution to this is to manage replication slots yourself and define a permanen Please note that this should be used only if it is actually needed. If there is no real need for that or full understanding why, then it is better to allow the table engine to create and manage its own replication slot. -**Example (from [@bchrobot](https://github.com/bchrobot))** +**Example (from [@bchrobot](https://github.com/bchrobot))** 1. Configure replication slot in PostgreSQL. diff --git a/docs/en/engines/table-engines/integrations/materialized-postgresql.md b/docs/en/engines/table-engines/integrations/materialized-postgresql.md index d02a11257c2..fa349e49af5 100644 --- a/docs/en/engines/table-engines/integrations/materialized-postgresql.md +++ b/docs/en/engines/table-engines/integrations/materialized-postgresql.md @@ -7,7 +7,7 @@ toc_title: MaterializedPostgreSQL Creates ClickHouse table with an initial data dump of PostgreSQL table and starts replication process, i.e. executes background job to apply new changes as they happen on PostgreSQL table in the remote PostgreSQL database. -If more than one table is required, it is highly recommended to use the [MaterializedPostgreSQL](../../../engines/database-engines/materialized-postgresql.md) database engine instead of the table engine and use the [materialized_postgresql_tables_list](../../../operations/settings/settings.md#materialized-postgresql-tables-list) setting, which specifies the tables to be replicated. It will be much better in terms of CPU, fewer connections and fewer replication slots inside the remote PostgreSQL database. +If more than one table is required, it is highly recommended to use the [MaterializedPostgreSQL](../../../engines/database-engines/materialized-postgresql.md) database engine instead of the table engine and use the `materialized_postgresql_tables_list` setting, which specifies the tables to be replicated (will also be possible to add database `schema`). It will be much better in terms of CPU, fewer connections and fewer replication slots inside the remote PostgreSQL database. ## Creating a Table {#creating-a-table} @@ -38,7 +38,7 @@ PRIMARY KEY key; - `_version` — Transaction counter. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). - `_sign` — Deletion mark. Type: [Int8](../../../sql-reference/data-types/int-uint.md). Possible values: - - `1` — Row is not deleted, + - `1` — Row is not deleted, - `-1` — Row is deleted. These columns do not need to be added when a table is created. They are always accessible in `SELECT` query. diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index 0bdb54e0c16..789759ec521 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -36,6 +36,31 @@ The table structure can differ from the original PostgreSQL table structure: - `schema` — Non-default table schema. Optional. - `on conflict ...` — example: `ON CONFLICT DO NOTHING`. Optional. Note: adding this option will make insertion less efficient. +or via config (since version 21.11): + +``` + + + + + + +
+
+ + + + + + +
+``` + +Some parameters can be overriden by key value arguments: +``` sql +SELECT * FROM postgresql(postgres1, schema='schema1', table='table1'); +``` + ## Implementation Details {#implementation-details} `SELECT` queries on PostgreSQL side run as `COPY (SELECT ...) TO STDOUT` inside read-only PostgreSQL transaction with commit after each `SELECT` query. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e56625fe948..9b4db0e026e 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1469,7 +1469,7 @@ Possible values: Default value: `1`. -**See Also** +**See Also** - [min_count_to_compile_aggregate_expression](#min_count_to_compile_aggregate_expression) @@ -2095,7 +2095,7 @@ Possible values: - 0 — Optimization disabled. - 1 — Optimization enabled. - + Default value: `1`. See also: @@ -3682,49 +3682,6 @@ Possible values: Default value: `0`. -## materialized_postgresql_max_block_size {#materialized-postgresql-max-block-size} - -Sets the number of rows collected in memory before flushing data into PostgreSQL database table. - -Possible values: - -- Positive integer. - -Default value: `65536`. - -## materialized_postgresql_tables_list {#materialized-postgresql-tables-list} - -Sets a comma-separated list of PostgreSQL database tables, which will be replicated via [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) database engine. - -Default value: empty list — means whole PostgreSQL database will be replicated. - -## materialized_postgresql_schema {#materialized-postgresql-schema} - -Default value: empty string. (Default schema is used) - -## materialized_postgresql_schema_list {#materialized-postgresql-schema-list} - -Default value: empty list. (Default schema is used) - -## materialized_postgresql_allow_automatic_update {#materialized-postgresql-allow-automatic-update} - -Allows reloading table in the background, when schema changes are detected. DDL queries on the PostgreSQL side are not replicated via ClickHouse [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) engine, because it is not allowed with PostgreSQL logical replication protocol, but the fact of DDL changes is detected transactionally. In this case, the default behaviour is to stop replicating those tables once DDL is detected. However, if this setting is enabled, then, instead of stopping the replication of those tables, they will be reloaded in the background via database snapshot without data losses and replication will continue for them. - -Possible values: - -- 0 — The table is not automatically updated in the background, when schema changes are detected. -- 1 — The table is automatically updated in the background, when schema changes are detected. - -Default value: `0`. - -## materialized_postgresql_replication_slot {#materialized-postgresql-replication-slot} - -A user-created replication slot. Must be used together with [materialized_postgresql_snapshot](#materialized-postgresql-snapshot). - -## materialized_postgresql_snapshot {#materialized-postgresql-snapshot} - -A text string identifying a snapshot, from which [initial dump of PostgreSQL tables](../../engines/database-engines/materialized-postgresql.md) will be performed. Must be used together with [materialized_postgresql_replication_slot](#materialized-postgresql-replication-slot). - ## allow_experimental_projection_optimization {#allow-experimental-projection-optimization} Enables or disables [projection](../../engines/table-engines/mergetree-family/mergetree.md#projections) optimization when processing `SELECT` queries. @@ -3993,8 +3950,8 @@ If [wait_for_async_insert](#wait-for-async-insert) is enabled, every client will Possible values: -- 0 — Insertions are made synchronously, one after another. -- 1 — Multiple asynchronous insertions enabled. +- 0 — Insertions are made synchronously, one after another. +- 1 — Multiple asynchronous insertions enabled. Default value: `0`. @@ -4066,7 +4023,7 @@ Default value: `0`. ## alter_partition_verbose_result {#alter-partition-verbose-result} -Enables or disables the display of information about the parts to which the manipulation operations with partitions and parts have been successfully applied. +Enables or disables the display of information about the parts to which the manipulation operations with partitions and parts have been successfully applied. Applicable to [ATTACH PARTITION|PART](../../sql-reference/statements/alter/partition.md#alter_attach-partition) and to [FREEZE PARTITION](../../sql-reference/statements/alter/partition.md#alter_freeze-partition). Possible values: From 057b7bea86d33e46e7a08aa85f72729db24ebda1 Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Sat, 25 Dec 2021 21:52:15 +0800 Subject: [PATCH 0365/1260] Update contributors.md --- docs/zh/operations/system-tables/contributors.md | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/docs/zh/operations/system-tables/contributors.md b/docs/zh/operations/system-tables/contributors.md index e9374a7dc9c..6c99d2dac22 100644 --- a/docs/zh/operations/system-tables/contributors.md +++ b/docs/zh/operations/system-tables/contributors.md @@ -1,15 +1,10 @@ ---- -machine_translated: true -machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 ---- +# system.contributors {#system-contributors} -# 系统。贡献者 {#system-contributors} - -包含有关贡献者的信息。 该顺序在查询执行时是随机的。 +此系统表包含有关贡献者的信息。在查询执行时,排列顺序是随机的。 列: -- `name` (String) — Contributor (author) name from git log. +- `name` (String) — git 日志中的贡献者 (作者) 名字。 **示例** @@ -32,7 +27,7 @@ SELECT * FROM system.contributors LIMIT 10 └──────────────────┘ ``` -要在表中找出自己,请使用查询: +要在表中找到你自己,请这样查询: ``` sql SELECT * FROM system.contributors WHERE name = 'Olga Khvostikova' @@ -43,3 +38,5 @@ SELECT * FROM system.contributors WHERE name = 'Olga Khvostikova' │ Olga Khvostikova │ └──────────────────┘ ``` + +[原文](https://clickhouse.com/docs/zh/operations/system-tables/contributors) From f8083450ab08b449679a170d93fd785fc2405fe3 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sat, 25 Dec 2021 17:43:00 +0300 Subject: [PATCH 0366/1260] Update ExternalDataSourceConfiguration.cpp --- src/Storages/ExternalDataSourceConfiguration.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 13e6f292602..215f424c839 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -28,7 +28,7 @@ static const std::unordered_set dictionary_allowed_keys = { "host", "port", "user", "password", "db", "database", "table", "schema", "replica", "update_field", "update_tag", "invalidate_query", "query", - "where", "name", "secure", "uri"}; + "where", "name", "secure", "uri", "collection"}; String ExternalDataSourceConfiguration::toString() const { From 32a9540a467debf157db2db79ba7e5b0c7293e00 Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Sat, 25 Dec 2021 23:27:20 +0800 Subject: [PATCH 0367/1260] Update metrics.md --- docs/en/operations/system-tables/metrics.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/metrics.md b/docs/en/operations/system-tables/metrics.md index 551c63d1aa3..21e5923e3a0 100644 --- a/docs/en/operations/system-tables/metrics.md +++ b/docs/en/operations/system-tables/metrics.md @@ -35,7 +35,7 @@ SELECT * FROM system.metrics LIMIT 10 - [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. - [system.events](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that occurred. -- [system.metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [system.metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` and `system.events`. - [Monitoring](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. [Original article](https://clickhouse.com/docs/en/operations/system-tables/metrics) From 84b1dd75b7b31b6025425e0fcc2ff983aaefd670 Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Sat, 25 Dec 2021 23:34:09 +0800 Subject: [PATCH 0368/1260] Update metrics.md --- docs/zh/operations/system-tables/metrics.md | 25 +++++++++------------ 1 file changed, 10 insertions(+), 15 deletions(-) diff --git a/docs/zh/operations/system-tables/metrics.md b/docs/zh/operations/system-tables/metrics.md index 34b7fa35681..5b5b4615f82 100644 --- a/docs/zh/operations/system-tables/metrics.md +++ b/docs/zh/operations/system-tables/metrics.md @@ -1,19 +1,14 @@ ---- -machine_translated: true -machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 ---- +# system.metrics {#system_tables-metrics} -# 系统。指标 {#system_tables-metrics} - -包含可以立即计算或具有当前值的指标。 例如,同时处理的查询的数量或当前副本的延迟。 此表始终是最新的。 +此系统表包含可以即时计算或具有当前值的指标。例如,同时处理的查询数量或当前的复制延迟。这个表始终是最新的。 列: -- `metric` ([字符串](../../sql-reference/data-types/string.md)) — Metric name. -- `value` ([Int64](../../sql-reference/data-types/int-uint.md)) — Metric value. -- `description` ([字符串](../../sql-reference/data-types/string.md)) — Metric description. +- `metric` ([字符串](../../sql-reference/data-types/string.md)) — 指标名称. +- `value` ([Int64](../../sql-reference/data-types/int-uint.md)) — 指标的值. +- `description` ([字符串](../../sql-reference/data-types/string.md)) — 指标的描述. -支持的指标列表,您可以在 [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp) ClickHouse的源文件。 +对于支持的指标列表,您可以查看 [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp) ClickHouse 的源文件。 **示例** @@ -38,7 +33,7 @@ SELECT * FROM system.metrics LIMIT 10 **另请参阅** -- [系统。asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. -- [系统。活动](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that occurred. -- [系统。metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. -- [监测](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. +- [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — 包含周期性的计算指标。 +- [system.events](../../operations/system-tables/events.md#system_tables-events) — 包含发生的一些事件。 +- [system.metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — 包含`system.metrics`表和`system.events`表的历史指标值。 +- [监控](../../operations/monitoring.md) — ClickHouse 监控的基本概念。 From a3dbebd47f0329023d68723deaa6ce553abe847c Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Sat, 25 Dec 2021 23:56:46 +0800 Subject: [PATCH 0369/1260] Update settings.md --- docs/zh/operations/system-tables/settings.md | 39 +++++++++----------- 1 file changed, 18 insertions(+), 21 deletions(-) diff --git a/docs/zh/operations/system-tables/settings.md b/docs/zh/operations/system-tables/settings.md index c717c8c9562..144eb0179c4 100644 --- a/docs/zh/operations/system-tables/settings.md +++ b/docs/zh/operations/system-tables/settings.md @@ -1,27 +1,22 @@ ---- -machine_translated: true -machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 ---- +# system.settings {#system-tables-system-settings} -# 系统。设置 {#system-tables-system-settings} - -包含有关当前用户的会话设置的信息。 +包含当前用户会话设置的相关信息。 列: -- `name` ([字符串](../../sql-reference/data-types/string.md)) — Setting name. -- `value` ([字符串](../../sql-reference/data-types/string.md)) — Setting value. -- `changed` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Shows whether a setting is changed from its default value. -- `description` ([字符串](../../sql-reference/data-types/string.md)) — Short setting description. -- `min` ([可为空](../../sql-reference/data-types/nullable.md)([字符串](../../sql-reference/data-types/string.md))) — Minimum value of the setting, if any is set via [制约因素](../../operations/settings/constraints-on-settings.md#constraints-on-settings). 如果设置没有最小值,则包含 [NULL](../../sql-reference/syntax.md#null-literal). -- `max` ([可为空](../../sql-reference/data-types/nullable.md)([字符串](../../sql-reference/data-types/string.md))) — Maximum value of the setting, if any is set via [制约因素](../../operations/settings/constraints-on-settings.md#constraints-on-settings). 如果设置没有最大值,则包含 [NULL](../../sql-reference/syntax.md#null-literal). -- `readonly` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Shows whether the current user can change the setting: - - `0` — Current user can change the setting. - - `1` — Current user can't change the setting. +- `name` ([字符串](../../sql-reference/data-types/string.md)) — 设置名称。 +- `value` ([字符串](../../sql-reference/data-types/string.md)) — 设置的值。 +- `changed` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 显示该设置是否从其默认值修改。 +- `description` ([字符串](../../sql-reference/data-types/string.md)) — 该设置的简要描述。 +- `min` ([可为空](../../sql-reference/data-types/nullable.md)([字符串](../../sql-reference/data-types/string.md))) — 该设置的最小值,如果有最小值,则是通过[约束](../../operations/settings/constraints-on-settings.md#constraints-on-settings)设置的。如果该设置没有最小值,则包含 [NULL](../../sql-reference/syntax.md#null-literal). +- `max` ([可为空](../../sql-reference/data-types/nullable.md)([字符串](../../sql-reference/data-types/string.md))) — 该设置的最大值, 如果有最大值,则是通过[约束](../../operations/settings/constraints-on-settings.md#constraints-on-settings)设置的。如果该设置没有最大值,则包含 [NULL](../../sql-reference/syntax.md#null-literal). +- `readonly` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 当前用户是否可以修改该设置: + - `0` — 当前用户可以修改此设置. + - `1` — 当前用户不能修改此设置. **示例** -下面的示例演示如何获取有关名称包含的设置的信息 `min_i`. +下面的例子显示了如何获得设置名称中包含`min_i`的设置信息。 ``` sql SELECT * @@ -37,10 +32,10 @@ WHERE name LIKE '%min_i%' └─────────────────────────────────────────────┴───────────┴─────────┴───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴──────┴──────┴──────────┘ ``` -使用 `WHERE changed` 可以是有用的,例如,当你想检查: +比如,当你想要检查以下情况时,使用 `WHERE changed` 会很有用: -- 配置文件中的设置是否正确加载并正在使用。 -- 在当前会话中更改的设置。 +- 配置文件中的设置是否正确加载,并正在使用。 +- 在当前会话中更改过的设置。 @@ -52,4 +47,6 @@ SELECT * FROM system.settings WHERE changed AND name='load_balancing' - [设置](../../operations/settings/index.md#session-settings-intro) - [查询权限](../../operations/settings/permissions-for-queries.md#settings_readonly) -- [对设置的限制](../../operations/settings/constraints-on-settings.md) +- [对设置的约束](../../operations/settings/constraints-on-settings.md) + +[原文](https://clickhouse.com/docs/zh/operations/system-tables/settings) From 1d8ddffa784ada89ecc265eb27663379c52a5db9 Mon Sep 17 00:00:00 2001 From: freedomDR <1640145602@qq.com> Date: Sat, 25 Dec 2021 16:44:12 +0000 Subject: [PATCH 0370/1260] add test --- tests/queries/0_stateless/02154_bitmap_contains.reference | 1 + tests/queries/0_stateless/02154_bitmap_contains.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/02154_bitmap_contains.reference create mode 100644 tests/queries/0_stateless/02154_bitmap_contains.sql diff --git a/tests/queries/0_stateless/02154_bitmap_contains.reference b/tests/queries/0_stateless/02154_bitmap_contains.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02154_bitmap_contains.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02154_bitmap_contains.sql b/tests/queries/0_stateless/02154_bitmap_contains.sql new file mode 100644 index 00000000000..3235e81e2bb --- /dev/null +++ b/tests/queries/0_stateless/02154_bitmap_contains.sql @@ -0,0 +1 @@ +select bitmapContains(bitmapBuild([9]), 964291337) From e663cb4574481a7d938fe2851265c3a1043ed65c Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 25 Dec 2021 18:35:38 +0300 Subject: [PATCH 0371/1260] Fix detaching-attaching table with schema --- .../PostgreSQLReplicationHandler.cpp | 23 +++++++---- .../test.py | 41 ++++++++++++++++++- 2 files changed, 54 insertions(+), 10 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 984a9cdd47a..a406ae62693 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -278,9 +278,8 @@ ASTPtr PostgreSQLReplicationHandler::getCreateNestedTableQuery(StorageMaterializ { postgres::Connection connection(connection_info); pqxx::nontransaction tx(connection.getRef()); - auto table_structure = std::make_unique(fetchPostgreSQLTableStructure(tx, table_name, postgres_schema, true, true, true)); - if (!table_structure) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to get PostgreSQL table structure"); + auto [postgres_table_schema, postgres_table_name] = getSchemaAndTableName(table_name); + auto table_structure = std::make_unique(fetchPostgreSQLTableStructure(tx, postgres_table_name, postgres_table_schema, true, true, true)); auto table_override = tryGetTableOverride(current_database_name, table_name); return storage->getCreateNestedTableQuery(std::move(table_structure), table_override ? table_override->as() : nullptr); @@ -516,17 +515,25 @@ void PostgreSQLReplicationHandler::dropPublication(pqxx::nontransaction & tx) void PostgreSQLReplicationHandler::addTableToPublication(pqxx::nontransaction & ntx, const String & table_name) { - std::string query_str = fmt::format("ALTER PUBLICATION {} ADD TABLE ONLY {}", publication_name, doubleQuoteString(table_name)); + std::string query_str = fmt::format("ALTER PUBLICATION {} ADD TABLE ONLY {}", publication_name, doubleQuoteWithSchema(table_name)); ntx.exec(query_str); - LOG_TRACE(log, "Added table `{}` to publication `{}`", table_name, publication_name); + LOG_TRACE(log, "Added table {} to publication `{}`", doubleQuoteWithSchema(table_name), publication_name); } void PostgreSQLReplicationHandler::removeTableFromPublication(pqxx::nontransaction & ntx, const String & table_name) { - std::string query_str = fmt::format("ALTER PUBLICATION {} DROP TABLE ONLY {}", publication_name, doubleQuoteString(table_name)); - ntx.exec(query_str); - LOG_TRACE(log, "Removed table `{}` from publication `{}`", table_name, publication_name); + try + { + std::string query_str = fmt::format("ALTER PUBLICATION {} DROP TABLE ONLY {}", publication_name, doubleQuoteWithSchema(table_name)); + ntx.exec(query_str); + LOG_TRACE(log, "Removed table `{}` from publication `{}`", doubleQuoteWithSchema(table_name), publication_name); + } + catch (const pqxx::undefined_table &) + { + /// Removing table from replication must succeed even if table does not exist in PostgreSQL. + LOG_WARNING(log, "Did not remove table {} from publication, because table does not exist in PostgreSQL", doubleQuoteWithSchema(table_name), publication_name); + } } diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index c8b63d8e667..725158920a1 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -178,7 +178,7 @@ def assert_number_of_columns(expected, table_name, database_name='test_database' def check_tables_are_synchronized(table_name, order_by='key', postgres_database='postgres_database', materialized_database='test_database', schema_name=''): assert_nested_table_is_created(table_name, materialized_database, schema_name) - print("Checking table is synchronized:", table_name) + print(f"Checking table is synchronized. Table name: {table_name}, table schema: {schema_name}") expected = instance.query('select * from {}.{} order by {};'.format(postgres_database, table_name, order_by)) if len(schema_name) == 0: result = instance.query('select * from {}.{} order by {};'.format(materialized_database, table_name, order_by)) @@ -356,6 +356,11 @@ def test_remove_table_from_replication(started_cluster): for i in range(NUM_TABLES): cursor.execute('drop table if exists postgresql_replica_{};'.format(i)) + # Removing from replication table which does not exist in PostgreSQL must be ok. + instance.query('DETACH TABLE test_database.postgresql_replica_0'); + assert instance.contains_in_log("from publication, because table does not exist in PostgreSQL") + drop_materialized_db() + def test_predefined_connection_configuration(started_cluster): drop_materialized_db() @@ -379,6 +384,7 @@ def test_database_with_single_non_default_schema(started_cluster): NUM_TABLES=5 schema_name = 'test_schema' + materialized_db = 'test_database' clickhouse_postgres_db = 'postgres_database_with_schema' global insert_counter insert_counter = 0 @@ -430,6 +436,14 @@ def test_database_with_single_non_default_schema(started_cluster): instance.query(f"INSERT INTO {clickhouse_postgres_db}.postgresql_replica_{altered_table} SELECT number, number, number from numbers(5000, 1000)") assert_number_of_columns(3, f'postgresql_replica_{altered_table}') check_tables_are_synchronized(f"postgresql_replica_{altered_table}", postgres_database=clickhouse_postgres_db); + + print('DETACH-ATTACH') + detached_table_name = "postgresql_replica_1" + instance.query(f"DETACH TABLE {materialized_db}.{detached_table_name}") + assert instance.contains_in_log("from publication, because table does not exist in PostgreSQL") == False + instance.query(f"ATTACH TABLE {materialized_db}.{detached_table_name}") + check_tables_are_synchronized(f"{detached_table_name}", postgres_database=clickhouse_postgres_db); + drop_materialized_db() @@ -440,6 +454,7 @@ def test_database_with_multiple_non_default_schemas_1(started_cluster): NUM_TABLES = 5 schema_name = 'test_schema' clickhouse_postgres_db = 'postgres_database_with_schema' + materialized_db = 'test_database' publication_tables = '' global insert_counter insert_counter = 0 @@ -494,6 +509,15 @@ def test_database_with_multiple_non_default_schemas_1(started_cluster): instance.query(f"INSERT INTO {clickhouse_postgres_db}.postgresql_replica_{altered_table} SELECT number, number, number from numbers(5000, 1000)") assert_number_of_columns(3, f'{schema_name}.postgresql_replica_{altered_table}') check_tables_are_synchronized(f"postgresql_replica_{altered_table}", schema_name=schema_name, postgres_database=clickhouse_postgres_db); + + print('DETACH-ATTACH') + detached_table_name = "postgresql_replica_1" + instance.query(f"DETACH TABLE {materialized_db}.`{schema_name}.{detached_table_name}`") + assert instance.contains_in_log("from publication, because table does not exist in PostgreSQL") == False + instance.query(f"ATTACH TABLE {materialized_db}.`{schema_name}.{detached_table_name}`") + assert_show_tables("test_schema.postgresql_replica_0\ntest_schema.postgresql_replica_1\ntest_schema.postgresql_replica_2\ntest_schema.postgresql_replica_3\ntest_schema.postgresql_replica_4\n") + check_tables_are_synchronized(f"{detached_table_name}", schema_name=schema_name, postgres_database=clickhouse_postgres_db); + drop_materialized_db() @@ -504,6 +528,7 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): NUM_TABLES = 2 schemas_num = 2 schema_list = 'schema0, schema1' + materialized_db = 'test_database' global insert_counter insert_counter = 0 @@ -557,11 +582,23 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): print('ALTER') altered_schema = random.randint(0, schemas_num-1) altered_table = random.randint(0, NUM_TABLES-1) + clickhouse_postgres_db = f'clickhouse_postgres_db{altered_schema}' cursor.execute(f"ALTER TABLE schema{altered_schema}.postgresql_replica_{altered_table} ADD COLUMN value2 integer") instance.query(f"INSERT INTO clickhouse_postgres_db{altered_schema}.postgresql_replica_{altered_table} SELECT number, number, number from numbers(1000 * {insert_counter}, 1000)") assert_number_of_columns(3, f'schema{altered_schema}.postgresql_replica_{altered_table}') - check_tables_are_synchronized(f"postgresql_replica_{altered_table}", schema_name=schema_name, postgres_database=clickhouse_postgres_db); + check_tables_are_synchronized(f"postgresql_replica_{altered_table}", schema_name=f"schema{altered_schema}", postgres_database=clickhouse_postgres_db); + + print('DETACH-ATTACH') + detached_table_name = "postgresql_replica_1" + detached_table_schema = "schema0" + clickhouse_postgres_db = f'clickhouse_postgres_db0' + instance.query(f"DETACH TABLE {materialized_db}.`{detached_table_schema}.{detached_table_name}`") + assert instance.contains_in_log("from publication, because table does not exist in PostgreSQL") == False + instance.query(f"ATTACH TABLE {materialized_db}.`{detached_table_schema}.{detached_table_name}`") + assert_show_tables("schema0.postgresql_replica_0\nschema0.postgresql_replica_1\nschema1.postgresql_replica_0\nschema1.postgresql_replica_1\n") + check_tables_are_synchronized(f"postgresql_replica_{altered_table}", schema_name=detached_table_schema, postgres_database=clickhouse_postgres_db); + drop_materialized_db() From 12a851b85c8b96caf8c37cff9f7ae8efeaadd46c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Dec 2021 20:21:30 +0300 Subject: [PATCH 0372/1260] Add simple tool to visualize flaky tests --- utils/tests-visualizer/index.html | 137 ++++++++++++++++++++++++++++++ 1 file changed, 137 insertions(+) create mode 100644 utils/tests-visualizer/index.html diff --git a/utils/tests-visualizer/index.html b/utils/tests-visualizer/index.html new file mode 100644 index 00000000000..690c42e486e --- /dev/null +++ b/utils/tests-visualizer/index.html @@ -0,0 +1,137 @@ + + + + + +

Loading (10 seconds, 20 MB)...

+ + + + + From 0a55fa3dc201a379bf2a90e7ae0ca4a5aa3e59d3 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 25 Dec 2021 20:30:31 +0300 Subject: [PATCH 0373/1260] Revert "Grouping sets dev" --- src/Core/ColumnNumbers.h | 1 - src/Core/NamesAndTypes.h | 2 - src/Interpreters/ActionsDAG.h | 2 +- src/Interpreters/Aggregator.h | 45 +---- src/Interpreters/ExpressionAnalyzer.cpp | 132 ++++----------- src/Interpreters/ExpressionAnalyzer.h | 4 - src/Interpreters/InterpreterSelectQuery.cpp | 154 +++++++----------- src/Interpreters/InterpreterSelectQuery.h | 11 +- src/Interpreters/TreeOptimizer.cpp | 18 +- src/Parsers/ASTExpressionList.cpp | 19 +-- src/Parsers/ASTSelectQuery.cpp | 17 +- src/Parsers/ASTSelectQuery.h | 1 - src/Parsers/ExpressionListParsers.cpp | 50 +----- src/Parsers/ExpressionListParsers.h | 14 -- src/Parsers/IAST.h | 1 - src/Parsers/ParserSelectQuery.cpp | 27 +-- src/Processors/QueryPlan/AggregatingStep.h | 2 - src/Processors/QueryPlan/GroupingSetsStep.cpp | 46 ------ src/Processors/QueryPlan/GroupingSetsStep.h | 25 --- .../Transforms/GroupingSetsTransform.cpp | 76 --------- .../Transforms/GroupingSetsTransform.h | 35 ---- src/QueryPipeline/Pipe.cpp | 2 +- src/QueryPipeline/Pipe.h | 3 - .../01883_with_grouping_sets.reference | 89 ---------- .../0_stateless/01883_with_grouping_sets.sql | 44 ----- 25 files changed, 104 insertions(+), 716 deletions(-) delete mode 100644 src/Processors/QueryPlan/GroupingSetsStep.cpp delete mode 100644 src/Processors/QueryPlan/GroupingSetsStep.h delete mode 100644 src/Processors/Transforms/GroupingSetsTransform.cpp delete mode 100644 src/Processors/Transforms/GroupingSetsTransform.h delete mode 100644 tests/queries/0_stateless/01883_with_grouping_sets.reference delete mode 100644 tests/queries/0_stateless/01883_with_grouping_sets.sql diff --git a/src/Core/ColumnNumbers.h b/src/Core/ColumnNumbers.h index 29b4c49dc83..9441f6485a7 100644 --- a/src/Core/ColumnNumbers.h +++ b/src/Core/ColumnNumbers.h @@ -8,6 +8,5 @@ namespace DB { using ColumnNumbers = std::vector; -using ColumnNumbersList = std::vector; } diff --git a/src/Core/NamesAndTypes.h b/src/Core/NamesAndTypes.h index 2719017a726..3ac9ad2fa02 100644 --- a/src/Core/NamesAndTypes.h +++ b/src/Core/NamesAndTypes.h @@ -109,8 +109,6 @@ public: std::optional tryGetByName(const std::string & name) const; }; -using NamesAndTypesLists = std::vector; - } namespace std diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index cfb85cd5369..9a5ad01a252 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -183,7 +183,7 @@ public: ActionsDAGPtr clone() const; /// Execute actions for header. Input block must have empty columns. - /// Result should be equal to the execution of ExpressionActions built from this DAG. + /// Result should be equal to the execution of ExpressionActions build form this DAG. /// Actions are not changed, no expressions are compiled. /// /// In addition, check that result constants are constants according to DAG. diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 587659c7eeb..c79c2c5ef64 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -878,10 +878,9 @@ public: Block intermediate_header; /// What to count. - ColumnNumbers keys; - const ColumnNumbersList keys_vector; + const ColumnNumbers keys; const AggregateDescriptions aggregates; - size_t keys_size; + const size_t keys_size; const size_t aggregates_size; /// The settings of approximate calculation of GROUP BY. @@ -939,46 +938,6 @@ public: { } - /// two dimensional vector of aggregating keys in params - Params( - const Block & src_header_, - const ColumnNumbers & keys_, - const ColumnNumbersList & keys_vector_, - const AggregateDescriptions & aggregates_, - bool overflow_row_, - size_t max_rows_to_group_by_, - OverflowMode group_by_overflow_mode_, - size_t group_by_two_level_threshold_, - size_t group_by_two_level_threshold_bytes_, - size_t max_bytes_before_external_group_by_, - bool empty_result_for_aggregation_by_empty_set_, - VolumePtr tmp_volume_, - size_t max_threads_, - size_t min_free_disk_space_, - bool compile_aggregate_expressions_, - size_t min_count_to_compile_aggregate_expression_, - const Block & intermediate_header_ = {}) - : src_header(src_header_) - , intermediate_header(intermediate_header_) - , keys(keys_) - , keys_vector(keys_vector_) - , aggregates(aggregates_) - , keys_size(keys.size()) - , aggregates_size(aggregates.size()) - , overflow_row(overflow_row_) - , max_rows_to_group_by(max_rows_to_group_by_) - , group_by_overflow_mode(group_by_overflow_mode_) - , group_by_two_level_threshold(group_by_two_level_threshold_) - , group_by_two_level_threshold_bytes(group_by_two_level_threshold_bytes_) - , max_bytes_before_external_group_by(max_bytes_before_external_group_by_) - , empty_result_for_aggregation_by_empty_set(empty_result_for_aggregation_by_empty_set_) - , tmp_volume(tmp_volume_) - , max_threads(max_threads_) - , min_free_disk_space(min_free_disk_space_) - , compile_aggregate_expressions(compile_aggregate_expressions_) - , min_count_to_compile_aggregate_expression(min_count_to_compile_aggregate_expression_) - {} - /// Only parameters that matter during merge. Params(const Block & intermediate_header_, const ColumnNumbers & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_threads_) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index f1d04e4b3a7..c195cb93c5e 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -40,7 +40,6 @@ #include #include -#include "Core/NamesAndTypes.h" #include @@ -342,101 +341,44 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false); - if (select_query->group_by_with_grouping_sets) + const auto & column_name = group_asts[i]->getColumnName(); + + const auto * node = temp_actions->tryFindInIndex(column_name); + if (!node) + throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER); + + /// Only removes constant keys if it's an initiator or distributed_group_by_no_merge is enabled. + if (getContext()->getClientInfo().distributed_depth == 0 || settings.distributed_group_by_no_merge > 0) { - ASTs group_elements_ast; - const ASTExpressionList * group_ast_element = group_asts[i]->as(); - if (!group_ast_element) - throw Exception("Grouping Sets element " + group_asts[i]->getColumnName() + " should be an expression type", ErrorCodes::UNKNOWN_IDENTIFIER); - group_elements_ast = group_ast_element->children; - - NamesAndTypesList grouping_set_list; - - for (ssize_t j = 0; j < ssize_t(group_elements_ast.size()); ++j) + /// Constant expressions have non-null column pointer at this stage. + if (node->column && isColumnConst(*node->column)) { - ssize_t group_size = group_elements_ast.size(); - const auto & column_name = group_elements_ast[j]->getColumnName(); - const auto * node = temp_actions->tryFindInIndex(column_name); - if (!node) - throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER); + select_query->group_by_with_constant_keys = true; - /// Only removes constant keys if it's an initiator or distributed_group_by_no_merge is enabled. - if (getContext()->getClientInfo().distributed_depth == 0 || settings.distributed_group_by_no_merge > 0) + /// But don't remove last key column if no aggregate functions, otherwise aggregation will not work. + if (!aggregate_descriptions.empty() || size > 1) { - /// Constant expressions have non-null column pointer at this stage. - if (node->column && isColumnConst(*node->column)) - { - /// But don't remove last key column if no aggregate functions, otherwise aggregation will not work. - if (!aggregate_descriptions.empty() || group_size > 1) - { - if (j + 1 < static_cast(group_size)) - group_elements_ast[j] = std::move(group_elements_ast.back()); + if (i + 1 < static_cast(size)) + group_asts[i] = std::move(group_asts.back()); - group_elements_ast.pop_back(); + group_asts.pop_back(); - --j; - continue; - } - } - } - - NameAndTypePair key{column_name, node->result_type}; - - grouping_set_list.push_back(key); - - /// Aggregation keys are unique. - if (!unique_keys.count(key.name)) - { - unique_keys.insert(key.name); - aggregation_keys.push_back(key); - - /// Key is no longer needed, therefore we can save a little by moving it. - aggregated_columns.push_back(std::move(key)); + --i; + continue; } } - - aggregation_keys_list.push_back(std::move(grouping_set_list)); } - else + + NameAndTypePair key{column_name, node->result_type}; + + /// Aggregation keys are uniqued. + if (!unique_keys.count(key.name)) { - const auto & column_name = group_asts[i]->getColumnName(); - const auto * node = temp_actions->tryFindInIndex(column_name); - if (!node) - throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER); + unique_keys.insert(key.name); + aggregation_keys.push_back(key); - /// Only removes constant keys if it's an initiator or distributed_group_by_no_merge is enabled. - if (getContext()->getClientInfo().distributed_depth == 0 || settings.distributed_group_by_no_merge > 0) - { - /// Constant expressions have non-null column pointer at this stage. - if (node->column && isColumnConst(*node->column)) - { - select_query->group_by_with_constant_keys = true; - - /// But don't remove last key column if no aggregate functions, otherwise aggregation will not work. - if (!aggregate_descriptions.empty() || size > 1) - { - if (i + 1 < static_cast(size)) - group_asts[i] = std::move(group_asts.back()); - - group_asts.pop_back(); - - --i; - continue; - } - } - } - - NameAndTypePair key{column_name, node->result_type}; - - /// Aggregation keys are uniqued. - if (!unique_keys.count(key.name)) - { - unique_keys.insert(key.name); - aggregation_keys.push_back(key); - - /// Key is no longer needed, therefore we can save a little by moving it. - aggregated_columns.push_back(std::move(key)); - } + /// Key is no longer needed, therefore we can save a little by moving it. + aggregated_columns.push_back(std::move(key)); } } @@ -1197,24 +1139,10 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain ExpressionActionsChain::Step & step = chain.lastStep(columns_after_join); ASTs asts = select_query->groupBy()->children; - if (select_query->group_by_with_grouping_sets) + for (const auto & ast : asts) { - for (const auto & ast : asts) - { - for (const auto & ast_element : ast->children) - { - step.addRequiredOutput(ast_element->getColumnName()); - getRootActions(ast_element, only_types, step.actions()); - } - } - } - else - { - for (const auto & ast : asts) - { - step.addRequiredOutput(ast->getColumnName()); - getRootActions(ast, only_types, step.actions()); - } + step.addRequiredOutput(ast->getColumnName()); + getRootActions(ast, only_types, step.actions()); } if (optimize_aggregation_in_order) diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 19054e795a6..b6bb3c5fad5 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -64,7 +64,6 @@ struct ExpressionAnalyzerData bool has_aggregation = false; NamesAndTypesList aggregation_keys; - NamesAndTypesLists aggregation_keys_list; bool has_const_aggregation_keys = false; AggregateDescriptions aggregate_descriptions; @@ -95,8 +94,6 @@ private: explicit ExtractedSettings(const Settings & settings_); }; - Poco::Logger * poco_log = &Poco::Logger::get("ExpressionAnalyzer"); - public: /// Ctor for non-select queries. Generally its usage is: /// auto actions = ExpressionAnalyzer(query, syntax, context).getActions(); @@ -324,7 +321,6 @@ public: const NamesAndTypesList & aggregationKeys() const { return aggregation_keys; } bool hasConstAggregationKeys() const { return has_const_aggregation_keys; } - const NamesAndTypesLists & aggregationKeysList() const { return aggregation_keys_list; } const AggregateDescriptions & aggregates() const { return aggregate_descriptions; } const PreparedSets & getPreparedSets() const { return prepared_sets; } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index fb3507eefef..5c0322ac1d9 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -44,7 +44,6 @@ #include #include #include -#include #include #include #include @@ -961,10 +960,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

QueryProcessingStage::WithMergeableState && - !query.group_by_with_totals && !query.group_by_with_rollup && !query.group_by_with_cube && !query.group_by_with_grouping_sets; - -// if (query.group_by_with_grouping_sets && query.group_by_with_totals) -// throw Exception("WITH TOTALS and GROUPING SETS are not supported together", ErrorCodes::NOT_IMPLEMENTED); + !query.group_by_with_totals && !query.group_by_with_rollup && !query.group_by_with_cube; if (query_info.projection && query_info.projection->desc->type == ProjectionDescription::Type::Aggregate) { @@ -1183,7 +1179,6 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( @@ -2043,79 +2039,6 @@ void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsD query_plan.addStep(std::move(where_step)); } -void InterpreterSelectQuery::initAggregatorParams( - const Block & current_data_stream_header, - AggregatorParamsPtr & params_ptr, - const AggregateDescriptions & aggregates, - bool overflow_row, const Settings & settings, - size_t group_by_two_level_threshold, size_t group_by_two_level_threshold_bytes) -{ - auto & query = getSelectQuery(); - if (query.group_by_with_grouping_sets) - { - ColumnNumbers keys; - ColumnNumbers all_keys; - ColumnNumbersList keys_vector; - std::unordered_set keys_set; - for (const auto & aggregation_keys : query_analyzer->aggregationKeysList()) - { - keys.clear(); - for (const auto & key : aggregation_keys) - { - size_t key_name_pos = current_data_stream_header.getPositionByName(key.name); - keys_set.insert(key_name_pos); - keys.push_back(key_name_pos); - } - keys_vector.push_back(keys); - } - all_keys.assign(keys_set.begin(), keys_set.end()); - - params_ptr = std::make_unique( - current_data_stream_header, - all_keys, - keys_vector, - aggregates, - overflow_row, - settings.max_rows_to_group_by, - settings.group_by_overflow_mode, - group_by_two_level_threshold, - group_by_two_level_threshold_bytes, - settings.max_bytes_before_external_group_by, - settings.empty_result_for_aggregation_by_empty_set - || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && keys.empty() - && query_analyzer->hasConstAggregationKeys()), - context->getTemporaryVolume(), - settings.max_threads, - settings.min_free_disk_space_for_temporary_data, - settings.compile_aggregate_expressions, - settings.min_count_to_compile_aggregate_expression); - } - else - { - ColumnNumbers keys; - for (const auto & key : query_analyzer->aggregationKeys()) - keys.push_back(current_data_stream_header.getPositionByName(key.name)); - - params_ptr = std::make_unique( - current_data_stream_header, - keys, - aggregates, - overflow_row, - settings.max_rows_to_group_by, - settings.group_by_overflow_mode, - group_by_two_level_threshold, - group_by_two_level_threshold_bytes, - settings.max_bytes_before_external_group_by, - settings.empty_result_for_aggregation_by_empty_set - || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && keys.empty() - && query_analyzer->hasConstAggregationKeys()), - context->getTemporaryVolume(), - settings.max_threads, - settings.min_free_disk_space_for_temporary_data, - settings.compile_aggregate_expressions, - settings.min_count_to_compile_aggregate_expression); - } -} void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info) { @@ -2127,6 +2050,9 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac return; const auto & header_before_aggregation = query_plan.getCurrentDataStream().header; + ColumnNumbers keys; + for (const auto & key : query_analyzer->aggregationKeys()) + keys.push_back(header_before_aggregation.getPositionByName(key.name)); AggregateDescriptions aggregates = query_analyzer->aggregates(); for (auto & descr : aggregates) @@ -2136,9 +2062,24 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac const Settings & settings = context->getSettingsRef(); - AggregatorParamsPtr params_ptr; - initAggregatorParams(header_before_aggregation, params_ptr, aggregates, overflow_row, settings, - settings.group_by_two_level_threshold, settings.group_by_two_level_threshold_bytes); + Aggregator::Params params( + header_before_aggregation, + keys, + aggregates, + overflow_row, + settings.max_rows_to_group_by, + settings.group_by_overflow_mode, + settings.group_by_two_level_threshold, + settings.group_by_two_level_threshold_bytes, + settings.max_bytes_before_external_group_by, + settings.empty_result_for_aggregation_by_empty_set + || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && keys.empty() + && query_analyzer->hasConstAggregationKeys()), + context->getTemporaryVolume(), + settings.max_threads, + settings.min_free_disk_space_for_temporary_data, + settings.compile_aggregate_expressions, + settings.min_count_to_compile_aggregate_expression); SortDescription group_by_sort_description; @@ -2156,7 +2097,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac auto aggregating_step = std::make_unique( query_plan.getCurrentDataStream(), - *params_ptr, + params, final, settings.max_block_size, settings.aggregation_in_order_max_block_bytes, @@ -2165,6 +2106,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac storage_has_evenly_distributed_read, std::move(group_by_info), std::move(group_by_sort_description)); + query_plan.addStep(std::move(aggregating_step)); } @@ -2216,27 +2158,47 @@ void InterpreterSelectQuery::executeTotalsAndHaving( query_plan.addStep(std::move(totals_having_step)); } + void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modificator modificator) { const auto & header_before_transform = query_plan.getCurrentDataStream().header; + ColumnNumbers keys; + + for (const auto & key : query_analyzer->aggregationKeys()) + keys.push_back(header_before_transform.getPositionByName(key.name)); + const Settings & settings = context->getSettingsRef(); - AggregatorParamsPtr params_ptr; - initAggregatorParams(header_before_transform, params_ptr, query_analyzer->aggregates(), false, settings, 0, 0); - auto transform_params = std::make_shared(*params_ptr, true); + Aggregator::Params params( + header_before_transform, + keys, + query_analyzer->aggregates(), + false, + settings.max_rows_to_group_by, + settings.group_by_overflow_mode, + 0, + 0, + settings.max_bytes_before_external_group_by, + settings.empty_result_for_aggregation_by_empty_set, + context->getTemporaryVolume(), + settings.max_threads, + settings.min_free_disk_space_for_temporary_data, + settings.compile_aggregate_expressions, + settings.min_count_to_compile_aggregate_expression); + + auto transform_params = std::make_shared(params, true); QueryPlanStepPtr step; if (modificator == Modificator::ROLLUP) step = std::make_unique(query_plan.getCurrentDataStream(), std::move(transform_params)); - else if (modificator == Modificator::CUBE) - step = std::make_unique(query_plan.getCurrentDataStream(), std::move(transform_params)); else - step = std::make_unique(query_plan.getCurrentDataStream(), std::move(transform_params)); + step = std::make_unique(query_plan.getCurrentDataStream(), std::move(transform_params)); query_plan.addStep(std::move(step)); } + void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description) { if (!expression) diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 7f75c4ea41f..4298cbbb794 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -14,7 +14,6 @@ #include #include -#include "Interpreters/ActionsDAG.h" namespace Poco { @@ -30,7 +29,6 @@ class QueryPlan; struct TreeRewriterResult; using TreeRewriterResultPtr = std::shared_ptr; -using AggregatorParamsPtr = std::unique_ptr; /** Interprets the SELECT query. Returns the stream of blocks with the results of the query before `to_stage` stage. @@ -128,12 +126,6 @@ private: /// Different stages of query execution. - void initAggregatorParams( - const Block & current_data_stream_header, - AggregatorParamsPtr & params_ptr, - const AggregateDescriptions & aggregates, - bool overflow_row, const Settings & settings, - size_t group_by_two_level_threshold, size_t group_by_two_level_threshold_bytes); void executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan); void executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter); void executeAggregation( @@ -164,8 +156,7 @@ private: enum class Modificator { ROLLUP = 0, - CUBE = 1, - GROUPING_SETS = 2 + CUBE = 1 }; void executeRollupOrCube(QueryPlan & query_plan, Modificator modificator); diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index a61d41f78fb..64b25ca9777 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -201,22 +201,10 @@ GroupByKeysInfo getGroupByKeysInfo(const ASTs & group_by_keys) /// filling set with short names of keys for (const auto & group_key : group_by_keys) { - /// for grouping sets case - if (group_key->as()) - { - const auto express_list_ast = group_key->as(); - for (const auto & group_elem : express_list_ast.children) - { - data.key_names.insert(group_elem->getColumnName()); - } - } - else - { - if (group_key->as()) - data.has_function = true; + if (group_key->as()) + data.has_function = true; - data.key_names.insert(group_key->getColumnName()); - } + data.key_names.insert(group_key->getColumnName()); } return data; diff --git a/src/Parsers/ASTExpressionList.cpp b/src/Parsers/ASTExpressionList.cpp index 2590c6b2941..2724465537f 100644 --- a/src/Parsers/ASTExpressionList.cpp +++ b/src/Parsers/ASTExpressionList.cpp @@ -26,15 +26,7 @@ void ASTExpressionList::formatImpl(const FormatSettings & settings, FormatState settings.ostr << ' '; } - if (frame.surround_each_list_element_with_parens) - settings.ostr << "("; - - FormatStateStacked frame_nested = frame; - frame_nested.surround_each_list_element_with_parens = false; - (*it)->formatImpl(settings, state, frame_nested); - - if (frame.surround_each_list_element_with_parens) - settings.ostr << ")"; + (*it)->formatImpl(settings, state, frame); } } @@ -49,7 +41,6 @@ void ASTExpressionList::formatImplMultiline(const FormatSettings & settings, For } ++frame.indent; - for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { if (it != children.begin()) @@ -63,15 +54,7 @@ void ASTExpressionList::formatImplMultiline(const FormatSettings & settings, For FormatStateStacked frame_nested = frame; frame_nested.expression_list_always_start_on_new_line = false; - frame_nested.surround_each_list_element_with_parens = false; - - if (frame.surround_each_list_element_with_parens) - settings.ostr << "("; - (*it)->formatImpl(settings, state, frame_nested); - - if (frame.surround_each_list_element_with_parens) - settings.ostr << ")"; } } diff --git a/src/Parsers/ASTSelectQuery.cpp b/src/Parsers/ASTSelectQuery.cpp index 5a96e1d4df9..1c5a4310f1b 100644 --- a/src/Parsers/ASTSelectQuery.cpp +++ b/src/Parsers/ASTSelectQuery.cpp @@ -114,12 +114,9 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F if (groupBy()) { s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "GROUP BY" << (s.hilite ? hilite_none : ""); - if (!group_by_with_grouping_sets) - { - s.one_line + s.one_line ? groupBy()->formatImpl(s, state, frame) : groupBy()->as().formatImplMultiline(s, state, frame); - } } if (group_by_with_rollup) @@ -128,18 +125,6 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F if (group_by_with_cube) s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH CUBE" << (s.hilite ? hilite_none : ""); - if (group_by_with_grouping_sets) - { - frame.surround_each_list_element_with_parens = true; - s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "GROUPING SETS" << (s.hilite ? hilite_none : ""); - s.ostr << " ("; - s.one_line - ? groupBy()->formatImpl(s, state, frame) - : groupBy()->as().formatImplMultiline(s, state, frame); - s.ostr << ")"; - frame.surround_each_list_element_with_parens = false; - } - if (group_by_with_totals) s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH TOTALS" << (s.hilite ? hilite_none : ""); diff --git a/src/Parsers/ASTSelectQuery.h b/src/Parsers/ASTSelectQuery.h index a6169c37013..2b004e9e040 100644 --- a/src/Parsers/ASTSelectQuery.h +++ b/src/Parsers/ASTSelectQuery.h @@ -83,7 +83,6 @@ public: bool group_by_with_rollup = false; bool group_by_with_cube = false; bool group_by_with_constant_keys = false; - bool group_by_with_grouping_sets = false; bool limit_with_ties = false; ASTPtr & refSelect() { return getExpression(Expression::SELECT); } diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 157b67a6be3..680d3f6031b 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -749,61 +749,13 @@ bool ParserNotEmptyExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected return nested_parser.parse(pos, node, expected) && !node->children.empty(); } + bool ParserOrderByExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { return ParserList(std::make_unique(), std::make_unique(TokenType::Comma), false) .parse(pos, node, expected); } -bool ParserGroupingSetsExpressionListElements::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - auto command_list = std::make_shared(); - node = command_list; - - ParserToken s_comma(TokenType::Comma); - ParserToken s_open(TokenType::OpeningRoundBracket); - ParserToken s_close(TokenType::ClosingRoundBracket); - ParserExpressionWithOptionalAlias p_expression(false); - ParserList p_command(std::make_unique(false), - std::make_unique(TokenType::Comma), true); - - do - { - Pos begin = pos; - ASTPtr command; - if (!s_open.ignore(pos, expected)) - { - pos = begin; - if (!p_expression.parse(pos, command, expected)) - { - return false; - } - auto list = std::make_shared(','); - list->children.push_back(command); - command = std::move(list); - } - else - { - if (!p_command.parse(pos, command, expected)) - return false; - - if (!s_close.ignore(pos, expected)) - break; - } - - command_list->children.push_back(command); - } - while (s_comma.ignore(pos, expected)); - - return true; -} - -bool ParserGroupingSetsExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - ParserGroupingSetsExpressionListElements grouping_sets_elements; - return grouping_sets_elements.parse(pos, node, expected); - -} bool ParserTTLExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 730f6b8e3cd..e9389f15bbb 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -511,20 +511,6 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; -class ParserGroupingSetsExpressionList : public IParserBase -{ -protected: - const char * getName() const override { return "grouping sets expression"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - -class ParserGroupingSetsExpressionListElements : public IParserBase -{ -protected: - const char * getName() const override { return "grouping sets expression elements"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - /// Parser for key-value pair, where value can be list of pairs. class ParserKeyValuePair : public IParserBase diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index a8b47d32413..2f30a1f7bee 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -224,7 +224,6 @@ public: bool need_parens = false; bool expression_list_always_start_on_new_line = false; /// Line feed and indent before expression list even if it's of single element. bool expression_list_prepend_whitespace = false; /// Prepend whitespace (if it is required) - bool surround_each_list_element_with_parens = false; const IAST * current_select = nullptr; }; diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index f767ee88b96..90ab5911d6b 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -50,7 +50,6 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_by("BY"); ParserKeyword s_rollup("ROLLUP"); ParserKeyword s_cube("CUBE"); - ParserKeyword s_grouping_sets("GROUPING SETS"); ParserKeyword s_top("TOP"); ParserKeyword s_with_ties("WITH TIES"); ParserKeyword s_offset("OFFSET"); @@ -66,7 +65,6 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserNotEmptyExpressionList exp_list_for_select_clause(true); /// Allows aliases without AS keyword. ParserExpressionWithOptionalAlias exp_elem(false); ParserOrderByExpressionList order_list; - ParserGroupingSetsExpressionList grouping_sets_list; ParserToken open_bracket(TokenType::OpeningRoundBracket); ParserToken close_bracket(TokenType::ClosingRoundBracket); @@ -186,39 +184,24 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) select_query->group_by_with_rollup = true; else if (s_cube.ignore(pos, expected)) select_query->group_by_with_cube = true; - else if (s_grouping_sets.ignore(pos, expected)) - select_query->group_by_with_grouping_sets = true; - if ((select_query->group_by_with_rollup || select_query->group_by_with_cube || select_query->group_by_with_grouping_sets) && - !open_bracket.ignore(pos, expected)) + if ((select_query->group_by_with_rollup || select_query->group_by_with_cube) && !open_bracket.ignore(pos, expected)) return false; - if (select_query->group_by_with_grouping_sets) - { - if (!grouping_sets_list.parse(pos, group_expression_list, expected)) - return false; - } - else - { - if (!exp_list.parse(pos, group_expression_list, expected)) - return false; - } + if (!exp_list.parse(pos, group_expression_list, expected)) + return false; - - if ((select_query->group_by_with_rollup || select_query->group_by_with_cube || select_query->group_by_with_grouping_sets) && - !close_bracket.ignore(pos, expected)) + if ((select_query->group_by_with_rollup || select_query->group_by_with_cube) && !close_bracket.ignore(pos, expected)) return false; } - /// WITH ROLLUP, CUBE, GROUPING SETS or TOTALS + /// WITH ROLLUP, CUBE or TOTALS if (s_with.ignore(pos, expected)) { if (s_rollup.ignore(pos, expected)) select_query->group_by_with_rollup = true; else if (s_cube.ignore(pos, expected)) select_query->group_by_with_cube = true; - else if (s_grouping_sets.ignore(pos, expected)) - select_query->group_by_with_grouping_sets = true; else if (s_totals.ignore(pos, expected)) select_query->group_by_with_totals = true; else diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 4c476a2841f..154ff0abc0a 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -56,8 +56,6 @@ private: Processors aggregating; - Poco::Logger * log = &Poco::Logger::get("AggregatingStep"); - }; } diff --git a/src/Processors/QueryPlan/GroupingSetsStep.cpp b/src/Processors/QueryPlan/GroupingSetsStep.cpp deleted file mode 100644 index 383ecf5ea1b..00000000000 --- a/src/Processors/QueryPlan/GroupingSetsStep.cpp +++ /dev/null @@ -1,46 +0,0 @@ -#include -#include -#include "QueryPipeline/QueryPipelineBuilder.h" - -namespace DB -{ - -static ITransformingStep::Traits getTraits() -{ - return ITransformingStep::Traits - { - { - .preserves_distinct_columns = false, - .returns_single_stream = true, - .preserves_number_of_streams = false, - .preserves_sorting = false, - }, - { - .preserves_number_of_rows = false, - } - }; -} - -GroupingSetsStep::GroupingSetsStep(const DataStream & input_stream_, AggregatingTransformParamsPtr params_) - : ITransformingStep(input_stream_, params_->getHeader(), getTraits()) - , params(std::move(params_)) -{ - /// Aggregation keys are distinct - for (auto key : params->params.keys) - output_stream->distinct_columns.insert(params->params.src_header.getByPosition(key).name); -} - -void GroupingSetsStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - pipeline.resize(1); - - pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr - { - if (stream_type == QueryPipelineBuilder::StreamType::Totals) - return nullptr; - - return std::make_shared(header, std::move(params)); - }); -} - -} diff --git a/src/Processors/QueryPlan/GroupingSetsStep.h b/src/Processors/QueryPlan/GroupingSetsStep.h deleted file mode 100644 index 3dc7c3b5a99..00000000000 --- a/src/Processors/QueryPlan/GroupingSetsStep.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once -#include -#include -#include "QueryPipeline/QueryPipelineBuilder.h" - -namespace DB -{ - -struct AggregatingTransformParams; -using AggregatingTransformParamsPtr = std::shared_ptr; - -class GroupingSetsStep : public ITransformingStep -{ -public: - GroupingSetsStep(const DataStream & input_stream_, AggregatingTransformParamsPtr params_); - - String getName() const override { return "GroupingSets"; } - - void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - -private: - AggregatingTransformParamsPtr params; -}; - -} diff --git a/src/Processors/Transforms/GroupingSetsTransform.cpp b/src/Processors/Transforms/GroupingSetsTransform.cpp deleted file mode 100644 index 960ea0e5d76..00000000000 --- a/src/Processors/Transforms/GroupingSetsTransform.cpp +++ /dev/null @@ -1,76 +0,0 @@ -#include -#include - -namespace DB -{ - -GroupingSetsTransform::GroupingSetsTransform(Block header, AggregatingTransformParamsPtr params_) - : IAccumulatingTransform(std::move(header), params_->getHeader()) - , params(std::move(params_)) - , keys(params->params.keys) - , keys_vector(params->params.keys_vector) - , keys_vector_idx(0) -{ -} - -void GroupingSetsTransform::consume(Chunk chunk) -{ - consumed_chunks.emplace_back(std::move(chunk)); -} - -Chunk GroupingSetsTransform::merge(Chunks && chunks, bool final) -{ - BlocksList grouping_sets_blocks; - for (auto & chunk : chunks) - grouping_sets_blocks.emplace_back(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns())); - - auto grouping_sets_block = params->aggregator.mergeBlocks(grouping_sets_blocks, final); - auto num_rows = grouping_sets_block.rows(); - return Chunk(grouping_sets_block.getColumns(), num_rows); -} - -Chunk GroupingSetsTransform::generate() -{ - if (!consumed_chunks.empty()) - { - if (consumed_chunks.size() > 1) - grouping_sets_chunk = merge(std::move(consumed_chunks), false); - else - grouping_sets_chunk = std::move(consumed_chunks.front()); - - consumed_chunks.clear(); - - auto num_rows = grouping_sets_chunk.getNumRows(); - - current_columns = grouping_sets_chunk.getColumns(); - current_zero_columns.clear(); - - for (auto key : keys) - current_zero_columns.emplace(key, current_columns[key]->cloneEmpty()->cloneResized(num_rows)); - } - - Chunk gen_chunk; - - if (!current_columns.empty() && keys_vector_idx < keys_vector.size()) - { - auto columns = current_columns; - std::set key_vector(keys_vector[keys_vector_idx].begin(), keys_vector[keys_vector_idx].end()); - - for (auto key : keys) - { - if (!key_vector.contains(key)) - columns[key] = current_zero_columns[key]; - } - - Chunks chunks; - chunks.emplace_back(std::move(columns), current_columns.front()->size()); - gen_chunk = merge(std::move(chunks), false); - - ++keys_vector_idx; - } - - finalizeChunk(gen_chunk); - return gen_chunk; -} - -} diff --git a/src/Processors/Transforms/GroupingSetsTransform.h b/src/Processors/Transforms/GroupingSetsTransform.h deleted file mode 100644 index e9b20867ef4..00000000000 --- a/src/Processors/Transforms/GroupingSetsTransform.h +++ /dev/null @@ -1,35 +0,0 @@ -#pragma once -#include -#include - -namespace DB -{ - -class GroupingSetsTransform : public IAccumulatingTransform -{ -public: - GroupingSetsTransform(Block header, AggregatingTransformParamsPtr params); - String getName() const override { return "GroupingSetsTransform"; } - -protected: - void consume(Chunk chunk) override; - Chunk generate() override; - -private: - AggregatingTransformParamsPtr params; - const ColumnNumbers & keys; - const ColumnNumbersList & keys_vector; - - Chunks consumed_chunks; - Chunk grouping_sets_chunk; - Columns current_columns; - std::unordered_map current_zero_columns; - - UInt64 keys_vector_idx = 0; - - Poco::Logger * log = &Poco::Logger::get("GroupingSetsTransform"); - - Chunk merge(Chunks && chunks, bool final); -}; - -} diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index 25c5f7c0781..6cef7cc28bd 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -497,7 +497,7 @@ void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort for (size_t i = 1; i < output_ports.size(); ++i) assertBlocksHaveEqualStructure(header, output_ports[i]->getHeader(), "Pipes"); - // Temporarily skip this check. TotalsHavingTransform may return finalized totals but not finalized data. + // Temporarily skip this check. TotaslHavingTransform may return finalized totals but not finalized data. // if (totals_port) // assertBlocksHaveEqualStructure(header, totals_port->getHeader(), "Pipes"); diff --git a/src/QueryPipeline/Pipe.h b/src/QueryPipeline/Pipe.h index 01a5d8bb961..613e92a782d 100644 --- a/src/QueryPipeline/Pipe.h +++ b/src/QueryPipeline/Pipe.h @@ -4,7 +4,6 @@ #include #include #include -#include namespace DB { @@ -137,8 +136,6 @@ private: /// It is needed for debug. See QueryPipelineProcessorsCollector. Processors * collected_processors = nullptr; - Poco::Logger * log = &Poco::Logger::get("Pipe"); - /// This methods are for QueryPipeline. It is allowed to complete graph only there. /// So, we may be sure that Pipe always has output port if not empty. bool isCompleted() const { return !empty() && output_ports.empty(); } diff --git a/tests/queries/0_stateless/01883_with_grouping_sets.reference b/tests/queries/0_stateless/01883_with_grouping_sets.reference deleted file mode 100644 index e52f72b84b9..00000000000 --- a/tests/queries/0_stateless/01883_with_grouping_sets.reference +++ /dev/null @@ -1,89 +0,0 @@ -1 0 1 4500 -1 0 3 4700 -1 0 5 4900 -1 0 7 5100 -1 0 9 5300 -1 1 0 4500 -1 2 0 5100 -1 3 0 4700 -1 4 0 5300 -1 5 0 4900 -2 0 2 4600 -2 0 4 4800 -2 0 6 5000 -2 0 8 5200 -2 0 10 5400 -2 1 0 5000 -2 2 0 4600 -2 3 0 5200 -2 4 0 4800 -2 5 0 5400 -0 0 1 1 4500 -0 0 2 2 4600 -0 0 3 3 4700 -0 0 4 4 4800 -0 0 5 5 4900 -0 0 6 6 5000 -0 0 7 7 5100 -0 0 8 8 5200 -0 0 9 9 5300 -0 0 10 10 5400 -1 1 0 0 4500 -1 2 0 0 5100 -1 3 0 0 4700 -1 4 0 0 5300 -1 5 0 0 4900 -2 1 0 0 5000 -2 2 0 0 4600 -2 3 0 0 5200 -2 4 0 0 4800 -2 5 0 0 5400 -0 0 0 49500 -0 0 1 4500 -0 0 2 4600 -0 0 3 4700 -0 0 4 4800 -0 0 5 4900 -0 0 6 5000 -0 0 7 5100 -0 0 8 5200 -0 0 9 5300 -0 0 10 5400 -1 1 0 4500 -1 2 0 5100 -1 3 0 4700 -1 4 0 5300 -1 5 0 4900 -2 1 0 5000 -2 2 0 4600 -2 3 0 5200 -2 4 0 4800 -2 5 0 5400 -1 0 24500 -1 1 4500 -1 3 4700 -1 5 4900 -1 7 5100 -1 9 5300 -2 0 25000 -2 2 4600 -2 4 4800 -2 6 5000 -2 8 5200 -2 10 5400 - -0 0 49500 -1 0 24500 -1 1 4500 -1 3 4700 -1 5 4900 -1 7 5100 -1 9 5300 -2 0 25000 -2 2 4600 -2 4 4800 -2 6 5000 -2 8 5200 -2 10 5400 - -0 0 49500 diff --git a/tests/queries/0_stateless/01883_with_grouping_sets.sql b/tests/queries/0_stateless/01883_with_grouping_sets.sql deleted file mode 100644 index 92a56be0a3f..00000000000 --- a/tests/queries/0_stateless/01883_with_grouping_sets.sql +++ /dev/null @@ -1,44 +0,0 @@ -DROP TABLE IF EXISTS grouping_sets; - -CREATE TABLE grouping_sets(fact_1_id Int32, fact_2_id Int32, fact_3_id Int32, fact_4_id Int32, sales_value Int32) ENGINE = Memory; - -SELECT fact_1_id, fact_3_id, sum(sales_value), count() from grouping_sets GROUP BY GROUPING SETS(fact_1_id, fact_3_id) ORDER BY fact_1_id, fact_3_id; - -INSERT INTO grouping_sets -SELECT - number % 2 + 1 AS fact_1_id, - number % 5 + 1 AS fact_2_id, - number % 10 + 1 AS fact_3_id, - number % 10 + 1 AS fact_4_id, - number % 100 AS sales_value -FROM system.numbers limit 1000; - -SELECT fact_1_id, fact_2_id, fact_3_id, SUM(sales_value) AS sales_value from grouping_sets -GROUP BY GROUPING SETS ((fact_1_id, fact_2_id), (fact_1_id, fact_3_id)) -ORDER BY fact_1_id, fact_2_id, fact_3_id; - -SELECT fact_1_id, fact_2_id, fact_3_id, fact_4_id, SUM(sales_value) AS sales_value from grouping_sets -GROUP BY GROUPING SETS ((fact_1_id, fact_2_id), (fact_3_id, fact_4_id)) -ORDER BY fact_1_id, fact_2_id, fact_3_id, fact_4_id; - -SELECT fact_1_id, fact_2_id, fact_3_id, SUM(sales_value) AS sales_value from grouping_sets -GROUP BY GROUPING SETS ((fact_1_id, fact_2_id), (fact_3_id), ()) -ORDER BY fact_1_id, fact_2_id, fact_3_id; - -SELECT - fact_1_id, - fact_3_id, - SUM(sales_value) AS sales_value -FROM grouping_sets -GROUP BY grouping sets ((fact_1_id), (fact_1_id, fact_3_id)) WITH TOTALS -ORDER BY fact_1_id, fact_3_id; - -SELECT - fact_1_id, - fact_3_id, - SUM(sales_value) AS sales_value -FROM grouping_sets -GROUP BY grouping sets (fact_1_id, (fact_1_id, fact_3_id)) WITH TOTALS -ORDER BY fact_1_id, fact_3_id; - -DROP TABLE grouping_sets; From b82b33d34b545951b26f43f144701a227122cd3a Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Sun, 26 Dec 2021 02:10:42 +0800 Subject: [PATCH 0374/1260] Update parts.md --- docs/zh/operations/system-tables/parts.md | 168 ++++++++++++++++------ 1 file changed, 125 insertions(+), 43 deletions(-) diff --git a/docs/zh/operations/system-tables/parts.md b/docs/zh/operations/system-tables/parts.md index e924ee27df3..dc98288305f 100644 --- a/docs/zh/operations/system-tables/parts.md +++ b/docs/zh/operations/system-tables/parts.md @@ -1,85 +1,167 @@ ---- -machine_translated: true -machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 ---- +# system.parts {#system_tables-parts} -# 系统。零件 {#system_tables-parts} +此系统表包含 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 表分区的相关信息。 -包含有关的部分信息 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 桌子 - -每行描述一个数据部分。 +每一行描述一个数据分区。 列: -- `partition` (String) – The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter.md#query_language_queries_alter) 查询。 +- `partition` ([String](../../sql-reference/data-types/string.md)) – 分区名称。请参阅 [ALTER](../../sql-reference/statements/alter/index.md#query_language_queries_alter) 查询的说明,来了解什么是分区。 格式: - `YYYYMM` 用于按月自动分区。 - - `any_string` 手动分区时。 + - `any_string` 手动分区时,是其他格式的字符串。 -- `name` (`String`) – Name of the data part. +- `name` ([String](../../sql-reference/data-types/string.md)) – 数据分区的名称。 -- `active` (`UInt8`) – Flag that indicates whether the data part is active. If a data part is active, it's used in a table. Otherwise, it's deleted. Inactive data parts remain after merging. +- `part_type` ([String](../../sql-reference/data-types/string.md)) — 数据分区的存储格式。 -- `marks` (`UInt64`) – The number of marks. To get the approximate number of rows in a data part, multiply `marks` 通过索引粒度(通常为8192)(此提示不适用于自适应粒度)。 + 可能的值: -- `rows` (`UInt64`) – The number of rows. + - `Wide` — 每一列在文件系统中的一个单独文件中存储。 + - `Compact` — 所有列在文件系统中的一个文件中存储。 -- `bytes_on_disk` (`UInt64`) – Total size of all the data part files in bytes. + 数据存储格式由 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 表的 `min_bytes_for_wide_part` 和 `min_rows_for_wide_part` 控制。 -- `data_compressed_bytes` (`UInt64`) – Total size of compressed data in the data part. All the auxiliary files (for example, files with marks) are not included. + - `active` ([UInt8](../../sql-reference/data-types/int-uint.md)) – 指示数据分区是否处于活动状态的标志。如果数据分区处于活动状态,则此数据正在被表使用。反之,则不活跃(deleted)。合并后仍会保留非活跃的数据分区。 -- `data_uncompressed_bytes` (`UInt64`) – Total size of uncompressed data in the data part. All the auxiliary files (for example, files with marks) are not included. +- `marks` ([UInt64](../../sql-reference/data-types/int-uint.md)) – 标记数。要获得数据分区中的大致行数:使用`marks`(标记数)乘以索引粒度(通常为 8192)。不适用于自适应颗粒度。 -- `marks_bytes` (`UInt64`) – The size of the file with marks. +- `rows` ([UInt64](../../sql-reference/data-types/int-uint.md)) – 行数. -- `modification_time` (`DateTime`) – The time the directory with the data part was modified. This usually corresponds to the time of data part creation.\| +- `bytes_on_disk` ([UInt64](../../sql-reference/data-types/int-uint.md)) – 数据总大小(以字节为单位)。 -- `remove_time` (`DateTime`) – The time when the data part became inactive. +- `data_compressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – 数据分区中压缩数据的总大小。不包括所有辅助文件(例如,带有标记的文件)。 -- `refcount` (`UInt32`) – The number of places where the data part is used. A value greater than 2 indicates that the data part is used in queries or merges. +- `data_uncompressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – 数据分区中未压缩数据的总大小。不包括所有辅助文件(例如,带有标记的文件)。 -- `min_date` (`Date`) – The minimum value of the date key in the data part. +- `marks_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – 带有标记的文件的大小。 -- `max_date` (`Date`) – The maximum value of the date key in the data part. +- `secondary_indices_compressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – 数据分区中二级索引的压缩数据总大小。所有的辅助文件(例如,带有标记的文件)都不包括在内。 -- `min_time` (`DateTime`) – The minimum value of the date and time key in the data part. +- `secondary_indices_uncompressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – 数据分区中二级索引的未压缩数据的总大小。所有的辅助文件(例如,带有标记的文件)都不包括在内。 -- `max_time`(`DateTime`) – The maximum value of the date and time key in the data part. +- `secondary_indices_marks_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – 带标记的二级索引的文件大小。 -- `partition_id` (`String`) – ID of the partition. +- `modification_time` ([DateTime](../../sql-reference/data-types/datetime.md)) – 包含数据分区的目录被修改的时间。这通常对应于数据部分创建的时间。 -- `min_block_number` (`UInt64`) – The minimum number of data parts that make up the current part after merging. +- `remove_time` ([DateTime](../../sql-reference/data-types/datetime.md)) – 数据分区变为非活动状态的时间。 -- `max_block_number` (`UInt64`) – The maximum number of data parts that make up the current part after merging. +- `refcount` ([UInt32](../../sql-reference/data-types/int-uint.md)) – 使用数据部分的位置数。大于 2 的值表示数据部分用于查询或是用于合并。 -- `level` (`UInt32`) – Depth of the merge tree. Zero means that the current part was created by insert rather than by merging other parts. +- `min_date` ([Date](../../sql-reference/data-types/date.md)) – 数据部分中日期键的最小值。 -- `data_version` (`UInt64`) – Number that is used to determine which mutations should be applied to the data part (mutations with a version higher than `data_version`). +- `max_date` ([Date](../../sql-reference/data-types/date.md)) – 数据部分中日期键的最大值。 -- `primary_key_bytes_in_memory` (`UInt64`) – The amount of memory (in bytes) used by primary key values. +- `min_time` ([DateTime](../../sql-reference/data-types/datetime.md)) – 数据部分中日期和时间键的最小值。 -- `primary_key_bytes_in_memory_allocated` (`UInt64`) – The amount of memory (in bytes) reserved for primary key values. +- `max_time`([DateTime](../../sql-reference/data-types/datetime.md)) – 数据部分中日期和时间键的最大值。 -- `is_frozen` (`UInt8`) – Flag that shows that a partition data backup exists. 1, the backup exists. 0, the backup doesn't exist. For more details, see [FREEZE PARTITION](../../sql-reference/statements/alter.md#alter_freeze-partition) +- `partition_id` ([String](../../sql-reference/data-types/string.md)) – 分区的 ID。 -- `database` (`String`) – Name of the database. +- `min_block_number` ([UInt64](../../sql-reference/data-types/int-uint.md)) – 合并后构成当前部分的最小数据部分数量。 -- `table` (`String`) – Name of the table. +- `max_block_number` ([UInt64](../../sql-reference/data-types/int-uint.md)) – 合并后构成当前部分的最大数据部分数量。 -- `engine` (`String`) – Name of the table engine without parameters. +- `level` ([UInt32](../../sql-reference/data-types/int-uint.md)) – 合并树的深度。值为 0 表示该分区是通过插入创建的,而不是通过合并创建的。 -- `path` (`String`) – Absolute path to the folder with data part files. +- `data_version` ([UInt64](../../sql-reference/data-types/int-uint.md)) – 用于确定应将哪些订正(mutations)应用于数据部分(版本高于 `data_version` 的订正(mutations))的数字。 -- `disk` (`String`) – Name of a disk that stores the data part. +- `primary_key_bytes_in_memory` ([UInt64](../../sql-reference/data-types/int-uint.md)) – 主键值使用的内存量(以字节为单位)。 -- `hash_of_all_files` (`String`) – [sipHash128](../../sql-reference/functions/hash-functions.md#hash_functions-siphash128) 的压缩文件。 +- `primary_key_bytes_in_memory_allocated` ([UInt64](../../sql-reference/data-types/int-uint.md)) – 为主键值保留的内存量(以字节为单位)。 -- `hash_of_uncompressed_files` (`String`) – [sipHash128](../../sql-reference/functions/hash-functions.md#hash_functions-siphash128) 未压缩的文件(带标记的文件,索引文件等。). +- `is_frozen` ([UInt8](../../sql-reference/data-types/int-uint.md)) – 显示分区数据备份存在的标志。1,备份存在。0,备份不存在。更多细节,见 [FREEZE PARTITION](../../sql-reference/statements/alter/partition.md#alter_freeze-partition)。 -- `uncompressed_hash_of_compressed_files` (`String`) – [sipHash128](../../sql-reference/functions/hash-functions.md#hash_functions-siphash128) 压缩文件中的数据,就好像它们是未压缩的。 +- `database` ([String](../../sql-reference/data-types/string.md)) – 数据库的名称。 -- `bytes` (`UInt64`) – Alias for `bytes_on_disk`. +- `table` ([String](../../sql-reference/data-types/string.md)) – 表的名称。 -- `marks_size` (`UInt64`) – Alias for `marks_bytes`. +- `engine` ([String](../../sql-reference/data-types/string.md)) – 不带参数的表引擎名称。 + +- `path` ([String](../../sql-reference/data-types/string.md)) – 包含数据部分文件的文件夹的绝对路径。 + +- `disk` ([String](../../sql-reference/data-types/string.md)) – 存储数据部分的磁盘的名称。 + +- `hash_of_all_files` ([String](../../sql-reference/data-types/string.md)) – 压缩文件的 [sipHash128](../../sql-reference/functions/hash-functions.md#hash_functions-siphash128)。 + +- `hash_of_uncompressed_files` ([String](../../sql-reference/data-types/string.md)) – 未压缩文件(带有标记的文件、索引文件等)的 [sipHash128](../../sql-reference/functions/hash-functions.md#hash_functions-siphash128)。 + +- `uncompressed_hash_of_compressed_files` ([String](../../sql-reference/data-types/string.md)) – 压缩文件中的数据(没有压缩时)的 [sipHash128](../../sql-reference/functions/hash-functions.md#hash_functions-siphash128)。 + +- `delete_ttl_info_min` ([DateTime](../../sql-reference/data-types/datetime.md)) — [TTL DELETE 规则](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl) 的日期和时间键的最小值。 + +- `delete_ttl_info_max` ([DateTime](../../sql-reference/data-types/datetime.md)) — [TTL DELETE 规则](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl) 的日期和时间键的最大值。 + +- `move_ttl_info.expression` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — 表达式的数组。 每个表达式定义一个 [TTL MOVE 规则](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). + + !!! note "警告" + 保留 `move_ttl_info.expression` 数组主要是为了向后兼容,现在检查 `TTL MOVE` 规则最简单的方法是使用 `move_ttl_info.min` 和 `move_ttl_info.max` 字段。 + +- `move_ttl_info.min` ([Array](../../sql-reference/data-types/array.md)([DateTime](../../sql-reference/data-types/datetime.md))) — 日期值和时间值的数组。数组中的每个元素都描述了一个 [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl) 的最小键值。 + +- `move_ttl_info.max` ([Array](../../sql-reference/data-types/array.md)([DateTime](../../sql-reference/data-types/datetime.md))) — 日期值和时间值的数组。数组中的每个元素都描述了一个 [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl) 的最大键值。 + +- `bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – `bytes_on_disk`的别名。 + +- `marks_size` ([UInt64](../../sql-reference/data-types/int-uint.md)) – `marks_bytes`的别名。 + +**示例** + +``` sql +SELECT * FROM system.parts LIMIT 1 FORMAT Vertical; +``` + +``` text +Row 1: +────── +partition: tuple() +name: all_1_4_1_6 +part_type: Wide +active: 1 +marks: 2 +rows: 6 +bytes_on_disk: 310 +data_compressed_bytes: 157 +data_uncompressed_bytes: 91 +secondary_indices_compressed_bytes: 58 +secondary_indices_uncompressed_bytes: 6 +secondary_indices_marks_bytes: 48 +marks_bytes: 144 +modification_time: 2020-06-18 13:01:49 +remove_time: 1970-01-01 00:00:00 +refcount: 1 +min_date: 1970-01-01 +max_date: 1970-01-01 +min_time: 1970-01-01 00:00:00 +max_time: 1970-01-01 00:00:00 +partition_id: all +min_block_number: 1 +max_block_number: 4 +level: 1 +data_version: 6 +primary_key_bytes_in_memory: 8 +primary_key_bytes_in_memory_allocated: 64 +is_frozen: 0 +database: default +table: months +engine: MergeTree +disk_name: default +path: /var/lib/clickhouse/data/default/months/all_1_4_1_6/ +hash_of_all_files: 2d0657a16d9430824d35e327fcbd87bf +hash_of_uncompressed_files: 84950cc30ba867c77a408ae21332ba29 +uncompressed_hash_of_compressed_files: 1ad78f1c6843bbfb99a2c931abe7df7d +delete_ttl_info_min: 1970-01-01 00:00:00 +delete_ttl_info_max: 1970-01-01 00:00:00 +move_ttl_info.expression: [] +move_ttl_info.min: [] +move_ttl_info.max: [] +``` + +**另请参阅** + +- [MergeTree(合并树)家族](../../engines/table-engines/mergetree-family/mergetree.md) +- [列和表的 TTL](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl) + +[原文](https://clickhouse.com/docs/zh/operations/system-tables/parts) From ef22ebad853a2331604d97cb4eae0eca2794271f Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 25 Dec 2021 17:10:21 +0300 Subject: [PATCH 0375/1260] Better startup --- .../DatabaseMaterializedPostgreSQL.cpp | 16 ++++--------- .../DatabaseMaterializedPostgreSQL.h | 2 ++ .../PostgreSQLReplicationHandler.cpp | 13 +++++++---- .../PostgreSQL/PostgreSQLReplicationHandler.h | 2 +- .../StorageMaterializedPostgreSQL.cpp | 23 ++----------------- .../StorageMaterializedPostgreSQL.h | 2 -- 6 files changed, 18 insertions(+), 40 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 5d699955ee5..e5ff9dd6105 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -51,6 +51,7 @@ DatabaseMaterializedPostgreSQL::DatabaseMaterializedPostgreSQL( , connection_info(connection_info_) , settings(std::move(settings_)) { + startup_task = getContext()->getSchedulePool().createTask("MaterializedPostgreSQLDatabaseStartup", [this]{ startSynchronization(); }); } @@ -104,24 +105,14 @@ void DatabaseMaterializedPostgreSQL::startSynchronization() } LOG_TRACE(log, "Loaded {} tables. Starting synchronization", materialized_tables.size()); - replication_handler->startup(); + replication_handler->startup(/* delayed */false); } void DatabaseMaterializedPostgreSQL::startupTables(ThreadPool & thread_pool, bool force_restore, bool force_attach) { DatabaseAtomic::startupTables(thread_pool, force_restore, force_attach); - try - { - startSynchronization(); - } - catch (...) - { - tryLogCurrentException(log, "Cannot load nested database objects for PostgreSQL database engine."); - - if (!force_attach) - throw; - } + startup_task->activateAndSchedule(); } @@ -376,6 +367,7 @@ StoragePtr DatabaseMaterializedPostgreSQL::detachTable(ContextPtr context_, cons void DatabaseMaterializedPostgreSQL::shutdown() { + startup_task->deactivate(); stopReplication(); DatabaseAtomic::shutdown(); } diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h index 3b7f0f9d29d..02dea446d47 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h @@ -86,6 +86,8 @@ private: std::map materialized_tables; mutable std::mutex tables_mutex; mutable std::mutex handler_mutex; + + BackgroundSchedulePool::TaskHolder startup_task; }; } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 984a9cdd47a..8016e60ee3d 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -78,11 +78,16 @@ void PostgreSQLReplicationHandler::addStorage(const std::string & table_name, St } -void PostgreSQLReplicationHandler::startup() +void PostgreSQLReplicationHandler::startup(bool delayed) { - /// We load tables in a separate thread, because this database is not created yet. - /// (will get "database is currently dropped or renamed") - startup_task->activateAndSchedule(); + if (delayed) + { + startup_task->activateAndSchedule(); + } + else + { + startSynchronization(/* throw_on_error */ true); + } } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index cf44101db76..5928e442de5 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -27,7 +27,7 @@ public: bool is_materialized_postgresql_database_); /// Activate task to be run from a separate thread: wait until connection is available and call startReplication(). - void startup(); + void startup(bool delayed); /// Stop replication without cleanup. void shutdown(); diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index aefd1aedbf7..1b5128299b0 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -87,14 +87,8 @@ StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL( *replication_settings, /* is_materialized_postgresql_database */false); - if (!is_attach) - { - replication_handler->addStorage(remote_table_name, this); - /// Start synchronization preliminary setup immediately and throw in case of failure. - /// It should be guaranteed that if MaterializedPostgreSQL table was created successfully, then - /// its nested table was also created. - replication_handler->startSynchronization(/* throw_on_error */ true); - } + replication_handler->addStorage(remote_table_name, this); + replication_handler->startup(/* delayed */is_attach); } @@ -234,19 +228,6 @@ void StorageMaterializedPostgreSQL::set(StoragePtr nested_storage) } -void StorageMaterializedPostgreSQL::startup() -{ - /// replication_handler != nullptr only in case of single table engine MaterializedPostgreSQL. - if (replication_handler && is_attach) - { - replication_handler->addStorage(remote_table_name, this); - /// In case of attach table use background startup in a separate thread. First wait until connection is reachable, - /// then check for nested table -- it should already be created. - replication_handler->startup(); - } -} - - void StorageMaterializedPostgreSQL::shutdown() { if (replication_handler) diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h index 9e11f314738..ff9b95cad7c 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h @@ -74,8 +74,6 @@ public: String getName() const override { return "MaterializedPostgreSQL"; } - void startup() override; - void shutdown() override; /// Used only for single MaterializedPostgreSQL storage. From c4db4309c29a01ce94c519553d46c5d617f13aeb Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 25 Dec 2021 21:47:16 +0300 Subject: [PATCH 0376/1260] Better --- .../materialized-postgresql.md | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 329034b70bb..605de3d24a6 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -46,6 +46,8 @@ After `MaterializedPostgreSQL` database is created, it does not automatically de ATTACH TABLE postgres_database.new_table; ``` +Warning: before version 21.13 adding table to replication left unremoved temprorary replication slot (named `{db_name}_ch_replication_slot_tmp`). If attaching tables in clickhouse version before 21.13, make sure to delete it manually (`SELECT pg_drop_replication_slot('{db_name}_ch_replication_slot_tmp')`). Otherwise disk usage will grow. Issue is fixed in 21.13. + ## Dynamically removing tables from replication {#dynamically-removing-table-from-replication} It is possible to remove specific tables from replication: @@ -250,3 +252,23 @@ SETTINGS ```bash kubectl exec acid-demo-cluster-0 -c postgres -- su postgres -c 'patronictl failover --candidate acid-demo-cluster-1 --force' ``` + +### Required permissions + +1. [CREATE PUBLICATION](https://postgrespro.ru/docs/postgresql/14/sql-createpublication) -- create query privilege. + +2. [CREATE_REPLICATION_SLOT](https://postgrespro.ru/docs/postgrespro/10/protocol-replication#PROTOCOL-REPLICATION-CREATE-SLOT) -- replication privelege. + +3. [pg_drop_replication_slot](https://postgrespro.ru/docs/postgrespro/9.5/functions-admin#functions-replication) -- replication privilege or superuser. + +4. [DROP PUBLICATION](https://postgrespro.ru/docs/postgresql/10/sql-droppublication) -- owner of publication (`username` in MaterializedPostgreSQL engine itself). + +It is possible to avoid executing `2` and `3` commands and having those permissions. Use settings `materialized_postgresql_replication_slot` and `materialized_postgresql_snapshot`. But with much care. + +Access to tables: + +1. pg_publication + +2. pg_replication_slots + +3. pg_publication_tables From 38ca7914656f20290dc0757b578e1fbe0f9f94b2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 25 Dec 2021 22:00:28 +0300 Subject: [PATCH 0377/1260] Fix --- programs/local/LocalServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 8da8641bcdc..53e295b7fbb 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -792,9 +792,9 @@ void LocalServer::processOptions(const OptionsDescription &, const CommandLineOp int mainEntryClickHouseLocal(int argc, char ** argv) { - DB::LocalServer app; try { + DB::LocalServer app; app.init(argc, argv); return app.run(); } From f7569f5e4305ee2e9074b9d729f225eccd7220e1 Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Sun, 26 Dec 2021 12:26:43 +0800 Subject: [PATCH 0378/1260] Update numbers.md --- docs/zh/operations/system-tables/numbers.md | 36 ++++++++++++++++----- 1 file changed, 28 insertions(+), 8 deletions(-) diff --git a/docs/zh/operations/system-tables/numbers.md b/docs/zh/operations/system-tables/numbers.md index c42c87053ca..fd67baa01a5 100644 --- a/docs/zh/operations/system-tables/numbers.md +++ b/docs/zh/operations/system-tables/numbers.md @@ -1,12 +1,32 @@ ---- -machine_translated: true -machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 ---- +# system.numbers {#system-numbers} -# 系统。数字 {#system-numbers} +这个表有一个名为 `number` 的 UInt64 列,包含了几乎所有从 0 开始的自然数。 -此表包含一个名为UInt64的列 `number` 它包含几乎所有从零开始的自然数。 +你可以用这个表进行测试,或者如果你需要进行暴力搜索。 -您可以使用此表进行测试,或者如果您需要进行暴力搜索。 +从该表的读取是不并行的。 -从此表中读取的内容不是并行的。 +**示例** + +```sql +:) SELECT * FROM system.numbers LIMIT 10; +``` + +```text +┌─number─┐ +│ 0 │ +│ 1 │ +│ 2 │ +│ 3 │ +│ 4 │ +│ 5 │ +│ 6 │ +│ 7 │ +│ 8 │ +│ 9 │ +└────────┘ + +10 rows in set. Elapsed: 0.001 sec. +``` + +[原文](https://clickhouse.com/docs/zh/operations/system-tables/numbers) From 2bcb122cd0e8cf733c6f360903708e45d3358096 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 25 Dec 2021 20:36:30 +0300 Subject: [PATCH 0379/1260] Clean up unused replication slots --- .../PostgreSQLReplicationHandler.cpp | 171 +++++++++++------- .../PostgreSQL/PostgreSQLReplicationHandler.h | 8 +- 2 files changed, 114 insertions(+), 65 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 984a9cdd47a..0f62f764381 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -20,6 +20,7 @@ namespace DB static const auto RESCHEDULE_MS = 1000; static const auto BACKOFF_TRESHOLD_MS = 10000; +static const auto CLEANUP_RESCHEDULE_MS = 600000 * 3; /// 30 min namespace ErrorCodes { @@ -28,6 +29,30 @@ namespace ErrorCodes extern const int POSTGRESQL_REPLICATION_INTERNAL_ERROR; } +class TemporaryReplicationSlot +{ +public: + TemporaryReplicationSlot( + PostgreSQLReplicationHandler * handler_, + std::shared_ptr tx_, + String & start_lsn, + String & snapshot_name) + : handler(handler_), tx(tx_) + { + handler->createReplicationSlot(*tx, start_lsn, snapshot_name, /* temporary */true); + } + + ~TemporaryReplicationSlot() + { + handler->dropReplicationSlot(*tx, /* temporary */true); + } + +private: + PostgreSQLReplicationHandler * handler; + std::shared_ptr tx; +}; + + PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const String & replication_identifier, const String & postgres_database_, @@ -69,6 +94,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( startup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ checkConnectionAndStart(); }); consumer_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ consumerFunc(); }); + cleanup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ cleanupFunc(); }); } @@ -148,6 +174,7 @@ void PostgreSQLReplicationHandler::shutdown() stop_synchronization.store(true); startup_task->deactivate(); consumer_task->deactivate(); + cleanup_task->deactivate(); } @@ -268,6 +295,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) (is_materialized_postgresql_database ? postgres_database : postgres_database + '.' + tables_list)); consumer_task->activateAndSchedule(); + cleanup_task->activateAndSchedule(); /// Do not rely anymore on saved storage pointers. materialized_storages.clear(); @@ -279,8 +307,6 @@ ASTPtr PostgreSQLReplicationHandler::getCreateNestedTableQuery(StorageMaterializ postgres::Connection connection(connection_info); pqxx::nontransaction tx(connection.getRef()); auto table_structure = std::make_unique(fetchPostgreSQLTableStructure(tx, table_name, postgres_schema, true, true, true)); - if (!table_structure) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to get PostgreSQL table structure"); auto table_override = tryGetTableOverride(current_database_name, table_name); return storage->getCreateNestedTableQuery(std::move(table_structure), table_override ? table_override->as() : nullptr); @@ -331,6 +357,21 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection & } +void PostgreSQLReplicationHandler::cleanupFunc() +{ + /// It is very important to make sure temporary replication slots are removed! + /// So just in case every 30 minutes check if one still exists. + postgres::Connection connection(connection_info); + String last_committed_lsn; + connection.execWithRetry([&](pqxx::nontransaction & tx) + { + if (isReplicationSlotExist(tx, last_committed_lsn, /* temporary */true)) + dropReplicationSlot(tx, /* temporary */true); + }); + cleanup_task->scheduleAfter(CLEANUP_RESCHEDULE_MS); +} + + void PostgreSQLReplicationHandler::consumerFunc() { std::vector> skipped_tables; @@ -767,10 +808,12 @@ void PostgreSQLReplicationHandler::addTableToReplication(StorageMaterializedPost StoragePtr nested_storage; { - pqxx::nontransaction tx(replication_connection.getRef()); - if (isReplicationSlotExist(tx, start_lsn, /* temporary */true)) - dropReplicationSlot(tx, /* temporary */true); - createReplicationSlot(tx, start_lsn, snapshot_name, /* temporary */true); + auto tx = std::make_shared(replication_connection.getRef()); + + if (isReplicationSlotExist(*tx, start_lsn, /* temporary */true)) + dropReplicationSlot(*tx, /* temporary */true); + + TemporaryReplicationSlot temporary_slot(this, tx, start_lsn, snapshot_name); /// Protect against deadlock. auto nested = DatabaseCatalog::instance().tryGetTable(materialized_storage->getNestedStorageID(), materialized_storage->getNestedTableContext()); @@ -841,81 +884,81 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vector(replication_connection.getRef()); - String snapshot_name, start_lsn; - - if (isReplicationSlotExist(tx, start_lsn, /* temporary */true)) - dropReplicationSlot(tx, /* temporary */true); - - createReplicationSlot(tx, start_lsn, snapshot_name, /* temporary */true); - postgres::Connection tmp_connection(connection_info); - - for (const auto & [relation_id, table_name] : relation_data) { - auto storage = DatabaseCatalog::instance().getTable(StorageID(current_database_name, table_name), context); - auto * materialized_storage = storage->as (); - auto materialized_table_lock = materialized_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout); + String snapshot_name, start_lsn; + if (isReplicationSlotExist(*tx, start_lsn, /* temporary */true)) + dropReplicationSlot(*tx, /* temporary */true); - /// If for some reason this temporary table already exists - also drop it. - auto temp_materialized_storage = materialized_storage->createTemporary(); + TemporaryReplicationSlot temporary_slot(this, tx, start_lsn, snapshot_name); + postgres::Connection tmp_connection(connection_info); - /// This snapshot is valid up to the end of the transaction, which exported it. - StoragePtr temp_nested_storage = loadFromSnapshot(tmp_connection, snapshot_name, table_name, - temp_materialized_storage->as ()); - - auto table_id = materialized_storage->getNestedStorageID(); - auto temp_table_id = temp_nested_storage->getStorageID(); - - LOG_DEBUG(log, "Starting background update of table {} ({} with {})", - table_name, table_id.getNameForLogs(), temp_table_id.getNameForLogs()); - - auto ast_rename = std::make_shared(); - ASTRenameQuery::Element elem + for (const auto & [relation_id, table_name] : relation_data) { - ASTRenameQuery::Table{table_id.database_name, table_id.table_name}, - ASTRenameQuery::Table{temp_table_id.database_name, temp_table_id.table_name} - }; - ast_rename->elements.push_back(std::move(elem)); - ast_rename->exchange = true; + auto storage = DatabaseCatalog::instance().getTable(StorageID(current_database_name, table_name), context); + auto * materialized_storage = storage->as (); + auto materialized_table_lock = materialized_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout); - auto nested_context = materialized_storage->getNestedTableContext(); + /// If for some reason this temporary table already exists - also drop it. + auto temp_materialized_storage = materialized_storage->createTemporary(); - try - { - InterpreterRenameQuery(ast_rename, nested_context).execute(); + /// This snapshot is valid up to the end of the transaction, which exported it. + StoragePtr temp_nested_storage = loadFromSnapshot(tmp_connection, snapshot_name, table_name, + temp_materialized_storage->as ()); - auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, table_id.table_name, temp_table_id.uuid), nested_context); - materialized_storage->set(nested_storage); + auto table_id = materialized_storage->getNestedStorageID(); + auto temp_table_id = temp_nested_storage->getStorageID(); - auto nested_sample_block = nested_storage->getInMemoryMetadataPtr()->getSampleBlock(); - auto materialized_sample_block = materialized_storage->getInMemoryMetadataPtr()->getSampleBlock(); - assertBlocksHaveEqualStructure(nested_sample_block, materialized_sample_block, "while reloading table in the background"); + LOG_DEBUG(log, "Starting background update of table {} ({} with {})", + table_name, table_id.getNameForLogs(), temp_table_id.getNameForLogs()); - LOG_INFO(log, "Updated table {}. New structure: {}", - nested_storage->getStorageID().getNameForLogs(), nested_sample_block.dumpStructure()); + auto ast_rename = std::make_shared(); + ASTRenameQuery::Element elem + { + ASTRenameQuery::Table{table_id.database_name, table_id.table_name}, + ASTRenameQuery::Table{temp_table_id.database_name, temp_table_id.table_name} + }; + ast_rename->elements.push_back(std::move(elem)); + ast_rename->exchange = true; - /// Pass pointer to new nested table into replication consumer, remove current table from skip list and set start lsn position. - consumer->updateNested(table_name, nested_storage, relation_id, start_lsn); + auto nested_context = materialized_storage->getNestedTableContext(); - auto table_to_drop = DatabaseCatalog::instance().getTable(StorageID(temp_table_id.database_name, temp_table_id.table_name, table_id.uuid), nested_context); - auto drop_table_id = table_to_drop->getStorageID(); + try + { + InterpreterRenameQuery(ast_rename, nested_context).execute(); - if (drop_table_id == nested_storage->getStorageID()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cannot drop table because is has the same uuid as new table: {}", drop_table_id.getNameForLogs()); + auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, table_id.table_name, temp_table_id.uuid), nested_context); + materialized_storage->set(nested_storage); - LOG_DEBUG(log, "Dropping table {}", drop_table_id.getNameForLogs()); - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, nested_context, nested_context, drop_table_id, true); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); + auto nested_sample_block = nested_storage->getInMemoryMetadataPtr()->getSampleBlock(); + auto materialized_sample_block = materialized_storage->getInMemoryMetadataPtr()->getSampleBlock(); + assertBlocksHaveEqualStructure(nested_sample_block, materialized_sample_block, "while reloading table in the background"); + + LOG_INFO(log, "Updated table {}. New structure: {}", + nested_storage->getStorageID().getNameForLogs(), nested_sample_block.dumpStructure()); + + /// Pass pointer to new nested table into replication consumer, remove current table from skip list and set start lsn position. + consumer->updateNested(table_name, nested_storage, relation_id, start_lsn); + + auto table_to_drop = DatabaseCatalog::instance().getTable(StorageID(temp_table_id.database_name, temp_table_id.table_name, table_id.uuid), nested_context); + auto drop_table_id = table_to_drop->getStorageID(); + + if (drop_table_id == nested_storage->getStorageID()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot drop table because is has the same uuid as new table: {}", drop_table_id.getNameForLogs()); + + LOG_DEBUG(log, "Dropping table {}", drop_table_id.getNameForLogs()); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, nested_context, nested_context, drop_table_id, true); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } } - dropReplicationSlot(tx, /* temporary */true); - tx.commit(); + tx->commit(); } catch (...) { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index cf44101db76..c0a2a6f2559 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -15,6 +15,8 @@ struct SettingChange; class PostgreSQLReplicationHandler { +friend class TemporaryReplicationSlot; + public: PostgreSQLReplicationHandler( const String & replication_identifier, @@ -52,6 +54,8 @@ public: void setSetting(const SettingChange & setting); + void cleanupFunc(); + private: using MaterializedStorages = std::unordered_map; @@ -133,7 +137,9 @@ private: /// Replication consumer. Manages decoding of replication stream and syncing into tables. std::shared_ptr consumer; - BackgroundSchedulePool::TaskHolder startup_task, consumer_task; + BackgroundSchedulePool::TaskHolder startup_task; + BackgroundSchedulePool::TaskHolder consumer_task; + BackgroundSchedulePool::TaskHolder cleanup_task; std::atomic stop_synchronization = false; From 495faad49f8da3082bed13cfa4b2a7abaf2c0f96 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 26 Dec 2021 10:20:43 +0300 Subject: [PATCH 0380/1260] Update materialized-postgresql.md --- docs/en/engines/database-engines/materialized-postgresql.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 605de3d24a6..4dea156f32e 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -177,11 +177,11 @@ Default value: `65536`. 6. materialized_postgresql_replication_slot {#materialized-postgresql-replication-slot} -A user-created replication slot. Must be used together with [materialized_postgresql_snapshot](#materialized-postgresql-snapshot). +A user-created replication slot. Must be used together with `materialized_postgresql_snapshot`. 7. materialized_postgresql_snapshot {#materialized-postgresql-snapshot} -A text string identifying a snapshot, from which [initial dump of PostgreSQL tables](../../engines/database-engines/materialized-postgresql.md) will be performed. Must be used together with [materialized_postgresql_replication_slot](#materialized-postgresql-replication-slot). +A text string identifying a snapshot, from which [initial dump of PostgreSQL tables](../../engines/database-engines/materialized-postgresql.md) will be performed. Must be used together with `materialized_postgresql_replication_slot`. ``` sql CREATE DATABASE database1 @@ -203,7 +203,7 @@ ALTER DATABASE postgres_database MODIFY SETTING materialized_postgresql_max_bloc Logical Replication Slots which exist on the primary are not available on standby replicas. So if there is a failover, new primary (the old physical standby) won’t be aware of any slots which were existing with old primary. This will lead to a broken replication from PostgreSQL. -A solution to this is to manage replication slots yourself and define a permanent replication slot (some information can be found [here](https://patroni.readthedocs.io/en/latest/SETTINGS.html)). You'll need to pass slot name via [materialized_postgresql_replication_slot](../../operations/settings/settings.md#materialized-postgresql-replication-slot) setting, and it has to be exported with `EXPORT SNAPSHOT` option. The snapshot identifier needs to be passed via [materialized_postgresql_snapshot](../../operations/settings/settings.md#materialized-postgresql-snapshot) setting. +A solution to this is to manage replication slots yourself and define a permanent replication slot (some information can be found [here](https://patroni.readthedocs.io/en/latest/SETTINGS.html)). You'll need to pass slot name via `materialized_postgresql_replication_slot` setting, and it has to be exported with `EXPORT SNAPSHOT` option. The snapshot identifier needs to be passed via `materialized_postgresql_snapshot` setting. Please note that this should be used only if it is actually needed. If there is no real need for that or full understanding why, then it is better to allow the table engine to create and manage its own replication slot. From 8daeefeebe860a60ea3f21975354de8d1768f41e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sun, 26 Dec 2021 15:46:38 +0800 Subject: [PATCH 0381/1260] inject gitinfo --- src/Storages/System/CMakeLists.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/System/CMakeLists.txt b/src/Storages/System/CMakeLists.txt index 6689deac84b..133761cbe22 100644 --- a/src/Storages/System/CMakeLists.txt +++ b/src/Storages/System/CMakeLists.txt @@ -25,8 +25,9 @@ if(Git_FOUND) OUTPUT_VARIABLE GIT_BRANCH ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) # The date of the commit + SET(ENV{TZ} "UTC") execute_process(COMMAND - "${GIT_EXECUTABLE}" log -1 --format=%ad --date=local + "${GIT_EXECUTABLE}" log -1 --format=%ad --date=iso-local WORKING_DIRECTORY "${ClickHouse_SOURCE_DIR}" OUTPUT_VARIABLE GIT_DATE ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE) From d2e09847021015cd29382557ced45b4e33d1b89e Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Sun, 26 Dec 2021 16:15:36 +0800 Subject: [PATCH 0382/1260] Update detached_parts.md --- .../zh/operations/system-tables/detached_parts.md | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/docs/zh/operations/system-tables/detached_parts.md b/docs/zh/operations/system-tables/detached_parts.md index ba35444c551..efcbb61d37e 100644 --- a/docs/zh/operations/system-tables/detached_parts.md +++ b/docs/zh/operations/system-tables/detached_parts.md @@ -1,14 +1,11 @@ ---- -machine_translated: true -machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 ---- +# system.detached_parts {#system_tables-detached_parts} -# 系统。detached_parts {#system_tables-detached_parts} +包含关于 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 表的分离分区的信息。`reason` 列详细说明了该分区被分离的原因。 -包含有关分离部分的信息 [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) 桌子 该 `reason` 列指定分离部件的原因。 +对于用户分离的分区,原因是空的。你可以通过 [ALTER TABLE ATTACH PARTITION\|PART](../../sql-reference/statements/alter/partition.md#alter_attach-partition) 命令添加这些分区。 -对于用户分离的部件,原因是空的。 这些部件可以附加 [ALTER TABLE ATTACH PARTITION\|PART](../../sql-reference/statements/alter.md#alter_attach-partition) 指挥部 +关于其他列的描述,请参见 [system.parts](../../operations/system-tables/parts.md#system_tables-parts)。 -有关其他列的说明,请参阅 [系统。零件](../../operations/system-tables/parts.md#system_tables-parts). +如果分区名称无效,一些列的值可能是`NULL`。你可以通过[ALTER TABLE DROP DETACHED PART](../../sql-reference/statements/alter/partition.md#alter_drop-detached)来删除这些分区。 -如果部件名称无效,某些列的值可能为 `NULL`. 这些部分可以删除 [ALTER TABLE DROP DETACHED PART](../../sql-reference/statements/alter.md#alter_drop-detached). +[原文](https://clickhouse.com/docs/zh/operations/system-tables/detached_parts) From 975b8211b5bfe0cc8a5f9c8119a81935d88bfa8b Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Sun, 26 Dec 2021 16:25:45 +0800 Subject: [PATCH 0383/1260] Remove duplicates The duplicates was introduced by this [commit](https://github.com/ClickHouse/ClickHouse/commit/00a2dfb1bf85b7ff32602533f54419f7b03d9ed9) --- docs/en/operations/system-tables/columns.md | 19 +------------------ 1 file changed, 1 insertion(+), 18 deletions(-) diff --git a/docs/en/operations/system-tables/columns.md b/docs/en/operations/system-tables/columns.md index 5ba38ab3e67..55e4a8284a0 100644 --- a/docs/en/operations/system-tables/columns.md +++ b/docs/en/operations/system-tables/columns.md @@ -6,7 +6,7 @@ You can use this table to get information similar to the [DESCRIBE TABLE](../../ Columns from [temporary tables](../../sql-reference/statements/create/table.md#temporary-tables) are visible in the `system.columns` only in those session where they have been created. They are shown with the empty `database` field. -Columns: +The `system.columns` table contains the following columns (the column type is shown in brackets): - `database` ([String](../../sql-reference/data-types/string.md)) — Database name. - `table` ([String](../../sql-reference/data-types/string.md)) — Table name. @@ -86,21 +86,4 @@ numeric_scale: ᴺᵁᴸᴸ datetime_precision: ᴺᵁᴸᴸ ``` -The `system.columns` table contains the following columns (the column type is shown in brackets): - -- `database` (String) — Database name. -- `table` (String) — Table name. -- `name` (String) — Column name. -- `type` (String) — Column type. -- `default_kind` (String) — Expression type (`DEFAULT`, `MATERIALIZED`, `ALIAS`) for the default value, or an empty string if it is not defined. -- `default_expression` (String) — Expression for the default value, or an empty string if it is not defined. -- `data_compressed_bytes` (UInt64) — The size of compressed data, in bytes. -- `data_uncompressed_bytes` (UInt64) — The size of decompressed data, in bytes. -- `marks_bytes` (UInt64) — The size of marks, in bytes. -- `comment` (String) — Comment on the column, or an empty string if it is not defined. -- `is_in_partition_key` (UInt8) — Flag that indicates whether the column is in the partition expression. -- `is_in_sorting_key` (UInt8) — Flag that indicates whether the column is in the sorting key expression. -- `is_in_primary_key` (UInt8) — Flag that indicates whether the column is in the primary key expression. -- `is_in_sampling_key` (UInt8) — Flag that indicates whether the column is in the sampling key expression. - [Original article](https://clickhouse.com/docs/en/operations/system-tables/columns) From 6734b592bb59939a7a01136ab24b4033154be2fc Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 26 Dec 2021 09:04:12 +0000 Subject: [PATCH 0384/1260] decimal plus float --- src/Functions/FunctionBinaryArithmetic.h | 74 +++++++++++++++--------- src/Functions/plus.cpp | 7 ++- 2 files changed, 52 insertions(+), 29 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 69d65bfcf66..8d0ad5db208 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -7,37 +7,38 @@ #include #include -#include -#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include -#include -#include +#include #include +#include #include +#include +#include #include #include -#include -#include -#include -#include -#include -#include -#include -#include "Core/DecimalFunctions.h" -#include "IFunction.h" -#include "FunctionHelpers.h" -#include "IsOperation.h" -#include "DivisionUtils.h" -#include "castTypeToEither.h" -#include "FunctionFactory.h" -#include -#include -#include -#include #include #include +#include +#include +#include +#include +#include "Core/DecimalFunctions.h" +#include "DivisionUtils.h" +#include "FunctionFactory.h" +#include "FunctionHelpers.h" +#include "IFunction.h" +#include "IsOperation.h" +#include "castTypeToEither.h" #include @@ -134,11 +135,29 @@ public: Case && IsIntegralOrExtended, LeftDataType>, Case && IsIntegralOrExtended, RightDataType>, + /// e.g Decimal + Float64 = Float64 + Case::plus && IsDataTypeDecimal && IsFloatingPoint, + DataTypeFloat64>, + Case::plus && IsDataTypeDecimal && IsFloatingPoint, + DataTypeFloat64>, + + /// e.g Decimal - Float64 = Float64 + Case::minus && IsDataTypeDecimal && IsFloatingPoint, + DataTypeFloat64>, + Case::multiply && IsDataTypeDecimal && IsFloatingPoint, + DataTypeFloat64>, + /// e.g Decimal * Float64 = Float64 Case::multiply && IsDataTypeDecimal && IsFloatingPoint, - RightDataType>, + DataTypeFloat64>, Case::multiply && IsDataTypeDecimal && IsFloatingPoint, - LeftDataType>, + DataTypeFloat64>, + + /// e.g Decimal / Float64 = Float64 + Case::multiply && IsDataTypeDecimal && IsFloatingPoint, + DataTypeFloat64>, + Case::multiply && IsDataTypeDecimal && IsFloatingPoint, + DataTypeFloat64>, /// Decimal Real is not supported (traditional DBs convert Decimal Real to Real) Case && !IsIntegralOrExtendedOrDecimal, InvalidType>, @@ -966,9 +985,9 @@ class FunctionBinaryArithmetic : public IFunction const ResultDataType type = [&] { if constexpr (left_is_decimal && IsFloatingPoint) - return RightDataType(); + return DataTypeFloat64{}; else if constexpr (right_is_decimal && IsFloatingPoint) - return LeftDataType(); + return DataTypeFloat64{}; else return decimalResultType(left, right); }(); @@ -1219,8 +1238,7 @@ public: } else if constexpr ((IsDataTypeDecimal && IsFloatingPoint) || (IsDataTypeDecimal && IsFloatingPoint)) - type_res = std::make_shared, - LeftDataType, RightDataType>>(); + type_res = std::make_shared(); else if constexpr (IsDataTypeDecimal) type_res = std::make_shared(left.getPrecision(), left.getScale()); else if constexpr (IsDataTypeDecimal) diff --git a/src/Functions/plus.cpp b/src/Functions/plus.cpp index 997cae0dbed..96e05b57f12 100644 --- a/src/Functions/plus.cpp +++ b/src/Functions/plus.cpp @@ -32,7 +32,12 @@ struct PlusImpl template static inline bool apply(A a, B b, Result & c) { - return common::addOverflow(static_cast(a), b, c); + if constexpr (std::is_same_v) + { + c = static_cast(a) + b; + } + else + return common::addOverflow(static_cast(a), b, c); } #if USE_EMBEDDED_COMPILER From 9e7a80f581925e94e3cd1647b18eeb549a663d78 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Dec 2021 12:18:06 +0300 Subject: [PATCH 0385/1260] Add RockPi4 --- website/benchmark/hardware/index.html | 1 + .../benchmark/hardware/results/rock_pi.json | 54 +++++++++++++++++++ 2 files changed, 55 insertions(+) create mode 100644 website/benchmark/hardware/results/rock_pi.json diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index 260a928184d..00a75cf29ce 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -82,6 +82,7 @@ Comparison of EBS and EFS is from Ramazan Polat.
Results for Hetzner and Scaleway are from Anthony Najjar Simon (Panelbear).
Results for GCP are from Vy Nguyen Tan.
Results for ThinkPad P15 are from Mikhail Shiryaev.
+Results for RockPi4 are from Kirill Zholnay.

diff --git a/website/benchmark/hardware/results/rock_pi.json b/website/benchmark/hardware/results/rock_pi.json new file mode 100644 index 00000000000..210dc213a49 --- /dev/null +++ b/website/benchmark/hardware/results/rock_pi.json @@ -0,0 +1,54 @@ +[ + { + "system": "Rock Pi 4, 4GiB, NVMe", + "system_full": "Rock Pi 4, 4GiB C, NVMe", + "time": "2021-12-23 00:00:00", + "kind": "desktop", + "result": + [ +[0.007, 0.014, 0.005], +[0.229, 0.132, 0.215], +[0.489, 0.351, 0.306], +[0.879, 0.774, 0.768], +[1.034, 0.966, 0.879], +[2.491, 2.249, 2.493], +[0.379, 0.212, 0.213], +[0.227, 0.140, 0.152], +[3.944, 3.823, 3.805], +[5.272, 4.985, 5.069], +[2.356, 2.193, 2.254], +[2.819, 2.595, 2.568], +[9.124, 8.306, 8.529], +[11.857, 11.412, 11.290], +[9.796, 9.477, 9.610], +[8.846, 8.867, 8.909], +[null, null, null], +[null, null, null], +[null, null, null], +[1.293, 0.887, 0.980], +[15.018, 14.928, 14.748], +[19.179, 17.889, 18.021], +[45.524, 46.927, 46.909], +[23.904, 23.197, 23.511], +[5.264, 4.891, 4.936], +[4.211, 3.940, 4.047], +[5.113, 4.615, 4.783], +[17.910, 16.800, 16.410], +[23.537, 22.249, 22.172], +[16.549, 16.388, 16.337], +[9.562, 9.006, 9.260], +[17.097, 17.676, 17.585], +[null, null, null], +[null, null, null], +[null, null, null], +[null, null, null], +[1.668, 1.469, 1.342], +[0.463, 0.442, 0.353], +[0.486, 0.410, 0.346], +[2.190, 2.014, 1.878], +[0.263, 0.097, 0.201], +[0.173, 0.082, 0.139], +[0.188, 0.024, 0.016] + ] + } +] From 26aedf247ca06d0559c75cbb27e86ebb6897b847 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Dec 2021 12:23:09 +0300 Subject: [PATCH 0386/1260] Add one more result --- website/benchmark/hardware/index.html | 1 + .../hardware/results/xeon_gold_6266.json | 56 +++++++++++++++++++ 2 files changed, 57 insertions(+) create mode 100644 website/benchmark/hardware/results/xeon_gold_6266.json diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index 00a75cf29ce..d12a9dd9057 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -83,6 +83,7 @@ Results for Hetzner and Scaleway are from Anthony Najjar Simon (Panelbear Results for GCP are from Vy Nguyen Tan.
Results for ThinkPad P15 are from Mikhail Shiryaev.
Results for RockPi4 are from Kirill Zholnay.
+Results for Xeon 6266C are from David in Shanghai.

diff --git a/website/benchmark/hardware/results/xeon_gold_6266.json b/website/benchmark/hardware/results/xeon_gold_6266.json new file mode 100644 index 00000000000..4283b711091 --- /dev/null +++ b/website/benchmark/hardware/results/xeon_gold_6266.json @@ -0,0 +1,56 @@ +[ + { + "system": "Xeon Gold 6266C, 3GHz, 4vCPU", + "system_full": "Xeon Gold 6266C, 3GHz, 4vCPU, 16GiB RAM, vda1 40GB", + "cpu_vendor": "Intel", + "cpu_model": "Xeon Gold 6266C", + "time": "2021-12-23 00:00:00", + "kind": "cloud", + "result": + [ +[0.001, 0.001, 0.001], +[0.034, 0.023, 0.023], +[0.168, 0.105, 0.104], +[0.745, 0.162, 0.160], +[1.512, 0.328, 0.327], +[2.408, 1.162, 1.155], +[0.069, 0.052, 0.051], +[0.074, 0.027, 0.026], +[2.314, 1.833, 1.796], +[2.749, 2.014, 2.011], +[1.424, 0.618, 0.579], +[1.494, 0.681, 0.677], +[3.208, 2.457, 2.529], +[5.071, 3.329, 3.411], +[3.968, 3.289, 3.330], +[3.142, 2.925, 2.827], +[9.473, 9.034, 8.850], +[6.768, 6.256, 6.115], +[18.388, 17.790, 17.892], +[1.105, 0.195, 0.194], +[20.310, 3.459, 3.416], +[22.772, 3.811, 3.773], +[42.554, 8.738, 8.640], +[30.747, 4.013, 3.967], +[4.707, 0.973, 0.965], +[2.003, 0.845, 0.839], +[4.978, 0.991, 0.974], +[19.726, 3.293, 3.264], +[17.151, 5.171, 5.134], +[3.620, 3.600, 3.600], +[4.693, 2.172, 2.115], +[10.842, 2.686, 2.750], +[17.857, 17.086, 16.907], +[22.926, 13.070, 12.808], +[22.803, 12.727, 12.867], +[4.189, 3.888, 3.893], +[0.227, 0.176, 0.177], +[0.085, 0.068, 0.067], +[0.101, 0.064, 0.067], +[0.493, 0.438, 0.399], +[0.042, 0.022, 0.021], +[0.029, 0.017, 0.015], +[0.007, 0.005, 0.003] + ] + } +] From c61609c4b913ad503b0dd33206cc5a6e3a76ce26 Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Sun, 26 Dec 2021 17:23:12 +0800 Subject: [PATCH 0387/1260] Update columns.md --- docs/zh/operations/system-tables/columns.md | 104 +++++++++++++++----- 1 file changed, 82 insertions(+), 22 deletions(-) diff --git a/docs/zh/operations/system-tables/columns.md b/docs/zh/operations/system-tables/columns.md index 9a90561a07b..6d4299a9056 100644 --- a/docs/zh/operations/system-tables/columns.md +++ b/docs/zh/operations/system-tables/columns.md @@ -1,29 +1,89 @@ ---- -machine_translated: true -machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 ---- +# system.columns {#system-columns} -# 系统。列 {#system-columns} +此系统表包含所有表中列的信息。 -包含有关所有表中列的信息。 +你可以使用这个表来获得类似于 [DESCRIBE TABLE](../../sql-reference/statements/misc.md#misc-describe-table) 查询的信息,但是可以同时获得多个表的信息。 -您可以使用此表获取类似于以下内容的信息 [DESCRIBE TABLE](../../sql-reference/statements/misc.md#misc-describe-table) 查询,但对于多个表一次。 +[临时表](../../sql-reference/statements/create/table.md#temporary-tables)中的列只在创建它们的会话中的 `system.columns` 中才可见,并且它们的 `database` 字段显示为空。 -该 `system.columns` 表包含以下列(列类型显示在括号中): +`system.columns` 表包含以下列 (括号中显示的是列类型): -- `database` (String) — Database name. -- `table` (String) — Table name. -- `name` (String) — Column name. -- `type` (String) — Column type. -- `default_kind` (String) — Expression type (`DEFAULT`, `MATERIALIZED`, `ALIAS`)为默认值,如果没有定义,则为空字符串。 -- `default_expression` (String) — Expression for the default value, or an empty string if it is not defined. -- `data_compressed_bytes` (UInt64) — The size of compressed data, in bytes. -- `data_uncompressed_bytes` (UInt64) — The size of decompressed data, in bytes. -- `marks_bytes` (UInt64) — The size of marks, in bytes. -- `comment` (String) — Comment on the column, or an empty string if it is not defined. -- `is_in_partition_key` (UInt8) — Flag that indicates whether the column is in the partition expression. -- `is_in_sorting_key` (UInt8) — Flag that indicates whether the column is in the sorting key expression. -- `is_in_primary_key` (UInt8) — Flag that indicates whether the column is in the primary key expression. -- `is_in_sampling_key` (UInt8) — Flag that indicates whether the column is in the sampling key expression. +- `database` ([String](../../sql-reference/data-types/string.md)) — 数据库名称。 +- `table` ([String](../../sql-reference/data-types/string.md)) — 表名。 +- `name` ([String](../../sql-reference/data-types/string.md)) — 列名。 +- `type` ([String](../../sql-reference/data-types/string.md)) — 列类型。 +- `position` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 列在表中的顺序位置,从1开始。 +- `default_kind` ([String](../../sql-reference/data-types/string.md)) — 默认值的表达式类型(`DEFAULT`, `MATERIALIZED`, `ALIAS`) ,如果没有定义,则为空字符串。 +- `default_expression` ([String](../../sql-reference/data-types/string.md)) — 默认值的表达式,如果未定义则为空字符串。 +- `data_compressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 压缩数据的大小,以字节为单位。 +- `data_uncompressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 解压后的数据的大小,以字节为单位。 +- `marks_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 标记的大小,以字节为单位。 +- `comment` ([String](../../sql-reference/data-types/string.md)) — 列注释,如果没有定义,则为空字符串。 +- `is_in_partition_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — 列是否在分区表达式中的标志。 +- `is_in_sorting_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — 列是否在排序键表达式中的标志。 +- `is_in_primary_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — 列是否在主键表达式中的标志。 +- `is_in_sampling_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — 列是否在采样键表达式中的标志。 +- `compression_codec` ([String](../../sql-reference/data-types/string.md)) — 压缩编码的名称。 +- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 二进制数据、字符数据或文本数据和图像的最大长度(以字节为单位)。在 ClickHouse 中只对 `FixedString` 数据类型有意义。否则,将返回 `NULL` 值。 +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 近似数字型数据、精确数字型数据、整数型数据或货币数据的精度。在 ClickHouse 中,对于整数类型是比特率(bitness),对于 `Decimal` 类型是十进制精度。否则,将返回 `NULL` 值。 +- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 数字系统的基数是近似数字型数据、精确数字型数据、整数型数据或货币数据的精度。在 ClickHouse 中,对于整数类型是2,对于 `Decimal` 类型是10。否则,将返回 `NULL` 值。 +- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 近似数字型数据、精确数字型数据、整数型数据或货币数据的比例。在 ClickHouse 中只对 `Decimal` 类型有意义。否则,将返回 `NULL` 值。 +- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — `DateTime64` 数据类型的小数精度。对于其他数据类型,将返回 `NULL` 值。 + +**示例** + +```sql +SELECT * FROM system.columns LIMIT 2 FORMAT Vertical; +``` + +```text +Row 1: +────── +database: INFORMATION_SCHEMA +table: COLUMNS +name: table_catalog +type: String +position: 1 +default_kind: +default_expression: +data_compressed_bytes: 0 +data_uncompressed_bytes: 0 +marks_bytes: 0 +comment: +is_in_partition_key: 0 +is_in_sorting_key: 0 +is_in_primary_key: 0 +is_in_sampling_key: 0 +compression_codec: +character_octet_length: ᴺᵁᴸᴸ +numeric_precision: ᴺᵁᴸᴸ +numeric_precision_radix: ᴺᵁᴸᴸ +numeric_scale: ᴺᵁᴸᴸ +datetime_precision: ᴺᵁᴸᴸ + +Row 2: +────── +database: INFORMATION_SCHEMA +table: COLUMNS +name: table_schema +type: String +position: 2 +default_kind: +default_expression: +data_compressed_bytes: 0 +data_uncompressed_bytes: 0 +marks_bytes: 0 +comment: +is_in_partition_key: 0 +is_in_sorting_key: 0 +is_in_primary_key: 0 +is_in_sampling_key: 0 +compression_codec: +character_octet_length: ᴺᵁᴸᴸ +numeric_precision: ᴺᵁᴸᴸ +numeric_precision_radix: ᴺᵁᴸᴸ +numeric_scale: ᴺᵁᴸᴸ +datetime_precision: ᴺᵁᴸᴸ +``` [原文](https://clickhouse.com/docs/zh/operations/system-tables/columns) From a32513fd46affc1156cc478c04aa33dc8b976144 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 26 Dec 2021 09:23:42 +0000 Subject: [PATCH 0388/1260] fix --- src/Functions/FunctionsComparison.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 4c21346e338..4f98754e2a1 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -15,7 +15,6 @@ #include #include #include -#include #include #include #include @@ -1245,7 +1244,7 @@ public: /// Other systems like MySQL and Spark also do as this. if (left_is_float || right_is_float) { - const auto converted_type = DataTypeFactory::instance().get("Float64"); + const auto converted_type = std::make_shared(); ColumnPtr c0_converted = castColumn(col_with_type_and_name_left, converted_type); ColumnPtr c1_converted = castColumn(col_with_type_and_name_right, converted_type); return executeGenericIdenticalTypes(c0_converted.get(), c1_converted.get()); From 717b9a228671526abea14dadffc9a4194a3dc03f Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Sun, 26 Dec 2021 17:33:58 +0800 Subject: [PATCH 0389/1260] Update contributors.md --- docs/zh/operations/system-tables/contributors.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/operations/system-tables/contributors.md b/docs/zh/operations/system-tables/contributors.md index 6c99d2dac22..fd876da3594 100644 --- a/docs/zh/operations/system-tables/contributors.md +++ b/docs/zh/operations/system-tables/contributors.md @@ -1,6 +1,6 @@ # system.contributors {#system-contributors} -此系统表包含有关贡献者的信息。在查询执行时,排列顺序是随机的。 +此系统表包含有关贡献者的信息。排列顺序是在查询执行时随机生成的。 列: From 05a33abf265d979bcf4db4a9ce5459d36ebf6046 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Sun, 26 Dec 2021 13:38:22 +0300 Subject: [PATCH 0390/1260] OR operator in ON section for JOIN --- .../sql-reference/statements/select/join.md | 631 ++++++++++-------- 1 file changed, 336 insertions(+), 295 deletions(-) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index aa61348d2a0..f09f6fe62a5 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -1,295 +1,336 @@ ---- -toc_title: JOIN ---- - -# JOIN Clause {#select-join} - -Join produces a new table by combining columns from one or multiple tables by using values common to each. It is a common operation in databases with SQL support, which corresponds to [relational algebra](https://en.wikipedia.org/wiki/Relational_algebra#Joins_and_join-like_operators) join. The special case of one table join is often referred to as “self-join”. - -**Syntax** - -``` sql -SELECT -FROM -[GLOBAL] [INNER|LEFT|RIGHT|FULL|CROSS] [OUTER|SEMI|ANTI|ANY|ASOF] JOIN -(ON )|(USING ) ... -``` - -Expressions from `ON` clause and columns from `USING` clause are called “join keys”. Unless otherwise stated, join produces a [Cartesian product](https://en.wikipedia.org/wiki/Cartesian_product) from rows with matching “join keys”, which might produce results with much more rows than the source tables. - -## Supported Types of JOIN {#select-join-types} - -All standard [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) types are supported: - -- `INNER JOIN`, only matching rows are returned. -- `LEFT OUTER JOIN`, non-matching rows from left table are returned in addition to matching rows. -- `RIGHT OUTER JOIN`, non-matching rows from right table are returned in addition to matching rows. -- `FULL OUTER JOIN`, non-matching rows from both tables are returned in addition to matching rows. -- `CROSS JOIN`, produces cartesian product of whole tables, “join keys” are **not** specified. - -`JOIN` without specified type implies `INNER`. Keyword `OUTER` can be safely omitted. Alternative syntax for `CROSS JOIN` is specifying multiple tables in [FROM clause](../../../sql-reference/statements/select/from.md) separated by commas. - -Additional join types available in ClickHouse: - -- `LEFT SEMI JOIN` and `RIGHT SEMI JOIN`, a whitelist on “join keys”, without producing a cartesian product. -- `LEFT ANTI JOIN` and `RIGHT ANTI JOIN`, a blacklist on “join keys”, without producing a cartesian product. -- `LEFT ANY JOIN`, `RIGHT ANY JOIN` and `INNER ANY JOIN`, partially (for opposite side of `LEFT` and `RIGHT`) or completely (for `INNER` and `FULL`) disables the cartesian product for standard `JOIN` types. -- `ASOF JOIN` and `LEFT ASOF JOIN`, joining sequences with a non-exact match. `ASOF JOIN` usage is described below. - -!!! note "Note" - When [join_algorithm](../../../operations/settings/settings.md#settings-join_algorithm) is set to `partial_merge`, `RIGHT JOIN` and `FULL JOIN` are supported only with `ALL` strictness (`SEMI`, `ANTI`, `ANY`, and `ASOF` are not supported). - -## Settings {#join-settings} - -The default join type can be overridden using [join_default_strictness](../../../operations/settings/settings.md#settings-join_default_strictness) setting. - -The behavior of ClickHouse server for `ANY JOIN` operations depends on the [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys) setting. - -**See also** - -- [join_algorithm](../../../operations/settings/settings.md#settings-join_algorithm) -- [join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) -- [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) -- [partial_merge_join_optimizations](../../../operations/settings/settings.md#partial_merge_join_optimizations) -- [partial_merge_join_rows_in_right_blocks](../../../operations/settings/settings.md#partial_merge_join_rows_in_right_blocks) -- [join_on_disk_max_files_to_merge](../../../operations/settings/settings.md#join_on_disk_max_files_to_merge) -- [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys) - -## ON Section Conditions {on-section-conditions} - -An `ON` section can contain several conditions combined using the `AND` operator. Conditions specifying join keys must refer both left and right tables and must use the equality operator. Other conditions may use other logical operators but they must refer either the left or the right table of a query. -Rows are joined if the whole complex condition is met. If the conditions are not met, still rows may be included in the result depending on the `JOIN` type. Note that if the same conditions are placed in a `WHERE` section and they are not met, then rows are always filtered out from the result. - -!!! note "Note" - The `OR` operator inside an `ON` section is not supported yet. - -!!! note "Note" - If a condition refers columns from different tables, then only the equality operator (`=`) is supported so far. - -**Example** - -Consider `table_1` and `table_2`: - -``` -┌─Id─┬─name─┐ ┌─Id─┬─text───────────┬─scores─┐ -│ 1 │ A │ │ 1 │ Text A │ 10 │ -│ 2 │ B │ │ 1 │ Another text A │ 12 │ -│ 3 │ C │ │ 2 │ Text B │ 15 │ -└────┴──────┘ └────┴────────────────┴────────┘ -``` - -Query with one join key condition and an additional condition for `table_2`: - -``` sql -SELECT name, text FROM table_1 LEFT OUTER JOIN table_2 - ON table_1.Id = table_2.Id AND startsWith(table_2.text, 'Text'); -``` - -Note that the result contains the row with the name `C` and the empty text column. It is included into the result because an `OUTER` type of a join is used. - -``` -┌─name─┬─text───┐ -│ A │ Text A │ -│ B │ Text B │ -│ C │ │ -└──────┴────────┘ -``` - -Query with `INNER` type of a join and multiple conditions: - -``` sql -SELECT name, text, scores FROM table_1 INNER JOIN table_2 - ON table_1.Id = table_2.Id AND table_2.scores > 10 AND startsWith(table_2.text, 'Text'); -``` - -Result: - -``` -┌─name─┬─text───┬─scores─┐ -│ B │ Text B │ 15 │ -└──────┴────────┴────────┘ -``` - -## ASOF JOIN Usage {#asof-join-usage} - -`ASOF JOIN` is useful when you need to join records that have no exact match. - -Algorithm requires the special column in tables. This column: - -- Must contain an ordered sequence. -- Can be one of the following types: [Int, UInt](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md), [Date](../../../sql-reference/data-types/date.md), [DateTime](../../../sql-reference/data-types/datetime.md), [Decimal](../../../sql-reference/data-types/decimal.md). -- Can’t be the only column in the `JOIN` clause. - -Syntax `ASOF JOIN ... ON`: - -``` sql -SELECT expressions_list -FROM table_1 -ASOF LEFT JOIN table_2 -ON equi_cond AND closest_match_cond -``` - -You can use any number of equality conditions and exactly one closest match condition. For example, `SELECT count() FROM table_1 ASOF LEFT JOIN table_2 ON table_1.a == table_2.b AND table_2.t <= table_1.t`. - -Conditions supported for the closest match: `>`, `>=`, `<`, `<=`. - -Syntax `ASOF JOIN ... USING`: - -``` sql -SELECT expressions_list -FROM table_1 -ASOF JOIN table_2 -USING (equi_column1, ... equi_columnN, asof_column) -``` - -`ASOF JOIN` uses `equi_columnX` for joining on equality and `asof_column` for joining on the closest match with the `table_1.asof_column >= table_2.asof_column` condition. The `asof_column` column is always the last one in the `USING` clause. - -For example, consider the following tables: - - table_1 table_2 - event | ev_time | user_id event | ev_time | user_id - ----------|---------|---------- ----------|---------|---------- - ... ... - event_1_1 | 12:00 | 42 event_2_1 | 11:59 | 42 - ... event_2_2 | 12:30 | 42 - event_1_2 | 13:00 | 42 event_2_3 | 13:00 | 42 - ... ... - -`ASOF JOIN` can take the timestamp of a user event from `table_1` and find an event in `table_2` where the timestamp is closest to the timestamp of the event from `table_1` corresponding to the closest match condition. Equal timestamp values are the closest if available. Here, the `user_id` column can be used for joining on equality and the `ev_time` column can be used for joining on the closest match. In our example, `event_1_1` can be joined with `event_2_1` and `event_1_2` can be joined with `event_2_3`, but `event_2_2` can’t be joined. - -!!! note "Note" - `ASOF` join is **not** supported in the [Join](../../../engines/table-engines/special/join.md) table engine. - -## Distributed JOIN {#global-join} - -There are two ways to execute join involving distributed tables: - -- When using a normal `JOIN`, the query is sent to remote servers. Subqueries are run on each of them in order to make the right table, and the join is performed with this table. In other words, the right table is formed on each server separately. -- When using `GLOBAL ... JOIN`, first the requestor server runs a subquery to calculate the right table. This temporary table is passed to each remote server, and queries are run on them using the temporary data that was transmitted. - -Be careful when using `GLOBAL`. For more information, see the [Distributed subqueries](../../../sql-reference/operators/in.md#select-distributed-subqueries) section. - -## Implicit Type Conversion {#implicit-type-conversion} - -`INNER JOIN`, `LEFT JOIN`, `RIGHT JOIN`, and `FULL JOIN` queries support the implicit type conversion for "join keys". However the query can not be executed, if join keys from the left and the right tables cannot be converted to a single type (for example, there is no data type that can hold all values from both `UInt64` and `Int64`, or `String` and `Int32`). - -**Example** - -Consider the table `t_1`: -```text -┌─a─┬─b─┬─toTypeName(a)─┬─toTypeName(b)─┐ -│ 1 │ 1 │ UInt16 │ UInt8 │ -│ 2 │ 2 │ UInt16 │ UInt8 │ -└───┴───┴───────────────┴───────────────┘ -``` -and the table `t_2`: -```text -┌──a─┬────b─┬─toTypeName(a)─┬─toTypeName(b)───┐ -│ -1 │ 1 │ Int16 │ Nullable(Int64) │ -│ 1 │ -1 │ Int16 │ Nullable(Int64) │ -│ 1 │ 1 │ Int16 │ Nullable(Int64) │ -└────┴──────┴───────────────┴─────────────────┘ -``` - -The query -```sql -SELECT a, b, toTypeName(a), toTypeName(b) FROM t_1 FULL JOIN t_2 USING (a, b); -``` -returns the set: -```text -┌──a─┬────b─┬─toTypeName(a)─┬─toTypeName(b)───┐ -│ 1 │ 1 │ Int32 │ Nullable(Int64) │ -│ 2 │ 2 │ Int32 │ Nullable(Int64) │ -│ -1 │ 1 │ Int32 │ Nullable(Int64) │ -│ 1 │ -1 │ Int32 │ Nullable(Int64) │ -└────┴──────┴───────────────┴─────────────────┘ -``` - -## Usage Recommendations {#usage-recommendations} - -### Processing of Empty or NULL Cells {#processing-of-empty-or-null-cells} - -While joining tables, the empty cells may appear. The setting [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) define how ClickHouse fills these cells. - -If the `JOIN` keys are [Nullable](../../../sql-reference/data-types/nullable.md) fields, the rows where at least one of the keys has the value [NULL](../../../sql-reference/syntax.md#null-literal) are not joined. - -### Syntax {#syntax} - -The columns specified in `USING` must have the same names in both subqueries, and the other columns must be named differently. You can use aliases to change the names of columns in subqueries. - -The `USING` clause specifies one or more columns to join, which establishes the equality of these columns. The list of columns is set without brackets. More complex join conditions are not supported. - -### Syntax Limitations {#syntax-limitations} - -For multiple `JOIN` clauses in a single `SELECT` query: - -- Taking all the columns via `*` is available only if tables are joined, not subqueries. -- The `PREWHERE` clause is not available. - -For `ON`, `WHERE`, and `GROUP BY` clauses: - -- Arbitrary expressions cannot be used in `ON`, `WHERE`, and `GROUP BY` clauses, but you can define an expression in a `SELECT` clause and then use it in these clauses via an alias. - -### Performance {#performance} - -When running a `JOIN`, there is no optimization of the order of execution in relation to other stages of the query. The join (a search in the right table) is run before filtering in `WHERE` and before aggregation. - -Each time a query is run with the same `JOIN`, the subquery is run again because the result is not cached. To avoid this, use the special [Join](../../../engines/table-engines/special/join.md) table engine, which is a prepared array for joining that is always in RAM. - -In some cases, it is more efficient to use [IN](../../../sql-reference/operators/in.md) instead of `JOIN`. - -If you need a `JOIN` for joining with dimension tables (these are relatively small tables that contain dimension properties, such as names for advertising campaigns), a `JOIN` might not be very convenient due to the fact that the right table is re-accessed for every query. For such cases, there is an “external dictionaries” feature that you should use instead of `JOIN`. For more information, see the [External dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) section. - -### Memory Limitations {#memory-limitations} - -By default, ClickHouse uses the [hash join](https://en.wikipedia.org/wiki/Hash_join) algorithm. ClickHouse takes the right_table and creates a hash table for it in RAM. If `join_algorithm = 'auto'` is enabled, then after some threshold of memory consumption, ClickHouse falls back to [merge](https://en.wikipedia.org/wiki/Sort-merge_join) join algorithm. For `JOIN` algorithms description see the [join_algorithm](../../../operations/settings/settings.md#settings-join_algorithm) setting. - -If you need to restrict `JOIN` operation memory consumption use the following settings: - -- [max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) — Limits number of rows in the hash table. -- [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) — Limits size of the hash table. - -When any of these limits is reached, ClickHouse acts as the [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) setting instructs. - -## Examples {#examples} - -Example: - -``` sql -SELECT - CounterID, - hits, - visits -FROM -( - SELECT - CounterID, - count() AS hits - FROM test.hits - GROUP BY CounterID -) ANY LEFT JOIN -( - SELECT - CounterID, - sum(Sign) AS visits - FROM test.visits - GROUP BY CounterID -) USING CounterID -ORDER BY hits DESC -LIMIT 10 -``` - -``` text -┌─CounterID─┬───hits─┬─visits─┐ -│ 1143050 │ 523264 │ 13665 │ -│ 731962 │ 475698 │ 102716 │ -│ 722545 │ 337212 │ 108187 │ -│ 722889 │ 252197 │ 10547 │ -│ 2237260 │ 196036 │ 9522 │ -│ 23057320 │ 147211 │ 7689 │ -│ 722818 │ 90109 │ 17847 │ -│ 48221 │ 85379 │ 4652 │ -│ 19762435 │ 77807 │ 7026 │ -│ 722884 │ 77492 │ 11056 │ -└───────────┴────────┴────────┘ -``` +--- +toc_title: JOIN +--- + +# JOIN Clause {#select-join} + +Join produces a new table by combining columns from one or multiple tables by using values common to each. It is a common operation in databases with SQL support, which corresponds to [relational algebra](https://en.wikipedia.org/wiki/Relational_algebra#Joins_and_join-like_operators) join. The special case of one table join is often referred to as “self-join”. + +**Syntax** + +``` sql +SELECT +FROM +[GLOBAL] [INNER|LEFT|RIGHT|FULL|CROSS] [OUTER|SEMI|ANTI|ANY|ASOF] JOIN +(ON )|(USING ) ... +``` + +Expressions from `ON` clause and columns from `USING` clause are called “join keys”. Unless otherwise stated, join produces a [Cartesian product](https://en.wikipedia.org/wiki/Cartesian_product) from rows with matching “join keys”, which might produce results with much more rows than the source tables. + +## Supported Types of JOIN {#select-join-types} + +All standard [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) types are supported: + +- `INNER JOIN`, only matching rows are returned. +- `LEFT OUTER JOIN`, non-matching rows from left table are returned in addition to matching rows. +- `RIGHT OUTER JOIN`, non-matching rows from right table are returned in addition to matching rows. +- `FULL OUTER JOIN`, non-matching rows from both tables are returned in addition to matching rows. +- `CROSS JOIN`, produces cartesian product of whole tables, “join keys” are **not** specified. + +`JOIN` without specified type implies `INNER`. Keyword `OUTER` can be safely omitted. Alternative syntax for `CROSS JOIN` is specifying multiple tables in [FROM clause](../../../sql-reference/statements/select/from.md) separated by commas. + +Additional join types available in ClickHouse: + +- `LEFT SEMI JOIN` and `RIGHT SEMI JOIN`, a whitelist on “join keys”, without producing a cartesian product. +- `LEFT ANTI JOIN` and `RIGHT ANTI JOIN`, a blacklist on “join keys”, without producing a cartesian product. +- `LEFT ANY JOIN`, `RIGHT ANY JOIN` and `INNER ANY JOIN`, partially (for opposite side of `LEFT` and `RIGHT`) or completely (for `INNER` and `FULL`) disables the cartesian product for standard `JOIN` types. +- `ASOF JOIN` and `LEFT ASOF JOIN`, joining sequences with a non-exact match. `ASOF JOIN` usage is described below. + +!!! note "Note" + When [join_algorithm](../../../operations/settings/settings.md#settings-join_algorithm) is set to `partial_merge`, `RIGHT JOIN` and `FULL JOIN` are supported only with `ALL` strictness (`SEMI`, `ANTI`, `ANY`, and `ASOF` are not supported). + +## Settings {#join-settings} + +The default join type can be overridden using [join_default_strictness](../../../operations/settings/settings.md#settings-join_default_strictness) setting. + +The behavior of ClickHouse server for `ANY JOIN` operations depends on the [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys) setting. + +**See also** + +- [join_algorithm](../../../operations/settings/settings.md#settings-join_algorithm) +- [join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) +- [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) +- [partial_merge_join_optimizations](../../../operations/settings/settings.md#partial_merge_join_optimizations) +- [partial_merge_join_rows_in_right_blocks](../../../operations/settings/settings.md#partial_merge_join_rows_in_right_blocks) +- [join_on_disk_max_files_to_merge](../../../operations/settings/settings.md#join_on_disk_max_files_to_merge) +- [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys) + +## ON Section Conditions {on-section-conditions} + +An `ON` section can contain several conditions combined using the `AND` and `OR` operators. Conditions specifying join keys must refer both left and right tables and must use the equality operator. Other conditions may use other logical operators but they must refer either the left or the right table of a query. +Rows are joined if the whole complex condition is met. If the conditions are not met, still rows may be included in the result depending on the `JOIN` type. Note that if the same conditions are placed in a `WHERE` section and they are not met, then rows are always filtered out from the result. + +The `OR` operator inside the` ON` clause works using the hash join algorithm — for each `OR` argument with join keys for` JOIN`, a separate hash table is created, so memory consumption and query execution time grow linearly with an increase in the number of expressions `OR` of the `ON` clause. + +!!! note "Note" + If a condition refers columns from different tables, then only the equality operator (`=`) is supported so far. + +**Example** + +Consider `table_1` and `table_2`: + +``` +┌─Id─┬─name─┐ ┌─Id─┬─text───────────┬─scores─┐ +│ 1 │ A │ │ 1 │ Text A │ 10 │ +│ 2 │ B │ │ 1 │ Another text A │ 12 │ +│ 3 │ C │ │ 2 │ Text B │ 15 │ +└────┴──────┘ └────┴────────────────┴────────┘ +``` + +Query with one join key condition and an additional condition for `table_2`: + +``` sql +SELECT name, text FROM table_1 LEFT OUTER JOIN table_2 + ON table_1.Id = table_2.Id AND startsWith(table_2.text, 'Text'); +``` + +Note that the result contains the row with the name `C` and the empty text column. It is included into the result because an `OUTER` type of a join is used. + +``` +┌─name─┬─text───┐ +│ A │ Text A │ +│ B │ Text B │ +│ C │ │ +└──────┴────────┘ +``` + +Query with `INNER` type of a join and multiple conditions: + +``` sql +SELECT name, text, scores FROM table_1 INNER JOIN table_2 + ON table_1.Id = table_2.Id AND table_2.scores > 10 AND startsWith(table_2.text, 'Text'); +``` + +Result: + +``` +┌─name─┬─text───┬─scores─┐ +│ B │ Text B │ 15 │ +└──────┴────────┴────────┘ +``` + +Query with `INNER` and condition` OR`: + +``` sql +CREATE TABLE t1 (`a` Int64, `b` Int64) ENGINE = MergeTree() ORDER BY a; + +CREATE TABLE t2 (`key` Int32, `val` Int64) ENGINE = MergeTree() ORDER BY key; + +INSERT INTO t1 SELECT number as a, -a as b from numbers(5); + +INSERT INTO t2 SELECT if(number % 2 == 0, toInt64(number), -number) as key, number as val from numbers(5); + +SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key; +``` + +Result: + +``` +┌─a─┬──b─┬─val─┐ +│ 0 │ 0 │ 0 │ +│ 1 │ -1 │ 1 │ +│ 2 │ -2 │ 2 │ +│ 3 │ -3 │ 3 │ +│ 4 │ -4 │ 4 │ +└───┴────┴─────┘ +``` + +Query with `INNER` and condition `OR` and `AND`: + +``` +SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key AND t2.val > 3; +``` + +Result: + +``` +┌─a─┬──b─┬─val─┐ +│ 0 │ 0 │ 0 │ +│ 2 │ -2 │ 2 │ +│ 4 │ -4 │ 4 │ +└───┴────┴─────┘ +``` + +## ASOF JOIN Usage {#asof-join-usage} + +`ASOF JOIN` is useful when you need to join records that have no exact match. + +Algorithm requires the special column in tables. This column: + +- Must contain an ordered sequence. +- Can be one of the following types: [Int, UInt](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md), [Date](../../../sql-reference/data-types/date.md), [DateTime](../../../sql-reference/data-types/datetime.md), [Decimal](../../../sql-reference/data-types/decimal.md). +- Can’t be the only column in the `JOIN` clause. + +Syntax `ASOF JOIN ... ON`: + +``` sql +SELECT expressions_list +FROM table_1 +ASOF LEFT JOIN table_2 +ON equi_cond AND closest_match_cond +``` + +You can use any number of equality conditions and exactly one closest match condition. For example, `SELECT count() FROM table_1 ASOF LEFT JOIN table_2 ON table_1.a == table_2.b AND table_2.t <= table_1.t`. + +Conditions supported for the closest match: `>`, `>=`, `<`, `<=`. + +Syntax `ASOF JOIN ... USING`: + +``` sql +SELECT expressions_list +FROM table_1 +ASOF JOIN table_2 +USING (equi_column1, ... equi_columnN, asof_column) +``` + +`ASOF JOIN` uses `equi_columnX` for joining on equality and `asof_column` for joining on the closest match with the `table_1.asof_column >= table_2.asof_column` condition. The `asof_column` column is always the last one in the `USING` clause. + +For example, consider the following tables: + + table_1 table_2 + event | ev_time | user_id event | ev_time | user_id + ----------|---------|---------- ----------|---------|---------- + ... ... + event_1_1 | 12:00 | 42 event_2_1 | 11:59 | 42 + ... event_2_2 | 12:30 | 42 + event_1_2 | 13:00 | 42 event_2_3 | 13:00 | 42 + ... ... + +`ASOF JOIN` can take the timestamp of a user event from `table_1` and find an event in `table_2` where the timestamp is closest to the timestamp of the event from `table_1` corresponding to the closest match condition. Equal timestamp values are the closest if available. Here, the `user_id` column can be used for joining on equality and the `ev_time` column can be used for joining on the closest match. In our example, `event_1_1` can be joined with `event_2_1` and `event_1_2` can be joined with `event_2_3`, but `event_2_2` can’t be joined. + +!!! note "Note" + `ASOF` join is **not** supported in the [Join](../../../engines/table-engines/special/join.md) table engine. + +## Distributed JOIN {#global-join} + +There are two ways to execute join involving distributed tables: + +- When using a normal `JOIN`, the query is sent to remote servers. Subqueries are run on each of them in order to make the right table, and the join is performed with this table. In other words, the right table is formed on each server separately. +- When using `GLOBAL ... JOIN`, first the requestor server runs a subquery to calculate the right table. This temporary table is passed to each remote server, and queries are run on them using the temporary data that was transmitted. + +Be careful when using `GLOBAL`. For more information, see the [Distributed subqueries](../../../sql-reference/operators/in.md#select-distributed-subqueries) section. + +## Implicit Type Conversion {#implicit-type-conversion} + +`INNER JOIN`, `LEFT JOIN`, `RIGHT JOIN`, and `FULL JOIN` queries support the implicit type conversion for "join keys". However the query can not be executed, if join keys from the left and the right tables cannot be converted to a single type (for example, there is no data type that can hold all values from both `UInt64` and `Int64`, or `String` and `Int32`). + +**Example** + +Consider the table `t_1`: +```text +┌─a─┬─b─┬─toTypeName(a)─┬─toTypeName(b)─┐ +│ 1 │ 1 │ UInt16 │ UInt8 │ +│ 2 │ 2 │ UInt16 │ UInt8 │ +└───┴───┴───────────────┴───────────────┘ +``` +and the table `t_2`: +```text +┌──a─┬────b─┬─toTypeName(a)─┬─toTypeName(b)───┐ +│ -1 │ 1 │ Int16 │ Nullable(Int64) │ +│ 1 │ -1 │ Int16 │ Nullable(Int64) │ +│ 1 │ 1 │ Int16 │ Nullable(Int64) │ +└────┴──────┴───────────────┴─────────────────┘ +``` + +The query +```sql +SELECT a, b, toTypeName(a), toTypeName(b) FROM t_1 FULL JOIN t_2 USING (a, b); +``` +returns the set: +```text +┌──a─┬────b─┬─toTypeName(a)─┬─toTypeName(b)───┐ +│ 1 │ 1 │ Int32 │ Nullable(Int64) │ +│ 2 │ 2 │ Int32 │ Nullable(Int64) │ +│ -1 │ 1 │ Int32 │ Nullable(Int64) │ +│ 1 │ -1 │ Int32 │ Nullable(Int64) │ +└────┴──────┴───────────────┴─────────────────┘ +``` + +## Usage Recommendations {#usage-recommendations} + +### Processing of Empty or NULL Cells {#processing-of-empty-or-null-cells} + +While joining tables, the empty cells may appear. The setting [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) define how ClickHouse fills these cells. + +If the `JOIN` keys are [Nullable](../../../sql-reference/data-types/nullable.md) fields, the rows where at least one of the keys has the value [NULL](../../../sql-reference/syntax.md#null-literal) are not joined. + +### Syntax {#syntax} + +The columns specified in `USING` must have the same names in both subqueries, and the other columns must be named differently. You can use aliases to change the names of columns in subqueries. + +The `USING` clause specifies one or more columns to join, which establishes the equality of these columns. The list of columns is set without brackets. More complex join conditions are not supported. + +### Syntax Limitations {#syntax-limitations} + +For multiple `JOIN` clauses in a single `SELECT` query: + +- Taking all the columns via `*` is available only if tables are joined, not subqueries. +- The `PREWHERE` clause is not available. + +For `ON`, `WHERE`, and `GROUP BY` clauses: + +- Arbitrary expressions cannot be used in `ON`, `WHERE`, and `GROUP BY` clauses, but you can define an expression in a `SELECT` clause and then use it in these clauses via an alias. + +### Performance {#performance} + +When running a `JOIN`, there is no optimization of the order of execution in relation to other stages of the query. The join (a search in the right table) is run before filtering in `WHERE` and before aggregation. + +Each time a query is run with the same `JOIN`, the subquery is run again because the result is not cached. To avoid this, use the special [Join](../../../engines/table-engines/special/join.md) table engine, which is a prepared array for joining that is always in RAM. + +In some cases, it is more efficient to use [IN](../../../sql-reference/operators/in.md) instead of `JOIN`. + +If you need a `JOIN` for joining with dimension tables (these are relatively small tables that contain dimension properties, such as names for advertising campaigns), a `JOIN` might not be very convenient due to the fact that the right table is re-accessed for every query. For such cases, there is an “external dictionaries” feature that you should use instead of `JOIN`. For more information, see the [External dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) section. + +### Memory Limitations {#memory-limitations} + +By default, ClickHouse uses the [hash join](https://en.wikipedia.org/wiki/Hash_join) algorithm. ClickHouse takes the right_table and creates a hash table for it in RAM. If `join_algorithm = 'auto'` is enabled, then after some threshold of memory consumption, ClickHouse falls back to [merge](https://en.wikipedia.org/wiki/Sort-merge_join) join algorithm. For `JOIN` algorithms description see the [join_algorithm](../../../operations/settings/settings.md#settings-join_algorithm) setting. + +If you need to restrict `JOIN` operation memory consumption use the following settings: + +- [max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) — Limits number of rows in the hash table. +- [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) — Limits size of the hash table. + +When any of these limits is reached, ClickHouse acts as the [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) setting instructs. + +## Examples {#examples} + +Example: + +``` sql +SELECT + CounterID, + hits, + visits +FROM +( + SELECT + CounterID, + count() AS hits + FROM test.hits + GROUP BY CounterID +) ANY LEFT JOIN +( + SELECT + CounterID, + sum(Sign) AS visits + FROM test.visits + GROUP BY CounterID +) USING CounterID +ORDER BY hits DESC +LIMIT 10 +``` + +``` text +┌─CounterID─┬───hits─┬─visits─┐ +│ 1143050 │ 523264 │ 13665 │ +│ 731962 │ 475698 │ 102716 │ +│ 722545 │ 337212 │ 108187 │ +│ 722889 │ 252197 │ 10547 │ +│ 2237260 │ 196036 │ 9522 │ +│ 23057320 │ 147211 │ 7689 │ +│ 722818 │ 90109 │ 17847 │ +│ 48221 │ 85379 │ 4652 │ +│ 19762435 │ 77807 │ 7026 │ +│ 722884 │ 77492 │ 11056 │ +└───────────┴────────┴────────┘ +``` From e3e1fe74c11c9abff90baea0cf90c2b3b6369111 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Sun, 26 Dec 2021 13:55:49 +0300 Subject: [PATCH 0391/1260] update --- docs/en/sql-reference/statements/select/join.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index f09f6fe62a5..fd94020f9e9 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -60,7 +60,7 @@ The behavior of ClickHouse server for `ANY JOIN` operations depends on the [any_ An `ON` section can contain several conditions combined using the `AND` and `OR` operators. Conditions specifying join keys must refer both left and right tables and must use the equality operator. Other conditions may use other logical operators but they must refer either the left or the right table of a query. Rows are joined if the whole complex condition is met. If the conditions are not met, still rows may be included in the result depending on the `JOIN` type. Note that if the same conditions are placed in a `WHERE` section and they are not met, then rows are always filtered out from the result. -The `OR` operator inside the` ON` clause works using the hash join algorithm — for each `OR` argument with join keys for` JOIN`, a separate hash table is created, so memory consumption and query execution time grow linearly with an increase in the number of expressions `OR` of the `ON` clause. +The `OR` operator inside the `ON` clause works using the hash join algorithm — for each `OR` argument with join keys for `JOIN`, a separate hash table is created, so memory consumption and query execution time grow linearly with an increase in the number of expressions `OR` of the `ON` clause. !!! note "Note" If a condition refers columns from different tables, then only the equality operator (`=`) is supported so far. @@ -109,7 +109,7 @@ Result: └──────┴────────┴────────┘ ``` -Query with `INNER` and condition` OR`: +Query with `INNER` type of a join and condition `OR`: ``` sql CREATE TABLE t1 (`a` Int64, `b` Int64) ENGINE = MergeTree() ORDER BY a; @@ -135,7 +135,7 @@ Result: └───┴────┴─────┘ ``` -Query with `INNER` and condition `OR` and `AND`: +Query with `INNER` type of a join and condition `OR` and `AND`: ``` SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key AND t2.val > 3; From a4d364ab03e60e5ecef541faa8bd40b6b23f1030 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Sun, 26 Dec 2021 14:03:49 +0300 Subject: [PATCH 0392/1260] update text --- docs/en/sql-reference/statements/select/join.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index fd94020f9e9..228b50524ae 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -58,6 +58,7 @@ The behavior of ClickHouse server for `ANY JOIN` operations depends on the [any_ ## ON Section Conditions {on-section-conditions} An `ON` section can contain several conditions combined using the `AND` and `OR` operators. Conditions specifying join keys must refer both left and right tables and must use the equality operator. Other conditions may use other logical operators but they must refer either the left or the right table of a query. + Rows are joined if the whole complex condition is met. If the conditions are not met, still rows may be included in the result depending on the `JOIN` type. Note that if the same conditions are placed in a `WHERE` section and they are not met, then rows are always filtered out from the result. The `OR` operator inside the `ON` clause works using the hash join algorithm — for each `OR` argument with join keys for `JOIN`, a separate hash table is created, so memory consumption and query execution time grow linearly with an increase in the number of expressions `OR` of the `ON` clause. @@ -109,7 +110,7 @@ Result: └──────┴────────┴────────┘ ``` -Query with `INNER` type of a join and condition `OR`: +Query with `INNER` type of a join and condition with `OR`: ``` sql CREATE TABLE t1 (`a` Int64, `b` Int64) ENGINE = MergeTree() ORDER BY a; @@ -135,7 +136,7 @@ Result: └───┴────┴─────┘ ``` -Query with `INNER` type of a join and condition `OR` and `AND`: +Query with `INNER` type of a join and conditions with `OR` and `AND`: ``` SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key AND t2.val > 3; From 78f66c6d723273fce038627342c3dbb1dd2c1b82 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 26 Dec 2021 16:13:03 +0300 Subject: [PATCH 0393/1260] check-style: remove absolete filters Follow-up for: eaba6a7f59df475acb0c8cbfbfb8fcd8d71abe3d --- utils/check-style/check-style | 1 - 1 file changed, 1 deletion(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index c65099f2582..adefba7f67e 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -184,7 +184,6 @@ tables_with_database_column=( tests_with_database_column=( $( find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' | grep -vP $EXCLUDE_DIRS | - grep -v -x -e $ROOT_PATH/tests/queries/query_test.py | xargs grep --with-filename $(printf -- "-e %s " "${tables_with_database_column[@]}") | cut -d: -f1 | sort -u ) ) for test_case in "${tests_with_database_column[@]}"; do From 6d9d7df9d697c45f08a9a01b0b3b14cba4e72943 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 26 Dec 2021 16:22:04 +0300 Subject: [PATCH 0394/1260] check-style: check for event_time/event_date = today() queries --- utils/check-style/check-style | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index adefba7f67e..22b5faa0fcb 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -298,6 +298,20 @@ for src in "${sources_with_std_cerr_cout[@]}"; do fi done +# Queries with event_date should have yesterday() not today() +# +# NOTE: it is not that accuate, but at least something. +tests_with_event_time_date=( $( + find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' | + grep -vP $EXCLUDE_DIRS | + xargs grep --with-filename -e event_time -e event_date | cut -d: -f1 | sort -u +) ) +for test_case in "${tests_with_event_time_date[@]}"; do + cat "$test_case" | tr '\n' ' ' | grep -q -i -e 'WHERE.*event_date[ ]*=[ ]*today()' -e 'WHERE.*event_date[ ]*=[ ]*today()' && { + echo "event_time/event_date should be filtered using >=yesterday() in $test_case (to avoid flakiness)" + } +done + # Conflict markers find $ROOT_PATH/{src,base,programs,utils,tests,docs,website,cmake} -name '*.md' -or -name '*.cpp' -or -name '*.h' | xargs grep -P '^(<<<<<<<|=======|>>>>>>>)$' | grep -P '.' && echo "Conflict markers are found in files" From bcce1d70b23849a70fc90c4cbfbcb5211429cc09 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 26 Dec 2021 16:23:21 +0300 Subject: [PATCH 0395/1260] tests: update tests with event_time/event_date = today() to >= yesterday() MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This will fix failures like in [1], from query_log from artifacts: SELECT query, event_time FROM system.query_log WHERE (NOT is_initial_query) AND (query NOT LIKE '%system%query_log%') AND (query LIKE concat('WITH%', 'test_84qkvq', '%AS `id_no` %')) AND (type = 'QueryFinish') Query id: c5d70aba-b0aa-4f92-bdb3-29547b9aabb1 ┌─query──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬──────────event_time─┐ │ WITH _CAST('test_84qkvq', 'Nullable(String)') AS `id_no` SELECT `one`.`dummy`, ignore(`id_no`) FROM `system`.`one` WHERE `dummy` IN (0, 2) │ 2021-12-25 23:59:59 │ │ WITH _CAST('test_84qkvq', 'Nullable(String)') AS `id_no` SELECT `one`.`dummy`, ignore(`id_no`) FROM `system`.`one` WHERE `dummy` IN (0, 2) │ 2021-12-25 23:59:59 │ └────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴─────────────────────┘ 2 rows in set. Elapsed: 0.032 sec. [1]: https://s3.amazonaws.com/clickhouse-test-reports/33175/465a9bf615e1b233606460f956c09f71931c99a2/stateless_tests__debug__actions__[2/3].html --- .../01756_optimize_skip_unused_shards_rewrite_in.sql | 8 ++++---- .../0_stateless/01810_max_part_removal_threads_long.sh | 4 ++-- .../0_stateless/01814_distributed_push_down_limit.sh | 2 +- tests/queries/0_stateless/02015_global_in_threads.sh | 2 +- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql index 6e4337fc05f..220d5d91a0b 100644 --- a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql +++ b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql @@ -32,7 +32,7 @@ select '(0, 2)'; with (select currentDatabase()) as id_no select *, ignore(id_no) from dist_01756 where dummy in (0, 2); system flush logs; select query from system.query_log where - event_date = today() and + event_date >= yesterday() and event_time > now() - interval 1 hour and not is_initial_query and query not like '%system%query_log%' and @@ -51,7 +51,7 @@ select 'optimize_skip_unused_shards_rewrite_in(0, 2)'; with (select currentDatabase()) as id_02 select *, ignore(id_02) from dist_01756 where dummy in (0, 2); system flush logs; select query from system.query_log where - event_date = today() and + event_date >= yesterday() and event_time > now() - interval 1 hour and not is_initial_query and query not like '%system%query_log%' and @@ -63,7 +63,7 @@ select 'optimize_skip_unused_shards_rewrite_in(2,)'; with (select currentDatabase()) as id_2 select *, ignore(id_2) from dist_01756 where dummy in (2,); system flush logs; select query from system.query_log where - event_date = today() and + event_date >= yesterday() and event_time > now() - interval 1 hour and not is_initial_query and query not like '%system%query_log%' and @@ -75,7 +75,7 @@ select 'optimize_skip_unused_shards_rewrite_in(0,)'; with (select currentDatabase()) as id_0 select *, ignore(id_0) from dist_01756 where dummy in (0,); system flush logs; select query from system.query_log where - event_date = today() and + event_date >= yesterday() and event_time > now() - interval 1 hour and not is_initial_query and query not like '%system%query_log%' and diff --git a/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh b/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh index a2945de5b0c..c5aaa794ac9 100755 --- a/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh +++ b/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh @@ -20,7 +20,7 @@ $CLICKHOUSE_CLIENT -nm -q """ insert into data_01810 select * from numbers(50); drop table data_01810 settings log_queries=1; system flush logs; - select throwIf(length(thread_ids)<50) from system.query_log where event_date = today() and current_database = currentDatabase() and query = 'drop table data_01810 settings log_queries=1;' and type = 'QueryFinish' format Null; + select throwIf(length(thread_ids)<50) from system.query_log where event_date >= yesterday() and current_database = currentDatabase() and query = 'drop table data_01810 settings log_queries=1;' and type = 'QueryFinish' format Null; """ # ReplicatedMergeTree @@ -31,7 +31,7 @@ $CLICKHOUSE_CLIENT -nm -q """ insert into rep_data_01810 select * from numbers(50); drop table rep_data_01810 settings log_queries=1; system flush logs; - select throwIf(length(thread_ids)<50) from system.query_log where event_date = today() and current_database = currentDatabase() and query = 'drop table rep_data_01810 settings log_queries=1;' and type = 'QueryFinish' format Null; + select throwIf(length(thread_ids)<50) from system.query_log where event_date >= yesterday() and current_database = currentDatabase() and query = 'drop table rep_data_01810 settings log_queries=1;' and type = 'QueryFinish' format Null; """ $CLICKHOUSE_CLIENT -nm -q "drop database ordinary_$CLICKHOUSE_DATABASE" diff --git a/tests/queries/0_stateless/01814_distributed_push_down_limit.sh b/tests/queries/0_stateless/01814_distributed_push_down_limit.sh index 81ed4568092..1412ea3be65 100755 --- a/tests/queries/0_stateless/01814_distributed_push_down_limit.sh +++ b/tests/queries/0_stateless/01814_distributed_push_down_limit.sh @@ -69,7 +69,7 @@ function test_distributed_push_down_limit_with_query_log() system flush logs; select read_rows from system.query_log where - event_date = today() + event_date >= yesterday() and query_kind = 'Select' /* exclude DESC TABLE */ and initial_query_id = '$query_id' and initial_query_id != query_id; " | xargs # convert new lines to spaces diff --git a/tests/queries/0_stateless/02015_global_in_threads.sh b/tests/queries/0_stateless/02015_global_in_threads.sh index c112e47fe92..9437187d462 100755 --- a/tests/queries/0_stateless/02015_global_in_threads.sh +++ b/tests/queries/0_stateless/02015_global_in_threads.sh @@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --log_queries=1 --max_threads=32 --query_id "2015_${CLICKHOUSE_DATABASE}_query" -q "select count() from remote('127.0.0.{2,3}', numbers(10)) where number global in (select number % 5 from numbers_mt(1000000))" ${CLICKHOUSE_CLIENT} -q "system flush logs" -${CLICKHOUSE_CLIENT} -q "select length(thread_ids) >= 32 from system.query_log where event_date = today() and query_id = '2015_${CLICKHOUSE_DATABASE}_query' and type = 'QueryFinish' and current_database = currentDatabase()" +${CLICKHOUSE_CLIENT} -q "select length(thread_ids) >= 32 from system.query_log where event_date >= yesterday() and query_id = '2015_${CLICKHOUSE_DATABASE}_query' and type = 'QueryFinish' and current_database = currentDatabase()" From 3f752c3acb860da051bcc28d49331dbc20de51e4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 25 Dec 2021 10:33:33 +0300 Subject: [PATCH 0396/1260] Make test_input_format_parallel_parsing_memory_tracking less flaky By disabling system tables, that may consume memory too. Also note, that failures with "curl: (56) Recv failure: Connection reset by peer" like [1] is due to memory, since the server was under memory pressure, and was not able even to accept the connection:
2021.12.25 00:22:02.957658 [ 54 ] {} ThreadStatus: Code: 241. DB::Exception: Memory limit (total) exceeded: would use 2.81 GiB (attempt to allocate chunk of 2097182 bytes), maximum: 2.79 GiB. (MEMORY_LIMIT_EXCEEDED), Stack trace (when copying this message, always include the lines below): 0. ./obj-x86_64-linux-gnu/../contrib/libcxx/include/exception:0: Poco::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int) @ 0x1addca1b in /usr/bin/clickhouse 1. ./obj-x86_64-linux-gnu/../src/Common/Exception.cpp:57: DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int, bool) @ 0xa0e993c in /usr/bin/clickhouse 2. ./obj-x86_64-linux-gnu/../contrib/libcxx/include/string:1444: DB::Exception::Exception, std::__1::allocator >, long&, std::__1::basic_string, std::__1::allocator > >(int, std::__1::basic_string, std::__1::allocator > const&, char const*&&, char const*&&, std::__1::basic_string, std::__1::allocator >&&, long&, std::__1::basic_string, std::__1::allocator >&&) @ 0xa11344b in /usr/bin/clickhouse 3. ./obj-x86_64-linux-gnu/../src/Common/MemoryTracker.cpp:226: MemoryTracker::allocImpl(long, bool) @ 0xa11220c in /usr/bin/clickhouse 4. ./obj-x86_64-linux-gnu/../src/Common/MemoryTracker.cpp:258: MemoryTracker::allocImpl(long, bool) @ 0xa111faa in /usr/bin/clickhouse 5. ./obj-x86_64-linux-gnu/../src/Common/MemoryTracker.cpp:264: MemoryTracker::alloc(long) @ 0xa112bc6 in /usr/bin/clickhouse 6. ./obj-x86_64-linux-gnu/../src/Common/ThreadStatus.cpp:152: DB::ThreadStatus::~ThreadStatus() @ 0xa10e182 in /usr/bin/clickhouse 7. ./obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:518: DB::TCPHandler::runImpl() @ 0x17c4192d in /usr/bin/clickhouse 8. ./obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:1909: DB::TCPHandler::run() @ 0x17c55328 in /usr/bin/clickhouse 9. ./obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerConnection.cpp:57: Poco::Net::TCPServerConnection::start() @ 0x1ac19f03 in /usr/bin/clickhouse 10. ./obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:1397: Poco::Net::TCPServerDispatcher::run() @ 0x1ac1a773 in /usr/bin/clickhouse 11. ./obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/ThreadPool.cpp:213: Poco::PooledThread::run() @ 0x1ae7e796 in /usr/bin/clickhouse 12. ./obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Thread.cpp:56: Poco::(anonymous namespace)::RunnableHolder::run() @ 0x1ae7c930 in /usr/bin/clickhouse 13. ./obj-x86_64-linux-gnu/../contrib/poco/Foundation/include/Poco/SharedPtr.h:277: Poco::ThreadImpl::runnableEntry(void*) @ 0x1ae7afa8 in /usr/bin/clickhouse 14. __tsan_thread_start_func @ 0xa039dfd in /usr/bin/clickhouse 15. ? @ 0x7f4c565cf609 in ? 16. clone @ 0x7f4c564f6293 in ? (version 21.13.1.1305)
[1]: https://s3.amazonaws.com/clickhouse-test-reports/33140/b4420cfa3e834ab9026914978697ded3180122a0/integration_tests__thread__actions__[3/4].html --- .../configs/conf.xml | 14 ++++++++++++++ .../test.py | 7 ++----- 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_input_format_parallel_parsing_memory_tracking/configs/conf.xml b/tests/integration/test_input_format_parallel_parsing_memory_tracking/configs/conf.xml index 3e4c885d1f6..7fb89245a3c 100644 --- a/tests/integration/test_input_format_parallel_parsing_memory_tracking/configs/conf.xml +++ b/tests/integration/test_input_format_parallel_parsing_memory_tracking/configs/conf.xml @@ -1,4 +1,18 @@ 3000000000 + + + + + + + + + + + + + + diff --git a/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py b/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py index bc7f32bf544..693c1dc13ad 100644 --- a/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py +++ b/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py @@ -26,14 +26,11 @@ def start_cluster(): # max_memory_usage_for_user cannot be used, since the memory for user accounted # correctly, only total is not def test_memory_tracking_total(): - instance.query(''' - CREATE TABLE null (row String) ENGINE=Null; - ''') + instance.query('CREATE TABLE null (row String) ENGINE=Null') instance.exec_in_container(['bash', '-c', 'clickhouse local -q "SELECT arrayStringConcat(arrayMap(x->toString(cityHash64(x)), range(1000)), \' \') from numbers(10000)" > data.json']) for it in range(0, 20): # the problem can be triggered only via HTTP, # since clickhouse-client parses the data by itself. assert instance.exec_in_container(['curl', '--silent', '--show-error', '--data-binary', '@data.json', - 'http://127.1:8123/?query=INSERT%20INTO%20null%20FORMAT%20TSV']) == '', 'Failed on {} iteration'.format( - it) + 'http://127.1:8123/?query=INSERT%20INTO%20null%20FORMAT%20TSV']) == '', f'Failed on {it} iteration' From 906ebc9fd44acd4d32ea2e69988d24f4bd6ddd33 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 26 Dec 2021 16:50:14 +0300 Subject: [PATCH 0397/1260] test_input_format_parallel_parsing_memory_tracking: increase max_server_memory_usage Even after disabling all system tables TSan build still not passed, since it uses 2.8Gi of RAM just after start, so let's increase this memory limit a little. --- .../configs/conf.xml | 9 +++++++-- .../test.py | 2 +- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_input_format_parallel_parsing_memory_tracking/configs/conf.xml b/tests/integration/test_input_format_parallel_parsing_memory_tracking/configs/conf.xml index 7fb89245a3c..3adba1d402a 100644 --- a/tests/integration/test_input_format_parallel_parsing_memory_tracking/configs/conf.xml +++ b/tests/integration/test_input_format_parallel_parsing_memory_tracking/configs/conf.xml @@ -1,6 +1,11 @@ - - 3000000000 + + 4000000000 diff --git a/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py b/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py index 693c1dc13ad..1c686c7982e 100644 --- a/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py +++ b/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py @@ -24,7 +24,7 @@ def start_cluster(): # max_memory_usage_for_user cannot be used, since the memory for user accounted -# correctly, only total is not +# correctly, only total is not (it is set via conf.xml) def test_memory_tracking_total(): instance.query('CREATE TABLE null (row String) ENGINE=Null') instance.exec_in_container(['bash', '-c', From 9f17e7bad00d7a45304342e3c252ce367ac4e8d8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 26 Dec 2021 20:49:33 +0300 Subject: [PATCH 0398/1260] Fix ddl validation --- .../materialized-postgresql.md | 6 +- .../MaterializedPostgreSQLConsumer.cpp | 172 +++++++++++------- .../MaterializedPostgreSQLConsumer.h | 9 +- .../test.py | 35 ++++ 4 files changed, 157 insertions(+), 65 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 4dea156f32e..c1fd8cd5757 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -7,7 +7,7 @@ toc_title: MaterializedPostgreSQL Creates a ClickHouse database with tables from PostgreSQL database. Firstly, database with engine `MaterializedPostgreSQL` creates a snapshot of PostgreSQL database and loads required tables. Required tables can include any subset of tables from any subset of schemas from specified database. Along with the snapshot database engine acquires LSN and once initial dump of tables is performed - it starts pulling updates from WAL. After database is created, newly added tables to PostgreSQL database are not automatically added to replication. They have to be added manually with `ATTACH TABLE db.table` query. -Replication is implemented with PostgreSQL Logical Replication Protocol, which does not allow to replicate DDL, but allows to know whether replication breaking changes happened (column type changes, adding/removing columns). Such changes are detected and according tables stop receiving updates. Such tables can be automatically reloaded in the background in case required setting is turned on. Safest way for now is to use `ATTACH`/ `DETACH` queries to reload table completely. If DDL does not break replication (for example, renaming a column) table will still receive updates (insertion is done by position). +Replication is implemented with PostgreSQL Logical Replication Protocol, which does not allow to replicate DDL, but allows to know whether replication breaking changes happened (column type changes, adding/removing columns). Such changes are detected and according tables stop receiving updates. Such tables can be automatically reloaded in the background in case required setting is turned on (can be used starting from 22.1). Safest way for now is to use `ATTACH`/ `DETACH` queries to reload table completely. If DDL does not break replication (for example, renaming a column) table will still receive updates (insertion is done by position). ## Creating a Database {#creating-a-database} @@ -46,7 +46,7 @@ After `MaterializedPostgreSQL` database is created, it does not automatically de ATTACH TABLE postgres_database.new_table; ``` -Warning: before version 21.13 adding table to replication left unremoved temprorary replication slot (named `{db_name}_ch_replication_slot_tmp`). If attaching tables in clickhouse version before 21.13, make sure to delete it manually (`SELECT pg_drop_replication_slot('{db_name}_ch_replication_slot_tmp')`). Otherwise disk usage will grow. Issue is fixed in 21.13. +Warning: before version 22.1 adding table to replication left unremoved temprorary replication slot (named `{db_name}_ch_replication_slot_tmp`). If attaching tables in clickhouse version before 22.1, make sure to delete it manually (`SELECT pg_drop_replication_slot('{db_name}_ch_replication_slot_tmp')`). Otherwise disk usage will grow. Issue is fixed in 22.1. ## Dynamically removing tables from replication {#dynamically-removing-table-from-replication} @@ -156,6 +156,8 @@ Default value: empty list. (Default schema is used) 4. materialized_postgresql_allow_automatic_update {#materialized-postgresql-allow-automatic-update} +Do not use this setting before 22.1 version. + Allows reloading table in the background, when schema changes are detected. DDL queries on the PostgreSQL side are not replicated via ClickHouse [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) engine, because it is not allowed with PostgreSQL logical replication protocol, but the fact of DDL changes is detected transactionally. In this case, the default behaviour is to stop replicating those tables once DDL is detected. However, if this setting is enabled, then, instead of stopping the replication of those tables, they will be reloaded in the background via database snapshot without data losses and replication will continue for them. Possible values: diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 4c66eda2fed..be35c4e3572 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -18,6 +18,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int POSTGRESQL_REPLICATION_INTERNAL_ERROR; } MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( @@ -94,23 +95,31 @@ void MaterializedPostgreSQLConsumer::insertValue(Buffer & buffer, const std::str const auto & sample = buffer.description.sample_block.getByPosition(column_idx); bool is_nullable = buffer.description.types[column_idx].second; - if (is_nullable) + try { - ColumnNullable & column_nullable = assert_cast(*buffer.columns[column_idx]); - const auto & data_type = assert_cast(*sample.type); + if (is_nullable) + { + ColumnNullable & column_nullable = assert_cast(*buffer.columns[column_idx]); + const auto & data_type = assert_cast(*sample.type); - insertPostgreSQLValue( - column_nullable.getNestedColumn(), value, - buffer.description.types[column_idx].first, data_type.getNestedType(), buffer.array_info, column_idx); + insertPostgreSQLValue( + column_nullable.getNestedColumn(), value, + buffer.description.types[column_idx].first, data_type.getNestedType(), buffer.array_info, column_idx); - column_nullable.getNullMapData().emplace_back(0); + column_nullable.getNullMapData().emplace_back(0); + } + else + { + insertPostgreSQLValue( + *buffer.columns[column_idx], value, + buffer.description.types[column_idx].first, sample.type, + buffer.array_info, column_idx); + } } - else + catch (const pqxx::conversion_error & e) { - insertPostgreSQLValue( - *buffer.columns[column_idx], value, - buffer.description.types[column_idx].first, sample.type, - buffer.array_info, column_idx); + LOG_ERROR(log, "Conversion failed while inserting PostgreSQL value {}, will insert default value. Error: {}", value, e.what()); + insertDefaultValue(buffer, column_idx); } } @@ -190,6 +199,12 @@ void MaterializedPostgreSQLConsumer::readTupleData( { Int16 num_columns = readInt16(message, pos, size); + /// Sanity check. In fact, it was already checked. + if (static_cast(num_columns) + 2 != buffer.getColumnsNum()) /// +2 -- sign and version columns + throw Exception(ErrorCodes::POSTGRESQL_REPLICATION_INTERNAL_ERROR, + "Number of columns does not match. Got: {}, expected {}, current buffer structure: {}", + num_columns, buffer.getColumnsNum(), buffer.description.sample_block.dumpStructure()); + auto proccess_column_value = [&](Int8 identifier, Int16 column_idx) { switch (identifier) @@ -202,8 +217,15 @@ void MaterializedPostgreSQLConsumer::readTupleData( case 't': /// Text formatted value { Int32 col_len = readInt32(message, pos, size); - String value; + /// Sanity check for protocol misuse. + /// PostgreSQL uses a fixed page size (commonly 8 kB), and does not allow tuples to span multiple pages. + static constexpr Int32 sanity_check_max_col_len = 1024 * 8 * 2; /// *2 -- just in case. + if (unlikely(col_len > sanity_check_max_col_len)) + throw Exception(ErrorCodes::POSTGRESQL_REPLICATION_INTERNAL_ERROR, + "Column legth is suspiciously long: {}", col_len); + + String value; for (Int32 i = 0; i < col_len; ++i) value += readInt8(message, pos, size); @@ -276,9 +298,11 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl { Int32 relation_id = readInt32(replication_message, pos, size); const auto & table_name = relation_id_to_name[relation_id]; - /// FIXME:If table name is empty here, it means we failed to load it, but it was included in publication. Need to remove? if (table_name.empty()) - LOG_WARNING(log, "No table mapping for relation id: {}. Probably table failed to be loaded", relation_id); + { + LOG_ERROR(log, "No table mapping for relation id: {}. It's a bug", relation_id); + return; + } if (!isSyncAllowed(relation_id, table_name)) return; @@ -296,9 +320,11 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl { Int32 relation_id = readInt32(replication_message, pos, size); const auto & table_name = relation_id_to_name[relation_id]; - /// FIXME:If table name is empty here, it means we failed to load it, but it was included in publication. Need to remove? if (table_name.empty()) - LOG_WARNING(log, "No table mapping for relation id: {}. Probably table failed to be loaded", relation_id); + { + LOG_ERROR(log, "No table mapping for relation id: {}. It's a bug", relation_id); + return; + } if (!isSyncAllowed(relation_id, table_name)) return; @@ -347,9 +373,11 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl { Int32 relation_id = readInt32(replication_message, pos, size); const auto & table_name = relation_id_to_name[relation_id]; - /// FIXME:If table name is empty here, it means we failed to load it, but it was included in publication. Need to remove? if (table_name.empty()) - LOG_WARNING(log, "No table mapping for relation id: {}. Probably table failed to be loaded", relation_id); + { + LOG_ERROR(log, "No table mapping for relation id: {}. It's a bug", relation_id); + return; + } if (!isSyncAllowed(relation_id, table_name)) return; @@ -359,8 +387,8 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl auto buffer = buffers.find(table_name); assert(buffer != buffers.end()); - readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::DELETE); + readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::DELETE); break; } case 'C': // Commit @@ -379,7 +407,6 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl Int32 relation_id = readInt32(replication_message, pos, size); String relation_namespace, relation_name; - readString(replication_message, pos, size, relation_namespace); readString(replication_message, pos, size, relation_name); @@ -389,22 +416,33 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl else table_name = relation_name; + if (!relation_id_to_name.contains(relation_id)) + relation_id_to_name[relation_id] = table_name; + if (!isSyncAllowed(relation_id, relation_name)) return; if (storages.find(table_name) == storages.end()) { - markTableAsSkipped(relation_id, table_name); - /// TODO: This can happen if we created a publication with this table but then got an exception that this + /// FIXME: This can happen if we created a publication with this table but then got an exception that this /// table has primary key or something else. LOG_ERROR(log, - "Storage for table {} does not exist, but is included in replication stream. (Storages number: {})", + "Storage for table {} does not exist, but is included in replication stream. (Storages number: {})" + "Please manually remove this table from replication (DETACH TABLE query) to avoid redundant replication", table_name, storages.size()); + markTableAsSkipped(relation_id, table_name); return; } - assert(buffers.contains(table_name)); - + auto buffer_iter = buffers.find(table_name); + if (buffer_iter == buffers.end()) + { + /// Must never happen if previous check for storage existance passed, but just in case. + LOG_ERROR(log, "No buffer found for table `{}`.", table_name); + markTableAsSkipped(relation_id, table_name); + return; + } + const auto & buffer = buffer_iter->second; /// 'd' - default (primary key if any) /// 'n' - nothing @@ -412,7 +450,6 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl /// 'i' - user defined index with indisreplident set /// Only 'd' and 'i' - are supported. char replica_identity = readInt8(replication_message, pos, size); - if (replica_identity != 'd' && replica_identity != 'i') { LOG_WARNING(log, @@ -423,25 +460,36 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl Int16 num_columns = readInt16(replication_message, pos, size); - Int32 data_type_id; - Int32 type_modifier; /// For example, n in varchar(n) - - bool new_relation_definition = false; - if (schema_data.find(relation_id) == schema_data.end()) - { - relation_id_to_name[relation_id] = table_name; - schema_data.emplace(relation_id, SchemaData(num_columns)); - new_relation_definition = true; - } - - auto & current_schema_data = schema_data.find(relation_id)->second; - - if (current_schema_data.number_of_columns != num_columns) + if (static_cast(num_columns) + 2 != buffer.getColumnsNum()) /// +2 -- sign and version columns { + LOG_DEBUG(log, "Mismatch in columns size. Got {}, expected {}. Current table structure: {}", + num_columns, buffer.getColumnsNum(), buffer.description.sample_block.dumpStructure()); /// Not an error. markTableAsSkipped(relation_id, table_name); return; } + Int32 data_type_id; + Int32 type_modifier; /// For example, n in varchar(n) + + auto schema_def_for_table_iter = schema_data.find(relation_id); + bool new_relation_definition = schema_def_for_table_iter == schema_data.end(); + SchemaData current_schema_data; + if (new_relation_definition) + { + current_schema_data = SchemaData(num_columns); + } + else + { + current_schema_data = schema_def_for_table_iter->second; + if (current_schema_data.number_of_columns != num_columns) + { + markTableAsSkipped(relation_id, table_name); + return; + } + } + + /// FIXME:!!! If table was reloaded and server restarted before it commited lsn with update point, it + /// will be marked as skipped once again and reloaded one more time. for (uint16_t i = 0; i < num_columns; ++i) { String column_name; @@ -466,8 +514,10 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl } } - tables_to_sync.insert(table_name); + if (new_relation_definition) + schema_data.emplace(relation_id, current_schema_data); + tables_to_sync.insert(table_name); break; } case 'O': // Origin @@ -706,7 +756,17 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot() current_lsn = (*row)[0]; lsn_value = getLSNValue(current_lsn); - processReplicationMessage((*row)[1].c_str(), (*row)[1].size()); + try + { + // LOG_DEBUG(log, "Current message: {}", (*row)[1]); + processReplicationMessage((*row)[1].c_str(), (*row)[1].size()); + } + catch (const Exception & e) + { + if (e.code() == ErrorCodes::POSTGRESQL_REPLICATION_INTERNAL_ERROR) + continue; + throw; + } } } catch (const Exception &) @@ -737,11 +797,6 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot() LOG_ERROR(log, "Conversion error: {}", e.what()); return false; } - catch (const pqxx::in_doubt_error & e) - { - LOG_ERROR(log, "PostgreSQL library has some doubts: {}", e.what()); - return false; - } catch (const pqxx::internal_error & e) { LOG_ERROR(log, "PostgreSQL library internal error: {}", e.what()); @@ -749,16 +804,8 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot() } catch (...) { - /// Since reading is done from a background task, it is important to catch any possible error - /// in order to understand why something does not work. - try - { - std::rethrow_exception(std::current_exception()); - } - catch (const std::exception& e) - { - LOG_ERROR(log, "Unexpected error: {}", e.what()); - } + tryLogCurrentException(__PRETTY_FUNCTION__); + return false; } if (!tables_to_sync.empty()) @@ -770,6 +817,11 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot() bool MaterializedPostgreSQLConsumer::consume(std::vector> & skipped_tables) { + /// Read up to max_block_size changed (approximately - in same cases might be more). + /// false: no data was read, reschedule. + /// true: some data was read, schedule as soon as possible. + auto read_next = readFromReplicationSlot(); + /// Check if there are tables, which are skipped from being updated by changes from replication stream, /// because schema changes were detected. Update them, if it is allowed. if (allow_automatic_update && !skip_list.empty()) @@ -786,10 +838,6 @@ bool MaterializedPostgreSQLConsumer::consume(std::vector; static void insertDefaultValue(Buffer & buffer, size_t column_idx); - static void insertValue(Buffer & buffer, const std::string & value, size_t column_idx); + void insertValue(Buffer & buffer, const std::string & value, size_t column_idx); enum class PostgreSQLQuery { @@ -139,6 +145,7 @@ private: std::vector> column_identifiers; SchemaData(Int16 number_of_columns_) : number_of_columns(number_of_columns_) {} + SchemaData() = default; }; /// Cache for table schema data to be able to detect schema changes, because ddl is not diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index c8b63d8e667..4f04264d18f 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -589,6 +589,41 @@ def test_table_override(started_cluster): drop_postgres_table(cursor, table_name) +def test_table_schema_changes_2(started_cluster): + drop_materialized_db() + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True) + cursor = conn.cursor() + + table_name = "test_table" + + create_postgres_table(cursor, table_name, template=postgres_table_template_2); + instance.query(f"INSERT INTO postgres_database.{table_name} SELECT number, number, number, number from numbers(25)") + + create_materialized_db(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=["materialized_postgresql_allow_automatic_update = 1, materialized_postgresql_tables_list='test_table'"]) + + instance.query(f"INSERT INTO postgres_database.{table_name} SELECT number, number, number, number from numbers(25, 25)") + check_tables_are_synchronized(table_name); + + cursor.execute(f"ALTER TABLE {table_name} DROP COLUMN value1") + cursor.execute(f"ALTER TABLE {table_name} DROP COLUMN value2") + cursor.execute(f"ALTER TABLE {table_name} ADD COLUMN value1 Text") + cursor.execute(f"ALTER TABLE {table_name} ADD COLUMN value2 Text") + cursor.execute(f"ALTER TABLE {table_name} DROP COLUMN value3") + cursor.execute(f"ALTER TABLE {table_name} ADD COLUMN value3 Text") + cursor.execute(f"ALTER TABLE {table_name} ADD COLUMN value4 Text") + cursor.execute(f"UPDATE {table_name} SET value3 = 'kek' WHERE key%2=0") + check_tables_are_synchronized(table_name); + instance.query(f"INSERT INTO postgres_database.{table_name} SELECT number, toString(number), toString(number), toString(number), toString(number) from numbers(50, 25)") + cursor.execute(f"ALTER TABLE {table_name} ADD COLUMN value5 Integer") + cursor.execute(f"ALTER TABLE {table_name} DROP COLUMN value2") + instance.query(f"INSERT INTO postgres_database.{table_name} SELECT number, toString(number), toString(number), toString(number), number from numbers(75, 25)") + check_tables_are_synchronized(table_name); + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From 8dc07dff4ab72d33029b59164da035ee0f0dadb9 Mon Sep 17 00:00:00 2001 From: Yatian Xu Date: Sun, 26 Dec 2021 11:24:47 -0800 Subject: [PATCH 0399/1260] add Clarification for test case --- tests/integration/test_system_metrics/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_system_metrics/test.py b/tests/integration/test_system_metrics/test.py index 02d9a676efb..efcc6f88a24 100644 --- a/tests/integration/test_system_metrics/test.py +++ b/tests/integration/test_system_metrics/test.py @@ -59,6 +59,7 @@ def test_readonly_metrics(start_cluster): node1.query("ATTACH TABLE test.test_table") assert_eq_with_retry(node1, "SELECT value FROM system.metrics WHERE metric = 'ReadonlyReplica'", "0\n", retry_count=300, sleep_time=1) +#For LowCardinality-columns, the bytes for N rows is not N*size of 1 row. def test_metrics_storage_buffer_size(start_cluster): node1.query(''' CREATE TABLE test.test_mem_table From dc768173297f6baefb67d499ac5caceb92de3e1b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 26 Dec 2021 22:09:53 +0300 Subject: [PATCH 0400/1260] Fix 01370_client_autocomplete_word_break_characters test logic AFAICS right now it works only because completion loaded too quickly, since `expect "_"` will block for 60 seconds if completion is not available. --- ...01370_client_autocomplete_word_break_characters.expect | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect b/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect index e0d01d905bb..f3a28bbee9b 100755 --- a/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect +++ b/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect @@ -20,6 +20,7 @@ expect "SET max_distributed" # Wait for suggestions to load, they are loaded in background set is_done 0 +set timeout 1 while {$is_done == 0} { send -- "\t" expect { @@ -27,10 +28,15 @@ while {$is_done == 0} { set is_done 1 } default { - sleep 1 + # expect "_" will wait for timeout, + # if completion was not loaded it will fail, + # and we will retry, + # but for retry on timeout default should be reseted, + # this is what for this block. } } } +set timeout 60 send -- "\3\4" expect eof From e35a91eb1178d8b2e29071a302769d5693a37b22 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Mon, 27 Dec 2021 00:52:08 +0300 Subject: [PATCH 0401/1260] Update docs/ru/operations/settings/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 688afa8279b..860ac900ba5 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1649,7 +1649,7 @@ ClickHouse генерирует исключение: ## insert_quorum_parallel {#settings-insert_quorum_parallel} -Включает и выключает параллелизм для кворума запросов `INSERT`. Когда включена, можно выполнить дополнительные запросы `INSERT` в то время, пока предыдущие запросы еще не завершены. Когда выключена, допольнительные записи в ту же таблицу будут отклонены. +Включает и выключает параллелизм для кворума запросов `INSERT`. Когда включена, можно выполнить дополнительные запросы `INSERT` в то время, пока предыдущие запросы еще не завершены. Когда выключена, дополнительные записи в ту же таблицу будут отклонены. Возможные значения: From 51e43ae6194411a413fc48b5aed9a4c7127c02a3 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 27 Dec 2021 10:20:07 +0800 Subject: [PATCH 0402/1260] some modification in LRUCache --- src/Common/LRUCache.h | 24 +++++++++++++----------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 856a230674b..2ca5e007716 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -21,7 +21,7 @@ struct TrivialWeightFunction }; template -struct TrivialLRUCacheEvitPolicy +struct TrivialLRUCacheEvictPolicy { inline bool canRelease(std::shared_ptr) const { @@ -43,13 +43,21 @@ template , typename WeightFunction = TrivialWeightFunction, - typename EvictPolicy = TrivialLRUCacheEvitPolicy> + typename EvictPolicy = TrivialLRUCacheEvictPolicy> class LRUCache { public: using Key = TKey; using Mapped = TMapped; using MappedPtr = std::shared_ptr; + + struct Result + { + MappedPtr value; + bool cache_miss = true; + // set_successful is not trustworthy for getOrSet, because removeOverflow is called right after putting key in cache + bool set_successful = false; + }; /** Initialize LRUCache with max_size and max_elements_size. * max_elements_size == 0 means no elements size restrictions. @@ -89,12 +97,11 @@ public: return setImpl(key, mapped, lock); } - /// Returns std::pair of the cached value and a bool indicating whether the value was produced during this call. template std::pair getOrSet(const Key & key, LoadFunc && load_func) { - auto [value, is_loaded, _] = getOrTrySet(key, std::move(load_func)); - return std::make_pair(value, is_loaded); + auto result = getOrTrySet(key, std::move(load_func)); + return std::make_pair(result.value, result.cache_miss); } /// If the value for the key is in the cache, returns it. If it is not, calls load_func() to @@ -104,12 +111,8 @@ public: /// Exceptions occurring in load_func will be propagated to the caller. Another thread from the /// set of concurrent threads will then try to call its load_func etc. /// - /// return std::tuple is , where - /// - is_value_loaded indicates whether the value was produce during this call - /// - is_value_updated indicates whether the value is updated in the cache when is_value_loaded = true. - /// if is_value_loaded = false, is_value_updated = false template - std::tuple getOrTrySet(const Key &key, LoadFunc && load_func) + Result getOrTrySet(const Key &key, LoadFunc && load_func) { InsertTokenHolder token_holder; { @@ -345,7 +348,6 @@ private: if (inserted) { auto value_weight = mapped ? weight_function(*mapped) : 0; - // move removeOverflow() ahead here. In default, the final result is the same as the old implementation if (!removeOverflow(value_weight)) { // cannot find enough space to put in the new value From 568aeec0b3066e873d56eed8fdfc7edbb8d31c68 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 Dec 2021 09:51:29 +0300 Subject: [PATCH 0403/1260] Review fix --- .../test_postgresql_replica_database_engine_2/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 725158920a1..7aee454c4a9 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -440,9 +440,9 @@ def test_database_with_single_non_default_schema(started_cluster): print('DETACH-ATTACH') detached_table_name = "postgresql_replica_1" instance.query(f"DETACH TABLE {materialized_db}.{detached_table_name}") - assert instance.contains_in_log("from publication, because table does not exist in PostgreSQL") == False + assert not instance.contains_in_log("from publication, because table does not exist in PostgreSQL") instance.query(f"ATTACH TABLE {materialized_db}.{detached_table_name}") - check_tables_are_synchronized(f"{detached_table_name}", postgres_database=clickhouse_postgres_db); + check_tables_are_synchronized(detached_table_name, postgres_database=clickhouse_postgres_db); drop_materialized_db() @@ -513,10 +513,10 @@ def test_database_with_multiple_non_default_schemas_1(started_cluster): print('DETACH-ATTACH') detached_table_name = "postgresql_replica_1" instance.query(f"DETACH TABLE {materialized_db}.`{schema_name}.{detached_table_name}`") - assert instance.contains_in_log("from publication, because table does not exist in PostgreSQL") == False + assert not instance.contains_in_log("from publication, because table does not exist in PostgreSQL") instance.query(f"ATTACH TABLE {materialized_db}.`{schema_name}.{detached_table_name}`") assert_show_tables("test_schema.postgresql_replica_0\ntest_schema.postgresql_replica_1\ntest_schema.postgresql_replica_2\ntest_schema.postgresql_replica_3\ntest_schema.postgresql_replica_4\n") - check_tables_are_synchronized(f"{detached_table_name}", schema_name=schema_name, postgres_database=clickhouse_postgres_db); + check_tables_are_synchronized(detached_table_name, schema_name=schema_name, postgres_database=clickhouse_postgres_db); drop_materialized_db() @@ -594,7 +594,7 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): detached_table_schema = "schema0" clickhouse_postgres_db = f'clickhouse_postgres_db0' instance.query(f"DETACH TABLE {materialized_db}.`{detached_table_schema}.{detached_table_name}`") - assert instance.contains_in_log("from publication, because table does not exist in PostgreSQL") == False + assert not instance.contains_in_log("from publication, because table does not exist in PostgreSQL") instance.query(f"ATTACH TABLE {materialized_db}.`{detached_table_schema}.{detached_table_name}`") assert_show_tables("schema0.postgresql_replica_0\nschema0.postgresql_replica_1\nschema1.postgresql_replica_0\nschema1.postgresql_replica_1\n") check_tables_are_synchronized(f"postgresql_replica_{altered_table}", schema_name=detached_table_schema, postgres_database=clickhouse_postgres_db); From 078521496a765fe9a27b39d601e8c8abe6dfd421 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 27 Dec 2021 15:04:26 +0800 Subject: [PATCH 0404/1260] modifications based on pr review --- src/Common/LRUCache.h | 34 +++++------- .../Cache/ExternalDataSourceCache.cpp | 24 ++------- src/Storages/Cache/ExternalDataSourceCache.h | 2 +- src/Storages/Cache/RemoteCacheController.cpp | 54 +++++++++---------- src/Storages/Cache/RemoteCacheController.h | 3 +- src/Storages/Cache/RemoteFileCachePolicy.h | 2 +- src/Storages/Hive/StorageHive.cpp | 7 ++- 7 files changed, 54 insertions(+), 72 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index c503619a7d0..2ca5e007716 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -7,16 +7,10 @@ #include #include #include -#include namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_RELEASE; -} - template struct TrivialWeightFunction { @@ -27,15 +21,13 @@ struct TrivialWeightFunction }; template -struct TrivialLRUCacheEvitPolicy +struct TrivialLRUCacheEvictPolicy { - // To note that the arg could be null inline bool canRelease(std::shared_ptr) const { return true; } - // To note that the arg could be null inline void release(std::shared_ptr) { } @@ -51,13 +43,21 @@ template , typename WeightFunction = TrivialWeightFunction, - typename EvictPolicy = TrivialLRUCacheEvitPolicy> + typename EvictPolicy = TrivialLRUCacheEvictPolicy> class LRUCache { public: using Key = TKey; using Mapped = TMapped; using MappedPtr = std::shared_ptr; + + struct Result + { + MappedPtr value; + bool cache_miss = true; + // set_successful is not trustworthy for getOrSet, because removeOverflow is called right after putting key in cache + bool set_successful = false; + }; /** Initialize LRUCache with max_size and max_elements_size. * max_elements_size == 0 means no elements size restrictions. @@ -97,12 +97,11 @@ public: return setImpl(key, mapped, lock); } - /// Returns std::pair of the cached value and a bool indicating whether the value was produced during this call. template std::pair getOrSet(const Key & key, LoadFunc && load_func) { - auto [value, is_loaded, _] = getOrTrySet(key, std::move(load_func)); - return std::make_pair(value, is_loaded); + auto result = getOrTrySet(key, std::move(load_func)); + return std::make_pair(result.value, result.cache_miss); } /// If the value for the key is in the cache, returns it. If it is not, calls load_func() to @@ -112,12 +111,8 @@ public: /// Exceptions occurring in load_func will be propagated to the caller. Another thread from the /// set of concurrent threads will then try to call its load_func etc. /// - /// return std::tuple is , where - /// - is_value_loaded indicates whether the value was produce during this call - /// - is_value_updated indicates whether the value is updated in the cache when is_value_loaded = true. - /// if is_value_loaded = false, is_value_updated = false template - std::tuple getOrTrySet(const Key &key, LoadFunc && load_func) + Result getOrTrySet(const Key &key, LoadFunc && load_func) { InsertTokenHolder token_holder; { @@ -353,7 +348,6 @@ private: if (inserted) { auto value_weight = mapped ? weight_function(*mapped) : 0; - // move removeOverflow() ahead here. In default, the final result is the same as the old implementation if (!removeOverflow(value_weight)) { // cannot find enough space to put in the new value @@ -375,7 +369,7 @@ private: { if (!evict_policy.canRelease(cell.value)) { - // the old value is refered by someone, cannot release now + // the old value is referred by someone, cannot release now // in default policy, it is always true. return false; } diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp index 872ce815e58..f8d5c95f859 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.cpp +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -41,23 +41,8 @@ RemoteReadBuffer::~RemoteReadBuffer() file_cache_controller->deallocFile(std::move(file_buffer)); } -std::unique_ptr RemoteReadBuffer::create(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer) +std::unique_ptr RemoteReadBuffer::create(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer, size_t buff_size) { - auto * log = &Poco::Logger::get("RemoteReadBuffer"); - size_t buff_size = DBMS_DEFAULT_BUFFER_SIZE; - if (read_buffer) - buff_size = read_buffer->internalBuffer().size(); - /* - * in the new implement of ReadBufferFromHDFS, buffer size is 0. - * - * in the common case, we don't read bytes from readbuffer directly, so set buff_size = DBMS_DEFAULT_BUFFER_SIZE - * is OK. - * - * we need be careful with the case without local file reader. - */ - if (buff_size == 0) - buff_size = DBMS_DEFAULT_BUFFER_SIZE; - auto remote_path = remote_file_metadata->remote_path; auto remote_read_buffer = std::make_unique(buff_size); ErrorCodes::ErrorCode error; @@ -65,8 +50,6 @@ std::unique_ptr RemoteReadBuffer::create(ContextPtr context, IRemote std::tie(remote_read_buffer->file_cache_controller, read_buffer, error) = ExternalDataSourceCache::instance().createReader(context, remote_file_metadata, read_buffer); if (remote_read_buffer->file_cache_controller == nullptr) { - LOG_ERROR(log, "Failed to allocate local file for remote path: {}, reason: {}.", remote_path, error); - // read_buffer is the input one. return read_buffer; } else @@ -120,7 +103,10 @@ off_t RemoteReadBuffer::getPosition() ExternalDataSourceCache::ExternalDataSourceCache() = default; -ExternalDataSourceCache::~ExternalDataSourceCache() = default; +ExternalDataSourceCache::~ExternalDataSourceCache() +{ + recover_task_holder->deactivate(); +} ExternalDataSourceCache & ExternalDataSourceCache::instance() { diff --git a/src/Storages/Cache/ExternalDataSourceCache.h b/src/Storages/Cache/ExternalDataSourceCache.h index 9f350da65ca..24c1bcd8607 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.h +++ b/src/Storages/Cache/ExternalDataSourceCache.h @@ -34,7 +34,7 @@ class RemoteReadBuffer : public BufferWithOwnMemory public: explicit RemoteReadBuffer(size_t buff_size); ~RemoteReadBuffer() override; - static std::unique_ptr create(ContextPtr contex, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer); + static std::unique_ptr create(ContextPtr contex, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer, size_t buff_size); bool nextImpl() override; off_t seek(off_t off, int whence) override; diff --git a/src/Storages/Cache/RemoteCacheController.cpp b/src/Storages/Cache/RemoteCacheController.cpp index 3f866415a0b..0cee2733bee 100644 --- a/src/Storages/Cache/RemoteCacheController.cpp +++ b/src/Storages/Cache/RemoteCacheController.cpp @@ -1,6 +1,8 @@ #include #include #include +#include +#include #include #include #include @@ -15,20 +17,6 @@ namespace ErrorCodes extern const int OK; extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; - extern const int END_OF_FILE; -} - -bool RemoteCacheController::loadInnerInformation(const fs::path & file_path) -{ - if (!fs::exists(file_path)) - return false; - std::ifstream info_file(file_path); - Poco::JSON::Parser info_parser; - auto info_json = info_parser.parse(info_file).extract(); - file_status = static_cast(info_json->get("file_status").convert()); - metadata_class = info_json->get("metadata_class").convert(); - info_file.close(); - return true; } std::shared_ptr RemoteCacheController::recover(const std::filesystem::path & local_path_) @@ -37,13 +25,12 @@ std::shared_ptr RemoteCacheController::recover(const std: if (!std::filesystem::exists(local_path_ / "data.bin")) { - LOG_TRACE(log, "Invalid cached directory:{}", local_path_.string()); + LOG_TRACE(log, "Invalid cached directory: {}", local_path_.string()); return nullptr; } auto cache_controller = std::make_shared(nullptr, local_path_, 0); - if (!cache_controller->loadInnerInformation(local_path_ / "info.txt") - || cache_controller->file_status != DOWNLOADED) + if (cache_controller->file_status != DOWNLOADED) { LOG_INFO(log, "Recover cached file failed. local path:{}", local_path_.string()); return nullptr; @@ -67,12 +54,11 @@ std::shared_ptr RemoteCacheController::recover(const std: local_path_.string()); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid metadata class:{}", cache_controller->metadata_class); } - std::ifstream metadata_file(local_path_ / "metadata.txt"); - if (!cache_controller->file_metadata_ptr->fromString(std::string((std::istreambuf_iterator(metadata_file)), - std::istreambuf_iterator()))) + ReadBufferFromFile file_readbuffer((local_path_ / "metadata.txt").string()); + std::string metadata_content; + readStringUntilEOF(metadata_content, file_readbuffer); + if (!cache_controller->file_metadata_ptr->fromString(metadata_content)) { - LOG_ERROR(log, "Cannot load the metadata. The cached file is invalid and will be remove. path:{}", - local_path_.string()); throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid metadata file({}) for meta class {}", local_path_.string(), cache_controller->metadata_class); } @@ -93,7 +79,7 @@ RemoteCacheController::RemoteCacheController( , current_offset(0) { // on recover, file_metadata_ptr is null, but it will be allocated after loading from metadata.txt - // when we allocate a whole new file cache , file_metadata_ptr must not be null. + // when we allocate a whole new file cache,file_metadata_ptr must not be null. if (file_metadata_ptr) { metadata_class = file_metadata_ptr->getName(); @@ -102,9 +88,22 @@ RemoteCacheController::RemoteCacheController( metadata_file_writer->write(str_buf.c_str(), str_buf.size()); metadata_file_writer->close(); } + else + { + auto info_path = local_path_ / "info.txt"; + if (fs::exists(info_path)) + { + std::ifstream info_file(info_path); + Poco::JSON::Parser info_parser; + auto info_json = info_parser.parse(info_file).extract(); + file_status = static_cast(info_json->get("file_status").convert()); + metadata_class = info_json->get("metadata_class").convert(); + info_file.close(); + } + } } -ErrorCodes::ErrorCode RemoteCacheController::waitMoreData(size_t start_offset_, size_t end_offset_) +void RemoteCacheController::waitMoreData(size_t start_offset_, size_t end_offset_) { std::unique_lock lock{mutex}; if (file_status == DOWNLOADED) @@ -113,7 +112,7 @@ ErrorCodes::ErrorCode RemoteCacheController::waitMoreData(size_t start_offset_, if (start_offset_ >= current_offset) { lock.unlock(); - return ErrorCodes::END_OF_FILE; + return; } } else // block until more data is ready @@ -121,18 +120,17 @@ ErrorCodes::ErrorCode RemoteCacheController::waitMoreData(size_t start_offset_, if (current_offset >= end_offset_) { lock.unlock(); - return ErrorCodes::OK; + return; } else more_data_signal.wait(lock, [this, end_offset_] { return file_status == DOWNLOADED || current_offset >= end_offset_; }); } lock.unlock(); - return ErrorCodes::OK; } bool RemoteCacheController::isModified(IRemoteFileMetadataPtr file_metadata_) { - return !(file_metadata_ptr->getVersion() == file_metadata_->getVersion()); + return file_metadata_ptr->getVersion() != file_metadata_->getVersion(); } void RemoteCacheController::startBackgroundDownload(std::unique_ptr in_readbuffer_, BackgroundSchedulePool & thread_pool) diff --git a/src/Storages/Cache/RemoteCacheController.h b/src/Storages/Cache/RemoteCacheController.h index 19dd49d5182..c5dc4a510b6 100644 --- a/src/Storages/Cache/RemoteCacheController.h +++ b/src/Storages/Cache/RemoteCacheController.h @@ -58,7 +58,7 @@ public: * enough data be downloaded. * If the file has finished download, the process would unblocked */ - ErrorCodes::ErrorCode waitMoreData(size_t start_offset_, size_t end_offset_); + void waitMoreData(size_t start_offset_, size_t end_offset_); inline size_t size() const { return current_offset; } @@ -85,7 +85,6 @@ public: private: // flush file and status information void flush(bool need_flush_status = false); - bool loadInnerInformation(const std::filesystem::path & file_path); BackgroundSchedulePool::TaskHolder download_task_holder; void backgroundDownload(ReadBufferPtr remote_read_buffer); diff --git a/src/Storages/Cache/RemoteFileCachePolicy.h b/src/Storages/Cache/RemoteFileCachePolicy.h index 4866247ee6e..ebdb2c4bc48 100644 --- a/src/Storages/Cache/RemoteFileCachePolicy.h +++ b/src/Storages/Cache/RemoteFileCachePolicy.h @@ -13,7 +13,7 @@ struct RemoteFileCacheEvictPolicy { bool canRelease(std::shared_ptr cache) const { - return (!cache || cache->closable()); + return !cache || cache->closable(); } void release(std::shared_ptr cache) { diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index f4536c07d93..af357e13ca7 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -161,9 +161,14 @@ public: /// Use local cache for remote storage if enabled. std::unique_ptr remote_read_buf; if (ExternalDataSourceCache::instance().isInitialized() && getContext()->getSettingsRef().use_local_cache_for_remote_storage) + { + size_t buff_size = raw_read_buf->internalBuffer().size(); + if (buff_size == 0) + buff_size = DBMS_DEFAULT_BUFFER_SIZE; remote_read_buf = RemoteReadBuffer::create(getContext(), std::make_shared("Hive", getNameNodeCluster(hdfs_namenode_url), uri_with_path, curr_file->getSize(), curr_file->getLastModTs()), - std::move(raw_read_buf)); + std::move(raw_read_buf), buff_size); + } else remote_read_buf = std::move(raw_read_buf); From 2d6eb40879cb6c3f30a8201876b8fa669286b6d5 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 27 Dec 2021 15:31:24 +0800 Subject: [PATCH 0405/1260] fix style --- src/Common/LRUCache.h | 2 +- .../Cache/ExternalDataSourceCache.cpp | 1 - src/Storages/Cache/RemoteCacheController.cpp | 3 +-- src/Storages/Hive/StorageHiveMetadata.h | 21 ++++++++++++------- 4 files changed, 15 insertions(+), 12 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 2ca5e007716..8a38ab09ed0 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -50,7 +50,7 @@ public: using Key = TKey; using Mapped = TMapped; using MappedPtr = std::shared_ptr; - + struct Result { MappedPtr value; diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp index f8d5c95f859..1182bc2c6b0 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.cpp +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -14,7 +14,6 @@ #include #include -namespace fs = std::filesystem; namespace ProfileEvents { extern const Event ExternalDataSourceLocalCacheReadBytes; diff --git a/src/Storages/Cache/RemoteCacheController.cpp b/src/Storages/Cache/RemoteCacheController.cpp index 0cee2733bee..01394120cb8 100644 --- a/src/Storages/Cache/RemoteCacheController.cpp +++ b/src/Storages/Cache/RemoteCacheController.cpp @@ -14,7 +14,6 @@ namespace DB namespace fs = std::filesystem; namespace ErrorCodes { - extern const int OK; extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; } @@ -39,7 +38,7 @@ std::shared_ptr RemoteCacheController::recover(const std: { cache_controller->file_metadata_ptr = RemoteFileMetadataFactory::instance().get(cache_controller->metadata_class); } - catch(const Exception & e) + catch (const Exception & e) { LOG_ERROR(log, "Get metadata class failed for {}. {}", cache_controller->metadata_class, e.message()); cache_controller->file_metadata_ptr = nullptr; diff --git a/src/Storages/Hive/StorageHiveMetadata.h b/src/Storages/Hive/StorageHiveMetadata.h index 7cdbac9a9c8..3400f1324f6 100644 --- a/src/Storages/Hive/StorageHiveMetadata.h +++ b/src/Storages/Hive/StorageHiveMetadata.h @@ -6,15 +6,20 @@ class StorageHiveMetadata : public IRemoteFileMetadata { public: StorageHiveMetadata() = default; - StorageHiveMetadata(const String & schema_, - const String & cluster_, - const String & remote_path_, - size_t file_size_, - UInt64 last_modification_timestamp_) : schema(schema_), cluster(cluster_){ - remote_path = remote_path_; - file_size = file_size_; - last_modification_timestamp = last_modification_timestamp_; + + explicit StorageHiveMetadata( + const String & schema_, + const String & cluster_, + const String & remote_path_, + size_t file_size_, + UInt64 last_modification_timestamp_) + : schema(schema_), cluster(cluster_) + { + remote_path = remote_path_; + file_size = file_size_; + last_modification_timestamp = last_modification_timestamp_; } + ~StorageHiveMetadata() override; String getName() const override { return "StorageHiveMetadata"; } From e683fa146bb9a509523a6bc7f117c552e1ce09fd Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 27 Dec 2021 10:33:25 +0300 Subject: [PATCH 0406/1260] More debug info in jepsen check --- tests/ci/keeper_jepsen_check.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/ci/keeper_jepsen_check.py b/tests/ci/keeper_jepsen_check.py index 5c7582242a9..0def2fee26f 100644 --- a/tests/ci/keeper_jepsen_check.py +++ b/tests/ci/keeper_jepsen_check.py @@ -120,6 +120,8 @@ if __name__ == "__main__": pr_info = PRInfo() + logging.info("Start at PR number %s, commit sha %s", pr_info.number, pr_info.sha) + if pr_info.number != 0 and 'jepsen-test' not in pr_info.labels(): logging.info("Not jepsen test label in labels list, skipping") sys.exit(0) From 342c8231fa7d38b1a799e3b792c79be9881015cc Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 27 Dec 2021 10:33:44 +0300 Subject: [PATCH 0407/1260] Better --- tests/ci/keeper_jepsen_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/keeper_jepsen_check.py b/tests/ci/keeper_jepsen_check.py index 0def2fee26f..2c2b8b4783f 100644 --- a/tests/ci/keeper_jepsen_check.py +++ b/tests/ci/keeper_jepsen_check.py @@ -120,7 +120,7 @@ if __name__ == "__main__": pr_info = PRInfo() - logging.info("Start at PR number %s, commit sha %s", pr_info.number, pr_info.sha) + logging.info("Start at PR number %s, commit sha %s labels %s", pr_info.number, pr_info.sha, pr_info.labels) if pr_info.number != 0 and 'jepsen-test' not in pr_info.labels(): logging.info("Not jepsen test label in labels list, skipping") From 0a39273feed0537e27482eb387e1f2d943c7faca Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 27 Dec 2021 15:31:04 +0800 Subject: [PATCH 0408/1260] fixed code style --- src/Common/LRUCache.h | 1 - src/Storages/Cache/ExternalDataSourceCache.cpp | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 8a38ab09ed0..4249d9de478 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -50,7 +50,6 @@ public: using Key = TKey; using Mapped = TMapped; using MappedPtr = std::shared_ptr; - struct Result { MappedPtr value; diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp index 1182bc2c6b0..8744a19fdca 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.cpp +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include From 1bd493696148ab5985fe7edf082e35063b96f1e3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 Dec 2021 11:03:04 +0300 Subject: [PATCH 0409/1260] Add retry for postgres query --- src/Core/PostgreSQL/Connection.cpp | 12 +++---- src/Core/PostgreSQL/Connection.h | 31 +++++++++++++------ src/Core/PostgreSQL/ConnectionHolder.h | 10 ++++-- src/Core/PostgreSQL/PoolWithFailover.cpp | 20 ++++++------ src/Core/PostgreSQL/PoolWithFailover.h | 7 ++--- src/Core/PostgreSQL/Utils.cpp | 2 +- .../Transforms/PostgreSQLSource.cpp | 12 ++++++- .../TableFunctionPostgreSQL.cpp | 1 + 8 files changed, 62 insertions(+), 33 deletions(-) diff --git a/src/Core/PostgreSQL/Connection.cpp b/src/Core/PostgreSQL/Connection.cpp index 75786a51d92..f97a35a9e92 100644 --- a/src/Core/PostgreSQL/Connection.cpp +++ b/src/Core/PostgreSQL/Connection.cpp @@ -12,10 +12,7 @@ Connection::Connection(const ConnectionInfo & connection_info_, bool replication , log(&Poco::Logger::get("PostgreSQLReplicaConnection")) { if (replication) - { - connection_info = std::make_pair( - fmt::format("{} replication=database", connection_info.first), connection_info.second); - } + connection_info = {fmt::format("{} replication=database", connection_info.connection_string), connection_info.host_port}; } void Connection::execWithRetry(const std::function & exec) @@ -61,11 +58,14 @@ void Connection::updateConnection() { if (connection) connection->close(); + /// Always throws if there is no connection. - connection = std::make_unique(connection_info.first); + connection = std::make_unique(connection_info.connection_string); + if (replication) connection->set_variable("default_transaction_isolation", "'repeatable read'"); - LOG_DEBUG(&Poco::Logger::get("PostgreSQLConnection"), "New connection to {}", connection_info.second); + + LOG_DEBUG(&Poco::Logger::get("PostgreSQLConnection"), "New connection to {}", connection_info.host_port); } void Connection::connect() diff --git a/src/Core/PostgreSQL/Connection.h b/src/Core/PostgreSQL/Connection.h index d65c38643c1..8c5609dc66b 100644 --- a/src/Core/PostgreSQL/Connection.h +++ b/src/Core/PostgreSQL/Connection.h @@ -8,19 +8,26 @@ #include #include -/* Methods to work with PostgreSQL connection object. +/** Methods to work with PostgreSQL connection object. * Should only be used in case there has to be a single connection object, which * is long-lived and there are no concurrent connection queries. - * Now only use case - for replication handler for replication from PostgreSQL. - * In all other integration engine use pool with failover. - **/ + */ namespace Poco { class Logger; } +namespace pqxx +{ + using ConnectionPtr = std::unique_ptr; +} + namespace postgres { -using ConnectionInfo = std::pair; -using ConnectionPtr = std::unique_ptr; + +struct ConnectionInfo +{ + String connection_string; + String host_port; /// For logs. +}; class Connection : private boost::noncopyable { @@ -33,14 +40,17 @@ public: void connect(); + void updateConnection(); + void tryUpdateConnection(); const ConnectionInfo & getConnectionInfo() { return connection_info; } -private: - void updateConnection(); + String getInfoForLog() const { return connection_info.host_port; } - ConnectionPtr connection; +private: + + pqxx::ConnectionPtr connection; ConnectionInfo connection_info; bool replication; @@ -48,6 +58,9 @@ private: Poco::Logger * log; }; + +using ConnectionPtr = std::unique_ptr; + } #endif diff --git a/src/Core/PostgreSQL/ConnectionHolder.h b/src/Core/PostgreSQL/ConnectionHolder.h index d0d64935e91..38e321e222c 100644 --- a/src/Core/PostgreSQL/ConnectionHolder.h +++ b/src/Core/PostgreSQL/ConnectionHolder.h @@ -7,12 +7,12 @@ #include #include #include +#include "Connection.h" namespace postgres { -using ConnectionPtr = std::unique_ptr; using Pool = BorrowedObjectPool; using PoolPtr = std::shared_ptr; @@ -28,8 +28,12 @@ public: pqxx::connection & get() { - assert(connection != nullptr); - return *connection; + return connection->getRef(); + } + + void update() + { + connection->updateConnection(); } private: diff --git a/src/Core/PostgreSQL/PoolWithFailover.cpp b/src/Core/PostgreSQL/PoolWithFailover.cpp index 3addb511c3b..844c60087e0 100644 --- a/src/Core/PostgreSQL/PoolWithFailover.cpp +++ b/src/Core/PostgreSQL/PoolWithFailover.cpp @@ -32,9 +32,9 @@ PoolWithFailover::PoolWithFailover( { for (const auto & replica_configuration : configurations) { - auto connection_string = formatConnectionString(replica_configuration.database, - replica_configuration.host, replica_configuration.port, replica_configuration.username, replica_configuration.password).first; - replicas_with_priority[priority].emplace_back(connection_string, pool_size, getConnectionForLog(replica_configuration.host, replica_configuration.port)); + auto connection_info = formatConnectionString(replica_configuration.database, + replica_configuration.host, replica_configuration.port, replica_configuration.username, replica_configuration.password); + replicas_with_priority[priority].emplace_back(connection_info, pool_size); } } } @@ -52,8 +52,8 @@ PoolWithFailover::PoolWithFailover( for (const auto & [host, port] : configuration.addresses) { LOG_DEBUG(&Poco::Logger::get("PostgreSQLPoolWithFailover"), "Adding address host: {}, port: {} to connection pool", host, port); - auto connection_string = formatConnectionString(configuration.database, host, port, configuration.username, configuration.password).first; - replicas_with_priority[0].emplace_back(connection_string, pool_size, getConnectionForLog(host, port)); + auto connection_string = formatConnectionString(configuration.database, host, port, configuration.username, configuration.password); + replicas_with_priority[0].emplace_back(connection_string, pool_size); } } @@ -83,16 +83,18 @@ ConnectionHolderPtr PoolWithFailover::get() try { /// Create a new connection or reopen an old connection if it became invalid. - if (!connection || !connection->is_open()) + if (!connection) { - connection = std::make_unique(replica.connection_string); - LOG_DEBUG(log, "New connection to {}:{}", connection->hostname(), connection->port()); + connection = std::make_unique(replica.connection_info); + LOG_DEBUG(log, "New connection to {}", connection->getInfoForLog()); } + + connection->connect(); } catch (const pqxx::broken_connection & pqxx_error) { LOG_ERROR(log, "Connection error: {}", pqxx_error.what()); - error_message << "Try " << try_idx + 1 << ". Connection to `" << replica.name_for_log << "` failed: " << pqxx_error.what() << "\n"; + error_message << "Try " << try_idx + 1 << ". Connection to `" << replica.connection_info.host_port << "` failed: " << pqxx_error.what() << "\n"; replica.pool->returnObject(std::move(connection)); continue; diff --git a/src/Core/PostgreSQL/PoolWithFailover.h b/src/Core/PostgreSQL/PoolWithFailover.h index c59010a5d43..e6f691ed2dd 100644 --- a/src/Core/PostgreSQL/PoolWithFailover.h +++ b/src/Core/PostgreSQL/PoolWithFailover.h @@ -44,12 +44,11 @@ public: private: struct PoolHolder { - String connection_string; + ConnectionInfo connection_info; PoolPtr pool; - String name_for_log; - PoolHolder(const String & connection_string_, size_t pool_size, const String & name_for_log_) - : connection_string(connection_string_), pool(std::make_shared(pool_size)), name_for_log(name_for_log_) {} + PoolHolder(const ConnectionInfo & connection_info_, size_t pool_size) + : connection_info(connection_info_), pool(std::make_shared(pool_size)) {} }; /// Highest priority is 0, the bigger the number in map, the less the priority diff --git a/src/Core/PostgreSQL/Utils.cpp b/src/Core/PostgreSQL/Utils.cpp index 60b13218202..b4ad19c819a 100644 --- a/src/Core/PostgreSQL/Utils.cpp +++ b/src/Core/PostgreSQL/Utils.cpp @@ -17,7 +17,7 @@ ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, S << " user=" << DB::quote << user << " password=" << DB::quote << password << " connect_timeout=10"; - return std::make_pair(out.str(), host + ':' + DB::toString(port)); + return {out.str(), host + ':' + DB::toString(port)}; } String getConnectionForLog(const String & host, UInt16 port) diff --git a/src/Processors/Transforms/PostgreSQLSource.cpp b/src/Processors/Transforms/PostgreSQLSource.cpp index ac8408d8338..88f092a2533 100644 --- a/src/Processors/Transforms/PostgreSQLSource.cpp +++ b/src/Processors/Transforms/PostgreSQLSource.cpp @@ -74,7 +74,17 @@ template void PostgreSQLSource::onStart() { if (!tx) - tx = std::make_shared(connection_holder->get()); + { + try + { + tx = std::make_shared(connection_holder->get()); + } + catch (const pqxx::broken_connection &) + { + connection_holder->update(); + tx = std::make_shared(connection_holder->get()); + } + } stream = std::make_unique(*tx, pqxx::from_query, std::string_view(query_str)); } diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index bcfe8d5444c..d948f40588f 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -50,6 +50,7 @@ ColumnsDescription TableFunctionPostgreSQL::getActualTableStructure(ContextPtr c if (!columns) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table structure not returned"); + return ColumnsDescription{*columns}; } From ef1d7142f52f88a0d5a996ba0b669de0579d79f3 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 27 Dec 2021 16:12:39 +0800 Subject: [PATCH 0410/1260] remove getOrTrySet --- src/Common/LRUCache.h | 15 +++++---------- src/IO/MMappedFileCache.h | 4 ++-- src/IO/UncompressedCache.h | 4 ++-- src/Interpreters/Aggregator.cpp | 4 ++-- src/Interpreters/ExpressionJIT.cpp | 3 ++- src/Interpreters/MergeJoin.cpp | 2 +- .../Formats/Impl/AvroRowInputFormat.cpp | 4 ++-- src/Storages/MarkCache.h | 4 ++-- 8 files changed, 18 insertions(+), 22 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 2ca5e007716..f782812b04d 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -50,12 +50,14 @@ public: using Key = TKey; using Mapped = TMapped; using MappedPtr = std::shared_ptr; - + struct Result { MappedPtr value; + // if key is in cache, cache_miss is true bool cache_miss = true; - // set_successful is not trustworthy for getOrSet, because removeOverflow is called right after putting key in cache + // set_successful is false in default + // when value is loaded by load_fun in getOrSet(), and setImpl returns true, set_successful = true bool set_successful = false; }; @@ -97,13 +99,6 @@ public: return setImpl(key, mapped, lock); } - template - std::pair getOrSet(const Key & key, LoadFunc && load_func) - { - auto result = getOrTrySet(key, std::move(load_func)); - return std::make_pair(result.value, result.cache_miss); - } - /// If the value for the key is in the cache, returns it. If it is not, calls load_func() to /// produce it, saves the result in the cache and returns it. /// Only one of several concurrent threads calling getOrTrySet() will call load_func(), @@ -112,7 +107,7 @@ public: /// set of concurrent threads will then try to call its load_func etc. /// template - Result getOrTrySet(const Key &key, LoadFunc && load_func) + Result getOrSet(const Key &key, LoadFunc && load_func) { InsertTokenHolder token_holder; { diff --git a/src/IO/MMappedFileCache.h b/src/IO/MMappedFileCache.h index adbb85a18cf..7ee6957c7db 100644 --- a/src/IO/MMappedFileCache.h +++ b/src/IO/MMappedFileCache.h @@ -48,12 +48,12 @@ public: MappedPtr getOrSet(const Key & key, LoadFunc && load) { auto result = Base::getOrSet(key, load); - if (result.second) + if (result.cache_miss) ProfileEvents::increment(ProfileEvents::MMappedFileCacheMisses); else ProfileEvents::increment(ProfileEvents::MMappedFileCacheHits); - return result.first; + return result.value; } }; diff --git a/src/IO/UncompressedCache.h b/src/IO/UncompressedCache.h index 5826b7f020a..78f81c15a4a 100644 --- a/src/IO/UncompressedCache.h +++ b/src/IO/UncompressedCache.h @@ -63,12 +63,12 @@ public: { auto result = Base::getOrSet(key, std::forward(load)); - if (result.second) + if (result.cache_miss) ProfileEvents::increment(ProfileEvents::UncompressedCacheMisses); else ProfileEvents::increment(ProfileEvents::UncompressedCacheHits); - return result.first; + return result.value; } private: diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index ae5ce117c61..95341efa76a 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -354,14 +354,14 @@ void Aggregator::compileAggregateFunctionsIfNeeded() if (auto * compilation_cache = CompiledExpressionCacheFactory::instance().tryGetCache()) { - auto [compiled_function_cache_entry, _] = compilation_cache->getOrSet(aggregate_functions_description_hash_key, [&] () + auto result = compilation_cache->getOrSet(aggregate_functions_description_hash_key, [&] () { LOG_TRACE(log, "Compile expression {}", functions_description); auto compiled_aggregate_functions = compileAggregateFunctions(getJITInstance(), functions_to_compile, functions_description); return std::make_shared(std::move(compiled_aggregate_functions)); }); - + auto compiled_function_cache_entry = result.value; compiled_aggregate_functions_holder = std::static_pointer_cast(compiled_function_cache_entry); } else diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index 90292d17fae..d5017b18dc1 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -296,12 +296,13 @@ static FunctionBasePtr compile( if (auto * compilation_cache = CompiledExpressionCacheFactory::instance().tryGetCache()) { - auto [compiled_function_cache_entry, _] = compilation_cache->getOrSet(hash_key, [&] () + auto result = compilation_cache->getOrSet(hash_key, [&] () { LOG_TRACE(getLogger(), "Compile expression {}", llvm_function->getName()); auto compiled_function = compileFunction(getJITInstance(), *llvm_function); return std::make_shared(compiled_function); }); + auto compiled_function_cache_entry = result.value; std::shared_ptr compiled_function_holder = std::static_pointer_cast(compiled_function_cache_entry); llvm_function->setCompiledFunction(std::move(compiled_function_holder)); diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 7f22386f54b..d5ea1682dff 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -1033,7 +1033,7 @@ std::shared_ptr MergeJoin::loadRightBlock(size_t pos) const return std::make_shared(input.block_in->read()); }; - return cached_right_blocks->getOrSet(pos, load_func).first; + return cached_right_blocks->getOrSet(pos, load_func).value; } else return loaded_right_blocks[pos]; diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 11e56ecbe0c..c415f27d202 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -650,7 +650,7 @@ public: avro::ValidSchema getSchema(uint32_t id) { - auto [schema, loaded] = schema_cache.getOrSet( + auto [schema, loaded, _] = schema_cache.getOrSet( id, [this, id](){ return std::make_shared(fetchSchema(id)); } ); @@ -727,7 +727,7 @@ static LRUCache schema_registry_cache(SCH static std::shared_ptr getConfluentSchemaRegistry(const FormatSettings & format_settings) { const auto & base_url = format_settings.avro.schema_registry_url; - auto [schema_registry, loaded] = schema_registry_cache.getOrSet( + auto [schema_registry, loaded, _] = schema_registry_cache.getOrSet( base_url, [base_url]() { diff --git a/src/Storages/MarkCache.h b/src/Storages/MarkCache.h index 06143e954f8..3438b4a1b9b 100644 --- a/src/Storages/MarkCache.h +++ b/src/Storages/MarkCache.h @@ -59,12 +59,12 @@ public: MappedPtr getOrSet(const Key & key, LoadFunc && load) { auto result = Base::getOrSet(key, load); - if (result.second) + if (result.cache_miss) ProfileEvents::increment(ProfileEvents::MarkCacheMisses); else ProfileEvents::increment(ProfileEvents::MarkCacheHits); - return result.first; + return result.value; } }; From 3367755dcc71ea1c0d4159193fb0ef00b4989ff6 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 27 Dec 2021 16:21:10 +0800 Subject: [PATCH 0411/1260] update comment --- src/Common/LRUCache.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index f782812b04d..049d07c74f4 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -101,7 +101,7 @@ public: /// If the value for the key is in the cache, returns it. If it is not, calls load_func() to /// produce it, saves the result in the cache and returns it. - /// Only one of several concurrent threads calling getOrTrySet() will call load_func(), + /// Only one of several concurrent threads calling getOrSet() will call load_func(), /// others will wait for that call to complete and will use its result (this helps prevent cache stampede). /// Exceptions occurring in load_func will be propagated to the caller. Another thread from the /// set of concurrent threads will then try to call its load_func etc. From 6324d2fb31931cc79f35d789aad808f0219756d5 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 27 Dec 2021 11:25:37 +0300 Subject: [PATCH 0412/1260] add i18n symlinks --- docs/ja/faq/general/how-do-i-contribute-code-to-clickhouse.md | 1 + docs/ru/faq/general/how-do-i-contribute-code-to-clickhouse.md | 1 + docs/zh/faq/general/how-do-i-contribute-code-to-clickhouse.md | 1 + 3 files changed, 3 insertions(+) create mode 120000 docs/ja/faq/general/how-do-i-contribute-code-to-clickhouse.md create mode 120000 docs/ru/faq/general/how-do-i-contribute-code-to-clickhouse.md create mode 120000 docs/zh/faq/general/how-do-i-contribute-code-to-clickhouse.md diff --git a/docs/ja/faq/general/how-do-i-contribute-code-to-clickhouse.md b/docs/ja/faq/general/how-do-i-contribute-code-to-clickhouse.md new file mode 120000 index 00000000000..5ac9a615386 --- /dev/null +++ b/docs/ja/faq/general/how-do-i-contribute-code-to-clickhouse.md @@ -0,0 +1 @@ +../../../en/faq/general/how-do-i-contribute-code-to-clickhouse.md \ No newline at end of file diff --git a/docs/ru/faq/general/how-do-i-contribute-code-to-clickhouse.md b/docs/ru/faq/general/how-do-i-contribute-code-to-clickhouse.md new file mode 120000 index 00000000000..5ac9a615386 --- /dev/null +++ b/docs/ru/faq/general/how-do-i-contribute-code-to-clickhouse.md @@ -0,0 +1 @@ +../../../en/faq/general/how-do-i-contribute-code-to-clickhouse.md \ No newline at end of file diff --git a/docs/zh/faq/general/how-do-i-contribute-code-to-clickhouse.md b/docs/zh/faq/general/how-do-i-contribute-code-to-clickhouse.md new file mode 120000 index 00000000000..5ac9a615386 --- /dev/null +++ b/docs/zh/faq/general/how-do-i-contribute-code-to-clickhouse.md @@ -0,0 +1 @@ +../../../en/faq/general/how-do-i-contribute-code-to-clickhouse.md \ No newline at end of file From 46dc07310f83b0f3ba10165c95077224f541cfd5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Dec 2021 11:52:17 +0300 Subject: [PATCH 0413/1260] Add a blog post about flaky tests --- utils/tests-visualizer/index.html | 8 ---- website/blog/en/2021/tests-visualization.md | 45 +++++++++++++++++++++ 2 files changed, 45 insertions(+), 8 deletions(-) create mode 100644 website/blog/en/2021/tests-visualization.md diff --git a/utils/tests-visualizer/index.html b/utils/tests-visualizer/index.html index 690c42e486e..a15b09ea58e 100644 --- a/utils/tests-visualizer/index.html +++ b/utils/tests-visualizer/index.html @@ -69,12 +69,6 @@ function renderResponse(response) { document.body.style.height = canvas.height + 10 + 'px'; let ctx = canvas.getContext('2d'); - - ctx.imageSmoothingEnabled = false; - ctx.mozImageSmoothingEnabled = false; - ctx.webkitImageSmoothingEnabled = false; - ctx.msImageSmoothingEnabled = false; - let image = ctx.createImageData(canvas.width, canvas.height); let pixels = image.data; @@ -123,8 +117,6 @@ canvas.addEventListener('mousemove', event => { let pixel = canvas.getContext('2d').getImageData(x, y, 1, 1).data; - console.log(pixel); - let info = document.getElementById('info'); info.innerText = `${date}, ${test}`; diff --git a/website/blog/en/2021/tests-visualization.md b/website/blog/en/2021/tests-visualization.md new file mode 100644 index 00000000000..b540f275bd9 --- /dev/null +++ b/website/blog/en/2021/tests-visualization.md @@ -0,0 +1,45 @@ +--- +title: 'Decorating a Christmas Tree With the Help Of Flaky Tests' +image: 'https://blog-images.clickhouse.com/en/2021/tests-visualization/tests.png' +date: '2021-12-27' +author: '[Alexey Milovidov](https://github.com/alexey-milovidov)' +tags: ['tests', 'ci', 'flaky', 'christmas', 'visualization'] +--- + +Test suites and testing infrastructure are one of the main assets of ClickHouse. We have tons of functional, integration, unit, performance, stress and fuzz tests. Tests are run on a per commit basis and results are publicly available. + +We also save the results of all test runs into the database in ClickHouse. We started collecting results in June 2020, and we have 1 777 608 240 records so far. Now we run around 5 to 9 million tests every day. + +Tests are good (in general). Good test suite allows fast development iterations, stable releases, accept more contributions from the community. We love tests. If there's something strange in ClickHouse, what are we gonna do? Write more tests. + +Some tests can be flaky. The reasons for flakiness are uncountable - most of them are simple timing issues in the test script itself, but sometimes if a test has failed one of a thousand times it can uncover subtle logic errors in code. + +The problem is how to deal with flaky tests. Some people may suggest to automatically mute the "annoying" flaky tests. Or to add automatic retries in case of failure. In my opinion, this is all wrong. Instead of trying to ignore flaky tests, we do the opposite: we do maximum efforts to make the tests even more flaky! + +Our recipes for flaky tests: +— never mute or restart them; if the test failed once, always look and investigate the cause; +— randomize the environment for every test run so the test will have more possible reasons to fail; +— if new tests have added, run them 100 times and if at least once they fail, do not merge the pull request; +— if new tests have added, use them as a corpus for fuzzing - it will uncover corner cases even if author did not write tests for them; +— [randomize thread scheduling](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/ThreadFuzzer.h) and add random sleeps and switching between CPU cores at random places and before and after mutex locks/unlocks; +— run everything in parallel on slow machines; + +Key point: to prevent flaky tests, we make our tests as flaky as possible. + +## Nice Way To Visualize Flaky Tests + +There is a test suite named "[functional stateless tests](https://github.com/ClickHouse/ClickHouse/tree/master/tests/queries/0_stateless)", it has 3772 tests. Let's draw test results as a picture. For every day since 2020-06-13 (561 days) and every test (3772 tests) make a picture of size 561x3772 where every pixel is green if all test runs finished successfully in master branch during this day (for all commits and all combinations: release, debug+assertions, ASan, MSan, TSan, UBSan); red if at least one run has failed; transparent if the test did not exist that day. + +This visualization is a toy that I've made for fun: + +![Visualization](https://blog-images.clickhouse.com/en/2021/tests-visualization/tree_half.png) + +It looks like a Christmas Tree (you need a bit of imagination). If you have a different kind of imagination, you can see it as a green field with flowers. + +The time is from left to right. The tests are numbered with non-unique numbers (new tests usually get larger numbers), and these numbers are on the vertical axis (newer tests on top). + +If you see red dots in a horizontal line - it is a flaky test. If you see red dots in a vertical line - it means that one day we accidentally broke the master branch. If you see black horizontal lines or cuts in the tree - it means that the tests were added with some old numbers, most likely because some long living feature branch was merged. If you see black vertical lines - it means that some days tests were not run. + +The velocity of adding new tests is represented by how tall and narrow the Christmas tree is. When we add a large number of tests, the tree grows with almost vertical slope. + +The image is prepared by [HTML page](https://github.com/ClickHouse/ClickHouse/pull/33185) with some JavaScript that is querying ClickHouse database directly and writing to a canvas. It took around ten seconds to build this picture. I also prepared [interactive version](https://blog-images.clickhouse.com/en/2021/tests-visualization/demo.html) with already saved data where you can play and find your favorite tests. From a1aab3a82d61e2b228b5ec500ab4ed1c95d72236 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 Dec 2021 11:55:29 +0300 Subject: [PATCH 0414/1260] Fix --- .../transformQueryForExternalDatabase.cpp | 12 ++++++++++++ tests/integration/test_storage_postgresql/test.py | 15 +++++++++++++++ 2 files changed, 27 insertions(+) diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 4d6c1787a34..c42fb7fa965 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -306,6 +306,18 @@ String transformQueryForExternalDatabase( throw Exception("Query contains non-compatible expressions (and external_table_strict_query=true)", ErrorCodes::INCORRECT_QUERY); } + auto * literal_expr = typeid_cast(original_where.get()); + UInt64 value; + if (literal_expr && literal_expr->value.tryGet(value) && (value == 0 || value == 1)) + { + /// WHERE 1 -> WHERE 1=1, WHERE 0 -> WHERE 1=0. + if (value) + original_where = makeASTFunction("equals", std::make_shared(1), std::make_shared(1)); + else + original_where = makeASTFunction("equals", std::make_shared(1), std::make_shared(0)); + select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(original_where)); + } + ASTPtr select_ptr = select; dropAliases(select_ptr); diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 6f43036e64d..b6ac121cd0c 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -424,6 +424,21 @@ def test_predefined_connection_configuration(started_cluster): cursor.execute(f'DROP TABLE test_table ') +def test_where_false(started_cluster): + cursor = started_cluster.postgres_conn.cursor() + cursor.execute("DROP TABLE IF EXISTS test") + cursor.execute('CREATE TABLE test (a Integer)') + cursor.execute("INSERT INTO test SELECT 1") + + result = node1.query("SELECT count() FROM postgresql('postgres1:5432', 'postgres', 'test', 'postgres', 'mysecretpassword') WHERE 1=0") + assert(int(result) == 0) + result = node1.query("SELECT count() FROM postgresql('postgres1:5432', 'postgres', 'test', 'postgres', 'mysecretpassword') WHERE 0") + assert(int(result) == 0) + result = node1.query("SELECT count() FROM postgresql('postgres1:5432', 'postgres', 'test', 'postgres', 'mysecretpassword') WHERE 1=1") + assert(int(result) == 1) + cursor.execute("DROP TABLE test") + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From 593dc6cf9050a136e3ee75b6f574873400cc9d0d Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 27 Dec 2021 12:24:19 +0300 Subject: [PATCH 0415/1260] Fix some flaky integration tests --- tests/integration/helpers/cluster.py | 12 +++++++++ .../test.py | 25 +++++++++++-------- .../test_jemalloc_percpu_arena/test.py | 9 +++++-- .../test_parts_delete_zookeeper/test.py | 10 +++++--- 4 files changed, 40 insertions(+), 16 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index d440f2de0ca..db206bac05f 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1476,6 +1476,18 @@ class ClickHouseCluster: common_opts = ['--verbose', 'up', '-d'] + images_pull_cmd = self.base_cmd + ['pull'] + # sometimes dockerhub/proxy can be flaky + for i in range(5): + try: + run_and_check(images_pull_cmd) + break + except Exception as ex: + if i == 4: + raise ex + logging.info("Got exception pulling images: %s", ex) + time.sleep(i * 3) + if self.with_zookeeper_secure and self.base_zookeeper_cmd: logging.debug('Setup ZooKeeper Secure') logging.debug(f'Creating internal ZooKeeper dirs: {self.zookeeper_dirs_to_create}') diff --git a/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py b/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py index bc7f32bf544..cc7a8eec4d6 100644 --- a/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py +++ b/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py @@ -26,14 +26,17 @@ def start_cluster(): # max_memory_usage_for_user cannot be used, since the memory for user accounted # correctly, only total is not def test_memory_tracking_total(): - instance.query(''' - CREATE TABLE null (row String) ENGINE=Null; - ''') - instance.exec_in_container(['bash', '-c', - 'clickhouse local -q "SELECT arrayStringConcat(arrayMap(x->toString(cityHash64(x)), range(1000)), \' \') from numbers(10000)" > data.json']) - for it in range(0, 20): - # the problem can be triggered only via HTTP, - # since clickhouse-client parses the data by itself. - assert instance.exec_in_container(['curl', '--silent', '--show-error', '--data-binary', '@data.json', - 'http://127.1:8123/?query=INSERT%20INTO%20null%20FORMAT%20TSV']) == '', 'Failed on {} iteration'.format( - it) + if instance.is_built_with_memory_sanitizer() or instance.is_built_with_thread_sanitizer() or instance.is_built_with_address_sanitizer(): + print("Server built with sanitizer and memory consumption can be unpredictable, skipping test") + else: + instance.query(''' + CREATE TABLE null (row String) ENGINE=Null; + ''') + instance.exec_in_container(['bash', '-c', + 'clickhouse local -q "SELECT arrayStringConcat(arrayMap(x->toString(cityHash64(x)), range(1000)), \' \') from numbers(10000)" > data.json']) + for it in range(0, 20): + # the problem can be triggered only via HTTP, + # since clickhouse-client parses the data by itself. + assert instance.exec_in_container(['curl', '--silent', '--show-error', '--data-binary', '@data.json', + 'http://127.1:8123/?query=INSERT%20INTO%20null%20FORMAT%20TSV']) == '', 'Failed on {} iteration'.format( + it) diff --git a/tests/integration/test_jemalloc_percpu_arena/test.py b/tests/integration/test_jemalloc_percpu_arena/test.py index 19ce520295a..cff593f82b7 100755 --- a/tests/integration/test_jemalloc_percpu_arena/test.py +++ b/tests/integration/test_jemalloc_percpu_arena/test.py @@ -48,15 +48,20 @@ def skip_if_jemalloc_disabled(): SELECT value FROM system.build_options WHERE name = 'USE_JEMALLOC'" """).strip() if output != b'ON' and output != b'1': - pytest.skip(f'Compiled w/o jemalloc (USE_JEMALLOC={output})') + return True + return False # Ensure that clickhouse works even when number of online CPUs # (_SC_NPROCESSORS_ONLN) is smaller then available (_SC_NPROCESSORS_CONF). # # Refs: https://github.com/jemalloc/jemalloc/pull/2181 def test_jemalloc_percpu_arena(): - skip_if_jemalloc_disabled() + # TODO: revert me when https://github.com/ClickHouse/ClickHouse/pull/33162 + # will be ready + if skip_if_jemalloc_disabled(): + print("Jemalloc disabled, exiting") + return assert multiprocessing.cpu_count() > CPU_ID diff --git a/tests/integration/test_parts_delete_zookeeper/test.py b/tests/integration/test_parts_delete_zookeeper/test.py index 8a4aafaa55c..62e14b68bd1 100644 --- a/tests/integration/test_parts_delete_zookeeper/test.py +++ b/tests/integration/test_parts_delete_zookeeper/test.py @@ -49,12 +49,16 @@ def test_merge_doesnt_work_without_zookeeper(start_cluster): node1.query("INSERT INTO test_table VALUES ('2018-10-01', 1), ('2018-10-02', 2), ('2018-10-03', 3)") node1.query("INSERT INTO test_table VALUES ('2018-10-01', 4), ('2018-10-02', 5), ('2018-10-03', 6)") - assert node1.query("SELECT count(*) from system.parts where table = 'test_table'") == "2\n" + assert node1.query("SELECT count(*) from system.parts where table = 'test_table' and active") == "2\n" with PartitionManager() as pm: node1.query("OPTIMIZE TABLE test_table FINAL") pm.drop_instance_zk_connections(node1) - time.sleep(10) # > old_parts_lifetime - assert node1.query("SELECT count(*) from system.parts where table = 'test_table'") == "3\n" + # unfortunately we can be too fast and delete node before partition with ZK + if node1.query("SELECT count(*) from system.parts where table = 'test_table'") == "1\n": + print("We were too fast and deleted parts before partition with ZK") + else: + time.sleep(10) # > old_parts_lifetime + assert node1.query("SELECT count(*) from system.parts where table = 'test_table'") == "3\n" assert_eq_with_retry(node1, "SELECT count(*) from system.parts where table = 'test_table' and active = 1", "1") From 1cbd05ae9e1c632d2069fff43ed69d0366f2dec9 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 27 Dec 2021 15:38:34 +0800 Subject: [PATCH 0416/1260] Correctly prevent nullable primary keys if necessary. --- src/DataTypes/hasNullable.cpp | 33 +++++++++++++++++++ src/DataTypes/hasNullable.h | 10 ++++++ src/Storages/MergeTree/MergeTreeData.cpp | 8 +++-- .../01410_nullable_key_and_index.sql | 9 +++++ 4 files changed, 57 insertions(+), 3 deletions(-) create mode 100644 src/DataTypes/hasNullable.cpp create mode 100644 src/DataTypes/hasNullable.h diff --git a/src/DataTypes/hasNullable.cpp b/src/DataTypes/hasNullable.cpp new file mode 100644 index 00000000000..2c699806874 --- /dev/null +++ b/src/DataTypes/hasNullable.cpp @@ -0,0 +1,33 @@ +#include +#include +#include +#include + +namespace DB +{ + +bool hasNullable(const DataTypePtr & type) +{ + if (type->isNullable() || type->isLowCardinalityNullable()) + return true; + + if (const DataTypeArray * type_array = typeid_cast(type.get())) + return hasNullable(type_array->getNestedType()); + else if (const DataTypeTuple * type_tuple = typeid_cast(type.get())) + { + for (const auto & subtype : type_tuple->getElements()) + { + if (hasNullable(subtype)) + return true; + } + return false; + } + else if (const DataTypeMap * type_map = typeid_cast(type.get())) + { + // Key type cannot be nullable. We only check value type. + return hasNullable(type_map->getValueType()); + } + return false; +} + +} diff --git a/src/DataTypes/hasNullable.h b/src/DataTypes/hasNullable.h new file mode 100644 index 00000000000..271803496f1 --- /dev/null +++ b/src/DataTypes/hasNullable.h @@ -0,0 +1,10 @@ +#pragma once + +#include + +namespace DB +{ + +bool hasNullable(const DataTypePtr & type); + +} diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1b7be8ca98d..ffaadb41c64 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -358,10 +359,11 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam { const ColumnPtr & column = element.column; if (column && (isColumnConst(*column) || column->isDummy())) - throw Exception{key_name + " key cannot contain constants", ErrorCodes::ILLEGAL_COLUMN}; + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "{} key cannot contain constants", key_name); - if (!allow_nullable_key && element.type->isNullable()) - throw Exception{key_name + " key cannot contain nullable columns", ErrorCodes::ILLEGAL_COLUMN}; + if (!allow_nullable_key && hasNullable(element.type)) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "{} key contains nullable columns, but `setting allow_nullable_key` is disabled", key_name); } } diff --git a/tests/queries/0_stateless/01410_nullable_key_and_index.sql b/tests/queries/0_stateless/01410_nullable_key_and_index.sql index fd1712b5d24..46a58152700 100644 --- a/tests/queries/0_stateless/01410_nullable_key_and_index.sql +++ b/tests/queries/0_stateless/01410_nullable_key_and_index.sql @@ -65,3 +65,12 @@ CREATE TABLE xxxx_null (`ts` Nullable(DateTime)) ENGINE = MergeTree ORDER BY toS INSERT INTO xxxx_null SELECT '2021-11-11 00:00:00'; SELECT * FROM xxxx_null WHERE ts > '2021-10-11 00:00:00'; DROP TABLE xxxx_null; + +-- nullable keys are forbidden when `allow_nullable_key = 0` +CREATE TABLE invalid_null (id Nullable(String)) ENGINE = MergeTree ORDER BY id; -- { serverError 44 } +CREATE TABLE invalid_lc_null (id LowCardinality(Nullable(String))) ENGINE = MergeTree ORDER BY id; -- { serverError 44 } +CREATE TABLE invalid_array_null (id Array(Nullable(String))) ENGINE = MergeTree ORDER BY id; -- { serverError 44 } +CREATE TABLE invalid_tuple_null (id Tuple(Nullable(String), UInt8)) ENGINE = MergeTree ORDER BY id; -- { serverError 44 } +CREATE TABLE invalid_map_null (id Map(UInt8, Nullable(String))) ENGINE = MergeTree ORDER BY id; -- { serverError 44 } +CREATE TABLE invalid_simple_agg_state_null (id SimpleAggregateFunction(sum, Nullable(UInt64))) ENGINE = MergeTree ORDER BY id; -- { serverError 44 } +-- AggregateFunctions are not comparable and cannot be used in key expressions. No need to test it. From 8ec4547c175676669ab9fa59ba8994ae17c3cc94 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Dec 2021 12:50:21 +0300 Subject: [PATCH 0417/1260] Remove old trash --- utils/ci/jobs/quick-build/README.md | 5 - utils/ci/jobs/quick-build/run.sh | 30 - utils/ci/vagrant-freebsd/.gitignore | 1 - utils/ci/vagrant-freebsd/Vagrantfile | 3 - utils/grammar-fuzzer/ClickHouseUnlexer.py | 1771 ---------------- utils/grammar-fuzzer/ClickHouseUnparser.py | 1815 ----------------- utils/grammar-fuzzer/README.md | 41 - utils/grammar-fuzzer/SpaceTransformer.py | 38 - utils/grammar-fuzzer/__init__.py | 1 - utils/junit_to_html/junit-noframes.xsl | 390 ---- utils/junit_to_html/junit_to_html | 86 - utils/link-validate/link-validate.sh | 42 - utils/upload_test_results/README.md | 34 - utils/upload_test_results/upload_test_results | 127 -- 14 files changed, 4384 deletions(-) delete mode 100644 utils/ci/jobs/quick-build/README.md delete mode 100755 utils/ci/jobs/quick-build/run.sh delete mode 100644 utils/ci/vagrant-freebsd/.gitignore delete mode 100644 utils/ci/vagrant-freebsd/Vagrantfile delete mode 100644 utils/grammar-fuzzer/ClickHouseUnlexer.py delete mode 100644 utils/grammar-fuzzer/ClickHouseUnparser.py delete mode 100644 utils/grammar-fuzzer/README.md delete mode 100644 utils/grammar-fuzzer/SpaceTransformer.py delete mode 100644 utils/grammar-fuzzer/__init__.py delete mode 100644 utils/junit_to_html/junit-noframes.xsl delete mode 100755 utils/junit_to_html/junit_to_html delete mode 100755 utils/link-validate/link-validate.sh delete mode 100644 utils/upload_test_results/README.md delete mode 100755 utils/upload_test_results/upload_test_results diff --git a/utils/ci/jobs/quick-build/README.md b/utils/ci/jobs/quick-build/README.md deleted file mode 100644 index 803acae0f93..00000000000 --- a/utils/ci/jobs/quick-build/README.md +++ /dev/null @@ -1,5 +0,0 @@ -## Build with debug mode and without many libraries - -This job is intended as first check that build is not broken on wide variety of platforms. - -Results of this build are not intended for production usage. diff --git a/utils/ci/jobs/quick-build/run.sh b/utils/ci/jobs/quick-build/run.sh deleted file mode 100755 index af977d14465..00000000000 --- a/utils/ci/jobs/quick-build/run.sh +++ /dev/null @@ -1,30 +0,0 @@ -#!/usr/bin/env bash -set -e -x - -# How to run: -# From "ci" directory: -# jobs/quick-build/run.sh -# or: -# ./run-with-docker.sh ubuntu:bionic jobs/quick-build/run.sh - -cd "$(dirname $0)"/../.. - -. default-config - -SOURCES_METHOD=local -COMPILER=clang -COMPILER_INSTALL_METHOD=packages -COMPILER_PACKAGE_VERSION=6.0 -BUILD_METHOD=normal -BUILD_TARGETS=clickhouse -BUILD_TYPE=Debug -ENABLE_EMBEDDED_COMPILER=0 - -CMAKE_FLAGS="-D CMAKE_C_FLAGS_ADD=-g0 -D CMAKE_CXX_FLAGS_ADD=-g0 -D ENABLE_JEMALLOC=0 -D ENABLE_CAPNP=0 -D ENABLE_RDKAFKA=0 -D ENABLE_UNWIND=0 -D ENABLE_ICU=0 -D ENABLE_POCO_MONGODB=0 -D ENABLE_POCO_REDIS=0 -D ENABLE_POCO_NETSSL=0 -D ENABLE_ODBC=0 -D ENABLE_MYSQL=0 -D ENABLE_SSL=0 -D ENABLE_POCO_NETSSL=0 -D ENABLE_CASSANDRA=0 -D ENABLE_LDAP=0" - -[[ $(uname) == "FreeBSD" ]] && COMPILER_PACKAGE_VERSION=devel && export COMPILER_PATH=/usr/local/bin - -. get-sources.sh -. prepare-toolchain.sh -. install-libraries.sh -. build-normal.sh diff --git a/utils/ci/vagrant-freebsd/.gitignore b/utils/ci/vagrant-freebsd/.gitignore deleted file mode 100644 index 8000dd9db47..00000000000 --- a/utils/ci/vagrant-freebsd/.gitignore +++ /dev/null @@ -1 +0,0 @@ -.vagrant diff --git a/utils/ci/vagrant-freebsd/Vagrantfile b/utils/ci/vagrant-freebsd/Vagrantfile deleted file mode 100644 index c01ae5fa6e2..00000000000 --- a/utils/ci/vagrant-freebsd/Vagrantfile +++ /dev/null @@ -1,3 +0,0 @@ -Vagrant.configure("2") do |config| - config.vm.box = "generic/freebsd11" -end diff --git a/utils/grammar-fuzzer/ClickHouseUnlexer.py b/utils/grammar-fuzzer/ClickHouseUnlexer.py deleted file mode 100644 index c91522bd7be..00000000000 --- a/utils/grammar-fuzzer/ClickHouseUnlexer.py +++ /dev/null @@ -1,1771 +0,0 @@ -# Generated by Grammarinator 19.3 - -from itertools import chain -from grammarinator.runtime import * - -charset_0 = list(chain(*multirange_diff(printable_unicode_ranges, [(39, 40),(92, 93)]))) -charset_1 = list(chain(range(97, 98), range(65, 66))) -charset_2 = list(chain(range(98, 99), range(66, 67))) -charset_3 = list(chain(range(99, 100), range(67, 68))) -charset_4 = list(chain(range(100, 101), range(68, 69))) -charset_5 = list(chain(range(101, 102), range(69, 70))) -charset_6 = list(chain(range(102, 103), range(70, 71))) -charset_7 = list(chain(range(103, 104), range(71, 72))) -charset_8 = list(chain(range(104, 105), range(72, 73))) -charset_9 = list(chain(range(105, 106), range(73, 74))) -charset_10 = list(chain(range(106, 107), range(74, 75))) -charset_11 = list(chain(range(107, 108), range(75, 76))) -charset_12 = list(chain(range(108, 109), range(76, 77))) -charset_13 = list(chain(range(109, 110), range(77, 78))) -charset_14 = list(chain(range(110, 111), range(78, 79))) -charset_15 = list(chain(range(111, 112), range(79, 80))) -charset_16 = list(chain(range(112, 113), range(80, 81))) -charset_17 = list(chain(range(113, 114), range(81, 82))) -charset_18 = list(chain(range(114, 115), range(82, 83))) -charset_19 = list(chain(range(115, 116), range(83, 84))) -charset_20 = list(chain(range(116, 117), range(84, 85))) -charset_21 = list(chain(range(117, 118), range(85, 86))) -charset_22 = list(chain(range(118, 119), range(86, 87))) -charset_23 = list(chain(range(119, 120), range(87, 88))) -charset_24 = list(chain(range(120, 121), range(88, 89))) -charset_25 = list(chain(range(121, 122), range(89, 90))) -charset_26 = list(chain(range(122, 123), range(90, 91))) -charset_27 = list(chain(range(97, 123), range(65, 91))) -charset_28 = list(chain(range(48, 58))) -charset_29 = list(chain(range(48, 58), range(97, 103), range(65, 71))) -charset_30 = list(chain(*multirange_diff(printable_unicode_ranges, [(92, 93),(92, 93)]))) -charset_31 = list(chain(range(32, 33), range(11, 12), range(12, 13), range(9, 10), range(13, 14), range(10, 11))) - - -class ClickHouseUnlexer(Grammarinator): - - def __init__(self, *, max_depth=float('inf'), weights=None, cooldown=1.0): - super(ClickHouseUnlexer, self).__init__() - self.unlexer = self - self.max_depth = max_depth - self.weights = weights or dict() - self.cooldown = cooldown - - def EOF(self, *args, **kwargs): - pass - - @depthcontrol - def INTERVAL_TYPE(self): - current = self.create_node(UnlexerRule(name='INTERVAL_TYPE')) - choice = self.choice([0 if [2, 2, 2, 2, 2, 2, 2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_0', i), 1) for i, w in enumerate([1, 1, 1, 1, 1, 1, 1, 1])]) - self.unlexer.weights[('alt_0', choice)] = self.unlexer.weights.get(('alt_0', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.SECOND() - elif choice == 1: - current += self.unlexer.MINUTE() - elif choice == 2: - current += self.unlexer.HOUR() - elif choice == 3: - current += self.unlexer.DAY() - elif choice == 4: - current += self.unlexer.WEEK() - elif choice == 5: - current += self.unlexer.MONTH() - elif choice == 6: - current += self.unlexer.QUARTER() - elif choice == 7: - current += self.unlexer.YEAR() - return current - INTERVAL_TYPE.min_depth = 2 - - @depthcontrol - def ALIAS(self): - current = self.create_node(UnlexerRule(name='ALIAS')) - current += self.unlexer.A() - current += self.unlexer.L() - current += self.unlexer.I() - current += self.unlexer.A() - current += self.unlexer.S() - return current - ALIAS.min_depth = 1 - - @depthcontrol - def ALL(self): - current = self.create_node(UnlexerRule(name='ALL')) - current += self.unlexer.A() - current += self.unlexer.L() - current += self.unlexer.L() - return current - ALL.min_depth = 1 - - @depthcontrol - def AND(self): - current = self.create_node(UnlexerRule(name='AND')) - current += self.unlexer.A() - current += self.unlexer.N() - current += self.unlexer.D() - return current - AND.min_depth = 1 - - @depthcontrol - def ANTI(self): - current = self.create_node(UnlexerRule(name='ANTI')) - current += self.unlexer.A() - current += self.unlexer.N() - current += self.unlexer.T() - current += self.unlexer.I() - return current - ANTI.min_depth = 1 - - @depthcontrol - def ANY(self): - current = self.create_node(UnlexerRule(name='ANY')) - current += self.unlexer.A() - current += self.unlexer.N() - current += self.unlexer.Y() - return current - ANY.min_depth = 1 - - @depthcontrol - def ARRAY(self): - current = self.create_node(UnlexerRule(name='ARRAY')) - current += self.unlexer.A() - current += self.unlexer.R() - current += self.unlexer.R() - current += self.unlexer.A() - current += self.unlexer.Y() - return current - ARRAY.min_depth = 1 - - @depthcontrol - def AS(self): - current = self.create_node(UnlexerRule(name='AS')) - current += self.unlexer.A() - current += self.unlexer.S() - return current - AS.min_depth = 1 - - @depthcontrol - def ASCENDING(self): - current = self.create_node(UnlexerRule(name='ASCENDING')) - choice = self.choice([0 if [1, 1][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_9', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_9', choice)] = self.unlexer.weights.get(('alt_9', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.A() - current += self.unlexer.S() - current += self.unlexer.C() - elif choice == 1: - current += self.unlexer.A() - current += self.unlexer.S() - current += self.unlexer.C() - current += self.unlexer.E() - current += self.unlexer.N() - current += self.unlexer.D() - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.G() - return current - ASCENDING.min_depth = 1 - - @depthcontrol - def ASOF(self): - current = self.create_node(UnlexerRule(name='ASOF')) - current += self.unlexer.A() - current += self.unlexer.S() - current += self.unlexer.O() - current += self.unlexer.F() - return current - ASOF.min_depth = 1 - - @depthcontrol - def BETWEEN(self): - current = self.create_node(UnlexerRule(name='BETWEEN')) - current += self.unlexer.B() - current += self.unlexer.E() - current += self.unlexer.T() - current += self.unlexer.W() - current += self.unlexer.E() - current += self.unlexer.E() - current += self.unlexer.N() - return current - BETWEEN.min_depth = 1 - - @depthcontrol - def BOTH(self): - current = self.create_node(UnlexerRule(name='BOTH')) - current += self.unlexer.B() - current += self.unlexer.O() - current += self.unlexer.T() - current += self.unlexer.H() - return current - BOTH.min_depth = 1 - - @depthcontrol - def BY(self): - current = self.create_node(UnlexerRule(name='BY')) - current += self.unlexer.B() - current += self.unlexer.Y() - return current - BY.min_depth = 1 - - @depthcontrol - def CASE(self): - current = self.create_node(UnlexerRule(name='CASE')) - current += self.unlexer.C() - current += self.unlexer.A() - current += self.unlexer.S() - current += self.unlexer.E() - return current - CASE.min_depth = 1 - - @depthcontrol - def CAST(self): - current = self.create_node(UnlexerRule(name='CAST')) - current += self.unlexer.C() - current += self.unlexer.A() - current += self.unlexer.S() - current += self.unlexer.T() - return current - CAST.min_depth = 1 - - @depthcontrol - def CLUSTER(self): - current = self.create_node(UnlexerRule(name='CLUSTER')) - current += self.unlexer.C() - current += self.unlexer.L() - current += self.unlexer.U() - current += self.unlexer.S() - current += self.unlexer.T() - current += self.unlexer.E() - current += self.unlexer.R() - return current - CLUSTER.min_depth = 1 - - @depthcontrol - def COLLATE(self): - current = self.create_node(UnlexerRule(name='COLLATE')) - current += self.unlexer.C() - current += self.unlexer.O() - current += self.unlexer.L() - current += self.unlexer.L() - current += self.unlexer.A() - current += self.unlexer.T() - current += self.unlexer.E() - return current - COLLATE.min_depth = 1 - - @depthcontrol - def CREATE(self): - current = self.create_node(UnlexerRule(name='CREATE')) - current += self.unlexer.C() - current += self.unlexer.R() - current += self.unlexer.E() - current += self.unlexer.A() - current += self.unlexer.T() - current += self.unlexer.E() - return current - CREATE.min_depth = 1 - - @depthcontrol - def CROSS(self): - current = self.create_node(UnlexerRule(name='CROSS')) - current += self.unlexer.C() - current += self.unlexer.R() - current += self.unlexer.O() - current += self.unlexer.S() - current += self.unlexer.S() - return current - CROSS.min_depth = 1 - - @depthcontrol - def DATABASE(self): - current = self.create_node(UnlexerRule(name='DATABASE')) - current += self.unlexer.D() - current += self.unlexer.A() - current += self.unlexer.T() - current += self.unlexer.A() - current += self.unlexer.B() - current += self.unlexer.A() - current += self.unlexer.S() - current += self.unlexer.E() - return current - DATABASE.min_depth = 1 - - @depthcontrol - def DAY(self): - current = self.create_node(UnlexerRule(name='DAY')) - current += self.unlexer.D() - current += self.unlexer.A() - current += self.unlexer.Y() - return current - DAY.min_depth = 1 - - @depthcontrol - def DEFAULT(self): - current = self.create_node(UnlexerRule(name='DEFAULT')) - current += self.unlexer.D() - current += self.unlexer.E() - current += self.unlexer.F() - current += self.unlexer.A() - current += self.unlexer.U() - current += self.unlexer.L() - current += self.unlexer.T() - return current - DEFAULT.min_depth = 1 - - @depthcontrol - def DELETE(self): - current = self.create_node(UnlexerRule(name='DELETE')) - current += self.unlexer.D() - current += self.unlexer.E() - current += self.unlexer.L() - current += self.unlexer.E() - current += self.unlexer.T() - current += self.unlexer.E() - return current - DELETE.min_depth = 1 - - @depthcontrol - def DESCENDING(self): - current = self.create_node(UnlexerRule(name='DESCENDING')) - choice = self.choice([0 if [1, 1][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_12', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_12', choice)] = self.unlexer.weights.get(('alt_12', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.D() - current += self.unlexer.E() - current += self.unlexer.S() - current += self.unlexer.C() - elif choice == 1: - current += self.unlexer.D() - current += self.unlexer.E() - current += self.unlexer.S() - current += self.unlexer.C() - current += self.unlexer.E() - current += self.unlexer.N() - current += self.unlexer.D() - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.G() - return current - DESCENDING.min_depth = 1 - - @depthcontrol - def DISK(self): - current = self.create_node(UnlexerRule(name='DISK')) - current += self.unlexer.D() - current += self.unlexer.I() - current += self.unlexer.S() - current += self.unlexer.K() - return current - DISK.min_depth = 1 - - @depthcontrol - def DISTINCT(self): - current = self.create_node(UnlexerRule(name='DISTINCT')) - current += self.unlexer.D() - current += self.unlexer.I() - current += self.unlexer.S() - current += self.unlexer.T() - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.C() - current += self.unlexer.T() - return current - DISTINCT.min_depth = 1 - - @depthcontrol - def DROP(self): - current = self.create_node(UnlexerRule(name='DROP')) - current += self.unlexer.D() - current += self.unlexer.R() - current += self.unlexer.O() - current += self.unlexer.P() - return current - DROP.min_depth = 1 - - @depthcontrol - def ELSE(self): - current = self.create_node(UnlexerRule(name='ELSE')) - current += self.unlexer.E() - current += self.unlexer.L() - current += self.unlexer.S() - current += self.unlexer.E() - return current - ELSE.min_depth = 1 - - @depthcontrol - def END(self): - current = self.create_node(UnlexerRule(name='END')) - current += self.unlexer.E() - current += self.unlexer.N() - current += self.unlexer.D() - return current - END.min_depth = 1 - - @depthcontrol - def ENGINE(self): - current = self.create_node(UnlexerRule(name='ENGINE')) - current += self.unlexer.E() - current += self.unlexer.N() - current += self.unlexer.G() - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.E() - return current - ENGINE.min_depth = 1 - - @depthcontrol - def EXISTS(self): - current = self.create_node(UnlexerRule(name='EXISTS')) - current += self.unlexer.E() - current += self.unlexer.X() - current += self.unlexer.I() - current += self.unlexer.S() - current += self.unlexer.T() - current += self.unlexer.S() - return current - EXISTS.min_depth = 1 - - @depthcontrol - def EXTRACT(self): - current = self.create_node(UnlexerRule(name='EXTRACT')) - current += self.unlexer.E() - current += self.unlexer.X() - current += self.unlexer.T() - current += self.unlexer.R() - current += self.unlexer.A() - current += self.unlexer.C() - current += self.unlexer.T() - return current - EXTRACT.min_depth = 1 - - @depthcontrol - def FINAL(self): - current = self.create_node(UnlexerRule(name='FINAL')) - current += self.unlexer.F() - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.A() - current += self.unlexer.L() - return current - FINAL.min_depth = 1 - - @depthcontrol - def FIRST(self): - current = self.create_node(UnlexerRule(name='FIRST')) - current += self.unlexer.F() - current += self.unlexer.I() - current += self.unlexer.R() - current += self.unlexer.S() - current += self.unlexer.T() - return current - FIRST.min_depth = 1 - - @depthcontrol - def FORMAT(self): - current = self.create_node(UnlexerRule(name='FORMAT')) - current += self.unlexer.F() - current += self.unlexer.O() - current += self.unlexer.R() - current += self.unlexer.M() - current += self.unlexer.A() - current += self.unlexer.T() - return current - FORMAT.min_depth = 1 - - @depthcontrol - def FROM(self): - current = self.create_node(UnlexerRule(name='FROM')) - current += self.unlexer.F() - current += self.unlexer.R() - current += self.unlexer.O() - current += self.unlexer.M() - return current - FROM.min_depth = 1 - - @depthcontrol - def FULL(self): - current = self.create_node(UnlexerRule(name='FULL')) - current += self.unlexer.F() - current += self.unlexer.U() - current += self.unlexer.L() - current += self.unlexer.L() - return current - FULL.min_depth = 1 - - @depthcontrol - def GLOBAL(self): - current = self.create_node(UnlexerRule(name='GLOBAL')) - current += self.unlexer.G() - current += self.unlexer.L() - current += self.unlexer.O() - current += self.unlexer.B() - current += self.unlexer.A() - current += self.unlexer.L() - return current - GLOBAL.min_depth = 1 - - @depthcontrol - def GROUP(self): - current = self.create_node(UnlexerRule(name='GROUP')) - current += self.unlexer.G() - current += self.unlexer.R() - current += self.unlexer.O() - current += self.unlexer.U() - current += self.unlexer.P() - return current - GROUP.min_depth = 1 - - @depthcontrol - def HAVING(self): - current = self.create_node(UnlexerRule(name='HAVING')) - current += self.unlexer.H() - current += self.unlexer.A() - current += self.unlexer.V() - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.G() - return current - HAVING.min_depth = 1 - - @depthcontrol - def HOUR(self): - current = self.create_node(UnlexerRule(name='HOUR')) - current += self.unlexer.H() - current += self.unlexer.O() - current += self.unlexer.U() - current += self.unlexer.R() - return current - HOUR.min_depth = 1 - - @depthcontrol - def IF(self): - current = self.create_node(UnlexerRule(name='IF')) - current += self.unlexer.I() - current += self.unlexer.F() - return current - IF.min_depth = 1 - - @depthcontrol - def IN(self): - current = self.create_node(UnlexerRule(name='IN')) - current += self.unlexer.I() - current += self.unlexer.N() - return current - IN.min_depth = 1 - - @depthcontrol - def INF(self): - current = self.create_node(UnlexerRule(name='INF')) - choice = self.choice([0 if [1, 1][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_15', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_15', choice)] = self.unlexer.weights.get(('alt_15', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.F() - elif choice == 1: - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.F() - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.I() - current += self.unlexer.T() - current += self.unlexer.Y() - return current - INF.min_depth = 1 - - @depthcontrol - def INNER(self): - current = self.create_node(UnlexerRule(name='INNER')) - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.N() - current += self.unlexer.E() - current += self.unlexer.R() - return current - INNER.min_depth = 1 - - @depthcontrol - def INSERT(self): - current = self.create_node(UnlexerRule(name='INSERT')) - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.S() - current += self.unlexer.E() - current += self.unlexer.R() - current += self.unlexer.T() - return current - INSERT.min_depth = 1 - - @depthcontrol - def INTERVAL(self): - current = self.create_node(UnlexerRule(name='INTERVAL')) - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.T() - current += self.unlexer.E() - current += self.unlexer.R() - current += self.unlexer.V() - current += self.unlexer.A() - current += self.unlexer.L() - return current - INTERVAL.min_depth = 1 - - @depthcontrol - def INTO(self): - current = self.create_node(UnlexerRule(name='INTO')) - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.T() - current += self.unlexer.O() - return current - INTO.min_depth = 1 - - @depthcontrol - def IS(self): - current = self.create_node(UnlexerRule(name='IS')) - current += self.unlexer.I() - current += self.unlexer.S() - return current - IS.min_depth = 1 - - @depthcontrol - def JOIN(self): - current = self.create_node(UnlexerRule(name='JOIN')) - current += self.unlexer.J() - current += self.unlexer.O() - current += self.unlexer.I() - current += self.unlexer.N() - return current - JOIN.min_depth = 1 - - @depthcontrol - def KEY(self): - current = self.create_node(UnlexerRule(name='KEY')) - current += self.unlexer.K() - current += self.unlexer.E() - current += self.unlexer.Y() - return current - KEY.min_depth = 1 - - @depthcontrol - def LAST(self): - current = self.create_node(UnlexerRule(name='LAST')) - current += self.unlexer.L() - current += self.unlexer.A() - current += self.unlexer.S() - current += self.unlexer.T() - return current - LAST.min_depth = 1 - - @depthcontrol - def LEADING(self): - current = self.create_node(UnlexerRule(name='LEADING')) - current += self.unlexer.L() - current += self.unlexer.E() - current += self.unlexer.A() - current += self.unlexer.D() - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.G() - return current - LEADING.min_depth = 1 - - @depthcontrol - def LEFT(self): - current = self.create_node(UnlexerRule(name='LEFT')) - current += self.unlexer.L() - current += self.unlexer.E() - current += self.unlexer.F() - current += self.unlexer.T() - return current - LEFT.min_depth = 1 - - @depthcontrol - def LIKE(self): - current = self.create_node(UnlexerRule(name='LIKE')) - current += self.unlexer.L() - current += self.unlexer.I() - current += self.unlexer.K() - current += self.unlexer.E() - return current - LIKE.min_depth = 1 - - @depthcontrol - def LIMIT(self): - current = self.create_node(UnlexerRule(name='LIMIT')) - current += self.unlexer.L() - current += self.unlexer.I() - current += self.unlexer.M() - current += self.unlexer.I() - current += self.unlexer.T() - return current - LIMIT.min_depth = 1 - - @depthcontrol - def LOCAL(self): - current = self.create_node(UnlexerRule(name='LOCAL')) - current += self.unlexer.L() - current += self.unlexer.O() - current += self.unlexer.C() - current += self.unlexer.A() - current += self.unlexer.L() - return current - LOCAL.min_depth = 1 - - @depthcontrol - def MATERIALIZED(self): - current = self.create_node(UnlexerRule(name='MATERIALIZED')) - current += self.unlexer.M() - current += self.unlexer.A() - current += self.unlexer.T() - current += self.unlexer.E() - current += self.unlexer.R() - current += self.unlexer.I() - current += self.unlexer.A() - current += self.unlexer.L() - current += self.unlexer.I() - current += self.unlexer.Z() - current += self.unlexer.E() - current += self.unlexer.D() - return current - MATERIALIZED.min_depth = 1 - - @depthcontrol - def MINUTE(self): - current = self.create_node(UnlexerRule(name='MINUTE')) - current += self.unlexer.M() - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.U() - current += self.unlexer.T() - current += self.unlexer.E() - return current - MINUTE.min_depth = 1 - - @depthcontrol - def MONTH(self): - current = self.create_node(UnlexerRule(name='MONTH')) - current += self.unlexer.M() - current += self.unlexer.O() - current += self.unlexer.N() - current += self.unlexer.T() - current += self.unlexer.H() - return current - MONTH.min_depth = 1 - - @depthcontrol - def NAN_SQL(self): - current = self.create_node(UnlexerRule(name='NAN_SQL')) - current += self.unlexer.N() - current += self.unlexer.A() - current += self.unlexer.N() - return current - NAN_SQL.min_depth = 1 - - @depthcontrol - def NOT(self): - current = self.create_node(UnlexerRule(name='NOT')) - current += self.unlexer.N() - current += self.unlexer.O() - current += self.unlexer.T() - return current - NOT.min_depth = 1 - - @depthcontrol - def NULL_SQL(self): - current = self.create_node(UnlexerRule(name='NULL_SQL')) - current += self.unlexer.N() - current += self.unlexer.U() - current += self.unlexer.L() - current += self.unlexer.L() - return current - NULL_SQL.min_depth = 1 - - @depthcontrol - def NULLS(self): - current = self.create_node(UnlexerRule(name='NULLS')) - current += self.unlexer.N() - current += self.unlexer.U() - current += self.unlexer.L() - current += self.unlexer.L() - current += self.unlexer.S() - return current - NULLS.min_depth = 1 - - @depthcontrol - def OFFSET(self): - current = self.create_node(UnlexerRule(name='OFFSET')) - current += self.unlexer.O() - current += self.unlexer.F() - current += self.unlexer.F() - current += self.unlexer.S() - current += self.unlexer.E() - current += self.unlexer.T() - return current - OFFSET.min_depth = 1 - - @depthcontrol - def ON(self): - current = self.create_node(UnlexerRule(name='ON')) - current += self.unlexer.O() - current += self.unlexer.N() - return current - ON.min_depth = 1 - - @depthcontrol - def OR(self): - current = self.create_node(UnlexerRule(name='OR')) - current += self.unlexer.O() - current += self.unlexer.R() - return current - OR.min_depth = 1 - - @depthcontrol - def ORDER(self): - current = self.create_node(UnlexerRule(name='ORDER')) - current += self.unlexer.O() - current += self.unlexer.R() - current += self.unlexer.D() - current += self.unlexer.E() - current += self.unlexer.R() - return current - ORDER.min_depth = 1 - - @depthcontrol - def OUTER(self): - current = self.create_node(UnlexerRule(name='OUTER')) - current += self.unlexer.O() - current += self.unlexer.U() - current += self.unlexer.T() - current += self.unlexer.E() - current += self.unlexer.R() - return current - OUTER.min_depth = 1 - - @depthcontrol - def OUTFILE(self): - current = self.create_node(UnlexerRule(name='OUTFILE')) - current += self.unlexer.O() - current += self.unlexer.U() - current += self.unlexer.T() - current += self.unlexer.F() - current += self.unlexer.I() - current += self.unlexer.L() - current += self.unlexer.E() - return current - OUTFILE.min_depth = 1 - - @depthcontrol - def PARTITION(self): - current = self.create_node(UnlexerRule(name='PARTITION')) - current += self.unlexer.P() - current += self.unlexer.A() - current += self.unlexer.R() - current += self.unlexer.T() - current += self.unlexer.I() - current += self.unlexer.T() - current += self.unlexer.I() - current += self.unlexer.O() - current += self.unlexer.N() - return current - PARTITION.min_depth = 1 - - @depthcontrol - def PREWHERE(self): - current = self.create_node(UnlexerRule(name='PREWHERE')) - current += self.unlexer.P() - current += self.unlexer.R() - current += self.unlexer.E() - current += self.unlexer.W() - current += self.unlexer.H() - current += self.unlexer.E() - current += self.unlexer.R() - current += self.unlexer.E() - return current - PREWHERE.min_depth = 1 - - @depthcontrol - def PRIMARY(self): - current = self.create_node(UnlexerRule(name='PRIMARY')) - current += self.unlexer.P() - current += self.unlexer.R() - current += self.unlexer.I() - current += self.unlexer.M() - current += self.unlexer.A() - current += self.unlexer.R() - current += self.unlexer.Y() - return current - PRIMARY.min_depth = 1 - - @depthcontrol - def QUARTER(self): - current = self.create_node(UnlexerRule(name='QUARTER')) - current += self.unlexer.Q() - current += self.unlexer.U() - current += self.unlexer.A() - current += self.unlexer.R() - current += self.unlexer.T() - current += self.unlexer.E() - current += self.unlexer.R() - return current - QUARTER.min_depth = 1 - - @depthcontrol - def RIGHT(self): - current = self.create_node(UnlexerRule(name='RIGHT')) - current += self.unlexer.R() - current += self.unlexer.I() - current += self.unlexer.G() - current += self.unlexer.H() - current += self.unlexer.T() - return current - RIGHT.min_depth = 1 - - @depthcontrol - def SAMPLE(self): - current = self.create_node(UnlexerRule(name='SAMPLE')) - current += self.unlexer.S() - current += self.unlexer.A() - current += self.unlexer.M() - current += self.unlexer.P() - current += self.unlexer.L() - current += self.unlexer.E() - return current - SAMPLE.min_depth = 1 - - @depthcontrol - def SECOND(self): - current = self.create_node(UnlexerRule(name='SECOND')) - current += self.unlexer.S() - current += self.unlexer.E() - current += self.unlexer.C() - current += self.unlexer.O() - current += self.unlexer.N() - current += self.unlexer.D() - return current - SECOND.min_depth = 1 - - @depthcontrol - def SELECT(self): - current = self.create_node(UnlexerRule(name='SELECT')) - current += self.unlexer.S() - current += self.unlexer.E() - current += self.unlexer.L() - current += self.unlexer.E() - current += self.unlexer.C() - current += self.unlexer.T() - return current - SELECT.min_depth = 1 - - @depthcontrol - def SEMI(self): - current = self.create_node(UnlexerRule(name='SEMI')) - current += self.unlexer.S() - current += self.unlexer.E() - current += self.unlexer.M() - current += self.unlexer.I() - return current - SEMI.min_depth = 1 - - @depthcontrol - def SET(self): - current = self.create_node(UnlexerRule(name='SET')) - current += self.unlexer.S() - current += self.unlexer.E() - current += self.unlexer.T() - return current - SET.min_depth = 1 - - @depthcontrol - def SETTINGS(self): - current = self.create_node(UnlexerRule(name='SETTINGS')) - current += self.unlexer.S() - current += self.unlexer.E() - current += self.unlexer.T() - current += self.unlexer.T() - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.G() - current += self.unlexer.S() - return current - SETTINGS.min_depth = 1 - - @depthcontrol - def TABLE(self): - current = self.create_node(UnlexerRule(name='TABLE')) - current += self.unlexer.T() - current += self.unlexer.A() - current += self.unlexer.B() - current += self.unlexer.L() - current += self.unlexer.E() - return current - TABLE.min_depth = 1 - - @depthcontrol - def TEMPORARY(self): - current = self.create_node(UnlexerRule(name='TEMPORARY')) - current += self.unlexer.T() - current += self.unlexer.E() - current += self.unlexer.M() - current += self.unlexer.P() - current += self.unlexer.O() - current += self.unlexer.R() - current += self.unlexer.A() - current += self.unlexer.R() - current += self.unlexer.Y() - return current - TEMPORARY.min_depth = 1 - - @depthcontrol - def THEN(self): - current = self.create_node(UnlexerRule(name='THEN')) - current += self.unlexer.T() - current += self.unlexer.H() - current += self.unlexer.E() - current += self.unlexer.N() - return current - THEN.min_depth = 1 - - @depthcontrol - def TO(self): - current = self.create_node(UnlexerRule(name='TO')) - current += self.unlexer.T() - current += self.unlexer.O() - return current - TO.min_depth = 1 - - @depthcontrol - def TOTALS(self): - current = self.create_node(UnlexerRule(name='TOTALS')) - current += self.unlexer.T() - current += self.unlexer.O() - current += self.unlexer.T() - current += self.unlexer.A() - current += self.unlexer.L() - current += self.unlexer.S() - return current - TOTALS.min_depth = 1 - - @depthcontrol - def TRAILING(self): - current = self.create_node(UnlexerRule(name='TRAILING')) - current += self.unlexer.T() - current += self.unlexer.R() - current += self.unlexer.A() - current += self.unlexer.I() - current += self.unlexer.L() - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.G() - return current - TRAILING.min_depth = 1 - - @depthcontrol - def TRIM(self): - current = self.create_node(UnlexerRule(name='TRIM')) - current += self.unlexer.T() - current += self.unlexer.R() - current += self.unlexer.I() - current += self.unlexer.M() - return current - TRIM.min_depth = 1 - - @depthcontrol - def TTL(self): - current = self.create_node(UnlexerRule(name='TTL')) - current += self.unlexer.T() - current += self.unlexer.T() - current += self.unlexer.L() - return current - TTL.min_depth = 1 - - @depthcontrol - def UNION(self): - current = self.create_node(UnlexerRule(name='UNION')) - current += self.unlexer.U() - current += self.unlexer.N() - current += self.unlexer.I() - current += self.unlexer.O() - current += self.unlexer.N() - return current - UNION.min_depth = 1 - - @depthcontrol - def USING(self): - current = self.create_node(UnlexerRule(name='USING')) - current += self.unlexer.U() - current += self.unlexer.S() - current += self.unlexer.I() - current += self.unlexer.N() - current += self.unlexer.G() - return current - USING.min_depth = 1 - - @depthcontrol - def VALUES(self): - current = self.create_node(UnlexerRule(name='VALUES')) - current += self.unlexer.V() - current += self.unlexer.A() - current += self.unlexer.L() - current += self.unlexer.U() - current += self.unlexer.E() - current += self.unlexer.S() - return current - VALUES.min_depth = 1 - - @depthcontrol - def VOLUME(self): - current = self.create_node(UnlexerRule(name='VOLUME')) - current += self.unlexer.V() - current += self.unlexer.O() - current += self.unlexer.L() - current += self.unlexer.U() - current += self.unlexer.M() - current += self.unlexer.E() - return current - VOLUME.min_depth = 1 - - @depthcontrol - def WEEK(self): - current = self.create_node(UnlexerRule(name='WEEK')) - current += self.unlexer.W() - current += self.unlexer.E() - current += self.unlexer.E() - current += self.unlexer.K() - return current - WEEK.min_depth = 1 - - @depthcontrol - def WHEN(self): - current = self.create_node(UnlexerRule(name='WHEN')) - current += self.unlexer.W() - current += self.unlexer.H() - current += self.unlexer.E() - current += self.unlexer.N() - return current - WHEN.min_depth = 1 - - @depthcontrol - def WHERE(self): - current = self.create_node(UnlexerRule(name='WHERE')) - current += self.unlexer.W() - current += self.unlexer.H() - current += self.unlexer.E() - current += self.unlexer.R() - current += self.unlexer.E() - return current - WHERE.min_depth = 1 - - @depthcontrol - def WITH(self): - current = self.create_node(UnlexerRule(name='WITH')) - current += self.unlexer.W() - current += self.unlexer.I() - current += self.unlexer.T() - current += self.unlexer.H() - return current - WITH.min_depth = 1 - - @depthcontrol - def YEAR(self): - current = self.create_node(UnlexerRule(name='YEAR')) - current += self.unlexer.Y() - current += self.unlexer.E() - current += self.unlexer.A() - current += self.unlexer.R() - return current - YEAR.min_depth = 1 - - @depthcontrol - def IDENTIFIER(self): - current = self.create_node(UnlexerRule(name='IDENTIFIER')) - choice = self.choice([0 if [1, 1][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_18', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_18', choice)] = self.unlexer.weights.get(('alt_18', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.LETTER() - elif choice == 1: - current += self.unlexer.UNDERSCORE() - if self.unlexer.max_depth >= 1: - for _ in self.zero_or_more(): - choice = self.choice([0 if [1, 1, 1][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_22', i), 1) for i, w in enumerate([1, 1, 1])]) - self.unlexer.weights[('alt_22', choice)] = self.unlexer.weights.get(('alt_22', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.LETTER() - elif choice == 1: - current += self.unlexer.UNDERSCORE() - elif choice == 2: - current += self.unlexer.DEC_DIGIT() - - return current - IDENTIFIER.min_depth = 1 - - @depthcontrol - def FLOATING_LITERAL(self): - current = self.create_node(UnlexerRule(name='FLOATING_LITERAL')) - choice = self.choice([0 if [2, 2, 2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_26', i), 1) for i, w in enumerate([1, 1, 1, 1])]) - self.unlexer.weights[('alt_26', choice)] = self.unlexer.weights.get(('alt_26', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.HEXADECIMAL_LITERAL() - current += self.unlexer.DOT() - if self.unlexer.max_depth >= 1: - for _ in self.zero_or_more(): - current += self.unlexer.HEX_DIGIT() - - if self.unlexer.max_depth >= 1: - for _ in self.zero_or_one(): - choice = self.choice([0 if [1, 1][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_33', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_33', choice)] = self.unlexer.weights.get(('alt_33', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.P() - elif choice == 1: - current += self.unlexer.E() - if self.unlexer.max_depth >= 1: - for _ in self.zero_or_one(): - choice = self.choice([0 if [1, 1][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_37', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_37', choice)] = self.unlexer.weights.get(('alt_37', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.PLUS() - elif choice == 1: - current += self.unlexer.DASH() - - if self.unlexer.max_depth >= 0: - for _ in self.one_or_more(): - current += self.unlexer.DEC_DIGIT() - - - elif choice == 1: - current += self.unlexer.HEXADECIMAL_LITERAL() - choice = self.choice([0 if [1, 1][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_40', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_40', choice)] = self.unlexer.weights.get(('alt_40', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.P() - elif choice == 1: - current += self.unlexer.E() - if self.unlexer.max_depth >= 1: - for _ in self.zero_or_one(): - choice = self.choice([0 if [1, 1][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_44', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_44', choice)] = self.unlexer.weights.get(('alt_44', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.PLUS() - elif choice == 1: - current += self.unlexer.DASH() - - if self.unlexer.max_depth >= 0: - for _ in self.one_or_more(): - current += self.unlexer.DEC_DIGIT() - - elif choice == 2: - current += self.unlexer.INTEGER_LITERAL() - current += self.unlexer.DOT() - if self.unlexer.max_depth >= 1: - for _ in self.zero_or_more(): - current += self.unlexer.DEC_DIGIT() - - if self.unlexer.max_depth >= 1: - for _ in self.zero_or_one(): - current += self.unlexer.E() - if self.unlexer.max_depth >= 1: - for _ in self.zero_or_one(): - choice = self.choice([0 if [1, 1][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_50', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_50', choice)] = self.unlexer.weights.get(('alt_50', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.PLUS() - elif choice == 1: - current += self.unlexer.DASH() - - if self.unlexer.max_depth >= 0: - for _ in self.one_or_more(): - current += self.unlexer.DEC_DIGIT() - - - elif choice == 3: - current += self.unlexer.INTEGER_LITERAL() - current += self.unlexer.E() - if self.unlexer.max_depth >= 1: - for _ in self.zero_or_one(): - choice = self.choice([0 if [1, 1][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_54', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_54', choice)] = self.unlexer.weights.get(('alt_54', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.PLUS() - elif choice == 1: - current += self.unlexer.DASH() - - if self.unlexer.max_depth >= 0: - for _ in self.one_or_more(): - current += self.unlexer.DEC_DIGIT() - - return current - FLOATING_LITERAL.min_depth = 2 - - @depthcontrol - def HEXADECIMAL_LITERAL(self): - current = self.create_node(UnlexerRule(name='HEXADECIMAL_LITERAL')) - current += self.create_node(UnlexerRule(src='0')) - current += self.unlexer.X() - if self.unlexer.max_depth >= 0: - for _ in self.one_or_more(): - current += self.unlexer.HEX_DIGIT() - - return current - HEXADECIMAL_LITERAL.min_depth = 1 - - @depthcontrol - def INTEGER_LITERAL(self): - current = self.create_node(UnlexerRule(name='INTEGER_LITERAL')) - if self.unlexer.max_depth >= 0: - for _ in self.one_or_more(): - current += self.unlexer.DEC_DIGIT() - - return current - INTEGER_LITERAL.min_depth = 1 - - @depthcontrol - def STRING_LITERAL(self): - current = self.create_node(UnlexerRule(name='STRING_LITERAL')) - current += self.unlexer.QUOTE_SINGLE() - if self.unlexer.max_depth >= 0: - for _ in self.zero_or_more(): - choice = self.choice([0 if [0, 1][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_59', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_59', choice)] = self.unlexer.weights.get(('alt_59', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += UnlexerRule(src=self.char_from_list(charset_0)) - elif choice == 1: - current += self.unlexer.BACKSLASH() - current += UnlexerRule(src=self.any_char()) - - current += self.unlexer.QUOTE_SINGLE() - return current - STRING_LITERAL.min_depth = 1 - - @depthcontrol - def A(self): - current = self.create_node(UnlexerRule(name='A')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_1))) - return current - A.min_depth = 0 - - @depthcontrol - def B(self): - current = self.create_node(UnlexerRule(name='B')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_2))) - return current - B.min_depth = 0 - - @depthcontrol - def C(self): - current = self.create_node(UnlexerRule(name='C')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_3))) - return current - C.min_depth = 0 - - @depthcontrol - def D(self): - current = self.create_node(UnlexerRule(name='D')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_4))) - return current - D.min_depth = 0 - - @depthcontrol - def E(self): - current = self.create_node(UnlexerRule(name='E')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_5))) - return current - E.min_depth = 0 - - @depthcontrol - def F(self): - current = self.create_node(UnlexerRule(name='F')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_6))) - return current - F.min_depth = 0 - - @depthcontrol - def G(self): - current = self.create_node(UnlexerRule(name='G')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_7))) - return current - G.min_depth = 0 - - @depthcontrol - def H(self): - current = self.create_node(UnlexerRule(name='H')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_8))) - return current - H.min_depth = 0 - - @depthcontrol - def I(self): - current = self.create_node(UnlexerRule(name='I')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_9))) - return current - I.min_depth = 0 - - @depthcontrol - def J(self): - current = self.create_node(UnlexerRule(name='J')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_10))) - return current - J.min_depth = 0 - - @depthcontrol - def K(self): - current = self.create_node(UnlexerRule(name='K')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_11))) - return current - K.min_depth = 0 - - @depthcontrol - def L(self): - current = self.create_node(UnlexerRule(name='L')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_12))) - return current - L.min_depth = 0 - - @depthcontrol - def M(self): - current = self.create_node(UnlexerRule(name='M')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_13))) - return current - M.min_depth = 0 - - @depthcontrol - def N(self): - current = self.create_node(UnlexerRule(name='N')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_14))) - return current - N.min_depth = 0 - - @depthcontrol - def O(self): - current = self.create_node(UnlexerRule(name='O')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_15))) - return current - O.min_depth = 0 - - @depthcontrol - def P(self): - current = self.create_node(UnlexerRule(name='P')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_16))) - return current - P.min_depth = 0 - - @depthcontrol - def Q(self): - current = self.create_node(UnlexerRule(name='Q')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_17))) - return current - Q.min_depth = 0 - - @depthcontrol - def R(self): - current = self.create_node(UnlexerRule(name='R')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_18))) - return current - R.min_depth = 0 - - @depthcontrol - def S(self): - current = self.create_node(UnlexerRule(name='S')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_19))) - return current - S.min_depth = 0 - - @depthcontrol - def T(self): - current = self.create_node(UnlexerRule(name='T')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_20))) - return current - T.min_depth = 0 - - @depthcontrol - def U(self): - current = self.create_node(UnlexerRule(name='U')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_21))) - return current - U.min_depth = 0 - - @depthcontrol - def V(self): - current = self.create_node(UnlexerRule(name='V')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_22))) - return current - V.min_depth = 0 - - @depthcontrol - def W(self): - current = self.create_node(UnlexerRule(name='W')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_23))) - return current - W.min_depth = 0 - - @depthcontrol - def X(self): - current = self.create_node(UnlexerRule(name='X')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_24))) - return current - X.min_depth = 0 - - @depthcontrol - def Y(self): - current = self.create_node(UnlexerRule(name='Y')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_25))) - return current - Y.min_depth = 0 - - @depthcontrol - def Z(self): - current = self.create_node(UnlexerRule(name='Z')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_26))) - return current - Z.min_depth = 0 - - @depthcontrol - def LETTER(self): - current = self.create_node(UnlexerRule(name='LETTER')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_27))) - return current - LETTER.min_depth = 0 - - @depthcontrol - def DEC_DIGIT(self): - current = self.create_node(UnlexerRule(name='DEC_DIGIT')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_28))) - return current - DEC_DIGIT.min_depth = 0 - - @depthcontrol - def HEX_DIGIT(self): - current = self.create_node(UnlexerRule(name='HEX_DIGIT')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_29))) - return current - HEX_DIGIT.min_depth = 0 - - @depthcontrol - def ARROW(self): - current = self.create_node(UnlexerRule(name='ARROW')) - current += self.create_node(UnlexerRule(src='->')) - return current - ARROW.min_depth = 0 - - @depthcontrol - def ASTERISK(self): - current = self.create_node(UnlexerRule(name='ASTERISK')) - current += self.create_node(UnlexerRule(src='*')) - return current - ASTERISK.min_depth = 0 - - @depthcontrol - def BACKQUOTE(self): - current = self.create_node(UnlexerRule(name='BACKQUOTE')) - current += self.create_node(UnlexerRule(src='`')) - return current - BACKQUOTE.min_depth = 0 - - @depthcontrol - def BACKSLASH(self): - current = self.create_node(UnlexerRule(name='BACKSLASH')) - current += self.create_node(UnlexerRule(src='\\')) - return current - BACKSLASH.min_depth = 0 - - @depthcontrol - def COLON(self): - current = self.create_node(UnlexerRule(name='COLON')) - current += self.create_node(UnlexerRule(src=':')) - return current - COLON.min_depth = 0 - - @depthcontrol - def COMMA(self): - current = self.create_node(UnlexerRule(name='COMMA')) - current += self.create_node(UnlexerRule(src=',')) - return current - COMMA.min_depth = 0 - - @depthcontrol - def CONCAT(self): - current = self.create_node(UnlexerRule(name='CONCAT')) - current += self.create_node(UnlexerRule(src='||')) - return current - CONCAT.min_depth = 0 - - @depthcontrol - def DASH(self): - current = self.create_node(UnlexerRule(name='DASH')) - current += self.create_node(UnlexerRule(src='-')) - return current - DASH.min_depth = 0 - - @depthcontrol - def DOT(self): - current = self.create_node(UnlexerRule(name='DOT')) - current += self.create_node(UnlexerRule(src='.')) - return current - DOT.min_depth = 0 - - @depthcontrol - def EQ_DOUBLE(self): - current = self.create_node(UnlexerRule(name='EQ_DOUBLE')) - current += self.create_node(UnlexerRule(src='==')) - return current - EQ_DOUBLE.min_depth = 0 - - @depthcontrol - def EQ_SINGLE(self): - current = self.create_node(UnlexerRule(name='EQ_SINGLE')) - current += self.create_node(UnlexerRule(src='=')) - return current - EQ_SINGLE.min_depth = 0 - - @depthcontrol - def GE(self): - current = self.create_node(UnlexerRule(name='GE')) - current += self.create_node(UnlexerRule(src='>=')) - return current - GE.min_depth = 0 - - @depthcontrol - def GT(self): - current = self.create_node(UnlexerRule(name='GT')) - current += self.create_node(UnlexerRule(src='>')) - return current - GT.min_depth = 0 - - @depthcontrol - def LBRACKET(self): - current = self.create_node(UnlexerRule(name='LBRACKET')) - current += self.create_node(UnlexerRule(src='[')) - return current - LBRACKET.min_depth = 0 - - @depthcontrol - def LE(self): - current = self.create_node(UnlexerRule(name='LE')) - current += self.create_node(UnlexerRule(src='<=')) - return current - LE.min_depth = 0 - - @depthcontrol - def LPAREN(self): - current = self.create_node(UnlexerRule(name='LPAREN')) - current += self.create_node(UnlexerRule(src='(')) - return current - LPAREN.min_depth = 0 - - @depthcontrol - def LT(self): - current = self.create_node(UnlexerRule(name='LT')) - current += self.create_node(UnlexerRule(src='<')) - return current - LT.min_depth = 0 - - @depthcontrol - def NOT_EQ(self): - current = self.create_node(UnlexerRule(name='NOT_EQ')) - choice = self.choice([0 if [0, 0][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_79', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_79', choice)] = self.unlexer.weights.get(('alt_79', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.create_node(UnlexerRule(src='!=')) - elif choice == 1: - current += self.create_node(UnlexerRule(src='<>')) - return current - NOT_EQ.min_depth = 0 - - @depthcontrol - def PERCENT(self): - current = self.create_node(UnlexerRule(name='PERCENT')) - current += self.create_node(UnlexerRule(src='%')) - return current - PERCENT.min_depth = 0 - - @depthcontrol - def PLUS(self): - current = self.create_node(UnlexerRule(name='PLUS')) - current += self.create_node(UnlexerRule(src='+')) - return current - PLUS.min_depth = 0 - - @depthcontrol - def QUERY(self): - current = self.create_node(UnlexerRule(name='QUERY')) - current += self.create_node(UnlexerRule(src='?')) - return current - QUERY.min_depth = 0 - - @depthcontrol - def QUOTE_SINGLE(self): - current = self.create_node(UnlexerRule(name='QUOTE_SINGLE')) - current += self.create_node(UnlexerRule(src='\'')) - return current - QUOTE_SINGLE.min_depth = 0 - - @depthcontrol - def RBRACKET(self): - current = self.create_node(UnlexerRule(name='RBRACKET')) - current += self.create_node(UnlexerRule(src=']')) - return current - RBRACKET.min_depth = 0 - - @depthcontrol - def RPAREN(self): - current = self.create_node(UnlexerRule(name='RPAREN')) - current += self.create_node(UnlexerRule(src=')')) - return current - RPAREN.min_depth = 0 - - @depthcontrol - def SEMICOLON(self): - current = self.create_node(UnlexerRule(name='SEMICOLON')) - current += self.create_node(UnlexerRule(src=';')) - return current - SEMICOLON.min_depth = 0 - - @depthcontrol - def SLASH(self): - current = self.create_node(UnlexerRule(name='SLASH')) - current += self.create_node(UnlexerRule(src='/')) - return current - SLASH.min_depth = 0 - - @depthcontrol - def UNDERSCORE(self): - current = self.create_node(UnlexerRule(name='UNDERSCORE')) - current += self.create_node(UnlexerRule(src='_')) - return current - UNDERSCORE.min_depth = 0 - - @depthcontrol - def SINGLE_LINE_COMMENT(self): - current = self.create_node(UnlexerRule(name='SINGLE_LINE_COMMENT')) - current += self.create_node(UnlexerRule(src='--')) - if self.unlexer.max_depth >= 0: - for _ in self.zero_or_more(): - current += UnlexerRule(src=self.char_from_list(charset_30)) - - choice = self.choice([0 if [0, 0, 1][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_95', i), 1) for i, w in enumerate([1, 1, 1])]) - self.unlexer.weights[('alt_95', choice)] = self.unlexer.weights.get(('alt_95', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.create_node(UnlexerRule(src='\n')) - elif choice == 1: - current += self.create_node(UnlexerRule(src='\r')) - elif choice == 2: - current += self.unlexer.EOF() - return current - SINGLE_LINE_COMMENT.min_depth = 0 - - @depthcontrol - def MULTI_LINE_COMMENT(self): - current = self.create_node(UnlexerRule(name='MULTI_LINE_COMMENT')) - current += self.create_node(UnlexerRule(src='/*')) - if self.unlexer.max_depth >= 0: - for _ in self.zero_or_more(): - current += UnlexerRule(src=self.any_char()) - - current += self.create_node(UnlexerRule(src='*/')) - return current - MULTI_LINE_COMMENT.min_depth = 0 - - @depthcontrol - def WHITESPACE(self): - current = self.create_node(UnlexerRule(name='WHITESPACE')) - current += self.create_node(UnlexerRule(src=self.char_from_list(charset_31))) - return current - WHITESPACE.min_depth = 0 - diff --git a/utils/grammar-fuzzer/ClickHouseUnparser.py b/utils/grammar-fuzzer/ClickHouseUnparser.py deleted file mode 100644 index 7fa5eb96d31..00000000000 --- a/utils/grammar-fuzzer/ClickHouseUnparser.py +++ /dev/null @@ -1,1815 +0,0 @@ -# Generated by Grammarinator 19.3 - -from itertools import chain -from grammarinator.runtime import * - -import ClickHouseUnlexer - - -class ClickHouseUnparser(Grammarinator): - - def __init__(self, unlexer): - super(ClickHouseUnparser, self).__init__() - self.unlexer = unlexer - @depthcontrol - def queryList(self): - current = self.create_node(UnparserRule(name='queryList')) - current += self.queryStmt() - if self.unlexer.max_depth >= 8: - for _ in self.zero_or_more(): - current += self.unlexer.SEMICOLON() - current += self.queryStmt() - - if self.unlexer.max_depth >= 1: - for _ in self.zero_or_one(): - current += self.unlexer.SEMICOLON() - - current += self.unlexer.EOF() - return current - queryList.min_depth = 8 - - @depthcontrol - def queryStmt(self): - current = self.create_node(UnparserRule(name='queryStmt')) - current += self.query() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.INTO() - current += self.unlexer.OUTFILE() - current += self.unlexer.STRING_LITERAL() - - if self.unlexer.max_depth >= 3: - for _ in self.zero_or_one(): - current += self.unlexer.FORMAT() - current += self.identifier() - - return current - queryStmt.min_depth = 7 - - @depthcontrol - def query(self): - current = self.create_node(UnparserRule(name='query')) - choice = self.choice([0 if [6, 7, 6, 6][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_108', i), 1) for i, w in enumerate([1, 1, 1, 1])]) - self.unlexer.weights[('alt_108', choice)] = self.unlexer.weights.get(('alt_108', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.distributedStmt() - elif choice == 1: - current += self.insertStmt() - elif choice == 2: - current += self.selectUnionStmt() - elif choice == 3: - current += self.setStmt() - return current - query.min_depth = 6 - - @depthcontrol - def distributedStmt(self): - current = self.create_node(UnparserRule(name='distributedStmt')) - choice = self.choice([0 if [5, 6, 6][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_113', i), 1) for i, w in enumerate([1, 1, 1])]) - self.unlexer.weights[('alt_113', choice)] = self.unlexer.weights.get(('alt_113', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.createDatabaseStmt() - elif choice == 1: - current += self.createTableStmt() - elif choice == 2: - current += self.dropStmt() - if self.unlexer.max_depth >= 3: - for _ in self.zero_or_one(): - current += self.unlexer.ON() - current += self.unlexer.CLUSTER() - current += self.identifier() - - return current - distributedStmt.min_depth = 5 - - @depthcontrol - def createDatabaseStmt(self): - current = self.create_node(UnparserRule(name='createDatabaseStmt')) - current += self.unlexer.CREATE() - current += self.unlexer.DATABASE() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.IF() - current += self.unlexer.NOT() - current += self.unlexer.EXISTS() - - current += self.databaseIdentifier() - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_one(): - current += self.engineExpr() - - return current - createDatabaseStmt.min_depth = 4 - - @depthcontrol - def createTableStmt(self): - current = self.create_node(UnparserRule(name='createTableStmt')) - current += self.unlexer.CREATE() - current += self.unlexer.TABLE() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.IF() - current += self.unlexer.NOT() - current += self.unlexer.EXISTS() - - current += self.tableIdentifier() - current += self.schemaClause() - return current - createTableStmt.min_depth = 5 - - @depthcontrol - def schemaClause(self): - current = self.create_node(UnparserRule(name='schemaClause')) - choice = self.choice([0 if [8, 7, 5, 4][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_121', i), 1) for i, w in enumerate([1, 1, 1, 1])]) - self.unlexer.weights[('alt_121', choice)] = self.unlexer.weights.get(('alt_121', choice), 1) * self.unlexer.cooldown - if choice == 0: - current = self.schemaClause_SchemaDescriptionClause() - elif choice == 1: - current = self.schemaClause_SchemaAsSubqueryClause() - elif choice == 2: - current = self.schemaClause_SchemaAsTableClause() - elif choice == 3: - current = self.schemaClause_SchemaAsFunctionClause() - return current - schemaClause.min_depth = 4 - - @depthcontrol - def schemaClause_SchemaDescriptionClause(self): - current = self.create_node(UnparserRule(name='schemaClause_SchemaDescriptionClause')) - current += self.unlexer.LPAREN() - current += self.tableElementExpr() - if self.unlexer.max_depth >= 7: - for _ in self.zero_or_more(): - current += self.unlexer.COMMA() - current += self.tableElementExpr() - - current += self.unlexer.RPAREN() - current += self.engineClause() - return current - schemaClause_SchemaDescriptionClause.min_depth = 7 - - @depthcontrol - def schemaClause_SchemaAsSubqueryClause(self): - current = self.create_node(UnparserRule(name='schemaClause_SchemaAsSubqueryClause')) - if self.unlexer.max_depth >= 5: - for _ in self.zero_or_one(): - current += self.engineClause() - - current += self.unlexer.AS() - current += self.selectUnionStmt() - return current - schemaClause_SchemaAsSubqueryClause.min_depth = 6 - - @depthcontrol - def schemaClause_SchemaAsTableClause(self): - current = self.create_node(UnparserRule(name='schemaClause_SchemaAsTableClause')) - current += self.unlexer.AS() - current += self.tableIdentifier() - if self.unlexer.max_depth >= 5: - for _ in self.zero_or_one(): - current += self.engineClause() - - return current - schemaClause_SchemaAsTableClause.min_depth = 4 - - @depthcontrol - def schemaClause_SchemaAsFunctionClause(self): - current = self.create_node(UnparserRule(name='schemaClause_SchemaAsFunctionClause')) - current += self.unlexer.AS() - current += self.identifier() - current += self.unlexer.LPAREN() - if self.unlexer.max_depth >= 5: - for _ in self.zero_or_one(): - current += self.tableArgList() - - current += self.unlexer.RPAREN() - return current - schemaClause_SchemaAsFunctionClause.min_depth = 3 - - @depthcontrol - def engineClause(self): - current = self.create_node(UnparserRule(name='engineClause')) - current += self.engineExpr() - if self.unlexer.max_depth >= 6: - for _ in self.zero_or_one(): - current += self.orderByClause() - - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_one(): - current += self.partitionByClause() - - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_one(): - current += self.primaryKeyClause() - - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_one(): - current += self.sampleByClause() - - if self.unlexer.max_depth >= 5: - for _ in self.zero_or_one(): - current += self.ttlClause() - - if self.unlexer.max_depth >= 6: - for _ in self.zero_or_one(): - current += self.settingsClause() - - return current - engineClause.min_depth = 4 - - @depthcontrol - def partitionByClause(self): - current = self.create_node(UnparserRule(name='partitionByClause')) - current += self.unlexer.PARTITION() - current += self.unlexer.BY() - current += self.columnExpr() - return current - partitionByClause.min_depth = 3 - - @depthcontrol - def primaryKeyClause(self): - current = self.create_node(UnparserRule(name='primaryKeyClause')) - current += self.unlexer.PRIMARY() - current += self.unlexer.KEY() - current += self.columnExpr() - return current - primaryKeyClause.min_depth = 3 - - @depthcontrol - def sampleByClause(self): - current = self.create_node(UnparserRule(name='sampleByClause')) - current += self.unlexer.SAMPLE() - current += self.unlexer.BY() - current += self.columnExpr() - return current - sampleByClause.min_depth = 3 - - @depthcontrol - def ttlClause(self): - current = self.create_node(UnparserRule(name='ttlClause')) - current += self.unlexer.TTL() - current += self.ttlExpr() - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_more(): - current += self.unlexer.COMMA() - current += self.ttlExpr() - - return current - ttlClause.min_depth = 4 - - @depthcontrol - def engineExpr(self): - current = self.create_node(UnparserRule(name='engineExpr')) - current += self.unlexer.ENGINE() - if self.unlexer.max_depth >= 1: - for _ in self.zero_or_one(): - current += self.unlexer.EQ_SINGLE() - - current += self.identifier() - if self.unlexer.max_depth >= 1: - for _ in self.zero_or_one(): - current += self.unlexer.LPAREN() - if self.unlexer.max_depth >= 5: - for _ in self.zero_or_one(): - current += self.tableArgList() - - current += self.unlexer.RPAREN() - - return current - engineExpr.min_depth = 3 - - @depthcontrol - def tableElementExpr(self): - current = self.create_node(UnparserRule(name='tableElementExpr')) - current = self.tableElementExpr_TableElementColumn() - return current - tableElementExpr.min_depth = 6 - - @depthcontrol - def tableElementExpr_TableElementColumn(self): - current = self.create_node(UnparserRule(name='tableElementExpr_TableElementColumn')) - current += self.identifier() - current += self.columnTypeExpr() - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_one(): - current += self.tableColumnPropertyExpr() - - if self.unlexer.max_depth >= 3: - for _ in self.zero_or_one(): - current += self.unlexer.TTL() - current += self.columnExpr() - - return current - tableElementExpr_TableElementColumn.min_depth = 5 - - @depthcontrol - def tableColumnPropertyExpr(self): - current = self.create_node(UnparserRule(name='tableColumnPropertyExpr')) - choice = self.choice([0 if [2, 2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_142', i), 1) for i, w in enumerate([1, 1, 1])]) - self.unlexer.weights[('alt_142', choice)] = self.unlexer.weights.get(('alt_142', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.DEFAULT() - elif choice == 1: - current += self.unlexer.MATERIALIZED() - elif choice == 2: - current += self.unlexer.ALIAS() - current += self.columnExpr() - return current - tableColumnPropertyExpr.min_depth = 3 - - @depthcontrol - def ttlExpr(self): - current = self.create_node(UnparserRule(name='ttlExpr')) - current += self.columnExpr() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - choice = self.choice([0 if [2, 2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_147', i), 1) for i, w in enumerate([1, 1, 1])]) - self.unlexer.weights[('alt_147', choice)] = self.unlexer.weights.get(('alt_147', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.DELETE() - elif choice == 1: - current += self.unlexer.TO() - current += self.unlexer.DISK() - current += self.unlexer.STRING_LITERAL() - elif choice == 2: - current += self.unlexer.TO() - current += self.unlexer.VOLUME() - current += self.unlexer.STRING_LITERAL() - - return current - ttlExpr.min_depth = 3 - - @depthcontrol - def dropStmt(self): - current = self.create_node(UnparserRule(name='dropStmt')) - choice = self.choice([0 if [5, 5][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_151', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_151', choice)] = self.unlexer.weights.get(('alt_151', choice), 1) * self.unlexer.cooldown - if choice == 0: - current = self.dropStmt_DropDatabaseStmt() - elif choice == 1: - current = self.dropStmt_DropTableStmt() - return current - dropStmt.min_depth = 5 - - @depthcontrol - def dropStmt_DropDatabaseStmt(self): - current = self.create_node(UnparserRule(name='dropStmt_DropDatabaseStmt')) - current += self.unlexer.DROP() - current += self.unlexer.DATABASE() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.IF() - current += self.unlexer.EXISTS() - - current += self.databaseIdentifier() - return current - dropStmt_DropDatabaseStmt.min_depth = 4 - - @depthcontrol - def dropStmt_DropTableStmt(self): - current = self.create_node(UnparserRule(name='dropStmt_DropTableStmt')) - current += self.unlexer.DROP() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.TEMPORARY() - - current += self.unlexer.TABLE() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.IF() - current += self.unlexer.EXISTS() - - current += self.tableIdentifier() - return current - dropStmt_DropTableStmt.min_depth = 4 - - @depthcontrol - def insertStmt(self): - current = self.create_node(UnparserRule(name='insertStmt')) - current += self.unlexer.INSERT() - current += self.unlexer.INTO() - current += self.tableIdentifier() - if self.unlexer.max_depth >= 3: - for _ in self.zero_or_one(): - current += self.unlexer.LPAREN() - current += self.identifier() - if self.unlexer.max_depth >= 3: - for _ in self.zero_or_more(): - current += self.unlexer.COMMA() - current += self.identifier() - - current += self.unlexer.RPAREN() - - current += self.valuesClause() - return current - insertStmt.min_depth = 6 - - @depthcontrol - def valuesClause(self): - current = self.create_node(UnparserRule(name='valuesClause')) - choice = self.choice([0 if [5, 6][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_159', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_159', choice)] = self.unlexer.weights.get(('alt_159', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.VALUES() - current += self.valueTupleExpr() - if self.unlexer.max_depth >= 5: - for _ in self.zero_or_more(): - current += self.unlexer.COMMA() - current += self.valueTupleExpr() - - elif choice == 1: - current += self.selectUnionStmt() - return current - valuesClause.min_depth = 5 - - @depthcontrol - def valueTupleExpr(self): - current = self.create_node(UnparserRule(name='valueTupleExpr')) - current += self.unlexer.LPAREN() - current += self.valueExprList() - current += self.unlexer.RPAREN() - return current - valueTupleExpr.min_depth = 4 - - @depthcontrol - def selectUnionStmt(self): - current = self.create_node(UnparserRule(name='selectUnionStmt')) - current += self.selectStmt() - if self.unlexer.max_depth >= 5: - for _ in self.zero_or_more(): - current += self.unlexer.UNION() - current += self.unlexer.ALL() - current += self.selectStmt() - - return current - selectUnionStmt.min_depth = 5 - - @depthcontrol - def selectStmt(self): - current = self.create_node(UnparserRule(name='selectStmt')) - if self.unlexer.max_depth >= 5: - for _ in self.zero_or_one(): - current += self.withClause() - - current += self.unlexer.SELECT() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.DISTINCT() - - current += self.columnExprList() - if self.unlexer.max_depth >= 8: - for _ in self.zero_or_one(): - current += self.fromClause() - - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_one(): - current += self.sampleClause() - - if self.unlexer.max_depth >= 5: - for _ in self.zero_or_one(): - current += self.arrayJoinClause() - - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_one(): - current += self.prewhereClause() - - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_one(): - current += self.whereClause() - - if self.unlexer.max_depth >= 5: - for _ in self.zero_or_one(): - current += self.groupByClause() - - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_one(): - current += self.havingClause() - - if self.unlexer.max_depth >= 6: - for _ in self.zero_or_one(): - current += self.orderByClause() - - if self.unlexer.max_depth >= 5: - for _ in self.zero_or_one(): - current += self.limitByClause() - - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_one(): - current += self.limitClause() - - if self.unlexer.max_depth >= 6: - for _ in self.zero_or_one(): - current += self.settingsClause() - - return current - selectStmt.min_depth = 4 - - @depthcontrol - def withClause(self): - current = self.create_node(UnparserRule(name='withClause')) - current += self.unlexer.WITH() - current += self.columnExprList() - return current - withClause.min_depth = 4 - - @depthcontrol - def fromClause(self): - current = self.create_node(UnparserRule(name='fromClause')) - current += self.unlexer.FROM() - current += self.joinExpr() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.FINAL() - - return current - fromClause.min_depth = 7 - - @depthcontrol - def sampleClause(self): - current = self.create_node(UnparserRule(name='sampleClause')) - current += self.unlexer.SAMPLE() - current += self.ratioExpr() - if self.unlexer.max_depth >= 3: - for _ in self.zero_or_one(): - current += self.unlexer.OFFSET() - current += self.ratioExpr() - - return current - sampleClause.min_depth = 3 - - @depthcontrol - def arrayJoinClause(self): - current = self.create_node(UnparserRule(name='arrayJoinClause')) - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.LEFT() - - current += self.unlexer.ARRAY() - current += self.unlexer.JOIN() - current += self.columnExprList() - return current - arrayJoinClause.min_depth = 4 - - @depthcontrol - def prewhereClause(self): - current = self.create_node(UnparserRule(name='prewhereClause')) - current += self.unlexer.PREWHERE() - current += self.columnExpr() - return current - prewhereClause.min_depth = 3 - - @depthcontrol - def whereClause(self): - current = self.create_node(UnparserRule(name='whereClause')) - current += self.unlexer.WHERE() - current += self.columnExpr() - return current - whereClause.min_depth = 3 - - @depthcontrol - def groupByClause(self): - current = self.create_node(UnparserRule(name='groupByClause')) - current += self.unlexer.GROUP() - current += self.unlexer.BY() - current += self.columnExprList() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.WITH() - current += self.unlexer.TOTALS() - - return current - groupByClause.min_depth = 4 - - @depthcontrol - def havingClause(self): - current = self.create_node(UnparserRule(name='havingClause')) - current += self.unlexer.HAVING() - current += self.columnExpr() - return current - havingClause.min_depth = 3 - - @depthcontrol - def orderByClause(self): - current = self.create_node(UnparserRule(name='orderByClause')) - current += self.unlexer.ORDER() - current += self.unlexer.BY() - current += self.orderExprList() - return current - orderByClause.min_depth = 5 - - @depthcontrol - def limitByClause(self): - current = self.create_node(UnparserRule(name='limitByClause')) - current += self.unlexer.LIMIT() - current += self.limitExpr() - current += self.unlexer.BY() - current += self.columnExprList() - return current - limitByClause.min_depth = 4 - - @depthcontrol - def limitClause(self): - current = self.create_node(UnparserRule(name='limitClause')) - current += self.unlexer.LIMIT() - current += self.limitExpr() - return current - limitClause.min_depth = 3 - - @depthcontrol - def settingsClause(self): - current = self.create_node(UnparserRule(name='settingsClause')) - current += self.unlexer.SETTINGS() - current += self.settingExprList() - return current - settingsClause.min_depth = 5 - - @depthcontrol - def joinExpr(self): - current = self.create_node(UnparserRule(name='joinExpr')) - choice = self.choice([0 if [6, 8, 8, 8][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_181', i), 1) for i, w in enumerate([1, 1, 1, 1])]) - self.unlexer.weights[('alt_181', choice)] = self.unlexer.weights.get(('alt_181', choice), 1) * self.unlexer.cooldown - if choice == 0: - current = self.joinExpr_JoinExprTable() - elif choice == 1: - current = self.joinExpr_JoinExprParens() - elif choice == 2: - current = self.joinExpr_JoinExprOp() - elif choice == 3: - current = self.joinExpr_JoinExprCrossOp() - return current - joinExpr.min_depth = 6 - - @depthcontrol - def joinExpr_JoinExprTable(self): - current = self.create_node(UnparserRule(name='joinExpr_JoinExprTable')) - current += self.tableExpr() - return current - joinExpr_JoinExprTable.min_depth = 5 - - @depthcontrol - def joinExpr_JoinExprParens(self): - current = self.create_node(UnparserRule(name='joinExpr_JoinExprParens')) - current += self.unlexer.LPAREN() - current += self.joinExpr() - current += self.unlexer.RPAREN() - return current - joinExpr_JoinExprParens.min_depth = 7 - - @depthcontrol - def joinExpr_JoinExprOp(self): - current = self.create_node(UnparserRule(name='joinExpr_JoinExprOp')) - current += self.joinExpr() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - choice = self.choice([0 if [2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_187', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_187', choice)] = self.unlexer.weights.get(('alt_187', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.GLOBAL() - elif choice == 1: - current += self.unlexer.LOCAL() - - current += self.joinOp() - current += self.unlexer.JOIN() - current += self.joinExpr() - current += self.joinConstraintClause() - return current - joinExpr_JoinExprOp.min_depth = 7 - - @depthcontrol - def joinExpr_JoinExprCrossOp(self): - current = self.create_node(UnparserRule(name='joinExpr_JoinExprCrossOp')) - current += self.joinExpr() - current += self.joinOpCross() - current += self.joinExpr() - return current - joinExpr_JoinExprCrossOp.min_depth = 7 - - @depthcontrol - def joinOp(self): - current = self.create_node(UnparserRule(name='joinOp')) - choice = self.choice([0 if [3, 3, 3][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_190', i), 1) for i, w in enumerate([1, 1, 1])]) - self.unlexer.weights[('alt_190', choice)] = self.unlexer.weights.get(('alt_190', choice), 1) * self.unlexer.cooldown - if choice == 0: - current = self.joinOp_JoinOpInner() - elif choice == 1: - current = self.joinOp_JoinOpLeftRight() - elif choice == 2: - current = self.joinOp_JoinOpFull() - return current - joinOp.min_depth = 3 - - @depthcontrol - def joinOp_JoinOpInner(self): - current = self.create_node(UnparserRule(name='joinOp_JoinOpInner')) - choice = self.choice([0 if [2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_194', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_194', choice)] = self.unlexer.weights.get(('alt_194', choice), 1) * self.unlexer.cooldown - if choice == 0: - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.ANY() - - current += self.unlexer.INNER() - elif choice == 1: - current += self.unlexer.INNER() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.ANY() - - return current - joinOp_JoinOpInner.min_depth = 2 - - @depthcontrol - def joinOp_JoinOpLeftRight(self): - current = self.create_node(UnparserRule(name='joinOp_JoinOpLeftRight')) - choice = self.choice([0 if [2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_199', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_199', choice)] = self.unlexer.weights.get(('alt_199', choice), 1) * self.unlexer.cooldown - if choice == 0: - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - choice = self.choice([0 if [2, 2, 2, 2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_203', i), 1) for i, w in enumerate([1, 1, 1, 1, 1])]) - self.unlexer.weights[('alt_203', choice)] = self.unlexer.weights.get(('alt_203', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.OUTER() - elif choice == 1: - current += self.unlexer.SEMI() - elif choice == 2: - current += self.unlexer.ANTI() - elif choice == 3: - current += self.unlexer.ANY() - elif choice == 4: - current += self.unlexer.ASOF() - - choice = self.choice([0 if [2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_209', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_209', choice)] = self.unlexer.weights.get(('alt_209', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.LEFT() - elif choice == 1: - current += self.unlexer.RIGHT() - elif choice == 1: - choice = self.choice([0 if [2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_212', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_212', choice)] = self.unlexer.weights.get(('alt_212', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.LEFT() - elif choice == 1: - current += self.unlexer.RIGHT() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - choice = self.choice([0 if [2, 2, 2, 2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_216', i), 1) for i, w in enumerate([1, 1, 1, 1, 1])]) - self.unlexer.weights[('alt_216', choice)] = self.unlexer.weights.get(('alt_216', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.OUTER() - elif choice == 1: - current += self.unlexer.SEMI() - elif choice == 2: - current += self.unlexer.ANTI() - elif choice == 3: - current += self.unlexer.ANY() - elif choice == 4: - current += self.unlexer.ASOF() - - return current - joinOp_JoinOpLeftRight.min_depth = 2 - - @depthcontrol - def joinOp_JoinOpFull(self): - current = self.create_node(UnparserRule(name='joinOp_JoinOpFull')) - choice = self.choice([0 if [2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_222', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_222', choice)] = self.unlexer.weights.get(('alt_222', choice), 1) * self.unlexer.cooldown - if choice == 0: - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - choice = self.choice([0 if [2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_226', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_226', choice)] = self.unlexer.weights.get(('alt_226', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.OUTER() - elif choice == 1: - current += self.unlexer.ANY() - - current += self.unlexer.FULL() - elif choice == 1: - current += self.unlexer.FULL() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - choice = self.choice([0 if [2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_230', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_230', choice)] = self.unlexer.weights.get(('alt_230', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.OUTER() - elif choice == 1: - current += self.unlexer.ANY() - - return current - joinOp_JoinOpFull.min_depth = 2 - - @depthcontrol - def joinOpCross(self): - current = self.create_node(UnparserRule(name='joinOpCross')) - choice = self.choice([0 if [2, 1][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_233', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_233', choice)] = self.unlexer.weights.get(('alt_233', choice), 1) * self.unlexer.cooldown - if choice == 0: - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - choice = self.choice([0 if [2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_237', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_237', choice)] = self.unlexer.weights.get(('alt_237', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.GLOBAL() - elif choice == 1: - current += self.unlexer.LOCAL() - - current += self.unlexer.CROSS() - current += self.unlexer.JOIN() - elif choice == 1: - current += self.unlexer.COMMA() - return current - joinOpCross.min_depth = 1 - - @depthcontrol - def joinConstraintClause(self): - current = self.create_node(UnparserRule(name='joinConstraintClause')) - choice = self.choice([0 if [4, 4, 4][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_240', i), 1) for i, w in enumerate([1, 1, 1])]) - self.unlexer.weights[('alt_240', choice)] = self.unlexer.weights.get(('alt_240', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.ON() - current += self.columnExprList() - elif choice == 1: - current += self.unlexer.USING() - current += self.unlexer.LPAREN() - current += self.columnExprList() - current += self.unlexer.RPAREN() - elif choice == 2: - current += self.unlexer.USING() - current += self.columnExprList() - return current - joinConstraintClause.min_depth = 4 - - @depthcontrol - def limitExpr(self): - current = self.create_node(UnparserRule(name='limitExpr')) - current += self.unlexer.INTEGER_LITERAL() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - choice = self.choice([0 if [1, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_245', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_245', choice)] = self.unlexer.weights.get(('alt_245', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.COMMA() - elif choice == 1: - current += self.unlexer.OFFSET() - current += self.unlexer.INTEGER_LITERAL() - - return current - limitExpr.min_depth = 2 - - @depthcontrol - def orderExprList(self): - current = self.create_node(UnparserRule(name='orderExprList')) - current += self.orderExpr() - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_more(): - current += self.unlexer.COMMA() - current += self.orderExpr() - - return current - orderExprList.min_depth = 4 - - @depthcontrol - def orderExpr(self): - current = self.create_node(UnparserRule(name='orderExpr')) - current += self.columnExpr() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - choice = self.choice([0 if [2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_250', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_250', choice)] = self.unlexer.weights.get(('alt_250', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.ASCENDING() - elif choice == 1: - current += self.unlexer.DESCENDING() - - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.NULLS() - choice = self.choice([0 if [2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_254', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_254', choice)] = self.unlexer.weights.get(('alt_254', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.FIRST() - elif choice == 1: - current += self.unlexer.LAST() - - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.COLLATE() - current += self.unlexer.STRING_LITERAL() - - return current - orderExpr.min_depth = 3 - - @depthcontrol - def ratioExpr(self): - current = self.create_node(UnparserRule(name='ratioExpr')) - current += self.unlexer.INTEGER_LITERAL() - current += self.unlexer.SLASH() - current += self.unlexer.INTEGER_LITERAL() - return current - ratioExpr.min_depth = 2 - - @depthcontrol - def settingExprList(self): - current = self.create_node(UnparserRule(name='settingExprList')) - current += self.settingExpr() - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_more(): - current += self.unlexer.COMMA() - current += self.settingExpr() - - return current - settingExprList.min_depth = 4 - - @depthcontrol - def settingExpr(self): - current = self.create_node(UnparserRule(name='settingExpr')) - current += self.identifier() - current += self.unlexer.EQ_SINGLE() - current += self.literal() - return current - settingExpr.min_depth = 3 - - @depthcontrol - def setStmt(self): - current = self.create_node(UnparserRule(name='setStmt')) - current += self.unlexer.SET() - current += self.settingExprList() - return current - setStmt.min_depth = 5 - - @depthcontrol - def valueExprList(self): - current = self.create_node(UnparserRule(name='valueExprList')) - current += self.valueExpr() - if self.unlexer.max_depth >= 3: - for _ in self.zero_or_more(): - current += self.unlexer.COMMA() - current += self.valueExpr() - - return current - valueExprList.min_depth = 3 - - @depthcontrol - def valueExpr(self): - current = self.create_node(UnparserRule(name='valueExpr')) - choice = self.choice([0 if [4, 6, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_260', i), 1) for i, w in enumerate([1, 1, 1])]) - self.unlexer.weights[('alt_260', choice)] = self.unlexer.weights.get(('alt_260', choice), 1) * self.unlexer.cooldown - if choice == 0: - current = self.valueExpr_ValueExprLiteral() - elif choice == 1: - current = self.valueExpr_ValueExprTuple() - elif choice == 2: - current = self.valueExpr_ValueExprArray() - return current - valueExpr.min_depth = 2 - - @depthcontrol - def valueExpr_ValueExprLiteral(self): - current = self.create_node(UnparserRule(name='valueExpr_ValueExprLiteral')) - current += self.literal() - return current - valueExpr_ValueExprLiteral.min_depth = 3 - - @depthcontrol - def valueExpr_ValueExprTuple(self): - current = self.create_node(UnparserRule(name='valueExpr_ValueExprTuple')) - current += self.valueTupleExpr() - return current - valueExpr_ValueExprTuple.min_depth = 5 - - @depthcontrol - def valueExpr_ValueExprArray(self): - current = self.create_node(UnparserRule(name='valueExpr_ValueExprArray')) - current += self.unlexer.LBRACKET() - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_one(): - current += self.valueExprList() - - current += self.unlexer.RBRACKET() - return current - valueExpr_ValueExprArray.min_depth = 1 - - @depthcontrol - def columnTypeExpr(self): - current = self.create_node(UnparserRule(name='columnTypeExpr')) - choice = self.choice([0 if [4, 5, 4, 6][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_265', i), 1) for i, w in enumerate([1, 1, 1, 1])]) - self.unlexer.weights[('alt_265', choice)] = self.unlexer.weights.get(('alt_265', choice), 1) * self.unlexer.cooldown - if choice == 0: - current = self.columnTypeExpr_ColumnTypeExprSimple() - elif choice == 1: - current = self.columnTypeExpr_ColumnTypeExprParam() - elif choice == 2: - current = self.columnTypeExpr_ColumnTypeExprEnum() - elif choice == 3: - current = self.columnTypeExpr_ColumnTypeExprComplex() - return current - columnTypeExpr.min_depth = 4 - - @depthcontrol - def columnTypeExpr_ColumnTypeExprSimple(self): - current = self.create_node(UnparserRule(name='columnTypeExpr_ColumnTypeExprSimple')) - current += self.identifier() - return current - columnTypeExpr_ColumnTypeExprSimple.min_depth = 3 - - @depthcontrol - def columnTypeExpr_ColumnTypeExprParam(self): - current = self.create_node(UnparserRule(name='columnTypeExpr_ColumnTypeExprParam')) - current += self.identifier() - current += self.unlexer.LPAREN() - current += self.columnParamList() - current += self.unlexer.RPAREN() - return current - columnTypeExpr_ColumnTypeExprParam.min_depth = 4 - - @depthcontrol - def columnTypeExpr_ColumnTypeExprEnum(self): - current = self.create_node(UnparserRule(name='columnTypeExpr_ColumnTypeExprEnum')) - current += self.identifier() - current += self.unlexer.LPAREN() - current += self.enumValue() - if self.unlexer.max_depth >= 3: - for _ in self.zero_or_more(): - current += self.unlexer.COMMA() - current += self.enumValue() - - current += self.unlexer.RPAREN() - return current - columnTypeExpr_ColumnTypeExprEnum.min_depth = 3 - - @depthcontrol - def columnTypeExpr_ColumnTypeExprComplex(self): - current = self.create_node(UnparserRule(name='columnTypeExpr_ColumnTypeExprComplex')) - current += self.identifier() - current += self.unlexer.LPAREN() - current += self.columnTypeExpr() - if self.unlexer.max_depth >= 5: - for _ in self.zero_or_more(): - current += self.unlexer.COMMA() - current += self.columnTypeExpr() - - current += self.unlexer.RPAREN() - return current - columnTypeExpr_ColumnTypeExprComplex.min_depth = 5 - - @depthcontrol - def columnExprList(self): - current = self.create_node(UnparserRule(name='columnExprList')) - current += self.columnExpr() - if self.unlexer.max_depth >= 3: - for _ in self.zero_or_more(): - current += self.unlexer.COMMA() - current += self.columnExpr() - - return current - columnExprList.min_depth = 3 - - @depthcontrol - def columnExpr(self): - current = self.create_node(UnparserRule(name='columnExpr')) - choice = self.choice([0 if [4, 2, 5, 2, 4, 4, 4, 4, 5, 4, 4, 4, 4, 4, 4, 4, 4, 4][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_273', i), 1) for i, w in enumerate([1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1])]) - self.unlexer.weights[('alt_273', choice)] = self.unlexer.weights.get(('alt_273', choice), 1) * self.unlexer.cooldown - if choice == 0: - current = self.columnExpr_ColumnExprLiteral() - elif choice == 1: - current = self.columnExpr_ColumnExprAsterisk() - elif choice == 2: - current = self.columnExpr_ColumnExprTuple() - elif choice == 3: - current = self.columnExpr_ColumnExprArray() - elif choice == 4: - current = self.columnExpr_ColumnExprCase() - elif choice == 5: - current = self.columnExpr_ColumnExprExtract() - elif choice == 6: - current = self.columnExpr_ColumnExprTrim() - elif choice == 7: - current = self.columnExpr_ColumnExprInterval() - elif choice == 8: - current = self.columnExpr_ColumnExprIdentifier() - elif choice == 9: - current = self.columnExpr_ColumnExprFunction() - elif choice == 10: - current = self.columnExpr_ColumnExprArrayAccess() - elif choice == 11: - current = self.columnExpr_ColumnExprTupleAccess() - elif choice == 12: - current = self.columnExpr_ColumnExprUnaryOp() - elif choice == 13: - current = self.columnExpr_ColumnExprIsNull() - elif choice == 14: - current = self.columnExpr_ColumnExprBinaryOp() - elif choice == 15: - current = self.columnExpr_ColumnExprTernaryOp() - elif choice == 16: - current = self.columnExpr_ColumnExprBetween() - elif choice == 17: - current = self.columnExpr_ColumnExprAlias() - return current - columnExpr.min_depth = 2 - - @depthcontrol - def columnExpr_ColumnExprLiteral(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprLiteral')) - current += self.literal() - return current - columnExpr_ColumnExprLiteral.min_depth = 3 - - @depthcontrol - def columnExpr_ColumnExprAsterisk(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprAsterisk')) - current += self.unlexer.ASTERISK() - return current - columnExpr_ColumnExprAsterisk.min_depth = 1 - - @depthcontrol - def columnExpr_ColumnExprTuple(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprTuple')) - current += self.unlexer.LPAREN() - current += self.columnExprList() - current += self.unlexer.RPAREN() - return current - columnExpr_ColumnExprTuple.min_depth = 4 - - @depthcontrol - def columnExpr_ColumnExprArray(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprArray')) - current += self.unlexer.LBRACKET() - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_one(): - current += self.columnExprList() - - current += self.unlexer.RBRACKET() - return current - columnExpr_ColumnExprArray.min_depth = 1 - - @depthcontrol - def columnExpr_ColumnExprCase(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprCase')) - current += self.unlexer.CASE() - if self.unlexer.max_depth >= 3: - for _ in self.zero_or_one(): - current += self.columnExpr() - - if self.unlexer.max_depth >= 0: - for _ in self.one_or_more(): - current += self.unlexer.WHEN() - current += self.columnExpr() - current += self.unlexer.THEN() - current += self.columnExpr() - - if self.unlexer.max_depth >= 3: - for _ in self.zero_or_one(): - current += self.unlexer.ELSE() - current += self.columnExpr() - - current += self.unlexer.END() - return current - columnExpr_ColumnExprCase.min_depth = 3 - - @depthcontrol - def columnExpr_ColumnExprExtract(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprExtract')) - current += self.unlexer.EXTRACT() - current += self.unlexer.LPAREN() - current += self.unlexer.INTERVAL_TYPE() - current += self.unlexer.FROM() - current += self.columnExpr() - current += self.unlexer.RPAREN() - return current - columnExpr_ColumnExprExtract.min_depth = 3 - - @depthcontrol - def columnExpr_ColumnExprTrim(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprTrim')) - current += self.unlexer.TRIM() - current += self.unlexer.LPAREN() - choice = self.choice([0 if [2, 2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_295', i), 1) for i, w in enumerate([1, 1, 1])]) - self.unlexer.weights[('alt_295', choice)] = self.unlexer.weights.get(('alt_295', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.BOTH() - elif choice == 1: - current += self.unlexer.LEADING() - elif choice == 2: - current += self.unlexer.TRAILING() - current += self.unlexer.STRING_LITERAL() - current += self.unlexer.FROM() - current += self.columnExpr() - current += self.unlexer.RPAREN() - return current - columnExpr_ColumnExprTrim.min_depth = 3 - - @depthcontrol - def columnExpr_ColumnExprInterval(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprInterval')) - current += self.unlexer.INTERVAL() - current += self.columnExpr() - current += self.unlexer.INTERVAL_TYPE() - return current - columnExpr_ColumnExprInterval.min_depth = 3 - - @depthcontrol - def columnExpr_ColumnExprIdentifier(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprIdentifier')) - current += self.columnIdentifier() - return current - columnExpr_ColumnExprIdentifier.min_depth = 4 - - @depthcontrol - def columnExpr_ColumnExprFunction(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprFunction')) - current += self.identifier() - if self.unlexer.max_depth >= 1: - for _ in self.zero_or_one(): - current += self.unlexer.LPAREN() - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_one(): - current += self.columnParamList() - - current += self.unlexer.RPAREN() - - current += self.unlexer.LPAREN() - if self.unlexer.max_depth >= 5: - for _ in self.zero_or_one(): - current += self.columnArgList() - - current += self.unlexer.RPAREN() - return current - columnExpr_ColumnExprFunction.min_depth = 3 - - @depthcontrol - def columnExpr_ColumnExprArrayAccess(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprArrayAccess')) - current += self.columnExpr() - current += self.unlexer.LBRACKET() - current += self.columnExpr() - current += self.unlexer.RBRACKET() - return current - columnExpr_ColumnExprArrayAccess.min_depth = 3 - - @depthcontrol - def columnExpr_ColumnExprTupleAccess(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprTupleAccess')) - current += self.columnExpr() - current += self.unlexer.DOT() - current += self.unlexer.INTEGER_LITERAL() - return current - columnExpr_ColumnExprTupleAccess.min_depth = 3 - - @depthcontrol - def columnExpr_ColumnExprUnaryOp(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprUnaryOp')) - current += self.unaryOp() - current += self.columnExpr() - return current - columnExpr_ColumnExprUnaryOp.min_depth = 3 - - @depthcontrol - def columnExpr_ColumnExprIsNull(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprIsNull')) - current += self.columnExpr() - current += self.unlexer.IS() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.NOT() - - current += self.unlexer.NULL_SQL() - return current - columnExpr_ColumnExprIsNull.min_depth = 3 - - @depthcontrol - def columnExpr_ColumnExprBinaryOp(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprBinaryOp')) - current += self.columnExpr() - current += self.binaryOp() - current += self.columnExpr() - return current - columnExpr_ColumnExprBinaryOp.min_depth = 3 - - @depthcontrol - def columnExpr_ColumnExprTernaryOp(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprTernaryOp')) - current += self.columnExpr() - current += self.unlexer.QUERY() - current += self.columnExpr() - current += self.unlexer.COLON() - current += self.columnExpr() - return current - columnExpr_ColumnExprTernaryOp.min_depth = 3 - - @depthcontrol - def columnExpr_ColumnExprBetween(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprBetween')) - current += self.columnExpr() - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.NOT() - - current += self.unlexer.BETWEEN() - current += self.columnExpr() - current += self.unlexer.AND() - current += self.columnExpr() - return current - columnExpr_ColumnExprBetween.min_depth = 3 - - @depthcontrol - def columnExpr_ColumnExprAlias(self): - current = self.create_node(UnparserRule(name='columnExpr_ColumnExprAlias')) - current += self.columnExpr() - current += self.unlexer.AS() - current += self.identifier() - return current - columnExpr_ColumnExprAlias.min_depth = 3 - - @depthcontrol - def columnParamList(self): - current = self.create_node(UnparserRule(name='columnParamList')) - current += self.literal() - if self.unlexer.max_depth >= 3: - for _ in self.zero_or_more(): - current += self.unlexer.COMMA() - current += self.literal() - - return current - columnParamList.min_depth = 3 - - @depthcontrol - def columnArgList(self): - current = self.create_node(UnparserRule(name='columnArgList')) - current += self.columnArgExpr() - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_more(): - current += self.unlexer.COMMA() - current += self.columnArgExpr() - - return current - columnArgList.min_depth = 4 - - @depthcontrol - def columnArgExpr(self): - current = self.create_node(UnparserRule(name='columnArgExpr')) - choice = self.choice([0 if [4, 3][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_306', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_306', choice)] = self.unlexer.weights.get(('alt_306', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.columnLambdaExpr() - elif choice == 1: - current += self.columnExpr() - return current - columnArgExpr.min_depth = 3 - - @depthcontrol - def columnLambdaExpr(self): - current = self.create_node(UnparserRule(name='columnLambdaExpr')) - choice = self.choice([0 if [3, 3][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_309', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_309', choice)] = self.unlexer.weights.get(('alt_309', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.LPAREN() - current += self.identifier() - if self.unlexer.max_depth >= 3: - for _ in self.zero_or_more(): - current += self.unlexer.COMMA() - current += self.identifier() - - current += self.unlexer.RPAREN() - elif choice == 1: - current += self.identifier() - if self.unlexer.max_depth >= 3: - for _ in self.zero_or_more(): - current += self.unlexer.COMMA() - current += self.identifier() - - current += self.unlexer.ARROW() - current += self.columnExpr() - return current - columnLambdaExpr.min_depth = 3 - - @depthcontrol - def columnIdentifier(self): - current = self.create_node(UnparserRule(name='columnIdentifier')) - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_one(): - current += self.tableIdentifier() - current += self.unlexer.DOT() - - current += self.identifier() - return current - columnIdentifier.min_depth = 3 - - @depthcontrol - def tableExpr(self): - current = self.create_node(UnparserRule(name='tableExpr')) - choice = self.choice([0 if [5, 4, 7, 6][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_315', i), 1) for i, w in enumerate([1, 1, 1, 1])]) - self.unlexer.weights[('alt_315', choice)] = self.unlexer.weights.get(('alt_315', choice), 1) * self.unlexer.cooldown - if choice == 0: - current = self.tableExpr_TableExprIdentifier() - elif choice == 1: - current = self.tableExpr_TableExprFunction() - elif choice == 2: - current = self.tableExpr_TableExprSubquery() - elif choice == 3: - current = self.tableExpr_TableExprAlias() - return current - tableExpr.min_depth = 4 - - @depthcontrol - def tableExpr_TableExprIdentifier(self): - current = self.create_node(UnparserRule(name='tableExpr_TableExprIdentifier')) - current += self.tableIdentifier() - return current - tableExpr_TableExprIdentifier.min_depth = 4 - - @depthcontrol - def tableExpr_TableExprFunction(self): - current = self.create_node(UnparserRule(name='tableExpr_TableExprFunction')) - current += self.identifier() - current += self.unlexer.LPAREN() - if self.unlexer.max_depth >= 5: - for _ in self.zero_or_one(): - current += self.tableArgList() - - current += self.unlexer.RPAREN() - return current - tableExpr_TableExprFunction.min_depth = 3 - - @depthcontrol - def tableExpr_TableExprSubquery(self): - current = self.create_node(UnparserRule(name='tableExpr_TableExprSubquery')) - current += self.unlexer.LPAREN() - current += self.selectUnionStmt() - current += self.unlexer.RPAREN() - return current - tableExpr_TableExprSubquery.min_depth = 6 - - @depthcontrol - def tableExpr_TableExprAlias(self): - current = self.create_node(UnparserRule(name='tableExpr_TableExprAlias')) - current += self.tableExpr() - current += self.unlexer.AS() - current += self.identifier() - return current - tableExpr_TableExprAlias.min_depth = 5 - - @depthcontrol - def tableIdentifier(self): - current = self.create_node(UnparserRule(name='tableIdentifier')) - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_one(): - current += self.databaseIdentifier() - current += self.unlexer.DOT() - - current += self.identifier() - return current - tableIdentifier.min_depth = 3 - - @depthcontrol - def tableArgList(self): - current = self.create_node(UnparserRule(name='tableArgList')) - current += self.tableArgExpr() - if self.unlexer.max_depth >= 4: - for _ in self.zero_or_more(): - current += self.unlexer.COMMA() - current += self.tableArgExpr() - - return current - tableArgList.min_depth = 4 - - @depthcontrol - def tableArgExpr(self): - current = self.create_node(UnparserRule(name='tableArgExpr')) - choice = self.choice([0 if [3, 4][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_323', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_323', choice)] = self.unlexer.weights.get(('alt_323', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.literal() - elif choice == 1: - current += self.tableIdentifier() - return current - tableArgExpr.min_depth = 3 - - @depthcontrol - def databaseIdentifier(self): - current = self.create_node(UnparserRule(name='databaseIdentifier')) - current += self.identifier() - return current - databaseIdentifier.min_depth = 3 - - @depthcontrol - def literal(self): - current = self.create_node(UnparserRule(name='literal')) - choice = self.choice([0 if [2, 2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_326', i), 1) for i, w in enumerate([1, 1, 1])]) - self.unlexer.weights[('alt_326', choice)] = self.unlexer.weights.get(('alt_326', choice), 1) * self.unlexer.cooldown - if choice == 0: - if self.unlexer.max_depth >= 1: - for _ in self.zero_or_one(): - choice = self.choice([0 if [1, 1][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_331', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_331', choice)] = self.unlexer.weights.get(('alt_331', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.PLUS() - elif choice == 1: - current += self.unlexer.DASH() - - choice = self.choice([0 if [3, 2, 2, 2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_334', i), 1) for i, w in enumerate([1, 1, 1, 1, 1])]) - self.unlexer.weights[('alt_334', choice)] = self.unlexer.weights.get(('alt_334', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.FLOATING_LITERAL() - elif choice == 1: - current += self.unlexer.HEXADECIMAL_LITERAL() - elif choice == 2: - current += self.unlexer.INTEGER_LITERAL() - elif choice == 3: - current += self.unlexer.INF() - elif choice == 4: - current += self.unlexer.NAN_SQL() - elif choice == 1: - current += self.unlexer.STRING_LITERAL() - elif choice == 2: - current += self.unlexer.NULL_SQL() - return current - literal.min_depth = 2 - - @depthcontrol - def keyword(self): - current = self.create_node(UnparserRule(name='keyword')) - choice = self.choice([0 if [2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_340', i), 1) for i, w in enumerate([1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1])]) - self.unlexer.weights[('alt_340', choice)] = self.unlexer.weights.get(('alt_340', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.ALIAS() - elif choice == 1: - current += self.unlexer.ALL() - elif choice == 2: - current += self.unlexer.AND() - elif choice == 3: - current += self.unlexer.ANTI() - elif choice == 4: - current += self.unlexer.ANY() - elif choice == 5: - current += self.unlexer.ARRAY() - elif choice == 6: - current += self.unlexer.AS() - elif choice == 7: - current += self.unlexer.ASCENDING() - elif choice == 8: - current += self.unlexer.ASOF() - elif choice == 9: - current += self.unlexer.BETWEEN() - elif choice == 10: - current += self.unlexer.BOTH() - elif choice == 11: - current += self.unlexer.BY() - elif choice == 12: - current += self.unlexer.CASE() - elif choice == 13: - current += self.unlexer.CAST() - elif choice == 14: - current += self.unlexer.CLUSTER() - elif choice == 15: - current += self.unlexer.COLLATE() - elif choice == 16: - current += self.unlexer.CREATE() - elif choice == 17: - current += self.unlexer.CROSS() - elif choice == 18: - current += self.unlexer.DAY() - elif choice == 19: - current += self.unlexer.DATABASE() - elif choice == 20: - current += self.unlexer.DEFAULT() - elif choice == 21: - current += self.unlexer.DELETE() - elif choice == 22: - current += self.unlexer.DESCENDING() - elif choice == 23: - current += self.unlexer.DISK() - elif choice == 24: - current += self.unlexer.DISTINCT() - elif choice == 25: - current += self.unlexer.DROP() - elif choice == 26: - current += self.unlexer.ELSE() - elif choice == 27: - current += self.unlexer.END() - elif choice == 28: - current += self.unlexer.ENGINE() - elif choice == 29: - current += self.unlexer.EXISTS() - elif choice == 30: - current += self.unlexer.EXTRACT() - elif choice == 31: - current += self.unlexer.FINAL() - elif choice == 32: - current += self.unlexer.FIRST() - elif choice == 33: - current += self.unlexer.FORMAT() - elif choice == 34: - current += self.unlexer.FROM() - elif choice == 35: - current += self.unlexer.FULL() - elif choice == 36: - current += self.unlexer.GLOBAL() - elif choice == 37: - current += self.unlexer.GROUP() - elif choice == 38: - current += self.unlexer.HAVING() - elif choice == 39: - current += self.unlexer.HOUR() - elif choice == 40: - current += self.unlexer.IF() - elif choice == 41: - current += self.unlexer.IN() - elif choice == 42: - current += self.unlexer.INNER() - elif choice == 43: - current += self.unlexer.INSERT() - elif choice == 44: - current += self.unlexer.INTERVAL() - elif choice == 45: - current += self.unlexer.INTO() - elif choice == 46: - current += self.unlexer.IS() - elif choice == 47: - current += self.unlexer.JOIN() - elif choice == 48: - current += self.unlexer.KEY() - elif choice == 49: - current += self.unlexer.LAST() - elif choice == 50: - current += self.unlexer.LEADING() - elif choice == 51: - current += self.unlexer.LEFT() - elif choice == 52: - current += self.unlexer.LIKE() - elif choice == 53: - current += self.unlexer.LIMIT() - elif choice == 54: - current += self.unlexer.LOCAL() - elif choice == 55: - current += self.unlexer.MATERIALIZED() - elif choice == 56: - current += self.unlexer.MINUTE() - elif choice == 57: - current += self.unlexer.MONTH() - elif choice == 58: - current += self.unlexer.NOT() - elif choice == 59: - current += self.unlexer.NULLS() - elif choice == 60: - current += self.unlexer.OFFSET() - elif choice == 61: - current += self.unlexer.ON() - elif choice == 62: - current += self.unlexer.OR() - elif choice == 63: - current += self.unlexer.ORDER() - elif choice == 64: - current += self.unlexer.OUTER() - elif choice == 65: - current += self.unlexer.OUTFILE() - elif choice == 66: - current += self.unlexer.PARTITION() - elif choice == 67: - current += self.unlexer.PREWHERE() - elif choice == 68: - current += self.unlexer.PRIMARY() - elif choice == 69: - current += self.unlexer.QUARTER() - elif choice == 70: - current += self.unlexer.RIGHT() - elif choice == 71: - current += self.unlexer.SAMPLE() - elif choice == 72: - current += self.unlexer.SECOND() - elif choice == 73: - current += self.unlexer.SEMI() - elif choice == 74: - current += self.unlexer.SET() - elif choice == 75: - current += self.unlexer.SETTINGS() - elif choice == 76: - current += self.unlexer.TABLE() - elif choice == 77: - current += self.unlexer.TEMPORARY() - elif choice == 78: - current += self.unlexer.THEN() - elif choice == 79: - current += self.unlexer.TOTALS() - elif choice == 80: - current += self.unlexer.TRAILING() - elif choice == 81: - current += self.unlexer.TRIM() - elif choice == 82: - current += self.unlexer.TO() - elif choice == 83: - current += self.unlexer.TTL() - elif choice == 84: - current += self.unlexer.UNION() - elif choice == 85: - current += self.unlexer.USING() - elif choice == 86: - current += self.unlexer.VALUES() - elif choice == 87: - current += self.unlexer.VOLUME() - elif choice == 88: - current += self.unlexer.WEEK() - elif choice == 89: - current += self.unlexer.WHEN() - elif choice == 90: - current += self.unlexer.WHERE() - elif choice == 91: - current += self.unlexer.WITH() - elif choice == 92: - current += self.unlexer.YEAR() - return current - keyword.min_depth = 2 - - @depthcontrol - def identifier(self): - current = self.create_node(UnparserRule(name='identifier')) - choice = self.choice([0 if [2, 3, 3][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_434', i), 1) for i, w in enumerate([1, 1, 1])]) - self.unlexer.weights[('alt_434', choice)] = self.unlexer.weights.get(('alt_434', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.IDENTIFIER() - elif choice == 1: - current += self.unlexer.INTERVAL_TYPE() - elif choice == 2: - current += self.keyword() - return current - identifier.min_depth = 2 - - @depthcontrol - def unaryOp(self): - current = self.create_node(UnparserRule(name='unaryOp')) - choice = self.choice([0 if [1, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_438', i), 1) for i, w in enumerate([1, 1])]) - self.unlexer.weights[('alt_438', choice)] = self.unlexer.weights.get(('alt_438', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.DASH() - elif choice == 1: - current += self.unlexer.NOT() - return current - unaryOp.min_depth = 1 - - @depthcontrol - def binaryOp(self): - current = self.create_node(UnparserRule(name='binaryOp')) - choice = self.choice([0 if [1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 2, 2, 2, 2][i] > self.unlexer.max_depth else w * self.unlexer.weights.get(('alt_441', i), 1) for i, w in enumerate([1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1])]) - self.unlexer.weights[('alt_441', choice)] = self.unlexer.weights.get(('alt_441', choice), 1) * self.unlexer.cooldown - if choice == 0: - current += self.unlexer.CONCAT() - elif choice == 1: - current += self.unlexer.ASTERISK() - elif choice == 2: - current += self.unlexer.SLASH() - elif choice == 3: - current += self.unlexer.PLUS() - elif choice == 4: - current += self.unlexer.DASH() - elif choice == 5: - current += self.unlexer.PERCENT() - elif choice == 6: - current += self.unlexer.EQ_DOUBLE() - elif choice == 7: - current += self.unlexer.EQ_SINGLE() - elif choice == 8: - current += self.unlexer.NOT_EQ() - elif choice == 9: - current += self.unlexer.LE() - elif choice == 10: - current += self.unlexer.GE() - elif choice == 11: - current += self.unlexer.LT() - elif choice == 12: - current += self.unlexer.GT() - elif choice == 13: - current += self.unlexer.AND() - elif choice == 14: - current += self.unlexer.OR() - elif choice == 15: - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.NOT() - - current += self.unlexer.LIKE() - elif choice == 16: - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.GLOBAL() - - if self.unlexer.max_depth >= 2: - for _ in self.zero_or_one(): - current += self.unlexer.NOT() - - current += self.unlexer.IN() - return current - binaryOp.min_depth = 1 - - @depthcontrol - def enumValue(self): - current = self.create_node(UnparserRule(name='enumValue')) - current += self.unlexer.STRING_LITERAL() - current += self.unlexer.EQ_SINGLE() - current += self.unlexer.INTEGER_LITERAL() - return current - enumValue.min_depth = 2 - - default_rule = queryList - diff --git a/utils/grammar-fuzzer/README.md b/utils/grammar-fuzzer/README.md deleted file mode 100644 index b3f233c8648..00000000000 --- a/utils/grammar-fuzzer/README.md +++ /dev/null @@ -1,41 +0,0 @@ -How to use Fuzzer -=== - -The fuzzer consists of auto-generated files: - - ClickHouseUnlexer.py - ClickHouseUnparser.py - -They are generated from grammar files (.g4) using Grammarinator: - - pip3 install grammarinator - grammarinator-process ClickHouseLexer.g4 ClickHouseParser.g4 -o fuzzer/ - -Then you can generate test input for ClickHouse client: - - cd fuzzer - grammarinator-generate \ - -r query_list \ # top-level rule - -o /tmp/sql_test_%d.sql \ # template for output test names - -n 10 \ # number of tests - -c 0.3 \ - -d 20 \ # depth of recursion - -p ClickHouseUnparser.py -l ClickHouseUnlexer.py \ # auto-generated unparser and unlexer - --test-transformers SpaceTransformer.single_line_whitespace \ # transform function to insert whitespace - -For more details see `grammarinator-generate --help`. As a test-transformer function also can be used `SpaceTransformer.multi_line_transformer` - both functions reside in `fuzzer/SpaceTransformer.py` file. - - -Parsing steps -=== - -1. Replace all operators with corresponding functions. -2. Replace all asterisks with columns - if it's inside function call, then expand it as multiple arguments. Warn about undeterministic invocations when functions have positional arguments. - -Old vs. new parser -=== - -- `a as b [c]` - accessing aliased array expression is not possible. -- `a as b . 1` - accessing aliased tuple expression is not possible. -- `between a is not null and b` - `between` operator should have lower priority than `is null`. -- `*.1` - accessing asterisk tuple expression is not possible. diff --git a/utils/grammar-fuzzer/SpaceTransformer.py b/utils/grammar-fuzzer/SpaceTransformer.py deleted file mode 100644 index ad96845c7e2..00000000000 --- a/utils/grammar-fuzzer/SpaceTransformer.py +++ /dev/null @@ -1,38 +0,0 @@ -# -*- coding: utf-8 -*- - -from grammarinator.runtime.tree import * - -from itertools import tee, islice, zip_longest -import random - - -def single_line_whitespace(node): - return _whitespace(node, ' \t') - - -def multi_line_whitespace(node): - return _whitespace(node, ' \t\r\n') - - -def _whitespace(node, symbols): - for child in node.children: - _whitespace(child, symbols) - - # helper function to look ahead one child - def with_next(iterable): - items, nexts = tee(iterable, 2) - nexts = islice(nexts, 1, None) - return zip_longest(items, nexts) - - if isinstance(node, UnparserRule): - new_children = [] - for child, next_child in with_next(node.children): - if (not next_child or - next_child and isinstance(next_child, UnlexerRule) and next_child.name == 'DOT' or - isinstance(child, UnlexerRule) and child.name == 'DOT'): - new_children.append(child) - else: - new_children.extend([child, UnlexerRule(src=random.choice(symbols))]) - node.children = new_children - - return node diff --git a/utils/grammar-fuzzer/__init__.py b/utils/grammar-fuzzer/__init__.py deleted file mode 100644 index 40a96afc6ff..00000000000 --- a/utils/grammar-fuzzer/__init__.py +++ /dev/null @@ -1 +0,0 @@ -# -*- coding: utf-8 -*- diff --git a/utils/junit_to_html/junit-noframes.xsl b/utils/junit_to_html/junit-noframes.xsl deleted file mode 100644 index ae70e230ef6..00000000000 --- a/utils/junit_to_html/junit-noframes.xsl +++ /dev/null @@ -1,390 +0,0 @@ - - - - - - - - Test Results - - - - - - - - -
- - - - -
- - - - - - - - - - - - -

-
- - - - - - - - - -
-

- Back to top - - -

Summary

- - - - - - - - - - - - - - - - - Failure - Error - - - - - - - - -
TestsFailuresErrorsSuccess rateTime
- - - - - - - -
- - - - -
- Note: failures are anticipated and checked for with assertions while errors are unanticipated. -
-
- - - - -

Test Results

-
-
- - - Name - Tests - Errors - Failures - Time(s) - - - - - - Name - Tests - Errors - Failures - Time(s) - Time Stamp - Host - - - - - - Name - Status - Type - Time(s) - - - - - - - - - Failure - Error - - - - - - - - - - - - - - - - - - - - - Error - Failure - TableRowColor - - - - - - Failure - - - - Error - - - - Success - - - - - - - - - - - - -

- - - - - -
- - - -

- - - - - -
- - - - N/A - - - - - - -

- at line - - - , column - - -
-
-
- - - - - - - - - - 32 - - - - - - - - - - - - -
- - - -
- - -
- - - -
- - - -
-
- - - - - - - - -
diff --git a/utils/junit_to_html/junit_to_html b/utils/junit_to_html/junit_to_html deleted file mode 100755 index 132763c7d4c..00000000000 --- a/utils/junit_to_html/junit_to_html +++ /dev/null @@ -1,86 +0,0 @@ -#!/usr/bin/env python3 -# -*- coding: utf-8 -*- -import os -import lxml.etree as etree -import json -import argparse - -def export_testcases_json(report, path): - with open(os.path.join(path, "cases.jer"), "w") as testcases_file: - for testsuite in report.getroot(): - for testcase in testsuite: - row = {} - row["hostname"] = testsuite.get("hostname") - row["suite"] = testsuite.get("name") - row["suite_duration"] = testsuite.get("time") - row["timestamp"] = testsuite.get("timestamp") - row["testname"] = testcase.get("name") - row["classname"] = testcase.get("classname") - row["file"] = testcase.get("file") - row["line"] = testcase.get("line") - row["duration"] = testcase.get("time") - for el in testcase: - if el.tag == "system-err": - row["stderr"] = el.text - else: - row["stderr"] = "" - - if el.tag == "system-out": - row["stdout"] = el.text - else: - row["stdout"] = "" - - json.dump(row, testcases_file) - testcases_file.write("\n") - -def export_testsuites_json(report, path): - with open(os.path.join(path, "suites.jer"), "w") as testsuites_file: - for testsuite in report.getroot(): - row = {} - row["suite"] = testsuite.get("name") - row["errors"] = testsuite.get("errors") - row["failures"] = testsuite.get("failures") - row["hostname"] = testsuite.get("hostname") - row["skipped"] = testsuite.get("skipped") - row["duration"] = testsuite.get("time") - row["timestamp"] = testsuite.get("timestamp") - json.dump(row, testsuites_file) - testsuites_file.write("\n") - - -def _convert_junit_to_html(junit_path, result_path, export_cases, export_suites): - with open(os.path.join(os.path.dirname(__file__), "junit-noframes.xsl")) as xslt_file: - junit_to_html_xslt = etree.parse(xslt_file) - if not os.path.exists(result_path): - os.makedirs(result_path) - - with open(junit_path) as junit_file: - junit_xml = etree.parse(junit_file) - - if export_suites: - export_testsuites_json(junit_xml, result_path) - if export_cases: - export_testcases_json(junit_xml, result_path) - transform = etree.XSLT(junit_to_html_xslt) - html = etree.tostring(transform(junit_xml), encoding="utf-8") - - with open(os.path.join(result_path, "result.html"), "w") as html_file: - html_file.write(html) - -if __name__ == "__main__": - - parser = argparse.ArgumentParser(description='Convert JUnit XML.') - parser.add_argument('junit', help='path to junit.xml report') - parser.add_argument('result_dir', nargs='?', help='directory for result files. Default to junit.xml directory') - parser.add_argument('--export-cases', help='Export JSONEachRow result for testcases to upload in CI', action='store_true') - parser.add_argument('--export-suites', help='Export JSONEachRow result for testsuites to upload in CI', action='store_true') - - args = parser.parse_args() - - junit_path = args.junit - if args.result_dir: - result_path = args.result_dir - else: - result_path = os.path.dirname(junit_path) - print("junit_path: {}, result_path: {}, export cases:{}, export suites: {}".format(junit_path, result_path, args.export_cases, args.export_suites)) - _convert_junit_to_html(junit_path, result_path, args.export_cases, args.export_suites) diff --git a/utils/link-validate/link-validate.sh b/utils/link-validate/link-validate.sh deleted file mode 100755 index 2d8d57b95fc..00000000000 --- a/utils/link-validate/link-validate.sh +++ /dev/null @@ -1,42 +0,0 @@ -#/bin/sh -# -# This script is used to validate the shared libraries -# -# Authors: FoundationDB team, https://github.com/apple/foundationdb/blame/master/build/link-validate.sh -# License: Apache License 2.0 - -verlte() { - [ "$1" = "`echo -e "$1\n$2" | sort -V | head -n1`" ] -} - -ALLOWED_SHARED_LIBS=("libdl.so.2" "libpthread.so.0" "librt.so.1" "libm.so.6" "libc.so.6" "ld-linux-x86-64.so.2") - -if [ "$#" -lt 1 ]; then - echo "USAGE: link-validate.sh BINNAME GLIBC_VERSION" - exit 1 -fi - -# Step 1: glibc version - -for i in $(objdump -T "$1" | awk '{print $5}' | grep GLIBC | sed 's/ *$//g' | sed 's/GLIBC_//' | sort | uniq); do - if ! verlte "$i" "${2:-2.10}"; then - echo "Dependency on newer libc detected: $i" - exit 1 - fi -done - -# Step 2: Other dynamic dependencies - -for j in $(objdump -p "$1" | grep NEEDED | awk '{print $2}'); do - PRESENT=0 - for k in ${ALLOWED_SHARED_LIBS[@]}; do - if [[ "$k" == "$j" ]]; then - PRESENT=1 - break - fi - done - if ! [[ $PRESENT == 1 ]]; then - echo "Unexpected shared object dependency detected: $j" - exit 1 - fi -done diff --git a/utils/upload_test_results/README.md b/utils/upload_test_results/README.md deleted file mode 100644 index e6b361081a2..00000000000 --- a/utils/upload_test_results/README.md +++ /dev/null @@ -1,34 +0,0 @@ -## Tool to upload results to CI ClickHouse - -Currently allows to upload results from `junit_to_html` tool to ClickHouse CI - -``` -usage: upload_test_results [-h] --sha SHA --pr PR --file FILE --type - {suites,cases} [--user USER] --password PASSWORD - [--ca-cert CA_CERT] [--host HOST] [--db DB] - -Upload test result to CI ClickHouse. - -optional arguments: - -h, --help show this help message and exit - --sha SHA sha of current commit - --pr PR pr of current commit. 0 for master - --file FILE file to upload - --type {suites,cases} - Export type - --user USER user name - --password PASSWORD password - --ca-cert CA_CERT CA certificate path - --host HOST CI ClickHouse host - --db DB CI ClickHouse database name -``` - -$ ./upload_test_results --sha "cf7eaee3301d4634acdacbfa308ddbe0cc6a061d" --pr "0" --file xyz/cases.jer --type cases --password $PASSWD - -CI checks has single commit sha and pr identifier. -While uploading your local results for testing purposes try to use correct sha and pr. - -CA Certificate for ClickHouse CI can be obtained from Yandex.Cloud where CI database is hosted -``` bash -wget "https://storage.yandexcloud.net/cloud-certs/CA.pem" -O YandexInternalRootCA.crt -``` \ No newline at end of file diff --git a/utils/upload_test_results/upload_test_results b/utils/upload_test_results/upload_test_results deleted file mode 100755 index 5916d0d85e8..00000000000 --- a/utils/upload_test_results/upload_test_results +++ /dev/null @@ -1,127 +0,0 @@ -#!/usr/bin/env python3 -import requests -import argparse - -# CREATE TABLE test_suites -# ( -# sha String, -# pr UInt16, -# suite String, -# errors UInt16, -# failures UInt16, -# hostname String, -# skipped UInt16, -# duration Double, -# timestamp DateTime -# ) ENGINE = MergeTree ORDER BY tuple(timestamp, suite); - -QUERY_SUITES="INSERT INTO test_suites "\ - "SELECT '{sha}' AS sha, "\ - "{pr} AS pr, "\ - "suite, "\ - "errors, "\ - "failures, "\ - "hostname, "\ - "skipped, "\ - "duration, "\ - "timestamp "\ - "FROM input('"\ - "suite String, "\ - "errors UInt16, "\ - "failures UInt16, "\ - "hostname String, "\ - "skipped UInt16, "\ - "duration Double, "\ - "timestamp DateTime"\ - "') FORMAT JSONEachRow" - -# CREATE TABLE test_cases -# ( -# sha String, -# pr UInt16, -# hostname String, -# suite String, -# timestamp DateTime, -# testname String, -# classname String, -# file String, -# line UInt16, -# duration Double, -# suite_duration Double, -# stderr String, -# stdout String -# ) ENGINE = MergeTree ORDER BY tuple(timestamp, testname); - -QUERY_CASES="INSERT INTO test_cases "\ - "SELECT '{sha}' AS sha, "\ - "{pr} AS pr, "\ - "hostname, "\ - "suite, "\ - "timestamp, "\ - "testname, "\ - "classname, "\ - "file, "\ - "line, "\ - "duration, "\ - "suite_duration, "\ - "stderr,"\ - "stdout "\ - "FROM input('"\ - "hostname String, "\ - "suite String, "\ - "timestamp DateTime, "\ - "testname String, "\ - "classname String, "\ - "file String, "\ - "line UInt16, "\ - "duration Double, "\ - "suite_duration Double, "\ - "stderr String, "\ - "stdout String"\ - "') FORMAT JSONEachRow" - - -def upload_request(sha, pr, file, q_type, user, password, ca_cert, host, db): - with open(file) as upload_f: - query = QUERY_SUITES if q_type=="suites" else QUERY_CASES - query = query.format(sha=sha, pr=pr) - url = 'https://{host}:8443/?database={db}&query={query}&date_time_input_format=best_effort'.format( - host=host, - db=db, - query=query - ) - data=upload_f - auth = { - 'X-ClickHouse-User': user, - 'X-ClickHouse-Key': password, - } - - print(query); - - res = requests.post( - url, - data=data, - headers=auth, - verify=ca_cert) - res.raise_for_status() - return res.text - -if __name__ == "__main__": - - parser = argparse.ArgumentParser(description='Upload test result to CI ClickHouse.') - parser.add_argument('--sha', help='sha of current commit', type=str, required=True) - parser.add_argument('--pr', help='pr of current commit. 0 for master', type=int, required=True) - parser.add_argument('--file', help='file to upload', required=True) - parser.add_argument('--type', help='Export type', choices=['suites', 'cases'] , required=True) - parser.add_argument('--user', help='user name', type=str, default="clickhouse-ci") - parser.add_argument('--password', help='password', type=str, required=True) - parser.add_argument('--ca-cert', help='CA certificate path', type=str, default="/usr/local/share/ca-certificates/YandexInternalRootCA.crt") - parser.add_argument('--host', help='CI ClickHouse host', type=str, default="c1a-ity5agjmuhyu6nu9.mdb.yandexcloud.net") - parser.add_argument('--db', help='CI ClickHouse database name', type=str, default="clickhouse-ci") - - args = parser.parse_args() - - print((upload_request(args.sha, args.pr, args.file, args.type, args.user, args.password, args.ca_cert, args.host, args.db))) - - - From 54b2a42530c687068e33428dd6ee713296a9ee98 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 27 Dec 2021 18:02:02 +0800 Subject: [PATCH 0418/1260] update variable names --- src/Common/LRUCache.h | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 049d07c74f4..3f13b4352c3 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -148,20 +148,20 @@ public: /// Insert the new value only if the token is still in present in insert_tokens. /// (The token may be absent because of a concurrent reset() call). bool is_value_loaded = false; - bool is_value_updated = false; + bool is_value_loaded_and_set = false; auto token_it = insert_tokens.find(key); if (token_it != insert_tokens.end() && token_it->second.get() == token) { // setImpl() may fail, but the final behavior seems not be affected // next call of getOrTrySet() will still call load_func() - is_value_updated = setImpl(key, token->value, cache_lock); + is_value_loaded_and_set = setImpl(key, token->value, cache_lock); is_value_loaded = true; } if (!token->cleaned_up) token_holder.cleanup(token_lock, cache_lock); - return {token->value, is_value_loaded, is_value_updated}; + return {token->value, is_value_loaded, is_value_loaded_and_set}; } /// If key is not in cache or the element can be released, return is true. otherwise, return is false @@ -363,11 +363,7 @@ private: else { if (!evict_policy.canRelease(cell.value)) - { - // the old value is referred by someone, cannot release now - // in default policy, it is always true. return false; - } evict_policy.release(cell.value); // release the old value. this action is empty in default policy. current_size -= cell.size; queue.splice(queue.end(), queue, cell.queue_iterator); @@ -377,6 +373,7 @@ private: cell.size = cell.value ? weight_function(*cell.value) : 0; current_size += cell.size; + removeOverflow(); return true; } From 4c7affb64acfe51ee9cedfc310c51e9099f2b703 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 27 Dec 2021 18:02:53 +0800 Subject: [PATCH 0419/1260] Better cmake script for azure blob --- cmake/find/blob_storage.cmake | 37 +++++++++++++++++------------------ 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/cmake/find/blob_storage.cmake b/cmake/find/blob_storage.cmake index 74a907da7db..271b1e595a6 100644 --- a/cmake/find/blob_storage.cmake +++ b/cmake/find/blob_storage.cmake @@ -7,24 +7,23 @@ option(USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY if (ENABLE_AZURE_BLOB_STORAGE) set(USE_AZURE_BLOB_STORAGE 1) set(AZURE_BLOB_STORAGE_LIBRARY azure_sdk) + + if ((NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/azure/sdk" + OR NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/azure/cmake-modules") + AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) + message (WARNING "submodule contrib/azure is missing. to fix try run: \n git submodule update --init") + set(USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY OFF) + set(USE_AZURE_BLOB_STORAGE 0) + endif () + + if (NOT USE_INTERNAL_SSL_LIBRARY AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) + message (FATAL_ERROR "Currently Blob Storage support can be built only with internal SSL library") + endif() + + if (NOT USE_INTERNAL_CURL AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) + message (FATAL_ERROR "Currently Blob Storage support can be built only with internal curl library") + endif() + endif() -if ((NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/azure/sdk" - OR NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/azure/cmake-modules") - AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) - message (WARNING "submodule contrib/azure is missing. to fix try run: \n git submodule update --init") - set(USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY OFF) - set(USE_AZURE_BLOB_STORAGE 0) -endif () - -if (NOT USE_INTERNAL_SSL_LIBRARY AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) - message (FATAL_ERROR "Currently Blob Storage support can be built only with internal SSL library") -endif() - -if (NOT USE_INTERNAL_CURL AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) - message (FATAL_ERROR "Currently Blob Storage support can be built only with internal curl library") -endif() - -if (USE_AZURE_BLOB_STORAGE) - message (STATUS "Using Azure Blob Storage - ${USE_AZURE_BLOB_STORAGE}") -endif() +message (STATUS "Using Azure Blob Storage - ${USE_AZURE_BLOB_STORAGE}") From f86e3ef1ae24d2fbb5179900136d53ea17c60eb7 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 27 Dec 2021 18:12:40 +0800 Subject: [PATCH 0420/1260] update the evict action when overwrite a key --- src/Common/LRUCache.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 3f13b4352c3..700ef3b0ceb 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -339,10 +339,10 @@ private: std::forward_as_tuple()); Cell & cell = it->second; + auto value_weight = mapped ? weight_function(*mapped) : 0; if (inserted) { - auto value_weight = mapped ? weight_function(*mapped) : 0; if (!removeOverflow(value_weight)) { // cannot find enough space to put in the new value @@ -364,6 +364,8 @@ private: { if (!evict_policy.canRelease(cell.value)) return false; + if (value_weight > cell.size && !removeOverflow(value_weight - cell.size)) + return false; evict_policy.release(cell.value); // release the old value. this action is empty in default policy. current_size -= cell.size; queue.splice(queue.end(), queue, cell.queue_iterator); @@ -373,7 +375,6 @@ private: cell.size = cell.value ? weight_function(*cell.value) : 0; current_size += cell.size; - removeOverflow(); return true; } From 493f71470ba861020ab92a4dded02fc736f58bea Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 27 Dec 2021 13:14:29 +0300 Subject: [PATCH 0421/1260] Update client-libraries.md --- docs/en/interfaces/third-party/client-libraries.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/third-party/client-libraries.md b/docs/en/interfaces/third-party/client-libraries.md index 342b1c9a496..a116c8e2222 100644 --- a/docs/en/interfaces/third-party/client-libraries.md +++ b/docs/en/interfaces/third-party/client-libraries.md @@ -6,7 +6,7 @@ toc_title: Client Libraries # Client Libraries from Third-party Developers {#client-libraries-from-third-party-developers} !!! warning "Disclaimer" - Yandex does **not** maintain the libraries listed below and hasn’t done any extensive testing to ensure their quality. + ClickHouse Inc does **not** maintain the libraries listed below and hasn’t done any extensive testing to ensure their quality. - Python - [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm) From 33fd1a44ace1b900e89f817b88ede2f3439b972d Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 27 Dec 2021 13:26:16 +0300 Subject: [PATCH 0422/1260] Fix cancel lambda --- .../cancel_and_rerun_workflow_lambda/app.py | 20 +++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py index bd1dc394086..b79eb292dc6 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ b/tests/ci/cancel_and_rerun_workflow_lambda/app.py @@ -11,7 +11,6 @@ import boto3 NEED_RERUN_OR_CANCELL_WORKFLOWS = { 13241696, # PR 15834118, # Docs - 15522500, # MasterCI 15516108, # ReleaseCI 15797242, # BackportPR } @@ -86,10 +85,23 @@ WorkflowDescription = namedtuple('WorkflowDescription', def get_workflows_description_for_pull_request(pull_request_event): head_branch = pull_request_event['head']['ref'] print("PR", pull_request_event['number'], "has head ref", head_branch) - workflows = _exec_get_with_retry(API_URL + f"/actions/runs?branch={head_branch}") + workflows_data = [] + workflows = _exec_get_with_retry(API_URL + f"/actions/runs?branch={head_branch}&event=pull_request&page=1") + workflows_data += workflows['workflow_runs'] + i = 2 + while len(workflows['workflow_runs']) > 0: + workflows = _exec_get_with_retry(API_URL + f"/actions/runs?branch={head_branch}&event=pull_request&page={i}") + workflows_data += workflows['workflow_runs'] + i += 1 + if i > 30: + print("Too many workflows found") + break + workflow_descriptions = [] - for workflow in workflows['workflow_runs']: - if workflow['workflow_id'] in NEED_RERUN_OR_CANCELL_WORKFLOWS: + for workflow in workflows_data: + # unfortunately we cannot filter workflows from forks in request to API so doing it manually + if (workflow['head_repository']['full_name'] == pull_request_event['head']['repo']['full_name'] + and workflow['workflow_id'] in NEED_RERUN_OR_CANCELL_WORKFLOWS): workflow_descriptions.append(WorkflowDescription( run_id=workflow['id'], status=workflow['status'], From 79e50a9ff6e1c69650377353eed3384f95cbeabe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Dec 2021 13:58:53 +0300 Subject: [PATCH 0423/1260] Update benchmark info --- website/benchmark/hardware/results/xeon_gold_6266.json | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/website/benchmark/hardware/results/xeon_gold_6266.json b/website/benchmark/hardware/results/xeon_gold_6266.json index 4283b711091..3c8ea24e838 100644 --- a/website/benchmark/hardware/results/xeon_gold_6266.json +++ b/website/benchmark/hardware/results/xeon_gold_6266.json @@ -1,7 +1,7 @@ [ { - "system": "Xeon Gold 6266C, 3GHz, 4vCPU", - "system_full": "Xeon Gold 6266C, 3GHz, 4vCPU, 16GiB RAM, vda1 40GB", + "system": "Huawei Cloud c6.xlarge.4, 4vCPUs, 16 GiB" + "system_full": "Huawei Cloud c6.xlarge.4, Xeon Gold 6266C, 3GHz, 4vCPU, 16GiB RAM, vda1 40GB", "cpu_vendor": "Intel", "cpu_model": "Xeon Gold 6266C", "time": "2021-12-23 00:00:00", From cecfa36cf61cf2ca793b85bac8a943ebfb7ae8eb Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 27 Dec 2021 14:10:06 +0300 Subject: [PATCH 0424/1260] Update website/blog/en/2021/tests-visualization.md Co-authored-by: alesapin --- website/blog/en/2021/tests-visualization.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/blog/en/2021/tests-visualization.md b/website/blog/en/2021/tests-visualization.md index b540f275bd9..51e6c46632c 100644 --- a/website/blog/en/2021/tests-visualization.md +++ b/website/blog/en/2021/tests-visualization.md @@ -14,7 +14,7 @@ Tests are good (in general). Good test suite allows fast development iterations, Some tests can be flaky. The reasons for flakiness are uncountable - most of them are simple timing issues in the test script itself, but sometimes if a test has failed one of a thousand times it can uncover subtle logic errors in code. -The problem is how to deal with flaky tests. Some people may suggest to automatically mute the "annoying" flaky tests. Or to add automatic retries in case of failure. In my opinion, this is all wrong. Instead of trying to ignore flaky tests, we do the opposite: we do maximum efforts to make the tests even more flaky! +The problem is how to deal with flaky tests. Some people may suggest to automatically mute the "annoying" flaky tests. Or to add automatic retries in case of failure. We believe that this is all wrong. Instead of trying to ignore flaky tests, we do the opposite: we do maximum efforts to make the tests even more flaky! Our recipes for flaky tests: — never mute or restart them; if the test failed once, always look and investigate the cause; From 8bb6205293b12c917fde6e05aaf812d8c70608bb Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 27 Dec 2021 15:23:44 +0300 Subject: [PATCH 0425/1260] Fix keeper log messages --- src/Coordination/KeeperServer.cpp | 33 ++++++++++++++++++++----------- 1 file changed, 22 insertions(+), 11 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 171fa2986eb..82ea100bccb 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -75,6 +75,17 @@ std::string checkAndGetSuperdigest(const String & user_and_digest) return user_and_digest; } +int32_t getValueOrMaxInt32AndLogWarning(uint64_t value, const std::string & name, Poco::Logger * log) +{ + if (value > std::numeric_limits::max()) + { + LOG_WARNING(log, "Got {} value for setting '{}' which is bigger than int32_t max value, lowering value to {}.", value, name, std::numeric_limits::max()); + return std::numeric_limits::max(); + } + + return static_cast(value); +} + } KeeperServer::KeeperServer( @@ -134,18 +145,18 @@ void KeeperServer::startup() } nuraft::raft_params params; - params.heart_beat_interval_ = coordination_settings->heart_beat_interval_ms.totalMilliseconds(); - params.election_timeout_lower_bound_ = coordination_settings->election_timeout_lower_bound_ms.totalMilliseconds(); - params.election_timeout_upper_bound_ = coordination_settings->election_timeout_upper_bound_ms.totalMilliseconds(); - - params.reserved_log_items_ = coordination_settings->reserved_log_items; - params.snapshot_distance_ = coordination_settings->snapshot_distance; - params.stale_log_gap_ = coordination_settings->stale_log_gap; - params.fresh_log_gap_ = coordination_settings->fresh_log_gap; - params.client_req_timeout_ = coordination_settings->operation_timeout_ms.totalMilliseconds(); + params.heart_beat_interval_ = getValueOrMaxInt32AndLogWarning(coordination_settings->heart_beat_interval_ms.totalMilliseconds(), "heart_beat_interval_ms", log); + params.election_timeout_lower_bound_ = getValueOrMaxInt32AndLogWarning(coordination_settings->election_timeout_lower_bound_ms.totalMilliseconds(), "election_timeout_lower_bound_ms", log); + params.election_timeout_upper_bound_ = getValueOrMaxInt32AndLogWarning(coordination_settings->election_timeout_upper_bound_ms.totalMilliseconds(), "election_timeout_upper_bound_ms", log); + params.reserved_log_items_ = getValueOrMaxInt32AndLogWarning(coordination_settings->reserved_log_items, "reserved_log_items", log); + params.snapshot_distance_ = getValueOrMaxInt32AndLogWarning(coordination_settings->snapshot_distance, "snapshot_distance", log); + params.stale_log_gap_ = getValueOrMaxInt32AndLogWarning(coordination_settings->stale_log_gap, "stale_log_gap", log); + params.fresh_log_gap_ = getValueOrMaxInt32AndLogWarning(coordination_settings->fresh_log_gap, "fresh_log_gap", log); + params.client_req_timeout_ = getValueOrMaxInt32AndLogWarning(coordination_settings->operation_timeout_ms.totalMilliseconds(), "operation_timeout_ms", log); params.auto_forwarding_ = coordination_settings->auto_forwarding; - params.auto_forwarding_req_timeout_ = coordination_settings->operation_timeout_ms.totalMilliseconds() * 2; - params.max_append_size_ = coordination_settings->max_requests_batch_size; + params.auto_forwarding_req_timeout_ = std::max(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2, std::numeric_limits::max()); + params.auto_forwarding_req_timeout_ = getValueOrMaxInt32AndLogWarning(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2, "operation_timeout_ms", log); + params.max_append_size_ = getValueOrMaxInt32AndLogWarning(coordination_settings->max_requests_batch_size, "max_requests_batch_size", log); params.return_method_ = nuraft::raft_params::async_handler; From 94bb7cba6214e26aa88103263ad5406ab84e01a4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 11 Nov 2021 12:03:53 +0300 Subject: [PATCH 0426/1260] cluster discovery wip --- programs/server/Server.cpp | 2 + src/Interpreters/Cluster.cpp | 4 + src/Interpreters/Cluster.h | 1 - src/Interpreters/ClusterDiscovery.cpp | 103 ++++++++++++++++++++ src/Interpreters/ClusterDiscovery.h | 39 ++++++++ src/Interpreters/Context.cpp | 14 +++ src/Interpreters/Context.h | 3 + src/Interpreters/InterpreterSystemQuery.cpp | 2 +- 8 files changed, 166 insertions(+), 2 deletions(-) create mode 100644 src/Interpreters/ClusterDiscovery.cpp create mode 100644 src/Interpreters/ClusterDiscovery.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 14075f9fbf2..e52323a4135 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1555,6 +1555,8 @@ if (ThreadFuzzer::instance().isEffective()) server.start(); LOG_INFO(log, "Ready for connections."); + global_context->registerNodeForClusterDiscovery(); + SCOPE_EXIT_SAFE({ LOG_DEBUG(log, "Received termination signal."); LOG_DEBUG(log, "Waiting for current connections to close."); diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 30d0dd4cece..ca599bd83a7 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -489,6 +489,10 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, internal_replication }); } + else if (startsWith(key, "discovery")) + { + continue; + } else throw Exception("Unknown element in config: " + key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index a64e17264b1..1232e004ca7 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -295,7 +295,6 @@ public: void updateClusters(const Poco::Util::AbstractConfiguration & new_config, const Settings & settings, const String & config_prefix, Poco::Util::AbstractConfiguration * old_config = nullptr); -public: using Impl = std::map; Impl getContainer() const; diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp new file mode 100644 index 00000000000..7f8f44690d1 --- /dev/null +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -0,0 +1,103 @@ +#include +#include +#include +#include "base/logger_useful.h" +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +ClusterDiscovery::ClusterDiscovery( + const Poco::Util::AbstractConfiguration & config, + ContextMutablePtr context_, + const String & config_prefix) + : context(context_) + , node_name(toString(ServerUUID::get())) + , server_port(context->getTCPPort()) + , log(&Poco::Logger::get("ClusterDiscovery")) +{ + Poco::Util::AbstractConfiguration::Keys config_keys; + config.keys(config_prefix, config_keys); + + for (const auto & key : config_keys) + { + String path = config.getString(config_prefix + "." + key + ".path"); + trimRight(path, '/'); + clusters[key] = path; + } +} + +Strings ClusterDiscovery::getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & nodes) +{ + Strings result; + result.reserve(nodes.size()); + for (const auto & node : nodes) + { + /// TODO (vdimir@): use batch request? + if (bool ok = zk->tryGet(zk_root + "/" + node, result.emplace_back()); !ok) + { + result.pop_back(); + LOG_WARNING(log, "Cluster configuration was changed during update, skip nonexisting node"); + } + } + return result; +} + +void ClusterDiscovery::updateCluster(const String & cluster_name, const String & zk_root) +{ + auto zk = context->getZooKeeper(); + + auto watch_callback = [this, cluster_name, zk_root](const Coordination::WatchResponse &) + { + this->updateCluster(cluster_name, zk_root); + }; + + const auto nodes = zk->getChildrenWatch(zk_root, nullptr, watch_callback); + + Strings replicas = getNodes(zk, zk_root, nodes); + if (replicas.empty()) + return; + + std::vector> shards = {replicas}; + + bool secure = false; + auto maybe_secure_port = context->getTCPPortSecure(); + auto cluster = std::make_shared( + context->getSettings(), + shards, + context->getUserName(), + "", + (secure ? (maybe_secure_port ? *maybe_secure_port : DBMS_DEFAULT_SECURE_PORT) : context->getTCPPort()), + false /* treat_local_as_remote */, + context->getApplicationType() == Context::ApplicationType::LOCAL /* treat_local_port_as_remote */, + secure); + + context->setCluster(cluster_name, cluster); +} + +void ClusterDiscovery::start() +{ + auto zk = context->getZooKeeper(); + for (const auto & [cluster_name, zk_root] : clusters) + { + String node_path = zk_root + "/" + node_name; + zk->createAncestors(node_path); + + String info = DNSResolver::instance().getHostName() + ":" + toString(server_port); + + zk->createOrUpdate(node_path, info, zkutil::CreateMode::Ephemeral); + + LOG_DEBUG(log, "Current node {} registered in cluster {}", node_name, cluster_name); + + this->updateCluster(cluster_name, zk_root); + } +} + +} diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h new file mode 100644 index 00000000000..d134fcf0f00 --- /dev/null +++ b/src/Interpreters/ClusterDiscovery.h @@ -0,0 +1,39 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +class ClusterDiscovery +{ + +public: + ClusterDiscovery( + const Poco::Util::AbstractConfiguration & config, + ContextMutablePtr context_, + const String & config_prefix = "remote_servers_discovery"); + + void start(); + + +private: + Strings getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & nodes); + void updateCluster(const String & cluster_name, const String & zk_root); + + /// cluster name -> path in zk + std::unordered_map clusters; + + ContextMutablePtr context; + + String node_name; + UInt16 server_port; + + Poco::Logger * log; +}; + +} diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index db1d6a37877..2f712e75b17 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -86,6 +86,7 @@ #include #include #include +#include #include @@ -254,6 +255,7 @@ struct ContextSharedPart std::shared_ptr clusters; ConfigurationPtr clusters_config; /// Stores updated configs mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config + std::unique_ptr cluster_discovery; std::shared_ptr async_insert_queue; std::map server_ports; @@ -2195,12 +2197,24 @@ std::shared_ptr Context::getClusters() const return shared->clusters; } +void Context::registerNodeForClusterDiscovery() +{ + if (!shared->cluster_discovery) + return; + shared->cluster_discovery->start(); +} + /// On repeating calls updates existing clusters and adds new clusters, doesn't delete old clusters void Context::setClustersConfig(const ConfigurationPtr & config, const String & config_name) { std::lock_guard lock(shared->clusters_mutex); + if (!shared->cluster_discovery) + { + shared->cluster_discovery = std::make_unique(*config, shared_from_this()); + } + /// Do not update clusters if this part of config wasn't changed. if (shared->clusters && isSameConfiguration(*config, *shared->clusters_config, config_name)) return; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 823bc028c15..35a2d6c3536 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -751,6 +751,9 @@ public: std::shared_ptr getCluster(const std::string & cluster_name) const; std::shared_ptr tryGetCluster(const std::string & cluster_name) const; void setClustersConfig(const ConfigurationPtr & config, const String & config_name = "remote_servers"); + + void registerNodeForClusterDiscovery(); + /// Sets custom cluster, but doesn't update configuration void setCluster(const String & cluster_name, const std::shared_ptr & cluster); void reloadClusterConfig() const; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 960fddccb8c..b39ededaa91 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -469,7 +469,7 @@ void InterpreterSystemQuery::restoreReplica() { getContext()->checkAccess(AccessType::SYSTEM_RESTORE_REPLICA, table_id); - const zkutil::ZooKeeperPtr& zookeeper = getContext()->getZooKeeper(); + const zkutil::ZooKeeperPtr & zookeeper = getContext()->getZooKeeper(); if (zookeeper->expired()) throw Exception(ErrorCodes::NO_ZOOKEEPER, From 84fb4354bc7abd2466efd8eaa6be6a7efdd7909d Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 15 Nov 2021 17:52:52 +0300 Subject: [PATCH 0427/1260] cluster discovery worker thread --- src/Interpreters/ClusterDiscovery.cpp | 96 ++++++++++++++++++++++----- src/Interpreters/ClusterDiscovery.h | 22 +++++- 2 files changed, 100 insertions(+), 18 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 7f8f44690d1..979c0b2a8b0 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -1,19 +1,33 @@ +#include +#include + #include #include #include -#include "base/logger_useful.h" +#include + #include #include #include #include -#include -#include -#include namespace DB { +namespace +{ + +constexpr size_t MAX_QUEUE_SIZE = 16; +constexpr UInt64 QUEUE_OP_TIMEOUT_MS = 1000; + +fs::path getReplicasListPath(const String & zk_root) +{ + return fs::path(zk_root + "/replicas"); +} + +} + ClusterDiscovery::ClusterDiscovery( const Poco::Util::AbstractConfiguration & config, ContextMutablePtr context_, @@ -21,6 +35,7 @@ ClusterDiscovery::ClusterDiscovery( : context(context_) , node_name(toString(ServerUUID::get())) , server_port(context->getTCPPort()) + , queue(std::make_shared(MAX_QUEUE_SIZE)) , log(&Poco::Logger::get("ClusterDiscovery")) { Poco::Util::AbstractConfiguration::Keys config_keys; @@ -41,7 +56,7 @@ Strings ClusterDiscovery::getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_ for (const auto & node : nodes) { /// TODO (vdimir@): use batch request? - if (bool ok = zk->tryGet(zk_root + "/" + node, result.emplace_back()); !ok) + if (bool ok = zk->tryGet(getReplicasListPath(zk_root) / node, result.emplace_back()); !ok) { result.pop_back(); LOG_WARNING(log, "Cluster configuration was changed during update, skip nonexisting node"); @@ -50,18 +65,30 @@ Strings ClusterDiscovery::getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_ return result; } -void ClusterDiscovery::updateCluster(const String & cluster_name, const String & zk_root) +Strings ClusterDiscovery::getNodeNames(zkutil::ZooKeeperPtr & zk, const String & zk_root, const String & cluster_name) { - auto zk = context->getZooKeeper(); - - auto watch_callback = [this, cluster_name, zk_root](const Coordination::WatchResponse &) + auto watch_callback = [cluster_name, queue=queue, log=log](const Coordination::WatchResponse &) { - this->updateCluster(cluster_name, zk_root); + if (!queue->tryPush(cluster_name, QUEUE_OP_TIMEOUT_MS)) + { + if (queue->isFinished()) + return; + LOG_WARNING(log, "Cannot push update request for cluster '{}'", cluster_name); + } }; - const auto nodes = zk->getChildrenWatch(zk_root, nullptr, watch_callback); + return zk->getChildrenWatch(getReplicasListPath(zk_root), nullptr, watch_callback); +} +void ClusterDiscovery::updateCluster(const String & cluster_name, const String & zk_root) +{ + LOG_TRACE(log, "Updating cluster '{}'", cluster_name); + + auto zk = context->getZooKeeper(); + + Strings nodes = getNodeNames(zk, zk_root, cluster_name); Strings replicas = getNodes(zk, zk_root, nodes); + if (replicas.empty()) return; @@ -82,22 +109,61 @@ void ClusterDiscovery::updateCluster(const String & cluster_name, const String & context->setCluster(cluster_name, cluster); } +void ClusterDiscovery::updateCluster(const String & cluster_name) +{ + updateCluster(cluster_name, clusters[cluster_name]); +} + void ClusterDiscovery::start() { auto zk = context->getZooKeeper(); + + LOG_TRACE(log, "Starting working thread"); + main_thread = ThreadFromGlobalPool([this] { runMainThread(); }); + for (const auto & [cluster_name, zk_root] : clusters) { - String node_path = zk_root + "/" + node_name; + String node_path = getReplicasListPath(zk_root) / node_name; zk->createAncestors(node_path); - String info = DNSResolver::instance().getHostName() + ":" + toString(server_port); + String info = getFQDNOrHostName() + ":" + toString(server_port); zk->createOrUpdate(node_path, info, zkutil::CreateMode::Ephemeral); - LOG_DEBUG(log, "Current node {} registered in cluster {}", node_name, cluster_name); - this->updateCluster(cluster_name, zk_root); + updateCluster(cluster_name, zk_root); } } +void ClusterDiscovery::runMainThread() +{ + // setThreadName("ClusterDiscovery"); + LOG_TRACE(log, "Worker thread started"); + + while (!stop_flag) + { + std::string cluster_name; + if (bool ok = queue->tryPop(cluster_name, QUEUE_OP_TIMEOUT_MS)) + { + updateCluster(cluster_name); + } + } + LOG_TRACE(log, "Worker thread stopped"); +} + +void ClusterDiscovery::shutdown() +{ + LOG_TRACE(log, "Shutting down"); + + stop_flag.exchange(true); + queue->clearAndFinish(); + if (main_thread.joinable()) + main_thread.join(); +} + +ClusterDiscovery::~ClusterDiscovery() +{ + ClusterDiscovery::shutdown(); +} + } diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index d134fcf0f00..c5545b6bcbe 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -1,10 +1,14 @@ #pragma once -#include -#include -#include +#include +#include #include +#include + +#include + +#include namespace DB { @@ -20,11 +24,18 @@ public: void start(); + ~ClusterDiscovery(); private: + Strings getNodeNames(zkutil::ZooKeeperPtr & zk, const String & zk_root, const String & cluster_name); Strings getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & nodes); + + void updateCluster(const String & cluster_name); void updateCluster(const String & cluster_name, const String & zk_root); + void runMainThread(); + void shutdown(); + /// cluster name -> path in zk std::unordered_map clusters; @@ -33,6 +44,11 @@ private: String node_name; UInt16 server_port; + using UpdateQueue = ConcurrentBoundedQueue; + std::shared_ptr queue; + std::atomic stop_flag = false; + ThreadFromGlobalPool main_thread; + Poco::Logger * log; }; From 43bfcacd6122b3c993591b58d365c9a516a5e0cd Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 16 Nov 2021 11:38:49 +0300 Subject: [PATCH 0428/1260] Add test_cluster_discovery --- .../test_cluster_discovery/__init__.py | 0 .../test_cluster_discovery/config/config.xml | 7 +++ .../test_cluster_discovery/test.py | 63 +++++++++++++++++++ 3 files changed, 70 insertions(+) create mode 100644 tests/integration/test_cluster_discovery/__init__.py create mode 100644 tests/integration/test_cluster_discovery/config/config.xml create mode 100644 tests/integration/test_cluster_discovery/test.py diff --git a/tests/integration/test_cluster_discovery/__init__.py b/tests/integration/test_cluster_discovery/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_cluster_discovery/config/config.xml b/tests/integration/test_cluster_discovery/config/config.xml new file mode 100644 index 00000000000..00595c77441 --- /dev/null +++ b/tests/integration/test_cluster_discovery/config/config.xml @@ -0,0 +1,7 @@ + + + + /clickhouse/discovery/test_auto_cluster + + + diff --git a/tests/integration/test_cluster_discovery/test.py b/tests/integration/test_cluster_discovery/test.py new file mode 100644 index 00000000000..dfbbaa9dda6 --- /dev/null +++ b/tests/integration/test_cluster_discovery/test.py @@ -0,0 +1,63 @@ +import pytest +import time + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +nodes = [ + cluster.add_instance( + f'node{i}', + main_configs=['config/config.xml'], + stay_alive=True, + with_zookeeper=True + ) for i in range(5) +] + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def check_nodes_count_in_cluster(nodes, expected, cluster_name, *, retries): + """ + Check nodes count in system.clusters for specified cluster + """ + for retry in range(1, retries + 1): + nodes_cnt = [ + int(node.query(f"SELECT count() FROM system.clusters WHERE cluster = '{cluster_name}'")) + for node in nodes + ] + if all(actual == expected for actual in nodes_cnt): + break + + if retry != retries: + time.sleep(2 ** retry) + else: + raise Exception(f'Wrong nodes count in cluster: {nodes_cnt}, expected: {expected} (after {retries} retries)') + + +def test_cluster_discovery_startup_and_stop(start_cluster): + """ + Start cluster, check nodes count in system.clusters, + then stop/start some nodes and check that it (dis)appeared in cluster. + """ + + check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes), 'test_auto_cluster', retries=1) + + nodes[1].stop_clickhouse(kill=True) + check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes) - 1, 'test_auto_cluster', retries=5) + + nodes[3].stop_clickhouse() + check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes) - 2, 'test_auto_cluster', retries=5) + + nodes[1].start_clickhouse() + check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes) - 1, 'test_auto_cluster', retries=5) + + nodes[3].start_clickhouse() + check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes), 'test_auto_cluster', retries=5) From 5c47229797715fb1a501cec5d2a1ca643866edc3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 16 Nov 2021 11:59:54 +0300 Subject: [PATCH 0429/1260] fix whitespace in StorageSystemClusters.cpp --- src/Storages/System/StorageSystemClusters.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index 1f5def6d6b4..1e303d1aeaa 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -45,7 +45,8 @@ void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr co // get an error when trying to get the info about DB from ZK. // Just ignore these inaccessible databases. A good example of a // failing test is `01526_client_start_and_exit`. - try { + try + { writeCluster(res_columns, {name_and_database.first, replicated->getCluster()}); } catch (...) From aa0d79659b6fc206bb8e4ca327f07675fe99f6a3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 16 Nov 2021 12:02:44 +0300 Subject: [PATCH 0430/1260] check version in ClusterDiscovery::updateCluster --- src/Interpreters/ClusterDiscovery.cpp | 71 +++++++++++++++++---------- src/Interpreters/ClusterDiscovery.h | 11 +++-- 2 files changed, 54 insertions(+), 28 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 979c0b2a8b0..17fa2180e95 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -49,23 +49,11 @@ ClusterDiscovery::ClusterDiscovery( } } -Strings ClusterDiscovery::getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & nodes) -{ - Strings result; - result.reserve(nodes.size()); - for (const auto & node : nodes) - { - /// TODO (vdimir@): use batch request? - if (bool ok = zk->tryGet(getReplicasListPath(zk_root) / node, result.emplace_back()); !ok) - { - result.pop_back(); - LOG_WARNING(log, "Cluster configuration was changed during update, skip nonexisting node"); - } - } - return result; -} - -Strings ClusterDiscovery::getNodeNames(zkutil::ZooKeeperPtr & zk, const String & zk_root, const String & cluster_name) +Strings ClusterDiscovery::getNodeNames(zkutil::ZooKeeperPtr & zk, + const String & zk_root, + const String & cluster_name, + int * version, + bool set_callback) { auto watch_callback = [cluster_name, queue=queue, log=log](const Coordination::WatchResponse &) { @@ -77,20 +65,48 @@ Strings ClusterDiscovery::getNodeNames(zkutil::ZooKeeperPtr & zk, const String & } }; - return zk->getChildrenWatch(getReplicasListPath(zk_root), nullptr, watch_callback); + Coordination::Stat stat; + Strings nodes = zk->getChildrenWatch(getReplicasListPath(zk_root), &stat, set_callback ? watch_callback : Coordination::WatchCallback{}); + if (version) + *version = stat.cversion; + return nodes; } -void ClusterDiscovery::updateCluster(const String & cluster_name, const String & zk_root) +Strings ClusterDiscovery::getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & nodes) +{ + Strings result; + result.reserve(nodes.size()); + for (const auto & node : nodes) + { + bool ok = zk->tryGet(getReplicasListPath(zk_root) / node, result.emplace_back()); + if (!ok) + { + result.pop_back(); + LOG_WARNING(log, "Cluster configuration was changed during update, skip nonexisting node"); + } + } + return result; +} + +bool ClusterDiscovery::updateCluster(const String & cluster_name, const String & zk_root) { LOG_TRACE(log, "Updating cluster '{}'", cluster_name); auto zk = context->getZooKeeper(); - Strings nodes = getNodeNames(zk, zk_root, cluster_name); + int start_version; + Strings nodes = getNodeNames(zk, zk_root, cluster_name, &start_version, false); + Strings replicas = getNodes(zk, zk_root, nodes); if (replicas.empty()) - return; + return false; + + int current_version; + getNodeNames(zk, zk_root, cluster_name, ¤t_version, true); + + if (current_version != start_version) + return false; std::vector> shards = {replicas}; @@ -107,11 +123,12 @@ void ClusterDiscovery::updateCluster(const String & cluster_name, const String & secure); context->setCluster(cluster_name, cluster); + return true; } -void ClusterDiscovery::updateCluster(const String & cluster_name) +bool ClusterDiscovery::updateCluster(const String & cluster_name) { - updateCluster(cluster_name, clusters[cluster_name]); + return updateCluster(cluster_name, clusters[cluster_name]); } void ClusterDiscovery::start() @@ -143,9 +160,13 @@ void ClusterDiscovery::runMainThread() while (!stop_flag) { std::string cluster_name; - if (bool ok = queue->tryPop(cluster_name, QUEUE_OP_TIMEOUT_MS)) + if (queue->tryPop(cluster_name, QUEUE_OP_TIMEOUT_MS)) { - updateCluster(cluster_name); + bool ok = updateCluster(cluster_name); + if (!ok) + { + LOG_WARNING(log, "Error on updating cluster '{}', configuration changed during update, will retry", cluster_name); + } } } LOG_TRACE(log, "Worker thread stopped"); diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index c5545b6bcbe..4e7672fc9ee 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -27,11 +27,16 @@ public: ~ClusterDiscovery(); private: - Strings getNodeNames(zkutil::ZooKeeperPtr & zk, const String & zk_root, const String & cluster_name); + Strings getNodeNames(zkutil::ZooKeeperPtr & zk, + const String & zk_root, + const String & cluster_name, + int * version = nullptr, + bool set_callback = true); + Strings getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & nodes); - void updateCluster(const String & cluster_name); - void updateCluster(const String & cluster_name, const String & zk_root); + bool updateCluster(const String & cluster_name); + bool updateCluster(const String & cluster_name, const String & zk_root); void runMainThread(); void shutdown(); From 1f460f05fee24d6d9d2eabfca097582b31da4881 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 16 Nov 2021 12:03:15 +0300 Subject: [PATCH 0431/1260] add retry for first check in test_cluster_discovery_startup_and_stop --- tests/integration/test_cluster_discovery/test.py | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_cluster_discovery/test.py b/tests/integration/test_cluster_discovery/test.py index dfbbaa9dda6..a85babf3ebe 100644 --- a/tests/integration/test_cluster_discovery/test.py +++ b/tests/integration/test_cluster_discovery/test.py @@ -24,10 +24,11 @@ def start_cluster(): cluster.shutdown() -def check_nodes_count_in_cluster(nodes, expected, cluster_name, *, retries): +def check_nodes_count_in_cluster(nodes, expected, cluster_name, *, retries=5): """ Check nodes count in system.clusters for specified cluster """ + for retry in range(1, retries + 1): nodes_cnt = [ int(node.query(f"SELECT count() FROM system.clusters WHERE cluster = '{cluster_name}'")) @@ -48,16 +49,16 @@ def test_cluster_discovery_startup_and_stop(start_cluster): then stop/start some nodes and check that it (dis)appeared in cluster. """ - check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes), 'test_auto_cluster', retries=1) + check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes), 'test_auto_cluster') nodes[1].stop_clickhouse(kill=True) - check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes) - 1, 'test_auto_cluster', retries=5) + check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes) - 1, 'test_auto_cluster') nodes[3].stop_clickhouse() - check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes) - 2, 'test_auto_cluster', retries=5) + check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes) - 2, 'test_auto_cluster') nodes[1].start_clickhouse() - check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes) - 1, 'test_auto_cluster', retries=5) + check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes) - 1, 'test_auto_cluster') nodes[3].start_clickhouse() - check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes), 'test_auto_cluster', retries=5) + check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes), 'test_auto_cluster') From 4816d1afcc90f0fe7a59be4d7099bbbd38d733ef Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 16 Nov 2021 15:01:30 +0300 Subject: [PATCH 0432/1260] cluster discovery: check cluster is changed, keep info --- src/Interpreters/ClusterDiscovery.cpp | 168 +++++++++++++++++++------- src/Interpreters/ClusterDiscovery.h | 38 +++++- 2 files changed, 161 insertions(+), 45 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 17fa2180e95..3ffdfebd2f0 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -1,3 +1,7 @@ +#include +#include +#include + #include #include @@ -45,10 +49,11 @@ ClusterDiscovery::ClusterDiscovery( { String path = config.getString(config_prefix + "." + key + ".path"); trimRight(path, '/'); - clusters[key] = path; + clusters_info.emplace(key, ClusterInfo(key, path)); } } +/// List node in zookeper for cluster Strings ClusterDiscovery::getNodeNames(zkutil::ZooKeeperPtr & zk, const String & zk_root, const String & cluster_name, @@ -72,43 +77,42 @@ Strings ClusterDiscovery::getNodeNames(zkutil::ZooKeeperPtr & zk, return nodes; } -Strings ClusterDiscovery::getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & nodes) +/// Reads node information from scpecified zookeper nodes +ClusterDiscovery::NodesInfo ClusterDiscovery::getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & nodes) { - Strings result; - result.reserve(nodes.size()); - for (const auto & node : nodes) + NodesInfo result; + for (const auto & node_uuid : nodes) { - bool ok = zk->tryGet(getReplicasListPath(zk_root) / node, result.emplace_back()); + bool ok = zk->tryGet(getReplicasListPath(zk_root) / node_uuid, result[node_uuid]); if (!ok) { - result.pop_back(); + result.erase(node_uuid); LOG_WARNING(log, "Cluster configuration was changed during update, skip nonexisting node"); } } return result; } -bool ClusterDiscovery::updateCluster(const String & cluster_name, const String & zk_root) +/// Checks if custer nodes set is changed. +/// Returs true if update required. +/// It performs only shallow check (set of nodes' uuids). +/// So, if node's hostname are changed, then cluster won't be updated. +bool ClusterDiscovery::needUpdate(const Strings & node_uuids, const NodesInfo & nodes) { - LOG_TRACE(log, "Updating cluster '{}'", cluster_name); + bool has_difference = node_uuids.size() != nodes.size() || + std::any_of(node_uuids.begin(), node_uuids.end(), [&nodes] (auto u) { return !nodes.contains(u); }); - auto zk = context->getZooKeeper(); + return has_difference; +} - int start_version; - Strings nodes = getNodeNames(zk, zk_root, cluster_name, &start_version, false); +ClusterPtr ClusterDiscovery::getCluster(const ClusterInfo & cluster_info) +{ + Strings replica_adresses; + replica_adresses.reserve(cluster_info.nodes_info.size()); + for (const auto & node : cluster_info.nodes_info) + replica_adresses.emplace_back(node.second); - Strings replicas = getNodes(zk, zk_root, nodes); - - if (replicas.empty()) - return false; - - int current_version; - getNodeNames(zk, zk_root, cluster_name, ¤t_version, true); - - if (current_version != start_version) - return false; - - std::vector> shards = {replicas}; + std::vector> shards = {replica_adresses}; bool secure = false; auto maybe_secure_port = context->getTCPPortSecure(); @@ -121,14 +125,73 @@ bool ClusterDiscovery::updateCluster(const String & cluster_name, const String & false /* treat_local_as_remote */, context->getApplicationType() == Context::ApplicationType::LOCAL /* treat_local_port_as_remote */, secure); + return cluster; +} - context->setCluster(cluster_name, cluster); +/// Reads data from zookeeper and tries to update cluster. +/// Returns true on success (or no update required). +bool ClusterDiscovery::updateCluster(ClusterInfo & cluster_info) +{ + LOG_TRACE(log, "Updating cluster '{}'", cluster_info.name); + + auto zk = context->getZooKeeper(); + + int start_version; + Strings node_uuids = getNodeNames(zk, cluster_info.zk_root, cluster_info.name, &start_version, false); + + if (node_uuids.empty()) + { + LOG_ERROR(log, "Can't find any node in cluster '{}', will register again", cluster_info.name); + registerInZk(zk, cluster_info); + return false; + } + + auto & nodes_info = cluster_info.nodes_info; + if (!needUpdate(node_uuids, nodes_info)) + { + LOG_TRACE(log, "No update required for cluster '{}'", cluster_info.name); + return true; + } + + nodes_info = getNodes(zk, cluster_info.zk_root, node_uuids); + + if (nodes_info.empty()) + return false; + + int current_version; + getNodeNames(zk, cluster_info.zk_root, cluster_info.name, ¤t_version, true); + + if (current_version != start_version) + { + nodes_info.clear(); + return false; + } + + auto cluster = getCluster(cluster_info); + context->setCluster(cluster_info.name, cluster); return true; } bool ClusterDiscovery::updateCluster(const String & cluster_name) { - return updateCluster(cluster_name, clusters[cluster_name]); + auto cluster_info = clusters_info.find(cluster_name); + if (cluster_info == clusters_info.end()) + { + LOG_ERROR(log, "Unknown cluster '{}'", cluster_name); + return false; + } + return updateCluster(cluster_info->second); +} + +void ClusterDiscovery::registerInZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & info) +{ + String node_path = getReplicasListPath(info.zk_root) / node_name; + zk->createAncestors(node_path); + + String payload = getFQDNOrHostName() + ":" + toString(server_port); + + zk->createOrUpdate(node_path, payload, zkutil::CreateMode::Ephemeral); + LOG_DEBUG(log, "Current node {} registered in cluster {}", node_name, info.name); } void ClusterDiscovery::start() @@ -138,36 +201,59 @@ void ClusterDiscovery::start() LOG_TRACE(log, "Starting working thread"); main_thread = ThreadFromGlobalPool([this] { runMainThread(); }); - for (const auto & [cluster_name, zk_root] : clusters) + for (auto & [_, info] : clusters_info) { - String node_path = getReplicasListPath(zk_root) / node_name; - zk->createAncestors(node_path); + registerInZk(zk, info); + updateCluster(info); - String info = getFQDNOrHostName() + ":" + toString(server_port); - - zk->createOrUpdate(node_path, info, zkutil::CreateMode::Ephemeral); - LOG_DEBUG(log, "Current node {} registered in cluster {}", node_name, cluster_name); - - updateCluster(cluster_name, zk_root); + if (!updateCluster(info)) + LOG_WARNING(log, "Error on updating cluster '{}'", info.name); } } void ClusterDiscovery::runMainThread() { - // setThreadName("ClusterDiscovery"); LOG_TRACE(log, "Worker thread started"); + // setThreadName("ClusterDiscovery"); + + using namespace std::chrono_literals; + constexpr UInt64 full_update_interval = std::chrono::milliseconds(5min).count(); + + std::unordered_map last_cluster_update; + for (const auto & [cluster_name, _] : clusters_info) + last_cluster_update.emplace(cluster_name, Stopwatch()); + Stopwatch last_full_update; + + pcg64 rng(randomSeed()); while (!stop_flag) { - std::string cluster_name; - if (queue->tryPop(cluster_name, QUEUE_OP_TIMEOUT_MS)) { - bool ok = updateCluster(cluster_name); - if (!ok) + String cluster_name; + if (queue->tryPop(cluster_name, QUEUE_OP_TIMEOUT_MS)) { - LOG_WARNING(log, "Error on updating cluster '{}', configuration changed during update, will retry", cluster_name); + if (updateCluster(cluster_name)) + last_cluster_update[cluster_name].restart(); + else + LOG_WARNING(log, "Error on updating cluster '{}', configuration changed during update, will retry", cluster_name); } } + + auto jitter = std::uniform_real_distribution<>(1.0, 2.0)(rng); + if (last_full_update.elapsedMilliseconds() > UInt64(full_update_interval * jitter)) + { + for (const auto & lastupd : last_cluster_update) + { + if (lastupd.second.elapsedMilliseconds() > full_update_interval) + { + if (updateCluster(lastupd.first)) + last_cluster_update[lastupd.first].restart(); + else + LOG_WARNING(log, "Error on updating cluster '{}'", lastupd.first); + } + } + last_full_update.restart(); + } } LOG_TRACE(log, "Worker thread stopped"); } diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index 4e7672fc9ee..152a4cbb892 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -13,6 +14,13 @@ namespace DB { +/* + * Discover cluster nodes. + * + * Each node adds ephemernal node into specified path in zookeeper (each cluster have own path). + * Also node subscribed for updates for theese paths, and at each child node chanhe cluster updated. + * When node goes down ephemernal node are destroyed, cluster configuration is updated on other node and gone node is removed from cluster. + */ class ClusterDiscovery { @@ -27,30 +35,52 @@ public: ~ClusterDiscovery(); private: + // node uuid -> address ("host:port") + using NodesInfo = std::unordered_map; + + struct ClusterInfo + { + const String name; + const String zk_root; + NodesInfo nodes_info; + + explicit ClusterInfo(const String & name_, const String & zk_root_) : name(name_), zk_root(zk_root_) {} + }; + + void registerInZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & info); + Strings getNodeNames(zkutil::ZooKeeperPtr & zk, const String & zk_root, const String & cluster_name, int * version = nullptr, bool set_callback = true); - Strings getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & nodes); + NodesInfo getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & nodes); + ClusterPtr getCluster(const ClusterInfo & cluster_info); + + static bool needUpdate(const Strings & node_uuids, const NodesInfo & nodes); bool updateCluster(const String & cluster_name); - bool updateCluster(const String & cluster_name, const String & zk_root); + bool updateCluster(ClusterInfo & cluster_info); void runMainThread(); void shutdown(); - /// cluster name -> path in zk - std::unordered_map clusters; + /// cluster name -> cluster info (zk root, set of nodes) + std::unordered_map clusters_info; ContextMutablePtr context; String node_name; UInt16 server_port; + /// Cluster names to update using UpdateQueue = ConcurrentBoundedQueue; + + /// shared_ptr is used because it's passed to watch callback + /// it prevents accessing to invalid queue after ClusterDiscovery is destroyed std::shared_ptr queue; + std::atomic stop_flag = false; ThreadFromGlobalPool main_thread; From e7d3dbeebd22c131d7ef14067e35609bcd4e4dfd Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 16 Nov 2021 16:01:57 +0300 Subject: [PATCH 0433/1260] cluster discovery init only with zookeeper --- programs/copier/ClusterCopier.cpp | 2 +- programs/server/Server.cpp | 5 +++-- src/Interpreters/Context.cpp | 4 ++-- src/Interpreters/Context.h | 2 +- 4 files changed, 7 insertions(+), 6 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 536bb37199d..4d491a06795 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -46,7 +46,7 @@ void ClusterCopier::init() reloadTaskDescription(); task_cluster->loadTasks(*task_cluster_current_config); - getContext()->setClustersConfig(task_cluster_current_config, task_cluster->clusters_prefix); + getContext()->setClustersConfig(task_cluster_current_config, false, task_cluster->clusters_prefix); /// Set up shards and their priority task_cluster->random_engine.seed(task_cluster->random_device()); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index e52323a4135..d701778b982 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -841,7 +841,7 @@ if (ThreadFuzzer::instance().isEffective()) // in a lot of places. For now, disable updating log configuration without server restart. //setTextLog(global_context->getTextLog()); updateLevels(*config, logger()); - global_context->setClustersConfig(config); + global_context->setClustersConfig(config, has_zookeeper); global_context->setMacros(std::make_unique(*config, "macros", log)); global_context->setExternalAuthenticatorsConfig(*config); @@ -1555,7 +1555,8 @@ if (ThreadFuzzer::instance().isEffective()) server.start(); LOG_INFO(log, "Ready for connections."); - global_context->registerNodeForClusterDiscovery(); + if (has_zookeeper) + global_context->registerNodeForClusterDiscovery(); SCOPE_EXIT_SAFE({ LOG_DEBUG(log, "Received termination signal."); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 2f712e75b17..a36b0805a60 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2206,11 +2206,11 @@ void Context::registerNodeForClusterDiscovery() /// On repeating calls updates existing clusters and adds new clusters, doesn't delete old clusters -void Context::setClustersConfig(const ConfigurationPtr & config, const String & config_name) +void Context::setClustersConfig(const ConfigurationPtr & config, bool enable_discovery, const String & config_name) { std::lock_guard lock(shared->clusters_mutex); - if (!shared->cluster_discovery) + if (enable_discovery && !shared->cluster_discovery) { shared->cluster_discovery = std::make_unique(*config, shared_from_this()); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 35a2d6c3536..c528e995ae9 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -750,7 +750,7 @@ public: std::shared_ptr getClusters() const; std::shared_ptr getCluster(const std::string & cluster_name) const; std::shared_ptr tryGetCluster(const std::string & cluster_name) const; - void setClustersConfig(const ConfigurationPtr & config, const String & config_name = "remote_servers"); + void setClustersConfig(const ConfigurationPtr & config, bool enable_discovery = false, const String & config_name = "remote_servers"); void registerNodeForClusterDiscovery(); From 5bbd151aae4a52358147c919d8d3f2813cdc4d5f Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 16 Nov 2021 16:39:54 +0300 Subject: [PATCH 0434/1260] style fix ClusterDiscovery --- src/Interpreters/ClusterDiscovery.cpp | 33 +++++++++++++-------------- src/Interpreters/ClusterDiscovery.h | 16 ++++++++++--- 2 files changed, 29 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 3ffdfebd2f0..cbc0bda4a49 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -25,9 +25,9 @@ namespace constexpr size_t MAX_QUEUE_SIZE = 16; constexpr UInt64 QUEUE_OP_TIMEOUT_MS = 1000; -fs::path getReplicasListPath(const String & zk_root) +fs::path getShardsListPath(const String & zk_root) { - return fs::path(zk_root + "/replicas"); + return fs::path(zk_root + "/shards"); } } @@ -71,30 +71,32 @@ Strings ClusterDiscovery::getNodeNames(zkutil::ZooKeeperPtr & zk, }; Coordination::Stat stat; - Strings nodes = zk->getChildrenWatch(getReplicasListPath(zk_root), &stat, set_callback ? watch_callback : Coordination::WatchCallback{}); + Strings nodes = zk->getChildrenWatch(getShardsListPath(zk_root), &stat, set_callback ? watch_callback : Coordination::WatchCallback{}); if (version) *version = stat.cversion; return nodes; } -/// Reads node information from scpecified zookeper nodes -ClusterDiscovery::NodesInfo ClusterDiscovery::getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & nodes) +/// Reads node information from specified zookeper nodes +ClusterDiscovery::NodesInfo ClusterDiscovery::getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & node_uuids) { NodesInfo result; - for (const auto & node_uuid : nodes) + for (const auto & node_uuid : node_uuids) { - bool ok = zk->tryGet(getReplicasListPath(zk_root) / node_uuid, result[node_uuid]); + String payload; + bool ok = zk->tryGet(getShardsListPath(zk_root) / node_uuid, payload); if (!ok) { - result.erase(node_uuid); - LOG_WARNING(log, "Cluster configuration was changed during update, skip nonexisting node"); + LOG_WARNING(log, "Cluster configuration was changed during update, found nonexisting node"); + return {}; } + result.emplace(node_uuid, NodeInfo(payload)); } return result; } -/// Checks if custer nodes set is changed. -/// Returs true if update required. +/// Checks if cluster nodes set is changed. +/// Returns true if update required. /// It performs only shallow check (set of nodes' uuids). /// So, if node's hostname are changed, then cluster won't be updated. bool ClusterDiscovery::needUpdate(const Strings & node_uuids, const NodesInfo & nodes) @@ -109,8 +111,8 @@ ClusterPtr ClusterDiscovery::getCluster(const ClusterInfo & cluster_info) { Strings replica_adresses; replica_adresses.reserve(cluster_info.nodes_info.size()); - for (const auto & node : cluster_info.nodes_info) - replica_adresses.emplace_back(node.second); + for (const auto & [_, node] : cluster_info.nodes_info) + replica_adresses.emplace_back(node.address); std::vector> shards = {replica_adresses}; @@ -154,7 +156,6 @@ bool ClusterDiscovery::updateCluster(ClusterInfo & cluster_info) } nodes_info = getNodes(zk, cluster_info.zk_root, node_uuids); - if (nodes_info.empty()) return false; @@ -185,7 +186,7 @@ bool ClusterDiscovery::updateCluster(const String & cluster_name) void ClusterDiscovery::registerInZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & info) { - String node_path = getReplicasListPath(info.zk_root) / node_name; + String node_path = getShardsListPath(info.zk_root) / node_name; zk->createAncestors(node_path); String payload = getFQDNOrHostName() + ":" + toString(server_port); @@ -204,8 +205,6 @@ void ClusterDiscovery::start() for (auto & [_, info] : clusters_info) { registerInZk(zk, info); - updateCluster(info); - if (!updateCluster(info)) LOG_WARNING(log, "Error on updating cluster '{}'", info.name); } diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index 152a4cbb892..bb910e80e82 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -18,7 +18,7 @@ namespace DB * Discover cluster nodes. * * Each node adds ephemernal node into specified path in zookeeper (each cluster have own path). - * Also node subscribed for updates for theese paths, and at each child node chanhe cluster updated. + * Also node subscribed for updates for these paths, and at each child node chanhe cluster updated. * When node goes down ephemernal node are destroyed, cluster configuration is updated on other node and gone node is removed from cluster. */ class ClusterDiscovery @@ -35,8 +35,18 @@ public: ~ClusterDiscovery(); private: + struct NodeInfo + { + /// host:port + String address; + + explicit NodeInfo(const String & data) + : address(data) + {} + }; + // node uuid -> address ("host:port") - using NodesInfo = std::unordered_map; + using NodesInfo = std::unordered_map; struct ClusterInfo { @@ -55,7 +65,7 @@ private: int * version = nullptr, bool set_callback = true); - NodesInfo getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & nodes); + NodesInfo getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & node_uuids); ClusterPtr getCluster(const ClusterInfo & cluster_info); From a2d37fe2ed47396c612b351f0f94005d8aea5556 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 17 Nov 2021 14:32:20 +0300 Subject: [PATCH 0435/1260] use map and condvar instead of queue in cluster discovery --- src/Interpreters/ClusterDiscovery.cpp | 127 +++++++++++++++----------- src/Interpreters/ClusterDiscovery.h | 13 +-- 2 files changed, 82 insertions(+), 58 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index cbc0bda4a49..aafd137494e 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -1,11 +1,14 @@ #include +#include #include +#include +#include #include #include #include -#include +#include #include #include #include @@ -19,12 +22,14 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace { -constexpr size_t MAX_QUEUE_SIZE = 16; -constexpr UInt64 QUEUE_OP_TIMEOUT_MS = 1000; - fs::path getShardsListPath(const String & zk_root) { return fs::path(zk_root + "/shards"); @@ -32,6 +37,56 @@ fs::path getShardsListPath(const String & zk_root) } +/* + * Holds boolean flags for fixed set of keys. + * Flags can be concurrently set from different threads, and consumer can wait for it. + */ +template +class ClusterDiscovery::ConcurrentFlags +{ +public: + template + ConcurrentFlags(It begin, It end) + { + for (auto it = begin; it != end; ++it) + flags.emplace(*it, false); + } + + void set(const T & val) + { + auto it = flags.find(val); + if (it == flags.end()) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown value '{}'", val); + it->second = true; + cv.notify_one(); + } + + void unset(const T & val) + { + auto it = flags.find(val); + if (it == flags.end()) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown value '{}'", val); + it->second = false; + } + + void wait(std::chrono::milliseconds timeout) + { + std::unique_lock lk(mu); + cv.wait_for(lk, timeout); + } + + const std::unordered_map & get() + { + return flags; + } + +private: + std::condition_variable cv; + std::mutex mu; + + std::unordered_map flags; +}; + ClusterDiscovery::ClusterDiscovery( const Poco::Util::AbstractConfiguration & config, ContextMutablePtr context_, @@ -39,7 +94,6 @@ ClusterDiscovery::ClusterDiscovery( : context(context_) , node_name(toString(ServerUUID::get())) , server_port(context->getTCPPort()) - , queue(std::make_shared(MAX_QUEUE_SIZE)) , log(&Poco::Logger::get("ClusterDiscovery")) { Poco::Util::AbstractConfiguration::Keys config_keys; @@ -51,6 +105,7 @@ ClusterDiscovery::ClusterDiscovery( trimRight(path, '/'); clusters_info.emplace(key, ClusterInfo(key, path)); } + clusters_to_update = std::make_shared(config_keys.begin(), config_keys.end()); } /// List node in zookeper for cluster @@ -60,15 +115,7 @@ Strings ClusterDiscovery::getNodeNames(zkutil::ZooKeeperPtr & zk, int * version, bool set_callback) { - auto watch_callback = [cluster_name, queue=queue, log=log](const Coordination::WatchResponse &) - { - if (!queue->tryPush(cluster_name, QUEUE_OP_TIMEOUT_MS)) - { - if (queue->isFinished()) - return; - LOG_WARNING(log, "Cannot push update request for cluster '{}'", cluster_name); - } - }; + auto watch_callback = [cluster_name, clusters_to_update=clusters_to_update](auto) { clusters_to_update->set(cluster_name); }; Coordination::Stat stat; Strings nodes = zk->getChildrenWatch(getShardsListPath(zk_root), &stat, set_callback ? watch_callback : Coordination::WatchCallback{}); @@ -77,7 +124,8 @@ Strings ClusterDiscovery::getNodeNames(zkutil::ZooKeeperPtr & zk, return nodes; } -/// Reads node information from specified zookeper nodes +/// Reads node information from specified zookeeper nodes +/// On error returns empty result ClusterDiscovery::NodesInfo ClusterDiscovery::getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & node_uuids) { NodesInfo result; @@ -86,10 +134,7 @@ ClusterDiscovery::NodesInfo ClusterDiscovery::getNodes(zkutil::ZooKeeperPtr & zk String payload; bool ok = zk->tryGet(getShardsListPath(zk_root) / node_uuid, payload); if (!ok) - { - LOG_WARNING(log, "Cluster configuration was changed during update, found nonexisting node"); return {}; - } result.emplace(node_uuid, NodeInfo(payload)); } return result; @@ -206,7 +251,10 @@ void ClusterDiscovery::start() { registerInZk(zk, info); if (!updateCluster(info)) - LOG_WARNING(log, "Error on updating cluster '{}'", info.name); + { + LOG_WARNING(log, "Error on updating cluster '{}', will retry", info.name); + clusters_to_update->set(info.name); + } } } @@ -216,42 +264,18 @@ void ClusterDiscovery::runMainThread() // setThreadName("ClusterDiscovery"); using namespace std::chrono_literals; - constexpr UInt64 full_update_interval = std::chrono::milliseconds(5min).count(); - - std::unordered_map last_cluster_update; - for (const auto & [cluster_name, _] : clusters_info) - last_cluster_update.emplace(cluster_name, Stopwatch()); - Stopwatch last_full_update; - - pcg64 rng(randomSeed()); while (!stop_flag) { + /// if some cluster update was ended with error on previous iteration, we will retry after timeout + clusters_to_update->wait(5s); + for (const auto & [cluster_name, need_update] : clusters_to_update->get()) { - String cluster_name; - if (queue->tryPop(cluster_name, QUEUE_OP_TIMEOUT_MS)) - { - if (updateCluster(cluster_name)) - last_cluster_update[cluster_name].restart(); - else - LOG_WARNING(log, "Error on updating cluster '{}', configuration changed during update, will retry", cluster_name); - } - } - - auto jitter = std::uniform_real_distribution<>(1.0, 2.0)(rng); - if (last_full_update.elapsedMilliseconds() > UInt64(full_update_interval * jitter)) - { - for (const auto & lastupd : last_cluster_update) - { - if (lastupd.second.elapsedMilliseconds() > full_update_interval) - { - if (updateCluster(lastupd.first)) - last_cluster_update[lastupd.first].restart(); - else - LOG_WARNING(log, "Error on updating cluster '{}'", lastupd.first); - } - } - last_full_update.restart(); + if (!need_update) + continue; + bool ok = updateCluster(cluster_name); + if (ok) + clusters_to_update->unset(cluster_name); } } LOG_TRACE(log, "Worker thread stopped"); @@ -262,7 +286,6 @@ void ClusterDiscovery::shutdown() LOG_TRACE(log, "Shutting down"); stop_flag.exchange(true); - queue->clearAndFinish(); if (main_thread.joinable()) main_thread.join(); } diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index bb910e80e82..53b97e75d8e 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -65,7 +65,7 @@ private: int * version = nullptr, bool set_callback = true); - NodesInfo getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & node_uuids); + static NodesInfo getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & node_uuids); ClusterPtr getCluster(const ClusterInfo & cluster_info); @@ -84,12 +84,13 @@ private: String node_name; UInt16 server_port; - /// Cluster names to update - using UpdateQueue = ConcurrentBoundedQueue; + template class ConcurrentFlags; + using UpdateFlags = ConcurrentFlags; - /// shared_ptr is used because it's passed to watch callback - /// it prevents accessing to invalid queue after ClusterDiscovery is destroyed - std::shared_ptr queue; + /// Cluster names to update. + /// The `shared_ptr` is used because it's passed to watch callback. + /// It prevents accessing to invalid object after ClusterDiscovery is destroyed. + std::shared_ptr clusters_to_update; std::atomic stop_flag = false; ThreadFromGlobalPool main_thread; From 76170156efd5c6c2e6ab217b5be50b474eec5b7c Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 17 Nov 2021 16:47:40 +0300 Subject: [PATCH 0436/1260] log nodes update in cluster discovery --- src/Interpreters/ClusterDiscovery.cpp | 27 +++++++++++++++++++++++++++ src/Interpreters/ClusterDiscovery.h | 2 +- 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index aafd137494e..4398c8d20b4 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -148,7 +148,34 @@ bool ClusterDiscovery::needUpdate(const Strings & node_uuids, const NodesInfo & { bool has_difference = node_uuids.size() != nodes.size() || std::any_of(node_uuids.begin(), node_uuids.end(), [&nodes] (auto u) { return !nodes.contains(u); }); + UNUSED(log); + #if defined(NDEBUG) || defined(ADDRESS_SANITIZER) || defined(THREAD_SANITIZER) || defined(MEMORY_SANITIZER) || defined(UNDEFINED_BEHAVIOR_SANITIZER) + { + /// Just to log updated nodes, suboptimal, but should be ok for expected update sizes. + /// Disabled on release build. + std::set new_names(node_uuids.begin(), node_uuids.end()); + std::set old_names; + for (const auto & [name, _] : nodes) + old_names.emplace(name); + auto format_cluster_update = [](const std::set & s1, const std::set & s2) + { + std::vector diff; + std::set_difference(s1.begin(), s1.end(), s2.begin(), s2.end(), std::back_inserter(diff)); + + constexpr size_t max_to_show = 3; + size_t sz = diff.size(); + if (sz > max_to_show) + diff.resize(max_to_show); + return fmt::format("{} nodes ({}{})", sz, fmt::join(diff, ", "), diff.size() == sz ? "" : ",..."); + }; + + LOG_DEBUG(log, "Cluster update: added: {}, removed: {}", + format_cluster_update(new_names, old_names), + format_cluster_update(old_names, new_names)); + + } + #endif return has_difference; } diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index 53b97e75d8e..0e32479ac7b 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -69,7 +69,7 @@ private: ClusterPtr getCluster(const ClusterInfo & cluster_info); - static bool needUpdate(const Strings & node_uuids, const NodesInfo & nodes); + bool needUpdate(const Strings & node_uuids, const NodesInfo & nodes); bool updateCluster(const String & cluster_name); bool updateCluster(ClusterInfo & cluster_info); From 8bf3d94f04116bb7ee722c4e7e0ca2e163726fa8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 17 Nov 2021 17:16:49 +0300 Subject: [PATCH 0437/1260] write data for cluster discovery to zk in json --- src/Interpreters/ClusterDiscovery.cpp | 52 ++++++++++++++++++++++++--- src/Interpreters/ClusterDiscovery.h | 10 +++--- 2 files changed, 53 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 4398c8d20b4..df705b51cc5 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -19,6 +19,11 @@ #include #include +#include +#include +#include +#include + namespace DB { @@ -132,10 +137,16 @@ ClusterDiscovery::NodesInfo ClusterDiscovery::getNodes(zkutil::ZooKeeperPtr & zk for (const auto & node_uuid : node_uuids) { String payload; - bool ok = zk->tryGet(getShardsListPath(zk_root) / node_uuid, payload); - if (!ok) + if (!zk->tryGet(getShardsListPath(zk_root) / node_uuid, payload)) + { + LOG_WARNING(log, "Error getting data from node '{}' in '{}'", node_uuid, zk_root); return {}; - result.emplace(node_uuid, NodeInfo(payload)); + } + if (!NodeInfo::parse(payload, result[node_uuid])) + { + LOG_WARNING(log, "Error parsing data from node '{}' in '{}'", node_uuid, zk_root); + return {}; + } } return result; } @@ -261,9 +272,9 @@ void ClusterDiscovery::registerInZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & inf String node_path = getShardsListPath(info.zk_root) / node_name; zk->createAncestors(node_path); - String payload = getFQDNOrHostName() + ":" + toString(server_port); + NodeInfo self_node(getFQDNOrHostName() + ":" + toString(server_port)); - zk->createOrUpdate(node_path, payload, zkutil::CreateMode::Ephemeral); + zk->createOrUpdate(node_path, self_node.serialize(), zkutil::CreateMode::Ephemeral); LOG_DEBUG(log, "Current node {} registered in cluster {}", node_name, info.name); } @@ -322,4 +333,35 @@ ClusterDiscovery::~ClusterDiscovery() ClusterDiscovery::shutdown(); } +bool ClusterDiscovery::NodeInfo::parse(const String & data, NodeInfo & result) +{ + try + { + Poco::JSON::Parser parser; + auto json = parser.parse(data).extract(); + + result.address = json->getValue("address"); + } + catch (Poco::Exception & e) + { + LOG_WARNING( + &Poco::Logger::get("ClusterDiscovery"), + "Can't parse '{}' from node: {}", + data.size() < 1024 ? data : "[data too long]", e.displayText()); + return false; + } + return true; +} + +String ClusterDiscovery::NodeInfo::serialize() const +{ + Poco::JSON::Object json; + json.set("address", address); + + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + return oss.str(); +} + } diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index 0e32479ac7b..6337aec37e7 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -40,9 +40,11 @@ private: /// host:port String address; - explicit NodeInfo(const String & data) - : address(data) - {} + NodeInfo() = default; + explicit NodeInfo(const String & address_) : address(address_) {} + + static bool parse(const String & data, NodeInfo & result); + String serialize() const; }; // node uuid -> address ("host:port") @@ -65,7 +67,7 @@ private: int * version = nullptr, bool set_callback = true); - static NodesInfo getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & node_uuids); + NodesInfo getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & node_uuids); ClusterPtr getCluster(const ClusterInfo & cluster_info); From 7404205f37bc256153e78798aef61508b2d91681 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 18 Nov 2021 11:57:26 +0300 Subject: [PATCH 0438/1260] fix cluster discovery startup race --- programs/server/Server.cpp | 3 +- src/Interpreters/ClusterDiscovery.cpp | 67 ++++++++++++++------------- src/Interpreters/ClusterDiscovery.h | 4 +- src/Interpreters/Context.cpp | 4 +- src/Interpreters/Context.h | 2 +- 5 files changed, 41 insertions(+), 39 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index d701778b982..ba2c2ae1e5e 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1555,8 +1555,7 @@ if (ThreadFuzzer::instance().isEffective()) server.start(); LOG_INFO(log, "Ready for connections."); - if (has_zookeeper) - global_context->registerNodeForClusterDiscovery(); + global_context->startClusterDiscovery(); SCOPE_EXIT_SAFE({ LOG_DEBUG(log, "Received termination signal."); diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index df705b51cc5..e2b20c8be83 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -57,22 +57,13 @@ public: flags.emplace(*it, false); } - void set(const T & val) + void set(const T & key) { - auto it = flags.find(val); - if (it == flags.end()) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown value '{}'", val); - it->second = true; + setFlag(key, true); cv.notify_one(); } - void unset(const T & val) - { - auto it = flags.find(val); - if (it == flags.end()) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown value '{}'", val); - it->second = false; - } + void unset(const T & key) { setFlag(key, false); } void wait(std::chrono::milliseconds timeout) { @@ -80,12 +71,18 @@ public: cv.wait_for(lk, timeout); } - const std::unordered_map & get() - { - return flags; - } + const std::unordered_map & get() { return flags; } private: + + void setFlag(const T & key, bool value) + { + auto it = flags.find(key); + if (it == flags.end()) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown value '{}'", key); + it->second = value; + } + std::condition_variable cv; std::mutex mu; @@ -94,9 +91,9 @@ private: ClusterDiscovery::ClusterDiscovery( const Poco::Util::AbstractConfiguration & config, - ContextMutablePtr context_, + ContextPtr context_, const String & config_prefix) - : context(context_) + : context(Context::createCopy(context_)) , node_name(toString(ServerUUID::get())) , server_port(context->getTCPPort()) , log(&Poco::Logger::get("ClusterDiscovery")) @@ -280,27 +277,33 @@ void ClusterDiscovery::registerInZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & inf void ClusterDiscovery::start() { - auto zk = context->getZooKeeper(); - + if (clusters_info.empty()) + { + LOG_DEBUG(log, "No defined clusters for discovery"); + return; + } LOG_TRACE(log, "Starting working thread"); main_thread = ThreadFromGlobalPool([this] { runMainThread(); }); - - for (auto & [_, info] : clusters_info) - { - registerInZk(zk, info); - if (!updateCluster(info)) - { - LOG_WARNING(log, "Error on updating cluster '{}', will retry", info.name); - clusters_to_update->set(info.name); - } - } } void ClusterDiscovery::runMainThread() { - LOG_TRACE(log, "Worker thread started"); + LOG_DEBUG(log, "Worker thread started"); // setThreadName("ClusterDiscovery"); + { + auto zk = context->getZooKeeper(); + for (auto & [_, info] : clusters_info) + { + registerInZk(zk, info); + if (!updateCluster(info)) + { + LOG_WARNING(log, "Error on updating cluster '{}', will retry", info.name); + clusters_to_update->set(info.name); + } + } + } + using namespace std::chrono_literals; while (!stop_flag) @@ -316,7 +319,7 @@ void ClusterDiscovery::runMainThread() clusters_to_update->unset(cluster_name); } } - LOG_TRACE(log, "Worker thread stopped"); + LOG_DEBUG(log, "Worker thread stopped"); } void ClusterDiscovery::shutdown() diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index 6337aec37e7..9ad7393582a 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -27,8 +27,8 @@ class ClusterDiscovery public: ClusterDiscovery( const Poco::Util::AbstractConfiguration & config, - ContextMutablePtr context_, - const String & config_prefix = "remote_servers_discovery"); + ContextPtr context_, + const String & config_prefix = "remote_servers_discovery"); // TODO(@vdimir) use `remote_servers` void start(); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a36b0805a60..aec0b4ee894 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2197,7 +2197,7 @@ std::shared_ptr Context::getClusters() const return shared->clusters; } -void Context::registerNodeForClusterDiscovery() +void Context::startClusterDiscovery() { if (!shared->cluster_discovery) return; @@ -2212,7 +2212,7 @@ void Context::setClustersConfig(const ConfigurationPtr & config, bool enable_dis if (enable_discovery && !shared->cluster_discovery) { - shared->cluster_discovery = std::make_unique(*config, shared_from_this()); + shared->cluster_discovery = std::make_unique(*config, getGlobalContext()); } /// Do not update clusters if this part of config wasn't changed. diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index c528e995ae9..6b0a4671efb 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -752,7 +752,7 @@ public: std::shared_ptr tryGetCluster(const std::string & cluster_name) const; void setClustersConfig(const ConfigurationPtr & config, bool enable_discovery = false, const String & config_name = "remote_servers"); - void registerNodeForClusterDiscovery(); + void startClusterDiscovery(); /// Sets custom cluster, but doesn't update configuration void setCluster(const String & cluster_name, const std::shared_ptr & cluster); From ceec643f90d8044519f47c34997ae36be58a652f Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 18 Nov 2021 12:45:57 +0300 Subject: [PATCH 0439/1260] update config structure: remote_servers_discovery -> remote_servers.discovery --- src/Interpreters/Cluster.cpp | 8 +++---- src/Interpreters/ClusterDiscovery.cpp | 5 +++- src/Interpreters/ClusterDiscovery.h | 2 +- .../test_cluster_discovery/config/config.xml | 24 ++++++++++++++++--- .../test_cluster_discovery/test.py | 3 +++ 5 files changed, 33 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index ca599bd83a7..ce7cb79e5e8 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -327,6 +327,10 @@ void Clusters::updateClusters(const Poco::Util::AbstractConfiguration & new_conf for (const auto & key : new_config_keys) { + if (new_config.has(config_prefix + "." + key + ".discovery")) + /// Handled in ClusterDiscovery + continue; + if (key.find('.') != String::npos) throw Exception("Cluster names with dots are not supported: '" + key + "'", ErrorCodes::SYNTAX_ERROR); @@ -489,10 +493,6 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, internal_replication }); } - else if (startsWith(key, "discovery")) - { - continue; - } else throw Exception("Unknown element in config: " + key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index e2b20c8be83..713cae6eec8 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -103,7 +103,10 @@ ClusterDiscovery::ClusterDiscovery( for (const auto & key : config_keys) { - String path = config.getString(config_prefix + "." + key + ".path"); + String prefix = config_prefix + "." + key + ".discovery"; + if (!config.has(prefix)) + continue; + String path = config.getString(prefix + ".path"); trimRight(path, '/'); clusters_info.emplace(key, ClusterInfo(key, path)); } diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index 9ad7393582a..8e2f0f48d95 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -28,7 +28,7 @@ public: ClusterDiscovery( const Poco::Util::AbstractConfiguration & config, ContextPtr context_, - const String & config_prefix = "remote_servers_discovery"); // TODO(@vdimir) use `remote_servers` + const String & config_prefix = "remote_servers"); void start(); diff --git a/tests/integration/test_cluster_discovery/config/config.xml b/tests/integration/test_cluster_discovery/config/config.xml index 00595c77441..2ead4142d5c 100644 --- a/tests/integration/test_cluster_discovery/config/config.xml +++ b/tests/integration/test_cluster_discovery/config/config.xml @@ -1,7 +1,25 @@ - + + - /clickhouse/discovery/test_auto_cluster + + /clickhouse/discovery/test_auto_cluster + - + + + + + + node1 + 9000 + + + node2 + 9000 + + + + + diff --git a/tests/integration/test_cluster_discovery/test.py b/tests/integration/test_cluster_discovery/test.py index a85babf3ebe..121da32c766 100644 --- a/tests/integration/test_cluster_discovery/test.py +++ b/tests/integration/test_cluster_discovery/test.py @@ -28,6 +28,7 @@ def check_nodes_count_in_cluster(nodes, expected, cluster_name, *, retries=5): """ Check nodes count in system.clusters for specified cluster """ + assert 1 <= retries <= 6 for retry in range(1, retries + 1): nodes_cnt = [ @@ -62,3 +63,5 @@ def test_cluster_discovery_startup_and_stop(start_cluster): nodes[3].start_clickhouse() check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes), 'test_auto_cluster') + + check_nodes_count_in_cluster([nodes[1], nodes[2]], 2, 'two_shards', retries=1) From 96a0371ecab491add972cfbcb6fb35d08ac13b0a Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 18 Nov 2021 12:52:33 +0300 Subject: [PATCH 0440/1260] add option allow_experimental_cluster_discovery --- src/Core/Settings.h | 2 ++ src/Interpreters/Context.cpp | 2 +- .../test_cluster_discovery/config/settings.xml | 12 ++++++++++++ tests/integration/test_cluster_discovery/test.py | 1 + 4 files changed, 16 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_cluster_discovery/config/settings.xml diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 70fb5604997..b955f5f54ed 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -549,6 +549,8 @@ class IColumn; /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ + \ + M(Bool, allow_experimental_cluster_discovery, false, "Enable automatic cluster discovery via ZooKeeper", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index aec0b4ee894..a7b2d2c2909 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2210,7 +2210,7 @@ void Context::setClustersConfig(const ConfigurationPtr & config, bool enable_dis { std::lock_guard lock(shared->clusters_mutex); - if (enable_discovery && !shared->cluster_discovery) + if (/* getSettingsRef().allow_experimental_cluster_discovery && */ enable_discovery && !shared->cluster_discovery) { shared->cluster_discovery = std::make_unique(*config, getGlobalContext()); } diff --git a/tests/integration/test_cluster_discovery/config/settings.xml b/tests/integration/test_cluster_discovery/config/settings.xml new file mode 100644 index 00000000000..69c24d8b2ec --- /dev/null +++ b/tests/integration/test_cluster_discovery/config/settings.xml @@ -0,0 +1,12 @@ + + + + 1 + + + + + default + + + diff --git a/tests/integration/test_cluster_discovery/test.py b/tests/integration/test_cluster_discovery/test.py index 121da32c766..d100d974df9 100644 --- a/tests/integration/test_cluster_discovery/test.py +++ b/tests/integration/test_cluster_discovery/test.py @@ -9,6 +9,7 @@ nodes = [ cluster.add_instance( f'node{i}', main_configs=['config/config.xml'], + user_configs=['config/settings.xml'], stay_alive=True, with_zookeeper=True ) for i in range(5) From 57cbbd9fcffaaf41f10e5d344cb14fe30ad02e6f Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 19 Nov 2021 10:20:37 +0300 Subject: [PATCH 0441/1260] global option allow_experimental_cluster_discovery --- src/Core/Settings.h | 2 -- src/Interpreters/Context.cpp | 3 +-- .../test_cluster_discovery/config/config.xml | 4 +--- .../test_cluster_discovery/config/settings.xml | 12 ------------ tests/integration/test_cluster_discovery/test.py | 1 - 5 files changed, 2 insertions(+), 20 deletions(-) delete mode 100644 tests/integration/test_cluster_discovery/config/settings.xml diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b955f5f54ed..70fb5604997 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -549,8 +549,6 @@ class IColumn; /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ - \ - M(Bool, allow_experimental_cluster_discovery, false, "Enable automatic cluster discovery via ZooKeeper", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a7b2d2c2909..69cf2dc11d0 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2209,8 +2209,7 @@ void Context::startClusterDiscovery() void Context::setClustersConfig(const ConfigurationPtr & config, bool enable_discovery, const String & config_name) { std::lock_guard lock(shared->clusters_mutex); - - if (/* getSettingsRef().allow_experimental_cluster_discovery && */ enable_discovery && !shared->cluster_discovery) + if (config->getBool("allow_experimental_cluster_discovery", false) && enable_discovery && !shared->cluster_discovery) { shared->cluster_discovery = std::make_unique(*config, getGlobalContext()); } diff --git a/tests/integration/test_cluster_discovery/config/config.xml b/tests/integration/test_cluster_discovery/config/config.xml index 2ead4142d5c..70cb010fe0e 100644 --- a/tests/integration/test_cluster_discovery/config/config.xml +++ b/tests/integration/test_cluster_discovery/config/config.xml @@ -1,12 +1,11 @@ + 1 - /clickhouse/discovery/test_auto_cluster - @@ -20,6 +19,5 @@ - diff --git a/tests/integration/test_cluster_discovery/config/settings.xml b/tests/integration/test_cluster_discovery/config/settings.xml deleted file mode 100644 index 69c24d8b2ec..00000000000 --- a/tests/integration/test_cluster_discovery/config/settings.xml +++ /dev/null @@ -1,12 +0,0 @@ - - - - 1 - - - - - default - - - diff --git a/tests/integration/test_cluster_discovery/test.py b/tests/integration/test_cluster_discovery/test.py index d100d974df9..121da32c766 100644 --- a/tests/integration/test_cluster_discovery/test.py +++ b/tests/integration/test_cluster_discovery/test.py @@ -9,7 +9,6 @@ nodes = [ cluster.add_instance( f'node{i}', main_configs=['config/config.xml'], - user_configs=['config/settings.xml'], stay_alive=True, with_zookeeper=True ) for i in range(5) From 7a387163603a3833a0386722094880320b4e0de0 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 19 Nov 2021 10:21:40 +0300 Subject: [PATCH 0442/1260] add 'secure' option to cluster discovery --- src/Interpreters/ClusterDiscovery.cpp | 28 +++++++++++++++++++-------- src/Interpreters/ClusterDiscovery.h | 2 ++ 2 files changed, 22 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 713cae6eec8..68203cf2eeb 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -194,22 +194,33 @@ ClusterPtr ClusterDiscovery::getCluster(const ClusterInfo & cluster_info) { Strings replica_adresses; replica_adresses.reserve(cluster_info.nodes_info.size()); + + std::optional secure; for (const auto & [_, node] : cluster_info.nodes_info) + { + if (secure && secure.value() != node.secure) + { + LOG_WARNING(log, "Nodes in cluster '{}' has different 'secure' value", cluster_info.name); + } + + secure = node.secure; replica_adresses.emplace_back(node.address); + } + + // TODO(vdimir@) save custom params from config to zookeeper and use here (like secure), split by shards std::vector> shards = {replica_adresses}; - bool secure = false; - auto maybe_secure_port = context->getTCPPortSecure(); + auto secure_port_opt = context->getTCPPortSecure(); auto cluster = std::make_shared( context->getSettings(), shards, - context->getUserName(), - "", - (secure ? (maybe_secure_port ? *maybe_secure_port : DBMS_DEFAULT_SECURE_PORT) : context->getTCPPort()), - false /* treat_local_as_remote */, - context->getApplicationType() == Context::ApplicationType::LOCAL /* treat_local_port_as_remote */, - secure); + /* username= */ context->getUserName(), + /* password= */ "", + /* clickhouse_port= */ (secure.value_or(false) ? secure_port_opt.value_or(DBMS_DEFAULT_SECURE_PORT) : context->getTCPPort()), + /* treat_local_as_remote= */ false, + /* treat_local_port_as_remote= */ context->getApplicationType() == Context::ApplicationType::LOCAL, + secure.value_or(false)); return cluster; } @@ -347,6 +358,7 @@ bool ClusterDiscovery::NodeInfo::parse(const String & data, NodeInfo & result) auto json = parser.parse(data).extract(); result.address = json->getValue("address"); + result.secure = json->optValue("secure", false); } catch (Poco::Exception & e) { diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index 8e2f0f48d95..a68e6c80ca3 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -39,6 +39,8 @@ private: { /// host:port String address; + /// is secure tcp port user + bool secure = false; NodeInfo() = default; explicit NodeInfo(const String & address_) : address(address_) {} From fbdb5c60bd4cac8731d1b463ddf3fd99f6625607 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 19 Nov 2021 12:42:00 +0300 Subject: [PATCH 0443/1260] initial initialization of cluster discovery in main thread --- programs/server/Server.cpp | 9 +++- src/Interpreters/ClusterDiscovery.cpp | 45 ++++++++++++------- src/Interpreters/ClusterDiscovery.h | 2 +- .../test_cluster_discovery/test.py | 8 ++-- 4 files changed, 41 insertions(+), 23 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ba2c2ae1e5e..30123004bbb 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1555,7 +1555,14 @@ if (ThreadFuzzer::instance().isEffective()) server.start(); LOG_INFO(log, "Ready for connections."); - global_context->startClusterDiscovery(); + try + { + global_context->startClusterDiscovery(); + } + catch (...) + { + tryLogCurrentException(log, "Caught exception while starting cluster discovery"); + } SCOPE_EXIT_SAFE({ LOG_DEBUG(log, "Received termination signal."); diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 68203cf2eeb..1fe4f4681f0 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -190,7 +190,7 @@ bool ClusterDiscovery::needUpdate(const Strings & node_uuids, const NodesInfo & return has_difference; } -ClusterPtr ClusterDiscovery::getCluster(const ClusterInfo & cluster_info) +ClusterPtr ClusterDiscovery::makeCluster(const ClusterInfo & cluster_info) { Strings replica_adresses; replica_adresses.reserve(cluster_info.nodes_info.size()); @@ -262,7 +262,7 @@ bool ClusterDiscovery::updateCluster(ClusterInfo & cluster_info) return false; } - auto cluster = getCluster(cluster_info); + auto cluster = makeCluster(cluster_info); context->setCluster(cluster_info.name, cluster); return true; } @@ -280,6 +280,8 @@ bool ClusterDiscovery::updateCluster(const String & cluster_name) void ClusterDiscovery::registerInZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & info) { + LOG_DEBUG(log, "Registering current node {} in cluster {}", node_name, info.name); + String node_path = getShardsListPath(info.zk_root) / node_name; zk->createAncestors(node_path); @@ -297,26 +299,35 @@ void ClusterDiscovery::start() return; } LOG_TRACE(log, "Starting working thread"); - main_thread = ThreadFromGlobalPool([this] { runMainThread(); }); + + auto zk = context->getZooKeeper(); + for (auto & [_, info] : clusters_info) + { + registerInZk(zk, info); + if (!updateCluster(info)) + { + LOG_WARNING(log, "Error on updating cluster '{}', will retry", info.name); + clusters_to_update->set(info.name); + } + } + + main_thread = ThreadFromGlobalPool([this] + { + try + { + runMainThread(); + } + catch (...) + { + tryLogCurrentException(log, "Caught exception in cluster discovery runMainThread"); + } + }); } void ClusterDiscovery::runMainThread() { + setThreadName("ClusterDiscover"); LOG_DEBUG(log, "Worker thread started"); - // setThreadName("ClusterDiscovery"); - - { - auto zk = context->getZooKeeper(); - for (auto & [_, info] : clusters_info) - { - registerInZk(zk, info); - if (!updateCluster(info)) - { - LOG_WARNING(log, "Error on updating cluster '{}', will retry", info.name); - clusters_to_update->set(info.name); - } - } - } using namespace std::chrono_literals; diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index a68e6c80ca3..c6fc17ca873 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -71,7 +71,7 @@ private: NodesInfo getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & node_uuids); - ClusterPtr getCluster(const ClusterInfo & cluster_info); + ClusterPtr makeCluster(const ClusterInfo & cluster_info); bool needUpdate(const Strings & node_uuids, const NodesInfo & nodes); bool updateCluster(const String & cluster_name); diff --git a/tests/integration/test_cluster_discovery/test.py b/tests/integration/test_cluster_discovery/test.py index 121da32c766..bdcd749f244 100644 --- a/tests/integration/test_cluster_discovery/test.py +++ b/tests/integration/test_cluster_discovery/test.py @@ -31,11 +31,11 @@ def check_nodes_count_in_cluster(nodes, expected, cluster_name, *, retries=5): assert 1 <= retries <= 6 for retry in range(1, retries + 1): - nodes_cnt = [ - int(node.query(f"SELECT count() FROM system.clusters WHERE cluster = '{cluster_name}'")) + nodes_cnt = { + node.name: int(node.query(f"SELECT count() FROM system.clusters WHERE cluster = '{cluster_name}'")) for node in nodes - ] - if all(actual == expected for actual in nodes_cnt): + } + if all(actual == expected for actual in nodes_cnt.values()): break if retry != retries: From d75a6b7abcd7af61c6380008c9d595279e018533 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 25 Nov 2021 16:45:38 +0300 Subject: [PATCH 0444/1260] more logging for cluster discovery --- src/Interpreters/ClusterDiscovery.cpp | 60 ++++++++++++++++----------- src/Interpreters/ClusterDiscovery.h | 2 +- 2 files changed, 36 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 1fe4f4681f0..48e998c357a 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -94,10 +94,12 @@ ClusterDiscovery::ClusterDiscovery( ContextPtr context_, const String & config_prefix) : context(Context::createCopy(context_)) - , node_name(toString(ServerUUID::get())) + , current_node_name(toString(ServerUUID::get())) , server_port(context->getTCPPort()) , log(&Poco::Logger::get("ClusterDiscovery")) { + LOG_DEBUG(log, "Cluster discovery is enabled"); + Poco::Util::AbstractConfiguration::Keys config_keys; config.keys(config_prefix, config_keys); @@ -137,14 +139,11 @@ ClusterDiscovery::NodesInfo ClusterDiscovery::getNodes(zkutil::ZooKeeperPtr & zk for (const auto & node_uuid : node_uuids) { String payload; - if (!zk->tryGet(getShardsListPath(zk_root) / node_uuid, payload)) + bool ok = zk->tryGet(getShardsListPath(zk_root) / node_uuid, payload) && + NodeInfo::parse(payload, result[node_uuid]); + if (!ok) { - LOG_WARNING(log, "Error getting data from node '{}' in '{}'", node_uuid, zk_root); - return {}; - } - if (!NodeInfo::parse(payload, result[node_uuid])) - { - LOG_WARNING(log, "Error parsing data from node '{}' in '{}'", node_uuid, zk_root); + LOG_WARNING(log, "Can't get data from node '{}' in '{}'", node_uuid, zk_root); return {}; } } @@ -159,11 +158,8 @@ bool ClusterDiscovery::needUpdate(const Strings & node_uuids, const NodesInfo & { bool has_difference = node_uuids.size() != nodes.size() || std::any_of(node_uuids.begin(), node_uuids.end(), [&nodes] (auto u) { return !nodes.contains(u); }); - UNUSED(log); - #if defined(NDEBUG) || defined(ADDRESS_SANITIZER) || defined(THREAD_SANITIZER) || defined(MEMORY_SANITIZER) || defined(UNDEFINED_BEHAVIOR_SANITIZER) { - /// Just to log updated nodes, suboptimal, but should be ok for expected update sizes. - /// Disabled on release build. + /// Just to log updated nodes, suboptimal, but should be ok for expected update sizes std::set new_names(node_uuids.begin(), node_uuids.end()); std::set old_names; for (const auto & [name, _] : nodes) @@ -176,17 +172,19 @@ bool ClusterDiscovery::needUpdate(const Strings & node_uuids, const NodesInfo & constexpr size_t max_to_show = 3; size_t sz = diff.size(); - if (sz > max_to_show) + bool need_crop = sz > max_to_show; + if (need_crop) diff.resize(max_to_show); - return fmt::format("{} nodes ({}{})", sz, fmt::join(diff, ", "), diff.size() == sz ? "" : ",..."); + + if (sz == 0) + return fmt::format("{} nodes", sz); + return fmt::format("{} nodes ({}{})", sz, fmt::join(diff, ", "), need_crop ? "" : ",..."); }; - LOG_DEBUG(log, "Cluster update: added: {}, removed: {}", + LOG_DEBUG(log, "Cluster update: added {}, removed {}", format_cluster_update(new_names, old_names), format_cluster_update(old_names, new_names)); - } - #endif return has_difference; } @@ -205,7 +203,6 @@ ClusterPtr ClusterDiscovery::makeCluster(const ClusterInfo & cluster_info) secure = node.secure; replica_adresses.emplace_back(node.address); - } // TODO(vdimir@) save custom params from config to zookeeper and use here (like secure), split by shards @@ -235,9 +232,9 @@ bool ClusterDiscovery::updateCluster(ClusterInfo & cluster_info) int start_version; Strings node_uuids = getNodeNames(zk, cluster_info.zk_root, cluster_info.name, &start_version, false); - if (node_uuids.empty()) + if (std::find(node_uuids.begin(), node_uuids.end(), current_node_name) == node_uuids.end()) { - LOG_ERROR(log, "Can't find any node in cluster '{}', will register again", cluster_info.name); + LOG_ERROR(log, "Can't find current node in cluster '{}', will register again", cluster_info.name); registerInZk(zk, cluster_info); return false; } @@ -251,17 +248,23 @@ bool ClusterDiscovery::updateCluster(ClusterInfo & cluster_info) nodes_info = getNodes(zk, cluster_info.zk_root, node_uuids); if (nodes_info.empty()) + { + LOG_WARNING(log, "Can't get nodes info for '{}'", cluster_info.name); return false; + } int current_version; getNodeNames(zk, cluster_info.zk_root, cluster_info.name, ¤t_version, true); if (current_version != start_version) { + LOG_TRACE(log, "Cluster '{}' configuration changed during update", cluster_info.name); nodes_info.clear(); return false; } + LOG_TRACE(log, "Updating system.clusters record for '{}' with {} nodes", cluster_info.name, cluster_info.nodes_info.size()); + auto cluster = makeCluster(cluster_info); context->setCluster(cluster_info.name, cluster); return true; @@ -280,15 +283,15 @@ bool ClusterDiscovery::updateCluster(const String & cluster_name) void ClusterDiscovery::registerInZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & info) { - LOG_DEBUG(log, "Registering current node {} in cluster {}", node_name, info.name); + LOG_DEBUG(log, "Registering current node {} in cluster {}", current_node_name, info.name); - String node_path = getShardsListPath(info.zk_root) / node_name; + String node_path = getShardsListPath(info.zk_root) / current_node_name; zk->createAncestors(node_path); NodeInfo self_node(getFQDNOrHostName() + ":" + toString(server_port)); zk->createOrUpdate(node_path, self_node.serialize(), zkutil::CreateMode::Ephemeral); - LOG_DEBUG(log, "Current node {} registered in cluster {}", node_name, info.name); + LOG_DEBUG(log, "Current node {} registered in cluster {}", current_node_name, info.name); } void ClusterDiscovery::start() @@ -339,9 +342,16 @@ void ClusterDiscovery::runMainThread() { if (!need_update) continue; - bool ok = updateCluster(cluster_name); - if (ok) + + if (updateCluster(cluster_name)) + { clusters_to_update->unset(cluster_name); + LOG_DEBUG(log, "Cluster '{}' updated successfully", cluster_name); + } + else + { + LOG_DEBUG(log, "Cluster '{}' are not updated, will retry", cluster_name); + } } } LOG_DEBUG(log, "Worker thread stopped"); diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index c6fc17ca873..91441caa086 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -85,7 +85,7 @@ private: ContextMutablePtr context; - String node_name; + String current_node_name; UInt16 server_port; template class ConcurrentFlags; From f47ce534f2e379a03c5a0cf7164fdd3cc39646c0 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Nov 2021 14:25:52 +0300 Subject: [PATCH 0445/1260] preserve auto clusters on config update --- src/Interpreters/Cluster.cpp | 16 ++++++++++++++-- src/Interpreters/Cluster.h | 6 ++++++ src/Interpreters/ClusterDiscovery.cpp | 7 ++++--- src/Interpreters/Context.cpp | 2 +- 4 files changed, 25 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index ce7cb79e5e8..b7b6b84439b 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -320,16 +320,28 @@ void Clusters::updateClusters(const Poco::Util::AbstractConfiguration & new_conf if (old_config) { for (const auto & key : deleted_keys) - impl.erase(key); + { + if (!automatic_clusters.contains(key)) + impl.erase(key); + } } else - impl.clear(); + { + if (!automatic_clusters.empty()) + std::erase_if(impl, [this](const auto & e) { return automatic_clusters.contains(e.first); }); + else + impl.clear(); + } + for (const auto & key : new_config_keys) { if (new_config.has(config_prefix + "." + key + ".discovery")) + { /// Handled in ClusterDiscovery + automatic_clusters.insert(key); continue; + } if (key.find('.') != String::npos) throw Exception("Cluster names with dots are not supported: '" + key + "'", ErrorCodes::SYNTAX_ERROR); diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index 1232e004ca7..3773dadaf13 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -6,6 +6,8 @@ #include #include +#include +#include namespace Poco { @@ -300,6 +302,10 @@ public: Impl getContainer() const; protected: + + /// setup outside of this class, stored to prevent deleting from impl on config update + std::unordered_set automatic_clusters; + Impl impl; mutable std::mutex mutex; }; diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 48e998c357a..0540bbd8641 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -178,10 +178,10 @@ bool ClusterDiscovery::needUpdate(const Strings & node_uuids, const NodesInfo & if (sz == 0) return fmt::format("{} nodes", sz); - return fmt::format("{} nodes ({}{})", sz, fmt::join(diff, ", "), need_crop ? "" : ",..."); + return fmt::format("{} node{} [{}{}]", sz, sz != 1 ? "s" : "", fmt::join(diff, ", "), need_crop ? ",..." : ""); }; - LOG_DEBUG(log, "Cluster update: added {}, removed {}", + LOG_TRACE(log, "Cluster update: added {}, removed {}", format_cluster_update(new_names, old_names), format_cluster_update(old_names, new_names)); } @@ -231,15 +231,16 @@ bool ClusterDiscovery::updateCluster(ClusterInfo & cluster_info) int start_version; Strings node_uuids = getNodeNames(zk, cluster_info.zk_root, cluster_info.name, &start_version, false); + auto & nodes_info = cluster_info.nodes_info; if (std::find(node_uuids.begin(), node_uuids.end(), current_node_name) == node_uuids.end()) { LOG_ERROR(log, "Can't find current node in cluster '{}', will register again", cluster_info.name); registerInZk(zk, cluster_info); + nodes_info.clear(); return false; } - auto & nodes_info = cluster_info.nodes_info; if (!needUpdate(node_uuids, nodes_info)) { LOG_TRACE(log, "No update required for cluster '{}'", cluster_info.name); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 69cf2dc11d0..14b0f65072a 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2222,7 +2222,7 @@ void Context::setClustersConfig(const ConfigurationPtr & config, bool enable_dis shared->clusters_config = config; if (!shared->clusters) - shared->clusters = std::make_unique(*shared->clusters_config, settings, config_name); + shared->clusters = std::make_shared(*shared->clusters_config, settings, config_name); else shared->clusters->updateClusters(*shared->clusters_config, settings, config_name, old_clusters_config); } From d3b1058c40f1bad51a5e3a4dff41e80e6f60e0f7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Nov 2021 15:35:28 +0300 Subject: [PATCH 0446/1260] cluster discovery configuration, support shards --- src/Interpreters/ClusterDiscovery.cpp | 52 +++++++++++-------- src/Interpreters/ClusterDiscovery.h | 19 +++++-- .../config/config_shard1.xml | 24 +++++++++ .../config/config_shard3.xml | 24 +++++++++ .../test_cluster_discovery/test.py | 40 +++++++++----- 5 files changed, 121 insertions(+), 38 deletions(-) create mode 100644 tests/integration/test_cluster_discovery/config/config_shard1.xml create mode 100644 tests/integration/test_cluster_discovery/config/config_shard3.xml diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 0540bbd8641..6f0bf9fe389 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -95,7 +95,6 @@ ClusterDiscovery::ClusterDiscovery( const String & config_prefix) : context(Context::createCopy(context_)) , current_node_name(toString(ServerUUID::get())) - , server_port(context->getTCPPort()) , log(&Poco::Logger::get("ClusterDiscovery")) { LOG_DEBUG(log, "Cluster discovery is enabled"); @@ -108,9 +107,17 @@ ClusterDiscovery::ClusterDiscovery( String prefix = config_prefix + "." + key + ".discovery"; if (!config.has(prefix)) continue; - String path = config.getString(prefix + ".path"); - trimRight(path, '/'); - clusters_info.emplace(key, ClusterInfo(key, path)); + + clusters_info.emplace( + key, + ClusterInfo( + /* name_= */ key, + /* zk_root_= */ config.getString(prefix + ".path"), + /* port= */ context->getTCPPort(), + /* secure= */ config.getBool(prefix + ".secure", false), + /* shard_id= */ config.getUInt(prefix + ".shard", 0) + ) + ); } clusters_to_update = std::make_shared(config_keys.begin(), config_keys.end()); } @@ -190,34 +197,35 @@ bool ClusterDiscovery::needUpdate(const Strings & node_uuids, const NodesInfo & ClusterPtr ClusterDiscovery::makeCluster(const ClusterInfo & cluster_info) { - Strings replica_adresses; - replica_adresses.reserve(cluster_info.nodes_info.size()); - - std::optional secure; - for (const auto & [_, node] : cluster_info.nodes_info) + std::vector> shards; { - if (secure && secure.value() != node.secure) + std::map replica_adresses; + + for (const auto & [_, node] : cluster_info.nodes_info) { - LOG_WARNING(log, "Nodes in cluster '{}' has different 'secure' value", cluster_info.name); + if (cluster_info.current_node.secure != node.secure) + { + LOG_WARNING(log, "Node '{}' in cluster '{}' has different 'secure' value, skipping it", node.address, cluster_info.name); + continue; + } + replica_adresses[node.shard_id].emplace_back(node.address); } - secure = node.secure; - replica_adresses.emplace_back(node.address); + shards.reserve(replica_adresses.size()); + for (auto & [_, replicas] : replica_adresses) + shards.emplace_back(std::move(replicas)); } - // TODO(vdimir@) save custom params from config to zookeeper and use here (like secure), split by shards - std::vector> shards = {replica_adresses}; - - auto secure_port_opt = context->getTCPPortSecure(); + bool secure = cluster_info.current_node.secure; auto cluster = std::make_shared( context->getSettings(), shards, /* username= */ context->getUserName(), /* password= */ "", - /* clickhouse_port= */ (secure.value_or(false) ? secure_port_opt.value_or(DBMS_DEFAULT_SECURE_PORT) : context->getTCPPort()), + /* clickhouse_port= */ secure ? context->getTCPPortSecure().value_or(DBMS_DEFAULT_SECURE_PORT) : context->getTCPPort(), /* treat_local_as_remote= */ false, /* treat_local_port_as_remote= */ context->getApplicationType() == Context::ApplicationType::LOCAL, - secure.value_or(false)); + /* secure= */ secure); return cluster; } @@ -289,9 +297,7 @@ void ClusterDiscovery::registerInZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & inf String node_path = getShardsListPath(info.zk_root) / current_node_name; zk->createAncestors(node_path); - NodeInfo self_node(getFQDNOrHostName() + ":" + toString(server_port)); - - zk->createOrUpdate(node_path, self_node.serialize(), zkutil::CreateMode::Ephemeral); + zk->createOrUpdate(node_path, info.current_node.serialize(), zkutil::CreateMode::Ephemeral); LOG_DEBUG(log, "Current node {} registered in cluster {}", current_node_name, info.name); } @@ -381,6 +387,7 @@ bool ClusterDiscovery::NodeInfo::parse(const String & data, NodeInfo & result) result.address = json->getValue("address"); result.secure = json->optValue("secure", false); + result.shard_id = json->optValue("shard_id", 0); } catch (Poco::Exception & e) { @@ -397,6 +404,7 @@ String ClusterDiscovery::NodeInfo::serialize() const { Poco::JSON::Object json; json.set("address", address); + json.set("shard_id", shard_id); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index 91441caa086..5b0f7211502 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -41,9 +42,15 @@ private: String address; /// is secure tcp port user bool secure = false; + /// shard number + size_t shard_id = 0; NodeInfo() = default; - explicit NodeInfo(const String & address_) : address(address_) {} + explicit NodeInfo(const String & address_, bool secure_, size_t shard_id_) + : address(address_) + , secure(secure_) + , shard_id(shard_id_) + {} static bool parse(const String & data, NodeInfo & result); String serialize() const; @@ -58,7 +65,14 @@ private: const String zk_root; NodesInfo nodes_info; - explicit ClusterInfo(const String & name_, const String & zk_root_) : name(name_), zk_root(zk_root_) {} + NodeInfo current_node; + + explicit ClusterInfo(const String & name_, const String & zk_root_, UInt16 port, bool secure, size_t shard_id) + : name(name_) + , zk_root(zk_root_) + , current_node(getFQDNOrHostName() + ":" + toString(port), secure, shard_id) + { + } }; void registerInZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & info); @@ -86,7 +100,6 @@ private: ContextMutablePtr context; String current_node_name; - UInt16 server_port; template class ConcurrentFlags; using UpdateFlags = ConcurrentFlags; diff --git a/tests/integration/test_cluster_discovery/config/config_shard1.xml b/tests/integration/test_cluster_discovery/config/config_shard1.xml new file mode 100644 index 00000000000..06a77a37263 --- /dev/null +++ b/tests/integration/test_cluster_discovery/config/config_shard1.xml @@ -0,0 +1,24 @@ + + 1 + + + + /clickhouse/discovery/test_auto_cluster + 1 + + + + + + + node1 + 9000 + + + node2 + 9000 + + + + + diff --git a/tests/integration/test_cluster_discovery/config/config_shard3.xml b/tests/integration/test_cluster_discovery/config/config_shard3.xml new file mode 100644 index 00000000000..ab66fdc2ab7 --- /dev/null +++ b/tests/integration/test_cluster_discovery/config/config_shard3.xml @@ -0,0 +1,24 @@ + + 1 + + + + /clickhouse/discovery/test_auto_cluster + 3 + + + + + + + node1 + 9000 + + + node2 + 9000 + + + + + diff --git a/tests/integration/test_cluster_discovery/test.py b/tests/integration/test_cluster_discovery/test.py index bdcd749f244..acddd855040 100644 --- a/tests/integration/test_cluster_discovery/test.py +++ b/tests/integration/test_cluster_discovery/test.py @@ -1,14 +1,21 @@ import pytest + +import functools import time from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) +shard_configs = { + i: f'config/config_shard{i}.xml' + for i in [1, 3] +} + nodes = [ cluster.add_instance( f'node{i}', - main_configs=['config/config.xml'], + main_configs=[shard_configs.get(i, 'config/config.xml')], stay_alive=True, with_zookeeper=True ) for i in range(5) @@ -24,24 +31,25 @@ def start_cluster(): cluster.shutdown() -def check_nodes_count_in_cluster(nodes, expected, cluster_name, *, retries=5): +def check_on_cluster(nodes, expected, *, what, cluster_name='test_auto_cluster', msg=None, retries=5): """ - Check nodes count in system.clusters for specified cluster + Select data from `system.clusters` on specified nodes and check the result """ assert 1 <= retries <= 6 for retry in range(1, retries + 1): - nodes_cnt = { - node.name: int(node.query(f"SELECT count() FROM system.clusters WHERE cluster = '{cluster_name}'")) + nodes_res = { + node.name: int(node.query(f"SELECT {what} FROM system.clusters WHERE cluster = '{cluster_name}'")) for node in nodes } - if all(actual == expected for actual in nodes_cnt.values()): + if all(actual == expected for actual in nodes_res.values()): break if retry != retries: time.sleep(2 ** retry) else: - raise Exception(f'Wrong nodes count in cluster: {nodes_cnt}, expected: {expected} (after {retries} retries)') + msg = msg or f"Wrong '{what}' result" + raise Exception(f'{msg}: {nodes_res}, expected: {expected} (after {retries} retries)') def test_cluster_discovery_startup_and_stop(start_cluster): @@ -50,18 +58,24 @@ def test_cluster_discovery_startup_and_stop(start_cluster): then stop/start some nodes and check that it (dis)appeared in cluster. """ - check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes), 'test_auto_cluster') + check_nodes_count = functools.partial(check_on_cluster, what='count()', msg='Wrong nodes count in cluster') + check_shard_num = functools.partial(check_on_cluster, what='count(DISTINCT shard_num)', msg='Wrong shard_num count in cluster') + + total_shards = len(shard_configs) + 1 + check_nodes_count([nodes[0], nodes[2]], len(nodes)) + check_shard_num([nodes[0], nodes[2]], total_shards) nodes[1].stop_clickhouse(kill=True) - check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes) - 1, 'test_auto_cluster') + check_nodes_count([nodes[0], nodes[2]], len(nodes) - 1) + check_shard_num([nodes[0], nodes[2]], total_shards - 1) nodes[3].stop_clickhouse() - check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes) - 2, 'test_auto_cluster') + check_nodes_count([nodes[0], nodes[2]], len(nodes) - 2) nodes[1].start_clickhouse() - check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes) - 1, 'test_auto_cluster') + check_nodes_count([nodes[0], nodes[2]], len(nodes) - 1) nodes[3].start_clickhouse() - check_nodes_count_in_cluster([nodes[0], nodes[2]], len(nodes), 'test_auto_cluster') + check_nodes_count([nodes[0], nodes[2]], len(nodes)) - check_nodes_count_in_cluster([nodes[1], nodes[2]], 2, 'two_shards', retries=1) + check_nodes_count([nodes[1], nodes[2]], 2, cluster_name='two_shards', retries=1) From 4f8a9cc539da36ee60dff218a3f7fd47746e7b46 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 Dec 2021 12:07:30 +0300 Subject: [PATCH 0447/1260] cluster discovery: versioning for format of data stored in zk --- src/Interpreters/ClusterDiscovery.cpp | 18 +++++++++++++++--- src/Interpreters/ClusterDiscovery.h | 3 +++ 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 6f0bf9fe389..f4836a909f8 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -385,9 +385,20 @@ bool ClusterDiscovery::NodeInfo::parse(const String & data, NodeInfo & result) Poco::JSON::Parser parser; auto json = parser.parse(data).extract(); - result.address = json->getValue("address"); - result.secure = json->optValue("secure", false); - result.shard_id = json->optValue("shard_id", 0); + size_t ver = json->optValue("version", data_ver); + if (ver == data_ver) + { + result.address = json->getValue("address"); + result.secure = json->optValue("secure", false); + result.shard_id = json->optValue("shard_id", 0); + } + else + { + LOG_ERROR( + &Poco::Logger::get("ClusterDiscovery"), + "Unsupported version '{}' of data in zk node '{}'", + ver, data.size() < 1024 ? data : "[data too long]"); + } } catch (Poco::Exception & e) { @@ -403,6 +414,7 @@ bool ClusterDiscovery::NodeInfo::parse(const String & data, NodeInfo & result) String ClusterDiscovery::NodeInfo::serialize() const { Poco::JSON::Object json; + json.set("version", data_ver); json.set("address", address); json.set("shard_id", shard_id); diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index 5b0f7211502..439a41389e6 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -38,6 +38,9 @@ public: private: struct NodeInfo { + /// versioning for format of data stored in zk + static constexpr size_t data_ver = 1; + /// host:port String address; /// is secure tcp port user From 72e63b4f12f6b97863d9f517f282b8a4955a2acc Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 Dec 2021 12:20:33 +0300 Subject: [PATCH 0448/1260] restart runMainThread in cluster discovery --- src/Interpreters/ClusterDiscovery.cpp | 25 ++++++++++++++++++------- src/Interpreters/ClusterDiscovery.h | 2 +- 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index f4836a909f8..35d4f0103da 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -323,18 +323,28 @@ void ClusterDiscovery::start() main_thread = ThreadFromGlobalPool([this] { - try + bool finish = false; + while (!finish) { - runMainThread(); - } - catch (...) - { - tryLogCurrentException(log, "Caught exception in cluster discovery runMainThread"); + try + { + finish = runMainThread(); + } + catch (...) + { + /* + * it can be zk error (will take new session) or other retriable error, + * should not stop discovery forever + */ + tryLogCurrentException(log, "Caught exception in cluster discovery runMainThread"); + } } }); } -void ClusterDiscovery::runMainThread() + +/// Returns `true` on gracefull shutdown (no restart required) +bool ClusterDiscovery::runMainThread() { setThreadName("ClusterDiscover"); LOG_DEBUG(log, "Worker thread started"); @@ -362,6 +372,7 @@ void ClusterDiscovery::runMainThread() } } LOG_DEBUG(log, "Worker thread stopped"); + return stop_flag; } void ClusterDiscovery::shutdown() diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index 439a41389e6..f0d0707d0a9 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -94,7 +94,7 @@ private: bool updateCluster(const String & cluster_name); bool updateCluster(ClusterInfo & cluster_info); - void runMainThread(); + bool runMainThread(); void shutdown(); /// cluster name -> cluster info (zk root, set of nodes) From 36cd9ecb148239223b4d655c330e785e952e0137 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 Dec 2021 12:45:32 +0300 Subject: [PATCH 0449/1260] log level debug in cluster discovery --- src/Interpreters/ClusterDiscovery.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 35d4f0103da..8839850b750 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -188,7 +188,7 @@ bool ClusterDiscovery::needUpdate(const Strings & node_uuids, const NodesInfo & return fmt::format("{} node{} [{}{}]", sz, sz != 1 ? "s" : "", fmt::join(diff, ", "), need_crop ? ",..." : ""); }; - LOG_TRACE(log, "Cluster update: added {}, removed {}", + LOG_DEBUG(log, "Cluster update: added {}, removed {}", format_cluster_update(new_names, old_names), format_cluster_update(old_names, new_names)); } @@ -233,7 +233,7 @@ ClusterPtr ClusterDiscovery::makeCluster(const ClusterInfo & cluster_info) /// Returns true on success (or no update required). bool ClusterDiscovery::updateCluster(ClusterInfo & cluster_info) { - LOG_TRACE(log, "Updating cluster '{}'", cluster_info.name); + LOG_DEBUG(log, "Updating cluster '{}'", cluster_info.name); auto zk = context->getZooKeeper(); @@ -251,7 +251,7 @@ bool ClusterDiscovery::updateCluster(ClusterInfo & cluster_info) if (!needUpdate(node_uuids, nodes_info)) { - LOG_TRACE(log, "No update required for cluster '{}'", cluster_info.name); + LOG_DEBUG(log, "No update required for cluster '{}'", cluster_info.name); return true; } @@ -267,12 +267,12 @@ bool ClusterDiscovery::updateCluster(ClusterInfo & cluster_info) if (current_version != start_version) { - LOG_TRACE(log, "Cluster '{}' configuration changed during update", cluster_info.name); + LOG_DEBUG(log, "Cluster '{}' configuration changed during update", cluster_info.name); nodes_info.clear(); return false; } - LOG_TRACE(log, "Updating system.clusters record for '{}' with {} nodes", cluster_info.name, cluster_info.nodes_info.size()); + LOG_DEBUG(log, "Updating system.clusters record for '{}' with {} nodes", cluster_info.name, cluster_info.nodes_info.size()); auto cluster = makeCluster(cluster_info); context->setCluster(cluster_info.name, cluster); @@ -308,7 +308,7 @@ void ClusterDiscovery::start() LOG_DEBUG(log, "No defined clusters for discovery"); return; } - LOG_TRACE(log, "Starting working thread"); + LOG_DEBUG(log, "Starting working thread"); auto zk = context->getZooKeeper(); for (auto & [_, info] : clusters_info) @@ -377,7 +377,7 @@ bool ClusterDiscovery::runMainThread() void ClusterDiscovery::shutdown() { - LOG_TRACE(log, "Shutting down"); + LOG_DEBUG(log, "Shutting down"); stop_flag.exchange(true); if (main_thread.joinable()) From 3d3fae126ea5cfc3f6ff2ed6c1bc6a4f5ee93152 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 Dec 2021 13:35:42 +0300 Subject: [PATCH 0450/1260] check if clusters updated successfully on current iteration of discovery --- src/Interpreters/ClusterDiscovery.cpp | 54 +++++++++++++++++++-------- 1 file changed, 38 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 8839850b750..918f53af305 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -63,20 +64,35 @@ public: cv.notify_one(); } - void unset(const T & key) { setFlag(key, false); } - - void wait(std::chrono::milliseconds timeout) + /// `need_update` expected to be value from `flags` corresponding to some key + void set(std::atomic_bool & need_update) { - std::unique_lock lk(mu); - cv.wait_for(lk, timeout); + any_need_update = true; + need_update = true; + cv.notify_one(); } - const std::unordered_map & get() { return flags; } + /// waits unit at least one flag is set + /// caller should handle all set flags (or set it again manually) + std::unordered_map & wait(std::atomic_bool & finished, std::chrono::milliseconds timeout) + { + std::unique_lock lk(mu); + cv.wait_for(lk, timeout, [this, &finished]() -> bool { return any_need_update || finished; }); + + /// all set flags expected to be handled by caller + any_need_update = false; + return flags; + } + + std::unique_lock getLock() { return std::unique_lock(mu); } + + void notify() { cv.notify_one(); } private: - void setFlag(const T & key, bool value) { + any_need_update = any_need_update || value; + auto it = flags.find(key); if (it == flags.end()) throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown value '{}'", key); @@ -86,7 +102,9 @@ private: std::condition_variable cv; std::mutex mu; + /// flag indicates that update is required std::unordered_map flags; + std::atomic_bool any_need_update = true; }; ClusterDiscovery::ClusterDiscovery( @@ -342,8 +360,7 @@ void ClusterDiscovery::start() }); } - -/// Returns `true` on gracefull shutdown (no restart required) +/// Returns `true` on graceful shutdown (no restart required) bool ClusterDiscovery::runMainThread() { setThreadName("ClusterDiscover"); @@ -353,21 +370,20 @@ bool ClusterDiscovery::runMainThread() while (!stop_flag) { - /// if some cluster update was ended with error on previous iteration, we will retry after timeout - clusters_to_update->wait(5s); - for (const auto & [cluster_name, need_update] : clusters_to_update->get()) + auto & clusters = clusters_to_update->wait(stop_flag, 5s); + for (auto & [cluster_name, need_update] : clusters) { - if (!need_update) + if (!need_update.exchange(false)) continue; if (updateCluster(cluster_name)) { - clusters_to_update->unset(cluster_name); LOG_DEBUG(log, "Cluster '{}' updated successfully", cluster_name); } else { - LOG_DEBUG(log, "Cluster '{}' are not updated, will retry", cluster_name); + clusters_to_update->set(need_update); + LOG_WARNING(log, "Cluster '{}' wasn't updated, will retry", cluster_name); } } } @@ -379,7 +395,13 @@ void ClusterDiscovery::shutdown() { LOG_DEBUG(log, "Shutting down"); - stop_flag.exchange(true); + /// need lock and notify because `clusters_to_update` uses `stop_flag` in stop condition + { + auto lk = clusters_to_update->getLock(); + stop_flag = true; + } + clusters_to_update->notify(); + if (main_thread.joinable()) main_thread.join(); } From be7efb14c67b6cc95deef2f6a167986f1f4307dd Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 Dec 2021 15:51:27 +0300 Subject: [PATCH 0451/1260] force updating cluster discovery periodically --- src/Interpreters/ClusterDiscovery.cpp | 35 +++++++++++++++------------ src/Interpreters/ClusterDiscovery.h | 4 ++- 2 files changed, 23 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 918f53af305..f0490d1b1a4 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -297,17 +297,6 @@ bool ClusterDiscovery::updateCluster(ClusterInfo & cluster_info) return true; } -bool ClusterDiscovery::updateCluster(const String & cluster_name) -{ - auto cluster_info = clusters_info.find(cluster_name); - if (cluster_info == clusters_info.end()) - { - LOG_ERROR(log, "Unknown cluster '{}'", cluster_name); - return false; - } - return updateCluster(cluster_info->second); -} - void ClusterDiscovery::registerInZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & info) { LOG_DEBUG(log, "Registering current node {} in cluster {}", current_node_name, info.name); @@ -368,16 +357,32 @@ bool ClusterDiscovery::runMainThread() using namespace std::chrono_literals; + constexpr auto force_update_interval = 2min; + while (!stop_flag) { auto & clusters = clusters_to_update->wait(stop_flag, 5s); for (auto & [cluster_name, need_update] : clusters) { - if (!need_update.exchange(false)) - continue; - - if (updateCluster(cluster_name)) + auto cluster_info_it = clusters_info.find(cluster_name); + if (cluster_info_it == clusters_info.end()) { + LOG_ERROR(log, "Unknown cluster '{}'", cluster_name); + continue; + } + auto & cluster_info = cluster_info_it->second; + + if (!need_update.exchange(false)) + { + /// force updating periodically + bool force_update = cluster_info.watch.elapsedSeconds() > std::chrono::seconds(force_update_interval).count(); + if (!force_update) + continue; + } + + if (updateCluster(cluster_info)) + { + cluster_info.watch.restart(); LOG_DEBUG(log, "Cluster '{}' updated successfully", cluster_name); } else diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index f0d0707d0a9..dee2f5f9a2a 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -68,6 +68,9 @@ private: const String zk_root; NodesInfo nodes_info; + /// Track last update time + Stopwatch watch; + NodeInfo current_node; explicit ClusterInfo(const String & name_, const String & zk_root_, UInt16 port, bool secure, size_t shard_id) @@ -91,7 +94,6 @@ private: ClusterPtr makeCluster(const ClusterInfo & cluster_info); bool needUpdate(const Strings & node_uuids, const NodesInfo & nodes); - bool updateCluster(const String & cluster_name); bool updateCluster(ClusterInfo & cluster_info); bool runMainThread(); From b90f53ffbae598d043899f88b3e415fa8847c8c1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 Dec 2021 15:31:49 +0300 Subject: [PATCH 0452/1260] Fix unit tests --- .../tests/gtest_transform_query_for_external_database.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index f161400630b..57b9e73bbbd 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -120,7 +120,7 @@ TEST(TransformQueryForExternalDatabase, InWithSingleElement) check(state, 1, "SELECT column FROM test.table WHERE 1 IN (1)", - R"(SELECT "column" FROM "test"."table" WHERE 1)"); + R"(SELECT "column" FROM "test"."table" WHERE 1 = 1)"); check(state, 1, "SELECT column FROM test.table WHERE column IN (1, 2)", R"(SELECT "column" FROM "test"."table" WHERE "column" IN (1, 2))"); @@ -135,7 +135,7 @@ TEST(TransformQueryForExternalDatabase, InWithMultipleColumns) check(state, 1, "SELECT column FROM test.table WHERE (1,1) IN ((1,1))", - R"(SELECT "column" FROM "test"."table" WHERE 1)"); + R"(SELECT "column" FROM "test"."table" WHERE 1 = 1)"); check(state, 1, "SELECT field, value FROM test.table WHERE (field, value) IN (('foo', 'bar'))", R"(SELECT "field", "value" FROM "test"."table" WHERE ("field", "value") IN (('foo', 'bar')))"); From 922dd40c05b8295303a3b7b9d981363018b2f057 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 27 Dec 2021 20:33:25 +0800 Subject: [PATCH 0453/1260] optimization --- src/Common/LRUCache.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 700ef3b0ceb..a5c4990fecd 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -372,7 +372,7 @@ private: } cell.value = mapped; - cell.size = cell.value ? weight_function(*cell.value) : 0; + cell.size = value_weight; current_size += cell.size; return true; From 47f0e3c72f9869958d1061f39be4fd3de478ae2b Mon Sep 17 00:00:00 2001 From: OnePiece <54787696+zhongyuankai@users.noreply.github.com> Date: Mon, 27 Dec 2021 20:46:51 +0800 Subject: [PATCH 0454/1260] Update MergeTreePartsMover.cpp --- src/Storages/MergeTree/MergeTreePartsMover.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 83ffcc41fd8..8c48815f4e2 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -122,6 +122,9 @@ bool MergeTreePartsMover::selectPartsForMove( time_t time_of_move = time(nullptr); auto metadata_snapshot = data->getInMemoryMetadataPtr(); + + if (need_to_move.empty() && !metadata_snapshot->hasAnyMoveTTL()) + return false; for (const auto & part : data_parts) { From 26e61061b0cf125cae246e981e2bf34dfa777da1 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 27 Dec 2021 20:46:51 +0800 Subject: [PATCH 0455/1260] update setImpl() --- src/Common/LRUCache.h | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index a5c4990fecd..652b02cad64 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -343,13 +343,6 @@ private: if (inserted) { - if (!removeOverflow(value_weight)) - { - // cannot find enough space to put in the new value - cells.erase(it); - return false; - } - try { cell.queue_iterator = queue.insert(queue.end(), key); @@ -359,6 +352,14 @@ private: cells.erase(it); throw; } + + if (!removeOverflow()) + { + // overflow is caused by inserting this element. + queue.erase(cell.queue_iterator); + cells.erase(it); + return false; + } } else { From a76e93209d2efe391e995b56ae5f9e8efbf2f171 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 10 Dec 2021 16:23:28 +0300 Subject: [PATCH 0456/1260] Backoff for ClusterDiscovery::runMainThread --- src/Interpreters/ClusterDiscovery.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index f0490d1b1a4..d32017873b9 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -328,8 +328,14 @@ void ClusterDiscovery::start() } } + using namespace std::chrono_literals; + constexpr static std::chrono::milliseconds DEFAULT_BACKOFF_TIMEOUT = 10ms; + + LOG_DEBUG(log, "Starting working thread"); main_thread = ThreadFromGlobalPool([this] { + std::chrono::milliseconds backoff_timeout = DEFAULT_BACKOFF_TIMEOUT; + bool finish = false; while (!finish) { @@ -345,6 +351,8 @@ void ClusterDiscovery::start() */ tryLogCurrentException(log, "Caught exception in cluster discovery runMainThread"); } + std::this_thread::sleep_for(backoff_timeout); + backoff_timeout = std::min(backoff_timeout * 2, std::chrono::milliseconds(3min)); } }); } From 3934e99dcebe397b1e7bb383ef33f40e1788538d Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 13 Dec 2021 14:40:07 +0300 Subject: [PATCH 0457/1260] update ClusterDiscovery::ConcurrentFlags::set --- src/Interpreters/ClusterDiscovery.cpp | 26 +++++++------------------- 1 file changed, 7 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index d32017873b9..69dcc8a79fe 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -60,20 +60,17 @@ public: void set(const T & key) { - setFlag(key, true); - cv.notify_one(); - } - - /// `need_update` expected to be value from `flags` corresponding to some key - void set(std::atomic_bool & need_update) - { + auto it = flags.find(key); + if (it == flags.end()) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown value '{}'", key); + it->second = true; any_need_update = true; - need_update = true; cv.notify_one(); } /// waits unit at least one flag is set /// caller should handle all set flags (or set it again manually) + /// note: keys of returen map should not be changed! std::unordered_map & wait(std::atomic_bool & finished, std::chrono::milliseconds timeout) { std::unique_lock lk(mu); @@ -89,16 +86,6 @@ public: void notify() { cv.notify_one(); } private: - void setFlag(const T & key, bool value) - { - any_need_update = any_need_update || value; - - auto it = flags.find(key); - if (it == flags.end()) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown value '{}'", key); - it->second = value; - } - std::condition_variable cv; std::mutex mu; @@ -395,7 +382,8 @@ bool ClusterDiscovery::runMainThread() } else { - clusters_to_update->set(need_update); + /// no need to trigger convar, will retry after timeout in `wait` + need_update = true; LOG_WARNING(log, "Cluster '{}' wasn't updated, will retry", cluster_name); } } From 5bd620fba19a72016e1090b831cabb7ea1900b7d Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 20 Dec 2021 18:43:11 +0300 Subject: [PATCH 0458/1260] Reset backoff when cluster discovery is up to date --- src/Interpreters/ClusterDiscovery.cpp | 12 +++++++++--- src/Interpreters/ClusterDiscovery.h | 2 +- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 69dcc8a79fe..28f4f720b38 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -328,7 +328,7 @@ void ClusterDiscovery::start() { try { - finish = runMainThread(); + finish = runMainThread([&backoff_timeout] { backoff_timeout = DEFAULT_BACKOFF_TIMEOUT; }); } catch (...) { @@ -345,7 +345,7 @@ void ClusterDiscovery::start() } /// Returns `true` on graceful shutdown (no restart required) -bool ClusterDiscovery::runMainThread() +bool ClusterDiscovery::runMainThread(std::function up_to_date_callback) { setThreadName("ClusterDiscover"); LOG_DEBUG(log, "Worker thread started"); @@ -353,9 +353,9 @@ bool ClusterDiscovery::runMainThread() using namespace std::chrono_literals; constexpr auto force_update_interval = 2min; - while (!stop_flag) { + bool all_up_to_date = true; auto & clusters = clusters_to_update->wait(stop_flag, 5s); for (auto & [cluster_name, need_update] : clusters) { @@ -382,11 +382,17 @@ bool ClusterDiscovery::runMainThread() } else { + all_up_to_date = false; /// no need to trigger convar, will retry after timeout in `wait` need_update = true; LOG_WARNING(log, "Cluster '{}' wasn't updated, will retry", cluster_name); } } + + if (all_up_to_date) + { + up_to_date_callback(); + } } LOG_DEBUG(log, "Worker thread stopped"); return stop_flag; diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index dee2f5f9a2a..16c186494a5 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -96,7 +96,7 @@ private: bool needUpdate(const Strings & node_uuids, const NodesInfo & nodes); bool updateCluster(ClusterInfo & cluster_info); - bool runMainThread(); + bool runMainThread(std::function up_to_date_callback); void shutdown(); /// cluster name -> cluster info (zk root, set of nodes) From 01a9199098289b1b0d9b546417f3b50df157569e Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 20 Dec 2021 18:57:35 +0300 Subject: [PATCH 0459/1260] Move stop_flag to clusters_to_update in ClusterDiscovery --- src/Interpreters/ClusterDiscovery.cpp | 31 ++++++++++++++------------- src/Interpreters/ClusterDiscovery.h | 1 - 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index 28f4f720b38..ed3eca9eb80 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -71,19 +71,24 @@ public: /// waits unit at least one flag is set /// caller should handle all set flags (or set it again manually) /// note: keys of returen map should not be changed! - std::unordered_map & wait(std::atomic_bool & finished, std::chrono::milliseconds timeout) + /// @param finished - output parameter indicates that stop() was called + std::unordered_map & wait(std::chrono::milliseconds timeout, bool & finished) { std::unique_lock lk(mu); - cv.wait_for(lk, timeout, [this, &finished]() -> bool { return any_need_update || finished; }); + cv.wait_for(lk, timeout, [this]() -> bool { return any_need_update || stop_flag; }); + finished = stop_flag; /// all set flags expected to be handled by caller any_need_update = false; return flags; } - std::unique_lock getLock() { return std::unique_lock(mu); } - - void notify() { cv.notify_one(); } + void stop() + { + std::unique_lock lk(mu); + stop_flag = true; + cv.notify_one(); + } private: std::condition_variable cv; @@ -92,6 +97,7 @@ private: /// flag indicates that update is required std::unordered_map flags; std::atomic_bool any_need_update = true; + bool stop_flag = false; }; ClusterDiscovery::ClusterDiscovery( @@ -353,10 +359,11 @@ bool ClusterDiscovery::runMainThread(std::function up_to_date_callback) using namespace std::chrono_literals; constexpr auto force_update_interval = 2min; - while (!stop_flag) + bool finished = false; + while (!finished) { bool all_up_to_date = true; - auto & clusters = clusters_to_update->wait(stop_flag, 5s); + auto & clusters = clusters_to_update->wait(5s, finished); for (auto & [cluster_name, need_update] : clusters) { auto cluster_info_it = clusters_info.find(cluster_name); @@ -395,19 +402,13 @@ bool ClusterDiscovery::runMainThread(std::function up_to_date_callback) } } LOG_DEBUG(log, "Worker thread stopped"); - return stop_flag; + return finished; } void ClusterDiscovery::shutdown() { LOG_DEBUG(log, "Shutting down"); - - /// need lock and notify because `clusters_to_update` uses `stop_flag` in stop condition - { - auto lk = clusters_to_update->getLock(); - stop_flag = true; - } - clusters_to_update->notify(); + clusters_to_update->stop(); if (main_thread.joinable()) main_thread.join(); diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index 16c186494a5..30c70bbb949 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -114,7 +114,6 @@ private: /// It prevents accessing to invalid object after ClusterDiscovery is destroyed. std::shared_ptr clusters_to_update; - std::atomic stop_flag = false; ThreadFromGlobalPool main_thread; Poco::Logger * log; From cd1aa89bb1bc0b799afc1b209cfabbf17e25a9b2 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 27 Dec 2021 21:20:01 +0800 Subject: [PATCH 0460/1260] Document the risk --- docs/en/operations/settings/settings.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 9b4db0e026e..70c351a2484 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3134,6 +3134,12 @@ Possible values: Default value: `0`. +!!! warning "Warning" + Nullable primary key usually indicates bad design. It is forbidden in almost all main stream DBMS. The feature is mainly for [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md) and is not heavily tested. Use with care. + +!!! warning "Warning" + Do not enable this feature in version `<= 21.8`. It's not properly implemented and may lead to server crash. + ## aggregate_functions_null_for_empty {#aggregate_functions_null_for_empty} Enables or disables rewriting all aggregate functions in a query, adding [-OrNull](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-ornull) suffix to them. Enable it for SQL standard compatibility. From 555b20bc2e6f1f67019c5f124c7a1cb228ef6017 Mon Sep 17 00:00:00 2001 From: OnePiece <54787696+zhongyuankai@users.noreply.github.com> Date: Mon, 27 Dec 2021 21:38:51 +0800 Subject: [PATCH 0461/1260] Update MergeTreePartsMover.cpp --- src/Storages/MergeTree/MergeTreePartsMover.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 8c48815f4e2..5a889ea5e8b 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -122,7 +122,7 @@ bool MergeTreePartsMover::selectPartsForMove( time_t time_of_move = time(nullptr); auto metadata_snapshot = data->getInMemoryMetadataPtr(); - + if (need_to_move.empty() && !metadata_snapshot->hasAnyMoveTTL()) return false; From 9e5f59659fce84c4e024d6ae7ee514fd9e937169 Mon Sep 17 00:00:00 2001 From: pdv-ru Date: Mon, 27 Dec 2021 17:22:38 +0300 Subject: [PATCH 0462/1260] Edit and translate to Russian (four letter commands to keeper) --- docs/ru/development/developer-instruction.md | 4 +- docs/ru/operations/clickhouse-keeper.md | 191 +++++++++++++++++++ 2 files changed, 193 insertions(+), 2 deletions(-) diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index 8466c709ad1..eb224adeb0b 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -40,7 +40,7 @@ ClickHouse не работает и не собирается на 32-битны Выполните в терминале: - git clone git@github.com:ClickHouse/ClickHouse.git + git clone --recursive git@github.com:your_github_username/ClickHouse.git cd ClickHouse Замените первое вхождение слова `ClickHouse` в команде для git на имя вашего аккаунта на GitHub. @@ -68,7 +68,7 @@ ClickHouse не работает и не собирается на 32-битны Вы также можете клонировать репозиторий по протоколу https: - git clone https://github.com/ClickHouse/ClickHouse.git + git clone --recursive https://github.com/ClickHouse/ClickHouse.git Этот вариант не подходит для отправки изменений на сервер. Вы можете временно его использовать, а затем добавить ssh ключи и заменить адрес репозитория с помощью команды `git remote`. diff --git a/docs/ru/operations/clickhouse-keeper.md b/docs/ru/operations/clickhouse-keeper.md index 9d6c4799008..b30749a8f76 100644 --- a/docs/ru/operations/clickhouse-keeper.md +++ b/docs/ru/operations/clickhouse-keeper.md @@ -54,6 +54,7 @@ ClickHouse Keeper может использоваться как равноце - `auto_forwarding` — разрешить пересылку запросов на запись от последователей лидеру (по умолчанию: true). - `shutdown_timeout` — время ожидания завершения внутренних подключений и выключения, в миллисекундах (по умолчанию: 5000). - `startup_timeout` — время отключения сервера, если он не подключается к другим участникам кворума, в миллисекундах (по умолчанию: 30000). +- `four_letter_word_white_list` — список разрешенных 4-х буквенных команд (по умолчанию: "conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro"). Конфигурация кворума находится в `.` и содержит описание серверов. @@ -109,6 +110,196 @@ ClickHouse Keeper входит в пакет `clickhouse-server`, просто clickhouse-keeper --config /etc/your_path_to_config/config.xml --daemon ``` +## 4-х буквенные команды + +ClickHouse Keeper также поддерживает 4-х буквенные команды, почти такие же, как у Zookeeper. Каждая команда состоит из 4-х символов, например, `mntr`, `stat` и т. д. Несколько интересных команд: `stat` предоставляет общую информацию о сервере и подключенных клиентах, а `srvr` и `cons` предоставляют расширенные сведения о сервере и подключениях соответственно. + +4-х буквенные команды имеют параметр для настройки разрешенного списка `four_letter_word_white_list`, который имеет значение по умолчанию "conf,cons,crst,envi,ruok,srst,srvr,stat, wchc,wchs,dirs,mntr,isro". + +You can issue the commands to ClickHouse Keeper via telnet or nc, at the client port. +Вы можете отправлять команды в ClickHouse Keeper через telnet или nc через клиентский порт. + +``` +echo mntr | nc localhost 9181 +``` + +Ниже приведен подробный список 4-х буквенных команд: + +- `ruok`: Проверяет, запущен ли сервер без ошибок. Если это так, сервер ответит `imok`. В противном случае он не ответит. Ответ `imok` не обязательно означает, что сервер присоединился к кворуму, просто процесс сервера активен и привязан к указанному клиентскому порту. Используйте команду `stat` для получения подробной информации о состоянии кворума и клиентском подключении. + +``` +imok +``` + +- `mntr`: Выводит список переменных, которые могут быть использованы для мониторинга работоспособности кластера. + +``` +zk_version v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 +zk_avg_latency 0 +zk_max_latency 0 +zk_min_latency 0 +zk_packets_received 68 +zk_packets_sent 68 +zk_num_alive_connections 1 +zk_outstanding_requests 0 +zk_server_state leader +zk_znode_count 4 +zk_watch_count 1 +zk_ephemerals_count 0 +zk_approximate_data_size 723 +zk_open_file_descriptor_count 310 +zk_max_file_descriptor_count 10240 +zk_followers 0 +zk_synced_followers 0 +``` + +- `srvr`: Выводит полную информацию о сервере. + +``` +ClickHouse Keeper version: v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 +Latency min/avg/max: 0/0/0 +Received: 2 +Sent : 2 +Connections: 1 +Outstanding: 0 +Zxid: 34 +Mode: leader +Node count: 4 +``` + +- `stat`: Выводит краткие сведения о сервере и подключенных клиентах. + +``` +ClickHouse Keeper version: v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 +Clients: + 192.168.1.1:52852(recved=0,sent=0) + 192.168.1.1:52042(recved=24,sent=48) +Latency min/avg/max: 0/0/0 +Received: 4 +Sent : 4 +Connections: 1 +Outstanding: 0 +Zxid: 36 +Mode: leader +Node count: 4 +``` + +- `srst`: Сбрасывает статистику сервера. Команда повлияет на результат `srvr`, `mntr` и `stat`. + +``` +Server stats reset. +``` + +- `conf`: Выводит подробную информацию о серверной конфигурации. + +``` +server_id=1 +tcp_port=2181 +four_letter_word_white_list=* +log_storage_path=./coordination/logs +snapshot_storage_path=./coordination/snapshots +max_requests_batch_size=100 +session_timeout_ms=30000 +operation_timeout_ms=10000 +dead_session_check_period_ms=500 +heart_beat_interval_ms=500 +election_timeout_lower_bound_ms=1000 +election_timeout_upper_bound_ms=2000 +reserved_log_items=1000000000000000 +snapshot_distance=10000 +auto_forwarding=true +shutdown_timeout=5000 +startup_timeout=240000 +raft_logs_level=information +snapshots_to_keep=3 +rotate_log_storage_interval=100000 +stale_log_gap=10000 +fresh_log_gap=200 +max_requests_batch_size=100 +quorum_reads=false +force_sync=false +compress_logs=true +compress_snapshots_with_zstd_format=true +configuration_change_tries_count=20 +``` + +- `cons`: Выводит полную информацию о подключении/сеансе для всех клиентов, подключенных к этому серверу. Включает информацию о количестве принятых/отправленных пакетов, идентификаторе сессии, задержках операций, последней выполненной операции и т. д. + +``` + 192.168.1.1:52163(recved=0,sent=0,sid=0xffffffffffffffff,lop=NA,est=1636454787393,to=30000,lzxid=0xffffffffffffffff,lresp=0,llat=0,minlat=0,avglat=0,maxlat=0) + 192.168.1.1:52042(recved=9,sent=18,sid=0x0000000000000001,lop=List,est=1636454739887,to=30000,lcxid=0x0000000000000005,lzxid=0x0000000000000005,lresp=1636454739892,llat=0,minlat=0,avglat=0,maxlat=0) +``` + +- `crst`: Сбрасывает статистику подключений/сессий для всех подключений. + +``` +Connection stats reset. +``` + +- `envi`: Выводит подробную информацию о серверном окружении. + +``` +Environment: +clickhouse.keeper.version=v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 +host.name=ZBMAC-C02D4054M.local +os.name=Darwin +os.arch=x86_64 +os.version=19.6.0 +cpu.count=12 +user.name=root +user.home=/Users/JackyWoo/ +user.dir=/Users/JackyWoo/project/jd/clickhouse/cmake-build-debug/programs/ +user.tmp=/var/folders/b4/smbq5mfj7578f2jzwn602tt40000gn/T/ +``` + + +- `dirs`: Показывает общий размер файлов снепшотов и журналов в байтах + +``` +snapshot_dir_size: 0 +log_dir_size: 3875 +``` + +- `isro`: Проверяет, работает ли сервер в режиме только для чтения. Сервер ответит `ro`, если он находится в режиме только для чтения, или `rw`, если нет. + +``` +rw +``` + +- `wchs`: Показывает краткую информацию о наблюдениях за сервером. + +``` +1 connections watching 1 paths +Total watches:1 +``` + +- `wchc`: Показывает подробную информацию о наблюдениях за сервером в разбивке по сессиям. При этом выводится список сессий (подключений) с соответствующими наблюдениями (путями). Обратите внимание, что в зависимости от количества наблюдений эта операция может быть дорогостоящей (т. е. повлиять на производительность сервера), используйте ее осторожно. + +``` +0x0000000000000001 + /clickhouse/task_queue/ddl +``` + +- `wchp`: Lists detailed information on watches for the server, by path. This outputs a list of paths (znodes) with associated sessions. Note, depending on the number of watches this operation may be expensive (i. e. impact server performance), use it carefully. +Показывает подробную информацию о наблюдениях за сервером по определенному пути. При этом выводится список путей (узлов) с соответствующими сессиями. Обратите внимание, что в зависимости от количества наблюдений эта операция может быть дорогостоящей (т. е. повлиять на производительность сервера), используйте ее осторожно. + +``` +/clickhouse/task_queue/ddl + 0x0000000000000001 +``` + +- `dump`: Выводит список незавершенных сеансов и эфемерные узлы. Это работает только с лидером. + +``` +Sessions dump (2): +0x0000000000000001 +0x0000000000000002 +Sessions with Ephemerals (1): +0x0000000000000001 + /clickhouse/task_queue/ddl +``` + + ## [экспериментально] Переход с ZooKeeper Плавный переход с ZooKeeper на ClickHouse Keeper невозможен, необходимо остановить кластер ZooKeeper, преобразовать данные и запустить ClickHouse Keeper. Утилита `clickhouse-keeper-converter` конвертирует журналы и снэпшоты ZooKeeper в снэпшот ClickHouse Keeper. Работа утилиты проверена только для версий ZooKeeper выше 3.4. Для миграции необходимо выполнить следующие шаги: From 55dfaef4decd8da8b7bf0efc33e90dec82f229f5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 Dec 2021 17:41:37 +0300 Subject: [PATCH 0463/1260] Settings changes as key value --- src/Databases/DatabaseFactory.cpp | 11 +++--- .../ExternalDataSourceConfiguration.cpp | 34 +++++++++++++++++-- .../ExternalDataSourceConfiguration.h | 12 +++++-- src/Storages/MySQL/MySQLHelpers.cpp | 8 +++++ src/Storages/MySQL/MySQLSettings.h | 5 ++- src/Storages/StorageExternalDistributed.cpp | 2 +- src/Storages/StorageMongoDB.cpp | 2 +- src/Storages/StorageMySQL.cpp | 12 ++++--- src/Storages/StorageMySQL.h | 2 +- src/Storages/StoragePostgreSQL.cpp | 2 +- src/TableFunctions/TableFunctionMySQL.cpp | 2 +- src/TableFunctions/TableFunctionRemote.cpp | 2 +- tests/integration/test_storage_mysql/test.py | 2 ++ 13 files changed, 74 insertions(+), 22 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 3f6cb49fda7..5cc334eaad4 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -156,13 +156,15 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String StorageMySQLConfiguration configuration; ASTs & arguments = engine->arguments->children; + MySQLSettings mysql_settings; - if (auto named_collection = getExternalDataSourceConfiguration(arguments, context, true)) + if (auto named_collection = getExternalDataSourceConfiguration(arguments, context, true, true, mysql_settings)) { - auto [common_configuration, storage_specific_args] = named_collection.value(); + auto [common_configuration, storage_specific_args, settings_changes] = named_collection.value(); configuration.set(common_configuration); configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; + mysql_settings.applyChanges(settings_changes); if (!storage_specific_args.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, @@ -200,7 +202,6 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (engine_name == "MySQL") { auto mysql_database_settings = std::make_unique(); - MySQLSettings mysql_settings; auto mysql_pool = createMySQLPoolWithFailover(configuration, mysql_settings); mysql_database_settings->loadFromQueryContext(context); @@ -299,7 +300,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (auto named_collection = getExternalDataSourceConfiguration(engine_args, context, true)) { - auto [common_configuration, storage_specific_args] = named_collection.value(); + auto [common_configuration, storage_specific_args, _] = named_collection.value(); configuration.set(common_configuration); configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; @@ -358,7 +359,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (auto named_collection = getExternalDataSourceConfiguration(engine_args, context, true)) { - auto [common_configuration, storage_specific_args] = named_collection.value(); + auto [common_configuration, storage_specific_args, _] = named_collection.value(); configuration.set(common_configuration); if (!storage_specific_args.empty()) diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 42b3b148551..f96e199ff6e 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -15,6 +15,9 @@ #if USE_RDKAFKA #include #endif +#if USE_MYSQL +#include +#endif namespace DB { @@ -24,6 +27,8 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +IMPLEMENT_SETTINGS_TRAITS(EmptySettingsTraits, EMPTY_SETTINGS) + String ExternalDataSourceConfiguration::toString() const { WriteBufferFromOwnString configuration_info; @@ -59,7 +64,9 @@ void ExternalDataSourceConfiguration::set(const ExternalDataSourceConfiguration } -std::optional getExternalDataSourceConfiguration(const ASTs & args, ContextPtr context, bool is_database_engine, bool throw_on_no_collection) +template +std::optional getExternalDataSourceConfiguration( + const ASTs & args, ContextPtr context, bool is_database_engine, bool throw_on_no_collection, const BaseSettings & storage_settings) { if (args.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); @@ -82,6 +89,15 @@ std::optional getExternalDataSourceConfiguration(const throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", collection->name()); } + SettingsChanges config_settings; + for (const auto & setting : storage_settings.all()) + { + const auto & setting_name = setting.getName(); + auto setting_value = config.getString(collection_prefix + '.' + setting_name, ""); + if (!setting_value.empty()) + config_settings.emplace_back(setting_name, setting_value); + } + configuration.host = config.getString(collection_prefix + ".host", ""); configuration.port = config.getInt(collection_prefix + ".port", 0); configuration.username = config.getString(collection_prefix + ".user", ""); @@ -123,6 +139,7 @@ std::optional getExternalDataSourceConfiguration(const if (arg_value_literal) { auto arg_value = arg_value_literal->value; + if (arg_name == "host") configuration.host = arg_value.safeGet(); else if (arg_name == "port") @@ -139,6 +156,8 @@ std::optional getExternalDataSourceConfiguration(const configuration.schema = arg_value.safeGet(); else if (arg_name == "addresses_expr") configuration.addresses_expr = arg_value.safeGet(); + else if (storage_settings.has(arg_name)) + config_settings.emplace_back(arg_name, arg_value); else non_common_args.emplace_back(std::make_pair(arg_name, arg_value_ast)); } @@ -153,8 +172,7 @@ std::optional getExternalDataSourceConfiguration(const } } - ExternalDataSourceConfig source_config{ .configuration = configuration, .specific_args = non_common_args }; - return source_config; + return ExternalDataSourceInfo{ .configuration = configuration, .specific_args = non_common_args, .settings_changes = config_settings }; } return std::nullopt; } @@ -425,4 +443,14 @@ bool getExternalDataSourceConfiguration(const ASTs & args, BaseSettings & settings, ContextPtr context); #endif + +template +std::optional getExternalDataSourceConfiguration( + const ASTs & args, ContextPtr context, bool is_database_engine, bool throw_on_no_collection, const BaseSettings & storage_settings); + +#if USE_MYSQL +template +std::optional getExternalDataSourceConfiguration( + const ASTs & args, ContextPtr context, bool is_database_engine, bool throw_on_no_collection, const BaseSettings & storage_settings); +#endif } diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index 502f8b800e3..dd83215c79f 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -7,6 +7,11 @@ namespace DB { +#define EMPTY_SETTINGS(M) +DECLARE_SETTINGS_TRAITS(EmptySettingsTraits, EMPTY_SETTINGS) + +struct EmptySettings : public BaseSettings {}; + struct ExternalDataSourceConfiguration { String host; @@ -46,10 +51,11 @@ struct StorageMongoDBConfiguration : ExternalDataSourceConfiguration using StorageSpecificArgs = std::vector>; -struct ExternalDataSourceConfig +struct ExternalDataSourceInfo { ExternalDataSourceConfiguration configuration; StorageSpecificArgs specific_args; + SettingsChanges settings_changes; }; /* If there is a storage engine's configuration specified in the named_collections, @@ -62,7 +68,9 @@ struct ExternalDataSourceConfig * Any key-value engine argument except common (`host`, `port`, `username`, `password`, `database`) * is returned in EngineArgs struct. */ -std::optional getExternalDataSourceConfiguration(const ASTs & args, ContextPtr context, bool is_database_engine = false, bool throw_on_no_collection = true); +template +std::optional getExternalDataSourceConfiguration( + const ASTs & args, ContextPtr context, bool is_database_engine = false, bool throw_on_no_collection = true, const BaseSettings & storage_settings = {}); std::optional getExternalDataSourceConfiguration( const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context); diff --git a/src/Storages/MySQL/MySQLHelpers.cpp b/src/Storages/MySQL/MySQLHelpers.cpp index e7745e6c0bb..edeb4ffca8a 100644 --- a/src/Storages/MySQL/MySQLHelpers.cpp +++ b/src/Storages/MySQL/MySQLHelpers.cpp @@ -8,9 +8,17 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + mysqlxx::PoolWithFailover createMySQLPoolWithFailover(const StorageMySQLConfiguration & configuration, const MySQLSettings & mysql_settings) { + if (!mysql_settings.connection_pool_size) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Connection pool cannot have zero size"); + return mysqlxx::PoolWithFailover( configuration.database, configuration.addresses, configuration.username, configuration.password, MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS, diff --git a/src/Storages/MySQL/MySQLSettings.h b/src/Storages/MySQL/MySQLSettings.h index aa2c2703d6b..be1e09c12e6 100644 --- a/src/Storages/MySQL/MySQLSettings.h +++ b/src/Storages/MySQL/MySQLSettings.h @@ -25,11 +25,14 @@ class ASTStorage; DECLARE_SETTINGS_TRAITS(MySQLSettingsTraits, LIST_OF_MYSQL_SETTINGS) +using MySQLBaseSettings = BaseSettings; + /** Settings for the MySQL family of engines. */ -struct MySQLSettings : public BaseSettings +struct MySQLSettings : public MySQLBaseSettings { void loadFromQuery(ASTStorage & storage_def); }; + } diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index 927c070826b..40a2ad0b85e 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -272,7 +272,7 @@ void registerStorageExternalDistributed(StorageFactory & factory) ExternalDataSourceConfiguration configuration; if (auto named_collection = getExternalDataSourceConfiguration(inner_engine_args, args.getLocalContext())) { - auto [common_configuration, storage_specific_args] = named_collection.value(); + auto [common_configuration, storage_specific_args, _] = named_collection.value(); configuration.set(common_configuration); for (const auto & [name, value] : storage_specific_args) diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 2c1b44d8685..9b25b44c0e7 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -117,7 +117,7 @@ StorageMongoDBConfiguration StorageMongoDB::getConfiguration(ASTs engine_args, C StorageMongoDBConfiguration configuration; if (auto named_collection = getExternalDataSourceConfiguration(engine_args, context)) { - auto [common_configuration, storage_specific_args] = named_collection.value(); + auto [common_configuration, storage_specific_args, _] = named_collection.value(); configuration.set(common_configuration); for (const auto & [arg_name, arg_value] : storage_specific_args) diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 66adf3ae272..83cf2b07b21 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -238,15 +238,17 @@ SinkToStoragePtr StorageMySQL::write(const ASTPtr & /*query*/, const StorageMeta } -StorageMySQLConfiguration StorageMySQL::getConfiguration(ASTs engine_args, ContextPtr context_) +StorageMySQLConfiguration StorageMySQL::getConfiguration(ASTs engine_args, ContextPtr context_, MySQLBaseSettings & storage_settings) { StorageMySQLConfiguration configuration; - if (auto named_collection = getExternalDataSourceConfiguration(engine_args, context_)) + if (auto named_collection = getExternalDataSourceConfiguration( + engine_args, context_, /* is_database_engine */false, /* throw_on_no_collection */true, storage_settings)) { - auto [common_configuration, storage_specific_args] = named_collection.value(); + auto [common_configuration, storage_specific_args, settings_changes] = named_collection.value(); configuration.set(common_configuration); configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; + storage_settings.applyChanges(settings_changes); for (const auto & [arg_name, arg_value] : storage_specific_args) { @@ -298,9 +300,9 @@ void registerStorageMySQL(StorageFactory & factory) { factory.registerStorage("MySQL", [](const StorageFactory::Arguments & args) { - auto configuration = StorageMySQL::getConfiguration(args.engine_args, args.getLocalContext()); - MySQLSettings mysql_settings; /// TODO: move some arguments from the arguments to the SETTINGS. + auto configuration = StorageMySQL::getConfiguration(args.engine_args, args.getLocalContext(), mysql_settings); + if (args.storage_def->settings) mysql_settings.loadFromQuery(*args.storage_def); diff --git a/src/Storages/StorageMySQL.h b/src/Storages/StorageMySQL.h index cc3673e50ca..fe2ee8439bc 100644 --- a/src/Storages/StorageMySQL.h +++ b/src/Storages/StorageMySQL.h @@ -53,7 +53,7 @@ public: SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; - static StorageMySQLConfiguration getConfiguration(ASTs engine_args, ContextPtr context_); + static StorageMySQLConfiguration getConfiguration(ASTs engine_args, ContextPtr context_, MySQLBaseSettings & storage_settings); private: friend class StorageMySQLSink; diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 8327bb92a38..5042f911149 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -390,7 +390,7 @@ StoragePostgreSQLConfiguration StoragePostgreSQL::getConfiguration(ASTs engine_a StoragePostgreSQLConfiguration configuration; if (auto named_collection = getExternalDataSourceConfiguration(engine_args, context)) { - auto [common_configuration, storage_specific_args] = named_collection.value(); + auto [common_configuration, storage_specific_args, _] = named_collection.value(); configuration.set(common_configuration); configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index e959fa754c9..cfed24caef6 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -37,8 +37,8 @@ void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, ContextPtr if (!args_func.arguments) throw Exception("Table function 'mysql' must have arguments.", ErrorCodes::LOGICAL_ERROR); - configuration = StorageMySQL::getConfiguration(args_func.arguments->children, context); MySQLSettings mysql_settings; + configuration = StorageMySQL::getConfiguration(args_func.arguments->children, context, mysql_settings); const auto & settings = context->getSettingsRef(); mysql_settings.connect_timeout = settings.external_storage_connect_timeout_sec; mysql_settings.read_write_timeout = settings.external_storage_rw_timeout_sec; diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index f7af6bee7d9..85857011616 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -60,7 +60,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr * Specific args (remote): sharding_key, or database (in case it is not ASTLiteral). * None of the common arguments is empty at this point, it is checked in getExternalDataSourceConfiguration. */ - auto [common_configuration, storage_specific_args] = named_collection.value(); + auto [common_configuration, storage_specific_args, _] = named_collection.value(); configuration.set(common_configuration); for (const auto & [arg_name, arg_value] : storage_specific_args) diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index c0ba0d8735e..59be9f5f879 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -418,6 +418,8 @@ def test_predefined_connection_configuration(started_cluster): ''') assert (node1.query(f"SELECT count() FROM test_table").rstrip() == '100') + assert 'Connection pool cannot have zero size' in node1.query_and_get_error(f"SELECT count() FROM mysql(mysql1, table='test_table', connection_pool_size=0)").rstrip() + # Regression for (k, v) IN ((k, v)) def test_mysql_in(started_cluster): From 5f82190cd3ad8b5a3d31109c014102cb573181ad Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 27 Dec 2021 15:41:09 +0300 Subject: [PATCH 0464/1260] Log exception on cluster discovery initial update --- src/Interpreters/ClusterDiscovery.cpp | 29 ++++++++++++++++++--------- src/Interpreters/ClusterDiscovery.h | 2 ++ 2 files changed, 22 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/ClusterDiscovery.cpp b/src/Interpreters/ClusterDiscovery.cpp index ed3eca9eb80..8b68ba02504 100644 --- a/src/Interpreters/ClusterDiscovery.cpp +++ b/src/Interpreters/ClusterDiscovery.cpp @@ -301,6 +301,20 @@ void ClusterDiscovery::registerInZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & inf LOG_DEBUG(log, "Current node {} registered in cluster {}", current_node_name, info.name); } +void ClusterDiscovery::initialUpdate() +{ + auto zk = context->getZooKeeper(); + for (auto & [_, info] : clusters_info) + { + registerInZk(zk, info); + if (!updateCluster(info)) + { + LOG_WARNING(log, "Error on initial cluster '{}' update, will retry in background", info.name); + clusters_to_update->set(info.name); + } + } +} + void ClusterDiscovery::start() { if (clusters_info.empty()) @@ -308,17 +322,14 @@ void ClusterDiscovery::start() LOG_DEBUG(log, "No defined clusters for discovery"); return; } - LOG_DEBUG(log, "Starting working thread"); - auto zk = context->getZooKeeper(); - for (auto & [_, info] : clusters_info) + try { - registerInZk(zk, info); - if (!updateCluster(info)) - { - LOG_WARNING(log, "Error on updating cluster '{}', will retry", info.name); - clusters_to_update->set(info.name); - } + initialUpdate(); + } + catch (...) + { + tryLogCurrentException(log, "Caught exception in cluster discovery initialization"); } using namespace std::chrono_literals; diff --git a/src/Interpreters/ClusterDiscovery.h b/src/Interpreters/ClusterDiscovery.h index 30c70bbb949..2098652c069 100644 --- a/src/Interpreters/ClusterDiscovery.h +++ b/src/Interpreters/ClusterDiscovery.h @@ -81,6 +81,8 @@ private: } }; + void initialUpdate(); + void registerInZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & info); Strings getNodeNames(zkutil::ZooKeeperPtr & zk, From 5117f3b2656c56fac35a4f65531a3767886be70d Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Mon, 27 Dec 2021 08:34:58 -0700 Subject: [PATCH 0465/1260] Proofreading edits --- website/blog/en/2021/tests-visualization.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/website/blog/en/2021/tests-visualization.md b/website/blog/en/2021/tests-visualization.md index 51e6c46632c..5fa39630e71 100644 --- a/website/blog/en/2021/tests-visualization.md +++ b/website/blog/en/2021/tests-visualization.md @@ -10,17 +10,17 @@ Test suites and testing infrastructure are one of the main assets of ClickHouse. We also save the results of all test runs into the database in ClickHouse. We started collecting results in June 2020, and we have 1 777 608 240 records so far. Now we run around 5 to 9 million tests every day. -Tests are good (in general). Good test suite allows fast development iterations, stable releases, accept more contributions from the community. We love tests. If there's something strange in ClickHouse, what are we gonna do? Write more tests. +Tests are good (in general). A good test suite allows for fast development iterations, stable releases, and accepts more contributions from the community. We love tests. If there's something strange in ClickHouse, what are we gonna do? Write more tests. Some tests can be flaky. The reasons for flakiness are uncountable - most of them are simple timing issues in the test script itself, but sometimes if a test has failed one of a thousand times it can uncover subtle logic errors in code. -The problem is how to deal with flaky tests. Some people may suggest to automatically mute the "annoying" flaky tests. Or to add automatic retries in case of failure. We believe that this is all wrong. Instead of trying to ignore flaky tests, we do the opposite: we do maximum efforts to make the tests even more flaky! +The problem is how to deal with flaky tests. Some people suggest automatically muting the "annoying" flaky tests. Or adding automatic retries in case of failure. We believe that this is all wrong. Instead of trying to ignore flaky tests, we do the opposite: we put maximum effort into making the tests even more flaky! Our recipes for flaky tests: — never mute or restart them; if the test failed once, always look and investigate the cause; — randomize the environment for every test run so the test will have more possible reasons to fail; -— if new tests have added, run them 100 times and if at least once they fail, do not merge the pull request; -— if new tests have added, use them as a corpus for fuzzing - it will uncover corner cases even if author did not write tests for them; +— if new tests are added, run them 100 times and if at least one fails, do not merge the pull request; +— if new tests are added, use them as a corpus for fuzzing - it will uncover corner cases even if author did not write tests for them; — [randomize thread scheduling](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/ThreadFuzzer.h) and add random sleeps and switching between CPU cores at random places and before and after mutex locks/unlocks; — run everything in parallel on slow machines; @@ -28,7 +28,7 @@ Key point: to prevent flaky tests, we make our tests as flaky as possible. ## Nice Way To Visualize Flaky Tests -There is a test suite named "[functional stateless tests](https://github.com/ClickHouse/ClickHouse/tree/master/tests/queries/0_stateless)", it has 3772 tests. Let's draw test results as a picture. For every day since 2020-06-13 (561 days) and every test (3772 tests) make a picture of size 561x3772 where every pixel is green if all test runs finished successfully in master branch during this day (for all commits and all combinations: release, debug+assertions, ASan, MSan, TSan, UBSan); red if at least one run has failed; transparent if the test did not exist that day. +There is a test suite named "[functional stateless tests](https://github.com/ClickHouse/ClickHouse/tree/master/tests/queries/0_stateless)" that has 3772 tests. For every day since 2020-06-13 (561 days) and every test (3772 tests), I drew a picture of size 561x3772 where a pixel is green if all test runs finished successfully in the master branch during this day (for all commits and all combinations: release, debug+assertions, ASan, MSan, TSan, UBSan), and a pixel is red if at least one run failed. The pixel will be transparent if the test did not exist that day. This visualization is a toy that I've made for fun: @@ -42,4 +42,4 @@ If you see red dots in a horizontal line - it is a flaky test. If you see red do The velocity of adding new tests is represented by how tall and narrow the Christmas tree is. When we add a large number of tests, the tree grows with almost vertical slope. -The image is prepared by [HTML page](https://github.com/ClickHouse/ClickHouse/pull/33185) with some JavaScript that is querying ClickHouse database directly and writing to a canvas. It took around ten seconds to build this picture. I also prepared [interactive version](https://blog-images.clickhouse.com/en/2021/tests-visualization/demo.html) with already saved data where you can play and find your favorite tests. +The image is prepared by [HTML page](https://github.com/ClickHouse/ClickHouse/pull/33185) with some JavaScript that is querying a ClickHouse database directly and writing to a canvas. It took around ten seconds to build this picture. I also prepared an [interactive version](https://blog-images.clickhouse.com/en/2021/tests-visualization/demo.html) with already-saved data where you can play and find your favorite tests. From d41ba4dc0fbac47e33f7c68dda2abe1c628d19d6 Mon Sep 17 00:00:00 2001 From: freedomDR <1640145602@qq.com> Date: Mon, 27 Dec 2021 15:39:35 +0000 Subject: [PATCH 0466/1260] improve projection describe --- docs/en/sql-reference/statements/alter/projection.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index 96cd8f5d607..c7ebc83c496 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -9,11 +9,12 @@ The following operations with [projections](../../../engines/table-engines/merge - `ALTER TABLE [db].name ADD PROJECTION name ( SELECT [GROUP BY] [ORDER BY] )` - Adds projection description to tables metadata. -- `ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. +- `ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). - `ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). -- `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. +- `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). + The commands `ADD`, `DROP` and `CLEAR` are lightweight in a sense that they only change metadata or remove files. From 0465aef15d9435bc3c4e2cf3ca07cee1fb791f99 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Mon, 27 Dec 2021 19:27:06 +0300 Subject: [PATCH 0467/1260] Fixes by code review responces --- src/Storages/IStorage.h | 3 - src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 4 +- src/Storages/MergeTree/MergeTreeData.cpp | 7 +- src/Storages/MergeTree/MergeTreeData.h | 5 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 69 ++++++++----------- src/Storages/StorageReplicatedMergeTree.h | 12 ++-- 8 files changed, 50 insertions(+), 53 deletions(-) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index be60c31ef96..f1fd25d6d12 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -114,9 +114,6 @@ public: /// The name of the table. StorageID getStorageID() const; - /// Unique ID, synchronized between replicas for replicated storage - virtual String getTableUniqID() const { return ""; } - /// Returns true if the storage receives data from a remote server or servers. virtual bool isRemote() const { return false; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 4c9373dd15c..24c0f25f3ec 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1623,7 +1623,7 @@ String IMergeTreeDataPart::getUniqueId() const } -UInt32 IMergeTreeDataPart::getRefCount() const +UInt32 IMergeTreeDataPart::getNumberOfRefereneces() const { return volume->getDisk()->getRefCount(fs::path(getFullRelativePath()) / "checksums.txt"); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 9a466f853b9..5b9861a3ecf 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -338,7 +338,7 @@ public: /// Changes only relative_dir_name, you need to update other metadata (name, is_temp) explicitly virtual void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) const; - /// Cleanup after change part + /// Cleanup shared locks made with old name after part renaming virtual void cleanupOldName(const String & old_part_name) const; /// Makes clone of a part in detached/ directory via hard links @@ -413,7 +413,7 @@ public: /// Return hardlink count for part. /// Required for keep data on remote FS when part has shadow copies. - UInt32 getRefCount() const; + UInt32 getNumberOfRefereneces() const; protected: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2d9cf5b478e..bc4a3659ae0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2572,6 +2572,7 @@ bool MergeTreeData::renameTempPartAndReplace( out_covered_parts->emplace_back(std::move(covered_part)); } + /// Cleanup shared locks made with old name part->cleanupOldName(old_part_name); return true; @@ -5216,7 +5217,9 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( localBackup(disk, src_part_path, backup_part_path); - freezeMetaData(disk, part, backup_part_path); + // Store metadata for replicated table. + // Do nothing for non-replocated. + createAndStoreFreezeMetadata(disk, part, backup_part_path); disk->removeFileIfExists(fs::path(backup_part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); @@ -5235,7 +5238,7 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( return result; } -void MergeTreeData::freezeMetaData(DiskPtr, DataPartPtr, String) const +void MergeTreeData::createAndStoreFreezeMetadata(DiskPtr, DataPartPtr, String) const { } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 04eb3e1a6c0..e5241771f91 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -884,8 +884,9 @@ public: /// Remove local files and remote files if needed virtual bool removeDetachedPart(DiskPtr disk, const String & path, const String & part_name, bool is_freezed); - /// Store some metadata for freezed part if needed - virtual void freezeMetaData(DiskPtr disk, DataPartPtr part, String backup_part_path) const; + /// Store metadata for replicated tables + /// Do nothing for non-replicated tables + virtual void createAndStoreFreezeMetadata(DiskPtr disk, DataPartPtr part, String backup_part_path) const; /// Parts that currently submerging (merging to bigger parts) or emerging /// (to be appeared after merging finished). These two variables have to be used diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 7c1f21409cd..674f4bded86 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -511,6 +511,7 @@ void ReplicatedMergeTreeSink::commitPart( waitForQuorum(zookeeper, part->name, quorum_info.status_path, quorum_info.is_active_node_value); } + /// Cleanup shared locks made with old name part->cleanupOldName(old_part_name); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a163bbe385c..930e9f6835f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -497,6 +497,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( createNewZooKeeperNodes(); syncPinnedPartUUIDs(); + + createTableSharedID(); } @@ -7087,12 +7089,18 @@ std::unique_ptr StorageReplicatedMergeTree::getDefaultSetting } -String StorageReplicatedMergeTree::getTableUniqID() const +String StorageReplicatedMergeTree::getTableSharedID() const { - if (table_global_id == UUIDHelpers::Nil) + return toString(table_shared_id); +} + + +void StorageReplicatedMergeTree::createTableSharedID() +{ + if (table_shared_id == UUIDHelpers::Nil) { zkutil::ZooKeeperPtr zookeeper = getZooKeeper(); - String zookeeper_table_id_path = fs::path(zookeeper_path) / "table_id"; + String zookeeper_table_id_path = fs::path(zookeeper_path) / "table_shared_id"; String id; if (!zookeeper->tryGet(zookeeper_table_id_path, id)) { @@ -7116,10 +7124,8 @@ String StorageReplicatedMergeTree::getTableUniqID() const } } - table_global_id = parseFromString(id); + table_shared_id = parseFromString(id); } - - return toString(table_global_id); } @@ -7138,7 +7144,7 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part) String id = part.getUniqueId(); boost::replace_all(id, "/", "_"); - Strings zc_zookeeper_paths = getZeroCopyPartPath(*getDefaultSettings(), disk->getType(), getTableUniqID(), + Strings zc_zookeeper_paths = getZeroCopyPartPath(*getSettings(), disk->getType(), getTableSharedID(), part.name, zookeeper_path); for (const auto & zc_zookeeper_path : zc_zookeeper_paths) { @@ -7168,11 +7174,11 @@ bool StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & par if (!zookeeper) return true; - auto ref_count = part.getRefCount(); + auto ref_count = part.getNumberOfRefereneces(); if (ref_count > 0) /// Keep part shard info for frozen backups return false; - return unlockSharedDataById(part.getUniqueId(), getTableUniqID(), name, replica_name, disk, zookeeper, *getDefaultSettings(), log, + return unlockSharedDataById(part.getUniqueId(), getTableSharedID(), name, replica_name, disk, zookeeper, *getSettings(), log, zookeeper_path); } @@ -7264,7 +7270,7 @@ String StorageReplicatedMergeTree::getSharedDataReplica( if (!zookeeper) return best_replica; - Strings zc_zookeeper_paths = getZeroCopyPartPath(*getDefaultSettings(), disk_type, getTableUniqID(), part.name, + Strings zc_zookeeper_paths = getZeroCopyPartPath(*getSettings(), disk_type, getTableSharedID(), part.name, zookeeper_path); std::set replicas; @@ -7598,6 +7604,9 @@ void StorageReplicatedMergeTree::createZeroCopyLockNode(const zkutil::ZooKeeperP } +namespace +{ + struct FreezeMetaData { public: @@ -7607,7 +7616,7 @@ public: is_remote = storage.isRemote(); replica_name = storage.getReplicaName(); zookeeper_name = storage.getZooKeeperName(); - table_uuid = storage.getTableUniqID(); + table_shared_id = storage.getTableSharedID(); } void save(DiskPtr disk, const String & path) const @@ -7624,7 +7633,7 @@ public: buffer->write("\n", 1); writeString(zookeeper_name, *buffer); buffer->write("\n", 1); - writeString(table_uuid, *buffer); + writeString(table_shared_id, *buffer); buffer->write("\n", 1); } @@ -7649,7 +7658,7 @@ public: DB::assertChar('\n', *buffer); readString(zookeeper_name, *buffer); DB::assertChar('\n', *buffer); - readString(table_uuid, *buffer); + readString(table_shared_id, *buffer); DB::assertChar('\n', *buffer); return true; } @@ -7671,9 +7680,10 @@ public: bool is_remote; String replica_name; String zookeeper_name; - String table_uuid; + String table_shared_id; }; +} bool StorageReplicatedMergeTree::removeDetachedPart(DiskPtr disk, const String & path, const String & part_name, bool is_freezed) { @@ -7685,14 +7695,14 @@ bool StorageReplicatedMergeTree::removeDetachedPart(DiskPtr disk, const String & if (meta.load(disk, path)) { FreezeMetaData::clean(disk, path); - return removeSharedDetachedPart(disk, path, part_name, meta.table_uuid, meta.zookeeper_name, meta.replica_name, ""); + return removeSharedDetachedPart(disk, path, part_name, meta.table_shared_id, meta.zookeeper_name, meta.replica_name, ""); } } else { - String table_uuid = getTableUniqID(); + String table_id = getTableSharedID(); - return removeSharedDetachedPart(disk, path, part_name, table_uuid, zookeeper_name, replica_name, zookeeper_path); + return removeSharedDetachedPart(disk, path, part_name, table_id, zookeeper_name, replica_name, zookeeper_path); } } @@ -7703,30 +7713,11 @@ bool StorageReplicatedMergeTree::removeDetachedPart(DiskPtr disk, const String & bool StorageReplicatedMergeTree::removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid, - const String & detached_zookeeper_name, const String & detached_replica_name, const String & detached_zookeeper_path) + const String &, const String & detached_replica_name, const String & detached_zookeeper_path) { bool keep_shared = false; - static constexpr auto default_zookeeper_name = "default"; - zkutil::ZooKeeperPtr zookeeper; - if (detached_zookeeper_name == default_zookeeper_name) - { - zookeeper = getContext()->getZooKeeper(); - } - else - { - try - { - zookeeper = getContext()->getAuxiliaryZooKeeper(detached_zookeeper_name); - } - catch (const Exception & e) - { - if (e.code() != ErrorCodes::BAD_ARGUMENTS) - throw; - /// No more stored non-default zookeeper - zookeeper = nullptr; - } - } + zkutil::ZooKeeperPtr zookeeper = getZooKeeper(); if (zookeeper) { @@ -7752,7 +7743,7 @@ bool StorageReplicatedMergeTree::removeSharedDetachedPart(DiskPtr disk, const St } -void StorageReplicatedMergeTree::freezeMetaData(DiskPtr disk, DataPartPtr, String backup_part_path) const +void StorageReplicatedMergeTree::createAndStoreFreezeMetadata(DiskPtr disk, DataPartPtr, String backup_part_path) const { if (disk->supportZeroCopyReplication()) { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index f2d7b44cbcb..e931efee735 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -274,9 +274,11 @@ public: bool createEmptyPartInsteadOfLost(zkutil::ZooKeeperPtr zookeeper, const String & lost_part_name); + // Return default or custom zookeeper name for table String getZooKeeperName() const { return zookeeper_name; } - virtual String getTableUniqID() const override; + // Return table id, common for different replicas + String getTableSharedID() const; private: std::atomic_bool are_restoring_replica {false}; @@ -745,7 +747,10 @@ private: bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid, const String & zookeeper_name, const String & replica_name, const String & zookeeper_path); - void freezeMetaData(DiskPtr disk, DataPartPtr part, String backup_part_path) const override; + void createAndStoreFreezeMetadata(DiskPtr disk, DataPartPtr part, String backup_part_path) const override; + + // Create table id if needed + void createTableSharedID(); protected: /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. @@ -765,8 +770,7 @@ protected: bool allow_renaming_); /// Global ID, synced via ZooKeeper between replicas - /// mutable because can getted from ZooKeeper when required - mutable UUID table_global_id; + UUID table_shared_id; }; String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info); From fc9f900078a8d1ed7fedb8aedd09ade2ba8ec970 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Dec 2021 19:30:12 +0300 Subject: [PATCH 0468/1260] Fix JSON --- website/benchmark/hardware/results/xeon_gold_6266.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/benchmark/hardware/results/xeon_gold_6266.json b/website/benchmark/hardware/results/xeon_gold_6266.json index 3c8ea24e838..0e68466a633 100644 --- a/website/benchmark/hardware/results/xeon_gold_6266.json +++ b/website/benchmark/hardware/results/xeon_gold_6266.json @@ -1,6 +1,6 @@ [ { - "system": "Huawei Cloud c6.xlarge.4, 4vCPUs, 16 GiB" + "system": "Huawei Cloud c6.xlarge.4, 4vCPUs, 16 GiB", "system_full": "Huawei Cloud c6.xlarge.4, Xeon Gold 6266C, 3GHz, 4vCPU, 16GiB RAM, vda1 40GB", "cpu_vendor": "Intel", "cpu_model": "Xeon Gold 6266C", From 343e138f1930a02ec603f8a89aef052d8086cea8 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 27 Dec 2021 19:34:28 +0300 Subject: [PATCH 0469/1260] Update tests-visualization.md --- website/blog/en/2021/tests-visualization.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/blog/en/2021/tests-visualization.md b/website/blog/en/2021/tests-visualization.md index 5fa39630e71..259cb4d8e34 100644 --- a/website/blog/en/2021/tests-visualization.md +++ b/website/blog/en/2021/tests-visualization.md @@ -10,7 +10,7 @@ Test suites and testing infrastructure are one of the main assets of ClickHouse. We also save the results of all test runs into the database in ClickHouse. We started collecting results in June 2020, and we have 1 777 608 240 records so far. Now we run around 5 to 9 million tests every day. -Tests are good (in general). A good test suite allows for fast development iterations, stable releases, and accepts more contributions from the community. We love tests. If there's something strange in ClickHouse, what are we gonna do? Write more tests. +Tests are good (in general). A good test suite allows for fast development iterations, stable releases, and to accept more contributions from the community. We love tests. If there's something strange in ClickHouse, what are we gonna do? Write more tests. Some tests can be flaky. The reasons for flakiness are uncountable - most of them are simple timing issues in the test script itself, but sometimes if a test has failed one of a thousand times it can uncover subtle logic errors in code. From 2f9d5af14e63395d2eae2f7d793d9210d6396cab Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 27 Dec 2021 19:54:14 +0300 Subject: [PATCH 0470/1260] Fix clang tidy --- programs/server/Server.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 43d2b64c4f2..c509b9ff8f7 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -569,8 +569,10 @@ if (ThreadFuzzer::instance().isEffective()) [&]() -> std::vector { std::vector metrics; + metrics.reserve(servers_to_start_before_tables.size()); for (const auto & server : servers_to_start_before_tables) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); + std::lock_guard lock(servers_lock); for (const auto & server : servers) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); From 38a40f27fbf8a1f404296027aa108098c13a1f52 Mon Sep 17 00:00:00 2001 From: Peignon Melvyn Date: Mon, 27 Dec 2021 18:12:19 +0100 Subject: [PATCH 0471/1260] Remove a code I removed a sentence that was in a code section, it didn't feel that it was intended initially. --- docs/en/sql-reference/aggregate-functions/reference/uniq.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/uniq.md b/docs/en/sql-reference/aggregate-functions/reference/uniq.md index 598af24c0de..33bfe72548b 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/uniq.md +++ b/docs/en/sql-reference/aggregate-functions/reference/uniq.md @@ -24,9 +24,7 @@ Function: - Calculates a hash for all parameters in the aggregate, then uses it in calculations. -- Uses an adaptive sampling algorithm. For the calculation state, the function uses a sample of element hash values up to 65536. - - This algorithm is very accurate and very efficient on the CPU. When the query contains several of these functions, using `uniq` is almost as fast as using other aggregate functions. +- Uses an adaptive sampling algorithm. For the calculation state, the function uses a sample of element hash values up to 65536. This algorithm is very accurate and very efficient on the CPU. When the query contains several of these functions, using `uniq` is almost as fast as using other aggregate functions. - Provides the result deterministically (it does not depend on the query processing order). From 8dc204c280451bc69b9cb0be5e2813e6c2a9c104 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Mon, 27 Dec 2021 21:23:09 +0300 Subject: [PATCH 0472/1260] fix diff and translate --- .../sql-reference/statements/select/join.md | 672 +++++++++--------- .../sql-reference/statements/select/join.md | 49 +- 2 files changed, 380 insertions(+), 341 deletions(-) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index 228b50524ae..a5b14ccbcfa 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -1,337 +1,335 @@ ---- -toc_title: JOIN ---- - -# JOIN Clause {#select-join} - -Join produces a new table by combining columns from one or multiple tables by using values common to each. It is a common operation in databases with SQL support, which corresponds to [relational algebra](https://en.wikipedia.org/wiki/Relational_algebra#Joins_and_join-like_operators) join. The special case of one table join is often referred to as “self-join”. - -**Syntax** - -``` sql -SELECT -FROM -[GLOBAL] [INNER|LEFT|RIGHT|FULL|CROSS] [OUTER|SEMI|ANTI|ANY|ASOF] JOIN -(ON )|(USING ) ... -``` - -Expressions from `ON` clause and columns from `USING` clause are called “join keys”. Unless otherwise stated, join produces a [Cartesian product](https://en.wikipedia.org/wiki/Cartesian_product) from rows with matching “join keys”, which might produce results with much more rows than the source tables. - -## Supported Types of JOIN {#select-join-types} - -All standard [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) types are supported: - -- `INNER JOIN`, only matching rows are returned. -- `LEFT OUTER JOIN`, non-matching rows from left table are returned in addition to matching rows. -- `RIGHT OUTER JOIN`, non-matching rows from right table are returned in addition to matching rows. -- `FULL OUTER JOIN`, non-matching rows from both tables are returned in addition to matching rows. -- `CROSS JOIN`, produces cartesian product of whole tables, “join keys” are **not** specified. - -`JOIN` without specified type implies `INNER`. Keyword `OUTER` can be safely omitted. Alternative syntax for `CROSS JOIN` is specifying multiple tables in [FROM clause](../../../sql-reference/statements/select/from.md) separated by commas. - -Additional join types available in ClickHouse: - -- `LEFT SEMI JOIN` and `RIGHT SEMI JOIN`, a whitelist on “join keys”, without producing a cartesian product. -- `LEFT ANTI JOIN` and `RIGHT ANTI JOIN`, a blacklist on “join keys”, without producing a cartesian product. -- `LEFT ANY JOIN`, `RIGHT ANY JOIN` and `INNER ANY JOIN`, partially (for opposite side of `LEFT` and `RIGHT`) or completely (for `INNER` and `FULL`) disables the cartesian product for standard `JOIN` types. -- `ASOF JOIN` and `LEFT ASOF JOIN`, joining sequences with a non-exact match. `ASOF JOIN` usage is described below. - -!!! note "Note" - When [join_algorithm](../../../operations/settings/settings.md#settings-join_algorithm) is set to `partial_merge`, `RIGHT JOIN` and `FULL JOIN` are supported only with `ALL` strictness (`SEMI`, `ANTI`, `ANY`, and `ASOF` are not supported). - -## Settings {#join-settings} - -The default join type can be overridden using [join_default_strictness](../../../operations/settings/settings.md#settings-join_default_strictness) setting. - -The behavior of ClickHouse server for `ANY JOIN` operations depends on the [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys) setting. - -**See also** - -- [join_algorithm](../../../operations/settings/settings.md#settings-join_algorithm) -- [join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) -- [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) -- [partial_merge_join_optimizations](../../../operations/settings/settings.md#partial_merge_join_optimizations) -- [partial_merge_join_rows_in_right_blocks](../../../operations/settings/settings.md#partial_merge_join_rows_in_right_blocks) -- [join_on_disk_max_files_to_merge](../../../operations/settings/settings.md#join_on_disk_max_files_to_merge) -- [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys) - -## ON Section Conditions {on-section-conditions} - -An `ON` section can contain several conditions combined using the `AND` and `OR` operators. Conditions specifying join keys must refer both left and right tables and must use the equality operator. Other conditions may use other logical operators but they must refer either the left or the right table of a query. - -Rows are joined if the whole complex condition is met. If the conditions are not met, still rows may be included in the result depending on the `JOIN` type. Note that if the same conditions are placed in a `WHERE` section and they are not met, then rows are always filtered out from the result. - -The `OR` operator inside the `ON` clause works using the hash join algorithm — for each `OR` argument with join keys for `JOIN`, a separate hash table is created, so memory consumption and query execution time grow linearly with an increase in the number of expressions `OR` of the `ON` clause. - -!!! note "Note" - If a condition refers columns from different tables, then only the equality operator (`=`) is supported so far. - -**Example** - -Consider `table_1` and `table_2`: - -``` -┌─Id─┬─name─┐ ┌─Id─┬─text───────────┬─scores─┐ -│ 1 │ A │ │ 1 │ Text A │ 10 │ -│ 2 │ B │ │ 1 │ Another text A │ 12 │ -│ 3 │ C │ │ 2 │ Text B │ 15 │ -└────┴──────┘ └────┴────────────────┴────────┘ -``` - -Query with one join key condition and an additional condition for `table_2`: - -``` sql -SELECT name, text FROM table_1 LEFT OUTER JOIN table_2 - ON table_1.Id = table_2.Id AND startsWith(table_2.text, 'Text'); -``` - -Note that the result contains the row with the name `C` and the empty text column. It is included into the result because an `OUTER` type of a join is used. - -``` -┌─name─┬─text───┐ -│ A │ Text A │ -│ B │ Text B │ -│ C │ │ -└──────┴────────┘ -``` - -Query with `INNER` type of a join and multiple conditions: - -``` sql -SELECT name, text, scores FROM table_1 INNER JOIN table_2 - ON table_1.Id = table_2.Id AND table_2.scores > 10 AND startsWith(table_2.text, 'Text'); -``` - -Result: - -``` -┌─name─┬─text───┬─scores─┐ -│ B │ Text B │ 15 │ -└──────┴────────┴────────┘ -``` - -Query with `INNER` type of a join and condition with `OR`: - -``` sql -CREATE TABLE t1 (`a` Int64, `b` Int64) ENGINE = MergeTree() ORDER BY a; - -CREATE TABLE t2 (`key` Int32, `val` Int64) ENGINE = MergeTree() ORDER BY key; - -INSERT INTO t1 SELECT number as a, -a as b from numbers(5); - -INSERT INTO t2 SELECT if(number % 2 == 0, toInt64(number), -number) as key, number as val from numbers(5); - -SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key; -``` - -Result: - -``` -┌─a─┬──b─┬─val─┐ -│ 0 │ 0 │ 0 │ -│ 1 │ -1 │ 1 │ -│ 2 │ -2 │ 2 │ -│ 3 │ -3 │ 3 │ -│ 4 │ -4 │ 4 │ -└───┴────┴─────┘ -``` - -Query with `INNER` type of a join and conditions with `OR` and `AND`: - -``` -SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key AND t2.val > 3; -``` - -Result: - -``` -┌─a─┬──b─┬─val─┐ -│ 0 │ 0 │ 0 │ -│ 2 │ -2 │ 2 │ -│ 4 │ -4 │ 4 │ -└───┴────┴─────┘ -``` - -## ASOF JOIN Usage {#asof-join-usage} - -`ASOF JOIN` is useful when you need to join records that have no exact match. - -Algorithm requires the special column in tables. This column: - -- Must contain an ordered sequence. -- Can be one of the following types: [Int, UInt](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md), [Date](../../../sql-reference/data-types/date.md), [DateTime](../../../sql-reference/data-types/datetime.md), [Decimal](../../../sql-reference/data-types/decimal.md). -- Can’t be the only column in the `JOIN` clause. - -Syntax `ASOF JOIN ... ON`: - -``` sql -SELECT expressions_list -FROM table_1 -ASOF LEFT JOIN table_2 -ON equi_cond AND closest_match_cond -``` - -You can use any number of equality conditions and exactly one closest match condition. For example, `SELECT count() FROM table_1 ASOF LEFT JOIN table_2 ON table_1.a == table_2.b AND table_2.t <= table_1.t`. - -Conditions supported for the closest match: `>`, `>=`, `<`, `<=`. - -Syntax `ASOF JOIN ... USING`: - -``` sql -SELECT expressions_list -FROM table_1 -ASOF JOIN table_2 -USING (equi_column1, ... equi_columnN, asof_column) -``` - -`ASOF JOIN` uses `equi_columnX` for joining on equality and `asof_column` for joining on the closest match with the `table_1.asof_column >= table_2.asof_column` condition. The `asof_column` column is always the last one in the `USING` clause. - -For example, consider the following tables: - - table_1 table_2 - event | ev_time | user_id event | ev_time | user_id - ----------|---------|---------- ----------|---------|---------- - ... ... - event_1_1 | 12:00 | 42 event_2_1 | 11:59 | 42 - ... event_2_2 | 12:30 | 42 - event_1_2 | 13:00 | 42 event_2_3 | 13:00 | 42 - ... ... - -`ASOF JOIN` can take the timestamp of a user event from `table_1` and find an event in `table_2` where the timestamp is closest to the timestamp of the event from `table_1` corresponding to the closest match condition. Equal timestamp values are the closest if available. Here, the `user_id` column can be used for joining on equality and the `ev_time` column can be used for joining on the closest match. In our example, `event_1_1` can be joined with `event_2_1` and `event_1_2` can be joined with `event_2_3`, but `event_2_2` can’t be joined. - -!!! note "Note" - `ASOF` join is **not** supported in the [Join](../../../engines/table-engines/special/join.md) table engine. - -## Distributed JOIN {#global-join} - -There are two ways to execute join involving distributed tables: - -- When using a normal `JOIN`, the query is sent to remote servers. Subqueries are run on each of them in order to make the right table, and the join is performed with this table. In other words, the right table is formed on each server separately. -- When using `GLOBAL ... JOIN`, first the requestor server runs a subquery to calculate the right table. This temporary table is passed to each remote server, and queries are run on them using the temporary data that was transmitted. - -Be careful when using `GLOBAL`. For more information, see the [Distributed subqueries](../../../sql-reference/operators/in.md#select-distributed-subqueries) section. - -## Implicit Type Conversion {#implicit-type-conversion} - -`INNER JOIN`, `LEFT JOIN`, `RIGHT JOIN`, and `FULL JOIN` queries support the implicit type conversion for "join keys". However the query can not be executed, if join keys from the left and the right tables cannot be converted to a single type (for example, there is no data type that can hold all values from both `UInt64` and `Int64`, or `String` and `Int32`). - -**Example** - -Consider the table `t_1`: -```text -┌─a─┬─b─┬─toTypeName(a)─┬─toTypeName(b)─┐ -│ 1 │ 1 │ UInt16 │ UInt8 │ -│ 2 │ 2 │ UInt16 │ UInt8 │ -└───┴───┴───────────────┴───────────────┘ -``` -and the table `t_2`: -```text -┌──a─┬────b─┬─toTypeName(a)─┬─toTypeName(b)───┐ -│ -1 │ 1 │ Int16 │ Nullable(Int64) │ -│ 1 │ -1 │ Int16 │ Nullable(Int64) │ -│ 1 │ 1 │ Int16 │ Nullable(Int64) │ -└────┴──────┴───────────────┴─────────────────┘ -``` - -The query -```sql -SELECT a, b, toTypeName(a), toTypeName(b) FROM t_1 FULL JOIN t_2 USING (a, b); -``` -returns the set: -```text -┌──a─┬────b─┬─toTypeName(a)─┬─toTypeName(b)───┐ -│ 1 │ 1 │ Int32 │ Nullable(Int64) │ -│ 2 │ 2 │ Int32 │ Nullable(Int64) │ -│ -1 │ 1 │ Int32 │ Nullable(Int64) │ -│ 1 │ -1 │ Int32 │ Nullable(Int64) │ -└────┴──────┴───────────────┴─────────────────┘ -``` - -## Usage Recommendations {#usage-recommendations} - -### Processing of Empty or NULL Cells {#processing-of-empty-or-null-cells} - -While joining tables, the empty cells may appear. The setting [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) define how ClickHouse fills these cells. - -If the `JOIN` keys are [Nullable](../../../sql-reference/data-types/nullable.md) fields, the rows where at least one of the keys has the value [NULL](../../../sql-reference/syntax.md#null-literal) are not joined. - -### Syntax {#syntax} - -The columns specified in `USING` must have the same names in both subqueries, and the other columns must be named differently. You can use aliases to change the names of columns in subqueries. - -The `USING` clause specifies one or more columns to join, which establishes the equality of these columns. The list of columns is set without brackets. More complex join conditions are not supported. - -### Syntax Limitations {#syntax-limitations} - -For multiple `JOIN` clauses in a single `SELECT` query: - -- Taking all the columns via `*` is available only if tables are joined, not subqueries. -- The `PREWHERE` clause is not available. - -For `ON`, `WHERE`, and `GROUP BY` clauses: - -- Arbitrary expressions cannot be used in `ON`, `WHERE`, and `GROUP BY` clauses, but you can define an expression in a `SELECT` clause and then use it in these clauses via an alias. - -### Performance {#performance} - -When running a `JOIN`, there is no optimization of the order of execution in relation to other stages of the query. The join (a search in the right table) is run before filtering in `WHERE` and before aggregation. - -Each time a query is run with the same `JOIN`, the subquery is run again because the result is not cached. To avoid this, use the special [Join](../../../engines/table-engines/special/join.md) table engine, which is a prepared array for joining that is always in RAM. - -In some cases, it is more efficient to use [IN](../../../sql-reference/operators/in.md) instead of `JOIN`. - -If you need a `JOIN` for joining with dimension tables (these are relatively small tables that contain dimension properties, such as names for advertising campaigns), a `JOIN` might not be very convenient due to the fact that the right table is re-accessed for every query. For such cases, there is an “external dictionaries” feature that you should use instead of `JOIN`. For more information, see the [External dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) section. - -### Memory Limitations {#memory-limitations} - -By default, ClickHouse uses the [hash join](https://en.wikipedia.org/wiki/Hash_join) algorithm. ClickHouse takes the right_table and creates a hash table for it in RAM. If `join_algorithm = 'auto'` is enabled, then after some threshold of memory consumption, ClickHouse falls back to [merge](https://en.wikipedia.org/wiki/Sort-merge_join) join algorithm. For `JOIN` algorithms description see the [join_algorithm](../../../operations/settings/settings.md#settings-join_algorithm) setting. - -If you need to restrict `JOIN` operation memory consumption use the following settings: - -- [max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) — Limits number of rows in the hash table. -- [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) — Limits size of the hash table. - -When any of these limits is reached, ClickHouse acts as the [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) setting instructs. - -## Examples {#examples} - -Example: - -``` sql -SELECT - CounterID, - hits, - visits -FROM -( - SELECT - CounterID, - count() AS hits - FROM test.hits - GROUP BY CounterID -) ANY LEFT JOIN -( - SELECT - CounterID, - sum(Sign) AS visits - FROM test.visits - GROUP BY CounterID -) USING CounterID -ORDER BY hits DESC -LIMIT 10 -``` - -``` text -┌─CounterID─┬───hits─┬─visits─┐ -│ 1143050 │ 523264 │ 13665 │ -│ 731962 │ 475698 │ 102716 │ -│ 722545 │ 337212 │ 108187 │ -│ 722889 │ 252197 │ 10547 │ -│ 2237260 │ 196036 │ 9522 │ -│ 23057320 │ 147211 │ 7689 │ -│ 722818 │ 90109 │ 17847 │ -│ 48221 │ 85379 │ 4652 │ -│ 19762435 │ 77807 │ 7026 │ -│ 722884 │ 77492 │ 11056 │ -└───────────┴────────┴────────┘ -``` +--- +toc_title: JOIN +--- + +# JOIN Clause {#select-join} + +Join produces a new table by combining columns from one or multiple tables by using values common to each. It is a common operation in databases with SQL support, which corresponds to [relational algebra](https://en.wikipedia.org/wiki/Relational_algebra#Joins_and_join-like_operators) join. The special case of one table join is often referred to as “self-join”. + +**Syntax** + +``` sql +SELECT +FROM +[GLOBAL] [INNER|LEFT|RIGHT|FULL|CROSS] [OUTER|SEMI|ANTI|ANY|ASOF] JOIN +(ON )|(USING ) ... +``` + +Expressions from `ON` clause and columns from `USING` clause are called “join keys”. Unless otherwise stated, join produces a [Cartesian product](https://en.wikipedia.org/wiki/Cartesian_product) from rows with matching “join keys”, which might produce results with much more rows than the source tables. + +## Supported Types of JOIN {#select-join-types} + +All standard [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) types are supported: + +- `INNER JOIN`, only matching rows are returned. +- `LEFT OUTER JOIN`, non-matching rows from left table are returned in addition to matching rows. +- `RIGHT OUTER JOIN`, non-matching rows from right table are returned in addition to matching rows. +- `FULL OUTER JOIN`, non-matching rows from both tables are returned in addition to matching rows. +- `CROSS JOIN`, produces cartesian product of whole tables, “join keys” are **not** specified. + +`JOIN` without specified type implies `INNER`. Keyword `OUTER` can be safely omitted. Alternative syntax for `CROSS JOIN` is specifying multiple tables in [FROM clause](../../../sql-reference/statements/select/from.md) separated by commas. + +Additional join types available in ClickHouse: + +- `LEFT SEMI JOIN` and `RIGHT SEMI JOIN`, a whitelist on “join keys”, without producing a cartesian product. +- `LEFT ANTI JOIN` and `RIGHT ANTI JOIN`, a blacklist on “join keys”, without producing a cartesian product. +- `LEFT ANY JOIN`, `RIGHT ANY JOIN` and `INNER ANY JOIN`, partially (for opposite side of `LEFT` and `RIGHT`) or completely (for `INNER` and `FULL`) disables the cartesian product for standard `JOIN` types. +- `ASOF JOIN` and `LEFT ASOF JOIN`, joining sequences with a non-exact match. `ASOF JOIN` usage is described below. + +!!! note "Note" + When [join_algorithm](../../../operations/settings/settings.md#settings-join_algorithm) is set to `partial_merge`, `RIGHT JOIN` and `FULL JOIN` are supported only with `ALL` strictness (`SEMI`, `ANTI`, `ANY`, and `ASOF` are not supported). + +## Settings {#join-settings} + +The default join type can be overridden using [join_default_strictness](../../../operations/settings/settings.md#settings-join_default_strictness) setting. + +The behavior of ClickHouse server for `ANY JOIN` operations depends on the [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys) setting. + +**See also** + +- [join_algorithm](../../../operations/settings/settings.md#settings-join_algorithm) +- [join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) +- [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) +- [partial_merge_join_optimizations](../../../operations/settings/settings.md#partial_merge_join_optimizations) +- [partial_merge_join_rows_in_right_blocks](../../../operations/settings/settings.md#partial_merge_join_rows_in_right_blocks) +- [join_on_disk_max_files_to_merge](../../../operations/settings/settings.md#join_on_disk_max_files_to_merge) +- [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys) + +## ON Section Conditions {#on-section-conditions} + +An `ON` section can contain several conditions combined using the `AND` and `OR` operators. Conditions specifying join keys must refer both left and right tables and must use the equality operator. Other conditions may use other logical operators but they must refer either the left or the right table of a query. + +Rows are joined if the whole complex condition is met. If the conditions are not met, still rows may be included in the result depending on the `JOIN` type. Note that if the same conditions are placed in a `WHERE` section and they are not met, then rows are always filtered out from the result. + +The `OR` operator inside the `ON` clause works using the hash join algorithm — for each `OR` argument with join keys for `JOIN`, a separate hash table is created, so memory consumption and query execution time grow linearly with an increase in the number of expressions `OR` of the `ON` clause. + +!!! note "Note" + If a condition refers columns from different tables, then only the equality operator (`=`) is supported so far. + +**Example** + +Consider `table_1` and `table_2`: + +``` +┌─Id─┬─name─┐ ┌─Id─┬─text───────────┬─scores─┐ +│ 1 │ A │ │ 1 │ Text A │ 10 │ +│ 2 │ B │ │ 1 │ Another text A │ 12 │ +│ 3 │ C │ │ 2 │ Text B │ 15 │ +└────┴──────┘ └────┴────────────────┴────────┘ +``` + +Query with one join key condition and an additional condition for `table_2`: + +``` sql +SELECT name, text FROM table_1 LEFT OUTER JOIN table_2 + ON table_1.Id = table_2.Id AND startsWith(table_2.text, 'Text'); +``` + +Note that the result contains the row with the name `C` and the empty text column. It is included into the result because an `OUTER` type of a join is used. + +``` +┌─name─┬─text───┐ +│ A │ Text A │ +│ B │ Text B │ +│ C │ │ +└──────┴────────┘ +``` + +Query with `INNER` type of a join and multiple conditions: + +``` sql +SELECT name, text, scores FROM table_1 INNER JOIN table_2 + ON table_1.Id = table_2.Id AND table_2.scores > 10 AND startsWith(table_2.text, 'Text'); +``` + +Result: + +``` +┌─name─┬─text───┬─scores─┐ +│ B │ Text B │ 15 │ +└──────┴────────┴────────┘ +``` +Query with `INNER` type of a join and condition with `OR`: + +``` sql +CREATE TABLE t1 (`a` Int64, `b` Int64) ENGINE = MergeTree() ORDER BY a; + +CREATE TABLE t2 (`key` Int32, `val` Int64) ENGINE = MergeTree() ORDER BY key; + +INSERT INTO t1 SELECT number as a, -a as b from numbers(5); + +INSERT INTO t2 SELECT if(number % 2 == 0, toInt64(number), -number) as key, number as val from numbers(5); + +SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key; +``` + +Result: + +``` +┌─a─┬──b─┬─val─┐ +│ 0 │ 0 │ 0 │ +│ 1 │ -1 │ 1 │ +│ 2 │ -2 │ 2 │ +│ 3 │ -3 │ 3 │ +│ 4 │ -4 │ 4 │ +└───┴────┴─────┘ +``` + +Query with `INNER` type of a join and conditions with `OR` and `AND`: + +``` +SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key AND t2.val > 3; +``` + +Result: + +``` +┌─a─┬──b─┬─val─┐ +│ 0 │ 0 │ 0 │ +│ 2 │ -2 │ 2 │ +│ 4 │ -4 │ 4 │ +└───┴────┴─────┘ +``` +## ASOF JOIN Usage {#asof-join-usage} + +`ASOF JOIN` is useful when you need to join records that have no exact match. + +Algorithm requires the special column in tables. This column: + +- Must contain an ordered sequence. +- Can be one of the following types: [Int, UInt](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md), [Date](../../../sql-reference/data-types/date.md), [DateTime](../../../sql-reference/data-types/datetime.md), [Decimal](../../../sql-reference/data-types/decimal.md). +- Can’t be the only column in the `JOIN` clause. + +Syntax `ASOF JOIN ... ON`: + +``` sql +SELECT expressions_list +FROM table_1 +ASOF LEFT JOIN table_2 +ON equi_cond AND closest_match_cond +``` + +You can use any number of equality conditions and exactly one closest match condition. For example, `SELECT count() FROM table_1 ASOF LEFT JOIN table_2 ON table_1.a == table_2.b AND table_2.t <= table_1.t`. + +Conditions supported for the closest match: `>`, `>=`, `<`, `<=`. + +Syntax `ASOF JOIN ... USING`: + +``` sql +SELECT expressions_list +FROM table_1 +ASOF JOIN table_2 +USING (equi_column1, ... equi_columnN, asof_column) +``` + +`ASOF JOIN` uses `equi_columnX` for joining on equality and `asof_column` for joining on the closest match with the `table_1.asof_column >= table_2.asof_column` condition. The `asof_column` column is always the last one in the `USING` clause. + +For example, consider the following tables: + + table_1 table_2 + event | ev_time | user_id event | ev_time | user_id + ----------|---------|---------- ----------|---------|---------- + ... ... + event_1_1 | 12:00 | 42 event_2_1 | 11:59 | 42 + ... event_2_2 | 12:30 | 42 + event_1_2 | 13:00 | 42 event_2_3 | 13:00 | 42 + ... ... + +`ASOF JOIN` can take the timestamp of a user event from `table_1` and find an event in `table_2` where the timestamp is closest to the timestamp of the event from `table_1` corresponding to the closest match condition. Equal timestamp values are the closest if available. Here, the `user_id` column can be used for joining on equality and the `ev_time` column can be used for joining on the closest match. In our example, `event_1_1` can be joined with `event_2_1` and `event_1_2` can be joined with `event_2_3`, but `event_2_2` can’t be joined. + +!!! note "Note" + `ASOF` join is **not** supported in the [Join](../../../engines/table-engines/special/join.md) table engine. + +## Distributed JOIN {#global-join} + +There are two ways to execute join involving distributed tables: + +- When using a normal `JOIN`, the query is sent to remote servers. Subqueries are run on each of them in order to make the right table, and the join is performed with this table. In other words, the right table is formed on each server separately. +- When using `GLOBAL ... JOIN`, first the requestor server runs a subquery to calculate the right table. This temporary table is passed to each remote server, and queries are run on them using the temporary data that was transmitted. + +Be careful when using `GLOBAL`. For more information, see the [Distributed subqueries](../../../sql-reference/operators/in.md#select-distributed-subqueries) section. + +## Implicit Type Conversion {#implicit-type-conversion} + +`INNER JOIN`, `LEFT JOIN`, `RIGHT JOIN`, and `FULL JOIN` queries support the implicit type conversion for "join keys". However the query can not be executed, if join keys from the left and the right tables cannot be converted to a single type (for example, there is no data type that can hold all values from both `UInt64` and `Int64`, or `String` and `Int32`). + +**Example** + +Consider the table `t_1`: +```text +┌─a─┬─b─┬─toTypeName(a)─┬─toTypeName(b)─┐ +│ 1 │ 1 │ UInt16 │ UInt8 │ +│ 2 │ 2 │ UInt16 │ UInt8 │ +└───┴───┴───────────────┴───────────────┘ +``` +and the table `t_2`: +```text +┌──a─┬────b─┬─toTypeName(a)─┬─toTypeName(b)───┐ +│ -1 │ 1 │ Int16 │ Nullable(Int64) │ +│ 1 │ -1 │ Int16 │ Nullable(Int64) │ +│ 1 │ 1 │ Int16 │ Nullable(Int64) │ +└────┴──────┴───────────────┴─────────────────┘ +``` + +The query +```sql +SELECT a, b, toTypeName(a), toTypeName(b) FROM t_1 FULL JOIN t_2 USING (a, b); +``` +returns the set: +```text +┌──a─┬────b─┬─toTypeName(a)─┬─toTypeName(b)───┐ +│ 1 │ 1 │ Int32 │ Nullable(Int64) │ +│ 2 │ 2 │ Int32 │ Nullable(Int64) │ +│ -1 │ 1 │ Int32 │ Nullable(Int64) │ +│ 1 │ -1 │ Int32 │ Nullable(Int64) │ +└────┴──────┴───────────────┴─────────────────┘ +``` + +## Usage Recommendations {#usage-recommendations} + +### Processing of Empty or NULL Cells {#processing-of-empty-or-null-cells} + +While joining tables, the empty cells may appear. The setting [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) define how ClickHouse fills these cells. + +If the `JOIN` keys are [Nullable](../../../sql-reference/data-types/nullable.md) fields, the rows where at least one of the keys has the value [NULL](../../../sql-reference/syntax.md#null-literal) are not joined. + +### Syntax {#syntax} + +The columns specified in `USING` must have the same names in both subqueries, and the other columns must be named differently. You can use aliases to change the names of columns in subqueries. + +The `USING` clause specifies one or more columns to join, which establishes the equality of these columns. The list of columns is set without brackets. More complex join conditions are not supported. + +### Syntax Limitations {#syntax-limitations} + +For multiple `JOIN` clauses in a single `SELECT` query: + +- Taking all the columns via `*` is available only if tables are joined, not subqueries. +- The `PREWHERE` clause is not available. + +For `ON`, `WHERE`, and `GROUP BY` clauses: + +- Arbitrary expressions cannot be used in `ON`, `WHERE`, and `GROUP BY` clauses, but you can define an expression in a `SELECT` clause and then use it in these clauses via an alias. + +### Performance {#performance} + +When running a `JOIN`, there is no optimization of the order of execution in relation to other stages of the query. The join (a search in the right table) is run before filtering in `WHERE` and before aggregation. + +Each time a query is run with the same `JOIN`, the subquery is run again because the result is not cached. To avoid this, use the special [Join](../../../engines/table-engines/special/join.md) table engine, which is a prepared array for joining that is always in RAM. + +In some cases, it is more efficient to use [IN](../../../sql-reference/operators/in.md) instead of `JOIN`. + +If you need a `JOIN` for joining with dimension tables (these are relatively small tables that contain dimension properties, such as names for advertising campaigns), a `JOIN` might not be very convenient due to the fact that the right table is re-accessed for every query. For such cases, there is an “external dictionaries” feature that you should use instead of `JOIN`. For more information, see the [External dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) section. + +### Memory Limitations {#memory-limitations} + +By default, ClickHouse uses the [hash join](https://en.wikipedia.org/wiki/Hash_join) algorithm. ClickHouse takes the right_table and creates a hash table for it in RAM. If `join_algorithm = 'auto'` is enabled, then after some threshold of memory consumption, ClickHouse falls back to [merge](https://en.wikipedia.org/wiki/Sort-merge_join) join algorithm. For `JOIN` algorithms description see the [join_algorithm](../../../operations/settings/settings.md#settings-join_algorithm) setting. + +If you need to restrict `JOIN` operation memory consumption use the following settings: + +- [max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) — Limits number of rows in the hash table. +- [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) — Limits size of the hash table. + +When any of these limits is reached, ClickHouse acts as the [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) setting instructs. + +## Examples {#examples} + +Example: + +``` sql +SELECT + CounterID, + hits, + visits +FROM +( + SELECT + CounterID, + count() AS hits + FROM test.hits + GROUP BY CounterID +) ANY LEFT JOIN +( + SELECT + CounterID, + sum(Sign) AS visits + FROM test.visits + GROUP BY CounterID +) USING CounterID +ORDER BY hits DESC +LIMIT 10 +``` + +``` text +┌─CounterID─┬───hits─┬─visits─┐ +│ 1143050 │ 523264 │ 13665 │ +│ 731962 │ 475698 │ 102716 │ +│ 722545 │ 337212 │ 108187 │ +│ 722889 │ 252197 │ 10547 │ +│ 2237260 │ 196036 │ 9522 │ +│ 23057320 │ 147211 │ 7689 │ +│ 722818 │ 90109 │ 17847 │ +│ 48221 │ 85379 │ 4652 │ +│ 19762435 │ 77807 │ 7026 │ +│ 722884 │ 77492 │ 11056 │ +└───────────┴────────┴────────┘ +``` diff --git a/docs/ru/sql-reference/statements/select/join.md b/docs/ru/sql-reference/statements/select/join.md index 9f6d38a024f..b1e739e6c77 100644 --- a/docs/ru/sql-reference/statements/select/join.md +++ b/docs/ru/sql-reference/statements/select/join.md @@ -55,13 +55,13 @@ FROM - [join_on_disk_max_files_to_merge](../../../operations/settings/settings.md#join_on_disk_max_files_to_merge) - [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys) -## Условия в секции ON {on-section-conditions} +## Условия в секции ON {#on-section-conditions} + +Секция `ON` может содержать несколько условий, связанных операторами `AND` и `OR`. Условия, задающие ключи соединения, должны содержать столбцы левой и правой таблицы и должны использовать оператор равенства. Прочие условия могут использовать другие логические операторы, но в отдельном условии могут использоваться столбцы либо только левой, либо только правой таблицы. -Секция `ON` может содержать несколько условий, связанных оператором `AND`. Условия, задающие ключи соединения, должны содержать столбцы левой и правой таблицы и должны использовать оператор равенства. Прочие условия могут использовать другие логические операторы, но в отдельном условии могут использоваться столбцы либо только левой, либо только правой таблицы. Строки объединяются только тогда, когда всё составное условие выполнено. Если оно не выполнено, то строки могут попасть в результат в зависимости от типа `JOIN`. Обратите внимание, что если то же самое условие поместить в секцию `WHERE`, то строки, для которых оно не выполняется, никогда не попаду в результат. -!!! note "Примечание" - Оператор `OR` внутри секции `ON` пока не поддерживается. +Оператор `OR` внутри секции `ON` работает, используя алгоритм хэш-соединения — на каждый агрумент `OR` с ключами соединений для `JOIN` создается отдельная хэш-таблица, поэтому потребление памяти и время выполнения запроса растет линейно при увеличении количества выражений `OR` секции `ON`. !!! note "Примечание" Если в условии использованы столбцы из разных таблиц, то пока поддерживается только оператор равенства (`=`). @@ -110,6 +110,47 @@ SELECT name, text, scores FROM table_1 INNER JOIN table_2 └──────┴────────┴────────┘ ``` +Запрос с типом соединения `INNER` и условием с оператором `OR`: + +``` sql +CREATE TABLE t1 (`a` Int64, `b` Int64) ENGINE = MergeTree() ORDER BY a; + +CREATE TABLE t2 (`key` Int32, `val` Int64) ENGINE = MergeTree() ORDER BY key; + +INSERT INTO t1 SELECT number as a, -a as b from numbers(5); + +INSERT INTO t2 SELECT if(number % 2 == 0, toInt64(number), -number) as key, number as val from numbers(5); + +SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key; +``` + +Результат: + +``` +┌─a─┬──b─┬─val─┐ +│ 0 │ 0 │ 0 │ +│ 1 │ -1 │ 1 │ +│ 2 │ -2 │ 2 │ +│ 3 │ -3 │ 3 │ +│ 4 │ -4 │ 4 │ +└───┴────┴─────┘ +``` + +Запрос с типом соединения `INNER` и условиями с операторами `OR` и `AND`: + +``` +SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key AND t2.val > 3; +``` + +Результат: + +``` +┌─a─┬──b─┬─val─┐ +│ 0 │ 0 │ 0 │ +│ 2 │ -2 │ 2 │ +│ 4 │ -4 │ 4 │ +└───┴────┴─────┘ +``` ## Использование ASOF JOIN {#asof-join-usage} `ASOF JOIN` применим в том случае, когда необходимо объединять записи, которые не имеют точного совпадения. From c7b758c88d833c18fa6b87cf260deb40958758e2 Mon Sep 17 00:00:00 2001 From: Peignon Melvyn Date: Mon, 27 Dec 2021 19:55:23 +0100 Subject: [PATCH 0473/1260] Removed an extra code block Removed an extra code block that should not be here. --- docs/en/sql-reference/data-types/aggregatefunction.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/en/sql-reference/data-types/aggregatefunction.md b/docs/en/sql-reference/data-types/aggregatefunction.md index 81945eeece6..e483a20eed9 100644 --- a/docs/en/sql-reference/data-types/aggregatefunction.md +++ b/docs/en/sql-reference/data-types/aggregatefunction.md @@ -11,9 +11,7 @@ Aggregate functions can have an implementation-defined intermediate state that c **Parameters** -- Name of the aggregate function. - - If the function is parametric, specify its parameters too. +- Name of the aggregate function. If the function is parametric, specify its parameters too. - Types of the aggregate function arguments. From 1f6ca619b780eb19020c07ff94997c9dfffcdfc0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 Dec 2021 22:42:56 +0300 Subject: [PATCH 0474/1260] Allow some killing --- .../Formats/Impl/ArrowBlockInputFormat.cpp | 12 +++++++++++- src/Processors/Formats/Impl/ArrowBlockInputFormat.h | 7 +++++++ src/Processors/Formats/Impl/ArrowBufferedStreams.cpp | 4 ++-- src/Processors/Formats/Impl/ArrowBufferedStreams.h | 2 +- src/Processors/Formats/Impl/ORCBlockInputFormat.cpp | 9 ++++++++- src/Processors/Formats/Impl/ORCBlockInputFormat.h | 7 +++++++ .../Formats/Impl/ParquetBlockInputFormat.cpp | 9 ++++++++- .../Formats/Impl/ParquetBlockInputFormat.h | 7 +++++++ src/Storages/HDFS/StorageHDFS.cpp | 6 ++++++ src/Storages/HDFS/StorageHDFS.h | 2 ++ src/Storages/StorageS3.cpp | 7 +++++++ src/Storages/StorageS3.h | 2 ++ src/Storages/StorageURL.cpp | 6 ++++++ 13 files changed, 74 insertions(+), 6 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index b1f9eaa59a1..1f6b530d72f 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -37,6 +37,9 @@ Chunk ArrowBlockInputFormat::generate() if (!stream_reader) prepareReader(); + if (is_stopped) + return {}; + batch_result = stream_reader->Next(); if (batch_result.ok() && !(*batch_result)) return res; @@ -46,6 +49,9 @@ Chunk ArrowBlockInputFormat::generate() if (!file_reader) prepareReader(); + if (is_stopped) + return {}; + if (record_batch_current >= record_batch_total) return res; @@ -94,7 +100,11 @@ void ArrowBlockInputFormat::prepareReader() } else { - auto file_reader_status = arrow::ipc::RecordBatchFileReader::Open(asArrowFile(*in, format_settings)); + auto arrow_file = asArrowFile(*in, format_settings, is_stopped); + if (is_stopped) + return; + + auto file_reader_status = arrow::ipc::RecordBatchFileReader::Open(std::move(arrow_file)); if (!file_reader_status.ok()) throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while opening a table: {}", file_reader_status.status().ToString()); diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index 44e18e3f852..bb8a000477c 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -27,6 +27,11 @@ public: private: Chunk generate() override; + void onCancel() override + { + is_stopped = 1; + } + // Whether to use ArrowStream format bool stream; // This field is only used for ArrowStream format @@ -42,6 +47,8 @@ private: const FormatSettings format_settings; void prepareReader(); + + std::atomic is_stopped{0}; }; } diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp index 148faabf352..86d278397c2 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp @@ -140,7 +140,7 @@ arrow::Status ArrowInputStreamFromReadBuffer::Close() return arrow::Status(); } -std::shared_ptr asArrowFile(ReadBuffer & in, const FormatSettings & settings) +std::shared_ptr asArrowFile(ReadBuffer & in, const FormatSettings & settings, std::atomic & is_cancelled) { if (auto * fd_in = dynamic_cast(&in)) { @@ -160,7 +160,7 @@ std::shared_ptr asArrowFile(ReadBuffer & in, const std::string file_data; { WriteBufferFromString file_buffer(file_data); - copyData(in, file_buffer); + copyData(in, file_buffer, is_cancelled); } return std::make_shared(arrow::Buffer::FromString(std::move(file_data))); diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.h b/src/Processors/Formats/Impl/ArrowBufferedStreams.h index 29c869e4152..d649c52557f 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.h +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.h @@ -86,7 +86,7 @@ private: ARROW_DISALLOW_COPY_AND_ASSIGN(ArrowInputStreamFromReadBuffer); }; -std::shared_ptr asArrowFile(ReadBuffer & in, const FormatSettings & settings); +std::shared_ptr asArrowFile(ReadBuffer & in, const FormatSettings & settings, std::atomic & is_cancelled); } diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index bac7f25a8e9..87eec459aa3 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -30,6 +30,9 @@ Chunk ORCBlockInputFormat::generate() if (!file_reader) prepareReader(); + if (is_stopped) + return {}; + std::shared_ptr batch_reader; auto result = file_reader->NextStripeReader(format_settings.orc.row_batch_size, include_indices); if (!result.ok()) @@ -86,7 +89,11 @@ static size_t countIndicesForType(std::shared_ptr type) void ORCBlockInputFormat::prepareReader() { - auto result = arrow::adapters::orc::ORCFileReader::Open(asArrowFile(*in, format_settings), arrow::default_memory_pool()); + auto arrow_file = asArrowFile(*in, format_settings, is_stopped); + if (is_stopped) + return; + + auto result = arrow::adapters::orc::ORCFileReader::Open(std::move(arrow_file), arrow::default_memory_pool()); if (!result.ok()) throw Exception(result.status().ToString(), ErrorCodes::BAD_ARGUMENTS); file_reader = std::move(result).ValueOrDie(); diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h index 639aaee73bb..c7dc1c4a710 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -29,6 +29,11 @@ public: protected: Chunk generate() override; + void onCancel() override + { + is_stopped = 1; + } + private: // TODO: check that this class implements every part of its parent @@ -45,6 +50,8 @@ private: const FormatSettings format_settings; void prepareReader(); + + std::atomic is_stopped{0}; }; } diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 901531d81cf..0b6cd006300 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -44,6 +44,9 @@ Chunk ParquetBlockInputFormat::generate() if (!file_reader) prepareReader(); + if (is_stopped) + return {}; + if (row_group_current >= row_group_total) return res; @@ -93,7 +96,11 @@ static size_t countIndicesForType(std::shared_ptr type) void ParquetBlockInputFormat::prepareReader() { - THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(asArrowFile(*in, format_settings), arrow::default_memory_pool(), &file_reader)); + auto arrow_file = asArrowFile(*in, format_settings, is_stopped); + if (is_stopped) + return; + + THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(std::move(arrow_file), arrow::default_memory_pool(), &file_reader)); row_group_total = file_reader->num_row_groups(); row_group_current = 0; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index 472aec66da3..3e04c523442 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -28,6 +28,11 @@ private: void prepareReader(); + void onCancel() override + { + is_stopped = 1; + } + std::unique_ptr file_reader; int row_group_total = 0; // indices of columns to read from Parquet file @@ -35,6 +40,8 @@ private: std::unique_ptr arrow_column_to_ch_column; int row_group_current = 0; const FormatSettings format_settings; + + std::atomic is_stopped{0}; }; } diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 306c010d2cd..2105228abf6 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -176,6 +176,12 @@ HDFSSource::HDFSSource( initialize(); } +void HDFSSource::onCancel() +{ + if (reader) + reader->cancel(); +} + bool HDFSSource::initialize() { current_path = (*file_iterator)(); diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 611ea3476b0..3e2f7a43127 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -118,6 +118,8 @@ public: Chunk generate() override; + void onCancel() override; + private: StorageHDFSPtr storage; StorageMetadataPtr metadata_snapshot; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 552aa1caa6b..3a03ac3906c 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -222,6 +222,13 @@ StorageS3Source::StorageS3Source( } +void StorageS3Source::onCancel() +{ + if (reader) + reader->cancel(); +} + + bool StorageS3Source::initialize() { String current_key = (*file_iterator)(); diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 8ce287ff681..248238379dc 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -68,6 +68,8 @@ public: Chunk generate() override; + void onCancel() override; + private: String name; String bucket; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 5fa16a25900..0eec77ac8e7 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -107,6 +107,12 @@ namespace }; using URIInfoPtr = std::shared_ptr; + void onCancel() override + { + if (reader) + reader->cancel(); + } + StorageURLSource( URIInfoPtr uri_info_, const std::string & http_method, From ffc77e5f21a2e9f878af3c69a90e4489dc656924 Mon Sep 17 00:00:00 2001 From: pdv-ru Date: Mon, 27 Dec 2021 22:45:45 +0300 Subject: [PATCH 0475/1260] Update clickhouse-keeper.md --- docs/ru/operations/clickhouse-keeper.md | 20 +++++++++----------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/docs/ru/operations/clickhouse-keeper.md b/docs/ru/operations/clickhouse-keeper.md index b30749a8f76..0598b97b47d 100644 --- a/docs/ru/operations/clickhouse-keeper.md +++ b/docs/ru/operations/clickhouse-keeper.md @@ -114,9 +114,8 @@ clickhouse-keeper --config /etc/your_path_to_config/config.xml --daemon ClickHouse Keeper также поддерживает 4-х буквенные команды, почти такие же, как у Zookeeper. Каждая команда состоит из 4-х символов, например, `mntr`, `stat` и т. д. Несколько интересных команд: `stat` предоставляет общую информацию о сервере и подключенных клиентах, а `srvr` и `cons` предоставляют расширенные сведения о сервере и подключениях соответственно. -4-х буквенные команды имеют параметр для настройки разрешенного списка `four_letter_word_white_list`, который имеет значение по умолчанию "conf,cons,crst,envi,ruok,srst,srvr,stat, wchc,wchs,dirs,mntr,isro". +У 4-х буквенных команд есть параметр для настройки разрешенного списка `four_letter_word_white_list`, который имеет значение по умолчанию "conf,cons,crst,envi,ruok,srst,srvr,stat, wchc,wchs,dirs,mntr,isro". -You can issue the commands to ClickHouse Keeper via telnet or nc, at the client port. Вы можете отправлять команды в ClickHouse Keeper через telnet или nc через клиентский порт. ``` @@ -125,13 +124,13 @@ echo mntr | nc localhost 9181 Ниже приведен подробный список 4-х буквенных команд: -- `ruok`: Проверяет, запущен ли сервер без ошибок. Если это так, сервер ответит `imok`. В противном случае он не ответит. Ответ `imok` не обязательно означает, что сервер присоединился к кворуму, просто процесс сервера активен и привязан к указанному клиентскому порту. Используйте команду `stat` для получения подробной информации о состоянии кворума и клиентском подключении. +- `ruok`: Проверяет, что сервер запущен без ошибок. В этом случае сервер ответит `imok`. В противном случае он не ответит. Ответ `imok` не обязательно означает, что сервер присоединился к кворуму, а указывает, что процесс сервера активен и привязан к указанному клиентскому порту. Используйте команду `stat` для получения подробной информации о состоянии кворума и клиентском подключении. ``` imok ``` -- `mntr`: Выводит список переменных, которые могут быть использованы для мониторинга работоспособности кластера. +- `mntr`: Выводит список переменных, которые используются для мониторинга работоспособности кластера. ``` zk_version v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 @@ -184,7 +183,7 @@ Mode: leader Node count: 4 ``` -- `srst`: Сбрасывает статистику сервера. Команда повлияет на результат `srvr`, `mntr` и `stat`. +- `srst`: Сбрасывает статистику сервера. Команда влияет на результат вывода `srvr`, `mntr` и `stat`. ``` Server stats reset. @@ -223,7 +222,7 @@ compress_snapshots_with_zstd_format=true configuration_change_tries_count=20 ``` -- `cons`: Выводит полную информацию о подключении/сеансе для всех клиентов, подключенных к этому серверу. Включает информацию о количестве принятых/отправленных пакетов, идентификаторе сессии, задержках операций, последней выполненной операции и т. д. +- `cons`: Выводит полную информацию о подключениях/сессиях для всех клиентов, подключенных к этому серверу. Включает информацию о количестве принятых/отправленных пакетов, идентификаторе сессии, задержках операций, последней выполненной операции и т. д. ``` 192.168.1.1:52163(recved=0,sent=0,sid=0xffffffffffffffff,lop=NA,est=1636454787393,to=30000,lzxid=0xffffffffffffffff,lresp=0,llat=0,minlat=0,avglat=0,maxlat=0) @@ -253,14 +252,14 @@ user.tmp=/var/folders/b4/smbq5mfj7578f2jzwn602tt40000gn/T/ ``` -- `dirs`: Показывает общий размер файлов снепшотов и журналов в байтах +- `dirs`: Показывает общий размер файлов снимков и журналов в байтах. ``` snapshot_dir_size: 0 log_dir_size: 3875 ``` -- `isro`: Проверяет, работает ли сервер в режиме только для чтения. Сервер ответит `ro`, если он находится в режиме только для чтения, или `rw`, если нет. +- `isro`: Проверяет, что сервер работает в режиме только для чтения. Сервер ответит `ro`, если он находится в режиме только для чтения, или `rw`, если нет. ``` rw @@ -280,15 +279,14 @@ Total watches:1 /clickhouse/task_queue/ddl ``` -- `wchp`: Lists detailed information on watches for the server, by path. This outputs a list of paths (znodes) with associated sessions. Note, depending on the number of watches this operation may be expensive (i. e. impact server performance), use it carefully. -Показывает подробную информацию о наблюдениях за сервером по определенному пути. При этом выводится список путей (узлов) с соответствующими сессиями. Обратите внимание, что в зависимости от количества наблюдений эта операция может быть дорогостоящей (т. е. повлиять на производительность сервера), используйте ее осторожно. +- `wchp`: Показывает подробную информацию о наблюдениях за сервером в разбивке по пути. При этом выводится список путей (узлов) с соответствующими сессиями. Обратите внимание, что в зависимости от количества наблюдений эта операция может быть дорогостоящей (т. е. повлиять на производительность сервера), используйте ее осторожно. ``` /clickhouse/task_queue/ddl 0x0000000000000001 ``` -- `dump`: Выводит список незавершенных сеансов и эфемерные узлы. Это работает только с лидером. +- `dump`: Выводит список незавершенных сеансов и эфемерных узлов. Это работает только с лидером. ``` Sessions dump (2): From 0e2d0d70bae34dbaa04a44a0b901dacbb090f4c5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 27 Dec 2021 23:16:53 +0300 Subject: [PATCH 0476/1260] add asynchronous inserts to query log --- src/Interpreters/executeQuery.cpp | 92 ++++++++++--------- .../02156_async_insert_query_log.reference | 4 + .../02156_async_insert_query_log.sh | 25 +++++ 3 files changed, 79 insertions(+), 42 deletions(-) create mode 100644 tests/queries/0_stateless/02156_async_insert_query_log.reference create mode 100755 tests/queries/0_stateless/02156_async_insert_query_log.sh diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index e177fd8e6b3..b192416f5bd 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -579,8 +579,14 @@ static std::tuple executeQueryImpl( } } else + { /// reset Input callbacks if query is not INSERT SELECT context->resetInputCallbacks(); + } + + StreamLocalLimits limits; + std::shared_ptr quota; + std::unique_ptr interpreter; auto * queue = context->getAsynchronousInsertQueue(); const bool async_insert = queue @@ -591,65 +597,64 @@ static std::tuple executeQueryImpl( { queue->push(ast, context); - BlockIO io; if (settings.wait_for_async_insert) { auto timeout = settings.wait_for_async_insert_timeout.totalMilliseconds(); auto query_id = context->getCurrentQueryId(); auto source = std::make_shared(query_id, timeout, *queue); - io.pipeline = QueryPipeline(Pipe(std::move(source))); + res.pipeline = QueryPipeline(Pipe(std::move(source))); } - return std::make_tuple(ast, std::move(io)); - } - - auto interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal)); - - std::shared_ptr quota; - if (!interpreter->ignoreQuota()) - { quota = context->getQuota(); if (quota) + quota->used(QuotaType::QUERY_INSERTS, 1); + } + else + { + interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal)); + + if (!interpreter->ignoreQuota()) { - if (ast->as() || ast->as()) + quota = context->getQuota(); + if (quota) { - quota->used(QuotaType::QUERY_SELECTS, 1); + if (ast->as() || ast->as()) + { + quota->used(QuotaType::QUERY_SELECTS, 1); + } + else if (ast->as()) + { + quota->used(QuotaType::QUERY_INSERTS, 1); + } + quota->used(QuotaType::QUERIES, 1); + quota->checkExceeded(QuotaType::ERRORS); } - else if (ast->as()) - { - quota->used(QuotaType::QUERY_INSERTS, 1); - } - quota->used(QuotaType::QUERIES, 1); - quota->checkExceeded(QuotaType::ERRORS); } - } - StreamLocalLimits limits; - if (!interpreter->ignoreLimits()) - { - limits.mode = LimitsMode::LIMITS_CURRENT; //-V1048 - limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode); - } - - { - std::unique_ptr span; - if (context->query_trace_context.trace_id != UUID()) + if (!interpreter->ignoreLimits()) { - auto * raw_interpreter_ptr = interpreter.get(); - std::string class_name(abi::__cxa_demangle(typeid(*raw_interpreter_ptr).name(), nullptr, nullptr, nullptr)); - span = std::make_unique(class_name + "::execute()"); + limits.mode = LimitsMode::LIMITS_CURRENT; //-V1048 + limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode); } - res = interpreter->execute(); - } - QueryPipeline & pipeline = res.pipeline; + { + std::unique_ptr span; + if (context->query_trace_context.trace_id != UUID()) + { + auto * raw_interpreter_ptr = interpreter.get(); + std::string class_name(abi::__cxa_demangle(typeid(*raw_interpreter_ptr).name(), nullptr, nullptr, nullptr)); + span = std::make_unique(class_name + "::execute()"); + } + res = interpreter->execute(); + } - if (const auto * insert_interpreter = typeid_cast(&*interpreter)) - { - /// Save insertion table (not table function). TODO: support remote() table function. - auto table_id = insert_interpreter->getDatabaseTable(); - if (!table_id.empty()) - context->setInsertionTable(std::move(table_id)); + if (const auto * insert_interpreter = typeid_cast(&*interpreter)) + { + /// Save insertion table (not table function). TODO: support remote() table function. + auto table_id = insert_interpreter->getDatabaseTable(); + if (!table_id.empty()) + context->setInsertionTable(std::move(table_id)); + } } if (process_list_entry) @@ -663,6 +668,8 @@ static std::tuple executeQueryImpl( /// Hold element of process list till end of query execution. res.process_list_entry = process_list_entry; + auto & pipeline = res.pipeline; + if (pipeline.pulling() || pipeline.completed()) { /// Limits on the result, the quota on the result, and also callback for progress. @@ -712,7 +719,8 @@ static std::tuple executeQueryImpl( elem.query_views = info.views; } - interpreter->extendQueryLogElem(elem, ast, context, query_database, query_table); + if (interpreter) + interpreter->extendQueryLogElem(elem, ast, context, query_database, query_table); if (settings.log_query_settings) elem.query_settings = std::make_shared(context->getSettingsRef()); diff --git a/tests/queries/0_stateless/02156_async_insert_query_log.reference b/tests/queries/0_stateless/02156_async_insert_query_log.reference new file mode 100644 index 00000000000..15fad7dd0ab --- /dev/null +++ b/tests/queries/0_stateless/02156_async_insert_query_log.reference @@ -0,0 +1,4 @@ +1 a +2 b +INSERT INTO async_inserts_2156 VALUES 1 0 +INSERT INTO async_inserts_2156 VALUES 1 diff --git a/tests/queries/0_stateless/02156_async_insert_query_log.sh b/tests/queries/0_stateless/02156_async_insert_query_log.sh new file mode 100755 index 00000000000..e5508273dbb --- /dev/null +++ b/tests/queries/0_stateless/02156_async_insert_query_log.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +url_wait="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=1" +url_no_wait="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=1" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts_2156" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE async_inserts_2156 (id UInt32, s String) ENGINE = Memory" + +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" -d "INSERT INTO async_inserts_2156 VALUES (1, 'a')" +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=1" -d "INSERT INTO async_inserts_2156 VALUES (2, 'b')" + +${CLICKHOUSE_CLIENT} -q "SELECT * FROM async_inserts_2156 ORDER BY id" + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +${CLICKHOUSE_CLIENT} -q "SELECT query, Settings['async_insert'], Settings['wait_for_async_insert'] FROM system.query_log \ + WHERE event_date = today() AND current_database = '$CLICKHOUSE_DATABASE' \ + AND query ILIKE 'INSERT INTO async_inserts_2156 VALUES%' AND type = 'QueryFinish' \ + ORDER BY query_start_time_microseconds" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE async_inserts_2156" From af159d1f2e33fd3a9e34c20d61abcca206b26364 Mon Sep 17 00:00:00 2001 From: pdv-ru Date: Mon, 27 Dec 2021 23:23:08 +0300 Subject: [PATCH 0477/1260] Update clickhouse-keeper.md --- docs/en/operations/clickhouse-keeper.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 6738f77cff9..afe1a3f3963 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -117,6 +117,7 @@ ClickHouse Keeper also provides 4lw commands which are almost the same with Zook The 4lw commands has a white list configuration `four_letter_word_white_list` which has default value "conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro". You can issue the commands to ClickHouse Keeper via telnet or nc, at the client port. + ``` echo mntr | nc localhost 9181 ``` From 413fb290bbab73923e858ab3da94fa359f455436 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 28 Dec 2021 09:19:21 +0800 Subject: [PATCH 0478/1260] remove comments --- src/Common/LRUCache.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 652b02cad64..97d70b5598c 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -152,8 +152,6 @@ public: auto token_it = insert_tokens.find(key); if (token_it != insert_tokens.end() && token_it->second.get() == token) { - // setImpl() may fail, but the final behavior seems not be affected - // next call of getOrTrySet() will still call load_func() is_value_loaded_and_set = setImpl(key, token->value, cache_lock); is_value_loaded = true; } From 2e141b77dbc8eb54d89bfc56cb026e5e13a52437 Mon Sep 17 00:00:00 2001 From: "Chun-Sheng, Li" Date: Tue, 28 Dec 2021 09:27:04 +0800 Subject: [PATCH 0479/1260] Add systemctl way to run server as daemon --- docs/en/getting-started/install.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 4a97ab6589d..b58303e9491 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -152,6 +152,12 @@ If you do not have `service` command, run as $ sudo /etc/init.d/clickhouse-server start ``` +If you have `systemctl` command, run as + +``` bash +$ sudo systemctl start clickhouse-server.service +``` + See the logs in the `/var/log/clickhouse-server/` directory. If the server does not start, check the configurations in the file `/etc/clickhouse-server/config.xml`. From 22b2d49d04a0c3c335339a95c44145eb1b72f05e Mon Sep 17 00:00:00 2001 From: peter279k Date: Tue, 28 Dec 2021 09:42:49 +0800 Subject: [PATCH 0480/1260] Add friendly message to notify user this instance --- docker/server/README.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/server/README.md b/docker/server/README.md index c63bb980c13..5a96a63bb05 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -17,6 +17,8 @@ $ docker run -d --name some-clickhouse-server --ulimit nofile=262144:262144 clic By default ClickHouse will be accessible only via docker network. See the [networking section below](#networking). +By default, starting above server instance will be run as default user without password. + ### connect to it from a native client ```bash $ docker run -it --rm --link some-clickhouse-server:clickhouse-server clickhouse/clickhouse-client --host clickhouse-server From 1ac697cad2015e5e3c43946ecb0fb29eda627f71 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 28 Dec 2021 11:26:39 +0800 Subject: [PATCH 0481/1260] some modifications --- src/Common/ErrorCodes.cpp | 5 -- .../Cache/ExternalDataSourceCache.cpp | 46 ++++++------------- src/Storages/Cache/ExternalDataSourceCache.h | 7 +-- src/Storages/Cache/RemoteCacheController.cpp | 9 ++-- 4 files changed, 17 insertions(+), 50 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f861d408a16..1ba512f7be3 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -606,11 +606,6 @@ M(650, SNAPPY_UNCOMPRESS_FAILED) \ M(651, SNAPPY_COMPRESS_FAILED) \ M(652, NO_HIVEMETASTORE) \ - M(653, NOT_INIT)\ - M(654, DISK_OVERFLOW)\ - M(655, FILE_BROKEN)\ - M(656, END_OF_FILE)\ - M(657, CANNOT_RELEASE)\ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp index 8744a19fdca..5dd28ebc0ff 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.cpp +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -24,10 +24,7 @@ namespace DB namespace fs = std::filesystem; namespace ErrorCodes { - extern const int OK; extern const int LOGICAL_ERROR; - extern const int NOT_INIT; - extern const int DISK_OVERFLOW; } @@ -45,9 +42,8 @@ std::unique_ptr RemoteReadBuffer::create(ContextPtr context, IRemote { auto remote_path = remote_file_metadata->remote_path; auto remote_read_buffer = std::make_unique(buff_size); - ErrorCodes::ErrorCode error; - std::tie(remote_read_buffer->file_cache_controller, read_buffer, error) = ExternalDataSourceCache::instance().createReader(context, remote_file_metadata, read_buffer); + std::tie(remote_read_buffer->file_cache_controller, read_buffer) = ExternalDataSourceCache::instance().createReader(context, remote_file_metadata, read_buffer); if (remote_read_buffer->file_cache_controller == nullptr) { return read_buffer; @@ -114,17 +110,14 @@ ExternalDataSourceCache & ExternalDataSourceCache::instance() return instance; } -void ExternalDataSourceCache::recoverCachedFilesMetadata( - const fs::path & current_path, - size_t current_depth, - size_t max_depth) +void ExternalDataSourceCache::recoverTask() { - if (current_depth >= max_depth) + std::vector invalid_paths; + for (auto const & group_dir : fs::directory_iterator{root_dir}) { - std::vector invalid_paths; - for (auto const & dir : fs::directory_iterator{current_path}) + for (auto const & cache_dir : fs::directory_iterator{group_dir.path()}) { - String path = dir.path(); + String path = cache_dir.path(); auto cache_controller = RemoteCacheController::recover(path); if (!cache_controller) { @@ -136,22 +129,9 @@ void ExternalDataSourceCache::recoverCachedFilesMetadata( invalid_paths.emplace_back(path); } } - for (auto & path : invalid_paths) - { - fs::remove_all(path); - } - return; } - - for (auto const & dir : fs::directory_iterator{current_path}) - { - recoverCachedFilesMetadata(dir.path(), current_depth + 1, max_depth); - } -} - -void ExternalDataSourceCache::recoverTask() -{ - recoverCachedFilesMetadata(root_dir, 1, 2); + for (auto & path : invalid_paths) + fs::remove_all(path); initialized = true; LOG_INFO(log, "Recovered from directory:{}", root_dir); } @@ -191,14 +171,14 @@ String ExternalDataSourceCache::calculateLocalPath(IRemoteFileMetadataPtr metada return fs::path(root_dir) / hashcode_str.substr(0, 3) / hashcode_str; } -std::tuple, ErrorCodes::ErrorCode> +std::pair> ExternalDataSourceCache::createReader(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr & read_buffer) { // If something is wrong on startup, rollback to read from the original ReadBuffer if (!isInitialized()) { LOG_ERROR(log, "ExternalDataSourceCache has not been initialized"); - return {nullptr, std::move(read_buffer), ErrorCodes::NOT_INIT}; + return {nullptr, std::move(read_buffer)}; } auto remote_path = remote_file_metadata->remote_path; @@ -221,7 +201,7 @@ ExternalDataSourceCache::createReader(ContextPtr context, IRemoteFileMetadataPtr } else { - return {cache, nullptr, ErrorCodes::OK}; + return {cache, nullptr}; } } @@ -235,10 +215,10 @@ ExternalDataSourceCache::createReader(ContextPtr context, IRemoteFileMetadataPtr LOG_ERROR(log, "Insert the new cache failed. new file size:{}, current total size:{}", remote_file_metadata->file_size, lru_caches->weight()); - return {nullptr, std::move(read_buffer), ErrorCodes::DISK_OVERFLOW}; + return {nullptr, std::move(read_buffer)}; } new_cache->startBackgroundDownload(std::move(read_buffer), context->getSchedulePool()); - return {new_cache, nullptr, ErrorCodes::OK}; + return {new_cache, nullptr}; } } diff --git a/src/Storages/Cache/ExternalDataSourceCache.h b/src/Storages/Cache/ExternalDataSourceCache.h index 24c1bcd8607..48faddd7cce 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.h +++ b/src/Storages/Cache/ExternalDataSourceCache.h @@ -60,7 +60,7 @@ public: inline bool isInitialized() const { return initialized; } - std::tuple, ErrorCodes::ErrorCode> + std::pair> createReader(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr & read_buffer); void updateTotalSize(size_t size) { total_size += size; } @@ -84,10 +84,5 @@ private: BackgroundSchedulePool::TaskHolder recover_task_holder; void recoverTask(); - void recoverCachedFilesMetadata( - const std::filesystem::path & current_path, - size_t current_depth, - size_t max_depth); }; - } diff --git a/src/Storages/Cache/RemoteCacheController.cpp b/src/Storages/Cache/RemoteCacheController.cpp index 01394120cb8..8da9a01fe34 100644 --- a/src/Storages/Cache/RemoteCacheController.cpp +++ b/src/Storages/Cache/RemoteCacheController.cpp @@ -31,6 +31,9 @@ std::shared_ptr RemoteCacheController::recover(const std: auto cache_controller = std::make_shared(nullptr, local_path_, 0); if (cache_controller->file_status != DOWNLOADED) { + // do not load this invalid cached file and clear it. the clear action is in + // ExternalDataSourceCache::recoverTask(), because deleting directories during iteration will + // cause unexpected behaviors LOG_INFO(log, "Recover cached file failed. local path:{}", local_path_.string()); return nullptr; } @@ -45,12 +48,6 @@ std::shared_ptr RemoteCacheController::recover(const std: } if (!cache_controller->file_metadata_ptr) { - // do not load this invalid cached file and clear it. the clear action is in - // ExternalDataSourceCache::recoverCachedFilesMetadata(), because deleting directories during iteration will - // cause unexpected behaviors - LOG_ERROR(log, "Cannot create the metadata class : {}. The cached file is invalid and will be remove. path:{}", - cache_controller->metadata_class, - local_path_.string()); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid metadata class:{}", cache_controller->metadata_class); } ReadBufferFromFile file_readbuffer((local_path_ / "metadata.txt").string()); From 6bada8b0ba323bc4247cae8c5d22b44f30bf26f6 Mon Sep 17 00:00:00 2001 From: pdv-ru <86398979+pdv-ru@users.noreply.github.com> Date: Tue, 28 Dec 2021 09:11:19 +0300 Subject: [PATCH 0482/1260] add anchors --- docs/en/operations/clickhouse-keeper.md | 12 ++++++------ docs/ru/operations/clickhouse-keeper.md | 12 ++++++------ 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index afe1a3f3963..7c13448cf97 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -3,14 +3,14 @@ toc_priority: 66 toc_title: ClickHouse Keeper --- -# [pre-production] ClickHouse Keeper +# [pre-production] ClickHouse Keeper {#clickHouse_keeper} ClickHouse server uses [ZooKeeper](https://zookeeper.apache.org/) coordination system for data [replication](../engines/table-engines/mergetree-family/replication.md) and [distributed DDL](../sql-reference/distributed-ddl.md) queries execution. ClickHouse Keeper is an alternative coordination system compatible with ZooKeeper. !!! warning "Warning" This feature is currently in the pre-production stage. We test it in our CI and on small internal installations. -## Implementation details +## Implementation details {#implementation_details} ZooKeeper is one of the first well-known open-source coordination systems. It's implemented in Java, has quite a simple and powerful data model. ZooKeeper's coordination algorithm called ZAB (ZooKeeper Atomic Broadcast) doesn't provide linearizability guarantees for reads, because each ZooKeeper node serves reads locally. Unlike ZooKeeper ClickHouse Keeper is written in C++ and uses [RAFT algorithm](https://raft.github.io/) [implementation](https://github.com/eBay/NuRaft). This algorithm allows to have linearizability for reads and writes, has several open-source implementations in different languages. @@ -21,7 +21,7 @@ ClickHouse Keeper supports Access Control List (ACL) the same way as [ZooKeeper] !!! info "Note" External integrations are not supported. -## Configuration +## Configuration {#configuration} ClickHouse Keeper can be used as a standalone replacement for ZooKeeper or as an internal part of the ClickHouse server, but in both cases configuration is almost the same `.xml` file. The main ClickHouse Keeper configuration tag is ``. Keeper configuration has the following parameters: @@ -102,7 +102,7 @@ Examples of configuration for quorum with three nodes can be found in [integrati ``` -## How to run +## How to run {#how_to_run} ClickHouse Keeper is bundled into the ClickHouse server package, just add configuration of `` and start ClickHouse server as always. If you want to run standalone ClickHouse Keeper you can start it in a similar way with: @@ -110,7 +110,7 @@ ClickHouse Keeper is bundled into the ClickHouse server package, just add config clickhouse-keeper --config /etc/your_path_to_config/config.xml --daemon ``` -## Four Letter Word Commands +## Four Letter Word Commands {#four_letter_word_commands} ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as `mntr`, `stat` etc. There are some more interesting commands: `stat` gives some general information about the server and connected clients, while `srvr` and `cons` give extended details on server and connections respectively. @@ -297,7 +297,7 @@ Sessions with Ephemerals (1): /clickhouse/task_queue/ddl ``` -## [experimental] Migration from ZooKeeper +## [experimental] Migration from ZooKeeper {#migration_from_zookeeper} Seamlessly migration from ZooKeeper to ClickHouse Keeper is impossible you have to stop your ZooKeeper cluster, convert data and start ClickHouse Keeper. `clickhouse-keeper-converter` tool allows converting ZooKeeper logs and snapshots to ClickHouse Keeper snapshot. It works only with ZooKeeper > 3.4. Steps for migration: diff --git a/docs/ru/operations/clickhouse-keeper.md b/docs/ru/operations/clickhouse-keeper.md index 0598b97b47d..d177294a5a1 100644 --- a/docs/ru/operations/clickhouse-keeper.md +++ b/docs/ru/operations/clickhouse-keeper.md @@ -3,14 +3,14 @@ toc_priority: 66 toc_title: ClickHouse Keeper --- -# [пре-продакшн] ClickHouse Keeper +# [пре-продакшн] ClickHouse Keeper {#clickHouse_keeper} Сервер ClickHouse использует сервис координации [ZooKeeper](https://zookeeper.apache.org/) для [репликации](../engines/table-engines/mergetree-family/replication.md) данных и выполнения [распределенных DDL запросов](../sql-reference/distributed-ddl.md). ClickHouse Keeper — это альтернативный сервис координации, совместимый с ZooKeeper. !!! warning "Предупреждение" ClickHouse Keeper находится в стадии пре-продакшн и тестируется в CI ClickHouse и на нескольких внутренних инсталляциях. -## Детали реализации +## Детали реализации {#implementation_details} ZooKeeper — один из первых широко известных сервисов координации с открытым исходным кодом. Он реализован на языке программирования Java, имеет достаточно простую и мощную модель данных. Алгоритм координации Zookeeper называется ZAB (ZooKeeper Atomic Broadcast). Он не гарантирует линеаризуемость операций чтения, поскольку каждый узел ZooKeeper обслуживает чтения локально. В отличие от ZooKeeper, ClickHouse Keeper реализован на C++ и использует алгоритм [RAFT](https://raft.github.io/), [реализация](https://github.com/eBay/NuRaft). Этот алгоритм позволяет достичь линеаризуемости чтения и записи, имеет несколько реализаций с открытым исходным кодом на разных языках. @@ -21,7 +21,7 @@ ZooKeeper — один из первых широко известных сер !!! info "Примечание" Внешние интеграции не поддерживаются. -## Конфигурация +## Конфигурация {#configuration} ClickHouse Keeper может использоваться как равноценная замена ZooKeeper или как внутренняя часть сервера ClickHouse, но в обоих случаях конфигурация представлена файлом `.xml`. Главный тег конфигурации ClickHouse Keeper — это ``. Параметры конфигурации: @@ -102,7 +102,7 @@ ClickHouse Keeper может использоваться как равноце ``` -## Как запустить +## Как запустить {#how_to_run} ClickHouse Keeper входит в пакет `clickhouse-server`, просто добавьте кофигурацию `` и запустите сервер ClickHouse как обычно. Если вы хотите запустить ClickHouse Keeper автономно, сделайте это аналогичным способом: @@ -110,7 +110,7 @@ ClickHouse Keeper входит в пакет `clickhouse-server`, просто clickhouse-keeper --config /etc/your_path_to_config/config.xml --daemon ``` -## 4-х буквенные команды +## 4-х буквенные команды {#four_letter_word_commands} ClickHouse Keeper также поддерживает 4-х буквенные команды, почти такие же, как у Zookeeper. Каждая команда состоит из 4-х символов, например, `mntr`, `stat` и т. д. Несколько интересных команд: `stat` предоставляет общую информацию о сервере и подключенных клиентах, а `srvr` и `cons` предоставляют расширенные сведения о сервере и подключениях соответственно. @@ -298,7 +298,7 @@ Sessions with Ephemerals (1): ``` -## [экспериментально] Переход с ZooKeeper +## [экспериментально] Переход с ZooKeeper {#migration_from_zookeeper} Плавный переход с ZooKeeper на ClickHouse Keeper невозможен, необходимо остановить кластер ZooKeeper, преобразовать данные и запустить ClickHouse Keeper. Утилита `clickhouse-keeper-converter` конвертирует журналы и снэпшоты ZooKeeper в снэпшот ClickHouse Keeper. Работа утилиты проверена только для версий ZooKeeper выше 3.4. Для миграции необходимо выполнить следующие шаги: From 2a9662bf6944c6db95fd9cf1c0b455dbb69d7998 Mon Sep 17 00:00:00 2001 From: pdv-ru <86398979+pdv-ru@users.noreply.github.com> Date: Tue, 28 Dec 2021 09:13:25 +0300 Subject: [PATCH 0483/1260] Update clickhouse-keeper.md --- docs/ru/operations/clickhouse-keeper.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/clickhouse-keeper.md b/docs/ru/operations/clickhouse-keeper.md index d177294a5a1..4a80651f879 100644 --- a/docs/ru/operations/clickhouse-keeper.md +++ b/docs/ru/operations/clickhouse-keeper.md @@ -252,7 +252,7 @@ user.tmp=/var/folders/b4/smbq5mfj7578f2jzwn602tt40000gn/T/ ``` -- `dirs`: Показывает общий размер файлов снимков и журналов в байтах. +- `dirs`: Показывает общий размер файлов снэпшотов и журналов в байтах. ``` snapshot_dir_size: 0 @@ -286,7 +286,7 @@ Total watches:1 0x0000000000000001 ``` -- `dump`: Выводит список незавершенных сеансов и эфемерных узлов. Это работает только с лидером. +- `dump`: Выводит список незавершенных сеансов и эфемерных узлов. Это работает только с репликой-лидером. ``` Sessions dump (2): From 72a9fe51d29acd0b5e53c591e3cc4bc514ed5522 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Tue, 28 Dec 2021 09:23:19 +0300 Subject: [PATCH 0484/1260] Update join.md --- docs/ru/sql-reference/statements/select/join.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/select/join.md b/docs/ru/sql-reference/statements/select/join.md index b1e739e6c77..70893d7cc8f 100644 --- a/docs/ru/sql-reference/statements/select/join.md +++ b/docs/ru/sql-reference/statements/select/join.md @@ -61,7 +61,7 @@ FROM Строки объединяются только тогда, когда всё составное условие выполнено. Если оно не выполнено, то строки могут попасть в результат в зависимости от типа `JOIN`. Обратите внимание, что если то же самое условие поместить в секцию `WHERE`, то строки, для которых оно не выполняется, никогда не попаду в результат. -Оператор `OR` внутри секции `ON` работает, используя алгоритм хэш-соединения — на каждый агрумент `OR` с ключами соединений для `JOIN` создается отдельная хэш-таблица, поэтому потребление памяти и время выполнения запроса растет линейно при увеличении количества выражений `OR` секции `ON`. +Оператор `OR` внутри секции `ON` работает, используя алгоритм хеш-соединения — на каждый агрумент `OR` с ключами соединений для `JOIN` создается отдельная хеш-таблица, поэтому потребление памяти и время выполнения запроса растет линейно при увеличении количества выражений `OR` секции `ON`. !!! note "Примечание" Если в условии использованы столбцы из разных таблиц, то пока поддерживается только оператор равенства (`=`). From 2e9979d67237f0b98c64df56a564277b55ab7b3f Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Tue, 28 Dec 2021 15:44:07 +0800 Subject: [PATCH 0485/1260] Fix ACLMap num --- src/Coordination/ACLMap.cpp | 3 ++- src/Coordination/ACLMap.h | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Coordination/ACLMap.cpp b/src/Coordination/ACLMap.cpp index 863dfdec281..70cb74cd502 100644 --- a/src/Coordination/ACLMap.cpp +++ b/src/Coordination/ACLMap.cpp @@ -46,7 +46,7 @@ uint64_t ACLMap::convertACLs(const Coordination::ACLs & acls) return acl_to_num[acls]; /// Start from one - auto index = acl_to_num.size() + 1; + auto index = max_acl_id++; acl_to_num[acls] = index; num_to_acl[index] = acls; @@ -69,6 +69,7 @@ void ACLMap::addMapping(uint64_t acls_id, const Coordination::ACLs & acls) { num_to_acl[acls_id] = acls; acl_to_num[acls] = acls_id; + max_acl_id = std::max(acls_id + 1, max_acl_id); /// max_acl_id pointer next slot } void ACLMap::addUsage(uint64_t acl_id) diff --git a/src/Coordination/ACLMap.h b/src/Coordination/ACLMap.h index 2313b3e7cd3..e1b2ce1eff6 100644 --- a/src/Coordination/ACLMap.h +++ b/src/Coordination/ACLMap.h @@ -31,6 +31,7 @@ private: ACLToNumMap acl_to_num; NumToACLMap num_to_acl; UsageCounter usage_counter; + uint64_t max_acl_id{1}; public: /// Convert ACL to number. If it's new ACL than adds it to map @@ -43,7 +44,7 @@ public: /// Mapping from numbers to ACLs vectors. Used during serialization. const NumToACLMap & getMapping() const { return num_to_acl; } - /// Add mapping to ACLMap. Used during deserialization. + /// Add mapping to ACLMap. Used during deserialization from snapshot. void addMapping(uint64_t acls_id, const Coordination::ACLs & acls); /// Add/remove usage of some id. Used to remove unused ACLs. From 78710d71b5d6e0df8c39fa2e4009eadd71a8ab3c Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Tue, 28 Dec 2021 10:49:35 +0300 Subject: [PATCH 0486/1260] Specify sql syntax in code snippets in join.md --- docs/en/sql-reference/statements/select/join.md | 2 +- docs/ru/sql-reference/statements/select/join.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index a5b14ccbcfa..3d302be561a 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -137,7 +137,7 @@ Result: Query with `INNER` type of a join and conditions with `OR` and `AND`: -``` +``` sql SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key AND t2.val > 3; ``` diff --git a/docs/ru/sql-reference/statements/select/join.md b/docs/ru/sql-reference/statements/select/join.md index 70893d7cc8f..bb9b7445083 100644 --- a/docs/ru/sql-reference/statements/select/join.md +++ b/docs/ru/sql-reference/statements/select/join.md @@ -138,7 +138,7 @@ SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key; Запрос с типом соединения `INNER` и условиями с операторами `OR` и `AND`: -``` +``` sql SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key AND t2.val > 3; ``` From 9303b5f0ce8160607eb6621a7baa79a7588b5dd2 Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Tue, 28 Dec 2021 15:54:00 +0800 Subject: [PATCH 0487/1260] better process empty acls --- src/Coordination/ACLMap.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Coordination/ACLMap.cpp b/src/Coordination/ACLMap.cpp index 70cb74cd502..41b759531cf 100644 --- a/src/Coordination/ACLMap.cpp +++ b/src/Coordination/ACLMap.cpp @@ -42,6 +42,9 @@ bool ACLMap::ACLsComparator::operator()(const Coordination::ACLs & left, const C uint64_t ACLMap::convertACLs(const Coordination::ACLs & acls) { + if (acls.empty()) + return 0; + if (acl_to_num.count(acls)) return acl_to_num[acls]; From 875eb5475434cc6189d5667a338a323d202e9c89 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 28 Dec 2021 11:21:33 +0300 Subject: [PATCH 0488/1260] Update test.py --- tests/integration/test_jemalloc_percpu_arena/test.py | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_jemalloc_percpu_arena/test.py b/tests/integration/test_jemalloc_percpu_arena/test.py index cff593f82b7..bdd0ada966f 100755 --- a/tests/integration/test_jemalloc_percpu_arena/test.py +++ b/tests/integration/test_jemalloc_percpu_arena/test.py @@ -48,20 +48,14 @@ def skip_if_jemalloc_disabled(): SELECT value FROM system.build_options WHERE name = 'USE_JEMALLOC'" """).strip() if output != b'ON' and output != b'1': - return True - - return False + pytest.skip(f'Compiled w/o jemalloc (USE_JEMALLOC={output})') # Ensure that clickhouse works even when number of online CPUs # (_SC_NPROCESSORS_ONLN) is smaller then available (_SC_NPROCESSORS_CONF). # # Refs: https://github.com/jemalloc/jemalloc/pull/2181 def test_jemalloc_percpu_arena(): - # TODO: revert me when https://github.com/ClickHouse/ClickHouse/pull/33162 - # will be ready - if skip_if_jemalloc_disabled(): - print("Jemalloc disabled, exiting") - return + skip_if_jemalloc_disabled() assert multiprocessing.cpu_count() > CPU_ID From 2d87f0a0e3560c336d695779846e375afe8ff635 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Tue, 28 Dec 2021 11:45:38 +0300 Subject: [PATCH 0489/1260] Fix debug build --- src/Disks/IDisk.h | 8 ++++---- src/Disks/IDiskRemote.h | 8 ++++---- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 28d3f0f615d..bf20c7bf354 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -252,16 +252,16 @@ public: /// Used for work with custom metadata. virtual std::unique_ptr readMetaFile( const String & path, - const ReadSettings & settings = ReadSettings{}, - std::optional size = {}) const; + const ReadSettings & settings, + std::optional size) const; /// Open the local file for write and return WriteBufferFromFileBase object. /// Overridden in IDiskRemote. /// Used for work with custom metadata. virtual std::unique_ptr writeMetaFile( const String & path, - size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, - WriteMode mode = WriteMode::Rewrite); + size_t buf_size, + WriteMode mode); virtual void removeMetaFileIfExists(const String & path); diff --git a/src/Disks/IDiskRemote.h b/src/Disks/IDiskRemote.h index 734bc5a11fa..c4f475f5b3e 100644 --- a/src/Disks/IDiskRemote.h +++ b/src/Disks/IDiskRemote.h @@ -138,13 +138,13 @@ public: virtual std::unique_ptr readMetaFile( const String & path, - const ReadSettings & settings = ReadSettings{}, - std::optional size = {}) const override; + const ReadSettings & settings, + std::optional size) const override; virtual std::unique_ptr writeMetaFile( const String & path, - size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, - WriteMode mode = WriteMode::Rewrite) override; + size_t buf_size, + WriteMode mode) override; virtual void removeMetaFileIfExists( const String & path) override; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 930e9f6835f..0eab161195c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7622,7 +7622,7 @@ public: void save(DiskPtr disk, const String & path) const { auto file_path = getFileName(path); - auto buffer = disk->writeMetaFile(file_path); + auto buffer = disk->writeMetaFile(file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite); writeIntText(version, *buffer); buffer->write("\n", 1); writeBoolText(is_replicated, *buffer); @@ -7642,7 +7642,7 @@ public: auto file_path = getFileName(path); if (!disk->exists(file_path)) return false; - auto buffer = disk->readMetaFile(file_path); + auto buffer = disk->readMetaFile(file_path, ReadSettings(), {}); readIntText(version, *buffer); if (version != 1) { From 26fcf1fbc71e175c3f73cd870997890882869581 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 28 Dec 2021 16:57:07 +0800 Subject: [PATCH 0490/1260] fixed a bug in allocating local file cache --- .../Cache/ExternalDataSourceCache.cpp | 56 +++++++++---------- src/Storages/Cache/ExternalDataSourceCache.h | 16 ++++-- src/Storages/Cache/RemoteCacheController.cpp | 3 +- 3 files changed, 42 insertions(+), 33 deletions(-) diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp index 5dd28ebc0ff..a168d5e5c3a 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.cpp +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -27,64 +27,64 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } - -RemoteReadBuffer::RemoteReadBuffer(size_t buff_size) : BufferWithOwnMemory(buff_size) +LocalFileHolder::LocalFileHolder(std::shared_ptr cache_controller):file_cache_controller(cache_controller) { + file_buffer = file_cache_controller->allocFile(); + if (!file_buffer) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Create file readbuffer failed. {}", + file_cache_controller->getLocalPath().string()); + } -RemoteReadBuffer::~RemoteReadBuffer() +LocalFileHolder::~LocalFileHolder() { if (file_cache_controller) file_cache_controller->deallocFile(std::move(file_buffer)); } +RemoteReadBuffer::RemoteReadBuffer(size_t buff_size) : BufferWithOwnMemory(buff_size) +{ +} + std::unique_ptr RemoteReadBuffer::create(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer, size_t buff_size) { auto remote_path = remote_file_metadata->remote_path; auto remote_read_buffer = std::make_unique(buff_size); - std::tie(remote_read_buffer->file_cache_controller, read_buffer) = ExternalDataSourceCache::instance().createReader(context, remote_file_metadata, read_buffer); - if (remote_read_buffer->file_cache_controller == nullptr) - { + std::tie(remote_read_buffer->local_file_holder, read_buffer) = ExternalDataSourceCache::instance().createReader(context, remote_file_metadata, read_buffer); + if (remote_read_buffer->local_file_holder == nullptr) return read_buffer; - } - else - { - remote_read_buffer->file_buffer = remote_read_buffer->file_cache_controller->allocFile(); - if (!remote_read_buffer->file_buffer) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Create file readbuffer failed. {}", - remote_read_buffer->file_cache_controller->getLocalPath().string()); - } remote_read_buffer->remote_file_size = remote_file_metadata->file_size; return remote_read_buffer; } bool RemoteReadBuffer::nextImpl() { - auto start_offset = file_buffer->getPosition(); - auto end_offset = start_offset + file_buffer->internalBuffer().size(); - file_cache_controller->waitMoreData(start_offset, end_offset); + auto start_offset = local_file_holder->file_buffer->getPosition(); + auto end_offset = start_offset + local_file_holder->file_buffer->internalBuffer().size(); + local_file_holder->file_cache_controller->waitMoreData(start_offset, end_offset); - auto status = file_buffer->next(); + auto status = local_file_holder->file_buffer->next(); if (status) { - BufferBase::set(file_buffer->buffer().begin(), - file_buffer->buffer().size(), - file_buffer->offset()); - ProfileEvents::increment(ProfileEvents::ExternalDataSourceLocalCacheReadBytes, file_buffer->available()); + BufferBase::set(local_file_holder->file_buffer->buffer().begin(), + local_file_holder->file_buffer->buffer().size(), + local_file_holder->file_buffer->offset()); + ProfileEvents::increment(ProfileEvents::ExternalDataSourceLocalCacheReadBytes, local_file_holder->file_buffer->available()); } return status; } off_t RemoteReadBuffer::seek(off_t offset, int whence) { - if (!file_buffer) + if (!local_file_holder->file_buffer) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot call seek() in this buffer. It's a bug!"); /* * Need to wait here. For example, the current file has been download at position X, but here we try to seek to * postition Y (Y > X), it would fail. */ - file_cache_controller->waitMoreData(offset, offset + file_buffer->internalBuffer().size()); + auto & file_buffer = local_file_holder->file_buffer; + local_file_holder->file_cache_controller->waitMoreData(offset, offset + file_buffer->internalBuffer().size()); auto ret = file_buffer->seek(offset, whence); BufferBase::set(file_buffer->buffer().begin(), file_buffer->buffer().size(), @@ -94,7 +94,7 @@ off_t RemoteReadBuffer::seek(off_t offset, int whence) off_t RemoteReadBuffer::getPosition() { - return file_buffer->getPosition(); + return local_file_holder->file_buffer->getPosition(); } ExternalDataSourceCache::ExternalDataSourceCache() = default; @@ -171,7 +171,7 @@ String ExternalDataSourceCache::calculateLocalPath(IRemoteFileMetadataPtr metada return fs::path(root_dir) / hashcode_str.substr(0, 3) / hashcode_str; } -std::pair> +std::pair, std::unique_ptr> ExternalDataSourceCache::createReader(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr & read_buffer) { // If something is wrong on startup, rollback to read from the original ReadBuffer @@ -201,7 +201,7 @@ ExternalDataSourceCache::createReader(ContextPtr context, IRemoteFileMetadataPtr } else { - return {cache, nullptr}; + return {std::make_unique(cache), nullptr}; } } @@ -218,7 +218,7 @@ ExternalDataSourceCache::createReader(ContextPtr context, IRemoteFileMetadataPtr return {nullptr, std::move(read_buffer)}; } new_cache->startBackgroundDownload(std::move(read_buffer), context->getSchedulePool()); - return {new_cache, nullptr}; + return {std::make_unique(new_cache), nullptr}; } } diff --git a/src/Storages/Cache/ExternalDataSourceCache.h b/src/Storages/Cache/ExternalDataSourceCache.h index 48faddd7cce..6f405ccbc7f 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.h +++ b/src/Storages/Cache/ExternalDataSourceCache.h @@ -28,12 +28,21 @@ namespace DB { +class LocalFileHolder +{ +public: + LocalFileHolder(std::shared_ptr cache_controller); + ~LocalFileHolder(); + + std::shared_ptr file_cache_controller; + std::unique_ptr file_buffer; +}; class RemoteReadBuffer : public BufferWithOwnMemory { public: explicit RemoteReadBuffer(size_t buff_size); - ~RemoteReadBuffer() override; + ~RemoteReadBuffer() override = default; static std::unique_ptr create(ContextPtr contex, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer, size_t buff_size); bool nextImpl() override; @@ -42,8 +51,7 @@ public: std::optional getTotalSize() override { return remote_file_size; } private: - std::shared_ptr file_cache_controller; - std::unique_ptr file_buffer; + std::unique_ptr local_file_holder; size_t remote_file_size = 0; }; @@ -60,7 +68,7 @@ public: inline bool isInitialized() const { return initialized; } - std::pair> + std::pair, std::unique_ptr> createReader(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr & read_buffer); void updateTotalSize(size_t size) { total_size += size; } diff --git a/src/Storages/Cache/RemoteCacheController.cpp b/src/Storages/Cache/RemoteCacheController.cpp index 8da9a01fe34..0fffaf0d5e2 100644 --- a/src/Storages/Cache/RemoteCacheController.cpp +++ b/src/Storages/Cache/RemoteCacheController.cpp @@ -203,7 +203,8 @@ void RemoteCacheController::close() { // delete directory LOG_TRACE(log, "Removing the local cache. local path: {}", local_path.string()); - std::filesystem::remove_all(local_path); + if (fs::exists(local_path)) + fs::remove_all(local_path); } std::unique_ptr RemoteCacheController::allocFile() From daa4c9e33bf19647f4be4724fb133d344bf767a7 Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Tue, 28 Dec 2021 17:17:01 +0800 Subject: [PATCH 0491/1260] clickhouse-keeper acl consistent with zookeeper, accept generated digest --- src/Coordination/KeeperStorage.cpp | 9 +++------ tests/integration/test_keeper_auth/test.py | 7 +++---- 2 files changed, 6 insertions(+), 10 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index a770451a733..a64a7d425f6 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -91,8 +91,7 @@ static bool checkACL(int32_t permission, const Coordination::ACLs & node_acls, c static bool fixupACL( const std::vector & request_acls, const std::vector & current_ids, - std::vector & result_acls, - bool hash_acls) + std::vector & result_acls) { if (request_acls.empty()) return true; @@ -125,8 +124,6 @@ static bool fixupACL( return false; valid_found = true; - if (hash_acls) - new_acl.id = generateDigest(new_acl.id); result_acls.push_back(new_acl); } } @@ -310,7 +307,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr KeeperStorage::Node created_node; Coordination::ACLs node_acls; - if (!fixupACL(request.acls, session_auth_ids, node_acls, !request.restored_from_zookeeper_log)) + if (!fixupACL(request.acls, session_auth_ids, node_acls)) { response.error = Coordination::Error::ZINVALIDACL; return {response_ptr, {}}; @@ -778,7 +775,7 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr auto & session_auth_ids = storage.session_and_auth[session_id]; Coordination::ACLs node_acls; - if (!fixupACL(request.acls, session_auth_ids, node_acls, !request.restored_from_zookeeper_log)) + if (!fixupACL(request.acls, session_auth_ids, node_acls)) { response.error = Coordination::Error::ZINVALIDACL; return {response_ptr, {}}; diff --git a/tests/integration/test_keeper_auth/test.py b/tests/integration/test_keeper_auth/test.py index 276fe3d8518..2df08cc94b7 100644 --- a/tests/integration/test_keeper_auth/test.py +++ b/tests/integration/test_keeper_auth/test.py @@ -43,12 +43,11 @@ def test_digest_auth_basic(started_cluster, get_zk): auth_connection.create("/test_no_acl", b"") auth_connection.create("/test_all_acl", b"data", acl=[make_acl("auth", "", all=True)]) - # for some reason original zookeeper accepts this ACL, but doesn't allow to do anything with this node - # even with correct credentials. - auth_connection.create("/test_all_digest_acl", b"dataX", acl=[make_acl("digest", "user1:password1", all=True)]) + # Consistent with zookeeper, accept generated digest + auth_connection.create("/test_all_digest_acl", b"dataX", acl=[make_acl("digest", "user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=", all=True)]) assert auth_connection.get("/test_all_acl")[0] == b"data" - #assert auth_connection.get("/test_all_digest_acl")[0] == b"dataX" + assert auth_connection.get("/test_all_digest_acl")[0] == b"dataX" no_auth_connection = get_zk() no_auth_connection.set("/test_no_acl", b"hello") From cb3c3970fe8ca5660d6bd9c17e5926a8c75cd15a Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 28 Dec 2021 12:40:07 +0300 Subject: [PATCH 0492/1260] Fix flaky test 01650 --- ...rt_and_deduplication_zookeeper_long.reference | 4 ++++ ...rop_part_and_deduplication_zookeeper_long.sql | 16 ++++++++++++---- 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.reference b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.reference index ebb0b033d5b..05f7d08de7d 100644 --- a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.reference +++ b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.reference @@ -1,3 +1,4 @@ +~~~~source parts~~~~~ 1 1_0_0_0 1 1_1_1_0 2 2_0_0_0 @@ -10,6 +11,7 @@ 2_ 2_1_1_0 3_ 3_0_0_0 3_ 3_1_1_0 +~~~~parts after deduplication~~~~~ 1 1_0_0_0 1 1_1_1_0 2 2_0_0_0 @@ -22,6 +24,7 @@ 2_ 2_1_1_0 3_ 3_0_0_0 3_ 3_1_1_0 +~~~~parts after drop 3_1_1_0~~~~~ 1 1_0_0_0 1 1_1_1_0 2 2_0_0_0 @@ -32,6 +35,7 @@ 2_ 2_0_0_0 2_ 2_1_1_0 3_ 3_0_0_0 +~~~~parts after new part without deduplication~~~~~ 1 1_0_0_0 1 1_1_1_0 2 2_0_0_0 diff --git a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql index 1aa568c1663..c77f29d89c2 100644 --- a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql +++ b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql @@ -17,25 +17,33 @@ SYSTEM STOP MERGES partitioned_table; INSERT INTO partitioned_table VALUES (1, 1, 'A'), (2, 2, 'B'), (3, 3, 'C'); INSERT INTO partitioned_table VALUES (11, 1, 'AA'), (22, 2, 'BB'), (33, 3, 'CC'); -SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AND database = currentDatabase() ORDER BY name; +SELECT '~~~~source parts~~~~~'; + +SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AND database = currentDatabase() and active ORDER BY name; SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/' || currentDatabase() || '/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; INSERT INTO partitioned_table VALUES (33, 3, 'CC'); -- must be deduplicated -SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AND database = currentDatabase() ORDER BY name; +SELECT '~~~~parts after deduplication~~~~~'; + +SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AND database = currentDatabase() and active ORDER BY name; SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/' || currentDatabase() || '/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; ALTER TABLE partitioned_table DROP PART '3_1_1_0'; -SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AND database = currentDatabase() ORDER BY name; +SELECT '~~~~parts after drop 3_1_1_0~~~~~'; + +SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AND database = currentDatabase() and active ORDER BY name; SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/' || currentDatabase() || '/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; INSERT INTO partitioned_table VALUES (33, 3, 'CC'); -- mustn't be deduplicated -SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AND database = currentDatabase() ORDER BY name; +SELECT '~~~~parts after new part without deduplication~~~~~'; + +SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AND database = currentDatabase() and active ORDER BY name; SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/' || currentDatabase() || '/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; From 948fde52a7c9db7684b94a8f4b5123090e1572cf Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 28 Dec 2021 13:13:51 +0300 Subject: [PATCH 0493/1260] Add retries for github api --- tests/ci/commit_status_helper.py | 32 ++++++++++++++++++++++++++------ 1 file changed, 26 insertions(+), 6 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index c420b76aaf3..8396303c5a3 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -1,13 +1,33 @@ #!/usr/bin/env python3 +import time from env_helper import GITHUB_REPOSITORY +RETRY = 5 + + +def get_commit(gh, commit_sha, retry_count=RETRY): + for i in range(retry_count): + try: + repo = gh.get_repo(GITHUB_REPOSITORY) + commit = repo.get_commit(commit_sha) + return commit + except Exception as ex: + if i == retry_count - 1: + raise ex + time.sleep(i) + + # just suppress warning + return None -def get_commit(gh, commit_sha): - repo = gh.get_repo(GITHUB_REPOSITORY) - commit = repo.get_commit(commit_sha) - return commit def post_commit_status(gh, sha, check_name, description, state, report_url): - commit = get_commit(gh, sha) - commit.create_status(context=check_name, description=description, state=state, target_url=report_url) + for i in range(RETRY): + try: + commit = get_commit(gh, sha, 1) + commit.create_status(context=check_name, description=description, state=state, target_url=report_url) + break + except Exception as ex: + if i == RETRY - 1: + raise ex + time.sleep(i) From 339443d0680c334a3a9f9ef9df3a3b970ffcfb21 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 28 Dec 2021 13:54:37 +0300 Subject: [PATCH 0494/1260] Update install.md --- docs/en/getting-started/install.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index b58303e9491..70a1b8349ff 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -142,6 +142,12 @@ On Gentoo, you can just use `emerge clickhouse` to install ClickHouse from sourc To start the server as a daemon, run: +``` bash +$ sudo clickhouse start +``` + +There are also another ways to run ClickHouse: + ``` bash $ sudo service clickhouse-server start ``` From 2f0982d38039380afdc263968960c920bdd0e5d4 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 28 Dec 2021 19:04:23 +0800 Subject: [PATCH 0495/1260] fix style --- src/Storages/Cache/ExternalDataSourceCache.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/Cache/ExternalDataSourceCache.h b/src/Storages/Cache/ExternalDataSourceCache.h index 6f405ccbc7f..4af7cbc51c8 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.h +++ b/src/Storages/Cache/ExternalDataSourceCache.h @@ -31,9 +31,9 @@ namespace DB class LocalFileHolder { public: - LocalFileHolder(std::shared_ptr cache_controller); + explicit LocalFileHolder(std::shared_ptr cache_controller); ~LocalFileHolder(); - + std::shared_ptr file_cache_controller; std::unique_ptr file_buffer; }; From e9b4439a47c4a914e2e42f1cd5b7f333fd10d33b Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 28 Dec 2021 15:08:40 +0300 Subject: [PATCH 0496/1260] Fix --- src/Storages/ExternalDataSourceConfiguration.cpp | 7 ++++--- tests/integration/test_dictionaries_postgresql/test.py | 4 ++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 215f424c839..5a93b688da6 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -172,8 +172,9 @@ static void validateConfigKeys( dict_config.keys(config_prefix, config_keys); for (const auto & config_key : config_keys) { - if (!allowed_keys.contains(config_key)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected key `{}` in dictionary source configuration", config_key); + if (allowed_keys.contains(config_key) || config_key.starts_with("replica")) + continue; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected key `{}` in dictionary source configuration", config_key); } } @@ -205,7 +206,7 @@ std::optional getExternalDataSourceConfiguratio if (configuration.host.empty() || configuration.port == 0 || configuration.username.empty() || configuration.table.empty()) { throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Named collection of connection parameters is missing some of the parameters and dictionary parameters are added"); + "Named collection of connection parameters is missing some of the parameters and dictionary parameters are not added"); } return configuration; } diff --git a/tests/integration/test_dictionaries_postgresql/test.py b/tests/integration/test_dictionaries_postgresql/test.py index b3914ea3a38..ce295e11586 100644 --- a/tests/integration/test_dictionaries_postgresql/test.py +++ b/tests/integration/test_dictionaries_postgresql/test.py @@ -388,8 +388,8 @@ def test_bad_configuration(started_cluster): LAYOUT(HASHED()); ''') - result = node1.query_and_get_error("SELECT dictGetUInt32(postgres_dict, 'value', toUInt64(1))") - assert 'Unexpected key in config: dbbb' in result + node1.query_and_get_error("SELECT dictGetUInt32(postgres_dict, 'value', toUInt64(1))") + assert node1.contains_in_log('Unexpected key `dbbb`') if __name__ == '__main__': From 333dfab6a62dbd37cce911526576c90852b7e4e0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 28 Dec 2021 16:12:51 +0300 Subject: [PATCH 0497/1260] Add retries to AST fuzzer --- docker/test/fuzzer/run-fuzzer.sh | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 844a7396134..1ebaed752a6 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -52,9 +52,21 @@ function clone } +function wget_with_retry +{ + for _ in 1 2 3 4; do + if wget -nv -nd -c "$1";then + return 0 + else + sleep 0.5 + fi + done + return 1 +} + function download { - wget -nv -nd -c "$BINARY_URL_TO_DOWNLOAD" + wget_with_retry "$BINARY_URL_TO_DOWNLOAD" chmod +x clickhouse ln -s ./clickhouse ./clickhouse-server From 5ae8188ab754cf030cb2ce128132d4dfff230945 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 28 Dec 2021 16:26:47 +0300 Subject: [PATCH 0498/1260] Fix flaky test --- .../0_stateless/02122_4letter_words_stress_zookeeper.sh | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh b/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh index 4a94beddbba..50b3ec92f5e 100755 --- a/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh +++ b/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh @@ -43,5 +43,12 @@ timeout $TIMEOUT bash -c create_drop_thread 2> /dev/null & wait +for i in $(seq 1 10); do + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_table$num" 2>/dev/null + while [ $? -ne 0 ]; do + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_table$num" 2>/dev/null + done +done + # still alive $CLICKHOUSE_CLIENT --query "SELECT 1" From a2aa32877f2efc5060fe701293a96f99ae4c8a52 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 28 Dec 2021 16:40:34 +0300 Subject: [PATCH 0499/1260] Update 02122_4letter_words_stress_zookeeper.sh --- .../0_stateless/02122_4letter_words_stress_zookeeper.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh b/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh index 50b3ec92f5e..2deaf788ecf 100755 --- a/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh +++ b/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh @@ -18,7 +18,7 @@ function four_letter_thread() function create_drop_thread() { while true; do - num=$RANDOM + num=$(($RANDOM % 10 + 1)) $CLICKHOUSE_CLIENT --query "CREATE TABLE test_table$num (key UInt64, value1 UInt8, value2 UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_table$num', '0') ORDER BY key" sleep 0.$RANDOM $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_table$num" @@ -43,7 +43,7 @@ timeout $TIMEOUT bash -c create_drop_thread 2> /dev/null & wait -for i in $(seq 1 10); do +for num in $(seq 1 10); do $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_table$num" 2>/dev/null while [ $? -ne 0 ]; do $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_table$num" 2>/dev/null From ee3a9e48f5e7213ddd7dfe9215601c08336ec805 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 28 Dec 2021 16:49:41 +0300 Subject: [PATCH 0500/1260] Add benchmark results from Lorenzo --- website/benchmark/hardware/index.html | 1 + .../hardware/results/cavium_4core.json | 54 +++++++++++++++++++ .../benchmark/hardware/results/ssdnodes.json | 54 +++++++++++++++++++ 3 files changed, 109 insertions(+) create mode 100644 website/benchmark/hardware/results/cavium_4core.json create mode 100644 website/benchmark/hardware/results/ssdnodes.json diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index d12a9dd9057..c6b1e2be275 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -84,6 +84,7 @@ Results for GCP are from Vy Nguyen Tan.
Results for ThinkPad P15 are from Mikhail Shiryaev.
Results for RockPi4 are from Kirill Zholnay.
Results for Xeon 6266C are from David in Shanghai.
+Results for SSDNodes and Cavium are from Lorenzo QXIP.

diff --git a/website/benchmark/hardware/results/cavium_4core.json b/website/benchmark/hardware/results/cavium_4core.json new file mode 100644 index 00000000000..a7cb96b2cd3 --- /dev/null +++ b/website/benchmark/hardware/results/cavium_4core.json @@ -0,0 +1,54 @@ +[ + { + "system": "Cavium ARM64 CPU (4 Core, 1.5 GHz, NVMe SSD)", + "system_full": "Cavium ARM64 CPU (4 Corem 1.5 GHz, NVMe SSD), 16 GiB", + "time": "2021-12-27 00:00:00", + "kind": "server", + "result": + [ +[0.004, 0.004, 0.004], +[0.196, 0.178, 0.180], +[0.495, 0.437, 0.426], +[0.715, 0.499, 0.499], +[0.992, 0.798, 0.795], +[3.958, 3.750, 3.751], +[0.288, 0.274, 0.273], +[0.236, 0.231, 0.239], +[3.129, 2.936, 2.918], +[4.221, 3.924, 3.934], +[2.395, 2.285, 2.226], +[2.832, 2.703, 2.644], +[6.510, 6.301, 6.262], +[7.933, 7.669, 7.704], +[7.397, 7.122, 7.146], +[4.692, 4.537, 4.540], +[15.194, 14.835, 15.051], +[10.446, 10.036, 10.072], +[26.472, 25.655, 25.809], +[0.879, 0.669, 0.694], +[14.614, 13.755, 13.726], +[16.876, 15.675, 15.703], +[34.715, 33.204, 33.250], +[18.850, 15.387, 15.332], +[4.455, 4.025, 4.016], +[3.667, 3.415, 3.457], +[4.507, 4.057, 4.049], +[14.344, 13.394, 13.390], +[17.519, 17.052, 17.067], +[8.606, 8.611, 8.545], +[6.936, 6.491, 6.496], +[10.020, 9.260, 9.233], +[39.793, 39.631, 39.553], +[30.310, 29.604, 29.572], +[30.485, 29.557, 29.649], +[8.539, 8.337, 8.342], +[0.931, 0.912, 0.912], +[0.523, 0.516, 0.507], +[0.460, 0.448, 0.450], +[1.880, 1.817, 1.884], +[0.141, 0.119, 0.117], +[0.116, 0.095, 0.092], +[0.021, 0.017, 0.014] + ] + } +] diff --git a/website/benchmark/hardware/results/ssdnodes.json b/website/benchmark/hardware/results/ssdnodes.json new file mode 100644 index 00000000000..623f4b49687 --- /dev/null +++ b/website/benchmark/hardware/results/ssdnodes.json @@ -0,0 +1,54 @@ +[ + { + "system": "SSDNodes G6", + "system_full": "G6 Performance+ 48GB RAM, 720GB NVMe, 12x Intel Silver vCPU, KVM", + "time": "2021-12-27 00:00:00", + "kind": "cloud", + "result": + [ +[0.002, 0.002, 0.002], +[0.021, 0.017, 0.017], +[0.053, 0.034, 0.039], +[0.090, 0.053, 0.047], +[0.146, 0.123, 0.117], +[0.358, 0.325, 0.323], +[0.025, 0.020, 0.021], +[0.042, 0.015, 0.014], +[0.566, 0.511, 0.524], +[0.704, 0.626, 0.591], +[0.229, 0.174, 0.194], +[0.255, 0.210, 0.206], +[0.849, 0.725, 0.701], +[0.984, 0.907, 0.948], +[0.952, 0.886, 0.899], +[0.772, 0.741, 0.738], +[2.945, 2.667, 2.703], +[1.645, 1.646, 1.576], +[5.342, 5.042, 5.306], +[0.088, 0.052, 0.051], +[1.176, 0.825, 0.839], +[1.261, 1.001, 0.933], +[2.977, 2.190, 2.193], +[1.872, 0.991, 0.956], +[0.368, 0.264, 0.275], +[0.300, 0.247, 0.241], +[0.329, 0.272, 0.277], +[1.124, 0.870, 0.824], +[1.545, 1.270, 1.281], +[1.478, 1.399, 1.463], +[0.809, 0.696, 0.677], +[1.095, 0.875, 0.832], +[5.164, 4.841, 4.613], +[3.859, 3.435, 3.396], +[4.054, 3.479, 3.496], +[1.325, 1.274, 1.294], +[0.261, 0.248, 0.266], +[0.102, 0.096, 0.104], +[0.102, 0.090, 0.094], +[0.600, 0.550, 0.566], +[0.041, 0.031, 0.028], +[0.029, 0.021, 0.025], +[0.007, 0.006, 0.005] + ] + } +] From 4956981e36b64042f29d1929363e46b1bd624cb7 Mon Sep 17 00:00:00 2001 From: pdv-ru <86398979+pdv-ru@users.noreply.github.com> Date: Tue, 28 Dec 2021 16:51:59 +0300 Subject: [PATCH 0501/1260] fix anchors --- docs/en/operations/clickhouse-keeper.md | 10 +++++----- docs/ru/operations/clickhouse-keeper.md | 10 +++++----- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 7c13448cf97..350ca835187 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -3,14 +3,14 @@ toc_priority: 66 toc_title: ClickHouse Keeper --- -# [pre-production] ClickHouse Keeper {#clickHouse_keeper} +# [pre-production] ClickHouse Keeper {#clickHouse-keeper} ClickHouse server uses [ZooKeeper](https://zookeeper.apache.org/) coordination system for data [replication](../engines/table-engines/mergetree-family/replication.md) and [distributed DDL](../sql-reference/distributed-ddl.md) queries execution. ClickHouse Keeper is an alternative coordination system compatible with ZooKeeper. !!! warning "Warning" This feature is currently in the pre-production stage. We test it in our CI and on small internal installations. -## Implementation details {#implementation_details} +## Implementation details {#implementation-details} ZooKeeper is one of the first well-known open-source coordination systems. It's implemented in Java, has quite a simple and powerful data model. ZooKeeper's coordination algorithm called ZAB (ZooKeeper Atomic Broadcast) doesn't provide linearizability guarantees for reads, because each ZooKeeper node serves reads locally. Unlike ZooKeeper ClickHouse Keeper is written in C++ and uses [RAFT algorithm](https://raft.github.io/) [implementation](https://github.com/eBay/NuRaft). This algorithm allows to have linearizability for reads and writes, has several open-source implementations in different languages. @@ -102,7 +102,7 @@ Examples of configuration for quorum with three nodes can be found in [integrati
``` -## How to run {#how_to_run} +## How to run {#how-to-run} ClickHouse Keeper is bundled into the ClickHouse server package, just add configuration of `` and start ClickHouse server as always. If you want to run standalone ClickHouse Keeper you can start it in a similar way with: @@ -110,7 +110,7 @@ ClickHouse Keeper is bundled into the ClickHouse server package, just add config clickhouse-keeper --config /etc/your_path_to_config/config.xml --daemon ``` -## Four Letter Word Commands {#four_letter_word_commands} +## Four Letter Word Commands {#four-letter-word-commands} ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as `mntr`, `stat` etc. There are some more interesting commands: `stat` gives some general information about the server and connected clients, while `srvr` and `cons` give extended details on server and connections respectively. @@ -297,7 +297,7 @@ Sessions with Ephemerals (1): /clickhouse/task_queue/ddl ``` -## [experimental] Migration from ZooKeeper {#migration_from_zookeeper} +## [experimental] Migration from ZooKeeper {#migration-from-zookeeper} Seamlessly migration from ZooKeeper to ClickHouse Keeper is impossible you have to stop your ZooKeeper cluster, convert data and start ClickHouse Keeper. `clickhouse-keeper-converter` tool allows converting ZooKeeper logs and snapshots to ClickHouse Keeper snapshot. It works only with ZooKeeper > 3.4. Steps for migration: diff --git a/docs/ru/operations/clickhouse-keeper.md b/docs/ru/operations/clickhouse-keeper.md index 4a80651f879..05784b2b942 100644 --- a/docs/ru/operations/clickhouse-keeper.md +++ b/docs/ru/operations/clickhouse-keeper.md @@ -3,14 +3,14 @@ toc_priority: 66 toc_title: ClickHouse Keeper --- -# [пре-продакшн] ClickHouse Keeper {#clickHouse_keeper} +# [пре-продакшн] ClickHouse Keeper {#clickHouse-keeper} Сервер ClickHouse использует сервис координации [ZooKeeper](https://zookeeper.apache.org/) для [репликации](../engines/table-engines/mergetree-family/replication.md) данных и выполнения [распределенных DDL запросов](../sql-reference/distributed-ddl.md). ClickHouse Keeper — это альтернативный сервис координации, совместимый с ZooKeeper. !!! warning "Предупреждение" ClickHouse Keeper находится в стадии пре-продакшн и тестируется в CI ClickHouse и на нескольких внутренних инсталляциях. -## Детали реализации {#implementation_details} +## Детали реализации {#implementation-details} ZooKeeper — один из первых широко известных сервисов координации с открытым исходным кодом. Он реализован на языке программирования Java, имеет достаточно простую и мощную модель данных. Алгоритм координации Zookeeper называется ZAB (ZooKeeper Atomic Broadcast). Он не гарантирует линеаризуемость операций чтения, поскольку каждый узел ZooKeeper обслуживает чтения локально. В отличие от ZooKeeper, ClickHouse Keeper реализован на C++ и использует алгоритм [RAFT](https://raft.github.io/), [реализация](https://github.com/eBay/NuRaft). Этот алгоритм позволяет достичь линеаризуемости чтения и записи, имеет несколько реализаций с открытым исходным кодом на разных языках. @@ -102,7 +102,7 @@ ClickHouse Keeper может использоваться как равноце ``` -## Как запустить {#how_to_run} +## Как запустить {#how-to-run} ClickHouse Keeper входит в пакет `clickhouse-server`, просто добавьте кофигурацию `` и запустите сервер ClickHouse как обычно. Если вы хотите запустить ClickHouse Keeper автономно, сделайте это аналогичным способом: @@ -110,7 +110,7 @@ ClickHouse Keeper входит в пакет `clickhouse-server`, просто clickhouse-keeper --config /etc/your_path_to_config/config.xml --daemon ``` -## 4-х буквенные команды {#four_letter_word_commands} +## 4-х буквенные команды {#four-letter-word-commands} ClickHouse Keeper также поддерживает 4-х буквенные команды, почти такие же, как у Zookeeper. Каждая команда состоит из 4-х символов, например, `mntr`, `stat` и т. д. Несколько интересных команд: `stat` предоставляет общую информацию о сервере и подключенных клиентах, а `srvr` и `cons` предоставляют расширенные сведения о сервере и подключениях соответственно. @@ -298,7 +298,7 @@ Sessions with Ephemerals (1): ``` -## [экспериментально] Переход с ZooKeeper {#migration_from_zookeeper} +## [экспериментально] Переход с ZooKeeper {#migration-from-zookeeper} Плавный переход с ZooKeeper на ClickHouse Keeper невозможен, необходимо остановить кластер ZooKeeper, преобразовать данные и запустить ClickHouse Keeper. Утилита `clickhouse-keeper-converter` конвертирует журналы и снэпшоты ZooKeeper в снэпшот ClickHouse Keeper. Работа утилиты проверена только для версий ZooKeeper выше 3.4. Для миграции необходимо выполнить следующие шаги: From a6e90a046d58d5ebc3ac28b384ef82cd6328ea2c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 28 Dec 2021 17:47:19 +0300 Subject: [PATCH 0502/1260] add more info to query log in async inserts --- src/Interpreters/InterpreterInsertQuery.cpp | 7 ++++++- src/Interpreters/InterpreterInsertQuery.h | 1 + src/Interpreters/executeQuery.cpp | 20 ++++++++++++++++--- .../02156_async_insert_query_log.reference | 4 ++-- .../02156_async_insert_query_log.sh | 5 +++-- 5 files changed, 29 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 8677cf59d79..a1f83c81a81 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -466,7 +466,7 @@ StorageID InterpreterInsertQuery::getDatabaseTable() const } -void InterpreterInsertQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr context_) const +void InterpreterInsertQuery::extendQueryLogElemImpl(QueryLogElement & elem, ContextPtr context_) { elem.query_kind = "Insert"; const auto & insert_table = context_->getInsertionTable(); @@ -477,4 +477,9 @@ void InterpreterInsertQuery::extendQueryLogElemImpl(QueryLogElement & elem, cons } } +void InterpreterInsertQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr context_) const +{ + extendQueryLogElemImpl(elem, context_); +} + } diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index e5733a8c28b..93de92a0680 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -40,6 +40,7 @@ public: ThreadStatus * thread_status = nullptr, std::atomic_uint64_t * elapsed_counter_ms = nullptr); + static void extendQueryLogElemImpl(QueryLogElement & elem, ContextPtr context_); void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context_) const override; StoragePtr getTable(ASTInsertQuery & query); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index b192416f5bd..bf7b5d86d90 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -556,9 +556,14 @@ static std::tuple executeQueryImpl( auto * insert_query = ast->as(); - if (insert_query && insert_query->table_id) - /// Resolve database before trying to use async insert feature - to properly hash the query. + /// Resolve database before trying to use async insert feature - to properly hash the query. + if (insert_query) + { + if (!insert_query->table_id) + insert_query->table_id = StorageID{insert_query->getDatabase(), insert_query->getTable()}; + insert_query->table_id = context->resolveStorageID(insert_query->table_id); + } if (insert_query && insert_query->select) { @@ -607,7 +612,14 @@ static std::tuple executeQueryImpl( quota = context->getQuota(); if (quota) + { quota->used(QuotaType::QUERY_INSERTS, 1); + quota->used(QuotaType::QUERIES, 1); + } + + const auto & table_id = insert_query->table_id; + if (!table_id.empty()) + context->setInsertionTable(table_id); } else { @@ -719,7 +731,9 @@ static std::tuple executeQueryImpl( elem.query_views = info.views; } - if (interpreter) + if (async_insert) + InterpreterInsertQuery::extendQueryLogElemImpl(elem, context); + else if (interpreter) interpreter->extendQueryLogElem(elem, ast, context, query_database, query_table); if (settings.log_query_settings) diff --git a/tests/queries/0_stateless/02156_async_insert_query_log.reference b/tests/queries/0_stateless/02156_async_insert_query_log.reference index 15fad7dd0ab..404dbfe753d 100644 --- a/tests/queries/0_stateless/02156_async_insert_query_log.reference +++ b/tests/queries/0_stateless/02156_async_insert_query_log.reference @@ -1,4 +1,4 @@ 1 a 2 b -INSERT INTO async_inserts_2156 VALUES 1 0 -INSERT INTO async_inserts_2156 VALUES 1 +INSERT INTO async_inserts_2156 VALUES 1 Insert 1 0 +INSERT INTO async_inserts_2156 VALUES 1 Insert 1 diff --git a/tests/queries/0_stateless/02156_async_insert_query_log.sh b/tests/queries/0_stateless/02156_async_insert_query_log.sh index e5508273dbb..4eb16f26def 100755 --- a/tests/queries/0_stateless/02156_async_insert_query_log.sh +++ b/tests/queries/0_stateless/02156_async_insert_query_log.sh @@ -17,8 +17,9 @@ ${CLICKHOUSE_CLIENT} -q "SELECT * FROM async_inserts_2156 ORDER BY id" ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "SELECT query, Settings['async_insert'], Settings['wait_for_async_insert'] FROM system.query_log \ - WHERE event_date = today() AND current_database = '$CLICKHOUSE_DATABASE' \ +${CLICKHOUSE_CLIENT} -q "SELECT query, arrayExists(x -> x LIKE '%async_inserts_2156', tables), \ + query_kind, Settings['async_insert'], Settings['wait_for_async_insert'] FROM system.query_log \ + WHERE event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' \ AND query ILIKE 'INSERT INTO async_inserts_2156 VALUES%' AND type = 'QueryFinish' \ ORDER BY query_start_time_microseconds" From b668691f6a7cd6cd05d61ec84f547e8e247a4ac8 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 29 Dec 2021 01:06:21 +0800 Subject: [PATCH 0503/1260] Fix broken tests --- .../00907_set_index_with_nullable_and_low_cardinality_bug.sql | 2 +- tests/queries/0_stateless/00945_bloom_filter_index.sql | 2 +- tests/queries/0_stateless/01045_bloom_filter_null_array.sql | 2 +- tests/queries/0_stateless/01414_low_cardinality_nullable.sql | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00907_set_index_with_nullable_and_low_cardinality_bug.sql b/tests/queries/0_stateless/00907_set_index_with_nullable_and_low_cardinality_bug.sql index 3a55a69c726..336d9984e69 100644 --- a/tests/queries/0_stateless/00907_set_index_with_nullable_and_low_cardinality_bug.sql +++ b/tests/queries/0_stateless/00907_set_index_with_nullable_and_low_cardinality_bug.sql @@ -8,7 +8,7 @@ CREATE TABLE null_lc_set_index ( INDEX test_user_idx (user) TYPE set(0) GRANULARITY 8192 ) ENGINE=MergeTree PARTITION BY toYYYYMMDD(timestamp) - ORDER BY (timestamp, action, cityHash64(user)); + ORDER BY (timestamp, action, cityHash64(user)) SETTINGS allow_nullable_key = 1; INSERT INTO null_lc_set_index VALUES (1550883010, 'subscribe', 'alice'); INSERT INTO null_lc_set_index VALUES (1550883020, 'follow', 'bob'); diff --git a/tests/queries/0_stateless/00945_bloom_filter_index.sql b/tests/queries/0_stateless/00945_bloom_filter_index.sql index f45c4c04290..33fd0393676 100644 --- a/tests/queries/0_stateless/00945_bloom_filter_index.sql +++ b/tests/queries/0_stateless/00945_bloom_filter_index.sql @@ -183,7 +183,7 @@ CREATE TABLE bloom_filter_array_lc_null_types_test ( fixed_string Array(LowCardinality(Nullable(FixedString(5)))), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) -ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6; +ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6, allow_nullable_key = 1; INSERT INTO bloom_filter_array_lc_null_types_test SELECT groupArray(number) AS order_key, diff --git a/tests/queries/0_stateless/01045_bloom_filter_null_array.sql b/tests/queries/0_stateless/01045_bloom_filter_null_array.sql index 3dfc04ae8ff..4a5741b4e72 100644 --- a/tests/queries/0_stateless/01045_bloom_filter_null_array.sql +++ b/tests/queries/0_stateless/01045_bloom_filter_null_array.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS bloom_filter_null_array; -CREATE TABLE bloom_filter_null_array (v Array(LowCardinality(Nullable(String))), INDEX idx v TYPE bloom_filter(0.1) GRANULARITY 1) ENGINE = MergeTree() ORDER BY v; +CREATE TABLE bloom_filter_null_array (v Array(LowCardinality(Nullable(String))), INDEX idx v TYPE bloom_filter(0.1) GRANULARITY 1) ENGINE = MergeTree() ORDER BY v SETTINGS allow_nullable_key = 1; INSERT INTO bloom_filter_null_array VALUES ([]); INSERT INTO bloom_filter_null_array VALUES (['1', '2']) ([]) ([]); diff --git a/tests/queries/0_stateless/01414_low_cardinality_nullable.sql b/tests/queries/0_stateless/01414_low_cardinality_nullable.sql index 596e90adfd6..c11e990cea8 100644 --- a/tests/queries/0_stateless/01414_low_cardinality_nullable.sql +++ b/tests/queries/0_stateless/01414_low_cardinality_nullable.sql @@ -19,7 +19,7 @@ CREATE TABLE lc_nullable ( str Array(LowCardinality(Nullable(String))), fixed_string Array(LowCardinality(Nullable(FixedString(5)))) -) ENGINE = MergeTree() ORDER BY order_key; +) ENGINE = MergeTree() ORDER BY order_key SETTINGS allow_nullable_key = 1; INSERT INTO lc_nullable SELECT groupArray(number) AS order_key, From a1dfebd7282de08108aa89c8b6adbb6c295986d1 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 27 Dec 2021 21:04:42 +0800 Subject: [PATCH 0504/1260] More nullable primary key tests. --- .../01410_nullable_key_more_tests.reference | 0 .../01410_nullable_key_more_tests.sh | 33 +++++++++++++++++++ 2 files changed, 33 insertions(+) create mode 100644 tests/queries/0_stateless/01410_nullable_key_more_tests.reference create mode 100755 tests/queries/0_stateless/01410_nullable_key_more_tests.sh diff --git a/tests/queries/0_stateless/01410_nullable_key_more_tests.reference b/tests/queries/0_stateless/01410_nullable_key_more_tests.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01410_nullable_key_more_tests.sh b/tests/queries/0_stateless/01410_nullable_key_more_tests.sh new file mode 100755 index 00000000000..03bebed324b --- /dev/null +++ b/tests/queries/0_stateless/01410_nullable_key_more_tests.sh @@ -0,0 +1,33 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +test_func() +{ + engine=$1 + + curl -d@- -sS "${CLICKHOUSE_URL}" <<< "drop table if exists table_with_nullable_keys" + + curl -d@- -sS "${CLICKHOUSE_URL}" <<< "create table table_with_nullable_keys (nullable_int Nullable(UInt32), nullable_str Nullable(String), nullable_lc LowCardinality(Nullable(String)), nullable_ints Array(Nullable(UInt32)), nullable_misc Tuple(Nullable(String), Nullable(UInt32)), nullable_val Map(UInt32, Nullable(String)), value UInt8) engine $engine order by (nullable_int, nullable_str, nullable_lc, nullable_ints, nullable_misc, nullable_val) settings allow_nullable_key = 1, index_granularity = 1" + + curl -d@- -sS "${CLICKHOUSE_URL}" <<< "insert into table_with_nullable_keys select * replace (cast(nullable_val as Map(UInt32, Nullable(String))) as nullable_val) from generateRandom('nullable_int Nullable(UInt32), nullable_str Nullable(String), nullable_lc Nullable(String), nullable_ints Array(Nullable(UInt32)), nullable_misc Tuple(Nullable(String), Nullable(UInt32)), nullable_val Array(Tuple(UInt32, Nullable(String))), value UInt8', 1, 30, 30) limit 1024" + + curl -d@- -sS "${CLICKHOUSE_URL}" <<< "select * from table_with_nullable_keys where nullable_str = (select randomPrintableASCII(30)) or nullable_str in (select randomPrintableASCII(30) from numbers(3)) format Null" + + curl -d@- -sS "${CLICKHOUSE_URL}" <<< "select * from table_with_nullable_keys where nullable_lc = (select randomPrintableASCII(30)) or nullable_lc in (select randomPrintableASCII(30) from numbers(3)) format Null" + + curl -d@- -sS "${CLICKHOUSE_URL}" <<< "select * from table_with_nullable_keys where nullable_ints = [1, 2, null] or nullable_ints in (select * from generateRandom('nullable_ints Array(Nullable(UInt32))', 1, 30, 30) limit 3) format Null" + + curl -d@- -sS "${CLICKHOUSE_URL}" <<< "select * from table_with_nullable_keys where nullable_misc = (select (randomPrintableASCII(30), rand())) or nullable_misc in (select arrayJoin([(randomPrintableASCII(30), null), (null, rand())]))" + + curl -d@- -sS "${CLICKHOUSE_URL}" <<< "select * from table_with_nullable_keys where nullable_val = (select map(rand(), randomPrintableASCII(10), rand(2), randomPrintableASCII(20), rand(3), null)) or nullable_val in (select cast(nullable_ints as Map(UInt32, Nullable(String))) from generateRandom('nullable_ints Array(Tuple(UInt32, Nullable(String)))', 1, 30, 30) limit 3) format Null" + + curl -d@- -sS "${CLICKHOUSE_URL}" <<< "drop table table_with_nullable_keys" +} + +test_func MergeTree +test_func AggregatingMergeTree +test_func ReplacingMergeTree From 6e70d690c8173c3eb3947f42ffc47718e6029699 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 28 Dec 2021 14:12:13 -0400 Subject: [PATCH 0505/1260] Update grant.md --- docs/ru/sql-reference/statements/grant.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/grant.md b/docs/ru/sql-reference/statements/grant.md index c970d4d24f3..887aa98a19e 100644 --- a/docs/ru/sql-reference/statements/grant.md +++ b/docs/ru/sql-reference/statements/grant.md @@ -21,7 +21,7 @@ GRANT [ON CLUSTER cluster_name] privilege[(column_name [,...])] [,...] ON {db.ta - `user` — Пользователь ClickHouse. `WITH GRANT OPTION` разрешает пользователю или роли выполнять запрос `GRANT`. Пользователь может выдавать только те привилегии, которые есть у него, той же или меньшей области действий. -`WITH REPLACE OPTION` заменяет все старые привилегии новыми привилегиями для `user` или `role`, Если не указано, добавьте новые привилегии для старых. +`WITH REPLACE OPTION` заменяет все старые привилегии новыми привилегиями для `user` или `role`, если не указано, добавляет новые привилегии. ## Синтаксис назначения ролей {#assign-role-syntax} From 035d757cc1059d4246addf86b367337b33af452a Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 28 Dec 2021 14:13:41 -0400 Subject: [PATCH 0506/1260] Update grant.md --- docs/en/sql-reference/statements/grant.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index f04952746a6..2b1262f7d3c 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -21,7 +21,7 @@ GRANT [ON CLUSTER cluster_name] privilege[(column_name [,...])] [,...] ON {db.ta - `user` — ClickHouse user account. The `WITH GRANT OPTION` clause grants `user` or `role` with permission to execute the `GRANT` query. Users can grant privileges of the same scope they have and less. -The `WITH REPLACE OPTION` clause replace old privileges by new privileges for the `user` or `role`, if not specified it is append privileges. +The `WITH REPLACE OPTION` clause replace old privileges by new privileges for the `user` or `role`, if is not specified it appends privileges. ## Assigning Role Syntax {#assign-role-syntax} @@ -33,7 +33,7 @@ GRANT [ON CLUSTER cluster_name] role [,...] TO {user | another_role | CURRENT_US - `user` — ClickHouse user account. The `WITH ADMIN OPTION` clause grants [ADMIN OPTION](#admin-option-privilege) privilege to `user` or `role`. -The `WITH REPLACE OPTION` clause replace old roles by new role for the `user` or `role`, if not specified it is append roles. +The `WITH REPLACE OPTION` clause replace old roles by new role for the `user` or `role`, if is not specified it appends roles. ## Usage {#grant-usage} From 9de3fd082e3f51d0e89564f2a734285e5b3b1a0f Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 28 Dec 2021 14:14:32 -0400 Subject: [PATCH 0507/1260] Update grant.md --- docs/ru/sql-reference/statements/grant.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/grant.md b/docs/ru/sql-reference/statements/grant.md index 887aa98a19e..06432193f9f 100644 --- a/docs/ru/sql-reference/statements/grant.md +++ b/docs/ru/sql-reference/statements/grant.md @@ -34,7 +34,7 @@ GRANT [ON CLUSTER cluster_name] role [,...] TO {user | another_role | CURRENT_US - `user` — Пользователь ClickHouse. `WITH ADMIN OPTION` присваивает привилегию [ADMIN OPTION](#admin-option-privilege) пользователю или роли. -`WITH REPLACE OPTION` заменяет все старые роли новыми ролями для пользователя `user` или `role`, Если не указано, добавьте новые роли в старые. +`WITH REPLACE OPTION` заменяет все старые роли новыми ролями для пользователя `user` или `role`, если не указано, добавляет новые новые роли. ## Использование {#grant-usage} From 67c5b41a58d18d842bb3d22581438958e70ea44d Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 28 Dec 2021 22:16:16 +0300 Subject: [PATCH 0508/1260] Fix --- .../integration/test_storage_rabbitmq/test.py | 24 ++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 2c2a9e41509..c5074156d88 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -35,6 +35,28 @@ def rabbitmq_check_result(result, check=False, ref_file='test_rabbitmq_json.refe else: return TSV(result) == TSV(reference) +def check_rabbitmq_is_available(rabbitmq_id): + p = subprocess.Popen(('docker', + 'exec', + '-i', + rabbitmq_id, + 'rabbitmqctl', + 'await_startup'), + stdout=subprocess.PIPE) + p.communicate() + return p.returncode == 0 + +def wait_rabbitmq_to_start(rabbitmq_docker_id, timeout=180): + start = time.time() + while time.time() - start < timeout: + try: + if check_rabbitmq_is_available(rabbitmq_docker_id): + logging.debug("RabbitMQ is available") + return + time.sleep(0.5) + except Exception as ex: + logging.debug("Can't connect to RabbitMQ " + str(ex)) + time.sleep(0.5) def kill_rabbitmq(rabbitmq_id): p = subprocess.Popen(('docker', 'stop', rabbitmq_id), stdout=subprocess.PIPE) @@ -45,7 +67,7 @@ def kill_rabbitmq(rabbitmq_id): def revive_rabbitmq(rabbitmq_id): p = subprocess.Popen(('docker', 'start', rabbitmq_id), stdout=subprocess.PIPE) p.communicate() - return p.returncode == 0 + wait_rabbitmq_to_start(rabbitmq_id) # Fixtures From eb4e400c54e41d2accc0c374ab5de51ede968f2b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 1 Nov 2021 14:22:21 +0300 Subject: [PATCH 0509/1260] Executable refactoring --- .../ExecutableDictionarySource.cpp | 60 ++-- src/Dictionaries/ExecutableDictionarySource.h | 6 +- .../ExecutablePoolDictionarySource.cpp | 69 ++-- .../ExecutablePoolDictionarySource.h | 11 +- ...alUserDefinedExecutableFunctionsLoader.cpp | 38 ++- .../UserDefinedExecutableFunction.cpp | 8 +- .../UserDefinedExecutableFunction.h | 31 +- .../UserDefinedExecutableFunctionFactory.cpp | 73 +---- src/Processors/Sources/ShellCommandSource.cpp | 300 ++++++++++++++++++ src/Processors/Sources/ShellCommandSource.h | 190 ++++------- src/Storages/ExecutableSettings.h | 7 +- src/Storages/StorageExecutable.cpp | 178 +++-------- src/Storages/StorageExecutable.h | 26 +- .../TableFunctionExecutable.cpp | 7 +- 14 files changed, 513 insertions(+), 491 deletions(-) create mode 100644 src/Processors/Sources/ShellCommandSource.cpp diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 8d10a6665cf..a4837027508 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -5,6 +5,7 @@ #include #include +#include #include #include @@ -31,11 +32,13 @@ ExecutableDictionarySource::ExecutableDictionarySource( const DictionaryStructure & dict_struct_, const Configuration & configuration_, Block & sample_block_, + std::shared_ptr coordinator_, ContextPtr context_) : log(&Poco::Logger::get("ExecutableDictionarySource")) , dict_struct(dict_struct_) , configuration(configuration_) - , sample_block{sample_block_} + , sample_block(sample_block_) + , coordinator(std::move(coordinator_)) , context(context_) { /// Remove keys from sample_block for implicit_key dictionary because @@ -58,6 +61,7 @@ ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionar , dict_struct(other.dict_struct) , configuration(other.configuration) , sample_block(other.sample_block) + , coordinator(other.coordinator) , context(Context::createCopy(other.context)) { } @@ -69,11 +73,7 @@ Pipe ExecutableDictionarySource::loadAll() LOG_TRACE(log, "loadAll {}", toString()); - ShellCommand::Config config(configuration.command); - auto process = ShellCommand::execute(config); - - Pipe pipe(std::make_unique(context, configuration.format, sample_block, std::move(process))); - return pipe; + return coordinator->createPipe(configuration.command, sample_block, context); } Pipe ExecutableDictionarySource::loadUpdatedAll() @@ -89,10 +89,7 @@ Pipe ExecutableDictionarySource::loadUpdatedAll() command_with_update_field += " " + configuration.update_field + " " + DB::toString(LocalDateTime(update_time - configuration.update_lag)); LOG_TRACE(log, "loadUpdatedAll {}", command_with_update_field); - ShellCommand::Config config(command_with_update_field); - auto process = ShellCommand::execute(config); - Pipe pipe(std::make_unique(context, configuration.format, sample_block, std::move(process))); - return pipe; + return coordinator->createPipe(command_with_update_field, sample_block, context); } Pipe ExecutableDictionarySource::loadIds(const std::vector & ids) @@ -113,32 +110,13 @@ Pipe ExecutableDictionarySource::loadKeys(const Columns & key_columns, const std Pipe ExecutableDictionarySource::getStreamForBlock(const Block & block) { - ShellCommand::Config config(configuration.command); - auto process = ShellCommand::execute(config); - auto * process_in = &process->in; + auto source = std::make_shared(block); + auto shell_input_pipe = Pipe(std::move(source)); - ShellCommandSource::SendDataTask task = {[process_in, block, this]() - { - auto & out = *process_in; + Pipes shell_input_pipes; + shell_input_pipes.emplace_back(std::move(shell_input_pipe)); - if (configuration.send_chunk_header) - { - writeText(block.rows(), out); - writeChar('\n', out); - } - - auto output_format = context->getOutputFormat(configuration.format, out, block.cloneEmpty()); - formatBlock(output_format, block); - out.close(); - }}; - std::vector tasks = {std::move(task)}; - - Pipe pipe(std::make_unique(context, configuration.format, sample_block, std::move(process), std::move(tasks))); - - if (configuration.implicit_key) - pipe.addTransform(std::make_shared(block, pipe.getHeader())); - - return pipe; + return coordinator->createPipe(configuration.command, std::move(shell_input_pipes), sample_block, context); } bool ExecutableDictionarySource::isModified() const @@ -192,14 +170,22 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) ExecutableDictionarySource::Configuration configuration { .command = config.getString(settings_config_prefix + ".command"), - .format = config.getString(settings_config_prefix + ".format"), .update_field = config.getString(settings_config_prefix + ".update_field", ""), .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1), .implicit_key = config.getBool(settings_config_prefix + ".implicit_key", false), - .send_chunk_header = config.getBool(settings_config_prefix + ".send_chunk_header", false) }; - return std::make_unique(dict_struct, configuration, sample_block, context); + ShellCommandCoordinator::Configuration shell_command_coordinator_configration + { + .format = config.getString(settings_config_prefix + ".format"), + .is_executable_pool = false, + .send_chunk_header = config.getBool(settings_config_prefix + ".send_chunk_header", false), + .execute_direct = false + }; + + std::shared_ptr coordinator = std::make_shared(shell_command_coordinator_configration); + + return std::make_unique(dict_struct, configuration, sample_block, std::move(coordinator), context); }; factory.registerSource("executable", create_table_source); diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index a7ffc8bebcb..f77da567efe 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -7,6 +7,7 @@ #include #include +#include namespace DB @@ -20,20 +21,18 @@ public: struct Configuration { std::string command; - std::string format; std::string update_field; UInt64 update_lag; /// Implicit key means that the source script will return only values, /// and the correspondence to the requested keys is determined implicitly - by the order of rows in the result. bool implicit_key; - /// Send number_of_rows\n before sending chunk to process - bool send_chunk_header; }; ExecutableDictionarySource( const DictionaryStructure & dict_struct_, const Configuration & configuration_, Block & sample_block_, + std::shared_ptr coordinator_, ContextPtr context_); ExecutableDictionarySource(const ExecutableDictionarySource & other); @@ -69,6 +68,7 @@ private: const DictionaryStructure dict_struct; const Configuration configuration; Block sample_block; + std::shared_ptr coordinator; ContextPtr context; }; diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index a0eb3435a11..c6c70654343 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -4,11 +4,12 @@ #include #include +#include +#include +#include #include #include -#include -#include #include #include @@ -30,13 +31,13 @@ ExecutablePoolDictionarySource::ExecutablePoolDictionarySource( const DictionaryStructure & dict_struct_, const Configuration & configuration_, Block & sample_block_, + std::shared_ptr coordinator_, ContextPtr context_) : dict_struct(dict_struct_) , configuration(configuration_) , sample_block(sample_block_) + , coordinator(std::move(coordinator_)) , context(context_) - /// If pool size == 0 then there is no size restrictions. Poco max size of semaphore is integer type. - , process_pool(std::make_shared(configuration.pool_size == 0 ? std::numeric_limits::max() : configuration.pool_size)) , log(&Poco::Logger::get("ExecutablePoolDictionarySource")) { /// Remove keys from sample_block for implicit_key dictionary because @@ -59,8 +60,8 @@ ExecutablePoolDictionarySource::ExecutablePoolDictionarySource(const ExecutableP : dict_struct(other.dict_struct) , configuration(other.configuration) , sample_block(other.sample_block) + , coordinator(other.coordinator) , context(Context::createCopy(other.context)) - , process_pool(std::make_shared(configuration.pool_size)) , log(&Poco::Logger::get("ExecutablePoolDictionarySource")) { } @@ -93,46 +94,17 @@ Pipe ExecutablePoolDictionarySource::loadKeys(const Columns & key_columns, const Pipe ExecutablePoolDictionarySource::getStreamForBlock(const Block & block) { - std::unique_ptr process; - bool result = process_pool->tryBorrowObject(process, [this]() - { - ShellCommand::Config config(configuration.command); - config.terminate_in_destructor_strategy = ShellCommand::DestructorStrategy{ true /*terminate_in_destructor*/, configuration.command_termination_timeout }; - auto shell_command = ShellCommand::execute(config); - return shell_command; - }, configuration.max_command_execution_time * 10000); - - if (!result) - throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, - "Could not get process from pool, max command execution timeout exceeded {} seconds", - configuration.max_command_execution_time); - - size_t rows_to_read = block.rows(); - auto * process_in = &process->in; - ShellCommandSource::SendDataTask task = [process_in, block, this]() mutable - { - auto & out = *process_in; - - if (configuration.send_chunk_header) - { - writeText(block.rows(), out); - writeChar('\n', out); - } - - auto output_format = context->getOutputFormat(configuration.format, out, block.cloneEmpty()); - formatBlock(output_format, block); - }; - std::vector tasks = {std::move(task)}; + auto source = std::make_shared(block); + auto shell_input_pipe = Pipe(std::move(source)); ShellCommandSourceConfiguration command_configuration; command_configuration.read_fixed_number_of_rows = true; - command_configuration.number_of_rows_to_read = rows_to_read; - Pipe pipe(std::make_unique(context, configuration.format, sample_block, std::move(process), std::move(tasks), command_configuration, process_pool)); + command_configuration.number_of_rows_to_read = block.rows(); - if (configuration.implicit_key) - pipe.addTransform(std::make_shared(block, pipe.getHeader())); + Pipes shell_input_pipes; + shell_input_pipes.emplace_back(std::move(shell_input_pipe)); - return pipe; + return coordinator->createPipe(configuration.command, std::move(shell_input_pipes), std::move(sample_block), context, command_configuration); } bool ExecutablePoolDictionarySource::isModified() const @@ -157,7 +129,8 @@ DictionarySourcePtr ExecutablePoolDictionarySource::clone() const std::string ExecutablePoolDictionarySource::toString() const { - return "ExecutablePool size: " + std::to_string(configuration.pool_size) + " command: " + configuration.command; + size_t pool_size = coordinator->getConfiguration().pool_size; + return "ExecutablePool size: " + std::to_string(pool_size) + " command: " + configuration.command; } void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) @@ -192,15 +165,23 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) ExecutablePoolDictionarySource::Configuration configuration { .command = config.getString(settings_config_prefix + ".command"), + .implicit_key = config.getBool(settings_config_prefix + ".implicit_key", false), + }; + + ShellCommandCoordinator::Configuration shell_command_coordinator_configration + { .format = config.getString(settings_config_prefix + ".format"), .pool_size = config.getUInt64(settings_config_prefix + ".size"), .command_termination_timeout = config.getUInt64(settings_config_prefix + ".command_termination_timeout", 10), .max_command_execution_time = max_command_execution_time, - .implicit_key = config.getBool(settings_config_prefix + ".implicit_key", false), - .send_chunk_header = config.getBool(settings_config_prefix + ".send_chunk_header", false) + .is_executable_pool = true, + .send_chunk_header = config.getBool(settings_config_prefix + ".send_chunk_header", false), + .execute_direct = false }; - return std::make_unique(dict_struct, configuration, sample_block, context); + std::shared_ptr coordinator = std::make_shared(shell_command_coordinator_configration); + + return std::make_unique(dict_struct, configuration, sample_block, std::move(coordinator), context); }; factory.registerSource("executable_pool", create_table_source); diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.h b/src/Dictionaries/ExecutablePoolDictionarySource.h index 51215b6311b..0071f2cc9d0 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.h +++ b/src/Dictionaries/ExecutablePoolDictionarySource.h @@ -28,21 +28,14 @@ public: struct Configuration { String command; - String format; - size_t pool_size; - size_t command_termination_timeout; - size_t max_command_execution_time; - /// Implicit key means that the source script will return only values, - /// and the correspondence to the requested keys is determined implicitly - by the order of rows in the result. bool implicit_key; - /// Send number_of_rows\n before sending chunk to process - bool send_chunk_header; }; ExecutablePoolDictionarySource( const DictionaryStructure & dict_struct_, const Configuration & configuration_, Block & sample_block_, + std::shared_ptr coordinator_, ContextPtr context_); ExecutablePoolDictionarySource(const ExecutablePoolDictionarySource & other); @@ -77,8 +70,8 @@ private: const Configuration configuration; Block sample_block; + std::shared_ptr coordinator; ContextPtr context; - std::shared_ptr process_pool; Poco::Logger * log; }; diff --git a/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp b/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp index 2de7b4b7846..8c5790a5235 100644 --- a/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp +++ b/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp @@ -54,18 +54,26 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS, "The aggregate function '{}' already exists", name); String type = config.getString(key_in_config + ".type"); - UserDefinedExecutableFunctionType function_type; + + bool is_executable_pool; if (type == "executable") - function_type = UserDefinedExecutableFunctionType::executable; + is_executable_pool = false; else if (type == "executable_pool") - function_type = UserDefinedExecutableFunctionType::executable_pool; + is_executable_pool = true; else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong user defined function type expected 'executable' or 'executable_pool' actual {}", - function_type); + type); + + String scipt_name_with_arguments_value = config.getString(key_in_config + ".command"); + + std::vector script_name_with_arguments; + boost::split(script_name_with_arguments, scipt_name_with_arguments_value, [](char c) { return c == ' '; }); + + auto script_path = script_name_with_arguments[0]; + script_name_with_arguments.erase(script_name_with_arguments.begin()); - String command = config.getString(key_in_config + ".command"); String format = config.getString(key_in_config + ".format"); DataTypePtr result_type = DataTypeFactory::instance().get(config.getString(key_in_config + ".return_type")); bool send_chunk_header = config.getBool(key_in_config + ".send_chunk_header", false); @@ -73,7 +81,7 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create size_t pool_size = 0; size_t command_termination_timeout = 0; size_t max_command_execution_time = 0; - if (function_type == UserDefinedExecutableFunctionType::executable_pool) + if (is_executable_pool) { pool_size = config.getUInt64(key_in_config + ".pool_size", 16); command_termination_timeout = config.getUInt64(key_in_config + ".command_termination_timeout", 10); @@ -106,19 +114,27 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create UserDefinedExecutableFunctionConfiguration function_configuration { - .type = function_type, .name = std::move(name), //-V1030 - .script_path = std::move(command), //-V1030 - .format = std::move(format), //-V1030 + .script_path = std::move(script_path), //-V1030 + .script_arguments = std::move(script_name_with_arguments), //-V1030 .argument_types = std::move(argument_types), //-V1030 .result_type = std::move(result_type), //-V1030 + }; + + ShellCommandCoordinator::Configuration shell_command_coordinator_configration + { + .format = std::move(format), //-V1030 .pool_size = pool_size, .command_termination_timeout = command_termination_timeout, .max_command_execution_time = max_command_execution_time, - .send_chunk_header = send_chunk_header + .is_executable_pool = is_executable_pool, + .send_chunk_header = send_chunk_header, + .execute_direct = true }; - return std::make_shared(function_configuration, lifetime); + std::shared_ptr coordinator = std::make_shared(shell_command_coordinator_configration); + + return std::make_shared(function_configuration, coordinator, lifetime); } } diff --git a/src/Interpreters/UserDefinedExecutableFunction.cpp b/src/Interpreters/UserDefinedExecutableFunction.cpp index d57978d0fd6..b8a84db64c8 100644 --- a/src/Interpreters/UserDefinedExecutableFunction.cpp +++ b/src/Interpreters/UserDefinedExecutableFunction.cpp @@ -13,14 +13,12 @@ namespace DB UserDefinedExecutableFunction::UserDefinedExecutableFunction( const UserDefinedExecutableFunctionConfiguration & configuration_, - const ExternalLoadableLifetime & lifetime_, - std::shared_ptr process_pool_) + std::shared_ptr coordinator_, + const ExternalLoadableLifetime & lifetime_) : configuration(configuration_) + , coordinator(std::move(coordinator_)) , lifetime(lifetime_) - , process_pool(process_pool_) { - if (!process_pool && configuration.type == UserDefinedExecutableFunctionType::executable_pool) - process_pool = std::make_shared(configuration.pool_size == 0 ? std::numeric_limits::max() : configuration.pool_size); } }; diff --git a/src/Interpreters/UserDefinedExecutableFunction.h b/src/Interpreters/UserDefinedExecutableFunction.h index 1cb1de47578..c466f43130b 100644 --- a/src/Interpreters/UserDefinedExecutableFunction.h +++ b/src/Interpreters/UserDefinedExecutableFunction.h @@ -10,26 +10,13 @@ namespace DB { -enum class UserDefinedExecutableFunctionType -{ - executable, - executable_pool -}; - struct UserDefinedExecutableFunctionConfiguration { - UserDefinedExecutableFunctionType type = UserDefinedExecutableFunctionType::executable; std::string name; std::string script_path; - std::string format; + std::vector script_arguments; std::vector argument_types; DataTypePtr result_type; - /// Pool settings - size_t pool_size = 0; - size_t command_termination_timeout = 0; - size_t max_command_execution_time = 0; - /// Send number_of_rows\n before sending chunk to process - bool send_chunk_header = false; }; class UserDefinedExecutableFunction final : public IExternalLoadable @@ -38,8 +25,8 @@ public: UserDefinedExecutableFunction( const UserDefinedExecutableFunctionConfiguration & configuration_, - const ExternalLoadableLifetime & lifetime_, - std::shared_ptr process_pool_ = nullptr); + std::shared_ptr coordinator_, + const ExternalLoadableLifetime & lifetime_); const ExternalLoadableLifetime & getLifetime() const override { @@ -63,7 +50,7 @@ public: std::shared_ptr clone() const override { - return std::make_shared(configuration, lifetime, process_pool); + return std::make_shared(configuration, coordinator, lifetime); } const UserDefinedExecutableFunctionConfiguration & getConfiguration() const @@ -71,9 +58,9 @@ public: return configuration; } - std::shared_ptr getProcessPool() const + std::shared_ptr getCoordinator() const { - return process_pool; + return coordinator; } std::shared_ptr shared_from_this() @@ -87,13 +74,9 @@ public: } private: - UserDefinedExecutableFunction(const UserDefinedExecutableFunctionConfiguration & configuration_, - std::shared_ptr process_pool_, - const ExternalLoadableLifetime & lifetime_); - UserDefinedExecutableFunctionConfiguration configuration; + std::shared_ptr coordinator; ExternalLoadableLifetime lifetime; - std::shared_ptr process_pool; }; } diff --git a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp index 4cb3e034b01..ea60741ba87 100644 --- a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp +++ b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include @@ -71,53 +72,35 @@ public: column_with_type = column_to_cast; } - std::unique_ptr process = getProcess(); - ColumnWithTypeAndName result(result_type, "result"); Block result_block({result}); Block arguments_block(arguments_copy); - auto * process_in = &process->in; + auto source = std::make_shared(std::move(arguments_block)); + auto shell_input_pipe = Pipe(std::move(source)); - auto process_pool = executable_function->getProcessPool(); - bool is_executable_pool_function = (process_pool != nullptr); + auto coordinator = executable_function->getCoordinator(); ShellCommandSourceConfiguration shell_command_source_configuration; - if (is_executable_pool_function) + if (coordinator->getConfiguration().is_executable_pool) { shell_command_source_configuration.read_fixed_number_of_rows = true; shell_command_source_configuration.number_of_rows_to_read = input_rows_count; } - ShellCommandSource::SendDataTask task = {[process_in, arguments_block, &configuration, is_executable_pool_function, this]() - { - auto & out = *process_in; + Pipes shell_input_pipes; + shell_input_pipes.emplace_back(std::move(shell_input_pipe)); - if (configuration.send_chunk_header) - { - writeText(arguments_block.rows(), out); - writeChar('\n', out); - } - - auto output_format = context->getOutputFormat(configuration.format, out, arguments_block.cloneEmpty()); - formatBlock(output_format, arguments_block); - if (!is_executable_pool_function) - out.close(); - }}; - std::vector tasks = {std::move(task)}; - - Pipe pipe(std::make_unique( + Pipe pipe = coordinator->createPipe( + configuration.script_path, + configuration.script_arguments, + std::move(shell_input_pipes), + result_block, context, - configuration.format, - result_block.cloneEmpty(), - std::move(process), - std::move(tasks), - shell_command_source_configuration, - process_pool)); + shell_command_source_configuration); QueryPipeline pipeline(std::move(pipe)); - PullingPipelineExecutor executor(pipeline); auto result_column = result_type->createColumn(); @@ -143,36 +126,6 @@ public: private: - std::unique_ptr getProcess() const - { - auto process_pool = executable_function->getProcessPool(); - auto executable_function_configuration = executable_function->getConfiguration(); - - std::unique_ptr process; - bool is_executable_pool_function = (process_pool != nullptr); - if (is_executable_pool_function) - { - bool result = process_pool->tryBorrowObject(process, [&]() - { - ShellCommand::Config process_config(executable_function_configuration.script_path); - process_config.terminate_in_destructor_strategy = ShellCommand::DestructorStrategy{ true /*terminate_in_destructor*/, executable_function_configuration.command_termination_timeout }; - auto shell_command = ShellCommand::execute(process_config); - return shell_command; - }, executable_function_configuration.max_command_execution_time * 1000); - - if (!result) - throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, - "Could not get process from pool, max command execution timeout exceeded {} seconds", - executable_function_configuration.max_command_execution_time); - } - else - { - process = ShellCommand::execute(executable_function_configuration.script_path); - } - - return process; - } - ExternalUserDefinedExecutableFunctionsLoader::UserDefinedExecutableFunctionPtr executable_function; ContextPtr context; }; diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp new file mode 100644 index 00000000000..6b84117405a --- /dev/null +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -0,0 +1,300 @@ +#include + +#include +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TIMEOUT_EXCEEDED; +} + +namespace +{ + /** A stream, that get child process and sends data using tasks in background threads. + * For each send data task background thread is created. Send data task must send data to process input pipes. + * ShellCommandPoolSource receives data from process stdout. + * + * If process_pool is passed in constructor then after source is destroyed process is returned to pool. + */ + class ShellCommandSource final : public SourceWithProgress + { + public: + + using SendDataTask = std::function; + + ShellCommandSource( + ContextPtr context, + const std::string & format, + const Block & sample_block, + std::unique_ptr && command_, + std::vector && send_data_tasks = {}, + const ShellCommandSourceConfiguration & configuration_ = {}, + std::shared_ptr process_pool_ = nullptr) + : SourceWithProgress(sample_block) + , command(std::move(command_)) + , configuration(configuration_) + , process_pool(process_pool_) + { + for (auto && send_data_task : send_data_tasks) + { + send_data_threads.emplace_back([task = std::move(send_data_task), this]() + { + try + { + task(); + } + catch (...) + { + std::lock_guard lock(send_data_lock); + exception_during_send_data = std::current_exception(); + } + }); + } + + size_t max_block_size = configuration.max_block_size; + + if (configuration.read_fixed_number_of_rows) + { + /** Currently parallel parsing input format cannot read exactly max_block_size rows from input, + * so it will be blocked on ReadBufferFromFileDescriptor because this file descriptor represent pipe that does not have eof. + */ + auto context_for_reading = Context::createCopy(context); + context_for_reading->setSetting("input_format_parallel_parsing", false); + context = context_for_reading; + + if (configuration.read_number_of_rows_from_process_output) + { + /// TODO: Move to generate + readText(configuration.number_of_rows_to_read, command->out); + char dummy; + readChar(dummy, command->out); + } + + max_block_size = configuration.number_of_rows_to_read; + } + + pipeline = QueryPipeline(Pipe(context->getInputFormat(format, command->out, sample_block, max_block_size))); + executor = std::make_unique(pipeline); + } + + ~ShellCommandSource() override + { + for (auto & thread : send_data_threads) + if (thread.joinable()) + thread.join(); + + if (command && process_pool) + process_pool->returnObject(std::move(command)); + } + + protected: + + Chunk generate() override + { + rethrowExceptionDuringSendDataIfNeeded(); + + if (configuration.read_fixed_number_of_rows && configuration.number_of_rows_to_read >= current_read_rows) + return {}; + + Chunk chunk; + + try + { + if (!executor->pull(chunk)) + return {}; + + current_read_rows += chunk.getNumRows(); + } + catch (...) + { + command = nullptr; + throw; + } + + return chunk; + } + + Status prepare() override + { + auto status = SourceWithProgress::prepare(); + + if (status == Status::Finished) + { + for (auto & thread : send_data_threads) + if (thread.joinable()) + thread.join(); + + rethrowExceptionDuringSendDataIfNeeded(); + } + + return status; + } + + String getName() const override { return "ShellCommandSource"; } + + private: + + void rethrowExceptionDuringSendDataIfNeeded() + { + std::lock_guard lock(send_data_lock); + if (exception_during_send_data) + { + command = nullptr; + std::rethrow_exception(exception_during_send_data); + } + } + + std::unique_ptr command; + ShellCommandSourceConfiguration configuration; + + size_t current_read_rows = 0; + + std::shared_ptr process_pool; + + QueryPipeline pipeline; + std::unique_ptr executor; + + std::vector send_data_threads; + std::mutex send_data_lock; + std::exception_ptr exception_during_send_data; + }; + + class SendingChunkHeaderTransform final : public ISimpleTransform + { + public: + SendingChunkHeaderTransform(const Block & header, WriteBuffer & buffer_) + : ISimpleTransform(header, header, false) + , buffer(buffer_) + { + } + + String getName() const override { return "SendingChunkHeaderTransform"; } + + protected: + + void transform(Chunk & chunk) override + { + writeText(chunk.getNumRows(), buffer); + writeChar('\n', buffer); + } + + private: + WriteBuffer & buffer; + }; + +} + +ShellCommandCoordinator::ShellCommandCoordinator(const Configuration & configuration_) + : configuration(configuration_) +{ + if (configuration.is_executable_pool) + process_pool = std::make_shared(configuration.pool_size ? configuration.pool_size : std::numeric_limits::max()); +} + +Pipe ShellCommandCoordinator::createPipe( + const std::string & command, + const std::vector & arguments, + std::vector && input_pipes, + Block sample_block, + ContextPtr context, + const ShellCommandSourceConfiguration & source_configuration) +{ + ShellCommand::Config command_config(command); + command_config.arguments = arguments; + for (size_t i = 1; i < input_pipes.size(); ++i) + command_config.write_fds.emplace_back(i + 2); + + std::unique_ptr process; + + bool is_executable_pool = (process_pool != nullptr); + if (is_executable_pool) + { + bool result = process_pool->tryBorrowObject( + process, + [&command_config, this]() + { + command_config.terminate_in_destructor_strategy + = ShellCommand::DestructorStrategy{true /*terminate_in_destructor*/, configuration.command_termination_timeout}; + + if (configuration.execute_direct) + return ShellCommand::executeDirect(command_config); + else + return ShellCommand::execute(command_config); + }, + configuration.max_command_execution_time * 10000); + + if (!result) + throw Exception( + ErrorCodes::TIMEOUT_EXCEEDED, + "Could not get process from pool, max command execution timeout exceeded {} seconds", + configuration.max_command_execution_time); + } + else + { + if (configuration.execute_direct) + process = ShellCommand::executeDirect(command_config); + else + process = ShellCommand::execute(command_config); + } + + std::vector tasks; + tasks.reserve(input_pipes.size()); + + for (size_t i = 0; i < input_pipes.size(); ++i) + { + WriteBufferFromFile * write_buffer = nullptr; + + if (i == 0) + { + write_buffer = &process->in; + } + else + { + auto descriptor = i + 2; + auto it = process->write_fds.find(descriptor); + if (it == process->write_fds.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Process does not contain descriptor to write {}", descriptor); + + write_buffer = &it->second; + } + + input_pipes[i].resize(1); + + if (configuration.send_chunk_header) + { + auto transform = std::make_shared(input_pipes[i].getHeader(), *write_buffer); + input_pipes[i].addTransform(std::move(transform)); + } + + auto pipeline = std::make_shared(std::move(input_pipes[i])); + auto out = context->getOutputFormat(configuration.format, *write_buffer, materializeBlock(pipeline->getHeader())); + out->setAutoFlush(); + pipeline->complete(std::move(out)); + + ShellCommandSource::SendDataTask task = [pipeline, write_buffer, is_executable_pool]() + { + CompletedPipelineExecutor executor(*pipeline); + executor.execute(); + + if (!is_executable_pool) + write_buffer->close(); + }; + + tasks.emplace_back(std::move(task)); + } + + Pipe pipe(std::make_unique(context, configuration.format, std::move(sample_block), std::move(process), std::move(tasks), source_configuration, process_pool)); + return pipe; +} + +} diff --git a/src/Processors/Sources/ShellCommandSource.h b/src/Processors/Sources/ShellCommandSource.h index 4974c33f290..aa18a289226 100644 --- a/src/Processors/Sources/ShellCommandSource.h +++ b/src/Processors/Sources/ShellCommandSource.h @@ -19,13 +19,6 @@ namespace DB { -/** A stream, that get child process and sends data using tasks in background threads. - * For each send data task background thread is created. Send data task must send data to process input pipes. - * ShellCommandPoolSource receives data from process stdout. - * - * If process_pool is passed in constructor then after source is destroyed process is returned to pool. - */ - using ProcessPool = BorrowedObjectPool>; struct ShellCommandSourceConfiguration @@ -37,148 +30,77 @@ struct ShellCommandSourceConfiguration /// Valid only if read_fixed_number_of_rows = true size_t number_of_rows_to_read = 0; /// Max block size - size_t max_block_size = DBMS_DEFAULT_BUFFER_SIZE; + size_t max_block_size = DEFAULT_BLOCK_SIZE; }; -class ShellCommandSource final : public SourceWithProgress +class ShellCommandCoordinator { public: - using SendDataTask = std::function; + struct Configuration + { - ShellCommandSource( + /// Script output format + std::string format; + + /// Pool size valid only if executable_pool = true + size_t pool_size = 16; + + /// Command termination timeout in seconds. Valid only if executable_pool = true + size_t command_termination_timeout = 10; + + /// Max command execution time in seconds. Valid only if executable_pool = true + size_t max_command_execution_time = 10; + + /// Should pool of processes be created. + bool is_executable_pool = false; + + /// Send number_of_rows\n before sending chunk to process. + bool send_chunk_header = false; + + /// Execute script direct or with /bin/bash. + bool execute_direct = true; + + }; + + explicit ShellCommandCoordinator(const Configuration & configuration_); + + const Configuration & getConfiguration() const + { + return configuration; + } + + Pipe createPipe( + const std::string & command, + const std::vector & arguments, + std::vector && input_pipes, + Block sample_block, ContextPtr context, - const std::string & format, - const Block & sample_block, - std::unique_ptr && command_, - std::vector && send_data_tasks = {}, - const ShellCommandSourceConfiguration & configuration_ = {}, - std::shared_ptr process_pool_ = nullptr) - : SourceWithProgress(sample_block) - , command(std::move(command_)) - , configuration(configuration_) - , process_pool(process_pool_) + const ShellCommandSourceConfiguration & source_configuration = {}); + + Pipe createPipe( + const std::string & command, + std::vector && input_pipes, + Block sample_block, + ContextPtr context, + const ShellCommandSourceConfiguration & source_configuration = {}) { - for (auto && send_data_task : send_data_tasks) - { - send_data_threads.emplace_back([task = std::move(send_data_task), this]() - { - try - { - task(); - } - catch (...) - { - std::lock_guard lock(send_data_lock); - exception_during_send_data = std::current_exception(); - } - }); - } - - size_t max_block_size = configuration.max_block_size; - - if (configuration.read_fixed_number_of_rows) - { - /** Currently parallel parsing input format cannot read exactly max_block_size rows from input, - * so it will be blocked on ReadBufferFromFileDescriptor because this file descriptor represent pipe that does not have eof. - */ - auto context_for_reading = Context::createCopy(context); - context_for_reading->setSetting("input_format_parallel_parsing", false); - context = context_for_reading; - - if (configuration.read_number_of_rows_from_process_output) - { - readText(configuration.number_of_rows_to_read, command->out); - char dummy; - readChar(dummy, command->out); - } - - max_block_size = configuration.number_of_rows_to_read; - } - - pipeline = QueryPipeline(Pipe(context->getInputFormat(format, command->out, sample_block, max_block_size))); - executor = std::make_unique(pipeline); + return createPipe(command, {}, std::move(input_pipes), std::move(sample_block), std::move(context), source_configuration); } - ~ShellCommandSource() override + Pipe createPipe( + const std::string & command, + Block sample_block, + ContextPtr context) { - for (auto & thread : send_data_threads) - if (thread.joinable()) - thread.join(); - - if (command && process_pool) - process_pool->returnObject(std::move(command)); + return createPipe(command, {}, {}, std::move(sample_block), std::move(context), {}); } -protected: - - Chunk generate() override - { - rethrowExceptionDuringSendDataIfNeeded(); - - if (configuration.read_fixed_number_of_rows && configuration.number_of_rows_to_read == current_read_rows) - return {}; - - Chunk chunk; - - try - { - if (!executor->pull(chunk)) - return {}; - - current_read_rows += chunk.getNumRows(); - } - catch (...) - { - command = nullptr; - throw; - } - - return chunk; - } - - Status prepare() override - { - auto status = SourceWithProgress::prepare(); - - if (status == Status::Finished) - { - for (auto & thread : send_data_threads) - if (thread.joinable()) - thread.join(); - - rethrowExceptionDuringSendDataIfNeeded(); - } - - return status; - } - - String getName() const override { return "ShellCommandSource"; } - private: - void rethrowExceptionDuringSendDataIfNeeded() - { - std::lock_guard lock(send_data_lock); - if (exception_during_send_data) - { - command = nullptr; - std::rethrow_exception(exception_during_send_data); - } - } + Configuration configuration; - std::unique_ptr command; - ShellCommandSourceConfiguration configuration; - - size_t current_read_rows = 0; - - std::shared_ptr process_pool; - - QueryPipeline pipeline; - std::unique_ptr executor; - - std::vector send_data_threads; - std::mutex send_data_lock; - std::exception_ptr exception_during_send_data; + std::shared_ptr process_pool = nullptr; }; + } diff --git a/src/Storages/ExecutableSettings.h b/src/Storages/ExecutableSettings.h index 9c0cfc05fa5..873ff3f45a7 100644 --- a/src/Storages/ExecutableSettings.h +++ b/src/Storages/ExecutableSettings.h @@ -9,7 +9,7 @@ namespace DB class ASTStorage; #define LIST_OF_EXECUTABLE_SETTINGS(M) \ - M(UInt64, send_chunk_header, false, "Send number_of_rows\n before sending chunk to process", 0) \ + M(Bool, send_chunk_header, false, "Send number_of_rows\n before sending chunk to process", 0) \ M(UInt64, pool_size, 16, "Processes pool size. If size == 0, then there is no size restrictions", 0) \ M(UInt64, max_command_execution_time, 10, "Max command execution time in seconds.", 0) \ M(UInt64, command_termination_timeout, 10, "Command termination timeout in seconds.", 0) \ @@ -19,6 +19,11 @@ DECLARE_SETTINGS_TRAITS(ExecutableSettingsTraits, LIST_OF_EXECUTABLE_SETTINGS) /// Settings for ExecutablePool engine. struct ExecutableSettings : public BaseSettings { + std::string script_name; + std::vector script_arguments; + + bool is_executable_pool = false; + void loadFromQuery(ASTStorage & storage_def); }; diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 51ecfc1e884..0eb3e03c49e 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -37,73 +37,37 @@ namespace ErrorCodes StorageExecutable::StorageExecutable( const StorageID & table_id_, - const String & script_name_, - const std::vector & arguments_, - const String & format_, - const std::vector & input_queries_, - const ColumnsDescription & columns, - const ConstraintsDescription & constraints) - : IStorage(table_id_) - , script_name(script_name_) - , arguments(arguments_) - , format(format_) - , input_queries(input_queries_) - , log(&Poco::Logger::get("StorageExecutable")) -{ - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns); - storage_metadata.setConstraints(constraints); - setInMemoryMetadata(storage_metadata); -} - -StorageExecutable::StorageExecutable( - const StorageID & table_id_, - const String & script_name_, - const std::vector & arguments_, - const String & format_, - const std::vector & input_queries_, + const String & format, const ExecutableSettings & settings_, + const std::vector & input_queries_, const ColumnsDescription & columns, const ConstraintsDescription & constraints) : IStorage(table_id_) - , script_name(script_name_) - , arguments(arguments_) - , format(format_) - , input_queries(input_queries_) , settings(settings_) - /// If pool size == 0 then there is no size restrictions. Poco max size of semaphore is integer type. - , process_pool(std::make_shared(settings.pool_size == 0 ? std::numeric_limits::max() : settings.pool_size)) - , log(&Poco::Logger::get("StorageExecutablePool")) + , input_queries(input_queries_) + , log(settings.is_executable_pool ? &Poco::Logger::get("StorageExecutablePool") : &Poco::Logger::get("StorageExecutable")) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns); storage_metadata.setConstraints(constraints); setInMemoryMetadata(storage_metadata); + + ShellCommandCoordinator::Configuration configuration + { + .format = format, + + .pool_size = settings.pool_size, + .command_termination_timeout = settings.command_termination_timeout, + .max_command_execution_time = settings.max_command_execution_time, + + .is_executable_pool = settings.is_executable_pool, + .send_chunk_header = settings.send_chunk_header, + .execute_direct = true + }; + + coordinator = std::make_unique(std::move(configuration)); } -class SendingChunkHeaderTransform final : public ISimpleTransform -{ -public: - SendingChunkHeaderTransform(const Block & header, WriteBuffer & buffer_) - : ISimpleTransform(header, header, false) - , buffer(buffer_) - { - } - - String getName() const override { return "SendingChunkHeaderTransform"; } - -protected: - - void transform(Chunk & chunk) override - { - writeText(chunk.getNumRows(), buffer); - writeChar('\n', buffer); - } - -private: - WriteBuffer & buffer; -}; - Pipe StorageExecutable::read( const Names & /*column_names*/, const StorageMetadataPtr & metadata_snapshot, @@ -113,6 +77,8 @@ Pipe StorageExecutable::read( size_t max_block_size, unsigned /*threads*/) { + auto & script_name = settings.script_name; + auto user_scripts_path = context->getUserScriptsPath(); auto script_path = user_scripts_path + '/' + script_name; @@ -128,86 +94,13 @@ Pipe StorageExecutable::read( script_name, user_scripts_path); - std::vector inputs; + Pipes inputs; inputs.reserve(input_queries.size()); for (auto & input_query : input_queries) { InterpreterSelectWithUnionQuery interpreter(input_query, context, {}); - inputs.emplace_back(interpreter.buildQueryPipeline()); - } - - ShellCommand::Config config(script_path); - config.arguments = arguments; - for (size_t i = 1; i < inputs.size(); ++i) - config.write_fds.emplace_back(i + 2); - - std::unique_ptr process; - - bool is_executable_pool = (process_pool != nullptr); - if (is_executable_pool) - { - bool result = process_pool->tryBorrowObject(process, [&config, this]() - { - config.terminate_in_destructor_strategy = ShellCommand::DestructorStrategy{ true /*terminate_in_destructor*/, settings.command_termination_timeout }; - auto shell_command = ShellCommand::executeDirect(config); - return shell_command; - }, settings.max_command_execution_time * 10000); - - if (!result) - throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, - "Could not get process from pool, max command execution timeout exceeded {} seconds", - settings.max_command_execution_time); - } - else - { - process = ShellCommand::executeDirect(config); - } - - std::vector tasks; - tasks.reserve(inputs.size()); - - for (size_t i = 0; i < inputs.size(); ++i) - { - WriteBufferFromFile * write_buffer = nullptr; - - if (i == 0) - { - write_buffer = &process->in; - } - else - { - auto descriptor = i + 2; - auto it = process->write_fds.find(descriptor); - if (it == process->write_fds.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Process does not contain descriptor to write {}", descriptor); - - write_buffer = &it->second; - } - - inputs[i].resize(1); - if (settings.send_chunk_header) - { - auto transform = std::make_shared(inputs[i].getHeader(), *write_buffer); - inputs[i].addTransform(std::move(transform)); - } - - auto pipeline = std::make_shared(QueryPipelineBuilder::getPipeline(std::move(inputs[i]))); - - auto out = context->getOutputFormat(format, *write_buffer, materializeBlock(pipeline->getHeader())); - out->setAutoFlush(); - pipeline->complete(std::move(out)); - - ShellCommandSource::SendDataTask task = [pipeline, write_buffer, is_executable_pool]() - { - CompletedPipelineExecutor executor(*pipeline); - executor.execute(); - - if (!is_executable_pool) - write_buffer->close(); - }; - - tasks.emplace_back(std::move(task)); + inputs.emplace_back(QueryPipelineBuilder::getPipe(interpreter.buildQueryPipeline())); } auto sample_block = metadata_snapshot->getSampleBlock(); @@ -215,14 +108,14 @@ Pipe StorageExecutable::read( ShellCommandSourceConfiguration configuration; configuration.max_block_size = max_block_size; - if (is_executable_pool) + if (coordinator->getConfiguration().is_executable_pool) { configuration.read_fixed_number_of_rows = true; configuration.read_number_of_rows_from_process_output = true; } - Pipe pipe(std::make_unique(context, format, std::move(sample_block), std::move(process), std::move(tasks), configuration, process_pool)); - return pipe; + /// TODO: Filter by column_names + return coordinator->createPipe(settings.script_name, settings.script_arguments, std::move(inputs), std::move(sample_block), context, configuration); } void registerStorageExecutable(StorageFactory & factory) @@ -262,6 +155,11 @@ void registerStorageExecutable(StorageFactory & factory) const auto & columns = args.columns; const auto & constraints = args.constraints; + ExecutableSettings settings; + settings.script_name = script_name; + settings.script_arguments = script_name_with_arguments; + settings.is_executable_pool = is_executable_pool; + if (is_executable_pool) { size_t max_command_execution_time = 10; @@ -270,17 +168,13 @@ void registerStorageExecutable(StorageFactory & factory) if (max_execution_time_seconds != 0 && max_command_execution_time > max_execution_time_seconds) max_command_execution_time = max_execution_time_seconds; - ExecutableSettings pool_settings; - pool_settings.max_command_execution_time = max_command_execution_time; + settings.max_command_execution_time = max_command_execution_time; if (args.storage_def->settings) - pool_settings.loadFromQuery(*args.storage_def); + settings.loadFromQuery(*args.storage_def); + } - return StorageExecutable::create(args.table_id, script_name, script_name_with_arguments, format, input_queries, pool_settings, columns, constraints); - } - else - { - return StorageExecutable::create(args.table_id, script_name, script_name_with_arguments, format, input_queries, columns, constraints); - } + auto global_context = args.getContext()->getGlobalContext(); + return StorageExecutable::create(args.table_id, format, settings, input_queries, columns, constraints); }; factory.registerStorage("Executable", [&](const StorageFactory::Arguments & args) diff --git a/src/Storages/StorageExecutable.h b/src/Storages/StorageExecutable.h index 74df17f1463..a12732281d2 100644 --- a/src/Storages/StorageExecutable.h +++ b/src/Storages/StorageExecutable.h @@ -23,7 +23,7 @@ public: String getName() const override { - if (process_pool) + if (coordinator->getConfiguration().is_executable_pool) return "ExecutablePool"; else return "Executable"; @@ -42,31 +42,17 @@ protected: StorageExecutable( const StorageID & table_id, - const String & script_name_, - const std::vector & arguments_, - const String & format_, - const std::vector & input_queries_, - const ColumnsDescription & columns, - const ConstraintsDescription & constraints); - - StorageExecutable( - const StorageID & table_id, - const String & script_name_, - const std::vector & arguments_, - const String & format_, - const std::vector & input_queries_, - const ExecutableSettings & settings_, + const String & format, + const ExecutableSettings & settings, + const std::vector & input_queries, const ColumnsDescription & columns, const ConstraintsDescription & constraints); private: - String script_name; - std::vector arguments; - String format; - std::vector input_queries; ExecutableSettings settings; - std::shared_ptr process_pool; + std::vector input_queries; Poco::Logger * log; + std::unique_ptr coordinator; }; } diff --git a/src/TableFunctions/TableFunctionExecutable.cpp b/src/TableFunctions/TableFunctionExecutable.cpp index 9edb75b0a69..41ba2db5c33 100644 --- a/src/TableFunctions/TableFunctionExecutable.cpp +++ b/src/TableFunctions/TableFunctionExecutable.cpp @@ -75,7 +75,12 @@ ColumnsDescription TableFunctionExecutable::getActualTableStructure(ContextPtr c StoragePtr TableFunctionExecutable::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { auto storage_id = StorageID(getDatabaseName(), table_name); - auto storage = StorageExecutable::create(storage_id, script_name, arguments, format, input_queries, getActualTableStructure(context), ConstraintsDescription{}); + auto global_context = context->getGlobalContext(); + ExecutableSettings settings; + settings.script_name = script_name; + settings.script_arguments = std::move(arguments); + + auto storage = StorageExecutable::create(storage_id, format, settings, input_queries, getActualTableStructure(context), ConstraintsDescription{}); storage->startup(); return storage; } From 2ffd83ee83ad5f53da26f07908c943c922a120d9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 22 Dec 2021 18:20:36 +0300 Subject: [PATCH 0510/1260] Fix shell command source start --- .../ExecutablePoolDictionarySource.cpp | 1 - ...alUserDefinedExecutableFunctionsLoader.cpp | 9 ++++--- .../UserDefinedExecutableFunction.h | 2 +- .../UserDefinedExecutableFunctionFactory.cpp | 25 +++++++++++++++++-- src/Processors/Sources/ShellCommandSource.cpp | 7 +++--- src/Storages/StorageExecutable.cpp | 11 +++----- 6 files changed, 37 insertions(+), 18 deletions(-) diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index c6c70654343..0208791a325 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -24,7 +24,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int DICTIONARY_ACCESS_DENIED; extern const int UNSUPPORTED_METHOD; - extern const int TIMEOUT_EXCEEDED; } ExecutablePoolDictionarySource::ExecutablePoolDictionarySource( diff --git a/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp b/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp index 8c5790a5235..593a29d5688 100644 --- a/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp +++ b/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp @@ -1,5 +1,7 @@ #include "ExternalUserDefinedExecutableFunctionsLoader.h" +#include + #include #include @@ -71,7 +73,7 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create std::vector script_name_with_arguments; boost::split(script_name_with_arguments, scipt_name_with_arguments_value, [](char c) { return c == ' '; }); - auto script_path = script_name_with_arguments[0]; + auto script_name = script_name_with_arguments[0]; script_name_with_arguments.erase(script_name_with_arguments.begin()); String format = config.getString(key_in_config + ".format"); @@ -115,7 +117,7 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create UserDefinedExecutableFunctionConfiguration function_configuration { .name = std::move(name), //-V1030 - .script_path = std::move(script_path), //-V1030 + .script_name = std::move(script_name), //-V1030 .script_arguments = std::move(script_name_with_arguments), //-V1030 .argument_types = std::move(argument_types), //-V1030 .result_type = std::move(result_type), //-V1030 @@ -133,8 +135,7 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create }; std::shared_ptr coordinator = std::make_shared(shell_command_coordinator_configration); - - return std::make_shared(function_configuration, coordinator, lifetime); + return std::make_shared(function_configuration, std::move(coordinator), lifetime); } } diff --git a/src/Interpreters/UserDefinedExecutableFunction.h b/src/Interpreters/UserDefinedExecutableFunction.h index c466f43130b..92a4c0985c5 100644 --- a/src/Interpreters/UserDefinedExecutableFunction.h +++ b/src/Interpreters/UserDefinedExecutableFunction.h @@ -13,7 +13,7 @@ namespace DB struct UserDefinedExecutableFunctionConfiguration { std::string name; - std::string script_path; + std::string script_name; std::vector script_arguments; std::vector argument_types; DataTypePtr result_type; diff --git a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp index ea60741ba87..41fc3fe4792 100644 --- a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp +++ b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp @@ -1,5 +1,9 @@ #include "UserDefinedExecutableFunctionFactory.h" +#include + +#include + #include #include @@ -20,7 +24,6 @@ namespace DB namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; - extern const int TIMEOUT_EXCEEDED; } class UserDefinedFunction final : public IFunction @@ -54,6 +57,24 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { const auto & configuration = executable_function->getConfiguration(); + + auto user_scripts_path = context->getUserScriptsPath(); + const auto & script_name = configuration.script_name; + + auto script_path = user_scripts_path + '/' + script_name; + + if (!fileOrSymlinkPathStartsWith(script_path, user_scripts_path)) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Executable file {} must be inside user scripts folder {}", + script_name, + user_scripts_path); + + if (!std::filesystem::exists(std::filesystem::path(script_path))) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Executable file {} does not exist inside user scripts folder {}", + script_name, + user_scripts_path); + auto arguments_copy = arguments; for (size_t i = 0; i < arguments.size(); ++i) @@ -93,7 +114,7 @@ public: shell_input_pipes.emplace_back(std::move(shell_input_pipe)); Pipe pipe = coordinator->createPipe( - configuration.script_path, + script_path, configuration.script_arguments, std::move(shell_input_pipes), result_block, diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 6b84117405a..21d07ecf126 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -15,6 +15,7 @@ namespace DB namespace ErrorCodes { + extern const int UNSUPPORTED_METHOD; extern const int TIMEOUT_EXCEEDED; } @@ -66,8 +67,8 @@ namespace if (configuration.read_fixed_number_of_rows) { /** Currently parallel parsing input format cannot read exactly max_block_size rows from input, - * so it will be blocked on ReadBufferFromFileDescriptor because this file descriptor represent pipe that does not have eof. - */ + * so it will be blocked on ReadBufferFromFileDescriptor because this file descriptor represent pipe that does not have eof. + */ auto context_for_reading = Context::createCopy(context); context_for_reading->setSetting("input_format_parallel_parsing", false); context = context_for_reading; @@ -263,7 +264,7 @@ Pipe ShellCommandCoordinator::createPipe( auto descriptor = i + 2; auto it = process->write_fds.find(descriptor); if (it == process->write_fds.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Process does not contain descriptor to write {}", descriptor); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Process does not contain descriptor to write {}", descriptor); write_buffer = &it->second; } diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 0eb3e03c49e..98c9b276497 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -2,6 +2,8 @@ #include +#include + #include #include @@ -21,8 +23,6 @@ #include #include -#include - namespace DB { @@ -30,9 +30,7 @@ namespace DB namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; - extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int TIMEOUT_EXCEEDED; } StorageExecutable::StorageExecutable( @@ -82,7 +80,7 @@ Pipe StorageExecutable::read( auto user_scripts_path = context->getUserScriptsPath(); auto script_path = user_scripts_path + '/' + script_name; - if (!pathStartsWith(script_path, user_scripts_path)) + if (!fileOrSymlinkPathStartsWith(script_path, user_scripts_path)) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Executable file {} must be inside user scripts folder {}", script_name, @@ -114,8 +112,7 @@ Pipe StorageExecutable::read( configuration.read_number_of_rows_from_process_output = true; } - /// TODO: Filter by column_names - return coordinator->createPipe(settings.script_name, settings.script_arguments, std::move(inputs), std::move(sample_block), context, configuration); + return coordinator->createPipe(script_path, settings.script_arguments, std::move(inputs), std::move(sample_block), context, configuration); } void registerStorageExecutable(StorageFactory & factory) From d3e507b9ef935370b8bb5c8f9571abb9510824d1 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 23 Dec 2021 13:01:44 +0300 Subject: [PATCH 0511/1260] Added executable function integrations tests --- .../ExecutableDictionarySource.cpp | 7 ++- .../ExecutablePoolDictionarySource.cpp | 3 +- src/Processors/Sources/ShellCommandSource.cpp | 3 +- tests/config/executable_pool_dictionary.xml | 5 +- .../__init__.py | 0 ...ecutable_user_defined_functions_config.xml | 2 + .../functions/test_function_config.xml | 59 ++++++++++++++++++ .../functions/test_function_config2.xml | 14 +++++ .../test.py | 61 +++++++++++++++++++ .../user_scripts/test_input.py | 7 +++ .../user_scripts/test_input.sh | 5 ++ .../user_scripts/test_input_sum.py | 9 +++ .../functions/test_function_config.xml | 3 +- .../functions/test_function_config2.xml | 3 +- .../test.py | 2 + .../user_scripts/test_input_1.sh | 5 ++ .../user_scripts/test_input_2.sh | 5 ++ 17 files changed, 184 insertions(+), 9 deletions(-) create mode 100644 tests/integration/test_executable_user_defined_function/__init__.py create mode 100644 tests/integration/test_executable_user_defined_function/config/executable_user_defined_functions_config.xml create mode 100644 tests/integration/test_executable_user_defined_function/functions/test_function_config.xml create mode 100644 tests/integration/test_executable_user_defined_function/functions/test_function_config2.xml create mode 100644 tests/integration/test_executable_user_defined_function/test.py create mode 100755 tests/integration/test_executable_user_defined_function/user_scripts/test_input.py create mode 100755 tests/integration/test_executable_user_defined_function/user_scripts/test_input.sh create mode 100755 tests/integration/test_executable_user_defined_function/user_scripts/test_input_sum.py create mode 100755 tests/integration/test_executable_user_defined_functions_config_reload/user_scripts/test_input_1.sh create mode 100755 tests/integration/test_executable_user_defined_functions_config_reload/user_scripts/test_input_2.sh diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index a4837027508..4f210d2e040 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -116,7 +116,12 @@ Pipe ExecutableDictionarySource::getStreamForBlock(const Block & block) Pipes shell_input_pipes; shell_input_pipes.emplace_back(std::move(shell_input_pipe)); - return coordinator->createPipe(configuration.command, std::move(shell_input_pipes), sample_block, context); + auto pipe = coordinator->createPipe(configuration.command, std::move(shell_input_pipes), sample_block, context); + + if (configuration.implicit_key) + pipe.addTransform(std::make_shared(block, pipe.getHeader())); + + return pipe; } bool ExecutableDictionarySource::isModified() const diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 0208791a325..568a11fec8b 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -103,7 +103,8 @@ Pipe ExecutablePoolDictionarySource::getStreamForBlock(const Block & block) Pipes shell_input_pipes; shell_input_pipes.emplace_back(std::move(shell_input_pipe)); - return coordinator->createPipe(configuration.command, std::move(shell_input_pipes), std::move(sample_block), context, command_configuration); + auto pipe = coordinator->createPipe(configuration.command, std::move(shell_input_pipes), sample_block, context, command_configuration); + return pipe; } bool ExecutablePoolDictionarySource::isModified() const diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 21d07ecf126..111ea367bea 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -104,8 +104,9 @@ namespace { rethrowExceptionDuringSendDataIfNeeded(); - if (configuration.read_fixed_number_of_rows && configuration.number_of_rows_to_read >= current_read_rows) + if (configuration.read_fixed_number_of_rows && current_read_rows >= configuration.number_of_rows_to_read) { return {}; + } Chunk chunk; diff --git a/tests/config/executable_pool_dictionary.xml b/tests/config/executable_pool_dictionary.xml index 13f34f0048e..212552a6776 100644 --- a/tests/config/executable_pool_dictionary.xml +++ b/tests/config/executable_pool_dictionary.xml @@ -61,10 +61,11 @@ - + TabSeparated while read read_data; do printf "$read_data\tvalue a\tvalue b\n"; done - + 5 + diff --git a/tests/integration/test_executable_user_defined_function/__init__.py b/tests/integration/test_executable_user_defined_function/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_executable_user_defined_function/config/executable_user_defined_functions_config.xml b/tests/integration/test_executable_user_defined_function/config/executable_user_defined_functions_config.xml new file mode 100644 index 00000000000..3cbf717bb67 --- /dev/null +++ b/tests/integration/test_executable_user_defined_function/config/executable_user_defined_functions_config.xml @@ -0,0 +1,2 @@ + + diff --git a/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml new file mode 100644 index 00000000000..18a1c055238 --- /dev/null +++ b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml @@ -0,0 +1,59 @@ + + + executable + test_function_bash + String + + UInt64 + + TabSeparated + test_input.sh + + + + executable_pool + test_function_pool_bash + String + + UInt64 + + TabSeparated + test_input.sh + + + + executable + test_function_python + String + + UInt64 + + TabSeparated + test_input.py + + + + executable + test_function_sum_python + String + + UInt64 + + + UInt64 + + TabSeparated + test_input_sum.py + + + + executable_pool + test_function_pool_python + String + + UInt64 + + TabSeparated + test_input.py + + diff --git a/tests/integration/test_executable_user_defined_function/functions/test_function_config2.xml b/tests/integration/test_executable_user_defined_function/functions/test_function_config2.xml new file mode 100644 index 00000000000..fe02146a6b8 --- /dev/null +++ b/tests/integration/test_executable_user_defined_function/functions/test_function_config2.xml @@ -0,0 +1,14 @@ + + + executable + test_function_2 + String + + UInt64 + + TabSeparated + while read read_data; do printf "Key_2 $read_data\n"; done + 0 + + + diff --git a/tests/integration/test_executable_user_defined_function/test.py b/tests/integration/test_executable_user_defined_function/test.py new file mode 100644 index 00000000000..9872668a901 --- /dev/null +++ b/tests/integration/test_executable_user_defined_function/test.py @@ -0,0 +1,61 @@ +import os +import sys +import time + +import pytest + +sys.path.insert(0, os.path.dirname(os.path.dirname(os.path.abspath(__file__)))) +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance('node', stay_alive=True, main_configs=[]) + + +def skip_test_msan(instance): + if instance.is_built_with_memory_sanitizer(): + pytest.skip("Memory Sanitizer cannot work with vfork") + +def copy_file_to_container(local_path, dist_path, container_id): + os.system("docker cp {local} {cont_id}:{dist}".format(local=local_path, cont_id=container_id, dist=dist_path)) + +config = ''' + /etc/clickhouse-server/functions/test_function_config.xml +''' + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + node.replace_config("/etc/clickhouse-server/config.d/executable_user_defined_functions_config.xml", config) + + copy_file_to_container(os.path.join(SCRIPT_DIR, 'functions/.'), '/etc/clickhouse-server/functions', node.docker_id) + copy_file_to_container(os.path.join(SCRIPT_DIR, 'user_scripts/.'), '/var/lib/clickhouse/user_scripts', node.docker_id) + + node.restart_clickhouse() + + yield cluster + + finally: + cluster.shutdown() + +def test_executable_function_bash(started_cluster): + skip_test_msan(node) + assert node.query("SELECT test_function_bash(toUInt64(1))") == 'Key 1\n' + assert node.query("SELECT test_function_bash(1)") == 'Key 1\n' + + assert node.query("SELECT test_function_pool_bash(toUInt64(1))") == 'Key 1\n' + assert node.query("SELECT test_function_pool_bash(1)") == 'Key 1\n' + +def test_executable_function_python(started_cluster): + skip_test_msan(node) + assert node.query("SELECT test_function_python(toUInt64(1))") == 'Key 1\n' + assert node.query("SELECT test_function_python(1)") == 'Key 1\n' + + assert node.query("SELECT test_function_sum_python(toUInt64(1), toUInt64(1))") == '2\n' + assert node.query("SELECT test_function_sum_python(1, 1)") == '2\n' + + # assert node.query("SELECT test_function_pool_python(toUInt64(1))") == 'Key 1\n' + # assert node.query("SELECT test_function_pool_python(1)") == 'Key 1\n' diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/test_input.py b/tests/integration/test_executable_user_defined_function/user_scripts/test_input.py new file mode 100755 index 00000000000..bdf95a9d3c4 --- /dev/null +++ b/tests/integration/test_executable_user_defined_function/user_scripts/test_input.py @@ -0,0 +1,7 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + for line in sys.stdin: + print("Key " + line, end='') diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/test_input.sh b/tests/integration/test_executable_user_defined_function/user_scripts/test_input.sh new file mode 100755 index 00000000000..aea51b82b1f --- /dev/null +++ b/tests/integration/test_executable_user_defined_function/user_scripts/test_input.sh @@ -0,0 +1,5 @@ +#!/bin/bash + +while read read_data; + do printf "Key $read_data\n"; +done diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/test_input_sum.py b/tests/integration/test_executable_user_defined_function/user_scripts/test_input_sum.py new file mode 100755 index 00000000000..a3580850f57 --- /dev/null +++ b/tests/integration/test_executable_user_defined_function/user_scripts/test_input_sum.py @@ -0,0 +1,9 @@ +#!/usr/bin/python3 + +import sys +import re + +if __name__ == '__main__': + for line in sys.stdin: + line_split = re.split(r'\t+', line) + print(int(line_split[0]) + int(line_split[1]), end='') diff --git a/tests/integration/test_executable_user_defined_functions_config_reload/functions/test_function_config.xml b/tests/integration/test_executable_user_defined_functions_config_reload/functions/test_function_config.xml index f2a7d6e67b1..d0bd6e5ab88 100644 --- a/tests/integration/test_executable_user_defined_functions_config_reload/functions/test_function_config.xml +++ b/tests/integration/test_executable_user_defined_functions_config_reload/functions/test_function_config.xml @@ -7,8 +7,7 @@ UInt64 TabSeparated - while read read_data; do printf "Key_1 $read_data\n"; done - 0 + test_input_1.sh diff --git a/tests/integration/test_executable_user_defined_functions_config_reload/functions/test_function_config2.xml b/tests/integration/test_executable_user_defined_functions_config_reload/functions/test_function_config2.xml index fe02146a6b8..80ae21a086d 100644 --- a/tests/integration/test_executable_user_defined_functions_config_reload/functions/test_function_config2.xml +++ b/tests/integration/test_executable_user_defined_functions_config_reload/functions/test_function_config2.xml @@ -7,8 +7,7 @@ UInt64 TabSeparated - while read read_data; do printf "Key_2 $read_data\n"; done - 0 + test_input_2.sh
diff --git a/tests/integration/test_executable_user_defined_functions_config_reload/test.py b/tests/integration/test_executable_user_defined_functions_config_reload/test.py index 3117b3e72b1..629c426a28c 100644 --- a/tests/integration/test_executable_user_defined_functions_config_reload/test.py +++ b/tests/integration/test_executable_user_defined_functions_config_reload/test.py @@ -28,6 +28,8 @@ def started_cluster(): cluster.start() copy_file_to_container(os.path.join(SCRIPT_DIR, 'functions/.'), '/etc/clickhouse-server/functions', node.docker_id) + copy_file_to_container(os.path.join(SCRIPT_DIR, 'user_scripts/.'), '/var/lib/clickhouse/user_scripts', node.docker_id) + node.restart_clickhouse() yield cluster diff --git a/tests/integration/test_executable_user_defined_functions_config_reload/user_scripts/test_input_1.sh b/tests/integration/test_executable_user_defined_functions_config_reload/user_scripts/test_input_1.sh new file mode 100755 index 00000000000..a6cffe83bba --- /dev/null +++ b/tests/integration/test_executable_user_defined_functions_config_reload/user_scripts/test_input_1.sh @@ -0,0 +1,5 @@ +#!/bin/bash + +while read read_data; + do printf "Key_1 $read_data\n"; +done diff --git a/tests/integration/test_executable_user_defined_functions_config_reload/user_scripts/test_input_2.sh b/tests/integration/test_executable_user_defined_functions_config_reload/user_scripts/test_input_2.sh new file mode 100755 index 00000000000..a673cfd18fb --- /dev/null +++ b/tests/integration/test_executable_user_defined_functions_config_reload/user_scripts/test_input_2.sh @@ -0,0 +1,5 @@ +#!/bin/bash + +while read read_data; + do printf "Key_2 $read_data\n"; +done From 5590cfa3aab47d8cdfdd5a39f0a2f8862c1a7402 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 24 Dec 2021 13:39:48 +0300 Subject: [PATCH 0512/1260] Updated executable function integration tests --- src/Common/ShellCommand.cpp | 20 +- .../ExecutableDictionarySource.cpp | 3 + .../ExecutablePoolDictionarySource.cpp | 6 +- ...alUserDefinedExecutableFunctionsLoader.cpp | 33 +- .../UserDefinedExecutableFunction.h | 4 +- .../UserDefinedExecutableFunctionFactory.cpp | 42 +-- src/Processors/Sources/ShellCommandSource.cpp | 306 ++++++++++++++++-- src/Processors/Sources/ShellCommandSource.h | 21 +- src/Storages/ExecutableSettings.h | 6 +- src/Storages/StorageExecutable.cpp | 6 +- .../functions/test_function_config.xml | 117 ++++++- .../functions/test_function_config2.xml | 14 - .../test.py | 49 ++- .../user_scripts/test_input.py | 1 + .../user_scripts/test_input_argument.py | 10 + .../user_scripts/test_input_chunk_header.py | 14 + .../user_scripts/test_input_signalled.py | 13 + .../user_scripts/test_input_slow.py | 12 + .../user_scripts/test_input_sum.py | 3 +- 19 files changed, 585 insertions(+), 95 deletions(-) delete mode 100644 tests/integration/test_executable_user_defined_function/functions/test_function_config2.xml create mode 100755 tests/integration/test_executable_user_defined_function/user_scripts/test_input_argument.py create mode 100755 tests/integration/test_executable_user_defined_function/user_scripts/test_input_chunk_header.py create mode 100755 tests/integration/test_executable_user_defined_function/user_scripts/test_input_signalled.py create mode 100755 tests/integration/test_executable_user_defined_function/user_scripts/test_input_slow.py diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index 99461862ef9..0093d72e766 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -65,14 +65,14 @@ ShellCommand::~ShellCommand() size_t try_wait_timeout = config.terminate_in_destructor_strategy.wait_for_normal_exit_before_termination_seconds; bool process_terminated_normally = tryWaitProcessWithTimeout(try_wait_timeout); - if (!process_terminated_normally) - { - LOG_TRACE(getLogger(), "Will kill shell command pid {} with SIGTERM", pid); + if (process_terminated_normally) + return; - int retcode = kill(pid, SIGTERM); - if (retcode != 0) - LOG_WARNING(getLogger(), "Cannot kill shell command pid {} errno '{}'", pid, errnoToString(retcode)); - } + LOG_TRACE(getLogger(), "Will kill shell command pid {} with SIGTERM", pid); + + int retcode = kill(pid, SIGTERM); + if (retcode != 0) + LOG_WARNING(getLogger(), "Cannot kill shell command pid {} errno '{}'", pid, errnoToString(retcode)); } else { @@ -91,7 +91,7 @@ bool ShellCommand::tryWaitProcessWithTimeout(size_t timeout_in_seconds) { int status = 0; - LOG_TRACE(getLogger(), "Try wait for shell command pid ({}) with timeout ({})", pid, timeout_in_seconds); + LOG_TRACE(getLogger(), "Try wait for shell command pid {} with timeout {}", pid, timeout_in_seconds); wait_called = true; struct timespec interval {.tv_sec = 1, .tv_nsec = 0}; @@ -119,7 +119,9 @@ bool ShellCommand::tryWaitProcessWithTimeout(size_t timeout_in_seconds) bool process_terminated_normally = (waitpid_res == pid); if (process_terminated_normally) + { return true; + } else if (waitpid_res == 0) { --timeout_in_seconds; @@ -128,7 +130,9 @@ bool ShellCommand::tryWaitProcessWithTimeout(size_t timeout_in_seconds) continue; } else if (waitpid_res == -1 && errno != EINTR) + { return false; + } } return false; diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 4f210d2e040..b801ad3ab04 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -183,6 +183,9 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) ShellCommandCoordinator::Configuration shell_command_coordinator_configration { .format = config.getString(settings_config_prefix + ".format"), + .command_termination_timeout_seconds = config.getUInt64(settings_config_prefix + ".command_termination_timeout", 10), + .command_read_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_read_timeout", 10000), + .command_write_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_write_timeout", 10000), .is_executable_pool = false, .send_chunk_header = config.getBool(settings_config_prefix + ".send_chunk_header", false), .execute_direct = false diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 568a11fec8b..7c010c1a264 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -171,9 +171,11 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) ShellCommandCoordinator::Configuration shell_command_coordinator_configration { .format = config.getString(settings_config_prefix + ".format"), + .command_termination_timeout_seconds = config.getUInt64(settings_config_prefix + ".command_termination_timeout", 10), + .command_read_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_read_timeout", 10000), + .command_write_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_write_timeout", 10000), .pool_size = config.getUInt64(settings_config_prefix + ".size"), - .command_termination_timeout = config.getUInt64(settings_config_prefix + ".command_termination_timeout", 10), - .max_command_execution_time = max_command_execution_time, + .max_command_execution_time_seconds = max_command_execution_time, .is_executable_pool = true, .send_chunk_header = config.getBool(settings_config_prefix + ".send_chunk_header", false), .execute_direct = false diff --git a/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp b/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp index 593a29d5688..6738b26c924 100644 --- a/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp +++ b/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp @@ -68,25 +68,32 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create "Wrong user defined function type expected 'executable' or 'executable_pool' actual {}", type); - String scipt_name_with_arguments_value = config.getString(key_in_config + ".command"); + bool execute_direct = config.getBool(key_in_config + ".execute_direct", true); - std::vector script_name_with_arguments; - boost::split(script_name_with_arguments, scipt_name_with_arguments_value, [](char c) { return c == ' '; }); + String command_value = config.getString(key_in_config + ".command"); + std::vector command_arguments; - auto script_name = script_name_with_arguments[0]; - script_name_with_arguments.erase(script_name_with_arguments.begin()); + if (execute_direct) + { + boost::split(command_arguments, command_value, [](char c) { return c == ' '; }); + + command_value = std::move(command_arguments[0]); + command_arguments.erase(command_arguments.begin()); + } String format = config.getString(key_in_config + ".format"); DataTypePtr result_type = DataTypeFactory::instance().get(config.getString(key_in_config + ".return_type")); bool send_chunk_header = config.getBool(key_in_config + ".send_chunk_header", false); + size_t command_termination_timeout_seconds = config.getUInt64(key_in_config + ".command_termination_timeout", 10); + size_t command_read_timeout_milliseconds = config.getUInt64(key_in_config + ".command_read_timeout", 10000); + size_t command_write_timeout_milliseconds = config.getUInt64(key_in_config + ".command_write_timeout", 10000); size_t pool_size = 0; - size_t command_termination_timeout = 0; size_t max_command_execution_time = 0; + if (is_executable_pool) { pool_size = config.getUInt64(key_in_config + ".pool_size", 16); - command_termination_timeout = config.getUInt64(key_in_config + ".command_termination_timeout", 10); max_command_execution_time = config.getUInt64(key_in_config + ".max_command_execution_time", 10); size_t max_execution_time_seconds = static_cast(getContext()->getSettings().max_execution_time.totalSeconds()); @@ -117,8 +124,8 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create UserDefinedExecutableFunctionConfiguration function_configuration { .name = std::move(name), //-V1030 - .script_name = std::move(script_name), //-V1030 - .script_arguments = std::move(script_name_with_arguments), //-V1030 + .command = std::move(command_value), //-V1030 + .command_arguments = std::move(command_arguments), //-V1030 .argument_types = std::move(argument_types), //-V1030 .result_type = std::move(result_type), //-V1030 }; @@ -126,12 +133,14 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create ShellCommandCoordinator::Configuration shell_command_coordinator_configration { .format = std::move(format), //-V1030 + .command_termination_timeout_seconds = command_termination_timeout_seconds, + .command_read_timeout_milliseconds = command_read_timeout_milliseconds, + .command_write_timeout_milliseconds = command_write_timeout_milliseconds, .pool_size = pool_size, - .command_termination_timeout = command_termination_timeout, - .max_command_execution_time = max_command_execution_time, + .max_command_execution_time_seconds = max_command_execution_time, .is_executable_pool = is_executable_pool, .send_chunk_header = send_chunk_header, - .execute_direct = true + .execute_direct = execute_direct }; std::shared_ptr coordinator = std::make_shared(shell_command_coordinator_configration); diff --git a/src/Interpreters/UserDefinedExecutableFunction.h b/src/Interpreters/UserDefinedExecutableFunction.h index 92a4c0985c5..f02a07b44a1 100644 --- a/src/Interpreters/UserDefinedExecutableFunction.h +++ b/src/Interpreters/UserDefinedExecutableFunction.h @@ -13,8 +13,8 @@ namespace DB struct UserDefinedExecutableFunctionConfiguration { std::string name; - std::string script_name; - std::vector script_arguments; + std::string command; + std::vector command_arguments; std::vector argument_types; DataTypePtr result_type; }; diff --git a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp index 41fc3fe4792..0cffd61eaf6 100644 --- a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp +++ b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp @@ -56,28 +56,36 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { + auto coordinator = executable_function->getCoordinator(); + const auto & coordinator_configuration = coordinator->getConfiguration(); const auto & configuration = executable_function->getConfiguration(); - auto user_scripts_path = context->getUserScriptsPath(); - const auto & script_name = configuration.script_name; + String command = configuration.command; - auto script_path = user_scripts_path + '/' + script_name; + if (coordinator_configuration.execute_direct) + { + auto user_scripts_path = context->getUserScriptsPath(); + auto script_path = user_scripts_path + '/' + command; - if (!fileOrSymlinkPathStartsWith(script_path, user_scripts_path)) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Executable file {} must be inside user scripts folder {}", - script_name, - user_scripts_path); + if (!fileOrSymlinkPathStartsWith(script_path, user_scripts_path)) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Executable file {} must be inside user scripts folder {}", + command, + user_scripts_path); - if (!std::filesystem::exists(std::filesystem::path(script_path))) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Executable file {} does not exist inside user scripts folder {}", - script_name, - user_scripts_path); + if (!std::filesystem::exists(std::filesystem::path(script_path))) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Executable file {} does not exist inside user scripts folder {}", + command, + user_scripts_path); + command = std::move(script_path); + } + + size_t argument_size = arguments.size(); auto arguments_copy = arguments; - for (size_t i = 0; i < arguments.size(); ++i) + for (size_t i = 0; i < argument_size; ++i) { auto & column_with_type = arguments_copy[i]; column_with_type.column = column_with_type.column->convertToFullColumnIfConst(); @@ -100,8 +108,6 @@ public: auto source = std::make_shared(std::move(arguments_block)); auto shell_input_pipe = Pipe(std::move(source)); - auto coordinator = executable_function->getCoordinator(); - ShellCommandSourceConfiguration shell_command_source_configuration; if (coordinator->getConfiguration().is_executable_pool) @@ -114,8 +120,8 @@ public: shell_input_pipes.emplace_back(std::move(shell_input_pipe)); Pipe pipe = coordinator->createPipe( - script_path, - configuration.script_arguments, + command, + configuration.command_arguments, std::move(shell_input_pipes), result_block, context, diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 111ea367bea..34f244d0d9e 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -1,5 +1,9 @@ #include +#include + +#include + #include #include @@ -17,8 +21,222 @@ namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; extern const int TIMEOUT_EXCEEDED; + extern const int CANNOT_FCNTL; + extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR; + extern const int CANNOT_SELECT; + extern const int CANNOT_CLOSE_FILE; + extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR; } +static bool tryMakeFdNonBlocking(int fd) +{ + int flags = fcntl(fd, F_GETFL, 0); + if (-1 == flags) + return false; + if (-1 == fcntl(fd, F_SETFL, flags | O_NONBLOCK)) + return false; + + return true; +} + +static void makeFdNonBlocking(int fd) +{ + bool result = tryMakeFdNonBlocking(fd); + if (!result) + throwFromErrno("Cannot set non-blocking mode of pipe", ErrorCodes::CANNOT_FCNTL); +} + +static bool tryMakeFdBlocking(int fd) +{ + int flags = fcntl(fd, F_GETFL, 0); + if (-1 == flags) + return false; + + if (-1 == fcntl(fd, F_SETFL, flags & (~O_NONBLOCK))) + return false; + + return true; +} + +static void makeFdBlocking(int fd) +{ + bool result = tryMakeFdBlocking(fd); + if (!result) + throwFromErrno("Cannot set blocking mode of pipe", ErrorCodes::CANNOT_FCNTL); +} + +static bool pollFd(int fd, size_t timeout_milliseconds, int events) +{ + pollfd pfd; + pfd.fd = fd; + pfd.events = events; + pfd.revents = 0; + + Stopwatch watch; + + int res; + + while (true) + { + res = poll(&pfd, 1, timeout_milliseconds); + + if (res < 0) + { + if (errno == EINTR) + { + watch.stop(); + timeout_milliseconds -= watch.elapsedMilliseconds(); + watch.start(); + + continue; + } + else + { + throwFromErrno("Cannot select", ErrorCodes::CANNOT_SELECT); + } + } + else + { + break; + } + } + + return res > 0; +} + +class TimeoutReadBufferFromFileDescriptor : public BufferWithOwnMemory +{ +public: + explicit TimeoutReadBufferFromFileDescriptor(int fd_, size_t timeout_milliseconds_) + : BufferWithOwnMemory() + , fd(fd_) + , timeout_milliseconds(timeout_milliseconds_) + { + makeFdNonBlocking(fd); + } + + bool nextImpl() override + { + size_t bytes_read = 0; + + while (!bytes_read) + { + if (!pollFd(fd, timeout_milliseconds, POLLIN)) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Pipe read timeout exceeded {} milliseconds", timeout_milliseconds); + + ssize_t res = ::read(fd, internal_buffer.begin(), internal_buffer.size()); + + if (-1 == res && errno != EINTR) + throwFromErrno("Cannot read from pipe ", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR); + + if (res == 0) { + break; + } + + if (res > 0) + bytes_read += res; + } + + if (bytes_read > 0) { + working_buffer = internal_buffer; + working_buffer.resize(bytes_read); + } + else + { + return false; + } + + return true; + } + + void reset() const + { + makeFdBlocking(fd); + } + + ~TimeoutReadBufferFromFileDescriptor() override + { + tryMakeFdBlocking(fd); + } + +private: + int fd; + size_t timeout_milliseconds; +}; + +class TimeoutWriteBufferFromFileDescriptor : public BufferWithOwnMemory +{ +public: + explicit TimeoutWriteBufferFromFileDescriptor(int fd_, size_t timeout_milliseconds_) + : BufferWithOwnMemory() + , fd(fd_) + , timeout_milliseconds(timeout_milliseconds_) + { + makeFdNonBlocking(fd); + } + + void nextImpl() override + { + if (!offset()) + return; + + size_t bytes_written = 0; + + while (bytes_written != offset()) + { + if (!pollFd(fd, timeout_milliseconds, POLLOUT)) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Pipe write timeout exceeded {} milliseconds", timeout_milliseconds); + + ssize_t res = ::write(fd, working_buffer.begin() + bytes_written, offset() - bytes_written); + + if ((-1 == res || 0 == res) && errno != EINTR) + throwFromErrno("Cannot write into pipe ", ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR); + + if (res > 0) + bytes_written += res; + } + } + + void reset() const { + makeFdBlocking(fd); + } + + ~TimeoutWriteBufferFromFileDescriptor() override + { + tryMakeFdBlocking(fd); + } + +private: + int fd; + size_t timeout_milliseconds; +}; + +class ShellCommandHolder +{ +public: + using ShellCommandBuilderFunc = std::function()>; + + explicit ShellCommandHolder(ShellCommandBuilderFunc && func_) + : func(std::move(func_)) + {} + + std::unique_ptr buildCommand() { + if (returned_command) { + return std::move(returned_command); + } + + return func(); + } + + void returnCommand(std::unique_ptr command) { + returned_command = std::move(command); + } + +private: + std::unique_ptr returned_command; + ShellCommandBuilderFunc func; +}; + namespace { /** A stream, that get child process and sends data using tasks in background threads. @@ -36,14 +254,18 @@ namespace ShellCommandSource( ContextPtr context, const std::string & format, + size_t command_read_timeout_milliseconds, const Block & sample_block, std::unique_ptr && command_, std::vector && send_data_tasks = {}, const ShellCommandSourceConfiguration & configuration_ = {}, + std::unique_ptr && command_holder_ = nullptr, std::shared_ptr process_pool_ = nullptr) : SourceWithProgress(sample_block) , command(std::move(command_)) , configuration(configuration_) + , timeout_command_out(command->out.getFD(), command_read_timeout_milliseconds) + , command_holder(std::move(command_holder_)) , process_pool(process_pool_) { for (auto && send_data_task : send_data_tasks) @@ -84,7 +306,7 @@ namespace max_block_size = configuration.number_of_rows_to_read; } - pipeline = QueryPipeline(Pipe(context->getInputFormat(format, command->out, sample_block, max_block_size))); + pipeline = QueryPipeline(Pipe(context->getInputFormat(format, timeout_command_out, sample_block, max_block_size))); executor = std::make_unique(pipeline); } @@ -94,8 +316,19 @@ namespace if (thread.joinable()) thread.join(); - if (command && process_pool) - process_pool->returnObject(std::move(command)); + if (command_is_invalid) { + command = nullptr; + } + + if (command_holder && process_pool) { + bool valid_command = configuration.read_fixed_number_of_rows && current_read_rows >= configuration.number_of_rows_to_read; + + if (command && valid_command) { + command_holder->returnCommand(std::move(command)); + } + + process_pool->returnObject(std::move(command_holder)); + } } protected: @@ -119,7 +352,7 @@ namespace } catch (...) { - command = nullptr; + command_is_invalid = true; throw; } @@ -159,22 +392,28 @@ namespace std::unique_ptr command; ShellCommandSourceConfiguration configuration; + TimeoutReadBufferFromFileDescriptor timeout_command_out; + size_t current_read_rows = 0; + ShellCommandHolderPtr command_holder; std::shared_ptr process_pool; QueryPipeline pipeline; std::unique_ptr executor; std::vector send_data_threads; + std::mutex send_data_lock; std::exception_ptr exception_during_send_data; + + std::atomic command_is_invalid {false}; }; class SendingChunkHeaderTransform final : public ISimpleTransform { public: - SendingChunkHeaderTransform(const Block & header, WriteBuffer & buffer_) + SendingChunkHeaderTransform(const Block & header, std::shared_ptr buffer_) : ISimpleTransform(header, header, false) , buffer(buffer_) { @@ -186,12 +425,12 @@ namespace void transform(Chunk & chunk) override { - writeText(chunk.getNumRows(), buffer); - writeChar('\n', buffer); + writeText(chunk.getNumRows(), *buffer); + writeChar('\n', *buffer); } private: - WriteBuffer & buffer; + std::shared_ptr buffer; }; } @@ -217,29 +456,39 @@ Pipe ShellCommandCoordinator::createPipe( command_config.write_fds.emplace_back(i + 2); std::unique_ptr process; + std::unique_ptr process_holder; + + auto destructor_strategy = ShellCommand::DestructorStrategy{true /*terminate_in_destructor*/, configuration.command_termination_timeout_seconds}; + command_config.terminate_in_destructor_strategy = destructor_strategy; bool is_executable_pool = (process_pool != nullptr); if (is_executable_pool) { - bool result = process_pool->tryBorrowObject( - process, - [&command_config, this]() - { - command_config.terminate_in_destructor_strategy - = ShellCommand::DestructorStrategy{true /*terminate_in_destructor*/, configuration.command_termination_timeout}; + bool execute_direct = configuration.execute_direct; - if (configuration.execute_direct) - return ShellCommand::executeDirect(command_config); - else - return ShellCommand::execute(command_config); + bool result = process_pool->tryBorrowObject( + process_holder, + [command_config, execute_direct]() + { + ShellCommandHolder::ShellCommandBuilderFunc func = [command_config, execute_direct]() mutable + { + if (execute_direct) + return ShellCommand::executeDirect(command_config); + else + return ShellCommand::execute(command_config); + }; + + return std::make_unique(std::move(func)); }, - configuration.max_command_execution_time * 10000); + configuration.max_command_execution_time_seconds * 10000); if (!result) throw Exception( ErrorCodes::TIMEOUT_EXCEEDED, "Could not get process from pool, max command execution timeout exceeded {} seconds", - configuration.max_command_execution_time); + configuration.max_command_execution_time_seconds); + + process = process_holder->buildCommand(); } else { @@ -270,32 +519,41 @@ Pipe ShellCommandCoordinator::createPipe( write_buffer = &it->second; } + int write_buffer_fd = write_buffer->getFD(); + auto timeout_write_buffer = std::make_shared(write_buffer_fd, configuration.command_write_timeout_milliseconds); + input_pipes[i].resize(1); if (configuration.send_chunk_header) { - auto transform = std::make_shared(input_pipes[i].getHeader(), *write_buffer); + auto transform = std::make_shared(input_pipes[i].getHeader(), timeout_write_buffer); input_pipes[i].addTransform(std::move(transform)); } auto pipeline = std::make_shared(std::move(input_pipes[i])); - auto out = context->getOutputFormat(configuration.format, *write_buffer, materializeBlock(pipeline->getHeader())); + auto out = context->getOutputFormat(configuration.format, *timeout_write_buffer, materializeBlock(pipeline->getHeader())); out->setAutoFlush(); pipeline->complete(std::move(out)); - ShellCommandSource::SendDataTask task = [pipeline, write_buffer, is_executable_pool]() + ShellCommandSource::SendDataTask task = [pipeline, timeout_write_buffer, write_buffer, is_executable_pool]() { CompletedPipelineExecutor executor(*pipeline); executor.execute(); if (!is_executable_pool) + { + timeout_write_buffer->next(); + timeout_write_buffer->reset(); + write_buffer->close(); + } }; tasks.emplace_back(std::move(task)); } - Pipe pipe(std::make_unique(context, configuration.format, std::move(sample_block), std::move(process), std::move(tasks), source_configuration, process_pool)); + Pipe pipe(std::make_unique( + context, configuration.format, configuration.command_read_timeout_milliseconds, std::move(sample_block), std::move(process), std::move(tasks), source_configuration, std::move(process_holder), process_pool)); return pipe; } diff --git a/src/Processors/Sources/ShellCommandSource.h b/src/Processors/Sources/ShellCommandSource.h index aa18a289226..5d5055f29e2 100644 --- a/src/Processors/Sources/ShellCommandSource.h +++ b/src/Processors/Sources/ShellCommandSource.h @@ -19,7 +19,10 @@ namespace DB { -using ProcessPool = BorrowedObjectPool>; +class ShellCommandHolder; +using ShellCommandHolderPtr = std::unique_ptr; + +using ProcessPool = BorrowedObjectPool; struct ShellCommandSourceConfiguration { @@ -43,14 +46,20 @@ public: /// Script output format std::string format; + /// Command termination timeout in seconds + size_t command_termination_timeout_seconds = 10; + + /// Timeout for reading data from command stdout + size_t command_read_timeout_milliseconds = 10000; + + /// Timeout for writing data to command stdin + size_t command_write_timeout_milliseconds = 10000; + /// Pool size valid only if executable_pool = true size_t pool_size = 16; - /// Command termination timeout in seconds. Valid only if executable_pool = true - size_t command_termination_timeout = 10; - - /// Max command execution time in seconds. Valid only if executable_pool = true - size_t max_command_execution_time = 10; + /// Max command execution time in milliseconds. Valid only if executable_pool = true + size_t max_command_execution_time_seconds = 10; /// Should pool of processes be created. bool is_executable_pool = false; diff --git a/src/Storages/ExecutableSettings.h b/src/Storages/ExecutableSettings.h index 873ff3f45a7..c6c1f0b9eb2 100644 --- a/src/Storages/ExecutableSettings.h +++ b/src/Storages/ExecutableSettings.h @@ -9,10 +9,12 @@ namespace DB class ASTStorage; #define LIST_OF_EXECUTABLE_SETTINGS(M) \ - M(Bool, send_chunk_header, false, "Send number_of_rows\n before sending chunk to process", 0) \ - M(UInt64, pool_size, 16, "Processes pool size. If size == 0, then there is no size restrictions", 0) \ + M(Bool, send_chunk_header, false, "Send number_of_rows\n before sending chunk to process.", 0) \ + M(UInt64, pool_size, 16, "Processes pool size. If size == 0, then there is no size restrictions.", 0) \ M(UInt64, max_command_execution_time, 10, "Max command execution time in seconds.", 0) \ M(UInt64, command_termination_timeout, 10, "Command termination timeout in seconds.", 0) \ + M(UInt64, command_read_timeout, 10000, "Timeout for reading data from command stdout in milliseconds.", 0) \ + M(UInt64, command_write_timeout, 10000, "Timeout for writing data to command stdin in milliseconds.", 0) DECLARE_SETTINGS_TRAITS(ExecutableSettingsTraits, LIST_OF_EXECUTABLE_SETTINGS) diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 98c9b276497..6a99a088571 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -53,10 +53,12 @@ StorageExecutable::StorageExecutable( ShellCommandCoordinator::Configuration configuration { .format = format, + .command_termination_timeout_seconds = settings.command_termination_timeout, + .command_read_timeout_milliseconds = settings.command_read_timeout, + .command_write_timeout_milliseconds = settings.command_write_timeout, .pool_size = settings.pool_size, - .command_termination_timeout = settings.command_termination_timeout, - .max_command_execution_time = settings.max_command_execution_time, + .max_command_execution_time_seconds = settings.max_command_execution_time, .is_executable_pool = settings.is_executable_pool, .send_chunk_header = settings.send_chunk_header, diff --git a/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml index 18a1c055238..2f07082696b 100644 --- a/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml +++ b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml @@ -32,6 +32,41 @@ test_input.py + + executable_pool + test_function_pool_python + String + + UInt64 + + TabSeparated + test_input.py + + + + executable + test_function_send_chunk_header_python + String + + UInt64 + + TabSeparated + 1 + test_input_chunk_header.py + + + + executable_pool + test_function_send_chunk_header_pool_python + String + + UInt64 + + TabSeparated + 1 + test_input_chunk_header.py + + executable test_function_sum_python @@ -48,12 +83,90 @@ executable_pool - test_function_pool_python + test_function_sum_pool_python + String + + UInt64 + + + UInt64 + + TabSeparated + test_input_sum.py + + + + executable + test_function_argument_python String UInt64 TabSeparated - test_input.py + test_input_argument.py 1 + + + executable_pool + test_function_argument_pool_python + String + + UInt64 + + TabSeparated + test_input_argument.py 1 + + + + executable + test_function_slow_python + String + + UInt64 + + TabSeparated + test_input_slow.py + 1 + 1000 + + + + executable_pool + test_function_slow_pool_python + String + + UInt64 + + TabSeparated + test_input_slow.py + 1 + 1000 + + + + executable + test_function_signalled_python + String + + UInt64 + + TabSeparated + test_input_signalled.py + 1 + 1000 + + + + executable_pool + test_function_signalled_pool_python + String + + UInt64 + + TabSeparated + test_input_signalled.py + 1 + 1000 + +
diff --git a/tests/integration/test_executable_user_defined_function/functions/test_function_config2.xml b/tests/integration/test_executable_user_defined_function/functions/test_function_config2.xml deleted file mode 100644 index fe02146a6b8..00000000000 --- a/tests/integration/test_executable_user_defined_function/functions/test_function_config2.xml +++ /dev/null @@ -1,14 +0,0 @@ - - - executable - test_function_2 - String - - UInt64 - - TabSeparated - while read read_data; do printf "Key_2 $read_data\n"; done - 0 - - - diff --git a/tests/integration/test_executable_user_defined_function/test.py b/tests/integration/test_executable_user_defined_function/test.py index 9872668a901..1899b01f3d6 100644 --- a/tests/integration/test_executable_user_defined_function/test.py +++ b/tests/integration/test_executable_user_defined_function/test.py @@ -54,8 +54,53 @@ def test_executable_function_python(started_cluster): assert node.query("SELECT test_function_python(toUInt64(1))") == 'Key 1\n' assert node.query("SELECT test_function_python(1)") == 'Key 1\n' + assert node.query("SELECT test_function_pool_python(toUInt64(1))") == 'Key 1\n' + assert node.query("SELECT test_function_pool_python(1)") == 'Key 1\n' + +def test_executable_function_send_chunk_header_python(started_cluster): + skip_test_msan(node) + assert node.query("SELECT test_function_send_chunk_header_python(toUInt64(1))") == 'Key 1\n' + assert node.query("SELECT test_function_send_chunk_header_python(1)") == 'Key 1\n' + + assert node.query("SELECT test_function_send_chunk_header_pool_python(toUInt64(1))") == 'Key 1\n' + assert node.query("SELECT test_function_send_chunk_header_pool_python(1)") == 'Key 1\n' + +def test_executable_function_sum_python(started_cluster): + skip_test_msan(node) assert node.query("SELECT test_function_sum_python(toUInt64(1), toUInt64(1))") == '2\n' assert node.query("SELECT test_function_sum_python(1, 1)") == '2\n' - # assert node.query("SELECT test_function_pool_python(toUInt64(1))") == 'Key 1\n' - # assert node.query("SELECT test_function_pool_python(1)") == 'Key 1\n' + assert node.query("SELECT test_function_sum_pool_python(toUInt64(1), toUInt64(1))") == '2\n' + assert node.query("SELECT test_function_sum_pool_python(1, 1)") == '2\n' + +def test_executable_function_argument_python(started_cluster): + skip_test_msan(node) + assert node.query("SELECT test_function_argument_python(toUInt64(1))") == 'Key 1 1\n' + assert node.query("SELECT test_function_argument_python(1)") == 'Key 1 1\n' + + assert node.query("SELECT test_function_argument_pool_python(toUInt64(1))") == 'Key 1 1\n' + assert node.query("SELECT test_function_argument_pool_python(1)") == 'Key 1 1\n' + +def test_executable_function_signalled_python(started_cluster): + skip_test_msan(node) + assert node.query_and_get_error("SELECT test_function_signalled_python(toUInt64(1))") + assert node.query_and_get_error("SELECT test_function_signalled_python(1)") + + assert node.query_and_get_error("SELECT test_function_signalled_pool_python(toUInt64(1))") + assert node.query_and_get_error("SELECT test_function_signalled_pool_python(1)") + +def test_executable_function_slow_python(started_cluster): + skip_test_msan(node) + assert node.query_and_get_error("SELECT test_function_slow_python(toUInt64(1))") + assert node.query_and_get_error("SELECT test_function_slow_python(1)") + + assert node.query_and_get_error("SELECT test_function_slow_pool_python(toUInt64(1))") + assert node.query_and_get_error("SELECT test_function_slow_pool_python(1)") + +def test_executable_function_signalled_python(started_cluster): + skip_test_msan(node) + assert node.query_and_get_error("SELECT test_function_signalled_python(toUInt64(1))") + assert node.query_and_get_error("SELECT test_function_signalled_python(1)") + + assert node.query_and_get_error("SELECT test_function_signalled_pool_python(toUInt64(1))") + assert node.query_and_get_error("SELECT test_function_signalled_pool_python(1)") diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/test_input.py b/tests/integration/test_executable_user_defined_function/user_scripts/test_input.py index bdf95a9d3c4..835ab1f441a 100755 --- a/tests/integration/test_executable_user_defined_function/user_scripts/test_input.py +++ b/tests/integration/test_executable_user_defined_function/user_scripts/test_input.py @@ -5,3 +5,4 @@ import sys if __name__ == '__main__': for line in sys.stdin: print("Key " + line, end='') + sys.stdout.flush() diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/test_input_argument.py b/tests/integration/test_executable_user_defined_function/user_scripts/test_input_argument.py new file mode 100755 index 00000000000..c1b2e5966d7 --- /dev/null +++ b/tests/integration/test_executable_user_defined_function/user_scripts/test_input_argument.py @@ -0,0 +1,10 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + arg = int(sys.argv[1]) + + for line in sys.stdin: + print("Key " + str(arg) + " " + line, end='') + sys.stdout.flush() diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/test_input_chunk_header.py b/tests/integration/test_executable_user_defined_function/user_scripts/test_input_chunk_header.py new file mode 100755 index 00000000000..5dc03e1c507 --- /dev/null +++ b/tests/integration/test_executable_user_defined_function/user_scripts/test_input_chunk_header.py @@ -0,0 +1,14 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + for chunk_header in sys.stdin: + chunk_length = int(chunk_header) + + while chunk_length != 0: + line = sys.stdin.readline() + chunk_length -= 1 + print("Key " + line, end='') + + sys.stdout.flush() diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/test_input_signalled.py b/tests/integration/test_executable_user_defined_function/user_scripts/test_input_signalled.py new file mode 100755 index 00000000000..27c8bc4840e --- /dev/null +++ b/tests/integration/test_executable_user_defined_function/user_scripts/test_input_signalled.py @@ -0,0 +1,13 @@ +#!/usr/bin/python3 + +import sys +import os +import signal +import time + +if __name__ == '__main__': + for line in sys.stdin: + os.signal(os.getpid(), signal.SIGTERM) + + print("Key " + line, end='') + sys.stdout.flush() diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/test_input_slow.py b/tests/integration/test_executable_user_defined_function/user_scripts/test_input_slow.py new file mode 100755 index 00000000000..648a9eac918 --- /dev/null +++ b/tests/integration/test_executable_user_defined_function/user_scripts/test_input_slow.py @@ -0,0 +1,12 @@ +#!/usr/bin/python3 + +import sys +import os +import signal +import time + +if __name__ == '__main__': + for line in sys.stdin: + time.sleep(5) + print("Key " + line, end='') + sys.stdout.flush() diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/test_input_sum.py b/tests/integration/test_executable_user_defined_function/user_scripts/test_input_sum.py index a3580850f57..432d7a13a2f 100755 --- a/tests/integration/test_executable_user_defined_function/user_scripts/test_input_sum.py +++ b/tests/integration/test_executable_user_defined_function/user_scripts/test_input_sum.py @@ -6,4 +6,5 @@ import re if __name__ == '__main__': for line in sys.stdin: line_split = re.split(r'\t+', line) - print(int(line_split[0]) + int(line_split[1]), end='') + print(int(line_split[0]) + int(line_split[1]), end='\n') + sys.stdout.flush() From 72909c67c25bb4a1326317cd19ad733ad3e4b3b7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 24 Dec 2021 13:51:16 +0300 Subject: [PATCH 0513/1260] Fix executable function tests --- tests/config/test_function.xml | 2 +- .../functions/test_function_config.xml | 24 +++++++++++++++++++ .../test.py | 16 +++++++++++++ 3 files changed, 41 insertions(+), 1 deletion(-) diff --git a/tests/config/test_function.xml b/tests/config/test_function.xml index 2e31c9677ec..928cbd75c78 100644 --- a/tests/config/test_function.xml +++ b/tests/config/test_function.xml @@ -11,6 +11,6 @@ TabSeparated cd /; clickhouse-local --input-format TabSeparated --output-format TabSeparated --structure 'x UInt64, y UInt64' --query "SELECT x + y FROM table" - 0 + 0
diff --git a/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml index 2f07082696b..5dd51c83a3f 100644 --- a/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml +++ b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml @@ -169,4 +169,28 @@ 1000 + + executable + test_function_non_direct_bash + String + + UInt64 + + TabSeparated + while read read_data; do printf "Key $read_data\n"; done + 0 + + + + executable_pool + test_function_non_direct_pool_bash + String + + UInt64 + + TabSeparated + while read read_data; do printf "Key $read_data\n"; done + 0 + +
diff --git a/tests/integration/test_executable_user_defined_function/test.py b/tests/integration/test_executable_user_defined_function/test.py index 1899b01f3d6..2282aca914a 100644 --- a/tests/integration/test_executable_user_defined_function/test.py +++ b/tests/integration/test_executable_user_defined_function/test.py @@ -104,3 +104,19 @@ def test_executable_function_signalled_python(started_cluster): assert node.query_and_get_error("SELECT test_function_signalled_pool_python(toUInt64(1))") assert node.query_and_get_error("SELECT test_function_signalled_pool_python(1)") + +def test_executable_function_signalled_python(started_cluster): + skip_test_msan(node) + assert node.query_and_get_error("SELECT test_function_signalled_python(toUInt64(1))") + assert node.query_and_get_error("SELECT test_function_signalled_python(1)") + + assert node.query_and_get_error("SELECT test_function_signalled_pool_python(toUInt64(1))") + assert node.query_and_get_error("SELECT test_function_signalled_pool_python(1)") + +def test_executable_function_non_direct_bash(started_cluster): + skip_test_msan(node) + assert node.query("SELECT test_function_non_direct_bash(toUInt64(1))") == 'Key 1\n' + assert node.query("SELECT test_function_non_direct_bash(1)") == 'Key 1\n' + + assert node.query("SELECT test_function_non_direct_pool_bash(toUInt64(1))") == 'Key 1\n' + assert node.query("SELECT test_function_non_direct_pool_bash(1)") == 'Key 1\n' From 04f5763f69ee0223ce6faf0c58f64c3866ac2ed0 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 25 Dec 2021 16:34:17 +0300 Subject: [PATCH 0514/1260] Added test_executable_dictionary tests --- .../ExecutableDictionarySource.cpp | 52 ++++++- src/Dictionaries/ExecutableDictionarySource.h | 1 + .../ExecutablePoolDictionarySource.cpp | 61 +++++++- .../ExecutablePoolDictionarySource.h | 1 + .../test_executable_dictionary/__init__.py | 0 .../config/dictionaries_config.xml | 2 + ...table_input_argument_python_dictionary.xml | 99 +++++++++++++ .../executable_input_bash_dictionary.xml | 99 +++++++++++++ .../executable_input_python_dictionary.xml | 99 +++++++++++++ ...ut_send_chunk_header_python_dictionary.xml | 103 ++++++++++++++ ...able_input_signalled_python_dictionary.xml | 103 ++++++++++++++ ...xecutable_input_slow_python_dictionary.xml | 103 ++++++++++++++ ...executable_input_sum_python_dictionary.xml | 128 +++++++++++++++++ .../executable_non_direct_dictionary.xml | 95 +++++++++++++ .../test_executable_dictionary/test.py | 132 ++++++++++++++++++ .../user_scripts/test_input.py | 11 ++ .../user_scripts/test_input.sh | 5 + .../user_scripts/test_input_argument.py | 11 ++ .../user_scripts/test_input_chunk_header.py | 15 ++ .../user_scripts/test_input_implicit.py | 8 ++ .../user_scripts/test_input_implicit.sh | 5 + .../test_input_implicit_argument.py | 10 ++ .../test_input_implicit_chunk_header.py | 14 ++ .../test_input_implicit_signalled.py | 13 ++ .../user_scripts/test_input_implicit_slow.py | 12 ++ .../user_scripts/test_input_implicit_sum.py | 10 ++ .../user_scripts/test_input_signalled.py | 13 ++ .../user_scripts/test_input_slow.py | 13 ++ .../user_scripts/test_input_sum.py | 12 ++ .../test.py | 16 --- 30 files changed, 1220 insertions(+), 26 deletions(-) create mode 100644 tests/integration/test_executable_dictionary/__init__.py create mode 100644 tests/integration/test_executable_dictionary/config/dictionaries_config.xml create mode 100644 tests/integration/test_executable_dictionary/dictionaries/executable_input_argument_python_dictionary.xml create mode 100644 tests/integration/test_executable_dictionary/dictionaries/executable_input_bash_dictionary.xml create mode 100644 tests/integration/test_executable_dictionary/dictionaries/executable_input_python_dictionary.xml create mode 100644 tests/integration/test_executable_dictionary/dictionaries/executable_input_send_chunk_header_python_dictionary.xml create mode 100644 tests/integration/test_executable_dictionary/dictionaries/executable_input_signalled_python_dictionary.xml create mode 100644 tests/integration/test_executable_dictionary/dictionaries/executable_input_slow_python_dictionary.xml create mode 100644 tests/integration/test_executable_dictionary/dictionaries/executable_input_sum_python_dictionary.xml create mode 100644 tests/integration/test_executable_dictionary/dictionaries/executable_non_direct_dictionary.xml create mode 100644 tests/integration/test_executable_dictionary/test.py create mode 100755 tests/integration/test_executable_dictionary/user_scripts/test_input.py create mode 100755 tests/integration/test_executable_dictionary/user_scripts/test_input.sh create mode 100755 tests/integration/test_executable_dictionary/user_scripts/test_input_argument.py create mode 100755 tests/integration/test_executable_dictionary/user_scripts/test_input_chunk_header.py create mode 100755 tests/integration/test_executable_dictionary/user_scripts/test_input_implicit.py create mode 100755 tests/integration/test_executable_dictionary/user_scripts/test_input_implicit.sh create mode 100755 tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_argument.py create mode 100755 tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_chunk_header.py create mode 100755 tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_signalled.py create mode 100755 tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_slow.py create mode 100755 tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_sum.py create mode 100755 tests/integration/test_executable_dictionary/user_scripts/test_input_signalled.py create mode 100755 tests/integration/test_executable_dictionary/user_scripts/test_input_slow.py create mode 100755 tests/integration/test_executable_dictionary/user_scripts/test_input_sum.py diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index b801ad3ab04..6c03fd7595b 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -1,7 +1,13 @@ #include "ExecutableDictionarySource.h" +#include + +#include + #include -#include +#include + +#include #include #include @@ -110,13 +116,37 @@ Pipe ExecutableDictionarySource::loadKeys(const Columns & key_columns, const std Pipe ExecutableDictionarySource::getStreamForBlock(const Block & block) { + String command = configuration.command; + const auto & coordinator_configuration = coordinator->getConfiguration(); + + if (coordinator_configuration.execute_direct) + { + auto global_context = context->getGlobalContext(); + auto user_scripts_path = global_context->getUserScriptsPath(); + auto script_path = user_scripts_path + '/' + command; + + if (!fileOrSymlinkPathStartsWith(script_path, user_scripts_path)) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Executable file {} must be inside user scripts folder {}", + command, + user_scripts_path); + + if (!std::filesystem::exists(std::filesystem::path(script_path))) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Executable file {} does not exist inside user scripts folder {}", + command, + user_scripts_path); + + command = std::move(script_path); + } + auto source = std::make_shared(block); auto shell_input_pipe = Pipe(std::move(source)); Pipes shell_input_pipes; shell_input_pipes.emplace_back(std::move(shell_input_pipe)); - auto pipe = coordinator->createPipe(configuration.command, std::move(shell_input_pipes), sample_block, context); + auto pipe = coordinator->createPipe(command, configuration.command_arguments, std::move(shell_input_pipes), sample_block, context); if (configuration.implicit_key) pipe.addTransform(std::make_shared(block, pipe.getHeader())); @@ -172,9 +202,22 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) std::string settings_config_prefix = config_prefix + ".executable"; + bool execute_direct = config.getBool(settings_config_prefix + ".execute_direct", false); + std::string command_value = config.getString(settings_config_prefix + ".command"); + std::vector command_arguments; + + if (execute_direct) + { + boost::split(command_arguments, command_value, [](char c) { return c == ' '; }); + + command_value = std::move(command_arguments[0]); + command_arguments.erase(command_arguments.begin()); + } + ExecutableDictionarySource::Configuration configuration { - .command = config.getString(settings_config_prefix + ".command"), + .command = std::move(command_value), + .command_arguments = std::move(command_arguments), .update_field = config.getString(settings_config_prefix + ".update_field", ""), .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1), .implicit_key = config.getBool(settings_config_prefix + ".implicit_key", false), @@ -188,11 +231,10 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) .command_write_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_write_timeout", 10000), .is_executable_pool = false, .send_chunk_header = config.getBool(settings_config_prefix + ".send_chunk_header", false), - .execute_direct = false + .execute_direct = config.getBool(settings_config_prefix + ".execute_direct", false) }; std::shared_ptr coordinator = std::make_shared(shell_command_coordinator_configration); - return std::make_unique(dict_struct, configuration, sample_block, std::move(coordinator), context); }; diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index f77da567efe..627915b8a46 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -21,6 +21,7 @@ public: struct Configuration { std::string command; + std::vector command_arguments; std::string update_field; UInt64 update_lag; /// Implicit key means that the source script will return only values, diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 7c010c1a264..1ba466aed7b 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -2,6 +2,11 @@ #include #include +#include + +#include + +#include #include #include @@ -93,6 +98,30 @@ Pipe ExecutablePoolDictionarySource::loadKeys(const Columns & key_columns, const Pipe ExecutablePoolDictionarySource::getStreamForBlock(const Block & block) { + String command = configuration.command; + const auto & coordinator_configuration = coordinator->getConfiguration(); + + if (coordinator_configuration.execute_direct) + { + auto global_context = context->getGlobalContext(); + auto user_scripts_path = global_context->getUserScriptsPath(); + auto script_path = user_scripts_path + '/' + command; + + if (!fileOrSymlinkPathStartsWith(script_path, user_scripts_path)) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Executable file {} must be inside user scripts folder {}", + command, + user_scripts_path); + + if (!std::filesystem::exists(std::filesystem::path(script_path))) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Executable file {} does not exist inside user scripts folder {}", + command, + user_scripts_path); + + command = std::move(script_path); + } + auto source = std::make_shared(block); auto shell_input_pipe = Pipe(std::move(source)); @@ -103,7 +132,17 @@ Pipe ExecutablePoolDictionarySource::getStreamForBlock(const Block & block) Pipes shell_input_pipes; shell_input_pipes.emplace_back(std::move(shell_input_pipe)); - auto pipe = coordinator->createPipe(configuration.command, std::move(shell_input_pipes), sample_block, context, command_configuration); + auto pipe = coordinator->createPipe( + command, + configuration.command_arguments, + std::move(shell_input_pipes), + sample_block, + context, + command_configuration); + + if (configuration.implicit_key) + pipe.addTransform(std::make_shared(block, pipe.getHeader())); + return pipe; } @@ -162,9 +201,22 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) if (max_execution_time_seconds != 0 && max_command_execution_time > max_execution_time_seconds) max_command_execution_time = max_execution_time_seconds; + bool execute_direct = config.getBool(settings_config_prefix + ".execute_direct", false); + std::string command_value = config.getString(settings_config_prefix + ".command"); + std::vector command_arguments; + + if (execute_direct) + { + boost::split(command_arguments, command_value, [](char c) { return c == ' '; }); + + command_value = std::move(command_arguments[0]); + command_arguments.erase(command_arguments.begin()); + } + ExecutablePoolDictionarySource::Configuration configuration { - .command = config.getString(settings_config_prefix + ".command"), + .command = std::move(command_value), + .command_arguments = std::move(command_arguments), .implicit_key = config.getBool(settings_config_prefix + ".implicit_key", false), }; @@ -174,15 +226,14 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) .command_termination_timeout_seconds = config.getUInt64(settings_config_prefix + ".command_termination_timeout", 10), .command_read_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_read_timeout", 10000), .command_write_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_write_timeout", 10000), - .pool_size = config.getUInt64(settings_config_prefix + ".size"), + .pool_size = config.getUInt64(settings_config_prefix + ".size", 16), .max_command_execution_time_seconds = max_command_execution_time, .is_executable_pool = true, .send_chunk_header = config.getBool(settings_config_prefix + ".send_chunk_header", false), - .execute_direct = false + .execute_direct = execute_direct }; std::shared_ptr coordinator = std::make_shared(shell_command_coordinator_configration); - return std::make_unique(dict_struct, configuration, sample_block, std::move(coordinator), context); }; diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.h b/src/Dictionaries/ExecutablePoolDictionarySource.h index 0071f2cc9d0..35f0f04bd48 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.h +++ b/src/Dictionaries/ExecutablePoolDictionarySource.h @@ -28,6 +28,7 @@ public: struct Configuration { String command; + std::vector command_arguments; bool implicit_key; }; diff --git a/tests/integration/test_executable_dictionary/__init__.py b/tests/integration/test_executable_dictionary/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_executable_dictionary/config/dictionaries_config.xml b/tests/integration/test_executable_dictionary/config/dictionaries_config.xml new file mode 100644 index 00000000000..3cbf717bb67 --- /dev/null +++ b/tests/integration/test_executable_dictionary/config/dictionaries_config.xml @@ -0,0 +1,2 @@ + + diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_input_argument_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_input_argument_python_dictionary.xml new file mode 100644 index 00000000000..424a19b1ba8 --- /dev/null +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_input_argument_python_dictionary.xml @@ -0,0 +1,99 @@ + + + executable_input_argument_python + + + TabSeparated + test_input_argument.py 1 + 1 + + + + + + + + input + + + result + String + + + + + + + executable_input_argument_pool_python + + + TabSeparated + test_input_argument.py 1 + 1 + + + + + + + + input + + + result + String + + + + + + + executable_implicit_input_argument_python + + + TabSeparated + test_input_implicit_argument.py 1 + 1 + 1 + + + + + + + + input + + + result + String + + + + + + + executable_implicit_input_argument_pool_python + + + TabSeparated + test_input_implicit_argument.py 1 + 1 + 1 + + + + + + + + input + + + result + String + + + + + \ No newline at end of file diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_input_bash_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_input_bash_dictionary.xml new file mode 100644 index 00000000000..744580600f4 --- /dev/null +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_input_bash_dictionary.xml @@ -0,0 +1,99 @@ + + + executable_input_bash + + + TabSeparated + test_input.sh + 1 + + + + + + + + input + + + result + String + + + + + + + executable_input_pool_bash + + + TabSeparated + test_input.sh + 1 + + + + + + + + input + + + result + String + + + + + + + executable_implicit_input_bash + + + TabSeparated + test_input_implicit.sh + 1 + 1 + + + + + + + + input + + + result + String + + + + + + + executable_implicit_input_pool_bash + + + TabSeparated + test_input_implicit.sh + 1 + 1 + + + + + + + + input + + + result + String + + + + + \ No newline at end of file diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_input_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_input_python_dictionary.xml new file mode 100644 index 00000000000..0ff81818b5d --- /dev/null +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_input_python_dictionary.xml @@ -0,0 +1,99 @@ + + + executable_input_python + + + TabSeparated + test_input.py + 1 + + + + + + + + input + + + result + String + + + + + + + executable_input_pool_python + + + TabSeparated + test_input.py + 1 + + + + + + + + input + + + result + String + + + + + + + executable_implicit_input_python + + + TabSeparated + test_input_implicit.py + 1 + 1 + + + + + + + + input + + + result + String + + + + + + + executable_implicit_input_pool_python + + + TabSeparated + test_input_implicit.py + 1 + 1 + + + + + + + + input + + + result + String + + + + + \ No newline at end of file diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_input_send_chunk_header_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_input_send_chunk_header_python_dictionary.xml new file mode 100644 index 00000000000..9310c3d2142 --- /dev/null +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_input_send_chunk_header_python_dictionary.xml @@ -0,0 +1,103 @@ + + + executable_input_send_chunk_header_python + + + TabSeparated + test_input_chunk_header.py + 1 + 1 + + + + + + + + input + + + result + String + + + + + + + executable_input_send_chunk_header_pool_python + + + TabSeparated + test_input_chunk_header.py + 1 + 1 + + + + + + + + input + + + result + String + + + + + + + executable_implicit_input_send_chunk_header_python + + + TabSeparated + test_input_implicit_chunk_header.py + 1 + 1 + 1 + + + + + + + + input + + + result + String + + + + + + + executable_implicit_input_send_chunk_header_pool_python + + + TabSeparated + test_input_implicit_chunk_header.py + 1 + 1 + 1 + + + + + + + + input + + + result + String + + + + + \ No newline at end of file diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_input_signalled_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_input_signalled_python_dictionary.xml new file mode 100644 index 00000000000..ecc3ef4e3a5 --- /dev/null +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_input_signalled_python_dictionary.xml @@ -0,0 +1,103 @@ + + + executable_input_signalled_python + + + TabSeparated + test_input_signalled.py + 1 + 1000 + + + + + + + + input + + + result + String + Default result + + + + + + executable_input_signalled_pool_python + + + TabSeparated + test_input_signalled.py + 1 + 1000 + + + + + + + + input + + + result + String + Default result + + + + + + executable_implicit_input_signalled_python + + + TabSeparated + test_input_implicit_signalled.py + 1 + 1 + 1000 + + + + + + + + input + + + result + String + Default result + + + + + + executable_implicit_input_signalled_pool_python + + + TabSeparated + test_input_implicit_signalled.py + 1 + 1 + 1000 + + + + + + + + input + + + result + String + Default result + + + + \ No newline at end of file diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_input_slow_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_input_slow_python_dictionary.xml new file mode 100644 index 00000000000..dbd596ee80b --- /dev/null +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_input_slow_python_dictionary.xml @@ -0,0 +1,103 @@ + + + executable_input_slow_python + + + TabSeparated + test_input_slow.py + 1 + 1000 + + + + + + + + input + + + result + String + + + + + + + executable_input_slow_pool_python + + + TabSeparated + test_input_slow.py + 1 + 1000 + + + + + + + + input + + + result + String + + + + + + + executable_implicit_input_slow_python + + + TabSeparated + test_input_implicit_slow.py + 1 + 1 + 1000 + + + + + + + + input + + + result + String + + + + + + + executable_implicit_input_slow_pool_python + + + TabSeparated + test_input_implicit_slow.py + 1 + 1 + 1000 + + + + + + + + input + + + result + String + + + + + \ No newline at end of file diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_input_sum_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_input_sum_python_dictionary.xml new file mode 100644 index 00000000000..a4504384e7d --- /dev/null +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_input_sum_python_dictionary.xml @@ -0,0 +1,128 @@ + + + executable_input_sum_python + + + TabSeparated + test_input_sum.py + 1 + + + + + + + + + first_argument + UInt64 + + + second_argument + UInt64 + + + + result + UInt64 + + + + + + + executable_input_sum_pool_python + + + TabSeparated + test_input_sum.py + 1 + + + + + + + + + first_argument + UInt64 + + + second_argument + UInt64 + + + + result + UInt64 + + + + + + + + executable_implicit_input_sum_python + + + TabSeparated + test_input_implicit_sum.py + 1 + 1 + + + + + + + + + first_argument + UInt64 + + + second_argument + UInt64 + + + + result + UInt64 + + + + + + + executable_implicit_input_sum_pool_python + + + TabSeparated + test_input_implicit_sum.py + 1 + 1 + + + + + + + + + first_argument + UInt64 + + + second_argument + UInt64 + + + + result + UInt64 + + + + + diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_non_direct_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_non_direct_dictionary.xml new file mode 100644 index 00000000000..3f77dae1ac6 --- /dev/null +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_non_direct_dictionary.xml @@ -0,0 +1,95 @@ + + + executable_input_non_direct_bash + + + TabSeparated + while read read_data; do printf "$read_data\tKey $read_data\n"; done + + + + + + + + input + + + result + String + + + + + + + executable_input_non_direct_pool_bash + + + TabSeparated + while read read_data; do printf "$read_data\tKey $read_data\n"; done + + + + + + + + input + + + result + String + + + + + + + executable_input_implicit_non_direct_bash + + + TabSeparated + while read read_data; do printf "Key $read_data\n"; done + 1 + + + + + + + + input + + + result + String + + + + + + + executable_input_implicit_non_direct_pool_bash + + + TabSeparated + while read read_data; do printf "Key $read_data\n"; done + 1 + + + + + + + + input + + + result + String + + + + + \ No newline at end of file diff --git a/tests/integration/test_executable_dictionary/test.py b/tests/integration/test_executable_dictionary/test.py new file mode 100644 index 00000000000..c8efdd400ce --- /dev/null +++ b/tests/integration/test_executable_dictionary/test.py @@ -0,0 +1,132 @@ +import os +import sys +import time + +import pytest + +sys.path.insert(0, os.path.dirname(os.path.dirname(os.path.abspath(__file__)))) +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance('node', stay_alive=True, main_configs=[]) + + +def skip_test_msan(instance): + if instance.is_built_with_memory_sanitizer(): + pytest.skip("Memory Sanitizer cannot work with vfork") + +def copy_file_to_container(local_path, dist_path, container_id): + os.system("docker cp {local} {cont_id}:{dist}".format(local=local_path, cont_id=container_id, dist=dist_path)) + +config = ''' + /etc/clickhouse-server/dictionaries/*_dictionary.xml +''' + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + node.replace_config("/etc/clickhouse-server/config.d/dictionaries_config.xml", config) + + copy_file_to_container(os.path.join(SCRIPT_DIR, 'dictionaries/.'), '/etc/clickhouse-server/dictionaries', node.docker_id) + copy_file_to_container(os.path.join(SCRIPT_DIR, 'user_scripts/.'), '/var/lib/clickhouse/user_scripts', node.docker_id) + + node.restart_clickhouse() + + yield cluster + + finally: + cluster.shutdown() + +def test_executable_input_bash(started_cluster): + skip_test_msan(node) + assert node.query("SELECT dictGet('executable_input_bash', 'result', toUInt64(1))") == 'Key 1\n' + assert node.query("SELECT dictGet('executable_input_pool_bash', 'result', toUInt64(1))") == 'Key 1\n' + +def test_executable_implicit_input_bash(started_cluster): + skip_test_msan(node) + assert node.query("SELECT dictGet('executable_implicit_input_bash', 'result', toUInt64(1))") == 'Key 1\n' + assert node.query("SELECT dictGet('executable_implicit_input_pool_bash', 'result', toUInt64(1))") == 'Key 1\n' + +def test_executable_input_python(started_cluster): + skip_test_msan(node) + assert node.query("SELECT dictGet('executable_input_python', 'result', toUInt64(1))") == 'Key 1\n' + assert node.query("SELECT dictGet('executable_input_pool_python', 'result', toUInt64(1))") == 'Key 1\n' + +def test_executable_implicit_input_python(started_cluster): + skip_test_msan(node) + assert node.query("SELECT dictGet('executable_implicit_input_python', 'result', toUInt64(1))") == 'Key 1\n' + assert node.query("SELECT dictGet('executable_implicit_input_pool_python', 'result', toUInt64(1))") == 'Key 1\n' + +def test_executable_input_send_chunk_header_python(started_cluster): + skip_test_msan(node) + assert node.query("SELECT dictGet('executable_input_send_chunk_header_python', 'result', toUInt64(1))") == 'Key 1\n' + assert node.query("SELECT dictGet('executable_input_send_chunk_header_pool_python', 'result', toUInt64(1))") == 'Key 1\n' + +def test_executable_implicit_input_send_chunk_header_python(started_cluster): + skip_test_msan(node) + assert node.query("SELECT dictGet('executable_implicit_input_send_chunk_header_python', 'result', toUInt64(1))") == 'Key 1\n' + assert node.query("SELECT dictGet('executable_implicit_input_send_chunk_header_pool_python', 'result', toUInt64(1))") == 'Key 1\n' + +def test_executable_input_sum_python(started_cluster): + skip_test_msan(node) + assert node.query("SELECT dictGet('executable_input_sum_python', 'result', tuple(toUInt64(1), toUInt64(1)))") == '2\n' + assert node.query("SELECT dictGet('executable_input_sum_pool_python', 'result', tuple(toUInt64(1), toUInt64(1)))") == '2\n' + +def test_executable_implicit_input_sum_python(started_cluster): + skip_test_msan(node) + assert node.query("SELECT dictGet('executable_implicit_input_sum_python', 'result', tuple(toUInt64(1), toUInt64(1)))") == '2\n' + assert node.query("SELECT dictGet('executable_implicit_input_sum_pool_python', 'result', tuple(toUInt64(1), toUInt64(1)))") == '2\n' + +def test_executable_input_argument_python(started_cluster): + skip_test_msan(node) + assert node.query("SELECT dictGet('executable_input_argument_python', 'result', toUInt64(1))") == 'Key 1 1\n' + assert node.query("SELECT dictGet('executable_input_argument_pool_python', 'result', toUInt64(1))") == 'Key 1 1\n' + +def test_executable_implicit_input_argument_python(started_cluster): + skip_test_msan(node) + assert node.query("SELECT dictGet('executable_implicit_input_argument_python', 'result', toUInt64(1))") == 'Key 1 1\n' + assert node.query("SELECT dictGet('executable_implicit_input_argument_pool_python', 'result', toUInt64(1))") == 'Key 1 1\n' + +def test_executable_input_signalled_python(started_cluster): + skip_test_msan(node) + assert node.query("SELECT dictGet('executable_input_signalled_python', 'result', toUInt64(1))") == 'Default result\n' + assert node.query("SELECT dictGet('executable_input_signalled_pool_python', 'result', toUInt64(1))") == 'Default result\n' + +def test_executable_implicit_input_signalled_python(started_cluster): + skip_test_msan(node) + assert node.query("SELECT dictGet('executable_implicit_input_signalled_python', 'result', toUInt64(1))") == 'Default result\n' + assert node.query("SELECT dictGet('executable_implicit_input_signalled_pool_python', 'result', toUInt64(1))") == 'Default result\n' + +def test_executable_input_slow_python(started_cluster): + skip_test_msan(node) + assert node.query_and_get_error("SELECT dictGet('executable_input_slow_python', 'result', toUInt64(1))") + assert node.query_and_get_error("SELECT dictGet('executable_input_slow_pool_python', 'result', toUInt64(1))") + +def test_executable_implicit_input_slow_python(started_cluster): + skip_test_msan(node) + assert node.query_and_get_error("SELECT dictGet('executable_implicit_input_slow_python', 'result', toUInt64(1))") + assert node.query_and_get_error("SELECT dictGet('executable_implicit_input_slow_pool_python', 'result', toUInt64(1))") + +def test_executable_input_slow_python(started_cluster): + skip_test_msan(node) + assert node.query_and_get_error("SELECT dictGet('executable_input_slow_python', 'result', toUInt64(1))") + assert node.query_and_get_error("SELECT dictGet('executable_input_slow_pool_python', 'result', toUInt64(1))") + +def test_executable_implicit_input_slow_python(started_cluster): + skip_test_msan(node) + assert node.query_and_get_error("SELECT dictGet('executable_implicit_input_slow_python', 'result', toUInt64(1))") + assert node.query_and_get_error("SELECT dictGet('executable_implicit_input_slow_pool_python', 'result', toUInt64(1))") + +def test_executable_non_direct_input_bash(started_cluster): + skip_test_msan(node) + assert node.query("SELECT dictGet('executable_input_non_direct_bash', 'result', toUInt64(1))") == 'Key 1\n' + assert node.query("SELECT dictGet('executable_input_non_direct_pool_bash', 'result', toUInt64(1))") == 'Key 1\n' + +def test_executable_implicit_non_direct_input_bash(started_cluster): + skip_test_msan(node) + assert node.query("SELECT dictGet('executable_input_implicit_non_direct_bash', 'result', toUInt64(1))") == 'Key 1\n' + assert node.query("SELECT dictGet('executable_input_implicit_non_direct_pool_bash', 'result', toUInt64(1))") == 'Key 1\n' diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input.py b/tests/integration/test_executable_dictionary/user_scripts/test_input.py new file mode 100755 index 00000000000..e711dd8e306 --- /dev/null +++ b/tests/integration/test_executable_dictionary/user_scripts/test_input.py @@ -0,0 +1,11 @@ +#!/usr/bin/python3 + +import sys +import os +import signal + +if __name__ == '__main__': + for line in sys.stdin: + updated_line = line.replace('\n', '') + print(updated_line + '\t' + "Key " + updated_line, end='\n') + sys.stdout.flush() diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input.sh b/tests/integration/test_executable_dictionary/user_scripts/test_input.sh new file mode 100755 index 00000000000..7712c392951 --- /dev/null +++ b/tests/integration/test_executable_dictionary/user_scripts/test_input.sh @@ -0,0 +1,5 @@ +#!/bin/bash + +while read read_data; + do printf "$read_data\tKey $read_data\n"; +done diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_argument.py b/tests/integration/test_executable_dictionary/user_scripts/test_input_argument.py new file mode 100755 index 00000000000..163f9c4183f --- /dev/null +++ b/tests/integration/test_executable_dictionary/user_scripts/test_input_argument.py @@ -0,0 +1,11 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + arg = int(sys.argv[1]) + + for line in sys.stdin: + updated_line = line.replace('\n', '') + print(updated_line + '\t' + "Key " + str(arg) + " " + updated_line, end='\n') + sys.stdout.flush() diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_chunk_header.py b/tests/integration/test_executable_dictionary/user_scripts/test_input_chunk_header.py new file mode 100755 index 00000000000..4eb00f64eb3 --- /dev/null +++ b/tests/integration/test_executable_dictionary/user_scripts/test_input_chunk_header.py @@ -0,0 +1,15 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + for chunk_header in sys.stdin: + chunk_length = int(chunk_header) + + while chunk_length != 0: + line = sys.stdin.readline() + updated_line = line.replace('\n', '') + chunk_length -= 1 + print(updated_line + '\t' + "Key " + updated_line, end='\n') + + sys.stdout.flush() diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit.py b/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit.py new file mode 100755 index 00000000000..835ab1f441a --- /dev/null +++ b/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit.py @@ -0,0 +1,8 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + for line in sys.stdin: + print("Key " + line, end='') + sys.stdout.flush() diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit.sh b/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit.sh new file mode 100755 index 00000000000..aea51b82b1f --- /dev/null +++ b/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit.sh @@ -0,0 +1,5 @@ +#!/bin/bash + +while read read_data; + do printf "Key $read_data\n"; +done diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_argument.py b/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_argument.py new file mode 100755 index 00000000000..c1b2e5966d7 --- /dev/null +++ b/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_argument.py @@ -0,0 +1,10 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + arg = int(sys.argv[1]) + + for line in sys.stdin: + print("Key " + str(arg) + " " + line, end='') + sys.stdout.flush() diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_chunk_header.py b/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_chunk_header.py new file mode 100755 index 00000000000..5dc03e1c507 --- /dev/null +++ b/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_chunk_header.py @@ -0,0 +1,14 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + for chunk_header in sys.stdin: + chunk_length = int(chunk_header) + + while chunk_length != 0: + line = sys.stdin.readline() + chunk_length -= 1 + print("Key " + line, end='') + + sys.stdout.flush() diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_signalled.py b/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_signalled.py new file mode 100755 index 00000000000..27c8bc4840e --- /dev/null +++ b/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_signalled.py @@ -0,0 +1,13 @@ +#!/usr/bin/python3 + +import sys +import os +import signal +import time + +if __name__ == '__main__': + for line in sys.stdin: + os.signal(os.getpid(), signal.SIGTERM) + + print("Key " + line, end='') + sys.stdout.flush() diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_slow.py b/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_slow.py new file mode 100755 index 00000000000..648a9eac918 --- /dev/null +++ b/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_slow.py @@ -0,0 +1,12 @@ +#!/usr/bin/python3 + +import sys +import os +import signal +import time + +if __name__ == '__main__': + for line in sys.stdin: + time.sleep(5) + print("Key " + line, end='') + sys.stdout.flush() diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_sum.py b/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_sum.py new file mode 100755 index 00000000000..432d7a13a2f --- /dev/null +++ b/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_sum.py @@ -0,0 +1,10 @@ +#!/usr/bin/python3 + +import sys +import re + +if __name__ == '__main__': + for line in sys.stdin: + line_split = re.split(r'\t+', line) + print(int(line_split[0]) + int(line_split[1]), end='\n') + sys.stdout.flush() diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_signalled.py b/tests/integration/test_executable_dictionary/user_scripts/test_input_signalled.py new file mode 100755 index 00000000000..a3a99f1e71e --- /dev/null +++ b/tests/integration/test_executable_dictionary/user_scripts/test_input_signalled.py @@ -0,0 +1,13 @@ +#!/usr/bin/python3 + +import sys +import os +import signal +import time + +if __name__ == '__main__': + for line in sys.stdin: + os.signal(os.getpid(), signal.SIGTERM) + updated_line = line.replace('\n', '') + print(updated_line + '\t' + "Key " + updated_line, end='\n') + sys.stdout.flush() diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_slow.py b/tests/integration/test_executable_dictionary/user_scripts/test_input_slow.py new file mode 100755 index 00000000000..a3b8c484b29 --- /dev/null +++ b/tests/integration/test_executable_dictionary/user_scripts/test_input_slow.py @@ -0,0 +1,13 @@ +#!/usr/bin/python3 + +import sys +import os +import signal +import time + +if __name__ == '__main__': + for line in sys.stdin: + time.sleep(5) + updated_line = line.replace('\n', '') + print(updated_line + '\t' + "Key " + updated_line, end='\n') + sys.stdout.flush() diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_sum.py b/tests/integration/test_executable_dictionary/user_scripts/test_input_sum.py new file mode 100755 index 00000000000..e9ec5028701 --- /dev/null +++ b/tests/integration/test_executable_dictionary/user_scripts/test_input_sum.py @@ -0,0 +1,12 @@ +#!/usr/bin/python3 + +import sys +import re + +if __name__ == '__main__': + for line in sys.stdin: + updated_line = line.replace('\n', '') + line_split = re.split(r'\t+', line) + sum = int(line_split[0]) + int(line_split[1]) + print(updated_line + '\t' + str(sum), end='\n') + sys.stdout.flush() diff --git a/tests/integration/test_executable_user_defined_function/test.py b/tests/integration/test_executable_user_defined_function/test.py index 2282aca914a..94afdf8d8a9 100644 --- a/tests/integration/test_executable_user_defined_function/test.py +++ b/tests/integration/test_executable_user_defined_function/test.py @@ -97,22 +97,6 @@ def test_executable_function_slow_python(started_cluster): assert node.query_and_get_error("SELECT test_function_slow_pool_python(toUInt64(1))") assert node.query_and_get_error("SELECT test_function_slow_pool_python(1)") -def test_executable_function_signalled_python(started_cluster): - skip_test_msan(node) - assert node.query_and_get_error("SELECT test_function_signalled_python(toUInt64(1))") - assert node.query_and_get_error("SELECT test_function_signalled_python(1)") - - assert node.query_and_get_error("SELECT test_function_signalled_pool_python(toUInt64(1))") - assert node.query_and_get_error("SELECT test_function_signalled_pool_python(1)") - -def test_executable_function_signalled_python(started_cluster): - skip_test_msan(node) - assert node.query_and_get_error("SELECT test_function_signalled_python(toUInt64(1))") - assert node.query_and_get_error("SELECT test_function_signalled_python(1)") - - assert node.query_and_get_error("SELECT test_function_signalled_pool_python(toUInt64(1))") - assert node.query_and_get_error("SELECT test_function_signalled_pool_python(1)") - def test_executable_function_non_direct_bash(started_cluster): skip_test_msan(node) assert node.query("SELECT test_function_non_direct_bash(toUInt64(1))") == 'Key 1\n' From cda8bd7f447af31985dbcd298f3af8f6b745f582 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 25 Dec 2021 19:50:16 +0300 Subject: [PATCH 0515/1260] Added test_executable_dictionary source tests --- .../ExecutableDictionarySource.cpp | 63 +++++++++++++++++-- src/Dictionaries/HashedDictionary.cpp | 2 + src/Processors/Sources/ShellCommandSource.h | 9 +++ ...able_source_argument_python_dictionary.xml | 54 ++++++++++++++++ .../executable_source_python_dictionary.xml | 54 ++++++++++++++++ ...table_source_updated_python_dictionary.xml | 56 +++++++++++++++++ .../test_executable_dictionary/test.py | 43 +++++++++++++ .../user_scripts/test_source.py | 10 +++ .../user_scripts/test_source_argument.py | 12 ++++ .../user_scripts/test_source_update.py | 12 ++++ 10 files changed, 309 insertions(+), 6 deletions(-) create mode 100644 tests/integration/test_executable_dictionary/dictionaries/executable_source_argument_python_dictionary.xml create mode 100644 tests/integration/test_executable_dictionary/dictionaries/executable_source_python_dictionary.xml create mode 100644 tests/integration/test_executable_dictionary/dictionaries/executable_source_updated_python_dictionary.xml create mode 100755 tests/integration/test_executable_dictionary/user_scripts/test_source.py create mode 100755 tests/integration/test_executable_dictionary/user_scripts/test_source_argument.py create mode 100755 tests/integration/test_executable_dictionary/user_scripts/test_source_update.py diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 6c03fd7595b..9314096e034 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -34,6 +34,35 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; } +namespace +{ + + void updateCommandIfNeeded(String & command, bool execute_direct, ContextPtr context) + { + if (!execute_direct) + return; + + auto global_context = context->getGlobalContext(); + auto user_scripts_path = global_context->getUserScriptsPath(); + auto script_path = user_scripts_path + '/' + command; + + if (!fileOrSymlinkPathStartsWith(script_path, user_scripts_path)) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Executable file {} must be inside user scripts folder {}", + command, + user_scripts_path); + + if (!std::filesystem::exists(std::filesystem::path(script_path))) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Executable file {} does not exist inside user scripts folder {}", + command, + user_scripts_path); + + command = std::move(script_path); + } + +} + ExecutableDictionarySource::ExecutableDictionarySource( const DictionaryStructure & dict_struct_, const Configuration & configuration_, @@ -79,7 +108,11 @@ Pipe ExecutableDictionarySource::loadAll() LOG_TRACE(log, "loadAll {}", toString()); - return coordinator->createPipe(configuration.command, sample_block, context); + const auto & coordinator_configuration = coordinator->getConfiguration(); + auto command = configuration.command; + updateCommandIfNeeded(command, coordinator_configuration.execute_direct, context); + + return coordinator->createPipe(command, configuration.command_arguments, sample_block, context); } Pipe ExecutableDictionarySource::loadUpdatedAll() @@ -88,14 +121,32 @@ Pipe ExecutableDictionarySource::loadUpdatedAll() throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutableDictionarySource with implicit_key does not support loadUpdatedAll method"); time_t new_update_time = time(nullptr); - SCOPE_EXIT(update_time = new_update_time); - std::string command_with_update_field = configuration.command; + const auto & coordinator_configuration = coordinator->getConfiguration(); + auto command = configuration.command; + updateCommandIfNeeded(command, coordinator_configuration.execute_direct, context); + + auto command_arguments = configuration.command_arguments; + if (update_time) - command_with_update_field += " " + configuration.update_field + " " + DB::toString(LocalDateTime(update_time - configuration.update_lag)); + { + auto update_difference = DB::toString(LocalDateTime(update_time - configuration.update_lag)); - LOG_TRACE(log, "loadUpdatedAll {}", command_with_update_field); - return coordinator->createPipe(command_with_update_field, sample_block, context); + if (coordinator_configuration.execute_direct) + { + command_arguments.emplace_back(configuration.update_field); + command_arguments.emplace_back(std::move(update_difference)); + } + else + { + command += ' ' + configuration.update_field + ' ' + update_difference; + } + } + + update_time = new_update_time; + + LOG_TRACE(log, "loadUpdatedAll {}", command); + return coordinator->createPipe(command, command_arguments, sample_block, context); } Pipe ExecutableDictionarySource::loadIds(const std::vector & ids) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index f7627d5817f..c48893bf24f 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -591,7 +591,9 @@ void HashedDictionary::loadData() } } else + { updateData(); + } if (configuration.require_nonempty && 0 == element_count) throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY, diff --git a/src/Processors/Sources/ShellCommandSource.h b/src/Processors/Sources/ShellCommandSource.h index 5d5055f29e2..3203a7a4dbb 100644 --- a/src/Processors/Sources/ShellCommandSource.h +++ b/src/Processors/Sources/ShellCommandSource.h @@ -97,6 +97,15 @@ public: return createPipe(command, {}, std::move(input_pipes), std::move(sample_block), std::move(context), source_configuration); } + Pipe createPipe( + const std::string & command, + const std::vector & arguments, + Block sample_block, + ContextPtr context) + { + return createPipe(command, arguments, {}, std::move(sample_block), std::move(context), {}); + } + Pipe createPipe( const std::string & command, Block sample_block, diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_source_argument_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_source_argument_python_dictionary.xml new file mode 100644 index 00000000000..5d9582654c0 --- /dev/null +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_source_argument_python_dictionary.xml @@ -0,0 +1,54 @@ + + + executable_source_simple_key_argument_python + + + TabSeparated + test_source_argument.py 1 + 1 + + + + + + 0 + + + input + + + result + String + + + + + + + executable_source_complex_key_argument_python + + + TabSeparated + test_source_argument.py 1 + 1 + + + + + + 0 + + + + input + UInt64 + + + + result + String + + + + + \ No newline at end of file diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_source_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_source_python_dictionary.xml new file mode 100644 index 00000000000..6d191e882ee --- /dev/null +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_source_python_dictionary.xml @@ -0,0 +1,54 @@ + + + executable_source_simple_key_python + + + TabSeparated + test_source.py + 1 + + + + + + 0 + + + input + + + result + String + + + + + + + executable_source_complex_key_python + + + TabSeparated + test_source.py + 1 + + + + + + 0 + + + + input + UInt64 + + + + result + String + + + + + \ No newline at end of file diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_source_updated_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_source_updated_python_dictionary.xml new file mode 100644 index 00000000000..0acd2c8d6b3 --- /dev/null +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_source_updated_python_dictionary.xml @@ -0,0 +1,56 @@ + + + executable_source_simple_key_update_python + + + TabSeparated + test_source_update.py + 1 + 1 + + + + + + 5 + + + input + + + result + String + + + + + + + executable_source_complex_key_update_python + + + TabSeparated + test_source_update.py + 1 + 1 + + + + + + 5 + + + + input + UInt64 + + + + result + String + + + + + \ No newline at end of file diff --git a/tests/integration/test_executable_dictionary/test.py b/tests/integration/test_executable_dictionary/test.py index c8efdd400ce..5e50a092a29 100644 --- a/tests/integration/test_executable_dictionary/test.py +++ b/tests/integration/test_executable_dictionary/test.py @@ -130,3 +130,46 @@ def test_executable_implicit_non_direct_input_bash(started_cluster): skip_test_msan(node) assert node.query("SELECT dictGet('executable_input_implicit_non_direct_bash', 'result', toUInt64(1))") == 'Key 1\n' assert node.query("SELECT dictGet('executable_input_implicit_non_direct_pool_bash', 'result', toUInt64(1))") == 'Key 1\n' + +def test_executable_source_python(started_cluster): + skip_test_msan(node) + assert node.query("SELECT * FROM dictionary(executable_source_simple_key_python) ORDER BY input") == '1\tValue 1\n2\tValue 2\n3\tValue 3\n' + assert node.query("SELECT dictGet('executable_source_simple_key_python', 'result', toUInt64(1))") == 'Value 1\n' + assert node.query("SELECT dictGet('executable_source_simple_key_python', 'result', toUInt64(2))") == 'Value 2\n' + assert node.query("SELECT dictGet('executable_source_simple_key_python', 'result', toUInt64(3))") == 'Value 3\n' + + assert node.query("SELECT * FROM dictionary('executable_source_complex_key_python') ORDER BY input") == '1\tValue 1\n2\tValue 2\n3\tValue 3\n' + assert node.query("SELECT dictGet('executable_source_complex_key_python', 'result', tuple(toUInt64(1)))") == 'Value 1\n' + assert node.query("SELECT dictGet('executable_source_complex_key_python', 'result', tuple(toUInt64(2)))") == 'Value 2\n' + assert node.query("SELECT dictGet('executable_source_complex_key_python', 'result', tuple(toUInt64(3)))") == 'Value 3\n' + +def test_executable_source_argument_python(started_cluster): + skip_test_msan(node) + assert node.query("SELECT * FROM dictionary(executable_source_simple_key_argument_python) ORDER BY input") == '1\tValue 1 1\n2\tValue 1 2\n3\tValue 1 3\n' + assert node.query("SELECT dictGet('executable_source_simple_key_argument_python', 'result', toUInt64(1))") == 'Value 1 1\n' + assert node.query("SELECT dictGet('executable_source_simple_key_argument_python', 'result', toUInt64(2))") == 'Value 1 2\n' + assert node.query("SELECT dictGet('executable_source_simple_key_argument_python', 'result', toUInt64(3))") == 'Value 1 3\n' + + assert node.query("SELECT * FROM dictionary(executable_source_complex_key_argument_python) ORDER BY input") == '1\tValue 1 1\n2\tValue 1 2\n3\tValue 1 3\n' + assert node.query("SELECT dictGet('executable_source_complex_key_argument_python', 'result', toUInt64(1))") == 'Value 1 1\n' + assert node.query("SELECT dictGet('executable_source_complex_key_argument_python', 'result', toUInt64(2))") == 'Value 1 2\n' + assert node.query("SELECT dictGet('executable_source_complex_key_argument_python', 'result', toUInt64(3))") == 'Value 1 3\n' + +def test_executable_source_updated_python(started_cluster): + skip_test_msan(node) + assert node.query("SELECT * FROM dictionary(executable_source_simple_key_update_python) ORDER BY input") == '1\tValue 0 1\n' + assert node.query("SELECT dictGet('executable_source_simple_key_update_python', 'result', toUInt64(1))") == 'Value 0 1\n' + + time.sleep(10) + + assert node.query("SELECT * FROM dictionary(executable_source_simple_key_update_python) ORDER BY input") == '1\tValue 1 1\n' + assert node.query("SELECT dictGet('executable_source_simple_key_update_python', 'result', toUInt64(1))") == 'Value 1 1\n' + + assert node.query("SELECT * FROM dictionary(executable_source_complex_key_update_python) ORDER BY input") == '1\tValue 0 1\n' + assert node.query("SELECT dictGet('executable_source_complex_key_update_python', 'result', toUInt64(1))") == 'Value 0 1\n' + + time.sleep(10) + + assert node.query("SELECT * FROM dictionary(executable_source_complex_key_update_python) ORDER BY input") == '1\tValue 1 1\n' + assert node.query("SELECT dictGet('executable_source_complex_key_update_python', 'result', toUInt64(1))") == 'Value 1 1\n' + diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_source.py b/tests/integration/test_executable_dictionary/user_scripts/test_source.py new file mode 100755 index 00000000000..e105773c467 --- /dev/null +++ b/tests/integration/test_executable_dictionary/user_scripts/test_source.py @@ -0,0 +1,10 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + print('1' + '\t' + 'Value 1', end='\n') + print('2' + '\t' + 'Value 2', end='\n') + print('3' + '\t' + 'Value 3', end='\n') + + sys.stdout.flush() diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_source_argument.py b/tests/integration/test_executable_dictionary/user_scripts/test_source_argument.py new file mode 100755 index 00000000000..881e73adc97 --- /dev/null +++ b/tests/integration/test_executable_dictionary/user_scripts/test_source_argument.py @@ -0,0 +1,12 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + arg = int(sys.argv[1]) + + print('1' + '\t' + 'Value ' + str(arg) + ' 1', end='\n') + print('2' + '\t' + 'Value ' + str(arg) + ' 2', end='\n') + print('3' + '\t' + 'Value ' + str(arg) + ' 3', end='\n') + + sys.stdout.flush() diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_source_update.py b/tests/integration/test_executable_dictionary/user_scripts/test_source_update.py new file mode 100755 index 00000000000..99388f9ada3 --- /dev/null +++ b/tests/integration/test_executable_dictionary/user_scripts/test_source_update.py @@ -0,0 +1,12 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + update_field_value = 0 + + if len(sys.argv) >= 2: + update_field_value = int(sys.argv[1]) + + print('1' + '\t' + 'Value ' + str(update_field_value) + ' 1', end='\n') + sys.stdout.flush() From de49a25185864b9b8fcbf51e83294b48aa4c425e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 25 Dec 2021 20:26:54 +0300 Subject: [PATCH 0516/1260] Fixed build --- .../ExecutableDictionarySource.cpp | 3 +- .../ExecutablePoolDictionarySource.cpp | 4 +-- src/Processors/Sources/ShellCommandSource.cpp | 31 +++++++++---------- .../test.py | 2 +- 4 files changed, 19 insertions(+), 21 deletions(-) diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 9314096e034..99d94b38d43 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -5,8 +5,7 @@ #include #include -#include - +#include #include #include diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 1ba466aed7b..17f5759b61c 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -1,11 +1,11 @@ #include "ExecutablePoolDictionarySource.h" -#include -#include #include #include +#include +#include #include #include diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 34f244d0d9e..db9e2365c42 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -24,7 +24,6 @@ namespace ErrorCodes extern const int CANNOT_FCNTL; extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR; extern const int CANNOT_SELECT; - extern const int CANNOT_CLOSE_FILE; extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR; } @@ -129,15 +128,15 @@ public: if (-1 == res && errno != EINTR) throwFromErrno("Cannot read from pipe ", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR); - if (res == 0) { + if (res == 0) break; - } if (res > 0) bytes_read += res; } - if (bytes_read > 0) { + if (bytes_read > 0) + { working_buffer = internal_buffer; working_buffer.resize(bytes_read); } @@ -197,7 +196,8 @@ public: } } - void reset() const { + void reset() const + { makeFdBlocking(fd); } @@ -220,15 +220,16 @@ public: : func(std::move(func_)) {} - std::unique_ptr buildCommand() { - if (returned_command) { + std::unique_ptr buildCommand() + { + if (returned_command) return std::move(returned_command); - } return func(); } - void returnCommand(std::unique_ptr command) { + void returnCommand(std::unique_ptr command) + { returned_command = std::move(command); } @@ -316,16 +317,15 @@ namespace if (thread.joinable()) thread.join(); - if (command_is_invalid) { + if (command_is_invalid) command = nullptr; - } - if (command_holder && process_pool) { + if (command_holder && process_pool) + { bool valid_command = configuration.read_fixed_number_of_rows && current_read_rows >= configuration.number_of_rows_to_read; - if (command && valid_command) { + if (command && valid_command) command_holder->returnCommand(std::move(command)); - } process_pool->returnObject(std::move(command_holder)); } @@ -337,9 +337,8 @@ namespace { rethrowExceptionDuringSendDataIfNeeded(); - if (configuration.read_fixed_number_of_rows && current_read_rows >= configuration.number_of_rows_to_read) { + if (configuration.read_fixed_number_of_rows && current_read_rows >= configuration.number_of_rows_to_read) return {}; - } Chunk chunk; diff --git a/tests/integration/test_table_functions_access_rights/test.py b/tests/integration/test_table_functions_access_rights/test.py index 16f18407960..90106303315 100644 --- a/tests/integration/test_table_functions_access_rights/test.py +++ b/tests/integration/test_table_functions_access_rights/test.py @@ -39,7 +39,7 @@ def test_merge(): instance.query("GRANT CREATE TEMPORARY TABLE ON *.* TO A") assert "no tables in database matches" in instance.query_and_get_error(select_query, user = 'A') - + instance.query("GRANT SELECT ON default.table1 TO A") assert instance.query(select_query, user = 'A') == "1\n" From c2977dbf580cb0e8ddff47efdb545e075714fb9c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 27 Dec 2021 15:14:43 +0300 Subject: [PATCH 0517/1260] Updated test_executable_table_function integration tests --- src/Processors/Sources/ShellCommandSource.cpp | 4 +- src/Storages/StorageExecutable.cpp | 12 +- .../test_executable_table_function/test.py | 175 +++++++++++++++--- .../user_scripts/test_argument.sh | 3 - .../user_scripts/test_input.py | 8 + .../user_scripts/test_input_argument.py | 10 + .../user_scripts/test_input_chunk_header.py | 14 ++ .../user_scripts/test_input_multiple_pipes.py | 19 ++ .../user_scripts/test_input_multiple_pipes.sh | 5 - .../user_scripts/test_input_signalled.py | 13 ++ .../user_scripts/test_input_slow.py | 10 + .../user_scripts/test_input_sum.py | 10 + .../user_scripts/test_no_input.sh | 4 +- 13 files changed, 243 insertions(+), 44 deletions(-) delete mode 100755 tests/integration/test_executable_table_function/user_scripts/test_argument.sh create mode 100755 tests/integration/test_executable_table_function/user_scripts/test_input.py create mode 100755 tests/integration/test_executable_table_function/user_scripts/test_input_argument.py create mode 100755 tests/integration/test_executable_table_function/user_scripts/test_input_chunk_header.py create mode 100755 tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes.py delete mode 100755 tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes.sh create mode 100755 tests/integration/test_executable_table_function/user_scripts/test_input_signalled.py create mode 100755 tests/integration/test_executable_table_function/user_scripts/test_input_slow.py create mode 100755 tests/integration/test_executable_table_function/user_scripts/test_input_sum.py diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index db9e2365c42..b0cf945b4f1 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -299,9 +299,9 @@ namespace if (configuration.read_number_of_rows_from_process_output) { /// TODO: Move to generate - readText(configuration.number_of_rows_to_read, command->out); + readText(configuration.number_of_rows_to_read, timeout_command_out); char dummy; - readChar(dummy, command->out); + readChar(dummy, timeout_command_out); } max_block_size = configuration.number_of_rows_to_read; diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 6a99a088571..5d2185133ed 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -168,23 +168,27 @@ void registerStorageExecutable(StorageFactory & factory) max_command_execution_time = max_execution_time_seconds; settings.max_command_execution_time = max_command_execution_time; - if (args.storage_def->settings) - settings.loadFromQuery(*args.storage_def); } + if (args.storage_def->settings) + settings.loadFromQuery(*args.storage_def); + auto global_context = args.getContext()->getGlobalContext(); return StorageExecutable::create(args.table_id, format, settings, input_queries, columns, constraints); }; + StorageFactory::StorageFeatures storage_features; + storage_features.supports_settings = true; + factory.registerStorage("Executable", [&](const StorageFactory::Arguments & args) { return register_storage(args, false /*is_executable_pool*/); - }); + }, storage_features); factory.registerStorage("ExecutablePool", [&](const StorageFactory::Arguments & args) { return register_storage(args, true /*is_executable_pool*/); - }); + }, storage_features); } }; diff --git a/tests/integration/test_executable_table_function/test.py b/tests/integration/test_executable_table_function/test.py index f5537e26b94..f713d34eeb5 100644 --- a/tests/integration/test_executable_table_function/test.py +++ b/tests/integration/test_executable_table_function/test.py @@ -30,69 +30,186 @@ def started_cluster(): copy_file_to_container(os.path.join(SCRIPT_DIR, 'user_scripts/.'), '/var/lib/clickhouse/user_scripts', node.docker_id) node.restart_clickhouse() + node.query("CREATE TABLE test_data_table (id UInt64) ENGINE=TinyLog;") + node.query("INSERT INTO test_data_table VALUES (0), (1), (2);") + yield cluster finally: cluster.shutdown() -def test_executable_function_no_input(started_cluster): +def test_executable_function_no_input_bash(started_cluster): skip_test_msan(node) - assert node.query("SELECT * FROM executable('test_no_input.sh', 'TabSeparated', 'value UInt64')") == '1\n' + assert node.query("SELECT * FROM executable('test_no_input.sh', 'TabSeparated', 'value String')") == 'Key 0\nKey 1\nKey 2\n' -def test_executable_function_input(started_cluster): +def test_executable_function_input_bash(started_cluster): skip_test_msan(node) - assert node.query("SELECT * FROM executable('test_input.sh', 'TabSeparated', 'value String', (SELECT 1))") == 'Key 1\n' -def test_executable_function_input_multiple_pipes(started_cluster): + query = "SELECT * FROM executable('test_input.sh', 'TabSeparated', 'value String', {source})" + assert node.query(query.format(source='(SELECT 1)')) == 'Key 1\n' + assert node.query(query.format(source='(SELECT id FROM test_data_table)')) == 'Key 0\nKey 1\nKey 2\n' + +def test_executable_function_input_python(started_cluster): skip_test_msan(node) - actual = node.query("SELECT * FROM executable('test_input_multiple_pipes.sh', 'TabSeparated', 'value String', (SELECT 1), (SELECT 2), (SELECT 3))") + + query = "SELECT * FROM executable('test_input.py', 'TabSeparated', 'value String', {source})" + assert node.query(query.format(source='(SELECT 1)')) == 'Key 1\n' + assert node.query(query.format(source='(SELECT id FROM test_data_table)')) == 'Key 0\nKey 1\nKey 2\n' + +def test_executable_function_input_sum_python(started_cluster): + skip_test_msan(node) + + query = "SELECT * FROM executable('test_input_sum.py', 'TabSeparated', 'value UInt64', {source})" + assert node.query(query.format(source='(SELECT 1, 1)')) == '2\n' + assert node.query(query.format(source='(SELECT id, id FROM test_data_table)')) == '0\n2\n4\n' + +def test_executable_function_input_argument_python(started_cluster): + skip_test_msan(node) + + query = "SELECT * FROM executable('test_input_argument.py 1', 'TabSeparated', 'value String', {source})" + assert node.query(query.format(source='(SELECT 1)')) == 'Key 1 1\n' + assert node.query(query.format(source='(SELECT id FROM test_data_table)')) == 'Key 1 0\nKey 1 1\nKey 1 2\n' + +def test_executable_function_input_signalled_python(started_cluster): + skip_test_msan(node) + + query = "SELECT * FROM executable('test_input_signalled.py', 'TabSeparated', 'value String', {source})" + assert node.query(query.format(source='(SELECT 1)')) == '' + assert node.query(query.format(source='(SELECT id FROM test_data_table)')) == '' + +def test_executable_function_input_slow_python(started_cluster): + skip_test_msan(node) + + query = "SELECT * FROM executable('test_input_slow.py', 'TabSeparated', 'value String', {source})" + assert node.query_and_get_error(query.format(source='(SELECT 1)')) + assert node.query_and_get_error(query.format(source='(SELECT id FROM test_data_table)')) + +def test_executable_function_input_multiple_pipes_python(started_cluster): + skip_test_msan(node) + query = "SELECT * FROM executable('test_input_multiple_pipes.py', 'TabSeparated', 'value String', {source})" + actual = node.query(query.format(source='(SELECT 1), (SELECT 2), (SELECT 3)')) expected = 'Key from 4 fd 3\nKey from 3 fd 2\nKey from 0 fd 1\n' assert actual == expected -def test_executable_function_argument(started_cluster): - skip_test_msan(node) - assert node.query("SELECT * FROM executable('test_argument.sh 1', 'TabSeparated', 'value String')") == 'Key 1\n' + actual = node.query(query.format(source='(SELECT id FROM test_data_table), (SELECT 2), (SELECT 3)')) + expected = 'Key from 4 fd 3\nKey from 3 fd 2\nKey from 0 fd 0\nKey from 0 fd 1\nKey from 0 fd 2\n' + assert actual == expected -def test_executable_storage_no_input(started_cluster): +def test_executable_storage_no_input_bash(started_cluster): skip_test_msan(node) node.query("DROP TABLE IF EXISTS test_table") node.query("CREATE TABLE test_table (value UInt64) ENGINE=Executable('test_no_input.sh', 'TabSeparated')") - assert node.query("SELECT * FROM test_table") == '1\n' + assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' node.query("DROP TABLE test_table") -def test_executable_storage_input(started_cluster): +def test_executable_storage_input_bash(started_cluster): skip_test_msan(node) + + query = "CREATE TABLE test_table (value String) ENGINE=Executable('test_input.sh', 'TabSeparated', {source})" + node.query("DROP TABLE IF EXISTS test_table") - node.query("CREATE TABLE test_table (value String) ENGINE=Executable('test_input.sh', 'TabSeparated', (SELECT 1))") + node.query(query.format(source='(SELECT 1)')) assert node.query("SELECT * FROM test_table") == 'Key 1\n' node.query("DROP TABLE test_table") -def test_executable_storage_input_multiple_pipes(started_cluster): - skip_test_msan(node) - node.query("DROP TABLE IF EXISTS test_table") - node.query("CREATE TABLE test_table (value String) ENGINE=Executable('test_input_multiple_pipes.sh', 'TabSeparated', (SELECT 1), (SELECT 2), (SELECT 3))") - actual = node.query("SELECT * FROM test_table") - expected = 'Key from 4 fd 3\nKey from 3 fd 2\nKey from 0 fd 1\n' - assert actual == expected + node.query(query.format(source='(SELECT id FROM test_data_table)')) + assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' node.query("DROP TABLE test_table") -def test_executable_storage_argument(started_cluster): +def test_executable_storage_input_python(started_cluster): skip_test_msan(node) + + query = "CREATE TABLE test_table (value String) ENGINE=Executable('test_input.py', 'TabSeparated', {source})" + node.query("DROP TABLE IF EXISTS test_table") - node.query("CREATE TABLE test_table (value String) ENGINE=Executable('test_argument.sh 1', 'TabSeparated')") + node.query(query.format(source='(SELECT 1)')) assert node.query("SELECT * FROM test_table") == 'Key 1\n' node.query("DROP TABLE test_table") -def test_executable_pool_storage(started_cluster): + node.query(query.format(source='(SELECT id FROM test_data_table)')) + assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' + node.query("DROP TABLE test_table") + +def test_executable_storage_input_send_chunk_header_python(started_cluster): skip_test_msan(node) + + query = "CREATE TABLE test_table (value String) ENGINE=Executable('test_input_chunk_header.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1" + node.query("DROP TABLE IF EXISTS test_table") - node.query("CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_process_pool.sh', 'TabSeparated', (SELECT 1))") + node.query(query.format(source='(SELECT 1)')) assert node.query("SELECT * FROM test_table") == 'Key 1\n' node.query("DROP TABLE test_table") -def test_executable_pool_storage_multiple_pipes(started_cluster): - skip_test_msan(node) - node.query("DROP TABLE IF EXISTS test_table") - node.query("CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_process_pool_multiple_pipes.sh', 'TabSeparated', (SELECT 1), (SELECT 2), (SELECT 3))") - assert node.query("SELECT * FROM test_table") == 'Key from 4 fd 3\nKey from 3 fd 2\nKey from 0 fd 1\n' + node.query(query.format(source='(SELECT id FROM test_data_table)')) + assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' node.query("DROP TABLE test_table") + +def test_executable_storage_input_sum_python(started_cluster): + skip_test_msan(node) + + query = "CREATE TABLE test_table (value UInt64) ENGINE=Executable('test_input_sum.py', 'TabSeparated', {source})" + + node.query("DROP TABLE IF EXISTS test_table") + node.query(query.format(source='(SELECT 1, 1)')) + assert node.query("SELECT * FROM test_table") == '2\n' + node.query("DROP TABLE test_table") + + node.query(query.format(source='(SELECT id, id FROM test_data_table)')) + assert node.query("SELECT * FROM test_table") == '0\n2\n4\n' + node.query("DROP TABLE test_table") + +def test_executable_storage_input_argument_python(started_cluster): + skip_test_msan(node) + + query = "CREATE TABLE test_table (value String) ENGINE=Executable('test_input_argument.py 1', 'TabSeparated', {source})" + + node.query("DROP TABLE IF EXISTS test_table") + node.query(query.format(source='(SELECT 1)')) + assert node.query("SELECT * FROM test_table") == 'Key 1 1\n' + node.query("DROP TABLE test_table") + + node.query(query.format(source='(SELECT id FROM test_data_table)')) + assert node.query("SELECT * FROM test_table") == 'Key 1 0\nKey 1 1\nKey 1 2\n' + node.query("DROP TABLE test_table") + +def test_executable_storage_input_signalled_python(started_cluster): + skip_test_msan(node) + + query = "CREATE TABLE test_table (value String) ENGINE=Executable('test_input_signalled.py', 'TabSeparated', {source})" + + node.query("DROP TABLE IF EXISTS test_table") + node.query(query.format(source='(SELECT 1)')) + assert node.query("SELECT * FROM test_table") == '' + node.query("DROP TABLE test_table") + + node.query(query.format(source='(SELECT id FROM test_data_table)')) + assert node.query("SELECT * FROM test_table") == '' + node.query("DROP TABLE test_table") + +def test_executable_storage_input_slow_python(started_cluster): + skip_test_msan(node) + + query = "CREATE TABLE test_table (value String) ENGINE=Executable('test_input_slow.py', 'TabSeparated', {source}) SETTINGS command_read_timeout=2500" + + node.query("DROP TABLE IF EXISTS test_table") + node.query(query.format(source='(SELECT 1)')) + assert node.query_and_get_error("SELECT * FROM test_table") + node.query("DROP TABLE test_table") + + node.query(query.format(source='(SELECT id FROM test_data_table)')) + assert node.query_and_get_error("SELECT * FROM test_table") + node.query("DROP TABLE test_table") + +# def test_executable_pool_storage(started_cluster): +# skip_test_msan(node) +# node.query("DROP TABLE IF EXISTS test_table") +# node.query("CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_process_pool.sh', 'TabSeparated', (SELECT 1))") +# assert node.query("SELECT * FROM test_table") == 'Key 1\n' +# node.query("DROP TABLE test_table") + +# def test_executable_pool_storage_multiple_pipes(started_cluster): +# skip_test_msan(node) +# node.query("DROP TABLE IF EXISTS test_table") +# node.query("CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_process_pool_multiple_pipes.sh', 'TabSeparated', (SELECT 1), (SELECT 2), (SELECT 3))") +# assert node.query("SELECT * FROM test_table") == 'Key from 4 fd 3\nKey from 3 fd 2\nKey from 0 fd 1\n' +# node.query("DROP TABLE test_table") diff --git a/tests/integration/test_executable_table_function/user_scripts/test_argument.sh b/tests/integration/test_executable_table_function/user_scripts/test_argument.sh deleted file mode 100755 index 89634031d2b..00000000000 --- a/tests/integration/test_executable_table_function/user_scripts/test_argument.sh +++ /dev/null @@ -1,3 +0,0 @@ -#!/bin/bash - -echo "Key $1" diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input.py b/tests/integration/test_executable_table_function/user_scripts/test_input.py new file mode 100755 index 00000000000..835ab1f441a --- /dev/null +++ b/tests/integration/test_executable_table_function/user_scripts/test_input.py @@ -0,0 +1,8 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + for line in sys.stdin: + print("Key " + line, end='') + sys.stdout.flush() diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_argument.py b/tests/integration/test_executable_table_function/user_scripts/test_input_argument.py new file mode 100755 index 00000000000..c1b2e5966d7 --- /dev/null +++ b/tests/integration/test_executable_table_function/user_scripts/test_input_argument.py @@ -0,0 +1,10 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + arg = int(sys.argv[1]) + + for line in sys.stdin: + print("Key " + str(arg) + " " + line, end='') + sys.stdout.flush() diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_chunk_header.py b/tests/integration/test_executable_table_function/user_scripts/test_input_chunk_header.py new file mode 100755 index 00000000000..5dc03e1c507 --- /dev/null +++ b/tests/integration/test_executable_table_function/user_scripts/test_input_chunk_header.py @@ -0,0 +1,14 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + for chunk_header in sys.stdin: + chunk_length = int(chunk_header) + + while chunk_length != 0: + line = sys.stdin.readline() + chunk_length -= 1 + print("Key " + line, end='') + + sys.stdout.flush() diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes.py b/tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes.py new file mode 100755 index 00000000000..64590cbc16a --- /dev/null +++ b/tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes.py @@ -0,0 +1,19 @@ +#!/usr/bin/python3 + +import sys +import os + +if __name__ == '__main__': + fd3 = os.fdopen(3) + fd4 = os.fdopen(4) + + for line in fd4: + print("Key from 4 fd " + line, end='') + + for line in fd3: + print("Key from 3 fd " + line, end='') + + for line in sys.stdin: + print("Key from 0 fd " + line, end='') + + sys.stdout.flush() diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes.sh b/tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes.sh deleted file mode 100755 index 1e53e3211dc..00000000000 --- a/tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes.sh +++ /dev/null @@ -1,5 +0,0 @@ -#!/bin/bash - -while read -t 250 -u 4 read_data; do printf "Key from 4 fd $read_data\n"; done -while read -t 250 -u 3 read_data; do printf "Key from 3 fd $read_data\n"; done -while read -t 250 read_data; do printf "Key from 0 fd $read_data\n"; done diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_signalled.py b/tests/integration/test_executable_table_function/user_scripts/test_input_signalled.py new file mode 100755 index 00000000000..27c8bc4840e --- /dev/null +++ b/tests/integration/test_executable_table_function/user_scripts/test_input_signalled.py @@ -0,0 +1,13 @@ +#!/usr/bin/python3 + +import sys +import os +import signal +import time + +if __name__ == '__main__': + for line in sys.stdin: + os.signal(os.getpid(), signal.SIGTERM) + + print("Key " + line, end='') + sys.stdout.flush() diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_slow.py b/tests/integration/test_executable_table_function/user_scripts/test_input_slow.py new file mode 100755 index 00000000000..4c2abe89e33 --- /dev/null +++ b/tests/integration/test_executable_table_function/user_scripts/test_input_slow.py @@ -0,0 +1,10 @@ +#!/usr/bin/python3 + +import sys +import time + +if __name__ == '__main__': + for line in sys.stdin: + time.sleep(25) + print("Key " + line, end='') + sys.stdout.flush() diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_sum.py b/tests/integration/test_executable_table_function/user_scripts/test_input_sum.py new file mode 100755 index 00000000000..432d7a13a2f --- /dev/null +++ b/tests/integration/test_executable_table_function/user_scripts/test_input_sum.py @@ -0,0 +1,10 @@ +#!/usr/bin/python3 + +import sys +import re + +if __name__ == '__main__': + for line in sys.stdin: + line_split = re.split(r'\t+', line) + print(int(line_split[0]) + int(line_split[1]), end='\n') + sys.stdout.flush() diff --git a/tests/integration/test_executable_table_function/user_scripts/test_no_input.sh b/tests/integration/test_executable_table_function/user_scripts/test_no_input.sh index 9e8b3be63d6..13d172a5be4 100755 --- a/tests/integration/test_executable_table_function/user_scripts/test_no_input.sh +++ b/tests/integration/test_executable_table_function/user_scripts/test_no_input.sh @@ -1,3 +1,5 @@ #!/bin/bash -echo "1" +printf "Key 0\n"; +printf "Key 1\n"; +printf "Key 2\n"; From fbe65658a1c6458de7ce85e39e47b81a973ddaac Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 27 Dec 2021 20:25:56 +0300 Subject: [PATCH 0518/1260] Fixed tests --- src/Processors/Sources/ShellCommandSource.cpp | 48 ++++++--- .../test_executable_table_function/test.py | 101 +++++++++++++++++- .../user_scripts/test_input_argument_pool.py | 17 +++ .../test_input_multiple_pipes_pool.py | 61 +++++++++++ .../user_scripts/test_input_pool.py | 15 +++ .../user_scripts/test_input_sum_pool.py | 17 +++ .../user_scripts/test_no_input.py | 9 ++ 7 files changed, 253 insertions(+), 15 deletions(-) create mode 100755 tests/integration/test_executable_table_function/user_scripts/test_input_argument_pool.py create mode 100755 tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes_pool.py create mode 100755 tests/integration/test_executable_table_function/user_scripts/test_input_pool.py create mode 100755 tests/integration/test_executable_table_function/user_scripts/test_input_sum_pool.py create mode 100755 tests/integration/test_executable_table_function/user_scripts/test_no_input.py diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index b0cf945b4f1..1b6e504f5cb 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -107,8 +107,7 @@ class TimeoutReadBufferFromFileDescriptor : public BufferWithOwnMemory() - , fd(fd_) + : fd(fd_) , timeout_milliseconds(timeout_milliseconds_) { makeFdNonBlocking(fd); @@ -167,8 +166,7 @@ class TimeoutWriteBufferFromFileDescriptor : public BufferWithOwnMemory() - , fd(fd_) + : fd(fd_) , timeout_milliseconds(timeout_milliseconds_) { makeFdNonBlocking(fd); @@ -253,16 +251,19 @@ namespace using SendDataTask = std::function; ShellCommandSource( - ContextPtr context, - const std::string & format, + ContextPtr context_, + const std::string & format_, size_t command_read_timeout_milliseconds, - const Block & sample_block, + const Block & sample_block_, std::unique_ptr && command_, std::vector && send_data_tasks = {}, const ShellCommandSourceConfiguration & configuration_ = {}, std::unique_ptr && command_holder_ = nullptr, std::shared_ptr process_pool_ = nullptr) - : SourceWithProgress(sample_block) + : SourceWithProgress(sample_block_) + , context(context_) + , format(format_) + , sample_block(sample_block_) , command(std::move(command_)) , configuration(configuration_) , timeout_command_out(command->out.getFD(), command_read_timeout_milliseconds) @@ -275,6 +276,7 @@ namespace { try { + std::cerr << "SendDataThread task start" << std::endl; task(); } catch (...) @@ -298,10 +300,8 @@ namespace if (configuration.read_number_of_rows_from_process_output) { - /// TODO: Move to generate - readText(configuration.number_of_rows_to_read, timeout_command_out); - char dummy; - readChar(dummy, timeout_command_out); + /// Initialize executor in generate + return; } max_block_size = configuration.number_of_rows_to_read; @@ -337,8 +337,22 @@ namespace { rethrowExceptionDuringSendDataIfNeeded(); - if (configuration.read_fixed_number_of_rows && current_read_rows >= configuration.number_of_rows_to_read) - return {}; + if (configuration.read_fixed_number_of_rows) + { + if (!executor && configuration.read_number_of_rows_from_process_output) + { + readText(configuration.number_of_rows_to_read, timeout_command_out); + char dummy; + readChar(dummy, timeout_command_out); + + size_t max_block_size = configuration.number_of_rows_to_read; + pipeline = QueryPipeline(Pipe(context->getInputFormat(format, timeout_command_out, sample_block, max_block_size))); + executor = std::make_unique(pipeline); + } + + if (current_read_rows >= configuration.number_of_rows_to_read) + return {}; + } Chunk chunk; @@ -388,6 +402,10 @@ namespace } } + ContextPtr context; + std::string format; + Block sample_block; + std::unique_ptr command; ShellCommandSourceConfiguration configuration; @@ -424,6 +442,8 @@ namespace void transform(Chunk & chunk) override { + std::cerr << "SendingChunkHeaderTransform::transform " << chunk.getNumRows() << std::endl; + writeText(chunk.getNumRows(), *buffer); writeChar('\n', *buffer); } diff --git a/tests/integration/test_executable_table_function/test.py b/tests/integration/test_executable_table_function/test.py index f713d34eeb5..f883b623ef7 100644 --- a/tests/integration/test_executable_table_function/test.py +++ b/tests/integration/test_executable_table_function/test.py @@ -42,6 +42,10 @@ def test_executable_function_no_input_bash(started_cluster): skip_test_msan(node) assert node.query("SELECT * FROM executable('test_no_input.sh', 'TabSeparated', 'value String')") == 'Key 0\nKey 1\nKey 2\n' +def test_executable_function_no_input_python(started_cluster): + skip_test_msan(node) + assert node.query("SELECT * FROM executable('test_no_input.py', 'TabSeparated', 'value String')") == 'Key 0\nKey 1\nKey 2\n' + def test_executable_function_input_bash(started_cluster): skip_test_msan(node) @@ -98,7 +102,14 @@ def test_executable_function_input_multiple_pipes_python(started_cluster): def test_executable_storage_no_input_bash(started_cluster): skip_test_msan(node) node.query("DROP TABLE IF EXISTS test_table") - node.query("CREATE TABLE test_table (value UInt64) ENGINE=Executable('test_no_input.sh', 'TabSeparated')") + node.query("CREATE TABLE test_table (value String) ENGINE=Executable('test_no_input.sh', 'TabSeparated')") + assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' + node.query("DROP TABLE test_table") + +def test_executable_storage_no_input_python(started_cluster): + skip_test_msan(node) + node.query("DROP TABLE IF EXISTS test_table") + node.query("CREATE TABLE test_table (value String) ENGINE=Executable('test_no_input.py', 'TabSeparated')") assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' node.query("DROP TABLE test_table") @@ -200,6 +211,94 @@ def test_executable_storage_input_slow_python(started_cluster): assert node.query_and_get_error("SELECT * FROM test_table") node.query("DROP TABLE test_table") +# def test_executable_pool_storage_input_python(started_cluster): +# skip_test_msan(node) + +# query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" + +# node.query("DROP TABLE IF EXISTS test_table") +# node.query(query.format(source='(SELECT 1)')) + +# assert node.query("SELECT * FROM test_table") == 'Key 1\n' +# assert node.query("SELECT * FROM test_table") == 'Key 1\n' +# assert node.query("SELECT * FROM test_table") == 'Key 1\n' + +# node.query("DROP TABLE test_table") + +# node.query(query.format(source='(SELECT id FROM test_data_table)')) + +# assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' +# assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' +# assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' + +# node.query("DROP TABLE test_table") + +# def test_executable_pool_storage_input_sum_python(started_cluster): +# skip_test_msan(node) + +# query = "CREATE TABLE test_table (value UInt64) ENGINE=ExecutablePool('test_input_sum_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" + +# node.query("DROP TABLE IF EXISTS test_table") +# node.query(query.format(source='(SELECT 1, 1)')) + +# assert node.query("SELECT * FROM test_table") == '2\n' +# assert node.query("SELECT * FROM test_table") == '2\n' +# assert node.query("SELECT * FROM test_table") == '2\n' + +# node.query("DROP TABLE test_table") + +# node.query(query.format(source='(SELECT id, id FROM test_data_table)')) + +# assert node.query("SELECT * FROM test_table") == '0\n2\n4\n' +# assert node.query("SELECT * FROM test_table") == '0\n2\n4\n' +# assert node.query("SELECT * FROM test_table") == '0\n2\n4\n' + +# node.query("DROP TABLE test_table") + +# def test_executable_pool_storage_input_argument_python(started_cluster): +# skip_test_msan(node) + +# query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_argument_pool.py 1', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" + +# node.query("DROP TABLE IF EXISTS test_table") +# node.query(query.format(source='(SELECT 1)')) + +# assert node.query("SELECT * FROM test_table") == 'Key 1 1\n' +# assert node.query("SELECT * FROM test_table") == 'Key 1 1\n' +# assert node.query("SELECT * FROM test_table") == 'Key 1 1\n' + +# node.query("DROP TABLE test_table") + +# node.query(query.format(source='(SELECT id FROM test_data_table)')) + +# assert node.query("SELECT * FROM test_table") == 'Key 1 0\nKey 1 1\nKey 1 2\n' +# assert node.query("SELECT * FROM test_table") == 'Key 1 0\nKey 1 1\nKey 1 2\n' +# assert node.query("SELECT * FROM test_table") == 'Key 1 0\nKey 1 1\nKey 1 2\n' + +# node.query("DROP TABLE test_table") + +# def test_executable_pool_storage_input_multiple_blocks_python(started_cluster): +# skip_test_msan(node) + +# query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_multiple_blocks_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" + +# node.query("DROP TABLE IF EXISTS test_table") +# node.query(query.format(source='(SELECT 1)')) + +# assert node.query("SELECT * FROM test_table") == 'Key 1\n' +# assert node.query("SELECT * FROM test_table") == 'Key 1\n' +# assert node.query("SELECT * FROM test_table") == 'Key 1\n' + +# node.query("DROP TABLE test_table") + +# node.query(query.format(source='(SELECT id FROM test_data_table)')) + +# assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' +# assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' +# assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' + +# node.query("DROP TABLE test_table") + # def test_executable_pool_storage(started_cluster): # skip_test_msan(node) # node.query("DROP TABLE IF EXISTS test_table") diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_argument_pool.py b/tests/integration/test_executable_table_function/user_scripts/test_input_argument_pool.py new file mode 100755 index 00000000000..378a6ef4391 --- /dev/null +++ b/tests/integration/test_executable_table_function/user_scripts/test_input_argument_pool.py @@ -0,0 +1,17 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + arg = int(sys.argv[1]) + + for chunk_header in sys.stdin: + chunk_length = int(chunk_header) + print(str(chunk_length), end='\n') + + while chunk_length != 0: + line = sys.stdin.readline() + chunk_length -= 1 + print("Key " + str(arg) + " " + line, end='') + + sys.stdout.flush() diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes_pool.py b/tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes_pool.py new file mode 100755 index 00000000000..fc52d67aeba --- /dev/null +++ b/tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes_pool.py @@ -0,0 +1,61 @@ +#!/usr/bin/python3 + +import sys +import os + +if __name__ == '__main__': + fd3 = os.fdopen(3) + fd4 = os.fdopen(4) + + for chunk_header in fd4: + fd4_chunk_length = int(chunk_header) + print(str(fd4_chunk_length), end='\n') + + while fd4_chunk_length != 0: + line = sys.stdin.readline() + fd4_chunk_length -= 1 + print("Key from fd 4 " + line, end='') + + sys.stdout.flush() + + for chunk_header in fd3: + fd3_chunk_length = int(chunk_header) + print(str(fd3_chunk_length), end='\n') + + while fd3_chunk_length != 0: + line = sys.stdin.readline() + fd3_chunk_length -= 1 + print("Key from fd 3 " + line, end='') + + sys.stdout.flush() + + for chunk_header in sys.stdin: + chunk_length = int(chunk_header) + print(str(chunk_length), end='\n') + + while chunk_length != 0: + line = sys.stdin.readline() + chunk_length -= 1 + print("Key " + line, end='') + + sys.stdout.flush() + +#!/usr/bin/python3 + +import sys +import os + +if __name__ == '__main__': + fd3 = os.fdopen(3) + fd4 = os.fdopen(4) + + for line in fd4: + print("Key from 4 fd " + line, end='') + + for line in fd3: + print("Key from 3 fd " + line, end='') + + for line in sys.stdin: + print("Key from 0 fd " + line, end='') + + sys.stdout.flush() diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_pool.py b/tests/integration/test_executable_table_function/user_scripts/test_input_pool.py new file mode 100755 index 00000000000..ec4e9af23cd --- /dev/null +++ b/tests/integration/test_executable_table_function/user_scripts/test_input_pool.py @@ -0,0 +1,15 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + for chunk_header in sys.stdin: + chunk_length = int(chunk_header) + print(str(chunk_length), end='\n') + + while chunk_length != 0: + line = sys.stdin.readline() + chunk_length -= 1 + print("Key " + line, end='') + + sys.stdout.flush() diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_sum_pool.py b/tests/integration/test_executable_table_function/user_scripts/test_input_sum_pool.py new file mode 100755 index 00000000000..cd0de25fe87 --- /dev/null +++ b/tests/integration/test_executable_table_function/user_scripts/test_input_sum_pool.py @@ -0,0 +1,17 @@ +#!/usr/bin/python3 + +import sys +import re + +if __name__ == '__main__': + for chunk_header in sys.stdin: + chunk_length = int(chunk_header) + print(str(chunk_length), end='\n') + + while chunk_length != 0: + line = sys.stdin.readline() + line_split = re.split(r'\t+', line) + print(int(line_split[0]) + int(line_split[1]), end='\n') + chunk_length -= 1 + + sys.stdout.flush() diff --git a/tests/integration/test_executable_table_function/user_scripts/test_no_input.py b/tests/integration/test_executable_table_function/user_scripts/test_no_input.py new file mode 100755 index 00000000000..65b78f3d755 --- /dev/null +++ b/tests/integration/test_executable_table_function/user_scripts/test_no_input.py @@ -0,0 +1,9 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + print("Key 0") + print("Key 1") + print("Key 2") + sys.stdout.flush() From 61b36f22b871ded2f4bac9e046ecd96cf414fdbe Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 27 Dec 2021 21:28:30 +0300 Subject: [PATCH 0519/1260] Fixed style check --- src/Processors/Sources/ShellCommandSource.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 1b6e504f5cb..2e845a91529 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -276,7 +276,6 @@ namespace { try { - std::cerr << "SendDataThread task start" << std::endl; task(); } catch (...) @@ -442,8 +441,6 @@ namespace void transform(Chunk & chunk) override { - std::cerr << "SendingChunkHeaderTransform::transform " << chunk.getNumRows() << std::endl; - writeText(chunk.getNumRows(), *buffer); writeChar('\n', *buffer); } From 338637805002a1ec09ba32f4f429b24fe9ca9505 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 28 Dec 2021 00:19:28 +0300 Subject: [PATCH 0520/1260] Updated test_executable_table_function tests --- src/Processors/Sources/ShellCommandSource.cpp | 14 +- src/Storages/StorageExecutable.cpp | 33 +++++ .../test_executable_table_function/test.py | 137 +++++++++--------- .../test_input_multiple_pipes_pool.py | 52 +++---- 4 files changed, 131 insertions(+), 105 deletions(-) diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 2e845a91529..0b3298ce8b5 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -568,8 +568,18 @@ Pipe ShellCommandCoordinator::createPipe( tasks.emplace_back(std::move(task)); } - Pipe pipe(std::make_unique( - context, configuration.format, configuration.command_read_timeout_milliseconds, std::move(sample_block), std::move(process), std::move(tasks), source_configuration, std::move(process_holder), process_pool)); + auto source = std::make_unique( + context, + configuration.format, + configuration.command_read_timeout_milliseconds, + std::move(sample_block), + std::move(process), + std::move(tasks), + source_configuration, + std::move(process_holder), + process_pool); + auto pipe = Pipe(std::move(source)); + return pipe; } diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 5d2185133ed..d1e7f1a17e1 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -103,6 +104,38 @@ Pipe StorageExecutable::read( inputs.emplace_back(QueryPipelineBuilder::getPipe(interpreter.buildQueryPipeline())); } + if (settings.is_executable_pool) + { + /// For executable pool we read data from input streams and convert it to single blocks streams. + size_t inputs_size = inputs.size(); + for (size_t i = 0; i < inputs_size; ++i) + { + auto && input = inputs[i]; + QueryPipeline input_pipeline(std::move(input)); + PullingPipelineExecutor input_pipeline_executor(input_pipeline); + + auto header = input_pipeline_executor.getHeader(); + auto result_block = header.cloneEmpty(); + + size_t result_block_columns = result_block.columns(); + + Block result; + while (input_pipeline_executor.pull(result)) + { + for (size_t result_block_index = 0; result_block_index < result_block_columns; ++result_block_index) + { + auto & block_column = result.safeGetByPosition(result_block_index); + auto & result_block_column = result_block.safeGetByPosition(result_block_index); + + result_block_column.column->assumeMutable()->insertRangeFrom(*block_column.column, 0, block_column.column->size()); + } + } + + auto source = std::make_shared(std::move(result_block)); + inputs[i] = Pipe(std::move(source)); + } + } + auto sample_block = metadata_snapshot->getSampleBlock(); ShellCommandSourceConfiguration configuration; diff --git a/tests/integration/test_executable_table_function/test.py b/tests/integration/test_executable_table_function/test.py index f883b623ef7..fa4bba16e8d 100644 --- a/tests/integration/test_executable_table_function/test.py +++ b/tests/integration/test_executable_table_function/test.py @@ -1,6 +1,5 @@ import os import sys -import time import pytest @@ -211,104 +210,104 @@ def test_executable_storage_input_slow_python(started_cluster): assert node.query_and_get_error("SELECT * FROM test_table") node.query("DROP TABLE test_table") -# def test_executable_pool_storage_input_python(started_cluster): -# skip_test_msan(node) +def test_executable_function_input_multiple_pipes_python(started_cluster): + skip_test_msan(node) -# query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" + query = "CREATE TABLE test_table (value String) ENGINE=Executable('test_input_multiple_pipes.py', 'TabSeparated', {source})" -# node.query("DROP TABLE IF EXISTS test_table") -# node.query(query.format(source='(SELECT 1)')) + node.query("DROP TABLE IF EXISTS test_table") + node.query(query.format(source='(SELECT 1), (SELECT 2), (SELECT 3)')) + assert node.query("SELECT * FROM test_table") == 'Key from 4 fd 3\nKey from 3 fd 2\nKey from 0 fd 1\n' + node.query("DROP TABLE test_table") -# assert node.query("SELECT * FROM test_table") == 'Key 1\n' -# assert node.query("SELECT * FROM test_table") == 'Key 1\n' -# assert node.query("SELECT * FROM test_table") == 'Key 1\n' + node.query(query.format(source='(SELECT id FROM test_data_table), (SELECT 2), (SELECT 3)')) + assert node.query("SELECT * FROM test_table") == 'Key from 4 fd 3\nKey from 3 fd 2\nKey from 0 fd 0\nKey from 0 fd 1\nKey from 0 fd 2\n' + node.query("DROP TABLE test_table") -# node.query("DROP TABLE test_table") +def test_executable_pool_storage_input_python(started_cluster): + skip_test_msan(node) -# node.query(query.format(source='(SELECT id FROM test_data_table)')) + query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" -# assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' -# assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' -# assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' + node.query("DROP TABLE IF EXISTS test_table") + node.query(query.format(source='(SELECT 1)')) -# node.query("DROP TABLE test_table") + assert node.query("SELECT * FROM test_table") == 'Key 1\n' + assert node.query("SELECT * FROM test_table") == 'Key 1\n' + assert node.query("SELECT * FROM test_table") == 'Key 1\n' -# def test_executable_pool_storage_input_sum_python(started_cluster): -# skip_test_msan(node) + node.query("DROP TABLE test_table") -# query = "CREATE TABLE test_table (value UInt64) ENGINE=ExecutablePool('test_input_sum_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" + node.query(query.format(source='(SELECT id FROM test_data_table)')) -# node.query("DROP TABLE IF EXISTS test_table") -# node.query(query.format(source='(SELECT 1, 1)')) + assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' + assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' + assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' -# assert node.query("SELECT * FROM test_table") == '2\n' -# assert node.query("SELECT * FROM test_table") == '2\n' -# assert node.query("SELECT * FROM test_table") == '2\n' + node.query("DROP TABLE test_table") -# node.query("DROP TABLE test_table") +def test_executable_pool_storage_input_sum_python(started_cluster): + skip_test_msan(node) -# node.query(query.format(source='(SELECT id, id FROM test_data_table)')) + query = "CREATE TABLE test_table (value UInt64) ENGINE=ExecutablePool('test_input_sum_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" -# assert node.query("SELECT * FROM test_table") == '0\n2\n4\n' -# assert node.query("SELECT * FROM test_table") == '0\n2\n4\n' -# assert node.query("SELECT * FROM test_table") == '0\n2\n4\n' + node.query("DROP TABLE IF EXISTS test_table") + node.query(query.format(source='(SELECT 1, 1)')) -# node.query("DROP TABLE test_table") + assert node.query("SELECT * FROM test_table") == '2\n' + assert node.query("SELECT * FROM test_table") == '2\n' + assert node.query("SELECT * FROM test_table") == '2\n' -# def test_executable_pool_storage_input_argument_python(started_cluster): -# skip_test_msan(node) + node.query("DROP TABLE test_table") -# query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_argument_pool.py 1', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" + node.query(query.format(source='(SELECT id, id FROM test_data_table)')) -# node.query("DROP TABLE IF EXISTS test_table") -# node.query(query.format(source='(SELECT 1)')) + assert node.query("SELECT * FROM test_table") == '0\n2\n4\n' + assert node.query("SELECT * FROM test_table") == '0\n2\n4\n' + assert node.query("SELECT * FROM test_table") == '0\n2\n4\n' -# assert node.query("SELECT * FROM test_table") == 'Key 1 1\n' -# assert node.query("SELECT * FROM test_table") == 'Key 1 1\n' -# assert node.query("SELECT * FROM test_table") == 'Key 1 1\n' + node.query("DROP TABLE test_table") -# node.query("DROP TABLE test_table") +def test_executable_pool_storage_input_argument_python(started_cluster): + skip_test_msan(node) -# node.query(query.format(source='(SELECT id FROM test_data_table)')) + query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_argument_pool.py 1', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" -# assert node.query("SELECT * FROM test_table") == 'Key 1 0\nKey 1 1\nKey 1 2\n' -# assert node.query("SELECT * FROM test_table") == 'Key 1 0\nKey 1 1\nKey 1 2\n' -# assert node.query("SELECT * FROM test_table") == 'Key 1 0\nKey 1 1\nKey 1 2\n' + node.query("DROP TABLE IF EXISTS test_table") + node.query(query.format(source='(SELECT 1)')) -# node.query("DROP TABLE test_table") + assert node.query("SELECT * FROM test_table") == 'Key 1 1\n' + assert node.query("SELECT * FROM test_table") == 'Key 1 1\n' + assert node.query("SELECT * FROM test_table") == 'Key 1 1\n' -# def test_executable_pool_storage_input_multiple_blocks_python(started_cluster): -# skip_test_msan(node) + node.query("DROP TABLE test_table") -# query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_multiple_blocks_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" + node.query(query.format(source='(SELECT id FROM test_data_table)')) -# node.query("DROP TABLE IF EXISTS test_table") -# node.query(query.format(source='(SELECT 1)')) + assert node.query("SELECT * FROM test_table") == 'Key 1 0\nKey 1 1\nKey 1 2\n' + assert node.query("SELECT * FROM test_table") == 'Key 1 0\nKey 1 1\nKey 1 2\n' + assert node.query("SELECT * FROM test_table") == 'Key 1 0\nKey 1 1\nKey 1 2\n' -# assert node.query("SELECT * FROM test_table") == 'Key 1\n' -# assert node.query("SELECT * FROM test_table") == 'Key 1\n' -# assert node.query("SELECT * FROM test_table") == 'Key 1\n' + node.query("DROP TABLE test_table") -# node.query("DROP TABLE test_table") +def test_executable_pool_storage_input_python(started_cluster): + skip_test_msan(node) -# node.query(query.format(source='(SELECT id FROM test_data_table)')) + query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_multiple_pipes_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" -# assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' -# assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' -# assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' + node.query("DROP TABLE IF EXISTS test_table") + node.query(query.format(source='(SELECT 1), (SELECT 2), (SELECT 3)')) -# node.query("DROP TABLE test_table") + assert node.query("SELECT * FROM test_table") == 'Key from 4 fd 3\nKey from 3 fd 2\nKey from 0 fd 1\n' + assert node.query("SELECT * FROM test_table") == 'Key from 4 fd 3\nKey from 3 fd 2\nKey from 0 fd 1\n' + assert node.query("SELECT * FROM test_table") == 'Key from 4 fd 3\nKey from 3 fd 2\nKey from 0 fd 1\n' -# def test_executable_pool_storage(started_cluster): -# skip_test_msan(node) -# node.query("DROP TABLE IF EXISTS test_table") -# node.query("CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_process_pool.sh', 'TabSeparated', (SELECT 1))") -# assert node.query("SELECT * FROM test_table") == 'Key 1\n' -# node.query("DROP TABLE test_table") + node.query("DROP TABLE test_table") -# def test_executable_pool_storage_multiple_pipes(started_cluster): -# skip_test_msan(node) -# node.query("DROP TABLE IF EXISTS test_table") -# node.query("CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_process_pool_multiple_pipes.sh', 'TabSeparated', (SELECT 1), (SELECT 2), (SELECT 3))") -# assert node.query("SELECT * FROM test_table") == 'Key from 4 fd 3\nKey from 3 fd 2\nKey from 0 fd 1\n' -# node.query("DROP TABLE test_table") + node.query(query.format(source='(SELECT id FROM test_data_table), (SELECT 2), (SELECT 3)')) + + assert node.query("SELECT * FROM test_table") == 'Key from 4 fd 3\nKey from 3 fd 2\nKey from 0 fd 0\nKey from 0 fd 1\nKey from 0 fd 2\n' + assert node.query("SELECT * FROM test_table") == 'Key from 4 fd 3\nKey from 3 fd 2\nKey from 0 fd 0\nKey from 0 fd 1\nKey from 0 fd 2\n' + assert node.query("SELECT * FROM test_table") == 'Key from 4 fd 3\nKey from 3 fd 2\nKey from 0 fd 0\nKey from 0 fd 1\nKey from 0 fd 2\n' + + node.query("DROP TABLE test_table") diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes_pool.py b/tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes_pool.py index fc52d67aeba..a3a515899f9 100755 --- a/tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes_pool.py +++ b/tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes_pool.py @@ -7,55 +7,39 @@ if __name__ == '__main__': fd3 = os.fdopen(3) fd4 = os.fdopen(4) - for chunk_header in fd4: - fd4_chunk_length = int(chunk_header) - print(str(fd4_chunk_length), end='\n') + lines = [] + + for chunk_header_fd4 in fd4: + fd4_chunk_length = int(chunk_header_fd4) while fd4_chunk_length != 0: - line = sys.stdin.readline() + line = fd4.readline() fd4_chunk_length -= 1 - print("Key from fd 4 " + line, end='') + lines.append("Key from 4 fd " + line) - sys.stdout.flush() - - for chunk_header in fd3: - fd3_chunk_length = int(chunk_header) - print(str(fd3_chunk_length), end='\n') + for chunk_header_fd3 in fd3: + fd3_chunk_length = int(chunk_header_fd3) while fd3_chunk_length != 0: - line = sys.stdin.readline() + line = fd3.readline() fd3_chunk_length -= 1 - print("Key from fd 3 " + line, end='') - - sys.stdout.flush() + lines.append("Key from 3 fd " + line) for chunk_header in sys.stdin: chunk_length = int(chunk_header) - print(str(chunk_length), end='\n') while chunk_length != 0: line = sys.stdin.readline() chunk_length -= 1 - print("Key " + line, end='') + lines.append("Key from 0 fd " + line) - sys.stdout.flush() + break + break -#!/usr/bin/python3 + print(str(len(lines)), end='\n') -import sys -import os + for line in lines: + print(line, end='') + lines.clear() -if __name__ == '__main__': - fd3 = os.fdopen(3) - fd4 = os.fdopen(4) - - for line in fd4: - print("Key from 4 fd " + line, end='') - - for line in fd3: - print("Key from 3 fd " + line, end='') - - for line in sys.stdin: - print("Key from 0 fd " + line, end='') - - sys.stdout.flush() + sys.stdout.flush() \ No newline at end of file From 41437b72f16d92cd0e8e57bf9dbf2a1b1f8d47d9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 28 Dec 2021 12:16:10 +0300 Subject: [PATCH 0521/1260] Fixed integration tests --- ...table_input_argument_python_dictionary.xml | 8 ++-- .../executable_input_bash_dictionary.xml | 8 ++-- .../executable_input_python_dictionary.xml | 8 ++-- ...ut_send_chunk_header_python_dictionary.xml | 8 ++-- ...able_input_signalled_python_dictionary.xml | 8 ++-- ...xecutable_input_slow_python_dictionary.xml | 8 ++-- ...executable_input_sum_python_dictionary.xml | 8 ++-- ...able_source_argument_python_dictionary.xml | 4 +- .../executable_source_python_dictionary.xml | 4 +- ...table_source_updated_python_dictionary.xml | 4 +- .../user_scripts/{test_input.py => input.py} | 0 .../user_scripts/{test_input.sh => input.sh} | 0 ...st_input_argument.py => input_argument.py} | 0 ..._chunk_header.py => input_chunk_header.py} | 0 ...st_input_implicit.py => input_implicit.py} | 0 ...st_input_implicit.sh => input_implicit.sh} | 0 ...argument.py => input_implicit_argument.py} | 0 ...ader.py => input_implicit_chunk_header.py} | 0 ...gnalled.py => input_implicit_signalled.py} | 0 ...mplicit_slow.py => input_implicit_slow.py} | 0 ..._implicit_sum.py => input_implicit_sum.py} | 0 ..._input_signalled.py => input_signalled.py} | 0 .../{test_input_slow.py => input_slow.py} | 0 .../{test_input_sum.py => input_sum.py} | 0 .../{test_source.py => source.py} | 0 ..._source_argument.py => source_argument.py} | 0 ...test_source_update.py => source_update.py} | 0 .../test_executable_table_function/test.py | 46 +++++++++---------- .../user_scripts/{test_input.py => input.py} | 0 .../user_scripts/{test_input.sh => input.sh} | 0 ...st_input_argument.py => input_argument.py} | 0 ...rgument_pool.py => input_argument_pool.py} | 0 ..._chunk_header.py => input_chunk_header.py} | 0 ...tiple_pipes.py => input_multiple_pipes.py} | 0 ...s_pool.py => input_multiple_pipes_pool.py} | 0 .../{test_input_pool.py => input_pool.py} | 0 ..._input_signalled.py => input_signalled.py} | 0 .../{test_input_slow.py => input_slow.py} | 0 .../{test_input_sum.py => input_sum.py} | 0 ...st_input_sum_pool.py => input_sum_pool.py} | 0 .../{test_no_input.py => no_input.py} | 0 .../{test_no_input.sh => no_input.sh} | 0 .../functions/test_function_config.xml | 28 +++++------ .../user_scripts/{test_input.py => input.py} | 0 .../user_scripts/{test_input.sh => input.sh} | 0 ...st_input_argument.py => input_argument.py} | 0 ..._chunk_header.py => input_chunk_header.py} | 0 ..._input_signalled.py => input_signalled.py} | 0 .../{test_input_slow.py => input_slow.py} | 0 .../{test_input_sum.py => input_sum.py} | 0 50 files changed, 71 insertions(+), 71 deletions(-) rename tests/integration/test_executable_dictionary/user_scripts/{test_input.py => input.py} (100%) rename tests/integration/test_executable_dictionary/user_scripts/{test_input.sh => input.sh} (100%) rename tests/integration/test_executable_dictionary/user_scripts/{test_input_argument.py => input_argument.py} (100%) rename tests/integration/test_executable_dictionary/user_scripts/{test_input_chunk_header.py => input_chunk_header.py} (100%) rename tests/integration/test_executable_dictionary/user_scripts/{test_input_implicit.py => input_implicit.py} (100%) rename tests/integration/test_executable_dictionary/user_scripts/{test_input_implicit.sh => input_implicit.sh} (100%) rename tests/integration/test_executable_dictionary/user_scripts/{test_input_implicit_argument.py => input_implicit_argument.py} (100%) rename tests/integration/test_executable_dictionary/user_scripts/{test_input_implicit_chunk_header.py => input_implicit_chunk_header.py} (100%) rename tests/integration/test_executable_dictionary/user_scripts/{test_input_implicit_signalled.py => input_implicit_signalled.py} (100%) rename tests/integration/test_executable_dictionary/user_scripts/{test_input_implicit_slow.py => input_implicit_slow.py} (100%) rename tests/integration/test_executable_dictionary/user_scripts/{test_input_implicit_sum.py => input_implicit_sum.py} (100%) rename tests/integration/test_executable_dictionary/user_scripts/{test_input_signalled.py => input_signalled.py} (100%) rename tests/integration/test_executable_dictionary/user_scripts/{test_input_slow.py => input_slow.py} (100%) rename tests/integration/test_executable_dictionary/user_scripts/{test_input_sum.py => input_sum.py} (100%) rename tests/integration/test_executable_dictionary/user_scripts/{test_source.py => source.py} (100%) rename tests/integration/test_executable_dictionary/user_scripts/{test_source_argument.py => source_argument.py} (100%) rename tests/integration/test_executable_dictionary/user_scripts/{test_source_update.py => source_update.py} (100%) rename tests/integration/test_executable_table_function/user_scripts/{test_input.py => input.py} (100%) rename tests/integration/test_executable_table_function/user_scripts/{test_input.sh => input.sh} (100%) rename tests/integration/test_executable_table_function/user_scripts/{test_input_argument.py => input_argument.py} (100%) rename tests/integration/test_executable_table_function/user_scripts/{test_input_argument_pool.py => input_argument_pool.py} (100%) rename tests/integration/test_executable_table_function/user_scripts/{test_input_chunk_header.py => input_chunk_header.py} (100%) rename tests/integration/test_executable_table_function/user_scripts/{test_input_multiple_pipes.py => input_multiple_pipes.py} (100%) rename tests/integration/test_executable_table_function/user_scripts/{test_input_multiple_pipes_pool.py => input_multiple_pipes_pool.py} (100%) rename tests/integration/test_executable_table_function/user_scripts/{test_input_pool.py => input_pool.py} (100%) rename tests/integration/test_executable_table_function/user_scripts/{test_input_signalled.py => input_signalled.py} (100%) rename tests/integration/test_executable_table_function/user_scripts/{test_input_slow.py => input_slow.py} (100%) rename tests/integration/test_executable_table_function/user_scripts/{test_input_sum.py => input_sum.py} (100%) rename tests/integration/test_executable_table_function/user_scripts/{test_input_sum_pool.py => input_sum_pool.py} (100%) rename tests/integration/test_executable_table_function/user_scripts/{test_no_input.py => no_input.py} (100%) rename tests/integration/test_executable_table_function/user_scripts/{test_no_input.sh => no_input.sh} (100%) rename tests/integration/test_executable_user_defined_function/user_scripts/{test_input.py => input.py} (100%) rename tests/integration/test_executable_user_defined_function/user_scripts/{test_input.sh => input.sh} (100%) rename tests/integration/test_executable_user_defined_function/user_scripts/{test_input_argument.py => input_argument.py} (100%) rename tests/integration/test_executable_user_defined_function/user_scripts/{test_input_chunk_header.py => input_chunk_header.py} (100%) rename tests/integration/test_executable_user_defined_function/user_scripts/{test_input_signalled.py => input_signalled.py} (100%) rename tests/integration/test_executable_user_defined_function/user_scripts/{test_input_slow.py => input_slow.py} (100%) rename tests/integration/test_executable_user_defined_function/user_scripts/{test_input_sum.py => input_sum.py} (100%) diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_input_argument_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_input_argument_python_dictionary.xml index 424a19b1ba8..ddbb8e95abb 100644 --- a/tests/integration/test_executable_dictionary/dictionaries/executable_input_argument_python_dictionary.xml +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_input_argument_python_dictionary.xml @@ -4,7 +4,7 @@ TabSeparated - test_input_argument.py 1 + input_argument.py 1 1 @@ -28,7 +28,7 @@ TabSeparated - test_input_argument.py 1 + input_argument.py 1 1 @@ -52,7 +52,7 @@ TabSeparated - test_input_implicit_argument.py 1 + input_implicit_argument.py 1 1 1 @@ -77,7 +77,7 @@ TabSeparated - test_input_implicit_argument.py 1 + input_implicit_argument.py 1 1 1 diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_input_bash_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_input_bash_dictionary.xml index 744580600f4..488a12de115 100644 --- a/tests/integration/test_executable_dictionary/dictionaries/executable_input_bash_dictionary.xml +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_input_bash_dictionary.xml @@ -4,7 +4,7 @@ TabSeparated - test_input.sh + input.sh 1 @@ -28,7 +28,7 @@ TabSeparated - test_input.sh + input.sh 1 @@ -52,7 +52,7 @@ TabSeparated - test_input_implicit.sh + input_implicit.sh 1 1 @@ -77,7 +77,7 @@ TabSeparated - test_input_implicit.sh + input_implicit.sh 1 1 diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_input_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_input_python_dictionary.xml index 0ff81818b5d..5b551e51951 100644 --- a/tests/integration/test_executable_dictionary/dictionaries/executable_input_python_dictionary.xml +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_input_python_dictionary.xml @@ -4,7 +4,7 @@ TabSeparated - test_input.py + input.py 1 @@ -28,7 +28,7 @@ TabSeparated - test_input.py + input.py 1 @@ -52,7 +52,7 @@ TabSeparated - test_input_implicit.py + input_implicit.py 1 1 @@ -77,7 +77,7 @@ TabSeparated - test_input_implicit.py + input_implicit.py 1 1 diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_input_send_chunk_header_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_input_send_chunk_header_python_dictionary.xml index 9310c3d2142..816cb0db2c5 100644 --- a/tests/integration/test_executable_dictionary/dictionaries/executable_input_send_chunk_header_python_dictionary.xml +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_input_send_chunk_header_python_dictionary.xml @@ -4,7 +4,7 @@ TabSeparated - test_input_chunk_header.py + input_chunk_header.py 1 1 @@ -29,7 +29,7 @@ TabSeparated - test_input_chunk_header.py + input_chunk_header.py 1 1 @@ -54,7 +54,7 @@ TabSeparated - test_input_implicit_chunk_header.py + input_implicit_chunk_header.py 1 1 1 @@ -80,7 +80,7 @@ TabSeparated - test_input_implicit_chunk_header.py + input_implicit_chunk_header.py 1 1 1 diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_input_signalled_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_input_signalled_python_dictionary.xml index ecc3ef4e3a5..71f8873b20e 100644 --- a/tests/integration/test_executable_dictionary/dictionaries/executable_input_signalled_python_dictionary.xml +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_input_signalled_python_dictionary.xml @@ -4,7 +4,7 @@ TabSeparated - test_input_signalled.py + input_signalled.py 1 1000 @@ -29,7 +29,7 @@ TabSeparated - test_input_signalled.py + input_signalled.py 1 1000 @@ -54,7 +54,7 @@ TabSeparated - test_input_implicit_signalled.py + input_implicit_signalled.py 1 1 1000 @@ -80,7 +80,7 @@ TabSeparated - test_input_implicit_signalled.py + input_implicit_signalled.py 1 1 1000 diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_input_slow_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_input_slow_python_dictionary.xml index dbd596ee80b..dee161a9b78 100644 --- a/tests/integration/test_executable_dictionary/dictionaries/executable_input_slow_python_dictionary.xml +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_input_slow_python_dictionary.xml @@ -4,7 +4,7 @@ TabSeparated - test_input_slow.py + input_slow.py 1 1000 @@ -29,7 +29,7 @@ TabSeparated - test_input_slow.py + input_slow.py 1 1000 @@ -54,7 +54,7 @@ TabSeparated - test_input_implicit_slow.py + input_implicit_slow.py 1 1 1000 @@ -80,7 +80,7 @@ TabSeparated - test_input_implicit_slow.py + input_implicit_slow.py 1 1 1000 diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_input_sum_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_input_sum_python_dictionary.xml index a4504384e7d..3f63e7b8671 100644 --- a/tests/integration/test_executable_dictionary/dictionaries/executable_input_sum_python_dictionary.xml +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_input_sum_python_dictionary.xml @@ -4,7 +4,7 @@ TabSeparated - test_input_sum.py + input_sum.py 1 @@ -35,7 +35,7 @@ TabSeparated - test_input_sum.py + input_sum.py 1 @@ -67,7 +67,7 @@ TabSeparated - test_input_implicit_sum.py + input_implicit_sum.py 1 1 @@ -99,7 +99,7 @@ TabSeparated - test_input_implicit_sum.py + input_implicit_sum.py 1 1 diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_source_argument_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_source_argument_python_dictionary.xml index 5d9582654c0..3173eb5500d 100644 --- a/tests/integration/test_executable_dictionary/dictionaries/executable_source_argument_python_dictionary.xml +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_source_argument_python_dictionary.xml @@ -4,7 +4,7 @@ TabSeparated - test_source_argument.py 1 + source_argument.py 1 1 @@ -29,7 +29,7 @@ TabSeparated - test_source_argument.py 1 + source_argument.py 1 1 diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_source_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_source_python_dictionary.xml index 6d191e882ee..a2036fc67bb 100644 --- a/tests/integration/test_executable_dictionary/dictionaries/executable_source_python_dictionary.xml +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_source_python_dictionary.xml @@ -4,7 +4,7 @@ TabSeparated - test_source.py + source.py 1 @@ -29,7 +29,7 @@ TabSeparated - test_source.py + source.py 1 diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_source_updated_python_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_source_updated_python_dictionary.xml index 0acd2c8d6b3..10d1b1ca0c6 100644 --- a/tests/integration/test_executable_dictionary/dictionaries/executable_source_updated_python_dictionary.xml +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_source_updated_python_dictionary.xml @@ -4,7 +4,7 @@ TabSeparated - test_source_update.py + source_update.py 1 1 @@ -30,7 +30,7 @@ TabSeparated - test_source_update.py + source_update.py 1 1 diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input.py b/tests/integration/test_executable_dictionary/user_scripts/input.py similarity index 100% rename from tests/integration/test_executable_dictionary/user_scripts/test_input.py rename to tests/integration/test_executable_dictionary/user_scripts/input.py diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input.sh b/tests/integration/test_executable_dictionary/user_scripts/input.sh similarity index 100% rename from tests/integration/test_executable_dictionary/user_scripts/test_input.sh rename to tests/integration/test_executable_dictionary/user_scripts/input.sh diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_argument.py b/tests/integration/test_executable_dictionary/user_scripts/input_argument.py similarity index 100% rename from tests/integration/test_executable_dictionary/user_scripts/test_input_argument.py rename to tests/integration/test_executable_dictionary/user_scripts/input_argument.py diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_chunk_header.py b/tests/integration/test_executable_dictionary/user_scripts/input_chunk_header.py similarity index 100% rename from tests/integration/test_executable_dictionary/user_scripts/test_input_chunk_header.py rename to tests/integration/test_executable_dictionary/user_scripts/input_chunk_header.py diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit.py b/tests/integration/test_executable_dictionary/user_scripts/input_implicit.py similarity index 100% rename from tests/integration/test_executable_dictionary/user_scripts/test_input_implicit.py rename to tests/integration/test_executable_dictionary/user_scripts/input_implicit.py diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit.sh b/tests/integration/test_executable_dictionary/user_scripts/input_implicit.sh similarity index 100% rename from tests/integration/test_executable_dictionary/user_scripts/test_input_implicit.sh rename to tests/integration/test_executable_dictionary/user_scripts/input_implicit.sh diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_argument.py b/tests/integration/test_executable_dictionary/user_scripts/input_implicit_argument.py similarity index 100% rename from tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_argument.py rename to tests/integration/test_executable_dictionary/user_scripts/input_implicit_argument.py diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_chunk_header.py b/tests/integration/test_executable_dictionary/user_scripts/input_implicit_chunk_header.py similarity index 100% rename from tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_chunk_header.py rename to tests/integration/test_executable_dictionary/user_scripts/input_implicit_chunk_header.py diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_signalled.py b/tests/integration/test_executable_dictionary/user_scripts/input_implicit_signalled.py similarity index 100% rename from tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_signalled.py rename to tests/integration/test_executable_dictionary/user_scripts/input_implicit_signalled.py diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_slow.py b/tests/integration/test_executable_dictionary/user_scripts/input_implicit_slow.py similarity index 100% rename from tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_slow.py rename to tests/integration/test_executable_dictionary/user_scripts/input_implicit_slow.py diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_sum.py b/tests/integration/test_executable_dictionary/user_scripts/input_implicit_sum.py similarity index 100% rename from tests/integration/test_executable_dictionary/user_scripts/test_input_implicit_sum.py rename to tests/integration/test_executable_dictionary/user_scripts/input_implicit_sum.py diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_signalled.py b/tests/integration/test_executable_dictionary/user_scripts/input_signalled.py similarity index 100% rename from tests/integration/test_executable_dictionary/user_scripts/test_input_signalled.py rename to tests/integration/test_executable_dictionary/user_scripts/input_signalled.py diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_slow.py b/tests/integration/test_executable_dictionary/user_scripts/input_slow.py similarity index 100% rename from tests/integration/test_executable_dictionary/user_scripts/test_input_slow.py rename to tests/integration/test_executable_dictionary/user_scripts/input_slow.py diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_input_sum.py b/tests/integration/test_executable_dictionary/user_scripts/input_sum.py similarity index 100% rename from tests/integration/test_executable_dictionary/user_scripts/test_input_sum.py rename to tests/integration/test_executable_dictionary/user_scripts/input_sum.py diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_source.py b/tests/integration/test_executable_dictionary/user_scripts/source.py similarity index 100% rename from tests/integration/test_executable_dictionary/user_scripts/test_source.py rename to tests/integration/test_executable_dictionary/user_scripts/source.py diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_source_argument.py b/tests/integration/test_executable_dictionary/user_scripts/source_argument.py similarity index 100% rename from tests/integration/test_executable_dictionary/user_scripts/test_source_argument.py rename to tests/integration/test_executable_dictionary/user_scripts/source_argument.py diff --git a/tests/integration/test_executable_dictionary/user_scripts/test_source_update.py b/tests/integration/test_executable_dictionary/user_scripts/source_update.py similarity index 100% rename from tests/integration/test_executable_dictionary/user_scripts/test_source_update.py rename to tests/integration/test_executable_dictionary/user_scripts/source_update.py diff --git a/tests/integration/test_executable_table_function/test.py b/tests/integration/test_executable_table_function/test.py index fa4bba16e8d..6b231228d0c 100644 --- a/tests/integration/test_executable_table_function/test.py +++ b/tests/integration/test_executable_table_function/test.py @@ -39,57 +39,57 @@ def started_cluster(): def test_executable_function_no_input_bash(started_cluster): skip_test_msan(node) - assert node.query("SELECT * FROM executable('test_no_input.sh', 'TabSeparated', 'value String')") == 'Key 0\nKey 1\nKey 2\n' + assert node.query("SELECT * FROM executable('no_input.sh', 'TabSeparated', 'value String')") == 'Key 0\nKey 1\nKey 2\n' def test_executable_function_no_input_python(started_cluster): skip_test_msan(node) - assert node.query("SELECT * FROM executable('test_no_input.py', 'TabSeparated', 'value String')") == 'Key 0\nKey 1\nKey 2\n' + assert node.query("SELECT * FROM executable('no_input.py', 'TabSeparated', 'value String')") == 'Key 0\nKey 1\nKey 2\n' def test_executable_function_input_bash(started_cluster): skip_test_msan(node) - query = "SELECT * FROM executable('test_input.sh', 'TabSeparated', 'value String', {source})" + query = "SELECT * FROM executable('input.sh', 'TabSeparated', 'value String', {source})" assert node.query(query.format(source='(SELECT 1)')) == 'Key 1\n' assert node.query(query.format(source='(SELECT id FROM test_data_table)')) == 'Key 0\nKey 1\nKey 2\n' def test_executable_function_input_python(started_cluster): skip_test_msan(node) - query = "SELECT * FROM executable('test_input.py', 'TabSeparated', 'value String', {source})" + query = "SELECT * FROM executable('input.py', 'TabSeparated', 'value String', {source})" assert node.query(query.format(source='(SELECT 1)')) == 'Key 1\n' assert node.query(query.format(source='(SELECT id FROM test_data_table)')) == 'Key 0\nKey 1\nKey 2\n' def test_executable_function_input_sum_python(started_cluster): skip_test_msan(node) - query = "SELECT * FROM executable('test_input_sum.py', 'TabSeparated', 'value UInt64', {source})" + query = "SELECT * FROM executable('input_sum.py', 'TabSeparated', 'value UInt64', {source})" assert node.query(query.format(source='(SELECT 1, 1)')) == '2\n' assert node.query(query.format(source='(SELECT id, id FROM test_data_table)')) == '0\n2\n4\n' def test_executable_function_input_argument_python(started_cluster): skip_test_msan(node) - query = "SELECT * FROM executable('test_input_argument.py 1', 'TabSeparated', 'value String', {source})" + query = "SELECT * FROM executable('input_argument.py 1', 'TabSeparated', 'value String', {source})" assert node.query(query.format(source='(SELECT 1)')) == 'Key 1 1\n' assert node.query(query.format(source='(SELECT id FROM test_data_table)')) == 'Key 1 0\nKey 1 1\nKey 1 2\n' def test_executable_function_input_signalled_python(started_cluster): skip_test_msan(node) - query = "SELECT * FROM executable('test_input_signalled.py', 'TabSeparated', 'value String', {source})" + query = "SELECT * FROM executable('input_signalled.py', 'TabSeparated', 'value String', {source})" assert node.query(query.format(source='(SELECT 1)')) == '' assert node.query(query.format(source='(SELECT id FROM test_data_table)')) == '' def test_executable_function_input_slow_python(started_cluster): skip_test_msan(node) - query = "SELECT * FROM executable('test_input_slow.py', 'TabSeparated', 'value String', {source})" + query = "SELECT * FROM executable('input_slow.py', 'TabSeparated', 'value String', {source})" assert node.query_and_get_error(query.format(source='(SELECT 1)')) assert node.query_and_get_error(query.format(source='(SELECT id FROM test_data_table)')) def test_executable_function_input_multiple_pipes_python(started_cluster): skip_test_msan(node) - query = "SELECT * FROM executable('test_input_multiple_pipes.py', 'TabSeparated', 'value String', {source})" + query = "SELECT * FROM executable('input_multiple_pipes.py', 'TabSeparated', 'value String', {source})" actual = node.query(query.format(source='(SELECT 1), (SELECT 2), (SELECT 3)')) expected = 'Key from 4 fd 3\nKey from 3 fd 2\nKey from 0 fd 1\n' assert actual == expected @@ -101,21 +101,21 @@ def test_executable_function_input_multiple_pipes_python(started_cluster): def test_executable_storage_no_input_bash(started_cluster): skip_test_msan(node) node.query("DROP TABLE IF EXISTS test_table") - node.query("CREATE TABLE test_table (value String) ENGINE=Executable('test_no_input.sh', 'TabSeparated')") + node.query("CREATE TABLE test_table (value String) ENGINE=Executable('no_input.sh', 'TabSeparated')") assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' node.query("DROP TABLE test_table") def test_executable_storage_no_input_python(started_cluster): skip_test_msan(node) node.query("DROP TABLE IF EXISTS test_table") - node.query("CREATE TABLE test_table (value String) ENGINE=Executable('test_no_input.py', 'TabSeparated')") + node.query("CREATE TABLE test_table (value String) ENGINE=Executable('no_input.py', 'TabSeparated')") assert node.query("SELECT * FROM test_table") == 'Key 0\nKey 1\nKey 2\n' node.query("DROP TABLE test_table") def test_executable_storage_input_bash(started_cluster): skip_test_msan(node) - query = "CREATE TABLE test_table (value String) ENGINE=Executable('test_input.sh', 'TabSeparated', {source})" + query = "CREATE TABLE test_table (value String) ENGINE=Executable('input.sh', 'TabSeparated', {source})" node.query("DROP TABLE IF EXISTS test_table") node.query(query.format(source='(SELECT 1)')) @@ -129,7 +129,7 @@ def test_executable_storage_input_bash(started_cluster): def test_executable_storage_input_python(started_cluster): skip_test_msan(node) - query = "CREATE TABLE test_table (value String) ENGINE=Executable('test_input.py', 'TabSeparated', {source})" + query = "CREATE TABLE test_table (value String) ENGINE=Executable('input.py', 'TabSeparated', {source})" node.query("DROP TABLE IF EXISTS test_table") node.query(query.format(source='(SELECT 1)')) @@ -143,7 +143,7 @@ def test_executable_storage_input_python(started_cluster): def test_executable_storage_input_send_chunk_header_python(started_cluster): skip_test_msan(node) - query = "CREATE TABLE test_table (value String) ENGINE=Executable('test_input_chunk_header.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1" + query = "CREATE TABLE test_table (value String) ENGINE=Executable('input_chunk_header.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1" node.query("DROP TABLE IF EXISTS test_table") node.query(query.format(source='(SELECT 1)')) @@ -157,7 +157,7 @@ def test_executable_storage_input_send_chunk_header_python(started_cluster): def test_executable_storage_input_sum_python(started_cluster): skip_test_msan(node) - query = "CREATE TABLE test_table (value UInt64) ENGINE=Executable('test_input_sum.py', 'TabSeparated', {source})" + query = "CREATE TABLE test_table (value UInt64) ENGINE=Executable('input_sum.py', 'TabSeparated', {source})" node.query("DROP TABLE IF EXISTS test_table") node.query(query.format(source='(SELECT 1, 1)')) @@ -171,7 +171,7 @@ def test_executable_storage_input_sum_python(started_cluster): def test_executable_storage_input_argument_python(started_cluster): skip_test_msan(node) - query = "CREATE TABLE test_table (value String) ENGINE=Executable('test_input_argument.py 1', 'TabSeparated', {source})" + query = "CREATE TABLE test_table (value String) ENGINE=Executable('input_argument.py 1', 'TabSeparated', {source})" node.query("DROP TABLE IF EXISTS test_table") node.query(query.format(source='(SELECT 1)')) @@ -185,7 +185,7 @@ def test_executable_storage_input_argument_python(started_cluster): def test_executable_storage_input_signalled_python(started_cluster): skip_test_msan(node) - query = "CREATE TABLE test_table (value String) ENGINE=Executable('test_input_signalled.py', 'TabSeparated', {source})" + query = "CREATE TABLE test_table (value String) ENGINE=Executable('input_signalled.py', 'TabSeparated', {source})" node.query("DROP TABLE IF EXISTS test_table") node.query(query.format(source='(SELECT 1)')) @@ -199,7 +199,7 @@ def test_executable_storage_input_signalled_python(started_cluster): def test_executable_storage_input_slow_python(started_cluster): skip_test_msan(node) - query = "CREATE TABLE test_table (value String) ENGINE=Executable('test_input_slow.py', 'TabSeparated', {source}) SETTINGS command_read_timeout=2500" + query = "CREATE TABLE test_table (value String) ENGINE=Executable('input_slow.py', 'TabSeparated', {source}) SETTINGS command_read_timeout=2500" node.query("DROP TABLE IF EXISTS test_table") node.query(query.format(source='(SELECT 1)')) @@ -213,7 +213,7 @@ def test_executable_storage_input_slow_python(started_cluster): def test_executable_function_input_multiple_pipes_python(started_cluster): skip_test_msan(node) - query = "CREATE TABLE test_table (value String) ENGINE=Executable('test_input_multiple_pipes.py', 'TabSeparated', {source})" + query = "CREATE TABLE test_table (value String) ENGINE=Executable('input_multiple_pipes.py', 'TabSeparated', {source})" node.query("DROP TABLE IF EXISTS test_table") node.query(query.format(source='(SELECT 1), (SELECT 2), (SELECT 3)')) @@ -227,7 +227,7 @@ def test_executable_function_input_multiple_pipes_python(started_cluster): def test_executable_pool_storage_input_python(started_cluster): skip_test_msan(node) - query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" + query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('input_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" node.query("DROP TABLE IF EXISTS test_table") node.query(query.format(source='(SELECT 1)')) @@ -249,7 +249,7 @@ def test_executable_pool_storage_input_python(started_cluster): def test_executable_pool_storage_input_sum_python(started_cluster): skip_test_msan(node) - query = "CREATE TABLE test_table (value UInt64) ENGINE=ExecutablePool('test_input_sum_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" + query = "CREATE TABLE test_table (value UInt64) ENGINE=ExecutablePool('input_sum_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" node.query("DROP TABLE IF EXISTS test_table") node.query(query.format(source='(SELECT 1, 1)')) @@ -271,7 +271,7 @@ def test_executable_pool_storage_input_sum_python(started_cluster): def test_executable_pool_storage_input_argument_python(started_cluster): skip_test_msan(node) - query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_argument_pool.py 1', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" + query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('input_argument_pool.py 1', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" node.query("DROP TABLE IF EXISTS test_table") node.query(query.format(source='(SELECT 1)')) @@ -293,7 +293,7 @@ def test_executable_pool_storage_input_argument_python(started_cluster): def test_executable_pool_storage_input_python(started_cluster): skip_test_msan(node) - query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('test_input_multiple_pipes_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" + query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('input_multiple_pipes_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" node.query("DROP TABLE IF EXISTS test_table") node.query(query.format(source='(SELECT 1), (SELECT 2), (SELECT 3)')) diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input.py b/tests/integration/test_executable_table_function/user_scripts/input.py similarity index 100% rename from tests/integration/test_executable_table_function/user_scripts/test_input.py rename to tests/integration/test_executable_table_function/user_scripts/input.py diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input.sh b/tests/integration/test_executable_table_function/user_scripts/input.sh similarity index 100% rename from tests/integration/test_executable_table_function/user_scripts/test_input.sh rename to tests/integration/test_executable_table_function/user_scripts/input.sh diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_argument.py b/tests/integration/test_executable_table_function/user_scripts/input_argument.py similarity index 100% rename from tests/integration/test_executable_table_function/user_scripts/test_input_argument.py rename to tests/integration/test_executable_table_function/user_scripts/input_argument.py diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_argument_pool.py b/tests/integration/test_executable_table_function/user_scripts/input_argument_pool.py similarity index 100% rename from tests/integration/test_executable_table_function/user_scripts/test_input_argument_pool.py rename to tests/integration/test_executable_table_function/user_scripts/input_argument_pool.py diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_chunk_header.py b/tests/integration/test_executable_table_function/user_scripts/input_chunk_header.py similarity index 100% rename from tests/integration/test_executable_table_function/user_scripts/test_input_chunk_header.py rename to tests/integration/test_executable_table_function/user_scripts/input_chunk_header.py diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes.py b/tests/integration/test_executable_table_function/user_scripts/input_multiple_pipes.py similarity index 100% rename from tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes.py rename to tests/integration/test_executable_table_function/user_scripts/input_multiple_pipes.py diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes_pool.py b/tests/integration/test_executable_table_function/user_scripts/input_multiple_pipes_pool.py similarity index 100% rename from tests/integration/test_executable_table_function/user_scripts/test_input_multiple_pipes_pool.py rename to tests/integration/test_executable_table_function/user_scripts/input_multiple_pipes_pool.py diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_pool.py b/tests/integration/test_executable_table_function/user_scripts/input_pool.py similarity index 100% rename from tests/integration/test_executable_table_function/user_scripts/test_input_pool.py rename to tests/integration/test_executable_table_function/user_scripts/input_pool.py diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_signalled.py b/tests/integration/test_executable_table_function/user_scripts/input_signalled.py similarity index 100% rename from tests/integration/test_executable_table_function/user_scripts/test_input_signalled.py rename to tests/integration/test_executable_table_function/user_scripts/input_signalled.py diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_slow.py b/tests/integration/test_executable_table_function/user_scripts/input_slow.py similarity index 100% rename from tests/integration/test_executable_table_function/user_scripts/test_input_slow.py rename to tests/integration/test_executable_table_function/user_scripts/input_slow.py diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_sum.py b/tests/integration/test_executable_table_function/user_scripts/input_sum.py similarity index 100% rename from tests/integration/test_executable_table_function/user_scripts/test_input_sum.py rename to tests/integration/test_executable_table_function/user_scripts/input_sum.py diff --git a/tests/integration/test_executable_table_function/user_scripts/test_input_sum_pool.py b/tests/integration/test_executable_table_function/user_scripts/input_sum_pool.py similarity index 100% rename from tests/integration/test_executable_table_function/user_scripts/test_input_sum_pool.py rename to tests/integration/test_executable_table_function/user_scripts/input_sum_pool.py diff --git a/tests/integration/test_executable_table_function/user_scripts/test_no_input.py b/tests/integration/test_executable_table_function/user_scripts/no_input.py similarity index 100% rename from tests/integration/test_executable_table_function/user_scripts/test_no_input.py rename to tests/integration/test_executable_table_function/user_scripts/no_input.py diff --git a/tests/integration/test_executable_table_function/user_scripts/test_no_input.sh b/tests/integration/test_executable_table_function/user_scripts/no_input.sh similarity index 100% rename from tests/integration/test_executable_table_function/user_scripts/test_no_input.sh rename to tests/integration/test_executable_table_function/user_scripts/no_input.sh diff --git a/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml index 5dd51c83a3f..d8f81a588a2 100644 --- a/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml +++ b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml @@ -7,7 +7,7 @@ UInt64 TabSeparated - test_input.sh + input.sh @@ -18,7 +18,7 @@ UInt64 TabSeparated - test_input.sh + input.sh @@ -29,7 +29,7 @@ UInt64 TabSeparated - test_input.py + input.py @@ -40,7 +40,7 @@ UInt64 TabSeparated - test_input.py + input.py @@ -52,7 +52,7 @@ TabSeparated 1 - test_input_chunk_header.py + input_chunk_header.py @@ -64,7 +64,7 @@ TabSeparated 1 - test_input_chunk_header.py + input_chunk_header.py @@ -78,7 +78,7 @@ UInt64 TabSeparated - test_input_sum.py + input_sum.py @@ -92,7 +92,7 @@ UInt64 TabSeparated - test_input_sum.py + input_sum.py @@ -103,7 +103,7 @@ UInt64 TabSeparated - test_input_argument.py 1 + input_argument.py 1 @@ -114,7 +114,7 @@ UInt64 TabSeparated - test_input_argument.py 1 + input_argument.py 1 @@ -125,7 +125,7 @@ UInt64 TabSeparated - test_input_slow.py + input_slow.py 1 1000 @@ -138,7 +138,7 @@ UInt64 TabSeparated - test_input_slow.py + input_slow.py 1 1000 @@ -151,7 +151,7 @@ UInt64 TabSeparated - test_input_signalled.py + input_signalled.py 1 1000 @@ -164,7 +164,7 @@ UInt64 TabSeparated - test_input_signalled.py + input_signalled.py 1 1000 diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/test_input.py b/tests/integration/test_executable_user_defined_function/user_scripts/input.py similarity index 100% rename from tests/integration/test_executable_user_defined_function/user_scripts/test_input.py rename to tests/integration/test_executable_user_defined_function/user_scripts/input.py diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/test_input.sh b/tests/integration/test_executable_user_defined_function/user_scripts/input.sh similarity index 100% rename from tests/integration/test_executable_user_defined_function/user_scripts/test_input.sh rename to tests/integration/test_executable_user_defined_function/user_scripts/input.sh diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/test_input_argument.py b/tests/integration/test_executable_user_defined_function/user_scripts/input_argument.py similarity index 100% rename from tests/integration/test_executable_user_defined_function/user_scripts/test_input_argument.py rename to tests/integration/test_executable_user_defined_function/user_scripts/input_argument.py diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/test_input_chunk_header.py b/tests/integration/test_executable_user_defined_function/user_scripts/input_chunk_header.py similarity index 100% rename from tests/integration/test_executable_user_defined_function/user_scripts/test_input_chunk_header.py rename to tests/integration/test_executable_user_defined_function/user_scripts/input_chunk_header.py diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/test_input_signalled.py b/tests/integration/test_executable_user_defined_function/user_scripts/input_signalled.py similarity index 100% rename from tests/integration/test_executable_user_defined_function/user_scripts/test_input_signalled.py rename to tests/integration/test_executable_user_defined_function/user_scripts/input_signalled.py diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/test_input_slow.py b/tests/integration/test_executable_user_defined_function/user_scripts/input_slow.py similarity index 100% rename from tests/integration/test_executable_user_defined_function/user_scripts/test_input_slow.py rename to tests/integration/test_executable_user_defined_function/user_scripts/input_slow.py diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/test_input_sum.py b/tests/integration/test_executable_user_defined_function/user_scripts/input_sum.py similarity index 100% rename from tests/integration/test_executable_user_defined_function/user_scripts/test_input_sum.py rename to tests/integration/test_executable_user_defined_function/user_scripts/input_sum.py From d1db3c9f42535e104ce561a0a5f4ff6e6c86c33e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 28 Dec 2021 12:43:30 +0300 Subject: [PATCH 0522/1260] Updated ShellCommandSource --- src/Common/ErrorCodes.cpp | 1 + .../ExecutableDictionarySource.cpp | 30 ++------ src/Dictionaries/ExecutableDictionarySource.h | 4 +- .../ExecutablePoolDictionarySource.cpp | 6 +- .../ExecutablePoolDictionarySource.h | 4 +- ...alUserDefinedExecutableFunctionsLoader.cpp | 6 +- .../UserDefinedExecutableFunction.cpp | 2 +- .../UserDefinedExecutableFunction.h | 6 +- src/Processors/Sources/ShellCommandSource.cpp | 8 +-- src/Processors/Sources/ShellCommandSource.h | 4 +- src/Storages/StorageExecutable.cpp | 72 ++++++++++--------- src/Storages/StorageExecutable.h | 4 +- 12 files changed, 67 insertions(+), 80 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 70d85433513..f2e5e018e1b 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -603,6 +603,7 @@ M(632, UNEXPECTED_DATA_AFTER_PARSED_VALUE) \ M(633, QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW) \ M(634, MONGODB_ERROR) \ + M(635, CANNOT_POLL) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 99d94b38d43..7a3550e7284 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -66,7 +66,7 @@ ExecutableDictionarySource::ExecutableDictionarySource( const DictionaryStructure & dict_struct_, const Configuration & configuration_, Block & sample_block_, - std::shared_ptr coordinator_, + std::shared_ptr coordinator_, ContextPtr context_) : log(&Poco::Logger::get("ExecutableDictionarySource")) , dict_struct(dict_struct_) @@ -166,29 +166,9 @@ Pipe ExecutableDictionarySource::loadKeys(const Columns & key_columns, const std Pipe ExecutableDictionarySource::getStreamForBlock(const Block & block) { - String command = configuration.command; const auto & coordinator_configuration = coordinator->getConfiguration(); - - if (coordinator_configuration.execute_direct) - { - auto global_context = context->getGlobalContext(); - auto user_scripts_path = global_context->getUserScriptsPath(); - auto script_path = user_scripts_path + '/' + command; - - if (!fileOrSymlinkPathStartsWith(script_path, user_scripts_path)) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Executable file {} must be inside user scripts folder {}", - command, - user_scripts_path); - - if (!std::filesystem::exists(std::filesystem::path(script_path))) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Executable file {} does not exist inside user scripts folder {}", - command, - user_scripts_path); - - command = std::move(script_path); - } + String command = configuration.command; + updateCommandIfNeeded(command, coordinator_configuration.execute_direct, context); auto source = std::make_shared(block); auto shell_input_pipe = Pipe(std::move(source)); @@ -273,7 +253,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) .implicit_key = config.getBool(settings_config_prefix + ".implicit_key", false), }; - ShellCommandCoordinator::Configuration shell_command_coordinator_configration + ShellCommandSourceCoordinator::Configuration shell_command_coordinator_configration { .format = config.getString(settings_config_prefix + ".format"), .command_termination_timeout_seconds = config.getUInt64(settings_config_prefix + ".command_termination_timeout", 10), @@ -284,7 +264,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) .execute_direct = config.getBool(settings_config_prefix + ".execute_direct", false) }; - std::shared_ptr coordinator = std::make_shared(shell_command_coordinator_configration); + auto coordinator = std::make_shared(shell_command_coordinator_configration); return std::make_unique(dict_struct, configuration, sample_block, std::move(coordinator), context); }; diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index 627915b8a46..6c5d2de3714 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -33,7 +33,7 @@ public: const DictionaryStructure & dict_struct_, const Configuration & configuration_, Block & sample_block_, - std::shared_ptr coordinator_, + std::shared_ptr coordinator_, ContextPtr context_); ExecutableDictionarySource(const ExecutableDictionarySource & other); @@ -69,7 +69,7 @@ private: const DictionaryStructure dict_struct; const Configuration configuration; Block sample_block; - std::shared_ptr coordinator; + std::shared_ptr coordinator; ContextPtr context; }; diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 17f5759b61c..bce56987899 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -35,7 +35,7 @@ ExecutablePoolDictionarySource::ExecutablePoolDictionarySource( const DictionaryStructure & dict_struct_, const Configuration & configuration_, Block & sample_block_, - std::shared_ptr coordinator_, + std::shared_ptr coordinator_, ContextPtr context_) : dict_struct(dict_struct_) , configuration(configuration_) @@ -220,7 +220,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) .implicit_key = config.getBool(settings_config_prefix + ".implicit_key", false), }; - ShellCommandCoordinator::Configuration shell_command_coordinator_configration + ShellCommandSourceCoordinator::Configuration shell_command_coordinator_configration { .format = config.getString(settings_config_prefix + ".format"), .command_termination_timeout_seconds = config.getUInt64(settings_config_prefix + ".command_termination_timeout", 10), @@ -233,7 +233,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) .execute_direct = execute_direct }; - std::shared_ptr coordinator = std::make_shared(shell_command_coordinator_configration); + auto coordinator = std::make_shared(shell_command_coordinator_configration); return std::make_unique(dict_struct, configuration, sample_block, std::move(coordinator), context); }; diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.h b/src/Dictionaries/ExecutablePoolDictionarySource.h index 35f0f04bd48..b9b3b8efb1b 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.h +++ b/src/Dictionaries/ExecutablePoolDictionarySource.h @@ -36,7 +36,7 @@ public: const DictionaryStructure & dict_struct_, const Configuration & configuration_, Block & sample_block_, - std::shared_ptr coordinator_, + std::shared_ptr coordinator_, ContextPtr context_); ExecutablePoolDictionarySource(const ExecutablePoolDictionarySource & other); @@ -71,7 +71,7 @@ private: const Configuration configuration; Block sample_block; - std::shared_ptr coordinator; + std::shared_ptr coordinator; ContextPtr context; Poco::Logger * log; }; diff --git a/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp b/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp index 6738b26c924..b266746642f 100644 --- a/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp +++ b/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp @@ -57,7 +57,7 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create String type = config.getString(key_in_config + ".type"); - bool is_executable_pool; + bool is_executable_pool = false; if (type == "executable") is_executable_pool = false; @@ -130,7 +130,7 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create .result_type = std::move(result_type), //-V1030 }; - ShellCommandCoordinator::Configuration shell_command_coordinator_configration + ShellCommandSourceCoordinator::Configuration shell_command_coordinator_configration { .format = std::move(format), //-V1030 .command_termination_timeout_seconds = command_termination_timeout_seconds, @@ -143,7 +143,7 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create .execute_direct = execute_direct }; - std::shared_ptr coordinator = std::make_shared(shell_command_coordinator_configration); + auto coordinator = std::make_shared(shell_command_coordinator_configration); return std::make_shared(function_configuration, std::move(coordinator), lifetime); } diff --git a/src/Interpreters/UserDefinedExecutableFunction.cpp b/src/Interpreters/UserDefinedExecutableFunction.cpp index b8a84db64c8..e5a852b0e75 100644 --- a/src/Interpreters/UserDefinedExecutableFunction.cpp +++ b/src/Interpreters/UserDefinedExecutableFunction.cpp @@ -13,7 +13,7 @@ namespace DB UserDefinedExecutableFunction::UserDefinedExecutableFunction( const UserDefinedExecutableFunctionConfiguration & configuration_, - std::shared_ptr coordinator_, + std::shared_ptr coordinator_, const ExternalLoadableLifetime & lifetime_) : configuration(configuration_) , coordinator(std::move(coordinator_)) diff --git a/src/Interpreters/UserDefinedExecutableFunction.h b/src/Interpreters/UserDefinedExecutableFunction.h index f02a07b44a1..a4fad8ceb7b 100644 --- a/src/Interpreters/UserDefinedExecutableFunction.h +++ b/src/Interpreters/UserDefinedExecutableFunction.h @@ -25,7 +25,7 @@ public: UserDefinedExecutableFunction( const UserDefinedExecutableFunctionConfiguration & configuration_, - std::shared_ptr coordinator_, + std::shared_ptr coordinator_, const ExternalLoadableLifetime & lifetime_); const ExternalLoadableLifetime & getLifetime() const override @@ -58,7 +58,7 @@ public: return configuration; } - std::shared_ptr getCoordinator() const + std::shared_ptr getCoordinator() const { return coordinator; } @@ -75,7 +75,7 @@ public: private: UserDefinedExecutableFunctionConfiguration configuration; - std::shared_ptr coordinator; + std::shared_ptr coordinator; ExternalLoadableLifetime lifetime; }; diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 0b3298ce8b5..f407664df86 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -23,7 +23,7 @@ namespace ErrorCodes extern const int TIMEOUT_EXCEEDED; extern const int CANNOT_FCNTL; extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR; - extern const int CANNOT_SELECT; + extern const int CANNOT_POLL; extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR; } @@ -91,7 +91,7 @@ static bool pollFd(int fd, size_t timeout_milliseconds, int events) } else { - throwFromErrno("Cannot select", ErrorCodes::CANNOT_SELECT); + throwFromErrno("Cannot poll", ErrorCodes::CANNOT_POLL); } } else @@ -451,14 +451,14 @@ namespace } -ShellCommandCoordinator::ShellCommandCoordinator(const Configuration & configuration_) +ShellCommandSourceCoordinator::ShellCommandSourceCoordinator(const Configuration & configuration_) : configuration(configuration_) { if (configuration.is_executable_pool) process_pool = std::make_shared(configuration.pool_size ? configuration.pool_size : std::numeric_limits::max()); } -Pipe ShellCommandCoordinator::createPipe( +Pipe ShellCommandSourceCoordinator::createPipe( const std::string & command, const std::vector & arguments, std::vector && input_pipes, diff --git a/src/Processors/Sources/ShellCommandSource.h b/src/Processors/Sources/ShellCommandSource.h index 3203a7a4dbb..649c713afcb 100644 --- a/src/Processors/Sources/ShellCommandSource.h +++ b/src/Processors/Sources/ShellCommandSource.h @@ -36,7 +36,7 @@ struct ShellCommandSourceConfiguration size_t max_block_size = DEFAULT_BLOCK_SIZE; }; -class ShellCommandCoordinator +class ShellCommandSourceCoordinator { public: @@ -72,7 +72,7 @@ public: }; - explicit ShellCommandCoordinator(const Configuration & configuration_); + explicit ShellCommandSourceCoordinator(const Configuration & configuration_); const Configuration & getConfiguration() const { diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index d1e7f1a17e1..21143438725 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -34,6 +34,40 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +namespace +{ + void transformToSingleBlockSources(Pipes & inputs) + { + size_t inputs_size = inputs.size(); + for (size_t i = 0; i < inputs_size; ++i) + { + auto && input = inputs[i]; + QueryPipeline input_pipeline(std::move(input)); + PullingPipelineExecutor input_pipeline_executor(input_pipeline); + + auto header = input_pipeline_executor.getHeader(); + auto result_block = header.cloneEmpty(); + + size_t result_block_columns = result_block.columns(); + + Block result; + while (input_pipeline_executor.pull(result)) + { + for (size_t result_block_index = 0; result_block_index < result_block_columns; ++result_block_index) + { + auto & block_column = result.safeGetByPosition(result_block_index); + auto & result_block_column = result_block.safeGetByPosition(result_block_index); + + result_block_column.column->assumeMutable()->insertRangeFrom(*block_column.column, 0, block_column.column->size()); + } + } + + auto source = std::make_shared(std::move(result_block)); + inputs[i] = Pipe(std::move(source)); + } + } +} + StorageExecutable::StorageExecutable( const StorageID & table_id_, const String & format, @@ -51,7 +85,7 @@ StorageExecutable::StorageExecutable( storage_metadata.setConstraints(constraints); setInMemoryMetadata(storage_metadata); - ShellCommandCoordinator::Configuration configuration + ShellCommandSourceCoordinator::Configuration configuration { .format = format, .command_termination_timeout_seconds = settings.command_termination_timeout, @@ -66,7 +100,7 @@ StorageExecutable::StorageExecutable( .execute_direct = true }; - coordinator = std::make_unique(std::move(configuration)); + coordinator = std::make_unique(std::move(configuration)); } Pipe StorageExecutable::read( @@ -104,44 +138,16 @@ Pipe StorageExecutable::read( inputs.emplace_back(QueryPipelineBuilder::getPipe(interpreter.buildQueryPipeline())); } + /// For executable pool we read data from input streams and convert it to single blocks streams. if (settings.is_executable_pool) - { - /// For executable pool we read data from input streams and convert it to single blocks streams. - size_t inputs_size = inputs.size(); - for (size_t i = 0; i < inputs_size; ++i) - { - auto && input = inputs[i]; - QueryPipeline input_pipeline(std::move(input)); - PullingPipelineExecutor input_pipeline_executor(input_pipeline); - - auto header = input_pipeline_executor.getHeader(); - auto result_block = header.cloneEmpty(); - - size_t result_block_columns = result_block.columns(); - - Block result; - while (input_pipeline_executor.pull(result)) - { - for (size_t result_block_index = 0; result_block_index < result_block_columns; ++result_block_index) - { - auto & block_column = result.safeGetByPosition(result_block_index); - auto & result_block_column = result_block.safeGetByPosition(result_block_index); - - result_block_column.column->assumeMutable()->insertRangeFrom(*block_column.column, 0, block_column.column->size()); - } - } - - auto source = std::make_shared(std::move(result_block)); - inputs[i] = Pipe(std::move(source)); - } - } + transformToSingleBlockSources(inputs); auto sample_block = metadata_snapshot->getSampleBlock(); ShellCommandSourceConfiguration configuration; configuration.max_block_size = max_block_size; - if (coordinator->getConfiguration().is_executable_pool) + if (settings.is_executable_pool) { configuration.read_fixed_number_of_rows = true; configuration.read_number_of_rows_from_process_output = true; diff --git a/src/Storages/StorageExecutable.h b/src/Storages/StorageExecutable.h index a12732281d2..b6248abae97 100644 --- a/src/Storages/StorageExecutable.h +++ b/src/Storages/StorageExecutable.h @@ -23,7 +23,7 @@ public: String getName() const override { - if (coordinator->getConfiguration().is_executable_pool) + if (settings.is_executable_pool) return "ExecutablePool"; else return "Executable"; @@ -52,7 +52,7 @@ private: ExecutableSettings settings; std::vector input_queries; Poco::Logger * log; - std::unique_ptr coordinator; + std::unique_ptr coordinator; }; } From b5698ea13498f82e5ccca03103cb888d91748498 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 28 Dec 2021 13:10:58 +0300 Subject: [PATCH 0523/1260] Updated test_executable_table_function tests --- src/Processors/Sources/ShellCommandSource.cpp | 36 +++++----- .../test_executable_table_function/test.py | 70 ++++++++++++++++++- .../user_scripts/input_count_pool.py | 15 ++++ .../user_scripts/input_signalled.py | 1 - .../user_scripts/input_signalled_pool.py | 19 +++++ .../user_scripts/input_slow_pool.py | 18 +++++ 6 files changed, 139 insertions(+), 20 deletions(-) create mode 100755 tests/integration/test_executable_table_function/user_scripts/input_count_pool.py create mode 100755 tests/integration/test_executable_table_function/user_scripts/input_signalled_pool.py create mode 100755 tests/integration/test_executable_table_function/user_scripts/input_slow_pool.py diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index f407664df86..dc272ace01e 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -336,27 +336,27 @@ namespace { rethrowExceptionDuringSendDataIfNeeded(); - if (configuration.read_fixed_number_of_rows) - { - if (!executor && configuration.read_number_of_rows_from_process_output) - { - readText(configuration.number_of_rows_to_read, timeout_command_out); - char dummy; - readChar(dummy, timeout_command_out); - - size_t max_block_size = configuration.number_of_rows_to_read; - pipeline = QueryPipeline(Pipe(context->getInputFormat(format, timeout_command_out, sample_block, max_block_size))); - executor = std::make_unique(pipeline); - } - - if (current_read_rows >= configuration.number_of_rows_to_read) - return {}; - } - Chunk chunk; try { + if (configuration.read_fixed_number_of_rows) + { + if (!executor && configuration.read_number_of_rows_from_process_output) + { + readText(configuration.number_of_rows_to_read, timeout_command_out); + char dummy; + readChar(dummy, timeout_command_out); + + size_t max_block_size = configuration.number_of_rows_to_read; + pipeline = QueryPipeline(Pipe(context->getInputFormat(format, timeout_command_out, sample_block, max_block_size))); + executor = std::make_unique(pipeline); + } + + if (current_read_rows >= configuration.number_of_rows_to_read) + return {}; + } + if (!executor->pull(chunk)) return {}; @@ -396,7 +396,7 @@ namespace std::lock_guard lock(send_data_lock); if (exception_during_send_data) { - command = nullptr; + command_is_invalid = true; std::rethrow_exception(exception_during_send_data); } } diff --git a/tests/integration/test_executable_table_function/test.py b/tests/integration/test_executable_table_function/test.py index 6b231228d0c..7820396d20f 100644 --- a/tests/integration/test_executable_table_function/test.py +++ b/tests/integration/test_executable_table_function/test.py @@ -290,7 +290,53 @@ def test_executable_pool_storage_input_argument_python(started_cluster): node.query("DROP TABLE test_table") -def test_executable_pool_storage_input_python(started_cluster): +def test_executable_pool_storage_input_signalled_python(started_cluster): + skip_test_msan(node) + + query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('input_signalled_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" + + node.query("DROP TABLE IF EXISTS test_table") + node.query(query.format(source='(SELECT 1)')) + + assert node.query_and_get_error("SELECT * FROM test_table") + assert node.query_and_get_error("SELECT * FROM test_table") + assert node.query_and_get_error("SELECT * FROM test_table") + + node.query("DROP TABLE test_table") + + node.query(query.format(source='(SELECT id FROM test_data_table)')) + + assert node.query_and_get_error("SELECT * FROM test_table") + assert node.query_and_get_error("SELECT * FROM test_table") + assert node.query_and_get_error("SELECT * FROM test_table") + + node.query("DROP TABLE test_table") + +def test_executable_pool_storage_input_slow_python(started_cluster): + skip_test_msan(node) + + query = """CREATE TABLE test_table (value String) + ENGINE=ExecutablePool('input_slow_pool.py', 'TabSeparated', {source}) + SETTINGS send_chunk_header=1, pool_size=1, command_read_timeout=2500""" + + node.query("DROP TABLE IF EXISTS test_table") + node.query(query.format(source='(SELECT 1)')) + + assert node.query_and_get_error("SELECT * FROM test_table") + assert node.query_and_get_error("SELECT * FROM test_table") + assert node.query_and_get_error("SELECT * FROM test_table") + + node.query("DROP TABLE test_table") + + node.query(query.format(source='(SELECT id FROM test_data_table)')) + + assert node.query_and_get_error("SELECT * FROM test_table") + assert node.query_and_get_error("SELECT * FROM test_table") + assert node.query_and_get_error("SELECT * FROM test_table") + + node.query("DROP TABLE test_table") + +def test_executable_pool_storage_input_multiple_pipes_python(started_cluster): skip_test_msan(node) query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('input_multiple_pipes_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" @@ -311,3 +357,25 @@ def test_executable_pool_storage_input_python(started_cluster): assert node.query("SELECT * FROM test_table") == 'Key from 4 fd 3\nKey from 3 fd 2\nKey from 0 fd 0\nKey from 0 fd 1\nKey from 0 fd 2\n' node.query("DROP TABLE test_table") + +def test_executable_pool_storage_input_count_python(started_cluster): + skip_test_msan(node) + + query = "CREATE TABLE test_table (value String) ENGINE=ExecutablePool('input_count_pool.py', 'TabSeparated', {source}) SETTINGS send_chunk_header=1, pool_size=1" + + node.query("DROP TABLE IF EXISTS test_table") + node.query(query.format(source='(SELECT 1)')) + + assert node.query("SELECT * FROM test_table") == '1\n' + assert node.query("SELECT * FROM test_table") == '1\n' + assert node.query("SELECT * FROM test_table") == '1\n' + + node.query("DROP TABLE test_table") + + node.query(query.format(source='(SELECT number FROM system.numbers LIMIT 250000)')) + + assert node.query("SELECT * FROM test_table") == '250000\n' + assert node.query("SELECT * FROM test_table") == '250000\n' + assert node.query("SELECT * FROM test_table") == '250000\n' + + node.query("DROP TABLE test_table") diff --git a/tests/integration/test_executable_table_function/user_scripts/input_count_pool.py b/tests/integration/test_executable_table_function/user_scripts/input_count_pool.py new file mode 100755 index 00000000000..8b744168a82 --- /dev/null +++ b/tests/integration/test_executable_table_function/user_scripts/input_count_pool.py @@ -0,0 +1,15 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + for chunk_header in sys.stdin: + chunk_length = int(chunk_header) + print(1, end='\n') + print(str(chunk_length), end='\n') + + while chunk_length != 0: + line = sys.stdin.readline() + chunk_length -= 1 + + sys.stdout.flush() diff --git a/tests/integration/test_executable_table_function/user_scripts/input_signalled.py b/tests/integration/test_executable_table_function/user_scripts/input_signalled.py index 27c8bc4840e..93ce20fa8e7 100755 --- a/tests/integration/test_executable_table_function/user_scripts/input_signalled.py +++ b/tests/integration/test_executable_table_function/user_scripts/input_signalled.py @@ -3,7 +3,6 @@ import sys import os import signal -import time if __name__ == '__main__': for line in sys.stdin: diff --git a/tests/integration/test_executable_table_function/user_scripts/input_signalled_pool.py b/tests/integration/test_executable_table_function/user_scripts/input_signalled_pool.py new file mode 100755 index 00000000000..1ea0eddbd8d --- /dev/null +++ b/tests/integration/test_executable_table_function/user_scripts/input_signalled_pool.py @@ -0,0 +1,19 @@ +#!/usr/bin/python3 + +import sys +import os +import signal + +if __name__ == '__main__': + for chunk_header in sys.stdin: + os.signal(os.getpid(), signal.SIGTERM) + + chunk_length = int(chunk_header) + print(str(chunk_length), end='\n') + + while chunk_length != 0: + line = sys.stdin.readline() + chunk_length -= 1 + print("Key " + line, end='') + + sys.stdout.flush() diff --git a/tests/integration/test_executable_table_function/user_scripts/input_slow_pool.py b/tests/integration/test_executable_table_function/user_scripts/input_slow_pool.py new file mode 100755 index 00000000000..c8df7e18c4c --- /dev/null +++ b/tests/integration/test_executable_table_function/user_scripts/input_slow_pool.py @@ -0,0 +1,18 @@ +#!/usr/bin/python3 + +import sys +import time + +if __name__ == '__main__': + for chunk_header in sys.stdin: + time.sleep(25) + + chunk_length = int(chunk_header) + print(str(chunk_length), end='\n') + + while chunk_length != 0: + line = sys.stdin.readline() + chunk_length -= 1 + print("Key " + line, end='') + + sys.stdout.flush() From 49d461989015974446e3984816a14cab1570a87d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 28 Dec 2021 13:20:50 +0300 Subject: [PATCH 0524/1260] Updated docs --- .../external-dicts-dict-sources.md | 13 +++++++++++-- docs/en/sql-reference/functions/index.md | 10 ++++++---- src/Dictionaries/ExecutablePoolDictionarySource.cpp | 2 +- 3 files changed, 18 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index 095f059513c..e606a19af6f 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -122,7 +122,12 @@ Setting fields: - `command` — The absolute path to the executable file, or the file name (if the program directory is written to `PATH`). - `format` — The file format. All the formats described in [Formats](../../../interfaces/formats.md#formats) are supported. +- `command_termination_timeout` — executable script should contain main read-write loop. After dictionary is destroyed, pipe is closed, and executable file will have `command_termination_timeout` seconds to shutdown, before ClickHouse will send SIGTERM signal to child process. Specified in seconds. Default value is 10. Optional parameter. +- `command_read_timeout` - timeout for reading data from command stdout in milliseconds. Default value 10000. Optional parameter. +- `command_read_timeout` - timeout for writing data to command stdin in milliseconds. Default value 10000. Optional parameter. - `implicit_key` — The executable source file can return only values, and the correspondence to the requested keys is determined implicitly — by the order of rows in the result. Default value is false. +- `execute_direct` - Executable source file will be searched inside `user_scripts` folder and executed directly. Arguments are passed using spaces. Example: `test_script arg_1 arg_2`. Default value is false. Optional parameter. +- `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`. That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled, otherwise, the DB user would be able to execute arbitrary binary on ClickHouse node. @@ -150,10 +155,14 @@ Setting fields: - `command` — The absolute path to the executable file, or the file name (if the program directory is written to `PATH`). - `format` — The file format. All the formats described in “[Formats](../../../interfaces/formats.md#formats)” are supported. -- `pool_size` — Size of pool. If 0 is specified as `pool_size` then there is no pool size restrictions. -- `command_termination_timeout` — Executable pool script should contain main read-write loop. After dictionary is destroyed, pipe is closed, and executable file will have `command_termination_timeout` seconds to shutdown, before ClickHouse will send SIGTERM signal to child process. Specified in seconds. Default value is 10. Optional parameter. +- `pool_size` — Size of pool. If 0 is specified as `pool_size` then there is no pool size restrictions. Default value is `16`. +- `command_termination_timeout` — executable script should contain main read-write loop. After dictionary is destroyed, pipe is closed, and executable file will have `command_termination_timeout` seconds to shutdown, before ClickHouse will send SIGTERM signal to child process. Specified in seconds. Default value is 10. Optional parameter. - `max_command_execution_time` — Maximum executable script command execution time for processing block of data. Specified in seconds. Default value is 10. Optional parameter. +- `command_read_timeout` - timeout for reading data from command stdout in milliseconds. Default value 10000. Optional parameter. +- `command_read_timeout` - timeout for writing data to command stdin in milliseconds. Default value 10000. Optional parameter. - `implicit_key` — The executable source file can return only values, and the correspondence to the requested keys is determined implicitly — by the order of rows in the result. Default value is false. Optional parameter. +- `execute_direct` - Executable source file will be searched inside `user_scripts` folder and executed directly. Additional arguments can be specified. Example: `test_script arg_1 arg_2`. Default value is false. Optional parameter. +- `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`. That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled, otherwise, the DB user would be able to execute arbitrary binary on ClickHouse node. diff --git a/docs/en/sql-reference/functions/index.md b/docs/en/sql-reference/functions/index.md index e86e6b37998..037078ba223 100644 --- a/docs/en/sql-reference/functions/index.md +++ b/docs/en/sql-reference/functions/index.md @@ -73,16 +73,19 @@ User defined function configurations are searched relative to the path specified A function configuration contains the following settings: - `name` - a function name. -- `command` - a command or a script to execute. +- `command` - script name to execute or command if `execute_direct` is false. - `argument` - argument description with the `type` of an argument. Each argument is described in a separate setting. - `format` - a [format](../../interfaces/formats.md) in which arguments are passed to the command. - `return_type` - the type of a returned value. - `type` - an executable type. If `type` is set to `executable` then single command is started. If it is set to `executable_pool` then a pool of commands is created. - `max_command_execution_time` - maximum execution time in seconds for processing block of data. This setting is valid for `executable_pool` commands only. Optional. Default value is `10`. -- `command_termination_timeout` - time in seconds during which a command should finish after its pipe is closed. After that time `SIGTERM` is sent to the process executing the command. This setting is valid for `executable_pool` commands only. Optional. Default value is `10`. +- `command_termination_timeout` - time in seconds during which a command should finish after its pipe is closed. After that time `SIGTERM` is sent to the process executing the command. Optional. Default value is `10`. +- `command_read_timeout` - timeout for reading data from command stdout in milliseconds. Default value 10000. Optional parameter. +- `command_read_timeout` - timeout for writing data to command stdin in milliseconds. Default value 10000. Optional parameter. - `pool_size` - the size of a command pool. Optional. Default value is `16`. -- `lifetime` - the reload interval of a function in seconds. If it is set to `0` then the function is not reloaded. - `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`. +- `execute_direct` - Executable source file will be searched inside `user_scripts` folder and executed directly. Additional arguments can be specified. Example: `test_script arg_1 arg_2`. Default value is true. Optional parameter. +- `lifetime` - the reload interval of a function in seconds. If it is set to `0` then the function is not reloaded. Default value is `0`. Optional parameter. The command must read arguments from `STDIN` and must output the result to `STDOUT`. The command must process arguments iteratively. That is after processing a chunk of arguments it must wait for the next chunk. @@ -102,7 +105,6 @@ Creating `test_function` using XML configuration: TabSeparated cd /; clickhouse-local --input-format TabSeparated --output-format TabSeparated --structure 'x UInt64, y UInt64' --query "SELECT x + y FROM table" - 0
``` diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index bce56987899..48ddeed7fa6 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -226,7 +226,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) .command_termination_timeout_seconds = config.getUInt64(settings_config_prefix + ".command_termination_timeout", 10), .command_read_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_read_timeout", 10000), .command_write_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_write_timeout", 10000), - .pool_size = config.getUInt64(settings_config_prefix + ".size", 16), + .pool_size = config.getUInt64(settings_config_prefix + ".pool_size", 16), .max_command_execution_time_seconds = max_command_execution_time, .is_executable_pool = true, .send_chunk_header = config.getBool(settings_config_prefix + ".send_chunk_header", false), From 12763976b11ef75af8a5a85843585eae1eb92582 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 28 Dec 2021 16:35:46 -0400 Subject: [PATCH 0525/1260] Update Settings.h obsolete max_alter_threads --- src/Core/Settings.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 70fb5604997..952009047d4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -46,7 +46,6 @@ class IColumn; M(UInt64, max_insert_threads, 0, "The maximum number of threads to execute the INSERT SELECT query. Values 0 or 1 means that INSERT SELECT is not run in parallel. Higher values will lead to higher memory usage. Parallel INSERT SELECT has effect only if the SELECT part is run on parallel, see 'max_threads' setting.", 0) \ M(UInt64, max_final_threads, 16, "The maximum number of threads to read from table with FINAL.", 0) \ M(MaxThreads, max_threads, 0, "The maximum number of threads to execute the request. By default, it is determined automatically.", 0) \ - M(MaxThreads, max_alter_threads, 0, "The maximum number of threads to execute the ALTER requests. By default, it is determined automatically.", 0) \ M(UInt64, max_read_buffer_size, DBMS_DEFAULT_BUFFER_SIZE, "The maximum size of the buffer to read from the filesystem.", 0) \ M(UInt64, max_distributed_connections, 1024, "The maximum number of connections for distributed processing of one query (should be greater than max_threads).", 0) \ M(UInt64, max_query_size, DBMS_DEFAULT_MAX_QUERY_SIZE, "Which part of the query can be read into RAM for parsing (the remaining data for INSERT, if any, is read later)", 0) \ @@ -572,7 +571,7 @@ class IColumn; MAKE_OBSOLETE(M, UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60) \ MAKE_OBSOLETE(M, UInt64, merge_tree_clear_old_parts_interval_seconds, 1) \ MAKE_OBSOLETE(M, UInt64, partial_merge_join_optimizations, 0) \ - + MAKE_OBSOLETE(M, MaxThreads, max_alter_threads, 0) \ /** The section above is for obsolete settings. Do not add anything there. */ From 031451ed17f3e6dfeda2d7f0fd6524f98b4f65b1 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 28 Dec 2021 16:48:34 -0400 Subject: [PATCH 0526/1260] replace obsolete max_alter_threads with max_final_threads in documentation examples --- docs/en/interfaces/http.md | 14 +++++++------- docs/ja/interfaces/http.md | 12 ++++++------ docs/ru/interfaces/http.md | 12 ++++++------ docs/zh/interfaces/http.md | 12 ++++++------ .../integration/test_http_handlers_config/test.py | 6 +++--- 5 files changed, 28 insertions(+), 28 deletions(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 313c6508b55..38e729fde0b 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -424,10 +424,10 @@ Next are the configuration methods for different `type`. `query` value is a predefined query of `predefined_query_handler`, which is executed by ClickHouse when an HTTP request is matched and the result of the query is returned. It is a must configuration. -The following example defines the values of [max_threads](../operations/settings/settings.md#settings-max_threads) and `max_alter_threads` settings, then queries the system table to check whether these settings were set successfully. +The following example defines the values of [max_threads](../operations/settings/settings.md#settings-max_threads) and `max_final_threads` settings, then queries the system table to check whether these settings were set successfully. !!! note "Warning" - To keep the default `handlers` such as` query`, `play`,` ping`, use the `` rule. + To keep the default `handlers` such as` query`, `play`,` ping`, use the `` rule. Example: @@ -451,9 +451,9 @@ Example: ``` ``` bash -$ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_threads' 'http://localhost:8123/query_param_with_url/1/max_threads/max_alter_threads?max_threads=1&max_alter_threads=2' +$ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_threads' 'http://localhost:8123/query_param_with_url/1/max_threads/max_final_threads?max_threads=1&max_final_threads=2' 1 -max_alter_threads 2 +max_final_threads 2 ``` !!! note "caution" @@ -465,7 +465,7 @@ In `dynamic_query_handler`, the query is written in the form of param of the HTT ClickHouse extracts and executes the value corresponding to the `query_param_name` value in the URL of the HTTP request. The default value of `query_param_name` is `/query` . It is an optional configuration. If there is no definition in the configuration file, the param is not passed in. -To experiment with this functionality, the example defines the values of [max_threads](../operations/settings/settings.md#settings-max_threads) and `max_alter_threads` and `queries` whether the settings were set successfully. +To experiment with this functionality, the example defines the values of [max_threads](../operations/settings/settings.md#settings-max_threads) and `max_final_threads` and `queries` whether the settings were set successfully. Example: @@ -484,9 +484,9 @@ Example: ``` ``` bash -$ curl -H 'XXX:TEST_HEADER_VALUE_DYNAMIC' 'http://localhost:8123/own?max_threads=1&max_alter_threads=2¶m_name_1=max_threads¶m_name_2=max_alter_threads&query_param=SELECT%20name,value%20FROM%20system.settings%20where%20name%20=%20%7Bname_1:String%7D%20OR%20name%20=%20%7Bname_2:String%7D' +$ curl -H 'XXX:TEST_HEADER_VALUE_DYNAMIC' 'http://localhost:8123/own?max_threads=1&max_final_threads=2¶m_name_1=max_threads¶m_name_2=max_final_threads&query_param=SELECT%20name,value%20FROM%20system.settings%20where%20name%20=%20%7Bname_1:String%7D%20OR%20name%20=%20%7Bname_2:String%7D' max_threads 1 -max_alter_threads 2 +max_final_threads 2 ``` ### static {#static} diff --git a/docs/ja/interfaces/http.md b/docs/ja/interfaces/http.md index 4ac9cd9e472..210e3f46d24 100644 --- a/docs/ja/interfaces/http.md +++ b/docs/ja/interfaces/http.md @@ -397,7 +397,7 @@ $ curl -v 'http://localhost:8123/predefined_query' `` 値は以下の定義済みクエリです `` これは、Http要求が一致し、クエリの結果が返されたときにClickHouseによって実行されます。 これは必須構成です。 -次の例では、次の値を定義します `max_threads` と `max_alter_threads` 設定、そしてクエリのテーブルから設定設定します。 +次の例では、次の値を定義します `max_threads` と `max_final_threads` 設定、そしてクエリのテーブルから設定設定します。 例: @@ -420,9 +420,9 @@ $ curl -v 'http://localhost:8123/predefined_query' ``` ``` bash -$ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_threads' 'http://localhost:8123/query_param_with_url/1/max_threads/max_alter_threads?max_threads=1&max_alter_threads=2' +$ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_threads' 'http://localhost:8123/query_param_with_url/1/max_threads/max_final_threads?max_threads=1&max_final_threads=2' 1 -max_alter_threads 2 +max_final_threads 2 ``` !!! note "注意" @@ -434,7 +434,7 @@ max_alter_threads 2 クリックハウスは、 `` HTTP要求のurlの値。 のデフォルト値 `` は `/query` . これはオプションの構成です。 設定ファイルに定義がない場合、paramは渡されません。 -この機能を試すために、この例ではmax_threadsとmax_alter_threadsの値を定義し、設定が正常に設定されたかどうかを照会します。 +この機能を試すために、この例ではmax_threadsとmax_final_threadsの値を定義し、設定が正常に設定されたかどうかを照会します。 例: @@ -452,9 +452,9 @@ max_alter_threads 2 ``` ``` bash -$ curl -H 'XXX:TEST_HEADER_VALUE_DYNAMIC' 'http://localhost:8123/own?max_threads=1&max_alter_threads=2¶m_name_1=max_threads¶m_name_2=max_alter_threads&query_param=SELECT%20name,value%20FROM%20system.settings%20where%20name%20=%20%7Bname_1:String%7D%20OR%20name%20=%20%7Bname_2:String%7D' +$ curl -H 'XXX:TEST_HEADER_VALUE_DYNAMIC' 'http://localhost:8123/own?max_threads=1&max_final_threads=2¶m_name_1=max_threads¶m_name_2=max_final_threads&query_param=SELECT%20name,value%20FROM%20system.settings%20where%20name%20=%20%7Bname_1:String%7D%20OR%20name%20=%20%7Bname_2:String%7D' max_threads 1 -max_alter_threads 2 +max_final_threads 2 ``` ## 静的 {#static} diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index 27a70a5c26d..8687201e1c9 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -422,7 +422,7 @@ $ curl -v 'http://localhost:8123/predefined_query' Значение `query` — это предопределенный запрос `predefined_query_handler`, который выполняется ClickHouse при совпадении HTTP-запроса и возврате результата запроса. Это обязательная настройка. -В следующем примере определяются настройки [max_threads](../operations/settings/settings.md#settings-max_threads) и `max_alter_threads`, а затем запрашивается системная таблица, чтобы проверить, были ли эти параметры успешно установлены. +В следующем примере определяются настройки [max_threads](../operations/settings/settings.md#settings-max_threads) и `max_final_threads`, а затем запрашивается системная таблица, чтобы проверить, были ли эти параметры успешно установлены. !!! note "Предупреждение" Чтобы сохранить стандартные `handlers` такие как `query`, `play`, `ping`, используйте правило ``. @@ -449,9 +449,9 @@ $ curl -v 'http://localhost:8123/predefined_query' ``` ``` bash -$ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_threads' 'http://localhost:8123/query_param_with_url/1/max_threads/max_alter_threads?max_threads=1&max_alter_threads=2' +$ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_threads' 'http://localhost:8123/query_param_with_url/1/max_threads/max_final_threads?max_threads=1&max_final_threads=2' 1 -max_alter_threads 2 +max_final_threads 2 ``` !!! note "Предупреждение" @@ -463,7 +463,7 @@ max_alter_threads 2 ClickHouse извлекает и выполняет значение, соответствующее значению `query_param_name` URL-адресе HTTP-запроса. Значение по умолчанию `query_param_name` — это `/query` . Это необязательная настройка. Если в файле конфигурации нет определения, параметр не передается. -Чтобы поэкспериментировать с этой функциональностью, в примере определяются значения [max_threads](../operations/settings/settings.md#settings-max_threads) и `max_alter_threads` и запрашивается, успешно ли были установлены настройки. +Чтобы поэкспериментировать с этой функциональностью, в примере определяются значения [max_threads](../operations/settings/settings.md#settings-max_threads) и `max_final_threads` и запрашивается, успешно ли были установлены настройки. Пример: @@ -482,9 +482,9 @@ ClickHouse извлекает и выполняет значение, соотв ``` ``` bash -$ curl -H 'XXX:TEST_HEADER_VALUE_DYNAMIC' 'http://localhost:8123/own?max_threads=1&max_alter_threads=2¶m_name_1=max_threads¶m_name_2=max_alter_threads&query_param=SELECT%20name,value%20FROM%20system.settings%20where%20name%20=%20%7Bname_1:String%7D%20OR%20name%20=%20%7Bname_2:String%7D' +$ curl -H 'XXX:TEST_HEADER_VALUE_DYNAMIC' 'http://localhost:8123/own?max_threads=1&max_final_threads=2¶m_name_1=max_threads¶m_name_2=max_final_threads&query_param=SELECT%20name,value%20FROM%20system.settings%20where%20name%20=%20%7Bname_1:String%7D%20OR%20name%20=%20%7Bname_2:String%7D' max_threads 1 -max_alter_threads 2 +max_final_threads 2 ``` ### static {#static} diff --git a/docs/zh/interfaces/http.md b/docs/zh/interfaces/http.md index cdce4f2f2e7..738b0365f46 100644 --- a/docs/zh/interfaces/http.md +++ b/docs/zh/interfaces/http.md @@ -407,7 +407,7 @@ $ curl -v 'http://localhost:8123/predefined_query' `query` 是一个预定义的`predefined_query_handler`查询,它由ClickHouse在匹配HTTP请求并返回查询结果时执行。这是一个必须的配置。 -以下是定义的[max_threads](../operations/settings/settings.md#settings-max_threads)和`max_alter_threads`设置, 然后查询系统表以检查这些设置是否设置成功。 +以下是定义的[max_threads](../operations/settings/settings.md#settings-max_threads)和`max_final_threads`设置, 然后查询系统表以检查这些设置是否设置成功。 示例: @@ -430,9 +430,9 @@ $ curl -v 'http://localhost:8123/predefined_query' ``` ``` bash -$ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_threads' 'http://localhost:8123/query_param_with_url/1/max_threads/max_alter_threads?max_threads=1&max_alter_threads=2' +$ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_threads' 'http://localhost:8123/query_param_with_url/1/max_threads/max_final_threads?max_threads=1&max_final_threads=2' 1 -max_alter_threads 2 +max_final_threads 2 ``` !!! note "警告" @@ -444,7 +444,7 @@ max_alter_threads 2 ClickHouse提取并执行与HTTP请求URL中的`query_param_name`值对应的值。`query_param_name`的默认值是`/query`。这是一个可选的配置。如果配置文件中没有定义,则不会传入参数。 -为了试验这个功能,示例定义了[max_threads](../operations/settings/settings.md#settings-max_threads)和`max_alter_threads`,`queries`设置是否成功的值。 +为了试验这个功能,示例定义了[max_threads](../operations/settings/settings.md#settings-max_threads)和`max_final_threads`,`queries`设置是否成功的值。 示例: @@ -462,9 +462,9 @@ ClickHouse提取并执行与HTTP请求URL中的`query_param_name`值对应的值 ``` ``` bash -$ curl -H 'XXX:TEST_HEADER_VALUE_DYNAMIC' 'http://localhost:8123/own?max_threads=1&max_alter_threads=2¶m_name_1=max_threads¶m_name_2=max_alter_threads&query_param=SELECT%20name,value%20FROM%20system.settings%20where%20name%20=%20%7Bname_1:String%7D%20OR%20name%20=%20%7Bname_2:String%7D' +$ curl -H 'XXX:TEST_HEADER_VALUE_DYNAMIC' 'http://localhost:8123/own?max_threads=1&max_final_threads=2¶m_name_1=max_threads¶m_name_2=max_final_threads&query_param=SELECT%20name,value%20FROM%20system.settings%20where%20name%20=%20%7Bname_1:String%7D%20OR%20name%20=%20%7Bname_2:String%7D' max_threads 1 -max_alter_threads 2 +max_final_threads 2 ``` ### static {#static} diff --git a/tests/integration/test_http_handlers_config/test.py b/tests/integration/test_http_handlers_config/test.py index 818a1e54640..1ea0f059b68 100644 --- a/tests/integration/test_http_handlers_config/test.py +++ b/tests/integration/test_http_handlers_config/test.py @@ -58,9 +58,9 @@ def test_predefined_query_handler(): 'test_predefined_handler_get?max_threads=1&setting_name=max_threads', method='GET', headers={'XXX': 'xxx'}).content - assert b'max_threads\t1\nmax_alter_threads\t1\n' == cluster.instance.http_request( - 'query_param_with_url/max_threads?max_threads=1&max_alter_threads=1', - headers={'XXX': 'max_alter_threads'}).content + assert b'max_threads\t1\nmax_final_threads\t1\n' == cluster.instance.http_request( + 'query_param_with_url/max_threads?max_threads=1&max_final_threads=1', + headers={'XXX': 'max_final_threads'}).content def test_fixed_static_handler(): From 9ef359ce2cccaa106a302e456b294f01264a55a4 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 28 Dec 2021 19:41:57 +0300 Subject: [PATCH 0527/1260] Dictionaries fix comment --- src/Databases/DatabaseDictionary.cpp | 3 ++ src/Databases/DatabasesCommon.cpp | 26 ++++++---- src/Interpreters/InterpreterAlterQuery.cpp | 5 ++ src/Storages/IStorage.cpp | 1 - src/Storages/StorageDictionary.cpp | 41 ++++++++++++++- src/Storages/StorageDictionary.h | 5 ++ .../System/StorageSystemDictionaries.cpp | 2 + .../02155_dictionary_comment.reference | 11 ++++ .../0_stateless/02155_dictionary_comment.sql | 50 +++++++++++++++++++ 9 files changed, 131 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/02155_dictionary_comment.reference create mode 100644 tests/queries/0_stateless/02155_dictionary_comment.sql diff --git a/src/Databases/DatabaseDictionary.cpp b/src/Databases/DatabaseDictionary.cpp index db7da95fb27..82766c1e384 100644 --- a/src/Databases/DatabaseDictionary.cpp +++ b/src/Databases/DatabaseDictionary.cpp @@ -29,10 +29,13 @@ namespace return nullptr; DictionaryStructure dictionary_structure = ExternalDictionariesLoader::getDictionaryStructure(*load_result.config); + auto comment = load_result.config->config->getString("dictionary.comment", ""); + return StorageDictionary::create( StorageID(database_name, load_result.name), load_result.name, dictionary_structure, + comment, StorageDictionary::Location::DictionaryDatabase, context); } diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index ffb39f5b113..1c3f417b431 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -30,27 +30,33 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo auto & ast_create_query = query->as(); bool has_structure = ast_create_query.columns_list && ast_create_query.columns_list->columns; + if (ast_create_query.as_table_function && !has_structure) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot alter table {} because it was created AS table function" " and doesn't have structure in metadata", backQuote(ast_create_query.getTable())); - assert(has_structure); - ASTPtr new_columns = InterpreterCreateQuery::formatColumns(metadata.columns); - ASTPtr new_indices = InterpreterCreateQuery::formatIndices(metadata.secondary_indices); - ASTPtr new_constraints = InterpreterCreateQuery::formatConstraints(metadata.constraints); - ASTPtr new_projections = InterpreterCreateQuery::formatProjections(metadata.projections); + if (!has_structure && !ast_create_query.is_dictionary) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot alter table {} metadata doesn't have structure", backQuote(ast_create_query.getTable())); - ast_create_query.columns_list->replace(ast_create_query.columns_list->columns, new_columns); - ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->indices, new_indices); - ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->constraints, new_constraints); - ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->projections, new_projections); + if (!ast_create_query.is_dictionary) + { + ASTPtr new_columns = InterpreterCreateQuery::formatColumns(metadata.columns); + ASTPtr new_indices = InterpreterCreateQuery::formatIndices(metadata.secondary_indices); + ASTPtr new_constraints = InterpreterCreateQuery::formatConstraints(metadata.constraints); + ASTPtr new_projections = InterpreterCreateQuery::formatProjections(metadata.projections); + + ast_create_query.columns_list->replace(ast_create_query.columns_list->columns, new_columns); + ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->indices, new_indices); + ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->constraints, new_constraints); + ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->projections, new_projections); + } if (metadata.select.select_query) { query->replace(ast_create_query.select, metadata.select.select_query); } - /// MaterializedView is one type of CREATE query without storage. + /// MaterializedView, Dictionary are types of CREATE query without storage. if (ast_create_query.storage) { ASTStorage & storage_ast = *ast_create_query.storage; diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 5f7c54e427f..2475d437acb 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -48,10 +48,15 @@ BlockIO InterpreterAlterQuery::execute() FunctionNameNormalizer().visit(query_ptr.get()); const auto & alter = query_ptr->as(); if (alter.alter_object == ASTAlterQuery::AlterObjectType::DATABASE) + { return executeToDatabase(alter); + } else if (alter.alter_object == ASTAlterQuery::AlterObjectType::TABLE || alter.alter_object == ASTAlterQuery::AlterObjectType::LIVE_VIEW) + { return executeToTable(alter); + } + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown alter object type"); } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 021335fea1f..a923258b111 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -139,7 +139,6 @@ void IStorage::alter(const AlterCommands & params, ContextPtr context, AlterLock setInMemoryMetadata(new_metadata); } - void IStorage::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /* context */) const { for (const auto & command : commands) diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index f6b330fe3df..7eb843d1f5b 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB @@ -111,10 +112,11 @@ StorageDictionary::StorageDictionary( const StorageID & table_id_, const String & dictionary_name_, const DictionaryStructure & dictionary_structure_, + const String & comment, Location location_, ContextPtr context_) : StorageDictionary( - table_id_, dictionary_name_, ColumnsDescription{getNamesAndTypes(dictionary_structure_)}, String{}, location_, context_) + table_id_, dictionary_name_, ColumnsDescription{getNamesAndTypes(dictionary_structure_)}, comment, location_, context_) { } @@ -126,6 +128,7 @@ StorageDictionary::StorageDictionary( table_id, table_id.getFullNameNotQuoted(), context_->getExternalDictionariesLoader().getDictionaryStructure(*dictionary_configuration), + dictionary_configuration->getString("dictionary.comment", ""), Location::SameDatabaseAndNameAsDictionary, context_) { @@ -230,7 +233,7 @@ void StorageDictionary::renameInMemory(const StorageID & new_table_id) if (move_to_atomic) configuration->setString("dictionary.uuid", toString(new_table_id.uuid)); else if (move_to_ordinary) - configuration->remove("dictionary.uuid"); + configuration->remove("dictionary.uuid"); } /// Dictionary is moving between databases of different engines or is renaming inside Ordinary database @@ -260,6 +263,40 @@ void StorageDictionary::renameInMemory(const StorageID & new_table_id) } } +void StorageDictionary::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /* context */) const +{ + for (const auto & command : commands) + { + if (location == Location::DictionaryDatabase || command.type != AlterCommand::COMMENT_TABLE) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}", + command.type, getName()); + } +} + +void StorageDictionary::alter(const AlterCommands & params, ContextPtr alter_context, AlterLockHolder & lock_holder) +{ + IStorage::alter(params, alter_context, lock_holder); + + if (location == Location::Custom) + return; + + auto new_comment = getInMemoryMetadataPtr()->comment; + + auto storage_id = getStorageID(); + const auto & external_dictionaries_loader = getContext()->getExternalDictionariesLoader(); + auto result = external_dictionaries_loader.getLoadResult(storage_id.getInternalDictionaryName()); + + if (result.object) + { + auto dictionary = std::static_pointer_cast(result.object); + auto * dictionary_non_const = const_cast(dictionary.get()); + dictionary_non_const->setDictionaryComment(new_comment); + } + + std::lock_guard lock(dictionary_config_mutex); + configuration->setString("dictionary.comment", std::move(new_comment)); +} + void registerStorageDictionary(StorageFactory & factory) { factory.registerStorage("Dictionary", [](const StorageFactory::Arguments & args) diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index 7d0af8c0ee3..855d02b0947 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -42,6 +42,10 @@ public: void renameInMemory(const StorageID & new_table_id) override; + void checkAlterIsPossible(const AlterCommands & commands, ContextPtr /* context */) const override; + + void alter(const AlterCommands & params, ContextPtr alter_context, AlterLockHolder &) override; + Poco::Timestamp getUpdateTime() const; LoadablesConfigurationPtr getConfiguration() const; @@ -89,6 +93,7 @@ private: const StorageID & table_id_, const String & dictionary_name_, const DictionaryStructure & dictionary_structure, + const String & comment, Location location_, ContextPtr context_); diff --git a/src/Storages/System/StorageSystemDictionaries.cpp b/src/Storages/System/StorageSystemDictionaries.cpp index d8f92d38081..c0d7d8cc4ed 100644 --- a/src/Storages/System/StorageSystemDictionaries.cpp +++ b/src/Storages/System/StorageSystemDictionaries.cpp @@ -142,7 +142,9 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, ContextPt res_columns[i++]->insertDefault(); if (dict_ptr) + { res_columns[i++]->insert(dict_ptr->getDictionaryComment()); + } else { if (load_result.config && load_result.config->config->has("dictionary.comment")) diff --git a/tests/queries/0_stateless/02155_dictionary_comment.reference b/tests/queries/0_stateless/02155_dictionary_comment.reference new file mode 100644 index 00000000000..69b871a6925 --- /dev/null +++ b/tests/queries/0_stateless/02155_dictionary_comment.reference @@ -0,0 +1,11 @@ +02155_test_dictionary +02155_test_dictionary 02155_test_dictionary_comment_0 +02155_test_dictionary 02155_test_dictionary_comment_0 +0 Value +02155_test_dictionary 02155_test_dictionary_comment_0 +02155_test_dictionary 02155_test_dictionary_comment_0 +02155_test_dictionary 02155_test_dictionary_comment_1 +02155_test_dictionary 02155_test_dictionary_comment_1 +0 Value +02155_test_dictionary_view 02155_test_dictionary_view_comment_0 +02155_test_dictionary_view 02155_test_dictionary_view_comment_0 diff --git a/tests/queries/0_stateless/02155_dictionary_comment.sql b/tests/queries/0_stateless/02155_dictionary_comment.sql new file mode 100644 index 00000000000..66d31004fb2 --- /dev/null +++ b/tests/queries/0_stateless/02155_dictionary_comment.sql @@ -0,0 +1,50 @@ +DROP TABLE IF EXISTS 02155_test_table; +CREATE TABLE 02155_test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO 02155_test_table VALUES (0, 'Value'); + +DROP DICTIONARY IF EXISTS 02155_test_dictionary; +CREATE DICTIONARY 02155_test_dictionary +( + id UInt64, + value String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE '02155_test_table')) +LAYOUT(DIRECT()); + +SELECT name, comment FROM system.dictionaries WHERE name == '02155_test_dictionary'; + +ALTER TABLE 02155_test_dictionary COMMENT COLUMN value 'value_column'; --{serverError 48} + +ALTER TABLE 02155_test_dictionary MODIFY COMMENT '02155_test_dictionary_comment_0'; +SELECT name, comment FROM system.dictionaries WHERE name == '02155_test_dictionary'; +SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary'; + +SELECT * FROM 02155_test_dictionary; +SELECT name, comment FROM system.dictionaries WHERE name == '02155_test_dictionary'; +SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary'; + +ALTER TABLE 02155_test_dictionary MODIFY COMMENT '02155_test_dictionary_comment_1'; +SELECT name, comment FROM system.dictionaries WHERE name == '02155_test_dictionary'; +SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary'; + +CREATE TABLE 02155_test_dictionary_view +( + id UInt64, + value String +) ENGINE=Dictionary('02155_test_dictionary'); + +SELECT * FROM 02155_test_dictionary_view; + +ALTER TABLE 02155_test_dictionary_view COMMENT COLUMN value 'value_column'; --{serverError 48} +ALTER TABLE 02155_test_dictionary_view MODIFY COMMENT '02155_test_dictionary_view_comment_0'; +SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary_view'; +SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary_view'; + +DROP TABLE 02155_test_dictionary_view; +DROP DICTIONARY 02155_test_dictionary; From df95a40bbf441ce3d3b5369bdabaa21b511f0ad0 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 28 Dec 2021 23:49:41 +0300 Subject: [PATCH 0528/1260] Fixed tests --- src/Storages/StorageDictionary.cpp | 1 + .../0_stateless/02155_dictionary_comment.sql | 23 +++++++++++-------- 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 7eb843d1f5b..da8c5f115b2 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -22,6 +22,7 @@ namespace ErrorCodes extern const int THERE_IS_NO_COLUMN; extern const int CANNOT_DETACH_DICTIONARY_AS_TABLE; extern const int DICTIONARY_ALREADY_EXISTS; + extern const int NOT_IMPLEMENTED; } namespace diff --git a/tests/queries/0_stateless/02155_dictionary_comment.sql b/tests/queries/0_stateless/02155_dictionary_comment.sql index 66d31004fb2..e31d9d28366 100644 --- a/tests/queries/0_stateless/02155_dictionary_comment.sql +++ b/tests/queries/0_stateless/02155_dictionary_comment.sql @@ -17,34 +17,37 @@ PRIMARY KEY id SOURCE(CLICKHOUSE(TABLE '02155_test_table')) LAYOUT(DIRECT()); -SELECT name, comment FROM system.dictionaries WHERE name == '02155_test_dictionary'; +SELECT name, comment FROM system.dictionaries WHERE name == '02155_test_dictionary' AND database == currentDatabase(); ALTER TABLE 02155_test_dictionary COMMENT COLUMN value 'value_column'; --{serverError 48} ALTER TABLE 02155_test_dictionary MODIFY COMMENT '02155_test_dictionary_comment_0'; -SELECT name, comment FROM system.dictionaries WHERE name == '02155_test_dictionary'; -SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary'; +SELECT name, comment FROM system.dictionaries WHERE name == '02155_test_dictionary' AND database == currentDatabase(); +SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary' AND database == currentDatabase(); SELECT * FROM 02155_test_dictionary; -SELECT name, comment FROM system.dictionaries WHERE name == '02155_test_dictionary'; -SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary'; +SELECT name, comment FROM system.dictionaries WHERE name == '02155_test_dictionary' AND database == currentDatabase(); +SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary' AND database == currentDatabase(); ALTER TABLE 02155_test_dictionary MODIFY COMMENT '02155_test_dictionary_comment_1'; -SELECT name, comment FROM system.dictionaries WHERE name == '02155_test_dictionary'; -SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary'; +SELECT name, comment FROM system.dictionaries WHERE name == '02155_test_dictionary' AND database == currentDatabase(); +SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary' AND database == currentDatabase(); +DROP TABLE IF EXISTS 02155_test_dictionary_view; CREATE TABLE 02155_test_dictionary_view ( id UInt64, value String -) ENGINE=Dictionary('02155_test_dictionary'); +) ENGINE=Dictionary(concat(currentDatabase(), '.02155_test_dictionary')); SELECT * FROM 02155_test_dictionary_view; ALTER TABLE 02155_test_dictionary_view COMMENT COLUMN value 'value_column'; --{serverError 48} + ALTER TABLE 02155_test_dictionary_view MODIFY COMMENT '02155_test_dictionary_view_comment_0'; -SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary_view'; -SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary_view'; +SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary_view' AND database == currentDatabase(); +SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary_view' AND database == currentDatabase(); DROP TABLE 02155_test_dictionary_view; +DROP TABLE 02155_test_table; DROP DICTIONARY 02155_test_dictionary; From c8dcde7843756575fb6061650a81a35dc5322fcd Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 28 Dec 2021 23:48:10 +0300 Subject: [PATCH 0529/1260] Fix --- .../test.py | 165 ++++++++++-------- 1 file changed, 93 insertions(+), 72 deletions(-) diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index 80cfed7d17c..92b9d52cf86 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -32,6 +32,27 @@ def cluster(): finally: cluster.shutdown() +# Note: use this for selects and inserts and create table queries. +# For inserts there is no guarantee that retries will not result in duplicates. +# But it is better to retry anyway because 'Connection was closed by the server' error +# happens in fact only for inserts because reads already have build-in retries in code. +def azure_query(node, query, try_num=3): + for i in range(try_num): + try: + return node.query(query) + except Exception as ex: + retriable_errors = [ + 'DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response', + ] + retry = False + for error in retriable_errors: + if error in str(ex): + retry = True + logging.info(f"Try num: {i}. Having retriable error: {ex}") + break + if not retry or i == try_num - 1: + raise Exception(ex) + continue def create_table(node, table_name, **additional_settings): settings = { @@ -53,8 +74,8 @@ def create_table(node, table_name, **additional_settings): SETTINGS {",".join((k+"="+repr(v) for k, v in settings.items()))}""" node.query(f"DROP TABLE IF EXISTS {table_name}") - node.query(create_table_statement) - assert node.query(f"SELECT COUNT(*) FROM {table_name} FORMAT Values") == "(0)" + azure_query(node, create_table_statement) + assert azure_query(node, f"SELECT COUNT(*) FROM {table_name} FORMAT Values") == "(0)" def test_create_table(cluster): @@ -68,13 +89,13 @@ def test_read_after_cache_is_wiped(cluster): values = "('2021-11-13',3,'hello'),('2021-11-14',4,'heyo')" - node.query(f"INSERT INTO {TABLE_NAME} VALUES {values}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {values}") # Wipe cache cluster.exec_in_container(cluster.get_container_id(NODE_NAME), ["rm", "-rf", "/var/lib/clickhouse/disks/blob_storage_disk/cache/"]) # After cache is populated again, only .bin files should be accessed from Blob Storage. - assert node.query(f"SELECT * FROM {TABLE_NAME} order by dt, id FORMAT Values") == values + assert azure_query(node, f"SELECT * FROM {TABLE_NAME} order by dt, id FORMAT Values") == values def test_simple_insert_select(cluster): @@ -82,8 +103,8 @@ def test_simple_insert_select(cluster): create_table(node, TABLE_NAME) values = "('2021-11-13',3,'hello')" - node.query(f"INSERT INTO {TABLE_NAME} VALUES {values}") - assert node.query(f"SELECT dt, id, data FROM {TABLE_NAME} FORMAT Values") == values + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {values}") + assert azure_query(node, f"SELECT dt, id, data FROM {TABLE_NAME} FORMAT Values") == values blob_container_client = cluster.blob_service_client.get_container_client(CONTAINER_NAME) assert len(list(blob_container_client.list_blobs())) >= 12 # 1 format file + 2 skip index files + 9 regular MergeTree files + leftovers from other tests @@ -93,14 +114,14 @@ def test_inserts_selects(cluster): create_table(node, TABLE_NAME) values1 = generate_values('2020-01-03', 4096) - node.query(f"INSERT INTO {TABLE_NAME} VALUES {values1}") - assert node.query(f"SELECT * FROM {TABLE_NAME} order by dt, id FORMAT Values") == values1 + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {values1}") + assert azure_query(node, f"SELECT * FROM {TABLE_NAME} order by dt, id FORMAT Values") == values1 values2 = generate_values('2020-01-04', 4096) - node.query(f"INSERT INTO {TABLE_NAME} VALUES {values2}") - assert node.query(f"SELECT * FROM {TABLE_NAME} ORDER BY dt, id FORMAT Values") == values1 + "," + values2 + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {values2}") + assert azure_query(node, f"SELECT * FROM {TABLE_NAME} ORDER BY dt, id FORMAT Values") == values1 + "," + values2 - assert node.query(f"SELECT count(*) FROM {TABLE_NAME} where id = 1 FORMAT Values") == "(2)" + assert azure_query(node, f"SELECT count(*) FROM {TABLE_NAME} where id = 1 FORMAT Values") == "(2)" @pytest.mark.parametrize( @@ -118,20 +139,20 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical): create_table(node, TABLE_NAME, **settings) node.query(f"SYSTEM STOP MERGES {TABLE_NAME}") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 1024)}") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 2048)}") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 1024, -1)}") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 2048, -1)}") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096, -1)}") - assert node.query(f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" - assert node.query(f"SELECT count(distinct(id)) FROM {TABLE_NAME} FORMAT Values") == "(8192)" + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 1024)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 2048)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 1024, -1)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 2048, -1)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096, -1)}") + assert azure_query(node, f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" + assert azure_query(node, f"SELECT count(distinct(id)) FROM {TABLE_NAME} FORMAT Values") == "(8192)" node.query(f"SYSTEM START MERGES {TABLE_NAME}") # Wait for merges and old parts deletion for attempt in range(0, 10): - parts_count = node.query(f"SELECT COUNT(*) FROM system.parts WHERE table = '{TABLE_NAME}' FORMAT Values") + parts_count = azure_query(node, f"SELECT COUNT(*) FROM system.parts WHERE table = '{TABLE_NAME}' FORMAT Values") if parts_count == "(1)": break @@ -140,64 +161,64 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical): time.sleep(1) - assert node.query(f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" - assert node.query(f"SELECT count(distinct(id)) FROM {TABLE_NAME} FORMAT Values") == "(8192)" + assert azure_query(node, f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" + assert azure_query(node, f"SELECT count(distinct(id)) FROM {TABLE_NAME} FORMAT Values") == "(8192)" def test_alter_table_columns(cluster): node = cluster.instances[NODE_NAME] create_table(node, TABLE_NAME) - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096, -1)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096, -1)}") node.query(f"ALTER TABLE {TABLE_NAME} ADD COLUMN col1 UInt64 DEFAULT 1") # To ensure parts have been merged node.query(f"OPTIMIZE TABLE {TABLE_NAME}") - assert node.query(f"SELECT sum(col1) FROM {TABLE_NAME} FORMAT Values") == "(8192)" - assert node.query(f"SELECT sum(col1) FROM {TABLE_NAME} WHERE id > 0 FORMAT Values") == "(4096)" + assert azure_query(node, f"SELECT sum(col1) FROM {TABLE_NAME} FORMAT Values") == "(8192)" + assert azure_query(node, f"SELECT sum(col1) FROM {TABLE_NAME} WHERE id > 0 FORMAT Values") == "(4096)" node.query(f"ALTER TABLE {TABLE_NAME} MODIFY COLUMN col1 String", settings={"mutations_sync": 2}) - assert node.query(f"SELECT distinct(col1) FROM {TABLE_NAME} FORMAT Values") == "('1')" + assert azure_query(node, f"SELECT distinct(col1) FROM {TABLE_NAME} FORMAT Values") == "('1')" def test_attach_detach_partition(cluster): node = cluster.instances[NODE_NAME] create_table(node, TABLE_NAME) - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096)}") - assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)" + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096)}") + assert azure_query(node, f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)" node.query(f"ALTER TABLE {TABLE_NAME} DETACH PARTITION '2020-01-03'") - assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(4096)" + assert azure_query(node, f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(4096)" node.query(f"ALTER TABLE {TABLE_NAME} ATTACH PARTITION '2020-01-03'") - assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)" + assert azure_query(node, f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)" node.query(f"ALTER TABLE {TABLE_NAME} DROP PARTITION '2020-01-03'") - assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(4096)" + assert azure_query(node, f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(4096)" node.query(f"ALTER TABLE {TABLE_NAME} DETACH PARTITION '2020-01-04'") node.query(f"ALTER TABLE {TABLE_NAME} DROP DETACHED PARTITION '2020-01-04'", settings={"allow_drop_detached": 1}) - assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(0)" + assert azure_query(node, f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(0)" def test_move_partition_to_another_disk(cluster): node = cluster.instances[NODE_NAME] create_table(node, TABLE_NAME) - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096)}") - assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)" + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096)}") + assert azure_query(node, f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)" node.query(f"ALTER TABLE {TABLE_NAME} MOVE PARTITION '2020-01-04' TO DISK '{LOCAL_DISK}'") - assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)" + assert azure_query(node, f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)" node.query(f"ALTER TABLE {TABLE_NAME} MOVE PARTITION '2020-01-04' TO DISK '{AZURE_BLOB_STORAGE_DISK}'") - assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)" + assert azure_query(node, f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)" def test_table_manipulations(cluster): @@ -210,17 +231,17 @@ def test_table_manipulations(cluster): node.query_with_retry(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096)}") node.query(f"RENAME TABLE {TABLE_NAME} TO {renamed_table}") - assert node.query(f"SELECT count(*) FROM {renamed_table} FORMAT Values") == "(8192)" + assert azure_query(node, f"SELECT count(*) FROM {renamed_table} FORMAT Values") == "(8192)" node.query(f"RENAME TABLE {renamed_table} TO {TABLE_NAME}") assert node.query(f"CHECK TABLE {TABLE_NAME} FORMAT Values") == "(1)" node.query(f"DETACH TABLE {TABLE_NAME}") node.query(f"ATTACH TABLE {TABLE_NAME}") - assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)" + assert azure_query(node, f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)" node.query(f"TRUNCATE TABLE {TABLE_NAME}") - assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(0)" + assert azure_query(node, f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(0)" @pytest.mark.long_run @@ -230,38 +251,38 @@ def test_move_replace_partition_to_another_table(cluster): table_clone_name = TABLE_NAME + "_clone" - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 256)}") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 256)}") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-05', 256, -1)}") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-06', 256, -1)}") - assert node.query(f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" - assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(1024)" + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 256)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 256)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-05', 256, -1)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-06', 256, -1)}") + assert azure_query(node, f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" + assert azure_query(node, f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(1024)" create_table(node, table_clone_name) node.query(f"ALTER TABLE {TABLE_NAME} MOVE PARTITION '2020-01-03' TO TABLE {table_clone_name}") node.query(f"ALTER TABLE {TABLE_NAME} MOVE PARTITION '2020-01-05' TO TABLE {table_clone_name}") - assert node.query(f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" - assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(512)" - assert node.query(f"SELECT sum(id) FROM {table_clone_name} FORMAT Values") == "(0)" - assert node.query(f"SELECT count(*) FROM {table_clone_name} FORMAT Values") == "(512)" + assert azure_query(node, f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" + assert azure_query(node, f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(512)" + assert azure_query(node, f"SELECT sum(id) FROM {table_clone_name} FORMAT Values") == "(0)" + assert azure_query(node, f"SELECT count(*) FROM {table_clone_name} FORMAT Values") == "(512)" # Add new partitions to source table, but with different values and replace them from copied table. - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 256, -1)}") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-05', 256)}") - assert node.query(f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" - assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(1024)" + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 256, -1)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-05', 256)}") + assert azure_query(node, f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" + assert azure_query(node, f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(1024)" node.query(f"ALTER TABLE {TABLE_NAME} REPLACE PARTITION '2020-01-03' FROM {table_clone_name}") node.query(f"ALTER TABLE {TABLE_NAME} REPLACE PARTITION '2020-01-05' FROM {table_clone_name}") - assert node.query(f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" - assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(1024)" - assert node.query(f"SELECT sum(id) FROM {table_clone_name} FORMAT Values") == "(0)" - assert node.query(f"SELECT count(*) FROM {table_clone_name} FORMAT Values") == "(512)" + assert azure_query(node, f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" + assert azure_query(node, f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(1024)" + assert azure_query(node, f"SELECT sum(id) FROM {table_clone_name} FORMAT Values") == "(0)" + assert azure_query(node, f"SELECT count(*) FROM {table_clone_name} FORMAT Values") == "(512)" node.query(f"DROP TABLE {table_clone_name} NO DELAY") - assert node.query(f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" - assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(1024)" + assert azure_query(node, f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" + assert azure_query(node, f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(1024)" node.query(f"ALTER TABLE {TABLE_NAME} FREEZE") @@ -275,12 +296,12 @@ def test_freeze_unfreeze(cluster): backup1 = 'backup1' backup2 = 'backup2' - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}") node.query(f"ALTER TABLE {TABLE_NAME} FREEZE WITH NAME '{backup1}'") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096)}") node.query(f"ALTER TABLE {TABLE_NAME} FREEZE WITH NAME '{backup2}'") - node.query(f"TRUNCATE TABLE {TABLE_NAME}") + azure_query(node, f"TRUNCATE TABLE {TABLE_NAME}") # Unfreeze single partition from backup1. node.query(f"ALTER TABLE {TABLE_NAME} UNFREEZE PARTITION '2020-01-03' WITH NAME '{backup1}'") @@ -292,7 +313,7 @@ def test_apply_new_settings(cluster): node = cluster.instances[NODE_NAME] create_table(node, TABLE_NAME) - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}") # Force multi-part upload mode. replace_config( @@ -301,7 +322,7 @@ def test_apply_new_settings(cluster): "4096") node.query("SYSTEM RELOAD CONFIG") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096, -1)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096, -1)}") # NOTE: this test takes a couple of minutes when run together with other tests @@ -313,14 +334,14 @@ def test_restart_during_load(cluster): # Force multi-part upload mode. replace_config(CONFIG_PATH, "false", "") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096)}") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-05', 4096, -1)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096)}") + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-05', 4096, -1)}") def read(): for ii in range(0, 5): logging.info(f"Executing {ii} query") - assert node.query(f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" + assert azure_query(node, f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" logging.info(f"Query {ii} executed") time.sleep(0.2) @@ -347,5 +368,5 @@ def test_restart_during_load(cluster): def test_big_insert(cluster): node = cluster.instances[NODE_NAME] create_table(node, TABLE_NAME) - node.query(f"INSERT INTO {TABLE_NAME} select '2020-01-03', number, toString(number) from numbers(5000000)") - assert int(node.query(f"SELECT count() FROM {TABLE_NAME}")) == 5000000 + azure_query(node, f"INSERT INTO {TABLE_NAME} select '2020-01-03', number, toString(number) from numbers(5000000)") + assert int(azure_query(node, f"SELECT count() FROM {TABLE_NAME}")) == 5000000 From d25560d37a9d7af4f314f4bb3b997b307cb7c750 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 29 Dec 2021 00:14:22 +0300 Subject: [PATCH 0530/1260] fix queries without table name --- src/Interpreters/executeQuery.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index bf7b5d86d90..bd3c35c12f6 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -559,10 +559,10 @@ static std::tuple executeQueryImpl( /// Resolve database before trying to use async insert feature - to properly hash the query. if (insert_query) { - if (!insert_query->table_id) - insert_query->table_id = StorageID{insert_query->getDatabase(), insert_query->getTable()}; - - insert_query->table_id = context->resolveStorageID(insert_query->table_id); + if (insert_query->table_id) + insert_query->table_id = context->resolveStorageID(insert_query->table_id); + else if (auto table = insert_query->getTable(); !table.empty()) + insert_query->table_id = context->resolveStorageID(StorageID{insert_query->getDatabase(), table}); } if (insert_query && insert_query->select) From 8ca12bb2e8790d6526feaa4e4af80b7b6a115fee Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 29 Dec 2021 00:43:33 +0300 Subject: [PATCH 0531/1260] fix style check --- tests/queries/0_stateless/02156_async_insert_query_log.sh | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/queries/0_stateless/02156_async_insert_query_log.sh b/tests/queries/0_stateless/02156_async_insert_query_log.sh index 4eb16f26def..d7177fbe70c 100755 --- a/tests/queries/0_stateless/02156_async_insert_query_log.sh +++ b/tests/queries/0_stateless/02156_async_insert_query_log.sh @@ -4,9 +4,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -url_wait="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=1" -url_no_wait="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=1" - ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts_2156" ${CLICKHOUSE_CLIENT} -q "CREATE TABLE async_inserts_2156 (id UInt32, s String) ENGINE = Memory" From e8e19b8b0d62c380aa88a4e364a00cb29b62ce87 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 29 Dec 2021 01:20:34 +0300 Subject: [PATCH 0532/1260] Always display resource usage in client --- src/Client/ClientBase.cpp | 61 ++++++++++------------ src/Common/ProgressIndication.cpp | 86 +++++++++++++++++++++---------- src/Common/ProgressIndication.h | 7 +-- 3 files changed, 88 insertions(+), 66 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 75e0588f786..8d7cdd1577d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -800,41 +800,38 @@ void ClientBase::onProfileEvents(Block & block) if (rows == 0) return; - if (progress_indication.print_hardware_utilization) + const auto & array_thread_id = typeid_cast(*block.getByName("thread_id").column).getData(); + const auto & names = typeid_cast(*block.getByName("name").column); + const auto & host_names = typeid_cast(*block.getByName("host_name").column); + const auto & array_values = typeid_cast(*block.getByName("value").column).getData(); + + const auto * user_time_name = ProfileEvents::getName(ProfileEvents::UserTimeMicroseconds); + const auto * system_time_name = ProfileEvents::getName(ProfileEvents::SystemTimeMicroseconds); + + HostToThreadTimesMap thread_times; + for (size_t i = 0; i < rows; ++i) { - const auto & array_thread_id = typeid_cast(*block.getByName("thread_id").column).getData(); - const auto & names = typeid_cast(*block.getByName("name").column); - const auto & host_names = typeid_cast(*block.getByName("host_name").column); - const auto & array_values = typeid_cast(*block.getByName("value").column).getData(); - - const auto * user_time_name = ProfileEvents::getName(ProfileEvents::UserTimeMicroseconds); - const auto * system_time_name = ProfileEvents::getName(ProfileEvents::SystemTimeMicroseconds); - - HostToThreadTimesMap thread_times; - for (size_t i = 0; i < rows; ++i) + auto thread_id = array_thread_id[i]; + auto host_name = host_names.getDataAt(i).toString(); + if (thread_id != 0) + progress_indication.addThreadIdToList(host_name, thread_id); + auto event_name = names.getDataAt(i); + auto value = array_values[i]; + if (event_name == user_time_name) { - auto thread_id = array_thread_id[i]; - auto host_name = host_names.getDataAt(i).toString(); - if (thread_id != 0) - progress_indication.addThreadIdToList(host_name, thread_id); - auto event_name = names.getDataAt(i); - auto value = array_values[i]; - if (event_name == user_time_name) - { - thread_times[host_name][thread_id].user_ms = value; - } - else if (event_name == system_time_name) - { - thread_times[host_name][thread_id].system_ms = value; - } - else if (event_name == MemoryTracker::USAGE_EVENT_NAME) - { - thread_times[host_name][thread_id].memory_usage = value; - } + thread_times[host_name][thread_id].user_ms = value; + } + else if (event_name == system_time_name) + { + thread_times[host_name][thread_id].system_ms = value; + } + else if (event_name == MemoryTracker::USAGE_EVENT_NAME) + { + thread_times[host_name][thread_id].memory_usage = value; } - auto elapsed_time = profile_events.watch.elapsedMicroseconds(); - progress_indication.updateThreadEventData(thread_times, elapsed_time); } + auto elapsed_time = profile_events.watch.elapsedMicroseconds(); + progress_indication.updateThreadEventData(thread_times, elapsed_time); if (profile_events.print) { @@ -1909,8 +1906,6 @@ void ClientBase::init(int argc, char ** argv) Poco::Logger::root().setLevel(options["log-level"].as()); if (options.count("server_logs_file")) server_logs_file = options["server_logs_file"].as(); - if (options.count("hardware-utilization")) - progress_indication.print_hardware_utilization = true; query_processing_stage = QueryProcessingStage::fromString(options["stage"].as()); profile_events.print = options.count("print-profile-events"); diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 33508f5ad5f..f41d5f14c5a 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -16,7 +16,7 @@ namespace { constexpr UInt64 ALL_THREADS = 0; - UInt64 calculateCoresNumber(DB::ThreadIdToTimeMap times, UInt64 elapsed) + double calculateCPUUsage(DB::ThreadIdToTimeMap times, UInt64 elapsed) { auto accumulated = std::accumulate(times.begin(), times.end(), 0, [](Int64 acc, const auto & elem) @@ -25,7 +25,7 @@ namespace return acc; return acc + elem.second.time(); }); - return (static_cast(accumulated) + elapsed - 1) / elapsed; + return static_cast(accumulated) / elapsed; } } @@ -53,7 +53,7 @@ void ProgressIndication::resetProgress() show_progress_bar = false; written_progress_chars = 0; write_progress_on_update = false; - host_active_cores.clear(); + host_cpu_usage.clear(); thread_data.clear(); } @@ -81,8 +81,7 @@ void ProgressIndication::updateThreadEventData(HostToThreadTimesMap & new_thread { for (auto & new_host_map : new_thread_data) { - auto new_cores = calculateCoresNumber(new_host_map.second, elapsed_time); - host_active_cores[new_host_map.first] = new_cores; + host_cpu_usage[new_host_map.first] = calculateCPUUsage(new_host_map.second, elapsed_time); thread_data[new_host_map.first] = std::move(new_host_map.second); } } @@ -96,13 +95,12 @@ size_t ProgressIndication::getUsedThreadsCount() const }); } -UInt64 ProgressIndication::getApproximateCoresNumber() const +double ProgressIndication::getCPUUsage() const { - return std::accumulate(host_active_cores.cbegin(), host_active_cores.cend(), 0, - [](UInt64 acc, auto const & elem) - { - return acc + elem.second; - }); + double res = 0; + for (const auto & elem : host_cpu_usage) + res += elem.second; + return res; } ProgressIndication::MemoryUsage ProgressIndication::getMemoryUsage() const @@ -116,6 +114,7 @@ ProgressIndication::MemoryUsage ProgressIndication::getMemoryUsage() const // memory consumption it's enough to look for data with thread id 0. if (auto it = host_data.second.find(ALL_THREADS); it != host_data.second.end()) host_usage = it->second.memory_usage; + return MemoryUsage{.total = acc.total + host_usage, .max = std::max(acc.max, host_usage)}; }); } @@ -183,27 +182,28 @@ void ProgressIndication::writeProgress() written_progress_chars = message.count() - prefix_size - (strlen(indicator) - 2); /// Don't count invisible output (escape sequences). - // If approximate cores number is known, display it. - auto cores_number = getApproximateCoresNumber(); std::string profiling_msg; - if (cores_number != 0 && print_hardware_utilization) + + // If approximate cores number is known, display it. + double cpu_usage = getCPUUsage(); + if (cpu_usage > 0) { WriteBufferFromOwnString profiling_msg_builder; - // Calculated cores number may be not accurate - // so it's better to print min(threads, cores). - UInt64 threads_number = getUsedThreadsCount(); - profiling_msg_builder << " Running " << threads_number << " threads on " - << std::min(cores_number, threads_number) << " cores"; + + profiling_msg_builder << "(" << fmt::format("{:.1f}", cpu_usage) << " CPU"; auto [memory_usage, max_host_usage] = getMemoryUsage(); if (memory_usage != 0) - profiling_msg_builder << " with " << formatReadableSizeWithDecimalSuffix(memory_usage) << " RAM used"; - if (thread_data.size() > 1 && max_host_usage) - profiling_msg_builder << " total (per host max: " << formatReadableSizeWithDecimalSuffix(max_host_usage) << ")"; - profiling_msg_builder << "."; + profiling_msg_builder << ", " << formatReadableSizeWithDecimalSuffix(memory_usage) << " RAM"; + if (max_host_usage < memory_usage) + profiling_msg_builder << ", " << formatReadableSizeWithDecimalSuffix(max_host_usage) << " max/host"; + + profiling_msg_builder << ")"; profiling_msg = profiling_msg_builder.str(); } + int64_t remaining_space = static_cast(terminal_width) - written_progress_chars; + /// If the approximate number of rows to process is known, we can display a progress bar and percentage. if (progress.total_rows_to_read || progress.total_raw_bytes_to_read) { @@ -230,14 +230,35 @@ void ProgressIndication::writeProgress() if (show_progress_bar) { - ssize_t width_of_progress_bar = static_cast(terminal_width) - written_progress_chars - strlen(" 99%") - profiling_msg.length(); + /// We will display profiling info only if there is enough space for it. + int64_t width_of_progress_bar = remaining_space - strlen(" 99%"); + + /// We need at least twice the space, because it will be displayed either + /// at right after progress bar or at left on top of the progress bar. + if (width_of_progress_bar <= 1 + 2 * static_cast(profiling_msg.size())) + profiling_msg.clear(); + else + width_of_progress_bar -= profiling_msg.size(); + if (width_of_progress_bar > 0) { - std::string bar - = UnicodeBar::render(UnicodeBar::getWidth(current_count, 0, max_count, width_of_progress_bar)); + size_t bar_width = UnicodeBar::getWidth(current_count, 0, max_count, width_of_progress_bar); + std::string bar = UnicodeBar::render(bar_width); + + /// Render profiling_msg at left on top of the progress bar. + bool render_profiling_msg_at_left = current_count * 2 >= max_count; + if (!profiling_msg.empty() && render_profiling_msg_at_left) + message << "\033[30;42m" << profiling_msg << "\033[0m"; + message << "\033[0;32m" << bar << "\033[0m"; - if (width_of_progress_bar > static_cast(bar.size() / UNICODE_BAR_CHAR_SIZE)) + + /// Whitespaces after the progress bar. + if (width_of_progress_bar > static_cast(bar.size() / UNICODE_BAR_CHAR_SIZE)) message << std::string(width_of_progress_bar - bar.size() / UNICODE_BAR_CHAR_SIZE, ' '); + + /// Render profiling_msg at right after the progress bar. + if (!profiling_msg.empty() && !render_profiling_msg_at_left) + message << "\033[2m" << profiling_msg << "\033[0m"; } } } @@ -245,8 +266,17 @@ void ProgressIndication::writeProgress() /// Underestimate percentage a bit to avoid displaying 100%. message << ' ' << (99 * current_count / max_count) << '%'; } + else + { + /// We can still display profiling info. + if (remaining_space >= static_cast(profiling_msg.size())) + { + if (remaining_space > static_cast(profiling_msg.size())) + message << std::string(remaining_space - profiling_msg.size(), ' '); + message << "\033[2m" << profiling_msg << "\033[0m"; + } + } - message << profiling_msg; message << CLEAR_TO_END_OF_LINE; ++increment; diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index b775279f73b..aad4a8c18e5 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -60,13 +60,10 @@ public: void updateThreadEventData(HostToThreadTimesMap & new_thread_data, UInt64 elapsed_time); - bool print_hardware_utilization = false; - private: - size_t getUsedThreadsCount() const; - UInt64 getApproximateCoresNumber() const; + double getCPUUsage() const; struct MemoryUsage { @@ -93,7 +90,7 @@ private: bool write_progress_on_update = false; - std::unordered_map host_active_cores; + std::unordered_map host_cpu_usage; HostToThreadTimesMap thread_data; }; From d456c5ddda1fc64e8e444058205726ac0b72b6c1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 29 Dec 2021 01:26:58 +0300 Subject: [PATCH 0533/1260] Always display resource usage in client --- src/Common/ProgressIndication.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index f41d5f14c5a..b9a8bc923f7 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -182,18 +182,19 @@ void ProgressIndication::writeProgress() written_progress_chars = message.count() - prefix_size - (strlen(indicator) - 2); /// Don't count invisible output (escape sequences). + /// Display resource usage if possible. std::string profiling_msg; - // If approximate cores number is known, display it. double cpu_usage = getCPUUsage(); - if (cpu_usage > 0) + auto [memory_usage, max_host_usage] = getMemoryUsage(); + + if (cpu_usage > 0 || memory_usage > 0) { WriteBufferFromOwnString profiling_msg_builder; profiling_msg_builder << "(" << fmt::format("{:.1f}", cpu_usage) << " CPU"; - auto [memory_usage, max_host_usage] = getMemoryUsage(); - if (memory_usage != 0) + if (memory_usage > 0) profiling_msg_builder << ", " << formatReadableSizeWithDecimalSuffix(memory_usage) << " RAM"; if (max_host_usage < memory_usage) profiling_msg_builder << ", " << formatReadableSizeWithDecimalSuffix(max_host_usage) << " max/host"; From 422061ce719176ab66e30709a45e718ceb4627d5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 29 Dec 2021 01:43:43 +0300 Subject: [PATCH 0534/1260] Update tests --- tests/queries/0_stateless/02151_clickhouse_client_hints.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02151_clickhouse_client_hints.sh b/tests/queries/0_stateless/02151_clickhouse_client_hints.sh index 3e6c6cb16a5..7221acc2504 100755 --- a/tests/queries/0_stateless/02151_clickhouse_client_hints.sh +++ b/tests/queries/0_stateless/02151_clickhouse_client_hints.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --hardware_utilization 2>&1 | grep -q "Code: 552. DB::Exception: Unrecognized option '--hardware_utilization'. Maybe you meant \['--hardware-utilization'\]. (UNRECOGNIZED_ARGUMENTS)" && echo 'OK' || echo 'FAIL' ||: +$CLICKHOUSE_CLIENT --secuer 2>&1 | grep -q "Code: 552. DB::Exception: Unrecognized option '--secuer'. Maybe you meant \['--secure'\]. (UNRECOGNIZED_ARGUMENTS)" && echo 'OK' || echo 'FAIL' ||: From 8b46053e8a3fb3386b9e774e9e972cd532b7ae39 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 28 Dec 2021 19:50:55 -0400 Subject: [PATCH 0535/1260] fix test. Output is in another order --- tests/integration/test_http_handlers_config/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_http_handlers_config/test.py b/tests/integration/test_http_handlers_config/test.py index 1ea0f059b68..01872a1d0c3 100644 --- a/tests/integration/test_http_handlers_config/test.py +++ b/tests/integration/test_http_handlers_config/test.py @@ -58,7 +58,7 @@ def test_predefined_query_handler(): 'test_predefined_handler_get?max_threads=1&setting_name=max_threads', method='GET', headers={'XXX': 'xxx'}).content - assert b'max_threads\t1\nmax_final_threads\t1\n' == cluster.instance.http_request( + assert b'max_final_threads\t1\nmax_threads\t1\n' == cluster.instance.http_request( 'query_param_with_url/max_threads?max_threads=1&max_final_threads=1', headers={'XXX': 'max_final_threads'}).content From a4fe8e52ed799c0e2c4d8d8fc86ef9b72abbb520 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 28 Dec 2021 21:20:23 -0400 Subject: [PATCH 0536/1260] test for csv strings started with slach #26920 --- ...2155_csv_with_strings_with_slash.reference | 62 +++++++++++++++++++ .../02155_csv_with_strings_with_slash.sh | 24 +++++++ .../0_stateless/data_csv/csv_with_slash.csv | 30 +++++++++ tests/queries/0_stateless/data_csv/slash.csv | 0 4 files changed, 116 insertions(+) create mode 100644 tests/queries/0_stateless/02155_csv_with_strings_with_slash.reference create mode 100755 tests/queries/0_stateless/02155_csv_with_strings_with_slash.sh create mode 100644 tests/queries/0_stateless/data_csv/csv_with_slash.csv create mode 100644 tests/queries/0_stateless/data_csv/slash.csv diff --git a/tests/queries/0_stateless/02155_csv_with_strings_with_slash.reference b/tests/queries/0_stateless/02155_csv_with_strings_with_slash.reference new file mode 100644 index 00000000000..db750f36364 --- /dev/null +++ b/tests/queries/0_stateless/02155_csv_with_strings_with_slash.reference @@ -0,0 +1,62 @@ +input_format_null_as_default = 1 +0 \\asdf 2000-01-01 +1 x\\x\\ 2000-01-01 +2 x\\x 2000-01-01 +3 x\\ 2000-01-01 +4 x\\ 2000-01-01 +5 \\x 2000-01-01 +6 2000-01-01 +7 \\r\\n 2000-01-01 +8 \\\\r\\\\n 2000-01-01 +9 x\\\\ 2000-01-01 +10 \\asdf 2000-01-01 +11 x\\x\\ 2000-01-01 +12 x\\x 2000-01-01 +13 x\\ 2000-01-01 +14 x\\ 2000-01-01 +15 \\x 2000-01-01 +16 \\N 2000-01-01 +17 \\r\\n 2000-01-01 +18 \\\\r\\\\n 2000-01-01 +19 x\\\\ 2000-01-01 +20 \\asdf 2000-01-01 +21 x\\x\\ 2000-01-01 +22 x\\x 2000-01-01 +23 x\\ 2000-01-01 +24 x\\ 2000-01-01 +25 \\x 2000-01-01 +26 \\N 2000-01-01 +27 \\r\\n 2000-01-01 +28 \\\\r\\\\n 2000-01-01 +29 x\\\\ 2000-01-01 +input_format_null_as_default = 0 +0 \\asdf 2000-01-01 +1 x\\x\\ 2000-01-01 +2 x\\x 2000-01-01 +3 x\\ 2000-01-01 +4 x\\ 2000-01-01 +5 \\x 2000-01-01 +6 \\N 2000-01-01 +7 \\r\\n 2000-01-01 +8 \\\\r\\\\n 2000-01-01 +9 x\\\\ 2000-01-01 +10 \\asdf 2000-01-01 +11 x\\x\\ 2000-01-01 +12 x\\x 2000-01-01 +13 x\\ 2000-01-01 +14 x\\ 2000-01-01 +15 \\x 2000-01-01 +16 \\N 2000-01-01 +17 \\r\\n 2000-01-01 +18 \\\\r\\\\n 2000-01-01 +19 x\\\\ 2000-01-01 +20 \\asdf 2000-01-01 +21 x\\x\\ 2000-01-01 +22 x\\x 2000-01-01 +23 x\\ 2000-01-01 +24 x\\ 2000-01-01 +25 \\x 2000-01-01 +26 \\N 2000-01-01 +27 \\r\\n 2000-01-01 +28 \\\\r\\\\n 2000-01-01 +29 x\\\\ 2000-01-01 diff --git a/tests/queries/0_stateless/02155_csv_with_strings_with_slash.sh b/tests/queries/0_stateless/02155_csv_with_strings_with_slash.sh new file mode 100755 index 00000000000..ab2577e6138 --- /dev/null +++ b/tests/queries/0_stateless/02155_csv_with_strings_with_slash.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test_02155_csv" + +${CLICKHOUSE_CLIENT} --query="create table test_02155_csv (A Int64, S String, D Date) Engine=Memory;" + + +echo "input_format_null_as_default = 1" +cat $CUR_DIR/data_csv/csv_with_slash.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_02155_csv FORMAT CSV SETTINGS input_format_null_as_default = 1" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM test_02155_csv" + +${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE test_02155_csv" + +echo "input_format_null_as_default = 0" +cat $CUR_DIR/data_csv/csv_with_slash.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_02155_csv FORMAT CSV SETTINGS input_format_null_as_default = 0" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM test_02155_csv" + + +${CLICKHOUSE_CLIENT} --query="DROP TABLE test_02155_csv" + diff --git a/tests/queries/0_stateless/data_csv/csv_with_slash.csv b/tests/queries/0_stateless/data_csv/csv_with_slash.csv new file mode 100644 index 00000000000..0f2c166faa8 --- /dev/null +++ b/tests/queries/0_stateless/data_csv/csv_with_slash.csv @@ -0,0 +1,30 @@ +0,\asdf,2000-01-01 +1,x\x\,2000-01-01 +2,x\x,2000-01-01 +3,x\,2000-01-01 +4,x\,2000-01-01 +5,\x,2000-01-01 +6,\N,2000-01-01 +7,\r\n,2000-01-01 +8,\\r\\n,2000-01-01 +9,x\\,2000-01-01 +10,'\asdf',2000-01-01 +11,'x\x\',2000-01-01 +12,'x\x',2000-01-01 +13,'x\',2000-01-01 +14,'x\',2000-01-01 +15,'\x',2000-01-01 +16,'\N',2000-01-01 +17,'\r\n',2000-01-01 +18,"\\r\\n",2000-01-01 +19,"x\\",2000-01-01 +20,"\asdf",2000-01-01 +21,"x\x\",2000-01-01 +22,"x\x",2000-01-01 +23,"x\",2000-01-01 +24,"x\",2000-01-01 +25,"\x",2000-01-01 +26,"\N",2000-01-01 +27,"\r\n",2000-01-01 +28,"\\r\\n",2000-01-01 +29,"x\\",2000-01-01 diff --git a/tests/queries/0_stateless/data_csv/slash.csv b/tests/queries/0_stateless/data_csv/slash.csv new file mode 100644 index 00000000000..e69de29bb2d From 464b30d954fd0a5b56c636712b47ebac1db97825 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 28 Dec 2021 21:21:39 -0400 Subject: [PATCH 0537/1260] test for csv strings started with slach #26920 --- tests/queries/0_stateless/data_csv/slash.csv | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 tests/queries/0_stateless/data_csv/slash.csv diff --git a/tests/queries/0_stateless/data_csv/slash.csv b/tests/queries/0_stateless/data_csv/slash.csv deleted file mode 100644 index e69de29bb2d..00000000000 From 9004e98992459a1d49c6a385f48dc3ec650fc625 Mon Sep 17 00:00:00 2001 From: benbiti Date: Wed, 29 Dec 2021 13:43:54 +0800 Subject: [PATCH 0538/1260] parseDateTimeBestEffort support Unix Timestamp with Milliseconds --- src/IO/parseDateTimeBestEffort.cpp | 13 ++++++++++++- .../01281_parseDateTime64BestEffort.reference | 4 ++++ .../0_stateless/01281_parseDateTime64BestEffort.sql | 7 ++++++- .../01442_date_time_with_params.reference | 9 +++++++++ .../0_stateless/01442_date_time_with_params.sql | 10 +++++++++- 5 files changed, 40 insertions(+), 3 deletions(-) diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index 4f00ec5f96d..3b05d8c76b6 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -151,7 +151,18 @@ ReturnType parseDateTimeBestEffortImpl( { num_digits = readDigits(digits, sizeof(digits), in); - if (num_digits == 10 && !year && !has_time) + if (num_digits == 13 && !year && !has_time) + { + /// This is unix timestamp with millisecond. + readDecimalNumber<10>(res, digits); + if (fractional) + { + fractional->digits = 3; + readDecimalNumber<3>(fractional->value, digits + 10); + } + return ReturnType(true); + } + else if (num_digits == 10 && !year && !has_time) { /// This is unix timestamp. readDecimalNumber<10>(res, digits); diff --git a/tests/queries/0_stateless/01281_parseDateTime64BestEffort.reference b/tests/queries/0_stateless/01281_parseDateTime64BestEffort.reference index 5d2507d2a08..b76db01a8ab 100644 --- a/tests/queries/0_stateless/01281_parseDateTime64BestEffort.reference +++ b/tests/queries/0_stateless/01281_parseDateTime64BestEffort.reference @@ -13,3 +13,7 @@ Formats 2020-05-14 03:37:03.253 2020-05-14 03:37:03.000 2020-05-14 03:37:03.000 +Unix Timestamp with Milliseconds +2021-12-28 00:00:00.123 +2021-12-28 00:00:00.1 +2021-12-28 00:00:00.123000 diff --git a/tests/queries/0_stateless/01281_parseDateTime64BestEffort.sql b/tests/queries/0_stateless/01281_parseDateTime64BestEffort.sql index 5c0bbe1b4c2..bf52314bb62 100644 --- a/tests/queries/0_stateless/01281_parseDateTime64BestEffort.sql +++ b/tests/queries/0_stateless/01281_parseDateTime64BestEffort.sql @@ -30,4 +30,9 @@ SELECT parseDateTime64BestEffort('2020-05-14T03:37:03.253184Z', 3, 'Europe/Minsk SELECT 'Formats'; SELECT parseDateTime64BestEffort('2020-05-14T03:37:03.253184', 3, 'UTC'); SELECT parseDateTime64BestEffort('2020-05-14T03:37:03', 3, 'UTC'); -SELECT parseDateTime64BestEffort('2020-05-14 03:37:03', 3, 'UTC'); \ No newline at end of file +SELECT parseDateTime64BestEffort('2020-05-14 03:37:03', 3, 'UTC'); + +SELECT 'Unix Timestamp with Milliseconds' +SELECT parseDateTime64BestEffort('1640649600123', 3, 'UTC') +SELECT parseDateTime64BestEffort('1640649600123', 1, 'UTC') +SELECT parseDateTime64BestEffort('1640649600123', 6, 'UTC') diff --git a/tests/queries/0_stateless/01442_date_time_with_params.reference b/tests/queries/0_stateless/01442_date_time_with_params.reference index 19f78c83f82..378b37de4a7 100644 --- a/tests/queries/0_stateless/01442_date_time_with_params.reference +++ b/tests/queries/0_stateless/01442_date_time_with_params.reference @@ -13,6 +13,8 @@ parseDateTimeBestEffort 2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') 2020-05-14 06:37:03.253 DateTime64(3, \'Europe/Minsk\') 2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') +2021-12-28 00:00:00.123 DateTime64(3, \'UTC\') +2021-12-28 00:00:00 DateTime(\'UTC\') parseDateTimeBestEffortOrNull \N Nullable(DateTime64(3)) 2020-05-14 03:37:03.000 Nullable(DateTime64(3, \'UTC\')) @@ -25,6 +27,8 @@ parseDateTimeBestEffortOrNull 2020-05-14 03:37:03.253 Nullable(DateTime64(3, \'UTC\')) 2020-05-14 06:37:03.253 Nullable(DateTime64(3, \'Europe/Minsk\')) 2020-05-14 03:37:03.253 Nullable(DateTime64(3, \'UTC\')) +2021-12-28 00:00:00.123 Nullable(DateTime64(3, \'UTC\')) +2021-12-28 00:00:00 Nullable(DateTime(\'UTC\')) parseDateTimeBestEffortOrZero 1970-01-01 00:00:00.000 DateTime64(3, \'UTC\') 2020-05-14 03:37:03.000 DateTime64(3, \'UTC\') @@ -37,6 +41,8 @@ parseDateTimeBestEffortOrZero 2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') 2020-05-14 06:37:03.253 DateTime64(3, \'Europe/Minsk\') 2020-05-14 03:37:03.253 DateTime64(3, \'UTC\') +2021-12-28 00:00:00.123 DateTime64(3, \'UTC\') +2021-12-28 00:00:00 DateTime(\'UTC\') parseDateTime32BestEffort 2020-05-14 03:37:03 DateTime(\'UTC\') 2020-05-14 03:37:03 DateTime(\'UTC\') @@ -48,6 +54,7 @@ parseDateTime32BestEffort 2020-05-14 03:37:03 DateTime(\'UTC\') 2020-05-14 06:37:03 DateTime(\'Europe/Minsk\') 2020-05-14 03:37:03 DateTime(\'UTC\') +2021-12-28 00:00:00 DateTime(\'UTC\') parseDateTime32BestEffortOrNull \N Nullable(DateTime) 2020-05-14 03:37:03 Nullable(DateTime(\'UTC\')) @@ -60,6 +67,7 @@ parseDateTime32BestEffortOrNull 2020-05-14 03:37:03 Nullable(DateTime(\'UTC\')) 2020-05-14 06:37:03 Nullable(DateTime(\'Europe/Minsk\')) 2020-05-14 03:37:03 Nullable(DateTime(\'UTC\')) +2021-12-28 00:00:00 Nullable(DateTime(\'UTC\')) parseDateTime32BestEffortOrZero 1970-01-01 00:00:00 DateTime(\'UTC\') 2020-05-14 03:37:03 DateTime(\'UTC\') @@ -72,3 +80,4 @@ parseDateTime32BestEffortOrZero 2020-05-14 03:37:03 DateTime(\'UTC\') 2020-05-14 06:37:03 DateTime(\'Europe/Minsk\') 2020-05-14 03:37:03 DateTime(\'UTC\') +2021-12-28 00:00:00 DateTime(\'UTC\') \ No newline at end of file diff --git a/tests/queries/0_stateless/01442_date_time_with_params.sql b/tests/queries/0_stateless/01442_date_time_with_params.sql index 52815460245..5a57aabdb0c 100644 --- a/tests/queries/0_stateless/01442_date_time_with_params.sql +++ b/tests/queries/0_stateless/01442_date_time_with_params.sql @@ -24,6 +24,8 @@ SELECT parseDateTimeBestEffort('2020-05-14T03:37:03.253184', 3, 'UTC') AS a, toT SELECT parseDateTimeBestEffort('2020-05-14T03:37:03.253184Z', 3, 'UTC') AS a, toTypeName(a); SELECT parseDateTimeBestEffort('2020-05-14T03:37:03.253184Z', 3, 'Europe/Minsk') AS a, toTypeName(a); SELECT parseDateTimeBestEffort(materialize('2020-05-14T03:37:03.253184Z'), 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffort('1640649600123', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffort('1640649600123', 'UTC') AS a, toTypeName(a); SELECT 'parseDateTimeBestEffortOrNull'; SELECT parseDateTimeBestEffortOrNull('', 3) AS a, toTypeName(a); @@ -37,6 +39,8 @@ SELECT parseDateTimeBestEffortOrNull('2020-05-14T03:37:03.253184', 3, 'UTC') AS SELECT parseDateTimeBestEffortOrNull('2020-05-14T03:37:03.253184Z', 3, 'UTC') AS a, toTypeName(a); SELECT parseDateTimeBestEffortOrNull('2020-05-14T03:37:03.253184Z', 3, 'Europe/Minsk') AS a, toTypeName(a); SELECT parseDateTimeBestEffortOrNull(materialize('2020-05-14T03:37:03.253184Z'), 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrNull('1640649600123', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrNull('1640649600123', 'UTC') AS a, toTypeName(a); SELECT 'parseDateTimeBestEffortOrZero'; SELECT parseDateTimeBestEffortOrZero('', 3, 'UTC') AS a, toTypeName(a); @@ -50,6 +54,8 @@ SELECT parseDateTimeBestEffortOrZero('2020-05-14T03:37:03.253184', 3, 'UTC') AS SELECT parseDateTimeBestEffortOrZero('2020-05-14T03:37:03.253184Z', 3, 'UTC') AS a, toTypeName(a); SELECT parseDateTimeBestEffortOrZero('2020-05-14T03:37:03.253184Z', 3, 'Europe/Minsk') AS a, toTypeName(a); SELECT parseDateTimeBestEffortOrZero(materialize('2020-05-14T03:37:03.253184Z'), 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrZero('1640649600123', 3, 'UTC') AS a, toTypeName(a); +SELECT parseDateTimeBestEffortOrZero('1640649600123', 'UTC') AS a, toTypeName(a); SELECT 'parseDateTime32BestEffort'; SELECT parseDateTime32BestEffort('') AS a, toTypeName(a); -- {serverError 41} @@ -63,6 +69,7 @@ SELECT parseDateTime32BestEffort('2020-05-14T03:37:03.253184', 'UTC') AS a, toTy SELECT parseDateTime32BestEffort('2020-05-14T03:37:03.253184Z', 'UTC') AS a, toTypeName(a); SELECT parseDateTime32BestEffort('2020-05-14T03:37:03.253184Z', 'Europe/Minsk') AS a, toTypeName(a); SELECT parseDateTime32BestEffort(materialize('2020-05-14T03:37:03.253184Z'), 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffort('1640649600123', 'UTC') AS a, toTypeName(a); SELECT 'parseDateTime32BestEffortOrNull'; SELECT parseDateTime32BestEffortOrNull('') AS a, toTypeName(a); @@ -76,6 +83,7 @@ SELECT parseDateTime32BestEffortOrNull('2020-05-14T03:37:03.253184', 'UTC') AS a SELECT parseDateTime32BestEffortOrNull('2020-05-14T03:37:03.253184Z', 'UTC') AS a, toTypeName(a); SELECT parseDateTime32BestEffortOrNull('2020-05-14T03:37:03.253184Z', 'Europe/Minsk') AS a, toTypeName(a); SELECT parseDateTime32BestEffortOrNull(materialize('2020-05-14T03:37:03.253184Z'), 'UTC') AS a, toTypeName(a); +SELECT parseDateTime32BestEffortOrNull('1640649600123', 'UTC') AS a, toTypeName(a); SELECT 'parseDateTime32BestEffortOrZero'; SELECT parseDateTime32BestEffortOrZero('', 'UTC') AS a, toTypeName(a); @@ -89,6 +97,6 @@ SELECT parseDateTime32BestEffortOrZero('2020-05-14T03:37:03.253184', 'UTC') AS a SELECT parseDateTime32BestEffortOrZero('2020-05-14T03:37:03.253184Z', 'UTC') AS a, toTypeName(a); SELECT parseDateTime32BestEffortOrZero('2020-05-14T03:37:03.253184Z', 'Europe/Minsk') AS a, toTypeName(a); SELECT parseDateTime32BestEffortOrZero(materialize('2020-05-14T03:37:03.253184Z'), 'UTC') AS a, toTypeName(a); - +SELECT parseDateTime32BestEffortOrZero('1640649600123', 'UTC') AS a, toTypeName(a); DROP TABLE IF EXISTS test; From a23bb79479b88c3576ef010bc035242e387bd353 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 28 Dec 2021 21:18:55 +0300 Subject: [PATCH 0539/1260] Fix parsing symbols from resources (for shared builds) In case of shared builds the symbol can be represented in multiple shared objects, and so if one SO contains _end symbol firstly, then it it's address can subtracted from the symbol from another SO, and this give overflow. Here is an example of such symbol: $ llvm-readelf -a programs/server/libclickhouse-server-libd.so | fgrep schemata_sql 552: 000000000010b869 0 NOTYPE GLOBAL DEFAULT 22 _binary_schemata_sql_size 1271: 000000000010b869 0 NOTYPE GLOBAL DEFAULT 22 _binary_schemata_sql_end 3627: 000000000010b480 0 NOTYPE GLOBAL DEFAULT 22 _binary_schemata_sql_start 5822: 000000000010b869 0 NOTYPE GLOBAL DEFAULT 22 _binary_schemata_sql_end 5828: 000000000010b869 0 NOTYPE GLOBAL DEFAULT 22 _binary_schemata_sql_size 5831: 000000000010b480 0 NOTYPE GLOBAL DEFAULT 22 _binary_schemata_sql_start $ llvm-readelf -W -a src/TableFunctions/libclickhouse_table_functionsd.* | fgrep schemata_sql 1709: 00000000001fc5a1 0 NOTYPE GLOBAL DEFAULT 22 _binary_schemata_sql_end 5107: 00000000001fc5a1 0 NOTYPE GLOBAL DEFAULT 22 _binary_schemata_sql_size 6090: 00000000001fc1b8 0 NOTYPE GLOBAL DEFAULT 22 _binary_schemata_sql_start 10975: 00000000001fc1b8 0 NOTYPE GLOBAL DEFAULT 22 _binary_schemata_sql_start 10976: 00000000001fc5a1 0 NOTYPE GLOBAL DEFAULT 22 _binary_schemata_sql_end 10977: 00000000001fc5a1 0 NOTYPE GLOBAL DEFAULT 22 _binary_schemata_sql_size To address this, add base address of the Elf into the resources to compare with. Follow-up for: #33134 (cc @alexey-milovidov) --- src/Common/SymbolIndex.cpp | 33 ++++++++++++++++++++++----------- src/Common/SymbolIndex.h | 14 ++++++++++++-- 2 files changed, 34 insertions(+), 13 deletions(-) diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index 568f633975b..32c1a15337c 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -86,7 +86,7 @@ namespace /// https://stackoverflow.com/questions/32088140/multiple-string-tables-in-elf-object -void updateResources(std::string_view name, const void * address, SymbolIndex::Resources & resources) +void updateResources(ElfW(Addr) base_address, std::string_view object_name, std::string_view name, const void * address, SymbolIndex::Resources & resources) { const char * char_address = static_cast(address); @@ -97,18 +97,23 @@ void updateResources(std::string_view name, const void * address, SymbolIndex::R name = name.substr((name[0] == '_') + strlen("binary_")); name = name.substr(0, name.size() - strlen("_start")); - resources.emplace(name, std::string_view{char_address, 0}); // NOLINT + resources.emplace(name, SymbolIndex::ResourcesBlob{ + base_address, + object_name, + std::string_view{char_address, 0}, // NOLINT + }); } else if (name.ends_with("_end")) { name = name.substr((name[0] == '_') + strlen("binary_")); name = name.substr(0, name.size() - strlen("_end")); - if (auto it = resources.find(name); it != resources.end() && it->second.empty()) + auto it = resources.find(name); + if (it != resources.end() && it->second.base_address == base_address && it->second.data.empty()) { - const char * start = it->second.data(); + const char * start = it->second.data.data(); assert(char_address >= start); - it->second = std::string_view{start, static_cast(char_address - start)}; + it->second.data = std::string_view{start, static_cast(char_address - start)}; } } } @@ -153,10 +158,12 @@ void collectSymbolsFromProgramHeaders( size_t sym_cnt = 0; for (const auto * it = dyn_begin; it->d_tag != DT_NULL; ++it) { + ElfW(Addr) base_address = correct_address(info->dlpi_addr, it->d_un.d_ptr); + // TODO: this branch leads to invalid address of the hash table. Need further investigation. // if (it->d_tag == DT_HASH) // { - // const ElfW(Word) * hash = reinterpret_cast(correct_address(info->dlpi_addr, it->d_un.d_ptr)); + // const ElfW(Word) * hash = reinterpret_cast(base_address); // sym_cnt = hash[1]; // break; // } @@ -167,7 +174,7 @@ void collectSymbolsFromProgramHeaders( const uint32_t * buckets = nullptr; const uint32_t * hashval = nullptr; - const ElfW(Word) * hash = reinterpret_cast(correct_address(info->dlpi_addr, it->d_un.d_ptr)); + const ElfW(Word) * hash = reinterpret_cast(base_address); buckets = hash + 4 + (hash[2] * sizeof(size_t) / 4); @@ -196,9 +203,11 @@ void collectSymbolsFromProgramHeaders( const char * strtab = nullptr; for (const auto * it = dyn_begin; it->d_tag != DT_NULL; ++it) { + ElfW(Addr) base_address = correct_address(info->dlpi_addr, it->d_un.d_ptr); + if (it->d_tag == DT_STRTAB) { - strtab = reinterpret_cast(correct_address(info->dlpi_addr, it->d_un.d_ptr)); + strtab = reinterpret_cast(base_address); break; } } @@ -208,10 +217,12 @@ void collectSymbolsFromProgramHeaders( for (const auto * it = dyn_begin; it->d_tag != DT_NULL; ++it) { + ElfW(Addr) base_address = correct_address(info->dlpi_addr, it->d_un.d_ptr); + if (it->d_tag == DT_SYMTAB) { /* Get the pointer to the first entry of the symbol table */ - const ElfW(Sym) * elf_sym = reinterpret_cast(correct_address(info->dlpi_addr, it->d_un.d_ptr)); + const ElfW(Sym) * elf_sym = reinterpret_cast(base_address); /* Iterate over the symbol table */ for (ElfW(Word) sym_index = 0; sym_index < ElfW(Word)(sym_cnt); ++sym_index) @@ -236,7 +247,7 @@ void collectSymbolsFromProgramHeaders( symbols.push_back(symbol); /// But resources can be represented by a pair of empty symbols (indicating their boundaries). - updateResources(symbol.name, symbol.address_begin, resources); + updateResources(base_address, info->dlpi_name, symbol.name, symbol.address_begin, resources); } break; @@ -299,7 +310,7 @@ void collectSymbolsFromELFSymbolTable( if (symbol_table_entry->st_size) symbols.push_back(symbol); - updateResources(symbol.name, symbol.address_begin, resources); + updateResources(info->dlpi_addr, info->dlpi_name, symbol.name, symbol.address_begin, resources); } } diff --git a/src/Common/SymbolIndex.h b/src/Common/SymbolIndex.h index 7c542980099..1331cf81cf7 100644 --- a/src/Common/SymbolIndex.h +++ b/src/Common/SymbolIndex.h @@ -51,7 +51,7 @@ public: std::string_view getResource(String name) const { if (auto it = data.resources.find(name); it != data.resources.end()) - return it->second; + return it->second.data; return {}; } @@ -59,7 +59,17 @@ public: String getBuildID() const { return data.build_id; } String getBuildIDHex() const; - using Resources = std::unordered_map; + struct ResourcesBlob + { + /// Symbol can be presented in multiple shared objects, + /// base_address will be used to compare only symbols from the same SO. + ElfW(Addr) base_address; + /// Just a human name of the SO. + std::string_view object_name; + /// Data blob. + std::string_view data; + }; + using Resources = std::unordered_map; struct Data { From a424b4ad5637677d6525db50a6d6885845d89ce6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 28 Dec 2021 21:16:04 +0300 Subject: [PATCH 0540/1260] Do not ignore eof in expect tests expect_after that adjusts default timeout handler, reseted eof handler, and this tell expect that it can ignore eof from read, consider the following example:
```expect #!/usr/bin/expect -f exp_internal 1 log_user 1 set timeout 4 match_max 100000 expect_after { # eof { exp_continue } timeout { exit 1 } } spawn bash -c "sleep 1; echo ':) '" expect ":) " ``` ``` $ ./expect.expect < /dev/null spawn bash -c sleep 1; echo ':) ' parent: waiting for sync byte parent: telling child to go ahead parent: now unsynchronized from child spawn: returns {6614} expect: does "" (spawn_id exp4) match glob pattern ":) "? no expect: read eof ^^^ expect: set expect_out(spawn_id) "exp0" expect: set expect_out(buffer) "" ``` And with uncommented eof handler: ``` $ ./expect.expect < /dev/null spawn bash -c sleep 1; echo ':) ' parent: waiting for sync byte parent: telling child to go ahead parent: now unsynchronized from child spawn: returns {17959} expect: does "" (spawn_id exp4) match glob pattern ":) "? no expect: read eof expect: set expect_out(spawn_id) "exp0" expect: set expect_out(buffer) "" expect: continuing expect after update expect: does "" (spawn_id exp4) match glob pattern ":) "? no expect: does ":) \r\n" (spawn_id exp4) match glob pattern ":) "? yes expect: set expect_out(0,string) ":) " expect: set expect_out(spawn_id) "exp4" expect: set expect_out(buffer) ":) " ```
--- .../01176_mysql_client_interactive.expect | 9 +++++---- .../01179_insert_values_semicolon.expect | 8 ++++---- .../01180_client_syntax_errors.expect | 8 ++++---- ..._client_interactive_vertical_multiline.expect | 8 ++++---- ...client_interactive_vertical_singleline.expect | 8 ++++---- ...ient_save_history_when_terminated_long.expect | 8 ++++---- ...ent_autocomplete_word_break_characters.expect | 8 ++++---- .../01520_client_print_query_id.expect | 8 ++++---- .../01565_reconnect_after_client_error.expect | 8 ++++---- .../01676_long_clickhouse_client_autocomplete.sh | 16 ++++++++-------- ...ighlight_multi_line_comment_regression.expect | 8 ++++---- ..._client_replxx_container_overflow_long.expect | 8 ++++---- .../01933_client_replxx_convert_history.expect | 8 ++++---- .../0_stateless/01945_show_debug_warning.expect | 8 ++++---- .../02003_memory_limit_in_client.expect | 8 ++++---- .../0_stateless/02047_client_exception.expect | 8 ++++---- .../02049_clickhouse_local_merge_tree.expect | 9 ++++----- .../02105_backslash_letter_commands.expect | 8 ++++---- ...ed_clickhouse_client_with_queries_file.expect | 9 ++++----- .../02112_delayed_clickhouse_local.expect | 8 ++++---- ...yed_clickhouse_local_with_queries_file.expect | 9 ++++----- .../0_stateless/02116_interactive_hello.expect | 8 ++++---- .../02132_client_history_navigation.expect | 9 +++++---- 23 files changed, 98 insertions(+), 99 deletions(-) diff --git a/tests/queries/0_stateless/01176_mysql_client_interactive.expect b/tests/queries/0_stateless/01176_mysql_client_interactive.expect index 37087dd85f7..5bbc77ccf14 100755 --- a/tests/queries/0_stateless/01176_mysql_client_interactive.expect +++ b/tests/queries/0_stateless/01176_mysql_client_interactive.expect @@ -5,11 +5,12 @@ log_user 0 set timeout 60 match_max 100000 -# A default timeout action is to do nothing, change it to fail + expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/01179_insert_values_semicolon.expect b/tests/queries/0_stateless/01179_insert_values_semicolon.expect index 0e65e5c4cbf..b8fab5be971 100755 --- a/tests/queries/0_stateless/01179_insert_values_semicolon.expect +++ b/tests/queries/0_stateless/01179_insert_values_semicolon.expect @@ -3,11 +3,11 @@ log_user 0 set timeout 60 match_max 100000 -# A default timeout action is to do nothing, change it to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/01180_client_syntax_errors.expect b/tests/queries/0_stateless/01180_client_syntax_errors.expect index c20982b2991..6e4e975988e 100755 --- a/tests/queries/0_stateless/01180_client_syntax_errors.expect +++ b/tests/queries/0_stateless/01180_client_syntax_errors.expect @@ -3,11 +3,11 @@ log_user 0 set timeout 60 match_max 100000 -# A default timeout action is to do nothing, change it to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect b/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect index 5e845754402..e4442047c87 100755 --- a/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect +++ b/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect @@ -4,11 +4,11 @@ log_user 0 set timeout 60 match_max 100000 -# A default timeout action is to do nothing, change it to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect b/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect index c68b153d5d3..2f871ab46d8 100755 --- a/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect +++ b/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect @@ -3,11 +3,11 @@ log_user 0 set timeout 60 match_max 100000 -# A default timeout action is to do nothing, change it to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect b/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect index 05d9d408228..ad5b7625929 100755 --- a/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect +++ b/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect @@ -4,11 +4,11 @@ log_user 0 set timeout 60 match_max 100000 -# A default timeout action is to do nothing, change it to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect b/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect index f3a28bbee9b..9c20b7c517e 100755 --- a/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect +++ b/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect @@ -3,11 +3,11 @@ log_user 0 set timeout 60 match_max 100000 -# A default timeout action is to do nothing, change it to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/01520_client_print_query_id.expect b/tests/queries/0_stateless/01520_client_print_query_id.expect index b0ff5d9d165..8b6e0e17a85 100755 --- a/tests/queries/0_stateless/01520_client_print_query_id.expect +++ b/tests/queries/0_stateless/01520_client_print_query_id.expect @@ -3,11 +3,11 @@ log_user 0 set timeout 60 match_max 100000 -# A default timeout action is to do nothing, change it to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/01565_reconnect_after_client_error.expect b/tests/queries/0_stateless/01565_reconnect_after_client_error.expect index 712fe4ff64a..71db4b1d452 100755 --- a/tests/queries/0_stateless/01565_reconnect_after_client_error.expect +++ b/tests/queries/0_stateless/01565_reconnect_after_client_error.expect @@ -7,11 +7,11 @@ log_user 0 set timeout 60 match_max 100000 -# A default timeout action is to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh b/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh index e029d90a686..1be082a6aae 100755 --- a/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh +++ b/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh @@ -20,11 +20,11 @@ function test_completion_word_client() log_user 0 set timeout 3 match_max 100000 -# A default timeout action is to do nothing, change it to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } spawn bash -c "$CLICKHOUSE_CLIENT_BINARY $CLICKHOUSE_CLIENT_OPT" @@ -104,11 +104,11 @@ function test_completion_word_local() log_user 0 set timeout 3 match_max 100000 -# A default timeout action is to do nothing, change it to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } spawn bash -c "$CLICKHOUSE_LOCAL" diff --git a/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect b/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect index 5543af4dd05..022320e2d4b 100755 --- a/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect +++ b/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect @@ -3,11 +3,11 @@ log_user 0 set timeout 60 match_max 100000 -# A default timeout action is to do nothing, change it to fail expect_after { - timeout { - exit 2 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/01910_client_replxx_container_overflow_long.expect b/tests/queries/0_stateless/01910_client_replxx_container_overflow_long.expect index 138727d296d..d5ce4c3cbf2 100755 --- a/tests/queries/0_stateless/01910_client_replxx_container_overflow_long.expect +++ b/tests/queries/0_stateless/01910_client_replxx_container_overflow_long.expect @@ -4,11 +4,11 @@ log_user 0 set timeout 60 match_max 100000 -# A default timeout action is to do nothing, change it to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/01933_client_replxx_convert_history.expect b/tests/queries/0_stateless/01933_client_replxx_convert_history.expect index 59231161d91..c5645179ab3 100755 --- a/tests/queries/0_stateless/01933_client_replxx_convert_history.expect +++ b/tests/queries/0_stateless/01933_client_replxx_convert_history.expect @@ -5,11 +5,11 @@ log_user 0 set timeout 60 match_max 100000 -# A default timeout action is to do nothing, change it to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 402ad9a1f35..2f74b6e33ae 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -7,11 +7,11 @@ log_user 0 set timeout 60 match_max 100000 -# A default timeout action is to do nothing, change it to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.expect b/tests/queries/0_stateless/02003_memory_limit_in_client.expect index 47ac4926537..29701f49746 100755 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.expect +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.expect @@ -8,11 +8,11 @@ log_user 0 set timeout 60 match_max 100000 -# A default timeout action is to do nothing, change it to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/02047_client_exception.expect b/tests/queries/0_stateless/02047_client_exception.expect index 57a38c4f6aa..f7d4bfb555d 100755 --- a/tests/queries/0_stateless/02047_client_exception.expect +++ b/tests/queries/0_stateless/02047_client_exception.expect @@ -4,11 +4,11 @@ log_user 0 set timeout 20 match_max 100000 -# A default timeout action is to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect b/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect index 89271805fb3..ffa25b964db 100755 --- a/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect +++ b/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect @@ -4,12 +4,11 @@ log_user 0 set timeout 20 match_max 100000 -# A default timeout action is to fail expect_after { - timeout { - exit 1 - } - + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/02105_backslash_letter_commands.expect b/tests/queries/0_stateless/02105_backslash_letter_commands.expect index 89d896fdedc..e67d60912fa 100755 --- a/tests/queries/0_stateless/02105_backslash_letter_commands.expect +++ b/tests/queries/0_stateless/02105_backslash_letter_commands.expect @@ -3,11 +3,11 @@ log_user 0 set timeout 02 match_max 100000 -# A default timeout action is to do nothing, change it to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/02112_delayed_clickhouse_client_with_queries_file.expect b/tests/queries/0_stateless/02112_delayed_clickhouse_client_with_queries_file.expect index 73b12637906..08d3b295632 100755 --- a/tests/queries/0_stateless/02112_delayed_clickhouse_client_with_queries_file.expect +++ b/tests/queries/0_stateless/02112_delayed_clickhouse_client_with_queries_file.expect @@ -5,14 +5,13 @@ log_user 0 set timeout 20 match_max 100000 -# A default timeout action is to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } - spawn bash -c "\$CLICKHOUSE_TESTS_DIR/helpers/02112_prepare.sh" set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/02112_delayed_clickhouse_local.expect b/tests/queries/0_stateless/02112_delayed_clickhouse_local.expect index fa146577234..c846464b011 100755 --- a/tests/queries/0_stateless/02112_delayed_clickhouse_local.expect +++ b/tests/queries/0_stateless/02112_delayed_clickhouse_local.expect @@ -4,11 +4,11 @@ log_user 0 set timeout 20 match_max 100000 -# A default timeout action is to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/02112_delayed_clickhouse_local_with_queries_file.expect b/tests/queries/0_stateless/02112_delayed_clickhouse_local_with_queries_file.expect index fbf79629f71..75ad5abe62d 100755 --- a/tests/queries/0_stateless/02112_delayed_clickhouse_local_with_queries_file.expect +++ b/tests/queries/0_stateless/02112_delayed_clickhouse_local_with_queries_file.expect @@ -5,14 +5,13 @@ log_user 0 set timeout 20 match_max 100000 -# A default timeout action is to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } - spawn bash -c "\$CLICKHOUSE_TESTS_DIR/helpers/02112_prepare.sh" set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/02116_interactive_hello.expect b/tests/queries/0_stateless/02116_interactive_hello.expect index 49a167e5a6e..84144ec25b2 100755 --- a/tests/queries/0_stateless/02116_interactive_hello.expect +++ b/tests/queries/0_stateless/02116_interactive_hello.expect @@ -4,11 +4,11 @@ log_user 0 set timeout 60 match_max 100000 -# A default timeout action is to do nothing, change it to fail expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } set basedir [file dirname $argv0] diff --git a/tests/queries/0_stateless/02132_client_history_navigation.expect b/tests/queries/0_stateless/02132_client_history_navigation.expect index cd83454c85e..b722a0af04c 100755 --- a/tests/queries/0_stateless/02132_client_history_navigation.expect +++ b/tests/queries/0_stateless/02132_client_history_navigation.expect @@ -3,11 +3,12 @@ log_user 0 set timeout 3 match_max 100000 -# A default timeout action is to do nothing, change it to fail + expect_after { - timeout { - exit 1 - } + # Do not ignore eof from expect + eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + timeout { exit 1 } } # useful debugging configuration From 06ac2cd6cdbeca9ce55b6275efedce06d91303ee Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 28 Dec 2021 22:59:42 +0300 Subject: [PATCH 0541/1260] tests: fix 02112_delayed_clickhouse_*_with_queries_file - fix variables inside prepare.sh - fix variables for invoking prepare.sh - fix path to queries file - wait for eof on client (exiting client is required to catch all possible issues, that can be reported by i.e. ASan on exit) - fix spawning of clean.sh by replace with system Cc: @kssenii --- ..._delayed_clickhouse_client_with_queries_file.expect | 10 ++++++---- ...2_delayed_clickhouse_local_with_queries_file.expect | 10 ++++++---- tests/queries/0_stateless/helpers/02112_clean.sh | 7 +++---- tests/queries/0_stateless/helpers/02112_prepare.sh | 8 +++----- 4 files changed, 18 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/02112_delayed_clickhouse_client_with_queries_file.expect b/tests/queries/0_stateless/02112_delayed_clickhouse_client_with_queries_file.expect index 08d3b295632..0abe25e60f4 100755 --- a/tests/queries/0_stateless/02112_delayed_clickhouse_client_with_queries_file.expect +++ b/tests/queries/0_stateless/02112_delayed_clickhouse_client_with_queries_file.expect @@ -12,15 +12,17 @@ expect_after { timeout { exit 1 } } -spawn bash -c "\$CLICKHOUSE_TESTS_DIR/helpers/02112_prepare.sh" - set basedir [file dirname $argv0] -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT --disable_suggestion --interactive --queries-file \$CURDIR/file_02112" + +system "$basedir/helpers/02112_prepare.sh" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT --disable_suggestion --interactive --queries-file $basedir/file_02112" expect ":) " send -- "select * from t format TSV\r" expect "1" expect ":) " -spawn bash -c "\$CLICKHOUSE_TESTS_DIR/helpers/02112_clean.sh" +send "" +expect eof +system "$basedir/helpers/02112_clean.sh" diff --git a/tests/queries/0_stateless/02112_delayed_clickhouse_local_with_queries_file.expect b/tests/queries/0_stateless/02112_delayed_clickhouse_local_with_queries_file.expect index 75ad5abe62d..c64f149a93c 100755 --- a/tests/queries/0_stateless/02112_delayed_clickhouse_local_with_queries_file.expect +++ b/tests/queries/0_stateless/02112_delayed_clickhouse_local_with_queries_file.expect @@ -12,15 +12,17 @@ expect_after { timeout { exit 1 } } -spawn bash -c "\$CLICKHOUSE_TESTS_DIR/helpers/02112_prepare.sh" - set basedir [file dirname $argv0] -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_LOCAL --disable_suggestion --interactive --queries-file \$CURDIR/file_02112" + +system "$basedir/helpers/02112_prepare.sh" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_LOCAL --disable_suggestion --interactive --queries-file $basedir/file_02112" expect ":) " send -- "select * from t format TSV\r" expect "1" expect ":) " -spawn bash -c "\$CLICKHOUSE_TESTS_DIR/helpers/02112_clean.sh" +send "" +expect eof +system "$basedir/helpers/02112_clean.sh" diff --git a/tests/queries/0_stateless/helpers/02112_clean.sh b/tests/queries/0_stateless/helpers/02112_clean.sh index 910c0709955..95af0cede9c 100755 --- a/tests/queries/0_stateless/helpers/02112_clean.sh +++ b/tests/queries/0_stateless/helpers/02112_clean.sh @@ -1,6 +1,5 @@ #!/usr/bin/env bash -FILE=${CURDIR}/file_02112 -if [ -f $FILE ]; then - rm $FILE -fi +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +FILE=${CURDIR}/../file_02112 +rm "$FILE" diff --git a/tests/queries/0_stateless/helpers/02112_prepare.sh b/tests/queries/0_stateless/helpers/02112_prepare.sh index 1f371789f86..c2791b01140 100755 --- a/tests/queries/0_stateless/helpers/02112_prepare.sh +++ b/tests/queries/0_stateless/helpers/02112_prepare.sh @@ -1,7 +1,5 @@ #!/usr/bin/env bash -FILE=${CURDIR}/file_02112 -if [ -f $FILE ]; then - rm $FILE -fi -echo "drop table if exists t;create table t(i Int32) engine=Memory; insert into t select 1" >> $FILE +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +FILE=${CURDIR}/../file_02112 +echo "drop table if exists t;create table t(i Int32) engine=Memory; insert into t select 1" > "$FILE" From c541ee386b39ff2038ec8143ef3bb580b948ae78 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 29 Dec 2021 10:02:12 +0300 Subject: [PATCH 0542/1260] Mark some expect tests as long CI: https://s3.amazonaws.com/clickhouse-test-reports/33263/765c7980b84ef6a7851a5fe2846a06f08382f03a/stateless_tests_flaky_check__address__actions_.html --- tests/queries/0_stateless/01179_insert_values_semicolon.expect | 1 + .../0_stateless/01565_reconnect_after_client_error.expect | 1 + tests/queries/0_stateless/02116_interactive_hello.expect | 1 + 3 files changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/01179_insert_values_semicolon.expect b/tests/queries/0_stateless/01179_insert_values_semicolon.expect index b8fab5be971..bf937c3a6a4 100755 --- a/tests/queries/0_stateless/01179_insert_values_semicolon.expect +++ b/tests/queries/0_stateless/01179_insert_values_semicolon.expect @@ -1,4 +1,5 @@ #!/usr/bin/expect -f +# Tags: long log_user 0 set timeout 60 diff --git a/tests/queries/0_stateless/01565_reconnect_after_client_error.expect b/tests/queries/0_stateless/01565_reconnect_after_client_error.expect index 71db4b1d452..819450ffd30 100755 --- a/tests/queries/0_stateless/01565_reconnect_after_client_error.expect +++ b/tests/queries/0_stateless/01565_reconnect_after_client_error.expect @@ -1,4 +1,5 @@ #!/usr/bin/expect -f +# Tags: long # This is a separate test, because we want to test the interactive mode. # https://github.com/ClickHouse/ClickHouse/issues/19353 diff --git a/tests/queries/0_stateless/02116_interactive_hello.expect b/tests/queries/0_stateless/02116_interactive_hello.expect index 84144ec25b2..e659cf8703c 100755 --- a/tests/queries/0_stateless/02116_interactive_hello.expect +++ b/tests/queries/0_stateless/02116_interactive_hello.expect @@ -1,4 +1,5 @@ #!/usr/bin/expect -f +# Tags: long log_user 0 set timeout 60 From 59cbd768800614227fa478e1e09c3bb092d69fee Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 29 Dec 2021 15:25:33 +0800 Subject: [PATCH 0543/1260] Add LRUResourceCache 1. add LRUResourceCache for managing resource cache in lru policy 2. rollback LRUCache to the original version 3. add remove() in LRUCache 4. add unit tests for LRUResourceCache and LRUCache --- src/Common/LRUCache.h | 131 +++------ src/Common/LRUResourceCache.h | 254 ++++++++++++++++++ src/Common/tests/gtest_lru_cache.cpp | 102 +++++++ src/Common/tests/gtest_lru_resource_cache.cpp | 212 +++++++++++++++ src/IO/MMappedFileCache.h | 4 +- src/IO/UncompressedCache.h | 4 +- src/Interpreters/Aggregator.cpp | 3 +- src/Interpreters/ExpressionJIT.cpp | 3 +- src/Interpreters/MergeJoin.cpp | 2 +- .../Formats/Impl/AvroRowInputFormat.cpp | 4 +- src/Storages/MarkCache.h | 4 +- 11 files changed, 610 insertions(+), 113 deletions(-) create mode 100644 src/Common/LRUResourceCache.h create mode 100644 src/Common/tests/gtest_lru_cache.cpp create mode 100644 src/Common/tests/gtest_lru_resource_cache.cpp diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 97d70b5598c..1058003a327 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -6,11 +6,13 @@ #include #include #include + #include namespace DB { + template struct TrivialWeightFunction { @@ -20,30 +22,13 @@ struct TrivialWeightFunction } }; -template -struct TrivialLRUCacheEvictPolicy -{ - inline bool canRelease(std::shared_ptr) const - { - return true; - } - - inline void release(std::shared_ptr) - { - } -}; - /// Thread-safe cache that evicts entries which are not used for a long time. /// WeightFunction is a functor that takes Mapped as a parameter and returns "weight" (approximate size) /// of that value. /// Cache starts to evict entries when their total weight exceeds max_size. /// Value weight should not change after insertion. -template , - typename WeightFunction = TrivialWeightFunction, - typename EvictPolicy = TrivialLRUCacheEvictPolicy> +template , typename WeightFunction = TrivialWeightFunction> class LRUCache { public: @@ -51,16 +36,6 @@ public: using Mapped = TMapped; using MappedPtr = std::shared_ptr; - struct Result - { - MappedPtr value; - // if key is in cache, cache_miss is true - bool cache_miss = true; - // set_successful is false in default - // when value is loaded by load_fun in getOrSet(), and setImpl returns true, set_successful = true - bool set_successful = false; - }; - /** Initialize LRUCache with max_size and max_elements_size. * max_elements_size == 0 means no elements size restrictions. */ @@ -85,18 +60,19 @@ public: void set(const Key & key, const MappedPtr & mapped) { std::lock_guard lock(mutex); + setImpl(key, mapped, lock); } - /** - * trySet() will fail (return false) if there is no space left and no keys could be evicted. - * Eviction permission of each key is defined by EvictPolicy. In default policy there is no restriction. - */ - bool trySet(const Key & key, const MappedPtr & mapped) + void remove(const Key & key) { std::lock_guard lock(mutex); - - return setImpl(key, mapped, lock); + auto it = cells.find(key); + if (it == cells.end()) + return; + auto & cell = it->second; + current_size -= cell.size; + cells.erase(it); } /// If the value for the key is in the cache, returns it. If it is not, calls load_func() to @@ -106,8 +82,9 @@ public: /// Exceptions occurring in load_func will be propagated to the caller. Another thread from the /// set of concurrent threads will then try to call its load_func etc. /// + /// Returns std::pair of the cached value and a bool indicating whether the value was produced during this call. template - Result getOrSet(const Key &key, LoadFunc && load_func) + std::pair getOrSet(const Key & key, LoadFunc && load_func) { InsertTokenHolder token_holder; { @@ -117,7 +94,7 @@ public: if (val) { ++hits; - return {val, false, false}; + return std::make_pair(val, false); } auto & token = insert_tokens[key]; @@ -137,7 +114,7 @@ public: { /// Another thread already produced the value while we waited for token->mutex. ++hits; - return {token->value, false, false}; + return std::make_pair(token->value, false); } ++misses; @@ -147,37 +124,18 @@ public: /// Insert the new value only if the token is still in present in insert_tokens. /// (The token may be absent because of a concurrent reset() call). - bool is_value_loaded = false; - bool is_value_loaded_and_set = false; + bool result = false; auto token_it = insert_tokens.find(key); if (token_it != insert_tokens.end() && token_it->second.get() == token) { - is_value_loaded_and_set = setImpl(key, token->value, cache_lock); - is_value_loaded = true; + setImpl(key, token->value, cache_lock); + result = true; } if (!token->cleaned_up) token_holder.cleanup(token_lock, cache_lock); - return {token->value, is_value_loaded, is_value_loaded_and_set}; - } - - /// If key is not in cache or the element can be released, return is true. otherwise, return is false - bool tryRemove(const Key & key) - { - std::lock_guard loc(mutex); - auto it = cells.find(key); - if (it == cells.end()) - return true; - auto & cell = it->second; - if (!evict_policy.canRelease(cell.value)) - return false; - evict_policy.release(cell.value); - - current_size -= cell.size; - cells.erase(it); - queue.erase(cell.queue_iterator); - return true; + return std::make_pair(token->value, result); } void getStats(size_t & out_hits, size_t & out_misses) const @@ -312,7 +270,6 @@ private: std::atomic misses {0}; WeightFunction weight_function; - EvictPolicy evict_policy; MappedPtr getImpl(const Key & key, [[maybe_unused]] std::lock_guard & cache_lock) { @@ -330,14 +287,13 @@ private: return cell.value; } - bool setImpl(const Key & key, const MappedPtr & mapped, [[maybe_unused]] std::lock_guard & cache_lock) + void setImpl(const Key & key, const MappedPtr & mapped, [[maybe_unused]] std::lock_guard & cache_lock) { auto [it, inserted] = cells.emplace(std::piecewise_construct, std::forward_as_tuple(key), std::forward_as_tuple()); Cell & cell = it->second; - auto value_weight = mapped ? weight_function(*mapped) : 0; if (inserted) { @@ -350,42 +306,28 @@ private: cells.erase(it); throw; } - - if (!removeOverflow()) - { - // overflow is caused by inserting this element. - queue.erase(cell.queue_iterator); - cells.erase(it); - return false; - } } else { - if (!evict_policy.canRelease(cell.value)) - return false; - if (value_weight > cell.size && !removeOverflow(value_weight - cell.size)) - return false; - evict_policy.release(cell.value); // release the old value. this action is empty in default policy. current_size -= cell.size; queue.splice(queue.end(), queue, cell.queue_iterator); } cell.value = mapped; - cell.size = value_weight; + cell.size = cell.value ? weight_function(*cell.value) : 0; current_size += cell.size; - return true; + removeOverflow(); } - bool removeOverflow(size_t required_size_to_remove = 0) + void removeOverflow() { size_t current_weight_lost = 0; size_t queue_size = cells.size(); - auto key_it = queue.begin(); - auto is_overflow = [&] { return (current_size + required_size_to_remove > max_size || (max_elements_size != 0 && queue_size > max_elements_size)); }; - while (is_overflow() && (queue_size > 1) && (key_it != queue.end())) + + while ((current_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) && (queue_size > 1)) { - const Key & key = *key_it; + const Key & key = queue.front(); auto it = cells.find(key); if (it == cells.end()) @@ -395,23 +337,13 @@ private: } const auto & cell = it->second; - if (evict_policy.canRelease(cell.value))// in default, it is true - { - // always call release() before erasing an element - // in default, it's an empty action - evict_policy.release(cell.value); - current_size -= cell.size; - current_weight_lost += cell.size; + current_size -= cell.size; + current_weight_lost += cell.size; - cells.erase(it); - key_it = queue.erase(key_it); - --queue_size; - } - else - { - key_it++; - } + cells.erase(it); + queue.pop_front(); + --queue_size; } onRemoveOverflowWeightLoss(current_weight_lost); @@ -421,7 +353,6 @@ private: LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); abort(); } - return !is_overflow(); } /// Override this method if you want to track how much weight was lost in removeOverflow method. diff --git a/src/Common/LRUResourceCache.h b/src/Common/LRUResourceCache.h new file mode 100644 index 00000000000..7250b44a055 --- /dev/null +++ b/src/Common/LRUResourceCache.h @@ -0,0 +1,254 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +template +struct TrivailLRUResourceCacheWeightFunction +{ + size_t operator()(const T &) const { return 1; } +}; + +/* + * A resource cache with key index. There is only one instance for every key which is not like the normal resource pool. + * Resource cache has max weight capacity and keys size limitation. If the limitation is exceeded, keys would be evicted + * by LRU policy. + * + * acquire and release must be used in pair. + */ +template < + typename TKey, + typename TMapped, + typename WeightFunction = TrivailLRUResourceCacheWeightFunction, + typename HashFunction = std::hash> +class LRUResourceCache +{ +public: + LRUResourceCache(size_t max_weight_, size_t max_element_size_ = 0) : max_weight(max_weight_), max_element_size(max_element_size_) { } + virtual ~LRUResourceCache() { } + using Key = TKey; + using Mapped = TMapped; + using MappedPtr = std::shared_ptr; + + // - load_func : when key is not exists in cache, load_func is called to generate a new key + // - return: is null when there is no more space for the new value or the old value is in used. + template + MappedPtr acquire(const Key & key, LoadFunc && load_func) + { + InsertToken * insert_token = nullptr; + { + std::lock_guard lock(mutex); + auto it = cells.find(key); + if (it != cells.end()) + { + hits++; + it->second.reference_count += 1; + return it->second.value; + } + misses++; + insert_token = acquireInsertToken(key); + } + Cell * cell_ptr = nullptr; + { + std::lock_guard lock(insert_token->mutex); + if (!insert_token->value) + { + insert_token->value = load_func(); + std::lock_guard cell_lock(mutex); + cell_ptr = insert_value(key, insert_token->value); + if (cell_ptr) + { + cell_ptr->reference_count += 1; + } + else + { + insert_token->value = nullptr; + } + } + } + + std::lock_guard lock(mutex); + releaseInsertToken(key); + if (cell_ptr) + { + return cell_ptr->value; + } + return nullptr; + } + + MappedPtr acquire(const Key & key) + { + std::lock_guard lock(mutex); + auto it = cells.find(key); + if (it == cells.end()) + { + misses++; + return nullptr; + } + hits++; + it->second.reference_count += 1; + queue.splice(queue.end(), queue, it->second.queue_iterator); + return it->second.value; + } + + // mark a reference is released + void release(const Key & key) + { + std::lock_guard lock(mutex); + auto it = cells.find(key); + if (it == cells.end() || it->second.reference_count == 0) + { + LOG_ERROR(&Poco::Logger::get("LRUResourceCache"), "try to release an invalid element"); + abort(); + } + it->second.reference_count -= 1; + } + + // If you want to update a value, call tryRemove() at first and then call acquire() with load_func. + bool tryRemove(const Key & key) + { + std::lock_guard guard(mutex); + auto it = cells.find(key); + if (it == cells.end()) + return true; + auto & cell = it->second; + if (cell.reference_count) + return false; + queue.erase(cell.queue_iterator); + current_weight -= cell.weight; + cells.erase(it); + return true; + } + + size_t weight() + { + std::lock_guard lock(mutex); + return current_weight; + } + + size_t size() + { + std::lock_guard lock(mutex); + return cells.size(); + } + + void getStats(size_t & out_hits, size_t & out_misses, size_t & out_evict_count) const + { + out_hits = hits; + out_misses = misses; + out_evict_count = evict_count; + } + +private: + mutable std::mutex mutex; + + using LRUQueue = std::list; + using LRUQueueIterator = typename LRUQueue::iterator; + + struct Cell + { + MappedPtr value; + size_t weight = 0; + LRUQueueIterator queue_iterator; + size_t reference_count = 0; + }; + + using Cells = std::unordered_map; + Cells cells; + LRUQueue queue; + size_t current_weight = 0; + size_t max_weight = 0; + size_t max_element_size = 0; + + struct InsertToken + { + std::mutex mutex; + MappedPtr value; + size_t reference_count = 0; + }; + using InsertTokens = std::unordered_map; + InsertTokens insert_tokens; + WeightFunction weight_function; + std::atomic hits{0}; + std::atomic misses{0}; + std::atomic evict_count{0}; + + InsertToken * acquireInsertToken(const Key & key) + { + auto & token = insert_tokens[key]; + token.reference_count += 1; + return &token; + } + + void releaseInsertToken(const Key & key) + { + auto it = insert_tokens.find(key); + if (it != insert_tokens.end()) + { + it->second.reference_count -= 1; + if (it->second.reference_count == 0) + insert_tokens.erase(it); + } + } + + // key mustn't be in the cache + Cell * insert_value(const Key & insert_key, MappedPtr value) + { + auto weight = value ? weight_function(*value) : 0; + auto queue_size = cells.size() + 1; + auto loss_weight = 0; + auto is_overflow = [&] { + return current_weight + weight - loss_weight > max_weight || (max_element_size != 0 && queue_size > max_element_size); + }; + auto key_it = queue.begin(); + std::unordered_set to_release_keys; + while (is_overflow() && queue_size > 1 && key_it != queue.end()) + { + const Key & key = *key_it; + auto cell_it = cells.find(key); + if (cell_it == cells.end()) + { + LOG_ERROR(&Poco::Logger::get("LRUResourceCache"), "LRUResourceCache became inconsistent. There must be a bug in it."); + abort(); + } + auto & cell = cell_it->second; + if (cell.reference_count == 0) + { + loss_weight += cell.weight; + queue_size -= 1; + to_release_keys.insert(key); + } + key_it++; + } + if (is_overflow()) + return nullptr; + if (loss_weight > current_weight + weight) + { + LOG_ERROR(&Poco::Logger::get("LRUResourceCache"), "LRUResourceCache became inconsistent. There must be a bug in it."); + abort(); + } + for (auto & key : to_release_keys) + { + auto & cell = cells[key]; + queue.erase(cell.queue_iterator); + cells.erase(key); + evict_count++; + } + current_weight = current_weight + weight - loss_weight; + + auto & new_cell = cells[insert_key]; + new_cell.value = value; + new_cell.weight = weight; + new_cell.queue_iterator = queue.insert(queue.end(), insert_key); + return &new_cell; + } +}; +} diff --git a/src/Common/tests/gtest_lru_cache.cpp b/src/Common/tests/gtest_lru_cache.cpp new file mode 100644 index 00000000000..42e404de379 --- /dev/null +++ b/src/Common/tests/gtest_lru_cache.cpp @@ -0,0 +1,102 @@ +#include +#include +#include +#include + +TEST(LRUCache, set) +{ + using SimpleLRUCache =DB::LRUCache; + auto lru_cache = SimpleLRUCache(10, 10); + lru_cache.set(1, std::make_shared(2)); + lru_cache.set(2,std::make_shared(3)); + + auto w = lru_cache.weight(); + auto n = lru_cache.count(); + ASSERT_EQ(w, 2); + ASSERT_EQ(n, 2); +} + +TEST(LRUCache, update) +{ + using SimpleLRUCache =DB::LRUCache; + auto lru_cache = SimpleLRUCache(10, 10); + lru_cache.set(1, std::make_shared(2)); + lru_cache.set(1,std::make_shared(3)); + auto val = lru_cache.get(1); + ASSERT_TRUE(val != nullptr); + ASSERT_TRUE(*val == 3); +} + +TEST(LRUCache, get) +{ + using SimpleLRUCache =DB::LRUCache; + auto lru_cache = SimpleLRUCache(10, 10); + lru_cache.set(1, std::make_shared(2)); + lru_cache.set(2, std::make_shared(3)); + SimpleLRUCache::MappedPtr value = lru_cache.get(1); + ASSERT_TRUE(value != nullptr); + ASSERT_EQ(*value, 2); + + value = lru_cache.get(2); + ASSERT_TRUE(value != nullptr); + ASSERT_EQ(*value, 3); +} + +struct ValueWeight +{ + size_t operator()(const size_t & x) const + { + return x; + } +}; + +TEST(LRUCache, evict_on_size) +{ + + using SimpleLRUCache =DB::LRUCache; + auto lru_cache = SimpleLRUCache(20, 3); + lru_cache.set(1, std::make_shared(2)); + lru_cache.set(2, std::make_shared(3)); + lru_cache.set(3, std::make_shared(4)); + lru_cache.set(4, std::make_shared(5)); + + auto n = lru_cache.count(); + ASSERT_EQ(n, 3); + + auto value = lru_cache.get(1); + ASSERT_TRUE(value == nullptr); +} + +TEST(LRUCache, evict_on_weight) +{ + + using SimpleLRUCache =DB::LRUCache, ValueWeight>; + auto lru_cache = SimpleLRUCache(10, 10); + lru_cache.set(1, std::make_shared(2)); + lru_cache.set(2, std::make_shared(3)); + lru_cache.set(3, std::make_shared(4)); + lru_cache.set(4, std::make_shared(5)); + + auto n = lru_cache.count(); + ASSERT_EQ(n, 2); + + auto w = lru_cache.weight(); + ASSERT_EQ(w, 9); + + auto value = lru_cache.get(1); + ASSERT_TRUE(value == nullptr); + value = lru_cache.get(2); + ASSERT_TRUE(value == nullptr); +} + +TEST(LRUCache, getOrSet) +{ + using SimpleLRUCache =DB::LRUCache, ValueWeight>; + auto lru_cache = SimpleLRUCache(10, 10); + size_t x = 10; + auto load_func = [&]{ return std::make_shared(x); }; + auto [value, loaded] = lru_cache.getOrSet(1, load_func); + ASSERT_TRUE(value != nullptr); + ASSERT_TRUE(*value == 10); +} + diff --git a/src/Common/tests/gtest_lru_resource_cache.cpp b/src/Common/tests/gtest_lru_resource_cache.cpp new file mode 100644 index 00000000000..79317d83936 --- /dev/null +++ b/src/Common/tests/gtest_lru_resource_cache.cpp @@ -0,0 +1,212 @@ +#include +#include +#include +#include + +TEST(LRUResourceCache, acquire) +{ + using MyCache = DB::LRUResourceCache; + auto mcache = MyCache(10, 10); + int x = 10; + auto load_int = [&] { return std::make_shared(x); }; + auto val = mcache.acquire(1, load_int); + x = 11; + val = mcache.acquire(2, load_int); + ASSERT_TRUE(val != nullptr); + ASSERT_TRUE(*val == 11); + + val = mcache.acquire(1); + ASSERT_TRUE(val != nullptr); + ASSERT_TRUE(*val == 10); +} + +TEST(LRUResourceCache, remove) +{ + using MyCache = DB::LRUResourceCache; + auto mcache = MyCache(10, 10); + int x = 10; + auto load_int = [&] { return std::make_shared(x); }; + auto val = mcache.acquire(1, load_int); + x = 11; + val = mcache.acquire(2, load_int); + + auto succ = mcache.tryRemove(3); + ASSERT_TRUE(succ); + + succ = mcache.tryRemove(1); + ASSERT_TRUE(!succ); + val = mcache.acquire(1); + ASSERT_TRUE(val != nullptr); + ASSERT_TRUE(*val == 10); + + mcache.release(1); + succ = mcache.tryRemove(1); + ASSERT_TRUE(!succ); + mcache.release(1); + succ = mcache.tryRemove(1); + ASSERT_TRUE(succ); + val = mcache.acquire(1); + ASSERT_TRUE(val == nullptr); +} + +struct MyWeight +{ + size_t operator()(const int & x) const { return static_cast(x); } +}; + +TEST(LRUResourceCache, evict_on_weight) +{ + using MyCache = DB::LRUResourceCache; + auto mcache = MyCache(5, 10); + int x = 2; + auto load_int = [&] { return std::make_shared(x); }; + auto val = mcache.acquire(1, load_int); + mcache.release(1); + + val = mcache.acquire(2, load_int); + mcache.release(2); + + x = 3; + val = mcache.acquire(3, load_int); + ASSERT_TRUE(val != nullptr); + + auto w = mcache.weight(); + ASSERT_EQ(w, 5); + auto n = mcache.size(); + ASSERT_EQ(n, 2); + + val = mcache.acquire(1); + ASSERT_TRUE(val == nullptr); + val = mcache.acquire(2); + ASSERT_TRUE(val != nullptr); + val = mcache.acquire(3); + ASSERT_TRUE(val != nullptr); +} + +TEST(LRUResourceCache, evict_on_size) +{ + using MyCache = DB::LRUResourceCache; + auto mcache = MyCache(5, 2); + int x = 2; + auto load_int = [&] { return std::make_shared(x); }; + auto val = mcache.acquire(1, load_int); + mcache.release(1); + + val = mcache.acquire(2, load_int); + mcache.release(2); + + x = 3; + val = mcache.acquire(3, load_int); + ASSERT_TRUE(val != nullptr); + + auto n = mcache.size(); + ASSERT_EQ(n, 2); + auto w = mcache.weight(); + ASSERT_EQ(w, 2); + + val = mcache.acquire(1); + ASSERT_TRUE(val == nullptr); + val = mcache.acquire(2); + ASSERT_TRUE(val != nullptr); + val = mcache.acquire(3); + ASSERT_TRUE(val != nullptr); +} + +TEST(LRUResourceCache, not_evict_used_element) +{ + using MyCache = DB::LRUResourceCache; + auto mcache = MyCache(7, 10); + int x = 2; + auto load_int = [&] { return std::make_shared(x); }; + auto val = mcache.acquire(1, load_int); + + val = mcache.acquire(2, load_int); + mcache.release(2); + + val = mcache.acquire(3, load_int); + mcache.release(3); + + x = 3; + val = mcache.acquire(4, load_int); + ASSERT_TRUE(val != nullptr); + + auto n = mcache.size(); + ASSERT_EQ(n, 3); + auto w = mcache.weight(); + ASSERT_EQ(w, 7); + + val = mcache.acquire(1); + ASSERT_TRUE(val != nullptr); + val = mcache.acquire(2); + ASSERT_TRUE(val == nullptr); + val = mcache.acquire(3); + ASSERT_TRUE(val != nullptr); + val = mcache.acquire(4); + ASSERT_TRUE(val != nullptr); +} + +TEST(LRUResourceCache, acquire_fail) +{ + using MyCache = DB::LRUResourceCache; + auto mcache = MyCache(5, 10); + int x = 2; + auto load_int = [&] { return std::make_shared(x); }; + auto val = mcache.acquire(1, load_int); + val = mcache.acquire(2, load_int); + val = mcache.acquire(3, load_int); + ASSERT_TRUE(val == nullptr); + + auto n = mcache.size(); + ASSERT_EQ(n, 2); + auto w = mcache.weight(); + ASSERT_EQ(w, 4); + val = mcache.acquire(1); + ASSERT_TRUE(val != nullptr); + val = mcache.acquire(2); + ASSERT_TRUE(val != nullptr); + val = mcache.acquire(3); + ASSERT_TRUE(val == nullptr); +} + +TEST(LRUResourceCache, dup_acquire) +{ + using MyCache = DB::LRUResourceCache; + auto mcache = MyCache(20, 10); + int x = 2; + auto load_int = [&] { return std::make_shared(x); }; + auto val = mcache.acquire(1, load_int); + mcache.release(1); + x = 11; + val = mcache.acquire(1, load_int); + ASSERT_TRUE(val != nullptr); + + auto n = mcache.size(); + ASSERT_EQ(n, 1); + auto w = mcache.weight(); + ASSERT_EQ(w, 2); + val = mcache.acquire(1); + ASSERT_TRUE(val != nullptr); + ASSERT_TRUE(*val == 2); +} + +TEST(LRUResourceCache, re_acquire) +{ + using MyCache = DB::LRUResourceCache; + auto mcache = MyCache(20, 10); + int x = 2; + auto load_int = [&] { return std::make_shared(x); }; + auto val = mcache.acquire(1, load_int); + mcache.release(1); + mcache.tryRemove(1); + x = 11; + val = mcache.acquire(1, load_int); + ASSERT_TRUE(val != nullptr); + + auto n = mcache.size(); + ASSERT_EQ(n, 1); + auto w = mcache.weight(); + ASSERT_EQ(w, 11); + val = mcache.acquire(1); + ASSERT_TRUE(val != nullptr); + ASSERT_TRUE(*val == 11); +} diff --git a/src/IO/MMappedFileCache.h b/src/IO/MMappedFileCache.h index 7ee6957c7db..adbb85a18cf 100644 --- a/src/IO/MMappedFileCache.h +++ b/src/IO/MMappedFileCache.h @@ -48,12 +48,12 @@ public: MappedPtr getOrSet(const Key & key, LoadFunc && load) { auto result = Base::getOrSet(key, load); - if (result.cache_miss) + if (result.second) ProfileEvents::increment(ProfileEvents::MMappedFileCacheMisses); else ProfileEvents::increment(ProfileEvents::MMappedFileCacheHits); - return result.value; + return result.first; } }; diff --git a/src/IO/UncompressedCache.h b/src/IO/UncompressedCache.h index 78f81c15a4a..5826b7f020a 100644 --- a/src/IO/UncompressedCache.h +++ b/src/IO/UncompressedCache.h @@ -63,12 +63,12 @@ public: { auto result = Base::getOrSet(key, std::forward(load)); - if (result.cache_miss) + if (result.second) ProfileEvents::increment(ProfileEvents::UncompressedCacheMisses); else ProfileEvents::increment(ProfileEvents::UncompressedCacheHits); - return result.value; + return result.first; } private: diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 95341efa76a..5c9d94d7c45 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -354,14 +354,13 @@ void Aggregator::compileAggregateFunctionsIfNeeded() if (auto * compilation_cache = CompiledExpressionCacheFactory::instance().tryGetCache()) { - auto result = compilation_cache->getOrSet(aggregate_functions_description_hash_key, [&] () + auto [compiled_function_cache_entry, _] = compilation_cache->getOrSet(aggregate_functions_description_hash_key, [&] () { LOG_TRACE(log, "Compile expression {}", functions_description); auto compiled_aggregate_functions = compileAggregateFunctions(getJITInstance(), functions_to_compile, functions_description); return std::make_shared(std::move(compiled_aggregate_functions)); }); - auto compiled_function_cache_entry = result.value; compiled_aggregate_functions_holder = std::static_pointer_cast(compiled_function_cache_entry); } else diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index d5017b18dc1..90292d17fae 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -296,13 +296,12 @@ static FunctionBasePtr compile( if (auto * compilation_cache = CompiledExpressionCacheFactory::instance().tryGetCache()) { - auto result = compilation_cache->getOrSet(hash_key, [&] () + auto [compiled_function_cache_entry, _] = compilation_cache->getOrSet(hash_key, [&] () { LOG_TRACE(getLogger(), "Compile expression {}", llvm_function->getName()); auto compiled_function = compileFunction(getJITInstance(), *llvm_function); return std::make_shared(compiled_function); }); - auto compiled_function_cache_entry = result.value; std::shared_ptr compiled_function_holder = std::static_pointer_cast(compiled_function_cache_entry); llvm_function->setCompiledFunction(std::move(compiled_function_holder)); diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index d5ea1682dff..7f22386f54b 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -1033,7 +1033,7 @@ std::shared_ptr MergeJoin::loadRightBlock(size_t pos) const return std::make_shared(input.block_in->read()); }; - return cached_right_blocks->getOrSet(pos, load_func).value; + return cached_right_blocks->getOrSet(pos, load_func).first; } else return loaded_right_blocks[pos]; diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index c415f27d202..11e56ecbe0c 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -650,7 +650,7 @@ public: avro::ValidSchema getSchema(uint32_t id) { - auto [schema, loaded, _] = schema_cache.getOrSet( + auto [schema, loaded] = schema_cache.getOrSet( id, [this, id](){ return std::make_shared(fetchSchema(id)); } ); @@ -727,7 +727,7 @@ static LRUCache schema_registry_cache(SCH static std::shared_ptr getConfluentSchemaRegistry(const FormatSettings & format_settings) { const auto & base_url = format_settings.avro.schema_registry_url; - auto [schema_registry, loaded, _] = schema_registry_cache.getOrSet( + auto [schema_registry, loaded] = schema_registry_cache.getOrSet( base_url, [base_url]() { diff --git a/src/Storages/MarkCache.h b/src/Storages/MarkCache.h index 3438b4a1b9b..06143e954f8 100644 --- a/src/Storages/MarkCache.h +++ b/src/Storages/MarkCache.h @@ -59,12 +59,12 @@ public: MappedPtr getOrSet(const Key & key, LoadFunc && load) { auto result = Base::getOrSet(key, load); - if (result.cache_miss) + if (result.second) ProfileEvents::increment(ProfileEvents::MarkCacheMisses); else ProfileEvents::increment(ProfileEvents::MarkCacheHits); - return result.value; + return result.first; } }; From 5a419a356b4e5f6475c84306ec828a0132fee9d5 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 29 Dec 2021 15:29:59 +0800 Subject: [PATCH 0544/1260] fix a bug --- src/Common/LRUResourceCache.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/LRUResourceCache.h b/src/Common/LRUResourceCache.h index 7250b44a055..2b1039ed9a5 100644 --- a/src/Common/LRUResourceCache.h +++ b/src/Common/LRUResourceCache.h @@ -51,6 +51,7 @@ public: { hits++; it->second.reference_count += 1; + queue.splice(queue.end(), queue, it->second.queue_iterator); return it->second.value; } misses++; From 5f85f7726650e6a4a621af20d17f5d0e64f6b988 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 29 Dec 2021 15:34:20 +0800 Subject: [PATCH 0545/1260] add a new test case --- src/Common/tests/gtest_lru_resource_cache.cpp | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/src/Common/tests/gtest_lru_resource_cache.cpp b/src/Common/tests/gtest_lru_resource_cache.cpp index 79317d83936..495b08137c4 100644 --- a/src/Common/tests/gtest_lru_resource_cache.cpp +++ b/src/Common/tests/gtest_lru_resource_cache.cpp @@ -83,6 +83,38 @@ TEST(LRUResourceCache, evict_on_weight) ASSERT_TRUE(val != nullptr); } +TEST(LRUResourceCache, evict_on_weight_v2) +{ + using MyCache = DB::LRUResourceCache; + auto mcache = MyCache(5, 10); + int x = 2; + auto load_int = [&] { return std::make_shared(x); }; + auto val = mcache.acquire(1, load_int); + mcache.release(1); + + val = mcache.acquire(2, load_int); + mcache.release(2); + + val = mcache.acquire(1); + mcache.release(1); + + x = 3; + val = mcache.acquire(3, load_int); + ASSERT_TRUE(val != nullptr); + + auto w = mcache.weight(); + ASSERT_EQ(w, 5); + auto n = mcache.size(); + ASSERT_EQ(n, 2); + + val = mcache.acquire(1); + ASSERT_TRUE(val != nullptr); + val = mcache.acquire(2); + ASSERT_TRUE(val == nullptr); + val = mcache.acquire(3); + ASSERT_TRUE(val != nullptr); +} + TEST(LRUResourceCache, evict_on_size) { using MyCache = DB::LRUResourceCache; From 0d474069b5ccfb3f1075b6fd5b97f0d106245c0d Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 29 Dec 2021 15:38:58 +0800 Subject: [PATCH 0546/1260] add a new test case --- src/Common/tests/gtest_lru_resource_cache.cpp | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/src/Common/tests/gtest_lru_resource_cache.cpp b/src/Common/tests/gtest_lru_resource_cache.cpp index 495b08137c4..f87855bf51d 100644 --- a/src/Common/tests/gtest_lru_resource_cache.cpp +++ b/src/Common/tests/gtest_lru_resource_cache.cpp @@ -115,6 +115,38 @@ TEST(LRUResourceCache, evict_on_weight_v2) ASSERT_TRUE(val != nullptr); } +TEST(LRUResourceCache, evict_on_weight_v3) +{ + using MyCache = DB::LRUResourceCache; + auto mcache = MyCache(5, 10); + int x = 2; + auto load_int = [&] { return std::make_shared(x); }; + auto val = mcache.acquire(1, load_int); + mcache.release(1); + + val = mcache.acquire(2, load_int); + mcache.release(2); + + val = mcache.acquire(1, load_int); + mcache.release(1); + + x = 3; + val = mcache.acquire(3, load_int); + ASSERT_TRUE(val != nullptr); + + auto w = mcache.weight(); + ASSERT_EQ(w, 5); + auto n = mcache.size(); + ASSERT_EQ(n, 2); + + val = mcache.acquire(1); + ASSERT_TRUE(val != nullptr); + val = mcache.acquire(2); + ASSERT_TRUE(val == nullptr); + val = mcache.acquire(3); + ASSERT_TRUE(val != nullptr); +} + TEST(LRUResourceCache, evict_on_size) { using MyCache = DB::LRUResourceCache; From 6aaeb285b0dbd6dba21d5ddf36fec85eb3ef2b22 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 29 Dec 2021 15:44:24 +0800 Subject: [PATCH 0547/1260] update destructor --- src/Common/LRUResourceCache.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/LRUResourceCache.h b/src/Common/LRUResourceCache.h index 2b1039ed9a5..221ba225f2c 100644 --- a/src/Common/LRUResourceCache.h +++ b/src/Common/LRUResourceCache.h @@ -33,7 +33,7 @@ class LRUResourceCache { public: LRUResourceCache(size_t max_weight_, size_t max_element_size_ = 0) : max_weight(max_weight_), max_element_size(max_element_size_) { } - virtual ~LRUResourceCache() { } + ~LRUResourceCache() = default; using Key = TKey; using Mapped = TMapped; using MappedPtr = std::shared_ptr; From 5a3b215f2448bdeace27a8a8b78fb5285d3ce73d Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 29 Dec 2021 16:00:57 +0800 Subject: [PATCH 0548/1260] fixed code style --- src/Common/tests/gtest_lru_cache.cpp | 27 +++++++++++---------------- 1 file changed, 11 insertions(+), 16 deletions(-) diff --git a/src/Common/tests/gtest_lru_cache.cpp b/src/Common/tests/gtest_lru_cache.cpp index 42e404de379..8a8b2ff3de8 100644 --- a/src/Common/tests/gtest_lru_cache.cpp +++ b/src/Common/tests/gtest_lru_cache.cpp @@ -5,11 +5,11 @@ TEST(LRUCache, set) { - using SimpleLRUCache =DB::LRUCache; + using SimpleLRUCache = DB::LRUCache; auto lru_cache = SimpleLRUCache(10, 10); lru_cache.set(1, std::make_shared(2)); - lru_cache.set(2,std::make_shared(3)); - + lru_cache.set(2, std::make_shared(3)); + auto w = lru_cache.weight(); auto n = lru_cache.count(); ASSERT_EQ(w, 2); @@ -18,10 +18,10 @@ TEST(LRUCache, set) TEST(LRUCache, update) { - using SimpleLRUCache =DB::LRUCache; + using SimpleLRUCache = DB::LRUCache; auto lru_cache = SimpleLRUCache(10, 10); lru_cache.set(1, std::make_shared(2)); - lru_cache.set(1,std::make_shared(3)); + lru_cache.set(1, std::make_shared(3)); auto val = lru_cache.get(1); ASSERT_TRUE(val != nullptr); ASSERT_TRUE(*val == 3); @@ -29,7 +29,7 @@ TEST(LRUCache, update) TEST(LRUCache, get) { - using SimpleLRUCache =DB::LRUCache; + using SimpleLRUCache = DB::LRUCache; auto lru_cache = SimpleLRUCache(10, 10); lru_cache.set(1, std::make_shared(2)); lru_cache.set(2, std::make_shared(3)); @@ -44,16 +44,12 @@ TEST(LRUCache, get) struct ValueWeight { - size_t operator()(const size_t & x) const - { - return x; - } + size_t operator()(const size_t & x) const { return x; } }; TEST(LRUCache, evict_on_size) { - - using SimpleLRUCache =DB::LRUCache; + using SimpleLRUCache = DB::LRUCache; auto lru_cache = SimpleLRUCache(20, 3); lru_cache.set(1, std::make_shared(2)); lru_cache.set(2, std::make_shared(3)); @@ -69,8 +65,7 @@ TEST(LRUCache, evict_on_size) TEST(LRUCache, evict_on_weight) { - - using SimpleLRUCache =DB::LRUCache, ValueWeight>; + using SimpleLRUCache = DB::LRUCache, ValueWeight>; auto lru_cache = SimpleLRUCache(10, 10); lru_cache.set(1, std::make_shared(2)); lru_cache.set(2, std::make_shared(3)); @@ -91,10 +86,10 @@ TEST(LRUCache, evict_on_weight) TEST(LRUCache, getOrSet) { - using SimpleLRUCache =DB::LRUCache, ValueWeight>; + using SimpleLRUCache = DB::LRUCache, ValueWeight>; auto lru_cache = SimpleLRUCache(10, 10); size_t x = 10; - auto load_func = [&]{ return std::make_shared(x); }; + auto load_func = [&] { return std::make_shared(x); }; auto [value, loaded] = lru_cache.getOrSet(1, load_func); ASSERT_TRUE(value != nullptr); ASSERT_TRUE(*value == 10); From 6784838047cce518700526b9956f2b3e9651f191 Mon Sep 17 00:00:00 2001 From: "Chun-Sheng, Li" Date: Wed, 29 Dec 2021 14:46:00 +0800 Subject: [PATCH 0549/1260] Adding cURL installation message to notice users --- docs/en/interfaces/http.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 313c6508b55..4fc1d709b36 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -9,6 +9,8 @@ The HTTP interface lets you use ClickHouse on any platform from any programming By default, `clickhouse-server` listens for HTTP on port 8123 (this can be changed in the config). +By default, `cURL` command is not available on user operating systems. Please refer this [documentation](https://curl.se/download.html) to install it before running the examples. + If you make a `GET /` request without parameters, it returns 200 response code and the string which defined in [http_server_default_response](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-http_server_default_response) default value “Ok.” (with a line feed at the end) ``` bash From e6712521310cab04c30c19c45dc817ef8ff6bd65 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 29 Dec 2021 11:40:12 +0300 Subject: [PATCH 0550/1260] [docs/faq] Does ClickHouse support multi-region replication? (#32700) * [docs/faq] Does ClickHouse support multi-region replication? * Update index.md * add i18n symlinks * fix symlinks --- docs/en/faq/operations/index.md | 1 + docs/en/faq/operations/multi-region-replication.md | 13 +++++++++++++ docs/ja/faq/operations/multi-region-replication.md | 1 + docs/ru/faq/operations/multi-region-replication.md | 1 + docs/zh/faq/operations/multi-region-replication.md | 1 + 5 files changed, 17 insertions(+) create mode 100644 docs/en/faq/operations/multi-region-replication.md create mode 120000 docs/ja/faq/operations/multi-region-replication.md create mode 120000 docs/ru/faq/operations/multi-region-replication.md create mode 120000 docs/zh/faq/operations/multi-region-replication.md diff --git a/docs/en/faq/operations/index.md b/docs/en/faq/operations/index.md index c0a6d85b66d..81aec18b9cf 100644 --- a/docs/en/faq/operations/index.md +++ b/docs/en/faq/operations/index.md @@ -11,6 +11,7 @@ Questions: - [Which ClickHouse version to use in production?](../../faq/operations/production.md) - [Is it possible to delete old records from a ClickHouse table?](../../faq/operations/delete-old-data.md) +- [Does ClickHouse support multi-region replication?](../../faq/operations/multi-region-replication.md) !!! info "Don’t see what you were looking for?" Check out [other F.A.Q. categories](../../faq/index.md) or browse around main documentation articles found in the left sidebar. diff --git a/docs/en/faq/operations/multi-region-replication.md b/docs/en/faq/operations/multi-region-replication.md new file mode 100644 index 00000000000..7d78737544a --- /dev/null +++ b/docs/en/faq/operations/multi-region-replication.md @@ -0,0 +1,13 @@ +--- +title: Does ClickHouse support multi-region replication? +toc_hidden: true +toc_priority: 30 +--- + +# Does ClickHouse support multi-region replication? {#does-clickhouse-support-multi-region-replication} + +The short answer is "yes". However, we recommend keeping latency between all regions/datacenters in two-digit range, otherwise write performance will suffer as it goes through distributed consensus protocol. For example, replication between US coasts will likely work fine, but between the US and Europe won't. + +Configuration-wise there's no difference compared to single-region replication, simply use hosts that are located in different locations for replicas. + +For more information, see [full article on data replication](../../engines/table-engines/mergetree-family/replication.md). diff --git a/docs/ja/faq/operations/multi-region-replication.md b/docs/ja/faq/operations/multi-region-replication.md new file mode 120000 index 00000000000..dbc985ee1fb --- /dev/null +++ b/docs/ja/faq/operations/multi-region-replication.md @@ -0,0 +1 @@ +../../../en/faq/operations/multi-region-replication.md \ No newline at end of file diff --git a/docs/ru/faq/operations/multi-region-replication.md b/docs/ru/faq/operations/multi-region-replication.md new file mode 120000 index 00000000000..dbc985ee1fb --- /dev/null +++ b/docs/ru/faq/operations/multi-region-replication.md @@ -0,0 +1 @@ +../../../en/faq/operations/multi-region-replication.md \ No newline at end of file diff --git a/docs/zh/faq/operations/multi-region-replication.md b/docs/zh/faq/operations/multi-region-replication.md new file mode 120000 index 00000000000..dbc985ee1fb --- /dev/null +++ b/docs/zh/faq/operations/multi-region-replication.md @@ -0,0 +1 @@ +../../../en/faq/operations/multi-region-replication.md \ No newline at end of file From a954de4560119129011b4be72de4ed9e4f911365 Mon Sep 17 00:00:00 2001 From: benbiti Date: Wed, 29 Dec 2021 16:46:06 +0800 Subject: [PATCH 0551/1260] fix missing ; in tests --- .../0_stateless/01281_parseDateTime64BestEffort.sql | 8 ++++---- .../0_stateless/01442_date_time_with_params.reference | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01281_parseDateTime64BestEffort.sql b/tests/queries/0_stateless/01281_parseDateTime64BestEffort.sql index bf52314bb62..ac1186284be 100644 --- a/tests/queries/0_stateless/01281_parseDateTime64BestEffort.sql +++ b/tests/queries/0_stateless/01281_parseDateTime64BestEffort.sql @@ -32,7 +32,7 @@ SELECT parseDateTime64BestEffort('2020-05-14T03:37:03.253184', 3, 'UTC'); SELECT parseDateTime64BestEffort('2020-05-14T03:37:03', 3, 'UTC'); SELECT parseDateTime64BestEffort('2020-05-14 03:37:03', 3, 'UTC'); -SELECT 'Unix Timestamp with Milliseconds' -SELECT parseDateTime64BestEffort('1640649600123', 3, 'UTC') -SELECT parseDateTime64BestEffort('1640649600123', 1, 'UTC') -SELECT parseDateTime64BestEffort('1640649600123', 6, 'UTC') +SELECT 'Unix Timestamp with Milliseconds'; +SELECT parseDateTime64BestEffort('1640649600123', 3, 'UTC'); +SELECT parseDateTime64BestEffort('1640649600123', 1, 'UTC'); +SELECT parseDateTime64BestEffort('1640649600123', 6, 'UTC'); diff --git a/tests/queries/0_stateless/01442_date_time_with_params.reference b/tests/queries/0_stateless/01442_date_time_with_params.reference index 378b37de4a7..726e59d4d35 100644 --- a/tests/queries/0_stateless/01442_date_time_with_params.reference +++ b/tests/queries/0_stateless/01442_date_time_with_params.reference @@ -80,4 +80,4 @@ parseDateTime32BestEffortOrZero 2020-05-14 03:37:03 DateTime(\'UTC\') 2020-05-14 06:37:03 DateTime(\'Europe/Minsk\') 2020-05-14 03:37:03 DateTime(\'UTC\') -2021-12-28 00:00:00 DateTime(\'UTC\') \ No newline at end of file +2021-12-28 00:00:00 DateTime(\'UTC\') From 8112a7123362d8286d2550e2353c0f58d8631a8a Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 15 Dec 2021 14:30:57 +0300 Subject: [PATCH 0552/1260] Implement schema inference for most input formats --- programs/local/LocalServer.cpp | 13 +- src/Common/ErrorCodes.cpp | 1 + src/Common/ZooKeeper/ZooKeeper.cpp | 51 ++ src/Common/ZooKeeper/ZooKeeper.h | 7 + src/Core/Settings.h | 3 + src/DataTypes/IDataType.h | 2 + src/Databases/DatabaseOnDisk.cpp | 14 +- src/Formats/CapnProtoUtils.cpp | 109 ++++ src/Formats/CapnProtoUtils.h | 1 + src/Formats/EscapingRuleUtils.cpp | 139 ++++- src/Formats/EscapingRuleUtils.h | 20 + src/Formats/FormatFactory.cpp | 64 ++- src/Formats/FormatFactory.h | 32 ++ src/Formats/FormatSettings.h | 6 + src/Formats/JSONEachRowUtils.cpp | 189 +++++++ src/Formats/JSONEachRowUtils.h | 16 + src/Formats/ParsedTemplateFormatString.cpp | 8 +- src/Formats/ParsedTemplateFormatString.h | 4 +- src/Formats/ProtobufSerializer.cpp | 103 +++- src/Formats/ProtobufSerializer.h | 3 + src/Formats/ReadSchemaUtils.cpp | 112 ++++ src/Formats/ReadSchemaUtils.h | 30 ++ src/Formats/registerFormats.cpp | 44 ++ src/IO/ReadHelpers.cpp | 59 +- src/IO/ReadHelpers.h | 5 + src/Interpreters/InterpreterCreateQuery.cpp | 10 +- src/Parsers/ASTCreateQuery.cpp | 4 +- src/Parsers/ASTCreateQuery.h | 6 + src/Parsers/ParserCreateQuery.cpp | 41 +- src/Parsers/ParserCreateQuery.h | 2 + src/Processors/Formats/ISchemaReader.cpp | 160 ++++++ src/Processors/Formats/ISchemaReader.h | 87 +++ .../Formats/Impl/ArrowBlockInputFormat.cpp | 82 ++- .../Formats/Impl/ArrowBlockInputFormat.h | 13 + .../Formats/Impl/ArrowColumnToCHColumn.cpp | 36 +- .../Formats/Impl/ArrowColumnToCHColumn.h | 8 +- .../Formats/Impl/AvroRowInputFormat.cpp | 103 ++++ .../Formats/Impl/AvroRowInputFormat.h | 15 + .../Formats/Impl/BinaryRowInputFormat.cpp | 51 +- .../Formats/Impl/BinaryRowInputFormat.h | 33 +- .../Formats/Impl/CSVRowInputFormat.cpp | 127 +++-- .../Formats/Impl/CSVRowInputFormat.h | 30 +- .../Formats/Impl/CapnProtoRowInputFormat.cpp | 28 + .../Formats/Impl/CapnProtoRowInputFormat.h | 12 + .../Impl/CustomSeparatedRowInputFormat.cpp | 194 +++++-- .../Impl/CustomSeparatedRowInputFormat.h | 49 +- .../Impl/JSONAsStringRowInputFormat.cpp | 8 + .../Formats/Impl/JSONAsStringRowInputFormat.h | 11 + .../Impl/JSONCompactEachRowRowInputFormat.cpp | 103 ++-- .../Impl/JSONCompactEachRowRowInputFormat.h | 26 +- .../Impl/JSONEachRowRowInputFormat.cpp | 42 +- .../Formats/Impl/JSONEachRowRowInputFormat.h | 13 + .../Impl/LineAsStringRowInputFormat.cpp | 9 + .../Formats/Impl/LineAsStringRowInputFormat.h | 11 + .../Formats/Impl/MsgPackRowInputFormat.cpp | 119 ++++- .../Formats/Impl/MsgPackRowInputFormat.h | 15 + src/Processors/Formats/Impl/NativeFormat.cpp | 26 + .../Formats/Impl/ORCBlockInputFormat.cpp | 41 +- .../Formats/Impl/ORCBlockInputFormat.h | 12 + .../Formats/Impl/ParquetBlockInputFormat.cpp | 50 +- .../Formats/Impl/ParquetBlockInputFormat.h | 12 + .../Formats/Impl/ProtobufRowInputFormat.cpp | 30 ++ .../Formats/Impl/ProtobufRowInputFormat.h | 13 + .../Formats/Impl/RawBLOBRowInputFormat.cpp | 9 + .../Formats/Impl/RawBLOBRowInputFormat.h | 11 + .../Formats/Impl/RegexpRowInputFormat.cpp | 132 +++-- .../Formats/Impl/RegexpRowInputFormat.h | 47 +- .../Formats/Impl/TSKVRowInputFormat.cpp | 63 +++ .../Formats/Impl/TSKVRowInputFormat.h | 13 + .../Impl/TabSeparatedRowInputFormat.cpp | 94 +++- .../Formats/Impl/TabSeparatedRowInputFormat.h | 32 +- .../Formats/Impl/TemplateRowInputFormat.cpp | 504 +++++++++++------- .../Formats/Impl/TemplateRowInputFormat.h | 78 ++- .../Formats/Impl/ValuesBlockInputFormat.cpp | 157 ++++-- .../Formats/Impl/ValuesBlockInputFormat.h | 17 +- .../RowInputFormatWithNamesAndTypes.cpp | 112 +++- .../Formats/RowInputFormatWithNamesAndTypes.h | 131 +++-- .../ExternalDataSourceConfiguration.h | 2 +- src/Storages/HDFS/StorageHDFS.cpp | 148 +++-- src/Storages/HDFS/StorageHDFS.h | 8 +- src/Storages/MergeTree/MergeTreeData.cpp | 1 - .../MergeTree/registerStorageMergeTree.cpp | 47 +- src/Storages/StorageBuffer.cpp | 9 +- src/Storages/StorageDistributed.cpp | 18 +- src/Storages/StorageFactory.h | 8 + src/Storages/StorageFile.cpp | 171 ++++-- src/Storages/StorageFile.h | 10 + src/Storages/StorageMerge.cpp | 17 +- src/Storages/StorageMerge.h | 2 + src/Storages/StorageReplicatedMergeTree.cpp | 65 +-- src/Storages/StorageReplicatedMergeTree.h | 2 + src/Storages/StorageS3.cpp | 95 +++- src/Storages/StorageS3.h | 24 +- src/Storages/StorageURL.cpp | 44 +- src/Storages/StorageURL.h | 8 + src/TableFunctions/ITableFunction.cpp | 12 +- src/TableFunctions/ITableFunction.h | 2 +- src/TableFunctions/ITableFunctionFileLike.cpp | 57 +- src/TableFunctions/ITableFunctionFileLike.h | 9 +- src/TableFunctions/TableFunctionFile.cpp | 16 + src/TableFunctions/TableFunctionFile.h | 8 +- src/TableFunctions/TableFunctionHDFS.cpp | 12 +- src/TableFunctions/TableFunctionHDFS.h | 4 +- src/TableFunctions/TableFunctionRemote.h | 1 + src/TableFunctions/TableFunctionS3.cpp | 42 +- src/TableFunctions/TableFunctionS3.h | 4 +- src/TableFunctions/TableFunctionURL.cpp | 35 +- src/TableFunctions/TableFunctionURL.h | 7 +- 108 files changed, 4029 insertions(+), 926 deletions(-) create mode 100644 src/Formats/ReadSchemaUtils.cpp create mode 100644 src/Formats/ReadSchemaUtils.h create mode 100644 src/Processors/Formats/ISchemaReader.cpp create mode 100644 src/Processors/Formats/ISchemaReader.h diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 53e295b7fbb..aa4747636c9 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -313,11 +313,11 @@ void LocalServer::cleanup() std::string LocalServer::getInitialCreateTableQuery() { - if (!config().has("table-structure")) + if (!config().has("table-structure") && !config().has("table-file")) return {}; auto table_name = backQuoteIfNeed(config().getString("table-name", "table")); - auto table_structure = config().getString("table-structure"); + auto table_structure = config().getString("table-structure", "auto"); auto data_format = backQuoteIfNeed(config().getString("table-data-format", "TSV")); String table_file; @@ -332,7 +332,12 @@ std::string LocalServer::getInitialCreateTableQuery() table_file = quoteString(config().getString("table-file")); } - return fmt::format("CREATE TABLE {} ({}) ENGINE = File({}, {});", + if (table_structure == "auto") + table_structure = ""; + else + table_structure = "(" + table_structure + ")"; + + return fmt::format("CREATE TABLE {} {} ENGINE = File({}, {});", table_name, table_structure, data_format, table_file); } @@ -422,7 +427,7 @@ try #else is_interactive = stdin_is_a_tty && (config().hasOption("interactive") - || (!config().has("query") && !config().has("table-structure") && queries_files.empty())); + || (!config().has("query") && !config().has("table-structure") && queries_files.empty() && !config().has("table-file"))); #endif if (!is_interactive) { diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 70d85433513..514aef09db0 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -603,6 +603,7 @@ M(632, UNEXPECTED_DATA_AFTER_PARSED_VALUE) \ M(633, QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW) \ M(634, MONGODB_ERROR) \ + M(635, CANNOT_EXTRACT_TABLE_STRUCTURE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index f05a10b8815..c8753c8edaf 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -26,6 +26,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; + extern const int BAD_ARGUMENTS; } } @@ -1133,4 +1134,54 @@ Coordination::RequestPtr makeCheckRequest(const std::string & path, int version) return request; } +std::string normalizeZooKeeperPath(std::string zookeeper_path, bool check_starts_with_slash, Poco::Logger * log) +{ + if (!zookeeper_path.empty() && zookeeper_path.back() == '/') + zookeeper_path.resize(zookeeper_path.size() - 1); + /// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it. + if (!zookeeper_path.empty() && zookeeper_path.front() != '/') + { + /// Do not allow this for new tables, print warning for tables created in old versions + if (check_starts_with_slash) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path must starts with '/', got '{}'", zookeeper_path); + if (log) + LOG_WARNING(log, "ZooKeeper path ('{}') does not start with '/'. It will not be supported in future releases"); + zookeeper_path = "/" + zookeeper_path; + } + + return zookeeper_path; +} + +String extractZooKeeperName(const String & path) +{ + static constexpr auto default_zookeeper_name = "default"; + if (path.empty()) + throw DB::Exception("ZooKeeper path should not be empty", DB::ErrorCodes::BAD_ARGUMENTS); + if (path[0] == '/') + return default_zookeeper_name; + auto pos = path.find(":/"); + if (pos != String::npos && pos < path.find('/')) + { + auto zookeeper_name = path.substr(0, pos); + if (zookeeper_name.empty()) + throw DB::Exception("Zookeeper path should start with '/' or ':/'", DB::ErrorCodes::BAD_ARGUMENTS); + return zookeeper_name; + } + return default_zookeeper_name; +} + +String extractZooKeeperPath(const String & path, bool check_starts_with_slash, Poco::Logger * log) +{ + if (path.empty()) + throw DB::Exception("ZooKeeper path should not be empty", DB::ErrorCodes::BAD_ARGUMENTS); + if (path[0] == '/') + return normalizeZooKeeperPath(path, check_starts_with_slash, log); + auto pos = path.find(":/"); + if (pos != String::npos && pos < path.find('/')) + { + return normalizeZooKeeperPath(path.substr(pos + 1, String::npos), check_starts_with_slash, log); + } + return normalizeZooKeeperPath(path, check_starts_with_slash, log); +} + } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 8e015b1f331..371f93f6df3 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -379,4 +379,11 @@ private: }; using EphemeralNodeHolderPtr = EphemeralNodeHolder::Ptr; + +String normalizeZooKeeperPath(std::string zookeeper_path, bool check_starts_with_slash, Poco::Logger * log = nullptr); + +String extractZooKeeperName(const String & path); + +String extractZooKeeperPath(const String & path, bool check_starts_with_slash, Poco::Logger * log = nullptr); + } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 70fb5604997..35d6c12685e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -597,6 +597,8 @@ class IColumn; M(Int64, input_format_orc_row_batch_size, 100'000, "Batch size when reading ORC stripes.", 0) \ M(Bool, input_format_parquet_import_nested, false, "Allow to insert array of structs into Nested table in Parquet input format.", 0) \ M(Bool, input_format_allow_seeks, true, "Allow seeks while reading in ORC/Parquet/Arrow input formats", 0) \ + M(UInt64, input_format_msgpack_number_of_columns, 0, "The number of columns in inserted MsgPack data. Used for automatic schema inference from data.", 0) \ + M(UInt64, input_format_max_rows_to_read_for_schema_inference, 100, "The maximum rows of data to read for automatic schema inference", 0) \ \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ @@ -662,6 +664,7 @@ class IColumn; M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \ \ M(EnumComparingMode, format_capn_proto_enum_comparising_mode, FormatSettings::EnumComparingMode::BY_VALUES, "How to map ClickHouse Enum and CapnProto Enum", 0)\ + // End of FORMAT_FACTORY_SETTINGS // Please add settings non-related to formats into the COMMON_SETTINGS above. diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index e74df5c327a..85644b6f6ca 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -377,6 +377,8 @@ struct WhichDataType constexpr bool isNullable() const { return idx == TypeIndex::Nullable; } constexpr bool isFunction() const { return idx == TypeIndex::Function; } constexpr bool isAggregateFunction() const { return idx == TypeIndex::AggregateFunction; } + + constexpr bool isLowCarnality() const { return idx == TypeIndex::LowCardinality; } }; /// IDataType helpers (alternative for IDataType virtual methods with single point of truth) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index e9944b592ed..165bad950f5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -76,10 +76,16 @@ std::pair createTableFromAST( /// - the database has not been loaded yet; /// - the code is simpler, since the query is already brought to a suitable form. if (!ast_create_query.columns_list || !ast_create_query.columns_list->columns) - throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); - - columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, true); - constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints); + { + if (!StorageFactory::instance().checkIfStorageSupportsSchemaInterface(ast_create_query.storage->engine->name)) + throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED); + /// Leave columns empty. + } + else + { + columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, true); + constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints); + } } return diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index ecfa5df8351..a76428a45d2 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -7,6 +7,8 @@ #include #include #include +#include +#include #include #include #include @@ -26,6 +28,7 @@ namespace ErrorCodes extern const int FILE_DOESNT_EXIST; extern const int UNKNOWN_EXCEPTION; extern const int INCORRECT_DATA; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } capnp::StructSchema CapnProtoSchemaParser::getMessageSchema(const FormatSchemaInfo & schema_info) @@ -427,6 +430,112 @@ void checkCapnProtoSchemaStructure(const capnp::StructSchema & schema, const Blo } } +template +static DataTypePtr getEnumDataTypeFromEnumerants(const capnp::EnumSchema::EnumerantList & enumerants) +{ + std::vector> values; + for (auto enumerant : enumerants) + values.emplace_back(enumerant.getProto().getName(), ValueType(enumerant.getOrdinal())); + return std::make_shared>(std::move(values)); +} + +static DataTypePtr getEnumDataTypeFromEnumSchema(const capnp::EnumSchema & enum_schema) +{ + auto enumerants = enum_schema.getEnumerants(); + if (enumerants.size() < 128) + return getEnumDataTypeFromEnumerants(enumerants); + if (enumerants.size() < 32768) + return getEnumDataTypeFromEnumerants(enumerants); + + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "ClickHouse supports only 8 and 16-but Enums"); +} + +static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type) +{ + switch (capnp_type.which()) + { + case capnp::schema::Type::INT8: + return std::make_shared(); + case capnp::schema::Type::INT16: + return std::make_shared(); + case capnp::schema::Type::INT32: + return std::make_shared(); + case capnp::schema::Type::INT64: + return std::make_shared(); + case capnp::schema::Type::BOOL: [[fallthrough]]; + case capnp::schema::Type::UINT8: + return std::make_shared(); + case capnp::schema::Type::UINT16: + return std::make_shared(); + case capnp::schema::Type::UINT32: + return std::make_shared(); + case capnp::schema::Type::UINT64: + return std::make_shared(); + case capnp::schema::Type::FLOAT32: + return std::make_shared(); + case capnp::schema::Type::FLOAT64: + return std::make_shared(); + case capnp::schema::Type::DATA: [[fallthrough]]; + case capnp::schema::Type::TEXT: + return std::make_shared(); + case capnp::schema::Type::ENUM: + return getEnumDataTypeFromEnumSchema(capnp_type.asEnum()); + case capnp::schema::Type::LIST: + { + auto list_schema = capnp_type.asList(); + auto nested_type = getDataTypeFromCapnProtoType(list_schema.getElementType()); + return std::make_shared(nested_type); + } + case capnp::schema::Type::STRUCT: + { + auto struct_schema = capnp_type.asStruct(); + if (checkIfStructContainsUnnamedUnion(struct_schema)) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Unnamed union is not supported"); + + /// Check if it can be Nullable. + if (checkIfStructIsNamedUnion(struct_schema)) + { + auto fields = struct_schema.getUnionFields(); + if (fields.size() != 2 || (!fields[0].getType().isVoid() && !fields[1].getType().isVoid())) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Unions are not supported"); + auto value_type = fields[0].getType().isVoid() ? fields[1].getType() : fields[0].getType(); + if (value_type.isStruct() || value_type.isList()) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Tuples and Lists cannot be inside Nullable"); + + auto nested_type = getDataTypeFromCapnProtoType(value_type); + return std::make_shared(nested_type); + } + + /// Treat Struct as Tuple. + DataTypes nested_types; + Names nested_names; + for (auto field : struct_schema.getNonUnionFields()) + { + nested_names.push_back(field.getProto().getName()); + nested_types.push_back(getDataTypeFromCapnProtoType(field.getType())); + } + return std::make_shared(std::move(nested_types), std::move(nested_names)); + } + default: + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Unsupported CapnProtoType: {}", getCapnProtoFullTypeName(capnp_type)); + } +} + +NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema) +{ + if (checkIfStructContainsUnnamedUnion(schema)) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Unnamed union is not supported"); + + NamesAndTypesList names_and_types; + for (auto field : schema.getNonUnionFields()) + { + auto name = field.getProto().getName(); + auto type = getDataTypeFromCapnProtoType(field.getType()); + names_and_types.emplace_back(name, type); + } + return names_and_types; +} + } #endif diff --git a/src/Formats/CapnProtoUtils.h b/src/Formats/CapnProtoUtils.h index 93ca0a5e616..51c152de17f 100644 --- a/src/Formats/CapnProtoUtils.h +++ b/src/Formats/CapnProtoUtils.h @@ -38,6 +38,7 @@ capnp::DynamicValue::Reader getReaderByColumnName(const capnp::DynamicStruct::Re void checkCapnProtoSchemaStructure(const capnp::StructSchema & schema, const Block & header, FormatSettings::EnumComparingMode mode); +NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema); } #endif diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index d956d9e6bfb..4c7f5aa35f9 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -1,7 +1,16 @@ #include +#include +#include #include +#include +#include +#include #include #include +#include +#include +#include +#include namespace DB { @@ -9,6 +18,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; } FormatSettings::EscapingRule stringToEscapingRule(const String & escaping_rule) @@ -193,30 +203,149 @@ void writeStringByEscapingRule(const String & value, WriteBuffer & out, FormatSe } } -String readStringByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings) +template +String readByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings) { String result; switch (escaping_rule) { case FormatSettings::EscapingRule::Quoted: - readQuotedString(result, buf); + if constexpr (read_string) + readQuotedString(result, buf); + else + readQuotedFieldIntoString(result, buf); break; case FormatSettings::EscapingRule::JSON: - readJSONString(result, buf); + if constexpr (read_string) + readJSONString(result, buf); + else + readJSONFieldIntoString(result, buf); break; case FormatSettings::EscapingRule::Raw: readString(result, buf); break; case FormatSettings::EscapingRule::CSV: - readCSVString(result, buf, format_settings.csv); + if constexpr (read_string) + readCSVString(result, buf, format_settings.csv); + else + readCSVField(result, buf, format_settings.csv); break; case FormatSettings::EscapingRule::Escaped: readEscapedString(result, buf); break; default: - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read string with {} escaping rule", escapingRuleToString(escaping_rule)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read value with {} escaping rule", escapingRuleToString(escaping_rule)); } return result; } +String readFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings) +{ + return readByEscapingRule(buf, escaping_rule, format_settings); +} + +String readStringByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings) +{ + return readByEscapingRule(buf, escaping_rule, format_settings); +} + +static bool evaluateConstantExpressionFromString(const StringRef & field, DataTypePtr & type, ContextPtr context) +{ + if (!context) + throw Exception(ErrorCodes::LOGICAL_ERROR, "You must provide context to evaluate constant expression"); + + ParserExpression parser; + Expected expected; + Tokens tokens(field.data, field.data + field.size); + IParser::Pos token_iterator(tokens, context->getSettingsRef().max_parser_depth); + ASTPtr ast; + + /// FIXME: Our parser cannot parse maps in the form of '{key : value}' that is used in text formats. + bool parsed = parser.parse(token_iterator, ast, expected); + if (!parsed) + return false; + + try + { + std::pair result = evaluateConstantExpression(ast, context); + type = generalizeDataType(result.second); + return true; + } + catch (...) + { + return false; + } +} + +DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, ContextPtr context) +{ + switch (escaping_rule) + { + case FormatSettings::EscapingRule::Quoted: + { + DataTypePtr type; + bool parsed = evaluateConstantExpressionFromString(field, type, context); + return parsed ? type : nullptr; + } + case FormatSettings::EscapingRule::JSON: + { + Poco::JSON::Parser parser; + Poco::Dynamic::Var var = parser.parse(field); + return getDataTypeFromJSONField(var); + } + case FormatSettings::EscapingRule::CSV: + { + if (field.empty() || field == format_settings.csv.null_representation) + return nullptr; + + if (field == format_settings.bool_false_representation || field == format_settings.bool_true_representation) + return std::make_shared(); + + DataTypePtr type; + bool parsed; + if (field[0] == '\'' || field[0] == '"') + { + /// Try to evaluate expression inside quotes. + parsed = evaluateConstantExpressionFromString(StringRef(field.data() + 1, field.size() - 2), type, context); + /// If it's a number in quotes we determine it as a string. + if (parsed && type && isNumber(removeNullable(type))) + return makeNullable(std::make_shared()); + } + else + parsed = evaluateConstantExpressionFromString(field, type, context); + + /// If we couldn't parse an expression, determine it as a string. + return parsed ? type : makeNullable(std::make_shared()); + } + case FormatSettings::EscapingRule::Raw: [[fallthrough]]; + case FormatSettings::EscapingRule::Escaped: + /// TODO: Try to use some heuristics here to determine the type of data. + return field.empty() ? nullptr : makeNullable(std::make_shared()); + default: + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot determine the type for value with {} escaping rule", escapingRuleToString(escaping_rule)); + } +} + +DataTypes determineDataTypesByEscapingRule(const std::vector & fields, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, ContextPtr context) +{ + DataTypes data_types; + data_types.reserve(fields.size()); + for (const auto & field : fields) + data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, escaping_rule, context)); + return data_types; +} + +DataTypePtr getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule escaping_rule) +{ + switch (escaping_rule) + { + case FormatSettings::EscapingRule::CSV: [[fallthrough]]; + case FormatSettings::EscapingRule::Escaped: [[fallthrough]]; + case FormatSettings::EscapingRule::Raw: + return makeNullable(std::make_shared()); + default: + return nullptr; + } +} + } diff --git a/src/Formats/EscapingRuleUtils.h b/src/Formats/EscapingRuleUtils.h index 02f027db74d..10147b29ad6 100644 --- a/src/Formats/EscapingRuleUtils.h +++ b/src/Formats/EscapingRuleUtils.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -33,5 +34,24 @@ void serializeFieldByEscapingRule( void writeStringByEscapingRule(const String & value, WriteBuffer & out, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings); String readStringByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings); +String readFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings); + +/// Try to determine the type of the field written by a specific escaping rule. +/// If cannot, return nullptr. +/// - For Quoted escaping rule we can interpret a single field as a constant +/// expression and get it's type by evaluation this expression. +/// - For JSON escaping rule we can use JSON parser to parse a single field +/// and then convert JSON type of this field to ClickHouse type. +/// - For CSV escaping rule we can do the next: +/// - If the field is an unquoted string, then we could try to evaluate it +/// as a constant expression, and if it fails, treat it as a String. +/// - If the field is a string in quotes, then we can try to evaluate +/// expression inside quotes as a constant expression, and if it fails or +/// the result is a number (we don't parse numbers in quotes) we treat it as a String. +/// - For TSV and TSVRaw we treat each field as a String (TODO: try to use some tweaks and heuristics here) +DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, ContextPtr context = nullptr); +DataTypes determineDataTypesByEscapingRule(const std::vector & fields, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, ContextPtr context = nullptr); + +DataTypePtr getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule escaping_rule); } diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index d292bbf551c..568a65f5003 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -13,9 +13,6 @@ #include #include -#include -#include - namespace DB { @@ -119,6 +116,8 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.defaults_for_omitted_fields = settings.input_format_defaults_for_omitted_fields; format_settings.capn_proto.enum_comparing_mode = settings.format_capn_proto_enum_comparising_mode; format_settings.seekable_read = settings.input_format_allow_seeks; + format_settings.msgpack.number_of_columns = settings.input_format_msgpack_number_of_columns; + format_settings.max_rows_to_read_for_schema_inference = settings.input_format_max_rows_to_read_for_schema_inference; /// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context if (format_settings.schema.is_server) @@ -200,7 +199,6 @@ InputFormatPtr FormatFactory::getInput( return format; } - InputFormatPtr FormatFactory::getInputFormat( const String & name, ReadBuffer & buf, @@ -325,6 +323,32 @@ String FormatFactory::getContentType( return format->getContentType(); } +SchemaReaderPtr FormatFactory::getSchemaReader( + const String & name, + ReadBuffer & buf, + ContextPtr context, + const std::optional & _format_settings) const +{ + const auto & schema_reader_creator = dict.at(name).schema_reader_creator; + if (!schema_reader_creator) + throw Exception("FormatFactory: Format " + name + " doesn't support schema inference.", ErrorCodes::LOGICAL_ERROR); + + auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); + return schema_reader_creator(buf, format_settings, context); +} + +ExternalSchemaReaderPtr FormatFactory::getExternalSchemaReader( + const String & name, + ContextPtr context, + const std::optional & _format_settings) const +{ + const auto & external_schema_reader_creator = dict.at(name).external_schema_reader_creator; + if (!external_schema_reader_creator) + throw Exception("FormatFactory: Format " + name + " doesn't support schema inference.", ErrorCodes::LOGICAL_ERROR); + + auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); + return external_schema_reader_creator(format_settings); +} void FormatFactory::registerInputFormat(const String & name, InputCreator input_creator) { @@ -358,6 +382,21 @@ void FormatFactory::registerFileSegmentationEngine(const String & name, FileSegm target = std::move(file_segmentation_engine); } +void FormatFactory::registerSchemaReader(const String & name, SchemaReaderCreator schema_reader_creator) +{ + auto & target = dict[name].schema_reader_creator; + if (target) + throw Exception("FormatFactory: Schema reader " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); + target = std::move(schema_reader_creator); +} + +void FormatFactory::registerExternalSchemaReader(const String & name, ExternalSchemaReaderCreator external_schema_reader_creator) +{ + auto & target = dict[name].external_schema_reader_creator; + if (target) + throw Exception("FormatFactory: Schema reader " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); + target = std::move(external_schema_reader_creator); +} void FormatFactory::markOutputFormatSupportsParallelFormatting(const String & name) { @@ -395,6 +434,23 @@ bool FormatFactory::isOutputFormat(const String & name) const return it != dict.end() && it->second.output_creator; } +bool FormatFactory::checkIfFormatHasSchemaReader(const String & name) +{ + const auto & target = getCreators(name); + return bool(target.schema_reader_creator); +} + +bool FormatFactory::checkIfFormatHasExternalSchemaReader(const String & name) +{ + const auto & target = getCreators(name); + return bool(target.external_schema_reader_creator); +} + +bool FormatFactory::checkIfFormatHasAnySchemaReader(const String & name) +{ + return checkIfFormatHasSchemaReader(name) || checkIfFormatHasExternalSchemaReader(name); +} + FormatFactory & FormatFactory::instance() { static FormatFactory ret; diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index ea285c47996..a62b32da0cc 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -4,7 +4,9 @@ #include #include #include +#include #include +#include #include @@ -31,6 +33,11 @@ class IOutputFormat; struct RowInputFormatParams; struct RowOutputFormatParams; +class ISchemaReader; +class IExternalSchemaReader; +using SchemaReaderPtr = std::shared_ptr; +using ExternalSchemaReaderPtr = std::shared_ptr; + using InputFormatPtr = std::shared_ptr; using OutputFormatPtr = std::shared_ptr; @@ -85,11 +92,16 @@ private: /// The checker should return true if parallel parsing should be disabled. using NonTrivialPrefixAndSuffixChecker = std::function; + using SchemaReaderCreator = std::function; + using ExternalSchemaReaderCreator = std::function; + struct Creators { InputCreator input_creator; OutputCreator output_creator; FileSegmentationEngine file_segmentation_engine; + SchemaReaderCreator schema_reader_creator; + ExternalSchemaReaderCreator external_schema_reader_creator; bool supports_parallel_formatting{false}; bool is_column_oriented{false}; NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker; @@ -138,6 +150,17 @@ public: ContextPtr context, const std::optional & format_settings = std::nullopt) const; + SchemaReaderPtr getSchemaReader( + const String & name, + ReadBuffer & buf, + ContextPtr context, + const std::optional & format_settings = std::nullopt) const; + + ExternalSchemaReaderPtr getExternalSchemaReader( + const String & name, + ContextPtr context, + const std::optional & format_settings = std::nullopt) const; + void registerFileSegmentationEngine(const String & name, FileSegmentationEngine file_segmentation_engine); void registerNonTrivialPrefixAndSuffixChecker(const String & name, NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker); @@ -146,11 +169,19 @@ public: void registerInputFormat(const String & name, InputCreator input_creator); void registerOutputFormat(const String & name, OutputCreator output_creator); + /// Register schema readers for format its name. + void registerSchemaReader(const String & name, SchemaReaderCreator schema_reader_creator); + void registerExternalSchemaReader(const String & name, ExternalSchemaReaderCreator external_schema_reader_creator); + void markOutputFormatSupportsParallelFormatting(const String & name); void markFormatAsColumnOriented(const String & name); bool checkIfFormatIsColumnOriented(const String & name); + bool checkIfFormatHasSchemaReader(const String & name); + bool checkIfFormatHasExternalSchemaReader(const String & name); + bool checkIfFormatHasAnySchemaReader(const String & name); + const FormatsDictionary & getAllFormats() const { return dict; @@ -163,6 +194,7 @@ private: FormatsDictionary dict; const Creators & getCreators(const String & name) const; + }; } diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index d9af07fdc9c..6298e959c3e 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -33,6 +33,7 @@ struct FormatSettings bool defaults_for_omitted_fields = true; bool seekable_read = true; + UInt64 max_rows_to_read_for_schema_inference = 100; enum class DateTimeInputFormat { @@ -217,6 +218,11 @@ struct FormatSettings { EnumComparingMode enum_comparing_mode = EnumComparingMode::BY_VALUES; } capn_proto; + + struct + { + UInt64 number_of_columns = 0; + } msgpack; }; } diff --git a/src/Formats/JSONEachRowUtils.cpp b/src/Formats/JSONEachRowUtils.cpp index b55e9f59cc7..df4ee77e586 100644 --- a/src/Formats/JSONEachRowUtils.cpp +++ b/src/Formats/JSONEachRowUtils.cpp @@ -1,7 +1,14 @@ #include #include +#include #include #include +#include +#include +#include +#include +#include +#include #include @@ -92,6 +99,148 @@ static std::pair fileSegmentationEngineJSONEachRowImpl(ReadBuffer return {loadAtPosition(in, memory, pos), number_of_rows}; } +template +static String readJSONEachRowLineIntoStringImpl(ReadBuffer & in) +{ + skipWhitespaceIfAny(in); + + if (in.eof()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read JSON object: unexpected end of file"); + + char * pos = in.position(); + if (*pos != opening_bracket) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read JSONEachRow line: {} expected, {} got", opening_bracket, *in.position()); + ++pos; + + Memory memory; + size_t balance = 1; + bool quotes = false; + while (loadAtPosition(in, memory, pos) && balance) + { + if (quotes) + { + pos = find_first_symbols<'\\', '"'>(pos, in.buffer().end()); + + if (pos == in.buffer().end()) + continue; + + if (*pos == '\\') + { + ++pos; + if (loadAtPosition(in, memory, pos)) + ++pos; + } + else if (*pos == '"') + { + ++pos; + quotes = false; + } + } + else + { + pos = find_first_symbols(pos, in.buffer().end()); + + if (pos == in.buffer().end()) + continue; + + else if (*pos == opening_bracket) + { + ++balance; + ++pos; + } + else if (*pos == closing_bracket) + { + --balance; + ++pos; + } + else if (*pos == '\\') + { + ++pos; + if (loadAtPosition(in, memory, pos)) + ++pos; + } + else if (*pos == '"') + { + quotes = true; + ++pos; + } + } + } + + if (balance) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read JSON object: unexpected end of file"); + + saveUpToPosition(in, memory, pos); + return String(memory.data(), memory.size()); +} + +DataTypePtr getDataTypeFromJSONField(const Poco::Dynamic::Var & field) +{ + if (field.isEmpty()) + return nullptr; + + if (field.isBoolean()) + return makeNullable(std::make_shared()); + + if (field.isNumeric()) + return makeNullable(std::make_shared()); + + if (field.isString()) + return makeNullable(std::make_shared()); + + if (field.isArray()) + { + Poco::JSON::Array::Ptr array = field.extract(); + + /// Return nullptr in case of empty array because we cannot determine nested type. + if (array->size() == 0) + return nullptr; + + DataTypes nested_data_types; + /// If this array contains fields with different types we will treat it as Tuple. + bool is_tuple = false; + for (size_t i = 0; i != array->size(); ++i) + { + auto type = getDataTypeFromJSONField(array->get(i)); + if (!type) + return nullptr; + + if (!nested_data_types.empty() && type->getName() != nested_data_types.back()->getName()) + is_tuple = true; + + nested_data_types.push_back(std::move(type)); + } + + if (is_tuple) + return std::make_shared(nested_data_types); + + return std::make_shared(nested_data_types.back()); + } + + throw Exception{ErrorCodes::INCORRECT_DATA, "Unexpected JSON type {}", field.type().name()}; +} + +using JSONEachRowFieldExtractor = std::function(const Poco::Dynamic::Var &)>; + +template +static DataTypes determineColumnDataTypesFromJSONEachRowDataImpl(ReadBuffer & in, bool /*json_strings*/, JSONEachRowFieldExtractor extractor) +{ + Poco::JSON::Parser parser; + DataTypes data_types; + + String line = readJSONEachRowLineIntoStringImpl(in); + auto var = parser.parse(line); + std::vector fields = extractor(var); + data_types.reserve(fields.size()); + for (const auto & field : fields) + data_types.push_back(getDataTypeFromJSONField(field)); + + /// TODO: For JSONStringsEachRow/JSONCompactStringsEach all types will be strings. + /// Should we try to parse data inside strings somehow in this case? + + return data_types; +} + std::pair fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) { return fileSegmentationEngineJSONEachRowImpl<'{', '}'>(in, memory, min_chunk_size, 1); @@ -102,6 +251,46 @@ std::pair fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in return fileSegmentationEngineJSONEachRowImpl<'[', ']'>(in, memory, min_chunk_size, min_rows); } +std::unordered_map readRowAndGetNamesAndDataTypesForJSONEachRow(ReadBuffer & in, bool json_strings) +{ + std::vector column_names; + + /// {..., "" : , ...} + auto extractor = [&](const Poco::Dynamic::Var & var) + { + Poco::JSON::Object::Ptr object = var.extract(); + column_names = object->getNames(); + + std::vector fields; + for (size_t i = 0; i != object->size(); ++i) + fields.push_back(object->get(column_names[i])); + return fields; + }; + + auto data_types = determineColumnDataTypesFromJSONEachRowDataImpl<'{', '}'>(in, json_strings, extractor); + std::unordered_map result; + for (size_t i = 0; i != column_names.size(); ++i) + result[column_names[i]] = data_types[i]; + return result; +} + +DataTypes readRowAndGetDataTypesForJSONCompactEachRow(ReadBuffer & in, bool json_strings) +{ + /// [..., , ...] + auto extractor = [](const Poco::Dynamic::Var & var) + { + Poco::JSON::Array::Ptr array = var.extract(); + std::vector fields; + fields.reserve(array->size()); + for (size_t i = 0; i != array->size(); ++i) + fields.push_back(array->get(i)); + return fields; + }; + + return determineColumnDataTypesFromJSONEachRowDataImpl<'[', ']'>(in, json_strings, extractor); +} + + bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf) { /// For JSONEachRow we can safely skip whitespace characters diff --git a/src/Formats/JSONEachRowUtils.h b/src/Formats/JSONEachRowUtils.h index 4a049aa1abd..68dc737a6df 100644 --- a/src/Formats/JSONEachRowUtils.h +++ b/src/Formats/JSONEachRowUtils.h @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace DB @@ -11,6 +12,21 @@ namespace DB std::pair fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size); std::pair fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, size_t min_rows); + +/// Convert JSON type to ClickHouse type. Make the result type always Nullable. +/// JSON array with different nested types is treated as Tuple. +/// If cannot convert (for example when field contains null), return nullptr. +DataTypePtr getDataTypeFromJSONField(const Poco::Dynamic::Var & field); + +/// Read row in JSONEachRow format and try to determine type for each field. +/// Return map {column_name : type}. +/// If cannot determine the type of some field, return nullptr for it. +std::unordered_map readRowAndGetNamesAndDataTypesForJSONEachRow(ReadBuffer & in, bool json_strings); + +/// Read row in JSONCompactEachRow format and try to determine type for each field. +/// If cannot determine the type of some field, return nullptr for it. +DataTypes readRowAndGetDataTypesForJSONCompactEachRow(ReadBuffer & in, bool json_strings); + bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf); bool readFieldImpl(ReadBuffer & in, IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, const String & column_name, const FormatSettings & format_settings, bool yield_strings); diff --git a/src/Formats/ParsedTemplateFormatString.cpp b/src/Formats/ParsedTemplateFormatString.cpp index 4966420f05b..8d1b987d01a 100644 --- a/src/Formats/ParsedTemplateFormatString.cpp +++ b/src/Formats/ParsedTemplateFormatString.cpp @@ -14,14 +14,14 @@ namespace ErrorCodes extern const int INVALID_TEMPLATE_FORMAT; } -ParsedTemplateFormatString::ParsedTemplateFormatString(const FormatSchemaInfo & schema, const ColumnIdxGetter & idx_by_name) +ParsedTemplateFormatString::ParsedTemplateFormatString(const FormatSchemaInfo & schema, const ColumnIdxGetter & idx_by_name, bool allow_indexes) { ReadBufferFromFile schema_file(schema.absoluteSchemaPath(), 4096); String format_string; readStringUntilEOF(format_string, schema_file); try { - parse(format_string, idx_by_name); + parse(format_string, idx_by_name, allow_indexes); } catch (DB::Exception & e) { @@ -33,7 +33,7 @@ ParsedTemplateFormatString::ParsedTemplateFormatString(const FormatSchemaInfo & } -void ParsedTemplateFormatString::parse(const String & format_string, const ColumnIdxGetter & idx_by_name) +void ParsedTemplateFormatString::parse(const String & format_string, const ColumnIdxGetter & idx_by_name, bool allow_indexes) { enum ParserState { @@ -100,6 +100,8 @@ void ParsedTemplateFormatString::parse(const String & format_string, const Colum column_idx = strtoull(column_names.back().c_str(), &col_idx_end, 10); if (col_idx_end != column_names.back().c_str() + column_names.back().size() || errno) column_idx = idx_by_name(column_names.back()); + else if (!allow_indexes) + throw Exception(ErrorCodes::INVALID_TEMPLATE_FORMAT, "Indexes instead of names are not allowed"); } format_idx_to_column_idx.emplace_back(column_idx); break; diff --git a/src/Formats/ParsedTemplateFormatString.h b/src/Formats/ParsedTemplateFormatString.h index ba0ebdf5aa8..c5617d0f0ef 100644 --- a/src/Formats/ParsedTemplateFormatString.h +++ b/src/Formats/ParsedTemplateFormatString.h @@ -31,9 +31,9 @@ struct ParsedTemplateFormatString typedef std::function(const String &)> ColumnIdxGetter; ParsedTemplateFormatString() = default; - ParsedTemplateFormatString(const FormatSchemaInfo & schema, const ColumnIdxGetter & idx_by_name); + ParsedTemplateFormatString(const FormatSchemaInfo & schema, const ColumnIdxGetter & idx_by_name, bool allow_indexes = true); - void parse(const String & format_string, const ColumnIdxGetter & idx_by_name); + void parse(const String & format_string, const ColumnIdxGetter & idx_by_name, bool allow_indexes = true); static const char * readMayBeQuotedColumnNameInto(const char * pos, size_t size, String & s); size_t columnsCount() const; diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index 5232b76b7fe..72ac94865e2 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -24,6 +24,7 @@ # include # include # include +# include # include # include # include @@ -56,6 +57,7 @@ namespace ErrorCodes extern const int PROTOBUF_FIELD_NOT_REPEATED; extern const int PROTOBUF_BAD_CAST; extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } namespace @@ -3230,8 +3232,98 @@ namespace std::function get_root_desc_function; std::shared_ptr root_serializer_ptr; }; -} + template + DataTypePtr getEnumDataType(const google::protobuf::EnumDescriptor * enum_descriptor) + { + std::vector> values; + for (int i = 0; i != enum_descriptor->value_count(); ++i) + { + const auto * enum_value_descriptor = enum_descriptor->value(i); + values.emplace_back(enum_value_descriptor->name(), enum_value_descriptor->number()); + } + return std::make_shared>(std::move(values)); + } + + NameAndTypePair getNameAndDataTypeFromField(const google::protobuf::FieldDescriptor * field_descriptor, bool allow_repeat = true) + { + if (allow_repeat && field_descriptor->is_repeated()) + { + auto name_and_type = getNameAndDataTypeFromField(field_descriptor, false); + return {name_and_type.name, std::make_shared(name_and_type.type)}; + } + + switch (field_descriptor->type()) + { + case FieldTypeId::TYPE_SFIXED32: [[fallthrough]]; + case FieldTypeId::TYPE_SINT32: [[fallthrough]]; + case FieldTypeId::TYPE_INT32: + return {field_descriptor->name(), std::make_shared()}; + case FieldTypeId::TYPE_SFIXED64: [[fallthrough]]; + case FieldTypeId::TYPE_SINT64: [[fallthrough]]; + case FieldTypeId::TYPE_INT64: + return {field_descriptor->name(), std::make_shared()}; + case FieldTypeId::TYPE_BOOL: + return {field_descriptor->name(), std::make_shared()}; + case FieldTypeId::TYPE_FLOAT: + return {field_descriptor->name(), std::make_shared()}; + case FieldTypeId::TYPE_DOUBLE: + return {field_descriptor->name(), std::make_shared()}; + case FieldTypeId::TYPE_UINT32: [[fallthrough]]; + case FieldTypeId::TYPE_FIXED32: + return {field_descriptor->name(), std::make_shared()}; + case FieldTypeId::TYPE_UINT64: [[fallthrough]]; + case FieldTypeId::TYPE_FIXED64: + return {field_descriptor->name(), std::make_shared()}; + case FieldTypeId::TYPE_BYTES: [[fallthrough]]; + case FieldTypeId::TYPE_STRING: + return {field_descriptor->name(), std::make_shared()}; + case FieldTypeId::TYPE_ENUM: + { + const auto * enum_descriptor = field_descriptor->enum_type(); + if (enum_descriptor->value_count() == 0) + throw Exception("Empty enum field", ErrorCodes::BAD_ARGUMENTS); + int max_abs = std::abs(enum_descriptor->value(0)->number()); + for (int i = 1; i != enum_descriptor->value_count(); ++i) + { + if (std::abs(enum_descriptor->value(i)->number()) > max_abs) + max_abs = std::abs(enum_descriptor->value(i)->number()); + } + if (max_abs < 128) + return {field_descriptor->name(), getEnumDataType(enum_descriptor)}; + else if (max_abs < 32768) + return {field_descriptor->name(), getEnumDataType(enum_descriptor)}; + else + throw Exception("ClickHouse supports only 8-bit and 16-bit enums", ErrorCodes::BAD_ARGUMENTS); + } + case FieldTypeId::TYPE_GROUP: [[fallthrough]]; + case FieldTypeId::TYPE_MESSAGE: + { + const auto * message_descriptor = field_descriptor->message_type(); + if (message_descriptor->field_count() == 1) + { + const auto * nested_field_descriptor = message_descriptor->field(0); + auto nested_name_and_type = getNameAndDataTypeFromField(nested_field_descriptor); + return {field_descriptor->name() + "_" + nested_name_and_type.name, nested_name_and_type.type}; + } + else + { + DataTypes nested_types; + Strings nested_names; + for (int i = 0; i != message_descriptor->field_count(); ++i) + { + auto nested_name_and_type = getNameAndDataTypeFromField(message_descriptor->field(i)); + nested_types.push_back(nested_name_and_type.type); + nested_names.push_back(nested_name_and_type.name); + } + return {field_descriptor->name(), std::make_shared(std::move(nested_types), std::move(nested_names))}; + } + } + } + + __builtin_unreachable(); + } +} std::unique_ptr ProtobufSerializer::create( const Strings & column_names, @@ -3254,5 +3346,14 @@ std::unique_ptr ProtobufSerializer::create( std::vector missing_column_indices; return ProtobufSerializerBuilder(writer).buildMessageSerializer(column_names, data_types, missing_column_indices, message_descriptor, with_length_delimiter); } + +NamesAndTypesList protobufSchemaToCHSchema(const google::protobuf::Descriptor * message_descriptor) +{ + NamesAndTypesList schema; + for (int i = 0; i != message_descriptor->field_count(); ++i) + schema.push_back(getNameAndDataTypeFromField(message_descriptor->field(i))); + return schema; +} + } #endif diff --git a/src/Formats/ProtobufSerializer.h b/src/Formats/ProtobufSerializer.h index 3eaca6a18d6..d9bed913517 100644 --- a/src/Formats/ProtobufSerializer.h +++ b/src/Formats/ProtobufSerializer.h @@ -4,6 +4,7 @@ #if USE_PROTOBUF # include +#include namespace google::protobuf { class Descriptor; } @@ -48,5 +49,7 @@ public: ProtobufWriter & writer); }; +NamesAndTypesList protobufSchemaToCHSchema(const google::protobuf::Descriptor * message_descriptor); + } #endif diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp new file mode 100644 index 00000000000..967cc3de54e --- /dev/null +++ b/src/Formats/ReadSchemaUtils.cpp @@ -0,0 +1,112 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + extern const int BAD_ARGUMENTS; +} + +ColumnsDescription readSchemaFromFormat(const String & format_name, const std::optional & format_settings, ReadBufferCreator read_buffer_creator, ContextPtr context) +{ + NamesAndTypesList names_and_types; + if (FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format_name)) + { + auto external_schema_reader = FormatFactory::instance().getExternalSchemaReader(format_name, context, format_settings); + try + { + names_and_types = external_schema_reader->readSchema(); + } + catch (const DB::Exception & e) + { + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file. Error: {}", format_name, e.message()); + } + } + else if (FormatFactory::instance().checkIfFormatHasSchemaReader(format_name)) + { + auto read_buf = read_buffer_creator(); + if (read_buf->eof()) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file, file is empty", format_name); + + auto schema_reader = FormatFactory::instance().getSchemaReader(format_name, *read_buf, context, format_settings); + try + { + names_and_types = schema_reader->readSchema(); + } + catch (const DB::Exception & e) + { + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file. Error: {}", format_name, e.message()); + } + } + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} file format doesn't support schema inference", format_name); + + return ColumnsDescription(names_and_types); +} + +DataTypePtr generalizeDataType(DataTypePtr type) +{ + WhichDataType which(type); + + if (which.isNothing()) + return nullptr; + + if (which.isNullable()) + { + const auto * nullable_type = assert_cast(type.get()); + return generalizeDataType(nullable_type->getNestedType()); + } + + if (isNumber(type)) + return makeNullable(std::make_shared()); + + if (which.isArray()) + { + const auto * array_type = assert_cast(type.get()); + auto nested_type = generalizeDataType(array_type->getNestedType()); + return nested_type ? std::make_shared(nested_type) : nullptr; + } + + if (which.isTuple()) + { + const auto * tuple_type = assert_cast(type.get()); + DataTypes nested_types; + for (const auto & element : tuple_type->getElements()) + { + auto nested_type = generalizeDataType(element); + if (!nested_type) + return nullptr; + nested_types.push_back(nested_type); + } + return std::make_shared(std::move(nested_types)); + } + + if (which.isMap()) + { + const auto * map_type = assert_cast(type.get()); + auto key_type = generalizeDataType(map_type->getKeyType()); + auto value_type = generalizeDataType(map_type->getValueType()); + return key_type && value_type ? std::make_shared(key_type, value_type) : nullptr; + } + + if (which.isLowCarnality()) + { + const auto * lc_type = assert_cast(type.get()); + auto nested_type = generalizeDataType(lc_type->getDictionaryType()); + return nested_type ? std::make_shared(nested_type) : nullptr; + } + + return makeNullable(type); +} + +} diff --git a/src/Formats/ReadSchemaUtils.h b/src/Formats/ReadSchemaUtils.h new file mode 100644 index 00000000000..3fba6ad9f70 --- /dev/null +++ b/src/Formats/ReadSchemaUtils.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// Try to determine the schema of the data in specifying format. +/// For formats that have an external schema reader, it will +/// use it and won't create a read buffer. +/// For formats that have a schema reader from the data, +/// read buffer will be created by the provided creator and +/// the schema will be extracted from the data. +/// If format doesn't have any schema reader or a schema reader +/// couldn't determine the schema, an exception will be thrown. +using ReadBufferCreator = std::function()>; +ColumnsDescription readSchemaFromFormat(const String & format_name, const std::optional & format_settings, ReadBufferCreator read_buffer_creator, ContextPtr context); + +/// Convert type to the most general type: +/// - IntN, UIntN, FloatN, Decimal -> Float64 +/// - Type -> Nullable(type) +/// - Array(Type) -> Array(Nullable(Type)) +/// - Tuple(Type1, ..., TypeN) -> Tuple(Nullable(Type1), ..., Nullable(TypeN)) +/// - Map(KeyType, ValueType) -> Map(Nullable(KeyType), Nullable(ValueType)) +/// - LowCardinality(Type) -> LowCardinality(Nullable(Type)) +/// If type is Nothing or one of the nested types is Nothing, return nullptr. +DataTypePtr generalizeDataType(DataTypePtr type); + +} diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp index 7425c6898de..1349c9e3323 100644 --- a/src/Formats/registerFormats.cpp +++ b/src/Formats/registerFormats.cpp @@ -81,6 +81,28 @@ void registerInputFormatCapnProto(FormatFactory & factory); void registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(FormatFactory & factory); void registerNonTrivialPrefixAndSuffixCheckerJSONAsString(FormatFactory & factory); +void registerArrowSchemaReader(FormatFactory & factory); +void registerParquetSchemaReader(FormatFactory & factory); +void registerORCSchemaReader(FormatFactory & factory); +void registerTSVSchemaReader(FormatFactory & factory); +void registerCSVSchemaReader(FormatFactory & factory); +void registerJSONCompactEachRowSchemaReader(FormatFactory & factory); +void registerJSONEachRowSchemaReader(FormatFactory & factory); +void registerNativeSchemaReader(FormatFactory & factory); +void registerRowBinaryWithNamesAndTypesSchemaReader(FormatFactory & factory); +void registerAvroSchemaReader(FormatFactory & factory); +void registerProtobufSchemaReader(FormatFactory & factory); +void registerLineAsStringSchemaReader(FormatFactory & factory); +void registerJSONAsStringSchemaReader(FormatFactory & factory); +void registerRawBLOBSchemaReader(FormatFactory & factory); +void registerMsgPackSchemaReader(FormatFactory & factory); +void registerCapnProtoSchemaReader(FormatFactory & factory); +void registerCustomSeparatedSchemaReader(FormatFactory & factory); +void registerRegexpSchemaReader(FormatFactory & factory); +void registerTSKVSchemaReader(FormatFactory & factory); +void registerValuesSchemaReader(FormatFactory & factory); +void registerTemplateSchemaReader(FormatFactory & factory); + void registerFormats() { auto & factory = FormatFactory::instance(); @@ -152,6 +174,28 @@ void registerFormats() registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(factory); registerNonTrivialPrefixAndSuffixCheckerJSONAsString(factory); + + registerArrowSchemaReader(factory); + registerParquetSchemaReader(factory); + registerORCSchemaReader(factory); + registerTSVSchemaReader(factory); + registerCSVSchemaReader(factory); + registerJSONCompactEachRowSchemaReader(factory); + registerJSONEachRowSchemaReader(factory); + registerNativeSchemaReader(factory); + registerRowBinaryWithNamesAndTypesSchemaReader(factory); + registerAvroSchemaReader(factory); + registerProtobufSchemaReader(factory); + registerLineAsStringSchemaReader(factory); + registerJSONAsStringSchemaReader(factory); + registerRawBLOBSchemaReader(factory); + registerMsgPackSchemaReader(factory); + registerCapnProtoSchemaReader(factory); + registerCustomSeparatedSchemaReader(factory); + registerRegexpSchemaReader(factory); + registerTSKVSchemaReader(factory); + registerValuesSchemaReader(factory); + registerTemplateSchemaReader(factory); } } diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index b0a6838b81e..48811a41edd 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -702,6 +702,25 @@ void readCSVString(String & s, ReadBuffer & buf, const FormatSettings::CSV & set readCSVStringInto(s, buf, settings); } +void readCSVField(String & s, ReadBuffer & buf, const FormatSettings::CSV & settings) +{ + s.clear(); + bool add_quote = false; + char quote = '\''; + + if (!buf.eof() && (*buf.position() == '\'' || *buf.position() == '"')) + { + quote = *buf.position(); + s.push_back(quote); + add_quote = true; + } + + readCSVStringInto(s, buf, settings); + + if (add_quote) + s.push_back(quote); +} + template void readCSVStringInto>(PaddedPODArray & s, ReadBuffer & buf, const FormatSettings::CSV & settings); @@ -1212,6 +1231,19 @@ void skipToNextRowOrEof(PeekableReadBuffer & buf, const String & row_after_delim } } +// Use PeekableReadBuffer to copy field to string after parsing. +template +static void readParsedValueIntoString(String & s, ReadBuffer & buf, ParseFunc parse_func) +{ + PeekableReadBuffer peekable_buf(buf); + peekable_buf.setCheckpoint(); + parse_func(peekable_buf); + peekable_buf.makeContinuousMemoryFromCheckpointToPos(); + auto * end = peekable_buf.position(); + peekable_buf.rollbackToCheckpoint(); + s.append(peekable_buf.position(), end); + peekable_buf.position() = end; +} template static void readQuotedFieldInBrackets(String & s, ReadBuffer & buf) @@ -1266,7 +1298,11 @@ void readQuotedFieldIntoString(String & s, ReadBuffer & buf) /// - Number: integer, float, decimal. if (*buf.position() == '\'') - readQuotedString(s, buf); + { + s.push_back('\''); + readQuotedStringInto(s, buf); + s.push_back('\''); + } else if (*buf.position() == '[') readQuotedFieldInBrackets<'[', ']'>(s, buf); else if (*buf.position() == '(') @@ -1290,18 +1326,19 @@ void readQuotedFieldIntoString(String & s, ReadBuffer & buf) else { /// It's an integer, float or decimal. They all can be parsed as float. - /// Use PeekableReadBuffer to copy field to string after parsing. - PeekableReadBuffer peekable_buf(buf); - peekable_buf.setCheckpoint(); - Float64 tmp; - readFloatText(tmp, peekable_buf); - peekable_buf.makeContinuousMemoryFromCheckpointToPos(); - auto * end = peekable_buf.position(); - peekable_buf.rollbackToCheckpoint(); - s.append(peekable_buf.position(), end); - peekable_buf.position() = end; + auto parse_func = [](ReadBuffer & in) + { + Float64 tmp; + readFloatText(tmp, in); + }; + readParsedValueIntoString(s, buf, parse_func); } } +void readJSONFieldIntoString(String & s, ReadBuffer & buf) +{ + auto parse_func = [](ReadBuffer & in) { skipJSONField(in, "json_field"); }; + readParsedValueIntoString(s, buf, parse_func); +} } diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index b2ad4035cdc..6d1023947a5 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -563,6 +563,8 @@ void readStringUntilWhitespace(String & s, ReadBuffer & buf); */ void readCSVString(String & s, ReadBuffer & buf, const FormatSettings::CSV & settings); +/// Differ from readCSVString in that it doesn't remove quotes around field if any. +void readCSVField(String & s, ReadBuffer & buf, const FormatSettings::CSV & settings); /// Read and append result to array of characters. template @@ -1381,4 +1383,7 @@ struct PcgDeserializer void readQuotedFieldIntoString(String & s, ReadBuffer & buf); +void readJSONFieldIntoString(String & s, ReadBuffer & buf); + } + diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 8f003e75a07..7ddb0c8c26e 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -637,13 +637,14 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti /// Table function without columns list. auto table_function = TableFunctionFactory::instance().get(create.as_table_function, getContext()); properties.columns = table_function->getActualTableStructure(getContext()); - assert(!properties.columns.empty()); } else if (create.is_dictionary) { return {}; } - else + /// We can have queries like "CREATE TABLE ENGINE=" if + /// supports schema inference (will determine table structure in it's constructor). + else if (!StorageFactory::instance().checkIfStorageSupportsSchemaInterface(create.storage->engine->name)) throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY); /// Even if query has list of columns, canonicalize it (unfold Nested columns). @@ -1083,7 +1084,10 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, { const auto & factory = TableFunctionFactory::instance(); auto table_func = factory.get(create.as_table_function, getContext()); - res = table_func->execute(create.as_table_function, getContext(), create.getTable(), properties.columns); + /// In case of CREATE AS table_function() query we should use global context + /// in storage creation because there will be no query context on server startup + /// and because storage lifetime is bigger than query context lifetime. + res = table_func->execute(create.as_table_function, getContext(), create.getTable(), properties.columns, /*use_global_context=*/true); res->renameInMemory({create.getDatabase(), create.getTable(), create.uuid}); } else diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index 3e77bee19a9..e61a0f55142 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -359,7 +359,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat if (as_table_function) { - if (columns_list) + if (columns_list && !columns_list->empty()) { frame.expression_list_always_start_on_new_line = true; settings.ostr << (settings.one_line ? " (" : "\n("); @@ -375,7 +375,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat frame.expression_list_always_start_on_new_line = true; - if (columns_list && !as_table_function) + if (columns_list && !columns_list->empty() && !as_table_function) { settings.ostr << (settings.one_line ? " (" : "\n("); FormatStateStacked frame_nested = frame; diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index 93fced7dba5..2e35731acad 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -50,6 +50,12 @@ public: ASTPtr clone() const override; void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + + bool empty() + { + return (!columns || columns->children.empty()) && (!indices || indices->children.empty()) && (!constraints || constraints->children.empty()) + && (!projections || projections->children.empty()); + } }; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index dbbea986404..7f47e1efb49 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -557,34 +557,43 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe } } } + /** Create queries without list of columns: + * - CREATE|ATTACH TABLE ... AS ... + * - CREATE|ATTACH TABLE ... ENGINE = engine + */ else { storage_p.parse(pos, storage, expected); - if (!s_as.ignore(pos, expected)) - return false; - - if (!select_p.parse(pos, select, expected)) /// AS SELECT ... + /// CREATE|ATTACH TABLE ... AS ... + if (s_as.ignore(pos, expected)) { - /// ENGINE can not be specified for table functions. - if (storage || !table_function_p.parse(pos, as_table_function, expected)) + if (!select_p.parse(pos, select, expected)) /// AS SELECT ... { - /// AS [db.]table - if (!name_p.parse(pos, as_table, expected)) - return false; - - if (s_dot.ignore(pos, expected)) + /// ENGINE can not be specified for table functions. + if (storage || !table_function_p.parse(pos, as_table_function, expected)) { - as_database = as_table; + /// AS [db.]table if (!name_p.parse(pos, as_table, expected)) return false; - } - /// Optional - ENGINE can be specified. - if (!storage) - storage_p.parse(pos, storage, expected); + if (s_dot.ignore(pos, expected)) + { + as_database = as_table; + if (!name_p.parse(pos, as_table, expected)) + return false; + } + + /// Optional - ENGINE can be specified. + if (!storage) + storage_p.parse(pos, storage, expected); + } } } + else if (!storage) + { + return false; + } } auto comment = parseComment(pos, expected); diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index bc1ebd65639..33aafb40d83 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -361,6 +361,8 @@ protected: * Or: * CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name [UUID 'uuid'] [ON CLUSTER cluster] AS ENGINE = engine SELECT ... * + * Or (for engines that supports schema inference): + * CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name [UUID 'uuid'] [ON CLUSTER cluster] ENGINE = engine */ class ParserCreateTableQuery : public IParserBase { diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp new file mode 100644 index 00000000000..73ad57e71e6 --- /dev/null +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -0,0 +1,160 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; +} + +IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_) + : ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_) +{ +} + +NamesAndTypesList IRowSchemaReader::readSchema() +{ + DataTypes data_types = readRowAndGetDataTypes(); + for (size_t row = 1; row < max_rows_to_read; ++row) + { + DataTypes new_data_types = readRowAndGetDataTypes(); + if (new_data_types.empty()) + /// We reached eof. + break; + + if (new_data_types.size() != data_types.size()) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Rows have different amount of values"); + + for (size_t i = 0; i != data_types.size(); ++i) + { + /// We couldn't determine the type of this column in a new row, just skip it. + if (!new_data_types[i]) + continue; + + /// If we couldn't determine the type of column yet, just set the new type. + if (!data_types[i]) + data_types[i] = new_data_types[i]; + /// If the new type and the previous type for this column are different, + /// we will use default type if we have it or throw an exception. + else if (data_types[i]->getName() != new_data_types[i]->getName()) + { + if (default_type) + data_types[i] = default_type; + else + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", new_data_types[i]->getName(), i + 1, row, data_types[i]->getName()); + } + } + } + + /// Check that we read at list one column. + if (data_types.empty()) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot read rows from the data"); + + /// If column names weren't set, use default names 'column_1', 'column_2', ... + if (column_names.empty()) + { + column_names.reserve(data_types.size()); + for (size_t i = 0; i != data_types.size(); ++i) + column_names.push_back("column_" + std::to_string(i + 1)); + } + /// If column names were set, check that the number of names match the number of types. + else if (column_names.size() != data_types.size()) + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "The number of column names {} differs with the number of types {}", column_names.size(), data_types.size()); + + NamesAndTypesList result; + for (size_t i = 0; i != data_types.size(); ++i) + { + /// Check that we could determine the type of this column. + if (!data_types[i]) + { + if (!default_type) + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot determine table structure by first {} rows of data, because some columns contain only Nulls. To increase the maximum " + "number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference", + max_rows_to_read); + + data_types[i] = default_type; + } + result.emplace_back(column_names[i], data_types[i]); + } + + return result; +} + +IRowWithNamesSchemaReader::IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_) + : ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_) +{ +} + +NamesAndTypesList IRowWithNamesSchemaReader::readSchema() +{ + auto names_and_types = readRowAndGetNamesAndDataTypes(); + for (size_t row = 1; row < max_rows_to_read; ++row) + { + auto new_names_and_types = readRowAndGetNamesAndDataTypes(); + if (new_names_and_types.empty()) + /// We reached eof. + break; + + for (const auto & [name, new_type] : new_names_and_types) + { + auto it = names_and_types.find(name); + /// If we didn't see this column before, just add it. + if (it == names_and_types.end()) + { + names_and_types[name] = new_type; + continue; + } + + auto & type = it->second; + /// If we couldn't determine the type of column yet, just set the new type. + if (!type) + type = new_type; + /// If the new type and the previous type for this column are different, + /// we will use default type if we have it or throw an exception. + else if (new_type && type->getName() != new_type->getName()) + { + if (default_type) + type = default_type; + else + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", type->getName(), name, row, new_type->getName()); + } + } + } + + /// Check that we read at list one column. + if (names_and_types.empty()) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot read rows from the data"); + + NamesAndTypesList result; + for (auto & [name, type] : names_and_types) + { + /// Check that we could determine the type of this column. + if (!type) + { + if (!default_type) + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot determine table structure by first {} rows of data, because some columns contain only Nulls. To increase the maximum " + "number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference", + max_rows_to_read); + + type = default_type; + } + result.emplace_back(name, type); + } + + return result; +} + +} diff --git a/src/Processors/Formats/ISchemaReader.h b/src/Processors/Formats/ISchemaReader.h new file mode 100644 index 00000000000..67a8eb88d61 --- /dev/null +++ b/src/Processors/Formats/ISchemaReader.h @@ -0,0 +1,87 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +/// Base class for schema inference for the data in some specific format. +/// It reads some data from read buffer and try to determine the schema +/// from read data. +class ISchemaReader +{ +public: + ISchemaReader(ReadBuffer & in_) : in(in_) {} + + virtual NamesAndTypesList readSchema() = 0; + + virtual ~ISchemaReader() = default; + +protected: + ReadBuffer & in; +}; + +/// Base class for schema inference for formats that read data row by row. +/// It reads data row by row (up to max_rows_to_read), determines types of columns +/// for each row and compare them with types from the previous rows. If some column +/// contains values with different types in different rows, the default type will be +/// used for this column or the exception will be thrown (if default type is not set). +class IRowSchemaReader : public ISchemaReader +{ +public: + IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr); + NamesAndTypesList readSchema() override; + +protected: + /// Read one row and determine types of columns in it. + /// Return types in the same order in which the values were in the row. + /// If it's impossible to determine the type for some column, return nullptr for it. + /// Return empty list if can't read more data. + virtual DataTypes readRowAndGetDataTypes() = 0; + + void setColumnNames(const std::vector & names) { column_names = names; } + +private: + size_t max_rows_to_read; + DataTypePtr default_type; + std::vector column_names; +}; + +/// Base class for schema inference for formats that read data row by row and each +/// row contains column names and values (ex: JSONEachRow, TSKV). +/// Differ from IRowSchemaReader in that after reading a row we get +/// a map {column_name : type} and some columns may be missed in a single row +/// (in this case we will use types from the previous rows for missed columns). +class IRowWithNamesSchemaReader : public ISchemaReader +{ +public: + IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr); + NamesAndTypesList readSchema() override; + +protected: + /// Read one row and determine types of columns in it. + /// Return map {column_name : type}. + /// If it's impossible to determine the type for some column, return nullptr for it. + /// Return empty map is can't read more data. + virtual std::unordered_map readRowAndGetNamesAndDataTypes() = 0; + +private: + size_t max_rows_to_read; + DataTypePtr default_type; +}; + +/// Base class for schema inference for formats that don't need any data to +/// determine the schema: formats with constant schema (ex: JSONAsString, LineAsString) +/// and formats that use external format schema (ex: Protobuf, CapnProto). +class IExternalSchemaReader +{ +public: + virtual NamesAndTypesList readSchema() = 0; + + virtual ~IExternalSchemaReader() = default; +}; + +} diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index 1f6b530d72f..4af2c651c39 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -85,31 +85,38 @@ void ArrowBlockInputFormat::resetParser() record_batch_current = 0; } +static std::shared_ptr createStreamReader(ReadBuffer & in) +{ + auto stream_reader_status = arrow::ipc::RecordBatchStreamReader::Open(std::make_unique(in)); + if (!stream_reader_status.ok()) + throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, + "Error while opening a table: {}", stream_reader_status.status().ToString()); + return *stream_reader_status; +} + +static std::shared_ptr createFileReader(ReadBuffer & in, const FormatSettings & format_settings, std::atomic & is_stopped) +{ + auto arrow_file = asArrowFile(in, format_settings, is_stopped); + if (is_stopped) + return nullptr; + + auto file_reader_status = arrow::ipc::RecordBatchFileReader::Open(std::move(arrow_file)); + if (!file_reader_status.ok()) + throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, + "Error while opening a table: {}", file_reader_status.status().ToString()); + return *file_reader_status; +} + + void ArrowBlockInputFormat::prepareReader() { - std::shared_ptr schema; - if (stream) - { - auto stream_reader_status = arrow::ipc::RecordBatchStreamReader::Open(std::make_unique(*in)); - if (!stream_reader_status.ok()) - throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, - "Error while opening a table: {}", stream_reader_status.status().ToString()); - stream_reader = *stream_reader_status; - schema = stream_reader->schema(); - } + stream_reader = createStreamReader(*in); else { - auto arrow_file = asArrowFile(*in, format_settings, is_stopped); - if (is_stopped) + file_reader = createFileReader(*in, format_settings, is_stopped); + if (!file_reader) return; - - auto file_reader_status = arrow::ipc::RecordBatchFileReader::Open(std::move(arrow_file)); - if (!file_reader_status.ok()) - throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, - "Error while opening a table: {}", file_reader_status.status().ToString()); - file_reader = *file_reader_status; - schema = file_reader->schema(); } arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), "Arrow", format_settings.arrow.import_nested); @@ -122,6 +129,27 @@ void ArrowBlockInputFormat::prepareReader() record_batch_current = 0; } +ArrowSchemaReader::ArrowSchemaReader(ReadBuffer & in_, bool stream_, const FormatSettings & format_settings_) + : ISchemaReader(in_), stream(stream_), format_settings(format_settings_) +{ +} + +NamesAndTypesList ArrowSchemaReader::readSchema() +{ + std::shared_ptr schema; + + if (stream) + schema = createStreamReader(in)->schema(); + else + { + std::atomic is_stopped = 0; + schema = createFileReader(in, format_settings, is_stopped)->schema(); + } + + auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader(*schema, stream ? "ArrowStream" : "Arrow"); + return header.getNamesAndTypesList(); +} + void registerInputFormatArrow(FormatFactory & factory) { factory.registerInputFormat( @@ -145,6 +173,20 @@ void registerInputFormatArrow(FormatFactory & factory) }); } +void registerArrowSchemaReader(FormatFactory & factory) +{ + factory.registerSchemaReader( + "Arrow", + [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + { + return std::make_shared(buf, false, settings); + }); + factory.registerSchemaReader( + "ArrowStream", + [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + { + return std::make_shared(buf, true, settings); + });} } #else @@ -154,6 +196,8 @@ class FormatFactory; void registerInputFormatArrow(FormatFactory &) { } + +void registerArrowSchemaReader(FormatFactory &) {} } #endif diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index bb8a000477c..62cbf949fc2 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -4,6 +4,7 @@ #if USE_ARROW #include +#include #include namespace arrow { class RecordBatchReader; } @@ -51,6 +52,18 @@ private: std::atomic is_stopped{0}; }; +class ArrowSchemaReader : public ISchemaReader +{ +public: + ArrowSchemaReader(ReadBuffer & in_, bool stream_, const FormatSettings & format_settings_); + + NamesAndTypesList readSchema() override; + +private: + bool stream; + const FormatSettings format_settings; +}; + } #endif diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 272907022a1..95d37d8422e 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -328,12 +328,13 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( const std::string & column_name, const std::string & format_name, bool is_nullable, - std::unordered_map> & dictionary_values) + std::unordered_map> & dictionary_values, + bool read_ints_as_dates) { if (!is_nullable && arrow_column->null_count() && arrow_column->type()->id() != arrow::Type::LIST && arrow_column->type()->id() != arrow::Type::MAP && arrow_column->type()->id() != arrow::Type::STRUCT) { - auto nested_column = readColumnFromArrowColumn(arrow_column, column_name, format_name, true, dictionary_values); + auto nested_column = readColumnFromArrowColumn(arrow_column, column_name, format_name, true, dictionary_values, read_ints_as_dates); auto nullmap_column = readByteMapFromArrowColumn(arrow_column); auto nullable_type = std::make_shared(std::move(nested_column.type)); auto nullable_column = ColumnNullable::create(std::move(nested_column.column), std::move(nullmap_column)); @@ -358,13 +359,15 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( case arrow::Type::UINT16: { auto column = readColumnWithNumericData(arrow_column, column_name); - column.type = std::make_shared(); + if (read_ints_as_dates) + column.type = std::make_shared(); return column; } case arrow::Type::UINT32: { auto column = readColumnWithNumericData(arrow_column, column_name); - column.type = std::make_shared(); + if (read_ints_as_dates) + column.type = std::make_shared(); return column; } case arrow::Type::TIMESTAMP: @@ -376,7 +379,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( case arrow::Type::MAP: { auto arrow_nested_column = getNestedArrowColumn(arrow_column); - auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values); + auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values, read_ints_as_dates); auto offsets_column = readOffsetsFromArrowListColumn(arrow_column); const auto * tuple_column = assert_cast(nested_column.column.get()); @@ -388,7 +391,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( case arrow::Type::LIST: { auto arrow_nested_column = getNestedArrowColumn(arrow_column); - auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values); + auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values, read_ints_as_dates); auto offsets_column = readOffsetsFromArrowListColumn(arrow_column); auto array_column = ColumnArray::create(std::move(nested_column.column), std::move(offsets_column)); auto array_type = std::make_shared(nested_column.type); @@ -413,7 +416,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( for (int i = 0; i != arrow_struct_type->num_fields(); ++i) { auto nested_arrow_column = std::make_shared(nested_arrow_columns[i]); - auto element = readColumnFromArrowColumn(nested_arrow_column, arrow_struct_type->field(i)->name(), format_name, false, dictionary_values); + auto element = readColumnFromArrowColumn(nested_arrow_column, arrow_struct_type->field(i)->name(), format_name, false, dictionary_values, read_ints_as_dates); tuple_elements.emplace_back(std::move(element.column)); tuple_types.emplace_back(std::move(element.type)); tuple_names.emplace_back(std::move(element.name)); @@ -436,7 +439,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( dict_array.emplace_back(dict_chunk.dictionary()); } auto arrow_dict_column = std::make_shared(dict_array); - auto dict_column = readColumnFromArrowColumn(arrow_dict_column, column_name, format_name, false, dictionary_values); + auto dict_column = readColumnFromArrowColumn(arrow_dict_column, column_name, format_name, false, dictionary_values, read_ints_as_dates); /// We should convert read column to ColumnUnique. auto tmp_lc_column = DataTypeLowCardinality(dict_column.type).createColumn(); @@ -483,7 +486,7 @@ static void checkStatus(const arrow::Status & status, const String & column_name throw Exception{ErrorCodes::UNKNOWN_EXCEPTION, "Error with a {} column '{}': {}.", format_name, column_name, status.ToString()}; } -static Block arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name) +Block ArrowColumnToCHColumn::arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name) { ColumnsWithTypeAndName sample_columns; for (const auto & field : schema.fields()) @@ -493,24 +496,21 @@ static Block arrowSchemaToCHHeader(const arrow::Schema & schema, const std::stri std::unique_ptr array_builder; arrow::Status status = MakeBuilder(pool, field->type(), &array_builder); checkStatus(status, field->name(), format_name); + std::shared_ptr arrow_array; status = array_builder->Finish(&arrow_array); checkStatus(status, field->name(), format_name); + arrow::ArrayVector array_vector = {arrow_array}; auto arrow_column = std::make_shared(array_vector); std::unordered_map> dict_values; - ColumnWithTypeAndName sample_column = readColumnFromArrowColumn(arrow_column, field->name(), format_name, false, dict_values); + ColumnWithTypeAndName sample_column = readColumnFromArrowColumn(arrow_column, field->name(), format_name, false, dict_values, false); + sample_columns.emplace_back(std::move(sample_column)); } return Block(std::move(sample_columns)); } -ArrowColumnToCHColumn::ArrowColumnToCHColumn( - const arrow::Schema & schema, const std::string & format_name_, bool import_nested_) - : header(arrowSchemaToCHHeader(schema, format_name_)), format_name(format_name_), import_nested(import_nested_) -{ -} - ArrowColumnToCHColumn::ArrowColumnToCHColumn( const Block & header_, const std::string & format_name_, bool import_nested_) : header(header_), format_name(format_name_), import_nested(import_nested_) @@ -553,7 +553,7 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & if (!nested_tables.contains(nested_table_name)) { std::shared_ptr arrow_column = name_to_column_ptr[nested_table_name]; - ColumnsWithTypeAndName cols = {readColumnFromArrowColumn(arrow_column, nested_table_name, format_name, false, dictionary_values)}; + ColumnsWithTypeAndName cols = {readColumnFromArrowColumn(arrow_column, nested_table_name, format_name, false, dictionary_values, true)}; Block block(cols); nested_tables[nested_table_name] = std::make_shared(Nested::flatten(block)); } @@ -573,7 +573,7 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & if (read_from_nested) column = nested_tables[nested_table_name]->getByName(header_column.name); else - column = readColumnFromArrowColumn(arrow_column, header_column.name, format_name, false, dictionary_values); + column = readColumnFromArrowColumn(arrow_column, header_column.name, format_name, false, dictionary_values, true); try { diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 46976093f0b..58f8f1536b5 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -23,16 +23,14 @@ public: ArrowColumnToCHColumn(const Block & header_, const std::string & format_name_, bool import_nested_); - /// Constructor that create header by arrow schema. It will be useful for inserting - /// data from file without knowing table structure. - ArrowColumnToCHColumn(const arrow::Schema & schema, const std::string & format_name, bool import_nested_); - void arrowTableToCHChunk(Chunk & res, std::shared_ptr & table); void arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & name_to_column_ptr); + static Block arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name); + private: - const Block header; + const Block & header; const std::string format_name; bool import_nested; diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 11e56ecbe0c..a372df41344 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -815,6 +815,92 @@ const AvroDeserializer & AvroConfluentRowInputFormat::getOrCreateDeserializer(Sc return it->second; } +AvroSchemaReader::AvroSchemaReader(ReadBuffer & in_, bool confluent_, const FormatSettings & format_settings_) + : ISchemaReader(in_), confluent(confluent_), format_settings(format_settings_) +{ +} + +NamesAndTypesList AvroSchemaReader::readSchema() +{ + avro::NodePtr root_node; + if (confluent) + { + UInt32 schema_id = readConfluentSchemaId(in); + root_node = getConfluentSchemaRegistry(format_settings)->getSchema(schema_id).root(); + } + else + { + auto file_reader_ptr = std::make_unique(std::make_unique(in)); + root_node = file_reader_ptr->dataSchema().root(); + } + + if (root_node->type() != avro::Type::AVRO_RECORD) + throw Exception("Root schema must be a record", ErrorCodes::TYPE_MISMATCH); + + NamesAndTypesList names_and_types; + for (size_t i = 0; i != root_node->leaves(); ++i) + names_and_types.emplace_back(root_node->nameAt(i), avroNodeToDataType(root_node->leafAt(i))); + + return names_and_types; +} + +DataTypePtr AvroSchemaReader::avroNodeToDataType(avro::NodePtr node) +{ + switch (node->type()) + { + case avro::Type::AVRO_INT: + return {std::make_shared()}; + case avro::Type::AVRO_LONG: + return std::make_shared(); + case avro::Type::AVRO_BOOL: + return std::make_shared(); + case avro::Type::AVRO_FLOAT: + return std::make_shared(); + case avro::Type::AVRO_DOUBLE: + return std::make_shared(); + case avro::Type::AVRO_STRING: + return std::make_shared(); + case avro::Type::AVRO_BYTES: + return std::make_shared(); + case avro::Type::AVRO_ENUM: + { + if (node->names() < 128) + { + EnumValues::Values values; + for (size_t i = 0; i != node->names(); ++i) + values.emplace_back(node->nameAt(i), i); + return std::make_shared(std::move(values)); + } + else if (node->names() < 32768) + { + EnumValues::Values values; + for (size_t i = 0; i != node->names(); ++i) + values.emplace_back(node->nameAt(i), i); + return std::make_shared(std::move(values)); + } + + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ClickHouse supports only 8 and 16-bit Enum."); + } + case avro::Type::AVRO_FIXED: + return std::make_shared(node->fixedSize()); + case avro::Type::AVRO_ARRAY: + return std::make_shared(avroNodeToDataType(node->leafAt(0))); + case avro::Type::AVRO_NULL: + return std::make_shared(); + case avro::Type::AVRO_UNION: + if (node->leaves() == 2 && (node->leafAt(0)->type() == avro::Type::AVRO_NULL || node->leafAt(1)->type() == avro::Type::AVRO_NULL)) + { + size_t nested_leaf_index = node->leafAt(0)->type() == avro::Type::AVRO_NULL ? 1 : 0; + return makeNullable(avroNodeToDataType(node->leafAt(nested_leaf_index))); + } + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Avro type UNION is not supported for inserting."); + case avro::Type::AVRO_SYMBOLIC: + return avroNodeToDataType(avro::resolveSymbol(node)); + default: + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Avro column {} is not supported for inserting."); + } +} + void registerInputFormatAvro(FormatFactory & factory) { factory.registerInputFormat("Avro", []( @@ -836,6 +922,21 @@ void registerInputFormatAvro(FormatFactory & factory) }); } +void registerAvroSchemaReader(FormatFactory & factory) +{ + factory.registerSchemaReader("Avro", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + { + return std::make_shared(buf, false, settings); + }); + + factory.registerSchemaReader("AvroConfluent", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + { + return std::make_shared(buf, true, settings); + }); + +} + + } #else @@ -846,6 +947,8 @@ class FormatFactory; void registerInputFormatAvro(FormatFactory &) { } + +void registerAvroSchemaReader(FormatFactory &) {} } #endif diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.h b/src/Processors/Formats/Impl/AvroRowInputFormat.h index 73237369e56..46e571d87ec 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.h +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.h @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -160,6 +161,20 @@ private: FormatSettings format_settings; }; +class AvroSchemaReader : public ISchemaReader +{ +public: + AvroSchemaReader(ReadBuffer & in_, bool confluent_, const FormatSettings & format_settings_); + + NamesAndTypesList readSchema() override; + +private: + DataTypePtr avroNodeToDataType(avro::NodePtr node); + + bool confluent; + const FormatSettings format_settings; +}; + } #endif diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp index 0506c539c0f..b356967a544 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp @@ -5,7 +5,6 @@ #include #include - namespace DB { @@ -15,11 +14,23 @@ namespace ErrorCodes } BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, Block header, Params params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_) - : RowInputFormatWithNamesAndTypes(std::move(header), in_, std::move(params_), with_names_, with_types_, format_settings_) + : RowInputFormatWithNamesAndTypes( + std::move(header), + in_, + std::move(params_), + with_names_, + with_types_, + format_settings_, + std::make_unique(in_, format_settings_)) { } -std::vector BinaryRowInputFormat::readHeaderRow() + +BinaryFormatReader::BinaryFormatReader(ReadBuffer & in_, const FormatSettings & format_settings_) : FormatWithNamesAndTypesReader(in_, format_settings_) +{ +} + +std::vector BinaryFormatReader::readHeaderRow() { std::vector fields; String field; @@ -31,13 +42,13 @@ std::vector BinaryRowInputFormat::readHeaderRow() return fields; } -std::vector BinaryRowInputFormat::readNames() +std::vector BinaryFormatReader::readNames() { readVarUInt(read_columns, *in); return readHeaderRow(); } -std::vector BinaryRowInputFormat::readTypes() +std::vector BinaryFormatReader::readTypes() { auto types = readHeaderRow(); for (const auto & type_name : types) @@ -45,31 +56,37 @@ std::vector BinaryRowInputFormat::readTypes() return types; } -bool BinaryRowInputFormat::readField(IColumn & column, const DataTypePtr & /*type*/, const SerializationPtr & serialization, bool /*is_last_file_column*/, const String & /*column_name*/) +bool BinaryFormatReader::readField(IColumn & column, const DataTypePtr & /*type*/, const SerializationPtr & serialization, bool /*is_last_file_column*/, const String & /*column_name*/) { serialization->deserializeBinary(column, *in); return true; } -void BinaryRowInputFormat::skipHeaderRow() +void BinaryFormatReader::skipHeaderRow() { String tmp; for (size_t i = 0; i < read_columns; ++i) readStringBinary(tmp, *in); } -void BinaryRowInputFormat::skipNames() +void BinaryFormatReader::skipNames() { readVarUInt(read_columns, *in); skipHeaderRow(); } -void BinaryRowInputFormat::skipTypes() +void BinaryFormatReader::skipTypes() { + if (read_columns == 0) + { + /// It's possible only when with_names = false and with_types = true + readVarUInt(read_columns, *in); + } + skipHeaderRow(); } -void BinaryRowInputFormat::skipField(size_t file_column) +void BinaryFormatReader::skipField(size_t file_column) { if (file_column >= read_data_types.size()) throw Exception(ErrorCodes::CANNOT_SKIP_UNKNOWN_FIELD, "Cannot skip unknown field in RowBinaryWithNames format, because it's type is unknown"); @@ -77,6 +94,11 @@ void BinaryRowInputFormat::skipField(size_t file_column) read_data_types[file_column]->getDefaultSerialization()->deserializeBinary(field, *in); } +BinaryWithNamesAndTypesSchemaReader::BinaryWithNamesAndTypesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) + : FormatWithNamesAndTypesSchemaReader(in_, 0, true, true, &reader), reader(in_, format_settings_) +{ +} + void registerInputFormatRowBinary(FormatFactory & factory) { auto register_func = [&](const String & format_name, bool with_names, bool with_types) @@ -94,4 +116,13 @@ void registerInputFormatRowBinary(FormatFactory & factory) registerWithNamesAndTypes("RowBinary", register_func); } +void registerRowBinaryWithNamesAndTypesSchemaReader(FormatFactory & factory) +{ + factory.registerSchemaReader("RowBinaryWithNamesAndTypes", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + { + return std::make_shared(buf, settings); + }); +} + + } diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.h b/src/Processors/Formats/Impl/BinaryRowInputFormat.h index 61d6df77522..d98e75bf621 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.h +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.h @@ -1,15 +1,19 @@ #pragma once #include -#include #include +#include namespace DB { -class ReadBuffer; +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} +class ReadBuffer; /** A stream for inputting data in a binary line-by-line format. */ @@ -24,9 +28,15 @@ public: /// in this format we cannot provide any DiagnosticInfo, because here we have /// just binary data. std::string getDiagnosticInfo() override { return {}; } +}; + +class BinaryFormatReader : public FormatWithNamesAndTypesReader +{ +public: + BinaryFormatReader(ReadBuffer & in_, const FormatSettings & format_settings_); -private: bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override; + void skipField(size_t file_column) override; void skipNames() override; @@ -37,9 +47,24 @@ private: std::vector readTypes() override; std::vector readHeaderRow(); +private: /// Data types read from input data. DataTypes read_data_types; - UInt64 read_columns = 0; + UInt64 read_columns; +}; + +class BinaryWithNamesAndTypesSchemaReader : public FormatWithNamesAndTypesSchemaReader +{ +public: + BinaryWithNamesAndTypesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_); + +private: + DataTypes readRowAndGetDataTypes() override + { + throw Exception{ErrorCodes::NOT_IMPLEMENTED, "Method readRowAndGetDataTypes is not implemented"}; + } + + BinaryFormatReader reader; }; } diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 9de2b908b1e..735a549d0a6 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -5,13 +5,16 @@ #include #include #include +#include +#include #include #include -#include +#include +#include + namespace DB { - namespace ErrorCodes { extern const int BAD_ARGUMENTS; @@ -26,7 +29,14 @@ CSVRowInputFormat::CSVRowInputFormat( bool with_names_, bool with_types_, const FormatSettings & format_settings_) - : RowInputFormatWithNamesAndTypes(header_, in_, params_, with_names_, with_types_, format_settings_) + : RowInputFormatWithNamesAndTypes( + header_, + in_, + params_, + with_names_, + with_types_, + format_settings_, + std::make_unique(in_, format_settings_)) { const String bad_delimiters = " \t\"'.UL"; if (bad_delimiters.find(format_settings.csv.delimiter) != String::npos) @@ -36,6 +46,11 @@ CSVRowInputFormat::CSVRowInputFormat( ErrorCodes::BAD_ARGUMENTS); } +void CSVRowInputFormat::syncAfterError() +{ + skipToNextLineOrEOF(*in); +} + static void skipEndOfLine(ReadBuffer & in) { /// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic) @@ -52,8 +67,10 @@ static void skipEndOfLine(ReadBuffer & in) if (!in.eof() && *in.position() == '\n') ++in.position(); else - throw Exception("Cannot parse CSV format: found \\r (CR) not followed by \\n (LF)." - " Line must end by \\n (LF) or \\r\\n (CR LF) or \\n\\r.", ErrorCodes::INCORRECT_DATA); + throw Exception( + "Cannot parse CSV format: found \\r (CR) not followed by \\n (LF)." + " Line must end by \\n (LF) or \\r\\n (CR LF) or \\n\\r.", + ErrorCodes::INCORRECT_DATA); } else if (!in.eof()) throw Exception("Expected end of line", ErrorCodes::INCORRECT_DATA); @@ -62,32 +79,38 @@ static void skipEndOfLine(ReadBuffer & in) /// Skip `whitespace` symbols allowed in CSV. static inline void skipWhitespacesAndTabs(ReadBuffer & in) { - while (!in.eof() - && (*in.position() == ' ' - || *in.position() == '\t')) + while (!in.eof() && (*in.position() == ' ' || *in.position() == '\t')) ++in.position(); } -void CSVRowInputFormat::skipFieldDelimiter() +CSVFormatReader::CSVFormatReader(ReadBuffer & in_, const FormatSettings & format_settings_) : FormatWithNamesAndTypesReader(in_, format_settings_) +{ +} + +void CSVFormatReader::skipFieldDelimiter() { skipWhitespacesAndTabs(*in); assertChar(format_settings.csv.delimiter, *in); } -String CSVRowInputFormat::readFieldIntoString() +template +String CSVFormatReader::readCSVFieldIntoString() { skipWhitespacesAndTabs(*in); String field; - readCSVString(field, *in, format_settings.csv); + if constexpr (read_string) + readCSVString(field, *in, format_settings.csv); + else + readCSVField(field, *in, format_settings.csv); return field; } -void CSVRowInputFormat::skipField() +void CSVFormatReader::skipField() { - readFieldIntoString(); + readCSVFieldIntoString(); } -void CSVRowInputFormat::skipRowEndDelimiter() +void CSVFormatReader::skipRowEndDelimiter() { skipWhitespacesAndTabs(*in); @@ -105,33 +128,32 @@ void CSVRowInputFormat::skipRowEndDelimiter() skipEndOfLine(*in); } -void CSVRowInputFormat::skipHeaderRow() +void CSVFormatReader::skipHeaderRow() { do { skipField(); skipWhitespacesAndTabs(*in); - } - while (checkChar(format_settings.csv.delimiter, *in)); + } while (checkChar(format_settings.csv.delimiter, *in)); skipRowEndDelimiter(); } -std::vector CSVRowInputFormat::readHeaderRow() +template +std::vector CSVFormatReader::readRowImpl() { std::vector fields; do { - fields.push_back(readFieldIntoString()); + fields.push_back(readCSVFieldIntoString()); skipWhitespacesAndTabs(*in); - } - while (checkChar(format_settings.csv.delimiter, *in)); + } while (checkChar(format_settings.csv.delimiter, *in)); skipRowEndDelimiter(); return fields; } -bool CSVRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) +bool CSVFormatReader::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) { const char delimiter = format_settings.csv.delimiter; @@ -144,7 +166,8 @@ bool CSVRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) { if (*in->position() == '\n' || *in->position() == '\r') { - out << "ERROR: Line feed found where delimiter (" << delimiter << ") is expected." + out << "ERROR: Line feed found where delimiter (" << delimiter + << ") is expected." " It's like your file has less columns than expected.\n" "And if your file has the right number of columns, maybe it has unescaped quotes in values.\n"; } @@ -160,7 +183,7 @@ bool CSVRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) return true; } -bool CSVRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out) +bool CSVFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out) { skipWhitespacesAndTabs(*in); @@ -191,23 +214,21 @@ bool CSVRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out) return true; } -void CSVRowInputFormat::syncAfterError() -{ - skipToNextLineOrEOF(*in); -} - -bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & /*column_name*/) +bool CSVFormatReader::readField( + IColumn & column, + const DataTypePtr & type, + const SerializationPtr & serialization, + bool is_last_file_column, + const String & /*column_name*/) { skipWhitespacesAndTabs(*in); const bool at_delimiter = !in->eof() && *in->position() == format_settings.csv.delimiter; - const bool at_last_column_line_end = is_last_file_column - && (in->eof() || *in->position() == '\n' || *in->position() == '\r'); + const bool at_last_column_line_end = is_last_file_column && (in->eof() || *in->position() == '\n' || *in->position() == '\r'); /// Note: Tuples are serialized in CSV as separate columns, but with empty_as_default or null_as_default /// only one empty or NULL column will be expected - if (format_settings.csv.empty_as_default - && (at_delimiter || at_last_column_line_end)) + if (format_settings.csv.empty_as_default && (at_delimiter || at_last_column_line_end)) { /// Treat empty unquoted column value as default value, if /// specified in the settings. Tuple columns might seem @@ -231,6 +252,31 @@ bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, co } } + +CSVSchemaReader::CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_setting_, ContextPtr context_) + : FormatWithNamesAndTypesSchemaReader( + in_, + format_setting_.max_rows_to_read_for_schema_inference, + with_names_, + with_types_, + &reader, + getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule::CSV)) + , reader(in_, format_setting_) + , context(context_) +{ +} + + +DataTypes CSVSchemaReader::readRowAndGetDataTypes() +{ + if (in.eof()) + return {}; + + auto fields = reader.readRow(); + return determineDataTypesByEscapingRule(fields, reader.getFormatSettings(), FormatSettings::EscapingRule::CSV, context); +} + + void registerInputFormatCSV(FormatFactory & factory) { auto register_func = [&](const String & format_name, bool with_names, bool with_types) @@ -326,4 +372,17 @@ void registerFileSegmentationEngineCSV(FormatFactory & factory) registerWithNamesAndTypes("CSV", register_func); } +void registerCSVSchemaReader(FormatFactory & factory) +{ + auto register_func = [&](const String & format_name, bool with_names, bool with_types) + { + factory.registerSchemaReader(format_name, [with_names, with_types](ReadBuffer & buf, const FormatSettings & settings, ContextPtr context) + { + return std::make_shared(buf, with_names, with_types, settings, context); + }); + }; + + registerWithNamesAndTypes("CSV", register_func); +} + } diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index d7c557b58d8..d723647595e 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -5,6 +5,7 @@ #include #include +#include #include @@ -28,6 +29,12 @@ public: private: bool allowSyncAfterError() const override { return true; } void syncAfterError() override; +}; + +class CSVFormatReader : public FormatWithNamesAndTypesReader +{ +public: + CSVFormatReader(ReadBuffer & in_, const FormatSettings & format_settings_); bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override; bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override; @@ -42,17 +49,34 @@ private: void skipField(size_t /*file_column*/) override { skipField(); } void skipField(); - void skipHeaderRow() ; + void skipHeaderRow(); void skipNames() override { skipHeaderRow(); } void skipTypes() override { skipHeaderRow(); } void skipFieldDelimiter() override; void skipRowEndDelimiter() override; - std::vector readHeaderRow(); std::vector readNames() override { return readHeaderRow(); } std::vector readTypes() override { return readHeaderRow(); } + std::vector readHeaderRow() { return readRowImpl(); } + std::vector readRow() { return readRowImpl(); } - String readFieldIntoString(); + template + std::vector readRowImpl(); + + template + String readCSVFieldIntoString(); +}; + +class CSVSchemaReader : public FormatWithNamesAndTypesSchemaReader +{ +public: + CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_setting_, ContextPtr context_); + +private: + DataTypes readRowAndGetDataTypes() override; + + CSVFormatReader reader; + ContextPtr context; }; } diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp index 4d000bb1f35..311f4742335 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp @@ -273,6 +273,7 @@ bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension #endif auto root_reader = msg.getRoot(root); + for (size_t i = 0; i != columns.size(); ++i) { auto value = getReaderByColumnName(root_reader, column_names[i]); @@ -282,6 +283,24 @@ bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension return true; } +CapnProtoSchemaReader::CapnProtoSchemaReader(const FormatSettings & format_settings_) : format_settings(format_settings_) +{ +} + +NamesAndTypesList CapnProtoSchemaReader::readSchema() +{ + auto schema_info = FormatSchemaInfo( + format_settings.schema.format_schema, + "CapnProto", + true, + format_settings.schema.is_server, + format_settings.schema.format_schema_path); + + auto schema_parser = CapnProtoSchemaParser(); + auto schema = schema_parser.getMessageSchema(schema_info); + return capnProtoSchemaToCHSchema(schema); +} + void registerInputFormatCapnProto(FormatFactory & factory) { factory.registerInputFormat( @@ -293,6 +312,14 @@ void registerInputFormatCapnProto(FormatFactory & factory) }); } +void registerCapnProtoSchemaReader(FormatFactory & factory) +{ + factory.registerExternalSchemaReader("CapnProto", [](const FormatSettings & settings) + { + return std::make_shared(settings); + }); +} + } #else @@ -301,6 +328,7 @@ namespace DB { class FormatFactory; void registerInputFormatCapnProto(FormatFactory &) {} + void registerCapnProtoSchemaReader(FormatFactory &) {} } #endif // USE_CAPNP diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h index 4c0f34d70a3..3db4f3de0d4 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -38,6 +39,17 @@ private: Names column_names; }; +class CapnProtoSchemaReader : public IExternalSchemaReader +{ +public: + CapnProtoSchemaReader(const FormatSettings & format_settings_); + + NamesAndTypesList readSchema() override; + +private: + const FormatSettings format_settings; +}; + } #endif // USE_CAPNP diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp index 56aa4345777..0366b8ff2e3 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp @@ -31,7 +31,7 @@ CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat( bool ignore_spaces_, const FormatSettings & format_settings_) : CustomSeparatedRowInputFormat( - header_, std::make_unique(in_buf_), params_, with_names_, with_types_, ignore_spaces_, format_settings_) + header_, std::make_unique(in_buf_), params_, with_names_, with_types_, ignore_spaces_, updateFormatSettings(format_settings_)) { } @@ -43,10 +43,15 @@ CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat( bool with_types_, bool ignore_spaces_, const FormatSettings & format_settings_) - : RowInputFormatWithNamesAndTypes(header_, *buf_, params_, with_names_, with_types_, updateFormatSettings(format_settings_)) + : RowInputFormatWithNamesAndTypes( + header_, + *buf_, + params_, + with_names_, + with_types_, + format_settings_, + std::make_unique(*buf_, ignore_spaces_, format_settings_)) , buf(std::move(buf_)) - , ignore_spaces(ignore_spaces_) - , escaping_rule(format_settings_.custom.escaping_rule) { /// In case of CustomSeparatedWithNames(AndTypes) formats and enabled setting input_format_with_names_use_header we don't know /// the exact number of columns in data (because it can contain unknown columns). So, if field_delimiter and row_after_delimiter are @@ -61,43 +66,76 @@ CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat( } } -void CustomSeparatedRowInputFormat::skipPrefixBeforeHeader() + +bool CustomSeparatedRowInputFormat::allowSyncAfterError() const +{ + return !format_settings.custom.row_after_delimiter.empty() || !format_settings.custom.row_between_delimiter.empty(); +} + +void CustomSeparatedRowInputFormat::syncAfterError() +{ + skipToNextRowOrEof(*buf, format_settings.custom.row_after_delimiter, format_settings.custom.row_between_delimiter, ignore_spaces); + end_of_stream = buf->eof(); + /// It can happen that buf->position() is not at the beginning of row + /// if some delimiters is similar to row_format.delimiters.back() and row_between_delimiter. + /// It will cause another parsing error. +} + +void CustomSeparatedRowInputFormat::setReadBuffer(ReadBuffer & in_) +{ + buf = std::make_unique(in_); + RowInputFormatWithNamesAndTypes::setReadBuffer(in_); +} + +CustomSeparatedFormatReader::CustomSeparatedFormatReader( + PeekableReadBuffer & buf_, bool ignore_spaces_, const FormatSettings & format_settings_) + : FormatWithNamesAndTypesReader(buf_, format_settings_), buf(&buf_), ignore_spaces(ignore_spaces_) +{ +} + +void CustomSeparatedRowInputFormat::resetParser() +{ + RowInputFormatWithNamesAndTypes::resetParser(); + buf->reset(); +} + +void CustomSeparatedFormatReader::skipPrefixBeforeHeader() { skipSpaces(); assertString(format_settings.custom.result_before_delimiter, *buf); } -void CustomSeparatedRowInputFormat::skipRowStartDelimiter() +void CustomSeparatedFormatReader::skipRowStartDelimiter() { skipSpaces(); assertString(format_settings.custom.row_before_delimiter, *buf); } -void CustomSeparatedRowInputFormat::skipFieldDelimiter() +void CustomSeparatedFormatReader::skipFieldDelimiter() { skipSpaces(); assertString(format_settings.custom.field_delimiter, *buf); } -void CustomSeparatedRowInputFormat::skipRowEndDelimiter() +void CustomSeparatedFormatReader::skipRowEndDelimiter() { skipSpaces(); assertString(format_settings.custom.row_after_delimiter, *buf); } -void CustomSeparatedRowInputFormat::skipRowBetweenDelimiter() +void CustomSeparatedFormatReader::skipRowBetweenDelimiter() { skipSpaces(); assertString(format_settings.custom.row_between_delimiter, *buf); } -void CustomSeparatedRowInputFormat::skipField() +void CustomSeparatedFormatReader::skipField() { skipSpaces(); - skipFieldByEscapingRule(*buf, escaping_rule, format_settings); + skipFieldByEscapingRule(*buf, format_settings.custom.escaping_rule, format_settings); } -bool CustomSeparatedRowInputFormat::checkEndOfRow() +bool CustomSeparatedFormatReader::checkEndOfRow() { PeekableReadBufferCheckpoint checkpoint{*buf, true}; @@ -118,43 +156,66 @@ bool CustomSeparatedRowInputFormat::checkEndOfRow() return checkForSuffixImpl(true); } -std::vector CustomSeparatedRowInputFormat::readHeaderRow() +template +String CustomSeparatedFormatReader::readFieldIntoString(bool is_first) +{ + if (!is_first) + skipFieldDelimiter(); + skipSpaces(); + if constexpr (is_header) + return readStringByEscapingRule(*buf, format_settings.custom.escaping_rule, format_settings); + else + return readFieldByEscapingRule(*buf, format_settings.custom.escaping_rule, format_settings); +} + +template +std::vector CustomSeparatedFormatReader::readRowImpl() { std::vector values; skipRowStartDelimiter(); - do + + if (columns == 0) { - if (!values.empty()) - skipFieldDelimiter(); - skipSpaces(); - values.push_back(readStringByEscapingRule(*buf, escaping_rule, format_settings)); + do + { + values.push_back(readFieldIntoString(values.empty())); + } while (!checkEndOfRow()); + columns = values.size(); + } + else + { + for (size_t i = 0; i != columns; ++i) + values.push_back(readFieldIntoString(i == 0)); } - while (!checkEndOfRow()); skipRowEndDelimiter(); return values; } -void CustomSeparatedRowInputFormat::skipHeaderRow() +void CustomSeparatedFormatReader::skipHeaderRow() { - size_t columns = getPort().getHeader().columns(); skipRowStartDelimiter(); - for (size_t i = 0; i != columns; ++i) + bool first = true; + do { - skipField(); - if (i + 1 != columns) + if (!first) skipFieldDelimiter(); + first = false; + + skipField(); } + while (!checkEndOfRow()); + skipRowEndDelimiter(); } -bool CustomSeparatedRowInputFormat::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool, const String &) +bool CustomSeparatedFormatReader::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool, const String &) { skipSpaces(); - return deserializeFieldByEscapingRule(type, serialization, column, *buf, escaping_rule, format_settings); + return deserializeFieldByEscapingRule(type, serialization, column, *buf, format_settings.custom.escaping_rule, format_settings); } -bool CustomSeparatedRowInputFormat::checkForSuffixImpl(bool check_eof) +bool CustomSeparatedFormatReader::checkForSuffixImpl(bool check_eof) { skipSpaces(); if (format_settings.custom.result_after_delimiter.empty()) @@ -177,7 +238,7 @@ bool CustomSeparatedRowInputFormat::checkForSuffixImpl(bool check_eof) return false; } -bool CustomSeparatedRowInputFormat::tryParseSuffixWithDiagnosticInfo(WriteBuffer & out) +bool CustomSeparatedFormatReader::tryParseSuffixWithDiagnosticInfo(WriteBuffer & out) { PeekableReadBufferCheckpoint checkpoint{*buf}; if (checkForSuffixImpl(false)) @@ -192,7 +253,7 @@ bool CustomSeparatedRowInputFormat::tryParseSuffixWithDiagnosticInfo(WriteBuffer return true; } -bool CustomSeparatedRowInputFormat::checkForSuffix() +bool CustomSeparatedFormatReader::checkForSuffix() { PeekableReadBufferCheckpoint checkpoint{*buf}; if (checkForSuffixImpl(true)) @@ -201,51 +262,60 @@ bool CustomSeparatedRowInputFormat::checkForSuffix() return false; } - -bool CustomSeparatedRowInputFormat::allowSyncAfterError() const -{ - return !format_settings.custom.row_after_delimiter.empty() || !format_settings.custom.row_between_delimiter.empty(); -} - -void CustomSeparatedRowInputFormat::syncAfterError() -{ - skipToNextRowOrEof(*buf, format_settings.custom.row_after_delimiter, format_settings.custom.row_between_delimiter, ignore_spaces); - end_of_stream = buf->eof(); - /// It can happen that buf->position() is not at the beginning of row - /// if some delimiters is similar to row_format.delimiters.back() and row_between_delimiter. - /// It will cause another parsing error. -} - -bool CustomSeparatedRowInputFormat::parseRowStartWithDiagnosticInfo(WriteBuffer & out) +bool CustomSeparatedFormatReader::parseRowStartWithDiagnosticInfo(WriteBuffer & out) { return parseDelimiterWithDiagnosticInfo(out, *buf, format_settings.custom.row_before_delimiter, "delimiter before first field", ignore_spaces); } -bool CustomSeparatedRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) +bool CustomSeparatedFormatReader::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) { return parseDelimiterWithDiagnosticInfo(out, *buf, format_settings.custom.field_delimiter, "delimiter between fields", ignore_spaces); } -bool CustomSeparatedRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out) +bool CustomSeparatedFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out) { return parseDelimiterWithDiagnosticInfo(out, *buf, format_settings.custom.row_after_delimiter, "delimiter after last field", ignore_spaces); } -bool CustomSeparatedRowInputFormat::parseRowBetweenDelimiterWithDiagnosticInfo(WriteBuffer & out) +bool CustomSeparatedFormatReader::parseRowBetweenDelimiterWithDiagnosticInfo(WriteBuffer & out) { return parseDelimiterWithDiagnosticInfo(out, *buf, format_settings.custom.row_between_delimiter, "delimiter between rows", ignore_spaces); } -void CustomSeparatedRowInputFormat::resetParser() +void CustomSeparatedFormatReader::setReadBuffer(ReadBuffer & in_) { - RowInputFormatWithNamesAndTypes::resetParser(); - buf->reset(); + buf = assert_cast(&in_); + FormatWithNamesAndTypesReader::setReadBuffer(in_); } -void CustomSeparatedRowInputFormat::setReadBuffer(ReadBuffer & in_) +CustomSeparatedSchemaReader::CustomSeparatedSchemaReader( + ReadBuffer & in_, bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_setting_, ContextPtr context_) + : FormatWithNamesAndTypesSchemaReader( + buf, + format_setting_.max_rows_to_read_for_schema_inference, + with_names_, + with_types_, + &reader, + getDefaultDataTypeForEscapingRule(format_setting_.custom.escaping_rule)) + , buf(in_) + , reader(buf, ignore_spaces_, updateFormatSettings(format_setting_)) + , context(context_) { - buf = std::make_unique(in_); - IInputFormat::setReadBuffer(*buf); +} + +DataTypes CustomSeparatedSchemaReader::readRowAndGetDataTypes() +{ + if (reader.checkForSuffix()) + return {}; + + if (!first_row || with_names || with_types) + reader.skipRowBetweenDelimiter(); + + if (first_row) + first_row = false; + + auto fields = reader.readRow(); + return determineDataTypesByEscapingRule(fields, reader.getFormatSettings(), reader.getEscapingRule(), context); } void registerInputFormatCustomSeparated(FormatFactory & factory) @@ -267,4 +337,20 @@ void registerInputFormatCustomSeparated(FormatFactory & factory) } } +void registerCustomSeparatedSchemaReader(FormatFactory & factory) +{ + for (bool ignore_spaces : {false, true}) + { + auto register_func = [&](const String & format_name, bool with_names, bool with_types) + { + factory.registerSchemaReader(format_name, [with_names, with_types, ignore_spaces](ReadBuffer & buf, const FormatSettings & settings, ContextPtr context) + { + return std::make_shared(buf, with_names, with_types, ignore_spaces, settings, context); + }); + }; + + registerWithNamesAndTypes(ignore_spaces ? "CustomSeparatedIgnoreSpaces" : "CustomSeparated", register_func); + } +} + } diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h index 6b572ca1417..d38d5bf0da4 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h @@ -19,7 +19,6 @@ public: void resetParser() override; String getName() const override { return "CustomSeparatedRowInputFormat"; } - void setReadBuffer(ReadBuffer & in_) override; private: @@ -28,6 +27,19 @@ private: std::unique_ptr in_buf_, const Params & params_, bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_settings_); + + bool allowSyncAfterError() const override; + void syncAfterError() override; + + std::unique_ptr buf; + bool ignore_spaces; +}; + +class CustomSeparatedFormatReader : public FormatWithNamesAndTypesReader +{ +public: + CustomSeparatedFormatReader(PeekableReadBuffer & buf_, bool ignore_spaces_, const FormatSettings & format_settings_); + using EscapingRule = FormatSettings::EscapingRule; bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override; @@ -46,9 +58,6 @@ private: bool checkForSuffix() override; - bool allowSyncAfterError() const override; - void syncAfterError() override; - bool parseRowStartWithDiagnosticInfo(WriteBuffer & out) override; bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override; bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override; @@ -57,15 +66,41 @@ private: std::vector readNames() override { return readHeaderRow(); } std::vector readTypes() override { return readHeaderRow(); } - std::vector readHeaderRow(); + std::vector readHeaderRow() {return readRowImpl(); } + + std::vector readRow() { return readRowImpl(); } bool checkEndOfRow(); bool checkForSuffixImpl(bool check_eof); inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(*buf); } - std::unique_ptr buf; + EscapingRule getEscapingRule() { return format_settings.custom.escaping_rule; } + + void setReadBuffer(ReadBuffer & in_) override; +private: + template + std::vector readRowImpl(); + + template + String readFieldIntoString(bool is_first); + + PeekableReadBuffer * buf; bool ignore_spaces; - EscapingRule escaping_rule; + size_t columns = 0; +}; + +class CustomSeparatedSchemaReader : public FormatWithNamesAndTypesSchemaReader +{ +public: + CustomSeparatedSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_setting_, ContextPtr context_); + +private: + DataTypes readRowAndGetDataTypes() override; + + PeekableReadBuffer buf; + CustomSeparatedFormatReader reader; + ContextPtr context; + bool first_row = true; }; } diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp index 476985c2509..56ba975dea1 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp @@ -202,4 +202,12 @@ void registerNonTrivialPrefixAndSuffixCheckerJSONAsString(FormatFactory & factor factory.registerNonTrivialPrefixAndSuffixChecker("JSONAsString", nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl); } +void registerJSONAsStringSchemaReader(FormatFactory & factory) +{ + factory.registerExternalSchemaReader("JSONAsString", [](const FormatSettings &) + { + return std::make_shared(); + }); +} + } diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h index d86142af795..ea6e9a1ed2f 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h @@ -1,8 +1,10 @@ #pragma once #include +#include #include #include +#include namespace DB { @@ -39,4 +41,13 @@ private: bool allow_new_rows = true; }; +class JSONAsStringExternalSchemaReader : public IExternalSchemaReader +{ +public: + NamesAndTypesList readSchema() override + { + return {{"json", std::make_shared()}}; + } +}; + } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index 88fb411ffbd..da6adced0a7 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -8,16 +9,13 @@ #include #include #include +#include +#include +#include namespace DB { -namespace ErrorCodes -{ - extern const int INCORRECT_DATA; -} - - JSONCompactEachRowRowInputFormat::JSONCompactEachRowRowInputFormat( const Block & header_, ReadBuffer & in_, @@ -26,24 +24,40 @@ JSONCompactEachRowRowInputFormat::JSONCompactEachRowRowInputFormat( bool with_types_, bool yield_strings_, const FormatSettings & format_settings_) - : RowInputFormatWithNamesAndTypes(header_, in_, std::move(params_), with_names_, with_types_, format_settings_) - , yield_strings(yield_strings_) + : RowInputFormatWithNamesAndTypes( + header_, + in_, + std::move(params_), + with_names_, + with_types_, + format_settings_, + std::make_unique(in_, yield_strings_, format_settings_)) { } -void JSONCompactEachRowRowInputFormat::skipRowStartDelimiter() +void JSONCompactEachRowRowInputFormat::syncAfterError() +{ + skipToUnescapedNextLineOrEOF(*in); +} + +JSONCompactEachRowFormatReader::JSONCompactEachRowFormatReader(ReadBuffer & in_, bool yield_strings_, const FormatSettings & format_settings_) + : FormatWithNamesAndTypesReader(in_, format_settings_), yield_strings(yield_strings_) +{ +} + +void JSONCompactEachRowFormatReader::skipRowStartDelimiter() { skipWhitespaceIfAny(*in); assertChar('[', *in); } -void JSONCompactEachRowRowInputFormat::skipFieldDelimiter() +void JSONCompactEachRowFormatReader::skipFieldDelimiter() { skipWhitespaceIfAny(*in); assertChar(',', *in); } -void JSONCompactEachRowRowInputFormat::skipRowEndDelimiter() +void JSONCompactEachRowFormatReader::skipRowEndDelimiter() { skipWhitespaceIfAny(*in); assertChar(']', *in); @@ -55,29 +69,18 @@ void JSONCompactEachRowRowInputFormat::skipRowEndDelimiter() skipWhitespaceIfAny(*in); } -String JSONCompactEachRowRowInputFormat::readFieldIntoString() +void JSONCompactEachRowFormatReader::skipField() { skipWhitespaceIfAny(*in); - String field; - readJSONString(field, *in); - return field; + skipJSONField(*in, "skipped_field"); } -void JSONCompactEachRowRowInputFormat::skipField(size_t file_column) -{ - skipWhitespaceIfAny(*in); - skipJSONField(*in, column_mapping->names_of_columns[file_column]); -} - -void JSONCompactEachRowRowInputFormat::skipHeaderRow() +void JSONCompactEachRowFormatReader::skipHeaderRow() { skipRowStartDelimiter(); - size_t i = 0; do { - if (i >= column_mapping->names_of_columns.size()) - throw Exception(ErrorCodes::INCORRECT_DATA, "The number of columns in a row differs from the number of column names"); - skipField(i++); + skipField(); skipWhitespaceIfAny(*in); } while (checkChar(',', *in)); @@ -85,13 +88,16 @@ void JSONCompactEachRowRowInputFormat::skipHeaderRow() skipRowEndDelimiter(); } -std::vector JSONCompactEachRowRowInputFormat::readHeaderRow() +std::vector JSONCompactEachRowFormatReader::readHeaderRow() { skipRowStartDelimiter(); std::vector fields; + String field; do { - fields.push_back(readFieldIntoString()); + skipWhitespaceIfAny(*in); + readJSONString(field, *in); + fields.push_back(field); skipWhitespaceIfAny(*in); } while (checkChar(',', *in)); @@ -100,18 +106,13 @@ std::vector JSONCompactEachRowRowInputFormat::readHeaderRow() return fields; } -bool JSONCompactEachRowRowInputFormat::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool /*is_last_file_column*/, const String & column_name) +bool JSONCompactEachRowFormatReader::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool /*is_last_file_column*/, const String & column_name) { skipWhitespaceIfAny(*in); return readFieldImpl(*in, column, type, serialization, column_name, format_settings, yield_strings); } -void JSONCompactEachRowRowInputFormat::syncAfterError() -{ - skipToUnescapedNextLineOrEOF(*in); -} - -bool JSONCompactEachRowRowInputFormat::parseRowStartWithDiagnosticInfo(WriteBuffer & out) +bool JSONCompactEachRowFormatReader::parseRowStartWithDiagnosticInfo(WriteBuffer & out) { skipWhitespaceIfAny(*in); if (!checkChar('[', *in)) @@ -123,7 +124,7 @@ bool JSONCompactEachRowRowInputFormat::parseRowStartWithDiagnosticInfo(WriteBuff return true; } -bool JSONCompactEachRowRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) +bool JSONCompactEachRowFormatReader::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) { try { @@ -150,7 +151,7 @@ bool JSONCompactEachRowRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(Wri return true; } -bool JSONCompactEachRowRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out) +bool JSONCompactEachRowFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out) { skipWhitespaceIfAny(*in); @@ -180,6 +181,19 @@ bool JSONCompactEachRowRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer return true; } +JSONCompactEachRowRowSchemaReader::JSONCompactEachRowRowSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool yield_strings_, const FormatSettings & format_settings_) + : FormatWithNamesAndTypesSchemaReader(in_, format_settings_.max_rows_to_read_for_schema_inference, with_names_, with_types_, &reader), reader(in_, yield_strings_, format_settings_) +{ +} + +DataTypes JSONCompactEachRowRowSchemaReader::readRowAndGetDataTypes() +{ + if (in.eof()) + return {}; + + return readRowAndGetDataTypesForJSONCompactEachRow(in, reader.yieldStrings()); +} + void registerInputFormatJSONCompactEachRow(FormatFactory & factory) { for (bool yield_strings : {true, false}) @@ -200,6 +214,21 @@ void registerInputFormatJSONCompactEachRow(FormatFactory & factory) } } +void registerJSONCompactEachRowSchemaReader(FormatFactory & factory) +{ + for (bool json_strings : {false, true}) + { + auto register_func = [&](const String & format_name, bool with_names, bool with_types) + { + factory.registerSchemaReader(format_name, [=](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + { + return std::make_shared(buf, with_names, with_types, json_strings, settings); + }); + }; + registerWithNamesAndTypes(json_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", register_func); + } +} + void registerFileSegmentationEngineJSONCompactEachRow(FormatFactory & factory) { auto register_func = [&](const String & format_name, bool with_names, bool with_types) diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h index e01a4f49b30..0551aa8b64e 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -10,6 +11,7 @@ namespace DB class ReadBuffer; + /** A stream for reading data in a bunch of formats: * - JSONCompactEachRow * - JSONCompactEachRowWithNamesAndTypes @@ -34,6 +36,13 @@ public: private: bool allowSyncAfterError() const override { return true; } void syncAfterError() override; +}; + +class JSONCompactEachRowFormatReader : public FormatWithNamesAndTypesReader +{ +public: + JSONCompactEachRowFormatReader(ReadBuffer & in_, bool yield_strings_, const FormatSettings & format_settings_); + bool parseRowStartWithDiagnosticInfo(WriteBuffer & out) override; bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override; @@ -45,7 +54,8 @@ private: bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override; - void skipField(size_t file_column) override; + void skipField(size_t /*column_index*/) override { skipField(); } + void skipField(); void skipHeaderRow(); void skipNames() override { skipHeaderRow(); } void skipTypes() override { skipHeaderRow(); } @@ -56,9 +66,21 @@ private: std::vector readHeaderRow(); std::vector readNames() override { return readHeaderRow(); } std::vector readTypes() override { return readHeaderRow(); } - String readFieldIntoString(); + bool yieldStrings() const { return yield_strings; } +private: bool yield_strings; }; +class JSONCompactEachRowRowSchemaReader : public FormatWithNamesAndTypesSchemaReader +{ +public: + JSONCompactEachRowRowSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool yield_strings_, const FormatSettings & format_settings_); + +private: + DataTypes readRowAndGetDataTypes() override; + + JSONCompactEachRowFormatReader reader; +}; + } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 28481313974..75beca955b9 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -286,11 +287,7 @@ void JSONEachRowRowInputFormat::readPrefix() skipBOMIfExists(*in); skipWhitespaceIfAny(*in); - if (!in->eof() && *in->position() == '[') - { - ++in->position(); - data_in_square_brackets = true; - } + data_in_square_brackets = checkChar('[', *in); } void JSONEachRowRowInputFormat::readSuffix() @@ -309,6 +306,28 @@ void JSONEachRowRowInputFormat::readSuffix() assertEOF(*in); } +JSONEachRowSchemaReader::JSONEachRowSchemaReader(ReadBuffer & in_, bool json_strings_, const FormatSettings & format_settings) + : IRowWithNamesSchemaReader(in_, format_settings.max_rows_to_read_for_schema_inference), json_strings(json_strings_) +{ +} + + +std::unordered_map JSONEachRowSchemaReader::readRowAndGetNamesAndDataTypes() +{ + if (first_row) + { + skipBOMIfExists(in); + skipWhitespaceIfAny(in); + checkChar('[', in); + first_row = false; + } + + skipWhitespaceIfAny(in); + if (in.eof()) + return {}; + + return readRowAndGetNamesAndDataTypesForJSONEachRow(in, json_strings); +} void registerInputFormatJSONEachRow(FormatFactory & factory) { @@ -343,4 +362,17 @@ void registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(FormatFactory & factory factory.registerNonTrivialPrefixAndSuffixChecker("JSONStringsEachRow", nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl); } +void registerJSONEachRowSchemaReader(FormatFactory & factory) +{ + factory.registerSchemaReader("JSONEachRow", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + { + return std::make_unique(buf, false, settings); + }); + + factory.registerSchemaReader("JSONStringsEachRow", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + { + return std::make_unique(buf, true, settings); + }); +} + } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h index 9810f2dc765..323909a7730 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -84,4 +85,16 @@ private: bool yield_strings; }; +class JSONEachRowSchemaReader : public IRowWithNamesSchemaReader +{ +public: + JSONEachRowSchemaReader(ReadBuffer & in_, bool json_strings, const FormatSettings & format_settings); + +private: + std::unordered_map readRowAndGetNamesAndDataTypes() override; + + bool json_strings; + bool first_row = true; +}; + } diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp index 1a05f61d36b..5983f3170e5 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -72,4 +72,13 @@ void registerInputFormatLineAsString(FormatFactory & factory) return std::make_shared(sample, buf, params); }); } + +void registerLineAsStringSchemaReader(FormatFactory & factory) +{ + factory.registerExternalSchemaReader("LineAsString", []( + const FormatSettings &) + { + return std::make_shared(); + }); +} } diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h index 1a6c6247558..c4c17c47dbe 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h @@ -1,7 +1,9 @@ #pragma once #include +#include #include +#include namespace DB { @@ -26,4 +28,13 @@ private: void readLineObject(IColumn & column); }; +class LinaAsStringSchemaReader : public IExternalSchemaReader +{ +public: + NamesAndTypesList readSchema() override + { + return {{"line", std::make_shared()}}; + } +}; + } diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index 60db32d879a..e4f727857ba 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include @@ -26,6 +27,8 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int INCORRECT_DATA; + extern const int BAD_ARGUMENTS; + extern const int UNEXPECTED_END_OF_FILE; } MsgPackRowInputFormat::MsgPackRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) @@ -369,7 +372,109 @@ bool MsgPackRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & void MsgPackRowInputFormat::setReadBuffer(ReadBuffer & in_) { buf = std::make_unique(in_); - IInputFormat::setReadBuffer(*buf); + IInputFormat::setReadBuffer(in_); +} + +MsgPackSchemaReader::MsgPackSchemaReader(ReadBuffer & in_, const FormatSettings & forma_settings_) + : IRowSchemaReader(buf, forma_settings_.max_rows_to_read_for_schema_inference), buf(in_), number_of_columns(forma_settings_.msgpack.number_of_columns) +{ + if (!number_of_columns) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "You must specify setting input_format_msgpack_number_of_columns to extract table schema from MsgPack data"); +} + + +msgpack::object_handle MsgPackSchemaReader::readObject() +{ + if (buf.eof()) + throw Exception(ErrorCodes::UNEXPECTED_END_OF_FILE, "Unexpected eof while parsing msgpack object"); + + PeekableReadBufferCheckpoint checkpoint{buf}; + size_t offset = 0; + bool need_more_data = true; + msgpack::object_handle object_handle; + while (need_more_data) + { + offset = 0; + try + { + object_handle = msgpack::unpack(buf.position(), buf.buffer().end() - buf.position(), offset); + need_more_data = false; + } + catch (msgpack::insufficient_bytes &) + { + buf.position() = buf.buffer().end(); + if (buf.eof()) + throw Exception("Unexpected end of file while parsing msgpack object", ErrorCodes::UNEXPECTED_END_OF_FILE); + buf.position() = buf.buffer().end(); + buf.makeContinuousMemoryFromCheckpointToPos(); + buf.rollbackToCheckpoint(); + } + } + buf.position() += offset; + return object_handle; +} + +DataTypePtr MsgPackSchemaReader::getDataType(const msgpack::object & object) +{ + switch (object.type) + { + case msgpack::type::object_type::POSITIVE_INTEGER: + return makeNullable(std::make_shared()); + case msgpack::type::object_type::NEGATIVE_INTEGER: + return makeNullable(std::make_shared()); + case msgpack::type::object_type::FLOAT32: + return makeNullable(std::make_shared()); + case msgpack::type::object_type::FLOAT64: + return makeNullable(std::make_shared()); + case msgpack::type::object_type::BOOLEAN: + return makeNullable(std::make_shared()); + case msgpack::type::object_type::BIN: [[fallthrough]]; + case msgpack::type::object_type::STR: + return makeNullable(std::make_shared()); + case msgpack::type::object_type::ARRAY: + { + msgpack::object_array object_array = object.via.array; + if (object_array.size) + { + auto nested_type = getDataType(object_array.ptr[0]); + if (nested_type) + return std::make_shared(getDataType(object_array.ptr[0])); + } + return nullptr; + } + case msgpack::type::object_type::MAP: + { + msgpack::object_map object_map = object.via.map; + if (object_map.size) + { + auto key_type = getDataType(object_map.ptr[0].key); + auto value_type = getDataType(object_map.ptr[1].val); + if (key_type && value_type) + return std::make_shared(key_type, value_type); + } + return nullptr; + } + case msgpack::type::object_type::NIL: + return nullptr; + default: + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Msgpack type is not supported"); + } +} + +DataTypes MsgPackSchemaReader::readRowAndGetDataTypes() +{ + if (buf.eof()) + return {}; + + DataTypes data_types; + data_types.reserve(number_of_columns); + for (size_t i = 0; i != number_of_columns; ++i) + { + auto object_handle = readObject(); + data_types.push_back(getDataType(object_handle.get())); + } + + return data_types; } void registerInputFormatMsgPack(FormatFactory & factory) @@ -384,6 +489,14 @@ void registerInputFormatMsgPack(FormatFactory & factory) }); } +void registerMsgPackSchemaReader(FormatFactory & factory) +{ + factory.registerSchemaReader("MsgPack", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + { + return std::make_shared(buf, settings); + }); +} + } #else @@ -394,6 +507,10 @@ class FormatFactory; void registerInputFormatMsgPack(FormatFactory &) { } + +void registerMsgPackSchemaReader(FormatFactory &) +{ +} } #endif diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h index bb3887695eb..dd5655c80fc 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h @@ -6,6 +6,7 @@ #if USE_MSGPACK #include +#include #include #include #include @@ -76,6 +77,20 @@ private: const DataTypes data_types; }; +class MsgPackSchemaReader : public IRowSchemaReader +{ +public: + MsgPackSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_); + +private: + msgpack::object_handle readObject(); + DataTypePtr getDataType(const msgpack::object & object); + DataTypes readRowAndGetDataTypes() override; + + PeekableReadBuffer buf; + UInt64 number_of_columns; +}; + } #endif diff --git a/src/Processors/Formats/Impl/NativeFormat.cpp b/src/Processors/Formats/Impl/NativeFormat.cpp index 07cf4670981..d7c70e1c37b 100644 --- a/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/src/Processors/Formats/Impl/NativeFormat.cpp @@ -1,8 +1,10 @@ #include #include + #include #include #include +#include #include @@ -82,6 +84,20 @@ private: NativeWriter writer; }; +class NativeSchemaReader : public ISchemaReader +{ +public: + NativeSchemaReader(ReadBuffer & in_) : ISchemaReader(in_) {} + + NamesAndTypesList readSchema() override + { + auto reader = NativeReader(in, 0); + auto block = reader.read(); + return block.getNamesAndTypesList(); + } +}; + + void registerInputFormatNative(FormatFactory & factory) { factory.registerInputFormat("Native", []( @@ -106,4 +122,14 @@ void registerOutputFormatNative(FormatFactory & factory) }); } + +void registerNativeSchemaReader(FormatFactory & factory) +{ + factory.registerSchemaReader("Native", [](ReadBuffer & buf, const FormatSettings &, ContextPtr) + { + return std::make_shared(buf); + }); +} + + } diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 87eec459aa3..44f6a44e492 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -87,7 +87,8 @@ static size_t countIndicesForType(std::shared_ptr type) return 1; } -void ORCBlockInputFormat::prepareReader() +static void getFileReaderAndSchema( + ReadBuffer & in, std::unique_ptr & file_reader, std::shared_ptr & schema, const FormatSettings & format_settings) { auto arrow_file = asArrowFile(*in, format_settings, is_stopped); if (is_stopped) @@ -101,7 +102,13 @@ void ORCBlockInputFormat::prepareReader() auto read_schema_result = file_reader->ReadSchema(); if (!read_schema_result.ok()) throw Exception(read_schema_result.status().ToString(), ErrorCodes::BAD_ARGUMENTS); - std::shared_ptr schema = std::move(read_schema_result).ValueOrDie(); + schema = std::move(read_schema_result).ValueOrDie(); +} + +void ORCBlockInputFormat::prepareReader() +{ + std::shared_ptr schema; + getFileReaderAndSchema(*in, file_reader, schema, format_settings); arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), "ORC", format_settings.orc.import_nested); @@ -128,7 +135,20 @@ void ORCBlockInputFormat::prepareReader() } } -void registerInputFormatORC(FormatFactory &factory) +ORCSchemaReader::ORCSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) : ISchemaReader(in_), format_settings(format_settings_) +{ +} + +NamesAndTypesList ORCSchemaReader::readSchema() +{ + std::unique_ptr file_reader; + std::shared_ptr schema; + getFileReaderAndSchema(in, file_reader, schema, format_settings); + auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader(*schema, "ORC"); + return header.getNamesAndTypesList(); +} + +void registerInputFormatORC(FormatFactory & factory) { factory.registerInputFormat( "ORC", @@ -142,6 +162,17 @@ void registerInputFormatORC(FormatFactory &factory) factory.markFormatAsColumnOriented("ORC"); } +void registerORCSchemaReader(FormatFactory & factory) +{ + factory.registerSchemaReader( + "ORC", + [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + { + return std::make_shared(buf, settings); + } + ); +} + } #else @@ -151,6 +182,10 @@ namespace DB void registerInputFormatORC(FormatFactory &) { } + + void registerORCSchemaReader(FormatFactory &) + { + } } #endif diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h index c7dc1c4a710..9b55747f552 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -3,6 +3,7 @@ #if USE_ORC #include +#include #include #include @@ -54,5 +55,16 @@ private: std::atomic is_stopped{0}; }; +class ORCSchemaReader : public ISchemaReader +{ +public: + ORCSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_); + + NamesAndTypesList readSchema() override; + +private: + const FormatSettings format_settings; +}; + } #endif diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 0b6cd006300..1d303014d31 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -94,19 +94,30 @@ static size_t countIndicesForType(std::shared_ptr type) return 1; } +static void getFileReaderAndSchema( + ReadBuffer & in, + std::unique_ptr & file_reader, + std::shared_ptr & schema, + const FormatSettings & format_settings, + std::atomic & is_stopped) +{ + auto arrow_file = asArrowFile(in, format_settings, is_stopped); + if (is_stopped) + return; + THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(std::move(arrow_file), arrow::default_memory_pool(), &file_reader)); + THROW_ARROW_NOT_OK(file_reader->GetSchema(&schema)); +} + void ParquetBlockInputFormat::prepareReader() { - auto arrow_file = asArrowFile(*in, format_settings, is_stopped); + std::shared_ptr schema; + getFileReaderAndSchema(*in, file_reader, schema, format_settings, is_stopped); if (is_stopped) return; - THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(std::move(arrow_file), arrow::default_memory_pool(), &file_reader)); row_group_total = file_reader->num_row_groups(); row_group_current = 0; - std::shared_ptr schema; - THROW_ARROW_NOT_OK(file_reader->GetSchema(&schema)); - arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), "Parquet", format_settings.parquet.import_nested); std::unordered_set nested_table_names; @@ -130,7 +141,21 @@ void ParquetBlockInputFormat::prepareReader() } } -void registerInputFormatParquet(FormatFactory &factory) +ParquetSchemaReader::ParquetSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) : ISchemaReader(in_), format_settings(format_settings_) +{ +} + +NamesAndTypesList ParquetSchemaReader::readSchema() +{ + std::unique_ptr file_reader; + std::shared_ptr schema; + std::atomic is_stopped = 0; + getFileReaderAndSchema(in, file_reader, schema, format_settings, is_stopped); + auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader(*schema, "Parquet"); + return header.getNamesAndTypesList(); +} + +void registerInputFormatParquet(FormatFactory & factory) { factory.registerInputFormat( "Parquet", @@ -144,6 +169,17 @@ void registerInputFormatParquet(FormatFactory &factory) factory.markFormatAsColumnOriented("Parquet"); } +void registerParquetSchemaReader(FormatFactory & factory) +{ + factory.registerSchemaReader( + "Parquet", + [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + { + return std::make_shared(buf, settings); + } + ); +} + } #else @@ -154,6 +190,8 @@ class FormatFactory; void registerInputFormatParquet(FormatFactory &) { } + +void registerParquetSchemaReader(FormatFactory &) {} } #endif diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index 3e04c523442..dbc99c08a35 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -3,6 +3,7 @@ #if USE_PARQUET #include +#include #include namespace parquet::arrow { class FileReader; } @@ -44,6 +45,17 @@ private: std::atomic is_stopped{0}; }; +class ParquetSchemaReader : public ISchemaReader +{ +public: + ParquetSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_); + + NamesAndTypesList readSchema() override; + +private: + const FormatSettings format_settings; +}; + } #endif diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp index df7b7102739..66da27e8829 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp @@ -73,6 +73,34 @@ void registerInputFormatProtobuf(FormatFactory & factory) } } +ProtobufSchemaReader::ProtobufSchemaReader(const FormatSettings & format_settings) + : schema_info( + format_settings.schema.format_schema, + "Protobuf", + true, + format_settings.schema.is_server, + format_settings.schema.format_schema_path) +{ +} + +NamesAndTypesList ProtobufSchemaReader::readSchema() +{ + const auto * message_descriptor = ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info); + return protobufSchemaToCHSchema(message_descriptor); +} + +void registerProtobufSchemaReader(FormatFactory & factory) +{ + factory.registerExternalSchemaReader("Protobuf", [](const FormatSettings & settings) + { + return std::make_shared(settings); + }); + factory.registerExternalSchemaReader("ProtobufSingle", [](const FormatSettings & settings) + { + return std::make_shared(settings); + }); +} + } #else @@ -81,6 +109,8 @@ namespace DB { class FormatFactory; void registerInputFormatProtobuf(FormatFactory &) {} + +void registerProtobufSchemaReader(FormatFactory &) {} } #endif diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h index 6f465e3f0b8..15305977619 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h @@ -3,7 +3,9 @@ #include "config_formats.h" #if USE_PROTOBUF +# include # include +# include namespace DB { @@ -42,5 +44,16 @@ private: std::unique_ptr serializer; }; +class ProtobufSchemaReader : public IExternalSchemaReader +{ +public: + ProtobufSchemaReader(const FormatSettings & format_settings); + + NamesAndTypesList readSchema() override; + +private: + FormatSchemaInfo schema_info; +}; + } #endif diff --git a/src/Processors/Formats/Impl/RawBLOBRowInputFormat.cpp b/src/Processors/Formats/Impl/RawBLOBRowInputFormat.cpp index 34424fffd34..91b1cc60fae 100644 --- a/src/Processors/Formats/Impl/RawBLOBRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/RawBLOBRowInputFormat.cpp @@ -51,5 +51,14 @@ void registerInputFormatRawBLOB(FormatFactory & factory) }); } +void registerRawBLOBSchemaReader(FormatFactory & factory) +{ + factory.registerExternalSchemaReader("RawBLOB", []( + const FormatSettings &) + { + return std::make_shared(); + }); +} + } diff --git a/src/Processors/Formats/Impl/RawBLOBRowInputFormat.h b/src/Processors/Formats/Impl/RawBLOBRowInputFormat.h index 343af9f4068..367ca04f9d8 100644 --- a/src/Processors/Formats/Impl/RawBLOBRowInputFormat.h +++ b/src/Processors/Formats/Impl/RawBLOBRowInputFormat.h @@ -1,6 +1,8 @@ #pragma once #include +#include +#include namespace DB @@ -22,5 +24,14 @@ private: bool readRow(MutableColumns & columns, RowReadExtension &) override; }; +class RawBLOBSchemaReader: public IExternalSchemaReader +{ +public: + NamesAndTypesList readSchema() override + { + return {{"raw_blob", std::make_shared()}}; + } +}; + } diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index 279ae89aba5..23b8c8212c0 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -14,18 +14,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -RegexpRowInputFormat::RegexpRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_) - : RegexpRowInputFormat(std::make_unique(in_), header_, params_, format_settings_) -{ -} - -RegexpRowInputFormat::RegexpRowInputFormat( - std::unique_ptr buf_, const Block & header_, Params params_, const FormatSettings & format_settings_) - : IRowInputFormat(header_, *buf_, std::move(params_)) - , buf(std::move(buf_)) - , format_settings(format_settings_) - , escaping_rule(format_settings_.regexp.escaping_rule) - , regexp(format_settings_.regexp.regexp) +RegexpFieldExtractor::RegexpFieldExtractor(const FormatSettings & format_settings) : regexp(format_settings.regexp.regexp), skip_unmatched(format_settings.regexp.skip_unmatched) { size_t fields_count = regexp.NumberOfCapturingGroups(); matched_fields.resize(fields_count); @@ -40,6 +29,50 @@ RegexpRowInputFormat::RegexpRowInputFormat( } } +bool RegexpFieldExtractor::parseRow(PeekableReadBuffer & buf) +{ + PeekableReadBufferCheckpoint checkpoint{buf}; + + size_t line_size = 0; + + do + { + char * pos = find_first_symbols<'\n', '\r'>(buf.position(), buf.buffer().end()); + line_size += pos - buf.position(); + buf.position() = pos; + } while (buf.position() == buf.buffer().end() && !buf.eof()); + + buf.makeContinuousMemoryFromCheckpointToPos(); + buf.rollbackToCheckpoint(); + + bool match = RE2::FullMatchN(re2::StringPiece(buf.position(), line_size), regexp, re2_arguments_ptrs.data(), re2_arguments_ptrs.size()); + + if (!match && !skip_unmatched) + throw Exception("Line \"" + std::string(buf.position(), line_size) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA); + + buf.position() += line_size; + checkChar('\r', buf); + if (!buf.eof() && !checkChar('\n', buf)) + throw Exception("No \\n after \\r at the end of line.", ErrorCodes::INCORRECT_DATA); + + return match; +} + +RegexpRowInputFormat::RegexpRowInputFormat( + ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_) + : RegexpRowInputFormat(std::make_unique(in_), header_, params_, format_settings_) +{ +} + +RegexpRowInputFormat::RegexpRowInputFormat( + std::unique_ptr buf_, const Block & header_, Params params_, const FormatSettings & format_settings_) + : IRowInputFormat(header_, *buf_, std::move(params_)) + , buf(std::move(buf_)) + , format_settings(format_settings_) + , escaping_rule(format_settings_.regexp.escaping_rule) + , field_extractor(RegexpFieldExtractor(format_settings_)) +{ +} void RegexpRowInputFormat::resetParser() { @@ -50,7 +83,8 @@ void RegexpRowInputFormat::resetParser() bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) { const auto & type = getPort().getHeader().getByPosition(index).type; - ReadBuffer field_buf(const_cast(matched_fields[index].data()), matched_fields[index].size(), 0); + auto matched_field = field_extractor.getField(index); + ReadBuffer field_buf(const_cast(matched_field.data()), matched_field.size(), 0); try { return deserializeFieldByEscapingRule(type, serializations[index], *columns[index], field_buf, escaping_rule, format_settings); @@ -64,7 +98,7 @@ bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) void RegexpRowInputFormat::readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext) { - if (matched_fields.size() != columns.size()) + if (field_extractor.getMatchedFieldsSize() != columns.size()) throw Exception("The number of matched fields in line doesn't match the number of columns.", ErrorCodes::INCORRECT_DATA); ext.read_columns.assign(columns.size(), false); @@ -79,39 +113,8 @@ bool RegexpRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & if (buf->eof()) return false; - PeekableReadBufferCheckpoint checkpoint{*buf}; - - size_t line_size = 0; - - do - { - char * pos = find_first_symbols<'\n', '\r'>(buf->position(), buf->buffer().end()); - line_size += pos - buf->position(); - buf->position() = pos; - } while (buf->position() == buf->buffer().end() && !buf->eof()); - - buf->makeContinuousMemoryFromCheckpointToPos(); - buf->rollbackToCheckpoint(); - - bool match = RE2::FullMatchN(re2::StringPiece(buf->position(), line_size), regexp, re2_arguments_ptrs.data(), re2_arguments_ptrs.size()); - bool read_line = true; - - if (!match) - { - if (!format_settings.regexp.skip_unmatched) - throw Exception("Line \"" + std::string(buf->position(), line_size) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA); - read_line = false; - } - - if (read_line) + if (field_extractor.parseRow(*buf)) readFieldsFromMatch(columns, ext); - - buf->position() += line_size; - - checkChar('\r', *buf); - if (!buf->eof() && !checkChar('\n', *buf)) - throw Exception("No \\n after \\r at the end of line.", ErrorCodes::INCORRECT_DATA); - return true; } @@ -121,6 +124,35 @@ void RegexpRowInputFormat::setReadBuffer(ReadBuffer & in_) IInputFormat::setReadBuffer(*buf); } +RegexpSchemaReader::RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) + : IRowSchemaReader( + buf, + format_settings_.max_rows_to_read_for_schema_inference, + getDefaultDataTypeForEscapingRule(format_settings_.regexp.escaping_rule)) + , format_settings(format_settings_) + , field_extractor(format_settings) + , buf(in_) +{ +} + +DataTypes RegexpSchemaReader::readRowAndGetDataTypes() +{ + if (buf.eof()) + return {}; + + field_extractor.parseRow(buf); + + DataTypes data_types; + data_types.reserve(field_extractor.getMatchedFieldsSize()); + for (size_t i = 0; i != field_extractor.getMatchedFieldsSize(); ++i) + { + String field(field_extractor.getField(i)); + data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, format_settings.regexp.escaping_rule)); + } + + return data_types; +} + void registerInputFormatRegexp(FormatFactory & factory) { factory.registerInputFormat("Regexp", []( @@ -172,4 +204,12 @@ void registerFileSegmentationEngineRegexp(FormatFactory & factory) factory.registerFileSegmentationEngine("Regexp", &fileSegmentationEngineRegexpImpl); } +void registerRegexpSchemaReader(FormatFactory & factory) +{ + factory.registerSchemaReader("Regexp", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + { + return std::make_shared(buf, settings); + }); +} + } diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.h b/src/Processors/Formats/Impl/RegexpRowInputFormat.h index dbce31a9b49..acd7b0d0934 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.h +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -16,6 +17,29 @@ namespace DB class ReadBuffer; +/// Class for extracting row fields from data by regexp. +class RegexpFieldExtractor +{ +public: + RegexpFieldExtractor(const FormatSettings & format_settings); + + /// Return true if row was successfully parsed and row fields were extracted. + bool parseRow(PeekableReadBuffer & buf); + + re2::StringPiece getField(size_t index) { return matched_fields[index]; } + size_t getMatchedFieldsSize() const { return matched_fields.size(); } + size_t getNumberOfGroups() const { return regexp.NumberOfCapturingGroups(); } + +private: + const RE2 regexp; + // The vector of fields extracted from line using regexp. + std::vector matched_fields; + // These two vectors are needed to use RE2::FullMatchN (function for extracting fields). + std::vector re2_arguments; + std::vector re2_arguments_ptrs; + bool skip_unmatched; +}; + /// Regexp input format. /// This format applies regular expression from format_regexp setting for every line of file /// (the lines must be separated by newline character ('\n') or DOS-style newline ("\r\n")). @@ -25,7 +49,6 @@ class ReadBuffer; class RegexpRowInputFormat : public IRowInputFormat { - using EscapingRule = FormatSettings::EscapingRule; public: RegexpRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_); @@ -36,6 +59,8 @@ public: private: RegexpRowInputFormat(std::unique_ptr buf_, const Block & header_, Params params_, const FormatSettings & format_settings_); + using EscapingRule = FormatSettings::EscapingRule; + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; bool readField(size_t index, MutableColumns & columns); @@ -44,13 +69,21 @@ private: std::unique_ptr buf; const FormatSettings format_settings; const EscapingRule escaping_rule; + RegexpFieldExtractor field_extractor; +}; - const RE2 regexp; - // The vector of fields extracted from line using regexp. - std::vector matched_fields; - // These two vectors are needed to use RE2::FullMatchN (function for extracting fields). - std::vector re2_arguments; - std::vector re2_arguments_ptrs; +class RegexpSchemaReader : public IRowSchemaReader +{ +public: + RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings); + +private: + DataTypes readRowAndGetDataTypes() override; + + using EscapingRule = FormatSettings::EscapingRule; + const FormatSettings format_settings; + RegexpFieldExtractor field_extractor; + PeekableReadBuffer buf; }; } diff --git a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp index eef97e15dd5..8a56c2ed5c7 100644 --- a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp @@ -1,7 +1,10 @@ #include #include #include +#include #include +#include +#include namespace DB @@ -211,6 +214,59 @@ void TSKVRowInputFormat::resetParser() name_buf.clear(); } +TSKVSchemaReader::TSKVSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) + : IRowWithNamesSchemaReader( + in_, + format_settings_.max_rows_to_read_for_schema_inference, + getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule::Escaped)) + , format_settings(format_settings_) +{ +} + +std::unordered_map TSKVSchemaReader::readRowAndGetNamesAndDataTypes() +{ + if (first_row) + { + skipBOMIfExists(in); + first_row = false; + } + + if (in.eof()) + return {}; + + if (*in.position() == '\n') + { + ++in.position(); + return {}; + } + + std::unordered_map names_and_types; + StringRef name_ref; + String name_tmp; + String value; + do + { + bool has_value = readName(in, name_ref, name_tmp); + if (has_value) + { + readEscapedString(value, in); + names_and_types[String(name_ref)] = determineDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Escaped); + } + else + { + /// The only thing that can go without value is `tskv` fragment that is ignored. + if (!(name_ref.size == 4 && 0 == memcmp(name_ref.data, "tskv", 4))) + throw Exception("Found field without value while parsing TSKV format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA); + } + + } + while (checkChar('\t', in)); + + assertChar('\n', in); + + return names_and_types; +} + void registerInputFormatTSKV(FormatFactory & factory) { factory.registerInputFormat("TSKV", []( @@ -222,5 +278,12 @@ void registerInputFormatTSKV(FormatFactory & factory) return std::make_shared(buf, sample, std::move(params), settings); }); } +void registerTSKVSchemaReader(FormatFactory & factory) +{ + factory.registerSchemaReader("TSKV", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + { + return std::make_shared(buf, settings); + }); +} } diff --git a/src/Processors/Formats/Impl/TSKVRowInputFormat.h b/src/Processors/Formats/Impl/TSKVRowInputFormat.h index 7d732bae691..6aef50a0f84 100644 --- a/src/Processors/Formats/Impl/TSKVRowInputFormat.h +++ b/src/Processors/Formats/Impl/TSKVRowInputFormat.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -52,4 +53,16 @@ private: /// for row like ..., non-nullable column name=\N, ... }; +class TSKVSchemaReader : public IRowWithNamesSchemaReader +{ +public: + TSKVSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_); + +private: + std::unordered_map readRowAndGetNamesAndDataTypes() override; + + const FormatSettings format_settings; + bool first_row = true; +}; + } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 1e6d238b202..bb844ec68ea 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -1,13 +1,15 @@ #include #include -#include -#include -#include -#include -#include -#include +#include +#include #include +#include +#include +#include +#include +#include +#include namespace DB { @@ -38,40 +40,50 @@ TabSeparatedRowInputFormat::TabSeparatedRowInputFormat( bool with_types_, bool is_raw_, const FormatSettings & format_settings_) - : RowInputFormatWithNamesAndTypes(header_, in_, params_, with_names_, with_types_, format_settings_), is_raw(is_raw_) + : RowInputFormatWithNamesAndTypes(header_, in_, params_, with_names_, with_types_, format_settings_, std::make_unique(in_, format_settings_, is_raw_)) { } -void TabSeparatedRowInputFormat::skipFieldDelimiter() +TabSeparatedFormatReader::TabSeparatedFormatReader(ReadBuffer & in_, const FormatSettings & format_settings_, bool is_raw_) + : FormatWithNamesAndTypesReader(in_, format_settings_), is_raw(is_raw_) +{ +} + +void TabSeparatedFormatReader::skipFieldDelimiter() { assertChar('\t', *in); } -void TabSeparatedRowInputFormat::skipRowEndDelimiter() +void TabSeparatedFormatReader::skipRowEndDelimiter() { if (in->eof()) return; - if (unlikely(row_num <= 1)) + if (unlikely(first_row)) + { checkForCarriageReturn(*in); + first_row = false; + } assertChar('\n', *in); } -String TabSeparatedRowInputFormat::readFieldIntoString() +String TabSeparatedFormatReader::readFieldIntoString() { String field; - readEscapedString(field, *in); + if (is_raw) + readString(field, *in); + else + readEscapedString(field, *in); return field; } -void TabSeparatedRowInputFormat::skipField() +void TabSeparatedFormatReader::skipField() { - NullOutput null_sink; - readEscapedStringInto(null_sink, *in); + readFieldIntoString(); } -void TabSeparatedRowInputFormat::skipHeaderRow() +void TabSeparatedFormatReader::skipHeaderRow() { do { @@ -82,7 +94,7 @@ void TabSeparatedRowInputFormat::skipHeaderRow() skipRowEndDelimiter(); } -std::vector TabSeparatedRowInputFormat::readHeaderRow() +std::vector TabSeparatedFormatReader::readRow() { std::vector fields; do @@ -95,7 +107,7 @@ std::vector TabSeparatedRowInputFormat::readHeaderRow() return fields; } -bool TabSeparatedRowInputFormat::readField(IColumn & column, const DataTypePtr & type, +bool TabSeparatedFormatReader::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & /*column_name*/) { const bool at_delimiter = !is_last_file_column && !in->eof() && *in->position() == '\t'; @@ -118,6 +130,7 @@ bool TabSeparatedRowInputFormat::readField(IColumn & column, const DataTypePtr & return true; } + if (as_nullable) return SerializationNullable::deserializeTextEscapedImpl(column, *in, format_settings, serialization); @@ -125,7 +138,7 @@ bool TabSeparatedRowInputFormat::readField(IColumn & column, const DataTypePtr & return true; } -bool TabSeparatedRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) +bool TabSeparatedFormatReader::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) { try { @@ -156,7 +169,7 @@ bool TabSeparatedRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuff return true; } -bool TabSeparatedRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out) +bool TabSeparatedFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out) { if (in->eof()) return true; @@ -190,7 +203,7 @@ bool TabSeparatedRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out return true; } -void TabSeparatedRowInputFormat::checkNullValueForNonNullable(DataTypePtr type) +void TabSeparatedFormatReader::checkNullValueForNonNullable(DataTypePtr type) { bool can_be_parsed_as_null = type->isNullable() || type->isLowCardinalityNullable() || format_settings.null_as_default; @@ -218,6 +231,28 @@ void TabSeparatedRowInputFormat::syncAfterError() skipToUnescapedNextLineOrEOF(*in); } +TabSeparatedSchemaReader::TabSeparatedSchemaReader( + ReadBuffer & in_, bool with_names_, bool with_types_, bool is_raw_, const FormatSettings & format_settings_) + : FormatWithNamesAndTypesSchemaReader( + in_, + format_settings_.max_rows_to_read_for_schema_inference, + with_names_, + with_types_, + &reader, + getDefaultDataTypeForEscapingRule(is_raw_ ? FormatSettings::EscapingRule::Raw : FormatSettings::EscapingRule::Escaped)) + , reader(in_, format_settings_, is_raw_) +{ +} + +DataTypes TabSeparatedSchemaReader::readRowAndGetDataTypes() +{ + if (in.eof()) + return {}; + + auto fields = reader.readRow(); + return determineDataTypesByEscapingRule(fields, reader.getFormatSettings(), reader.getEscapingRule()); +} + void registerInputFormatTabSeparated(FormatFactory & factory) { for (bool is_raw : {false, true}) @@ -239,6 +274,23 @@ void registerInputFormatTabSeparated(FormatFactory & factory) } } +void registerTSVSchemaReader(FormatFactory & factory) +{ + for (bool is_raw : {false, true}) + { + auto register_func = [&](const String & format_name, bool with_names, bool with_types) + { + factory.registerSchemaReader(format_name, [with_names, with_types, is_raw](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + { + return std::make_shared(buf, with_names, with_types, is_raw, settings); + }); + }; + + registerWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func); + registerWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func); + } +} + static std::pair fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, bool is_raw, size_t min_rows) { bool need_more_data = true; diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h index 6e2e283e792..1f2bfc255b8 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -24,6 +25,13 @@ public: private: bool allowSyncAfterError() const override { return true; } void syncAfterError() override; + bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override { return *pos != '\n' && *pos != '\t'; } +}; + +class TabSeparatedFormatReader : public FormatWithNamesAndTypesReader +{ +public: + TabSeparatedFormatReader(ReadBuffer & in_, const FormatSettings & format_settings, bool is_raw_); bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override; @@ -36,18 +44,34 @@ private: void skipFieldDelimiter() override; void skipRowEndDelimiter() override; - std::vector readHeaderRow(); - std::vector readNames() override { return readHeaderRow(); } - std::vector readTypes() override { return readHeaderRow(); } + std::vector readRow(); + std::vector readNames() override { return readRow(); } + std::vector readTypes() override { return readRow(); } String readFieldIntoString(); void checkNullValueForNonNullable(DataTypePtr type) override; bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override; bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override; - bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override { return *pos != '\n' && *pos != '\t'; } + FormatSettings::EscapingRule getEscapingRule() + { + return is_raw ? FormatSettings::EscapingRule::Raw : FormatSettings::EscapingRule::Escaped; + } +private: bool is_raw; + bool first_row = true; +}; + +class TabSeparatedSchemaReader : public FormatWithNamesAndTypesSchemaReader +{ +public: + TabSeparatedSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool is_raw_, const FormatSettings & format_settings); + +private: + DataTypes readRowAndGetDataTypes() override; + + TabSeparatedFormatReader reader; }; } diff --git a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp index fccf6eb10df..06d6ba06bcc 100644 --- a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include namespace DB @@ -12,13 +11,19 @@ namespace DB namespace ErrorCodes { -extern const int ATTEMPT_TO_READ_AFTER_EOF; -extern const int CANNOT_READ_ALL_DATA; -extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; -extern const int CANNOT_PARSE_QUOTED_STRING; -extern const int SYNTAX_ERROR; + extern const int ATTEMPT_TO_READ_AFTER_EOF; + extern const int CANNOT_READ_ALL_DATA; + extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; + extern const int CANNOT_PARSE_QUOTED_STRING; + extern const int SYNTAX_ERROR; } +[[noreturn]] static void throwUnexpectedEof(size_t row_num) +{ + throw ParsingException("Unexpected EOF while parsing row " + std::to_string(row_num) + ". " + "Maybe last row has wrong format or input doesn't contain specified suffix before EOF.", + ErrorCodes::CANNOT_READ_ALL_DATA); +} TemplateRowInputFormat::TemplateRowInputFormat( const Block & header_, @@ -41,37 +46,13 @@ TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, std::uniqu : RowInputFormatWithDiagnosticInfo(header_, *buf_, params_), buf(std::move(buf_)), data_types(header_.getDataTypes()), settings(std::move(settings_)), ignore_spaces(ignore_spaces_), format(std::move(format_)), row_format(std::move(row_format_)), - default_csv_delimiter(settings.csv.delimiter), row_between_delimiter(std::move(row_between_delimiter_)) + default_csv_delimiter(settings.csv.delimiter), row_between_delimiter(row_between_delimiter_), + format_reader(std::make_unique(*buf, ignore_spaces_, format, row_format, row_between_delimiter, settings)) { - /// Validate format string for result set - bool has_data = false; - for (size_t i = 0; i < format.columnsCount(); ++i) - { - if (format.format_idx_to_column_idx[i]) - { - if (*format.format_idx_to_column_idx[i] != 0) - format.throwInvalidFormat("Invalid input part", i); - if (has_data) - format.throwInvalidFormat("${data} can occur only once", i); - if (format.escaping_rules[i] != EscapingRule::None) - format.throwInvalidFormat("${data} must have empty or None deserialization type", i); - has_data = true; - format_data_idx = i; - } - else - { - if (format.escaping_rules[i] == EscapingRule::XML) - format.throwInvalidFormat("XML deserialization is not supported", i); - } - } - /// Validate format string for rows std::vector column_in_format(header_.columns(), false); for (size_t i = 0; i < row_format.columnsCount(); ++i) { - if (row_format.escaping_rules[i] == EscapingRule::XML) - row_format.throwInvalidFormat("XML deserialization is not supported", i); - if (row_format.format_idx_to_column_idx[i]) { if (header_.columns() <= *row_format.format_idx_to_column_idx[i]) @@ -94,69 +75,7 @@ TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, std::uniqu void TemplateRowInputFormat::readPrefix() { - size_t last_successfully_parsed_idx = 0; - try - { - tryReadPrefixOrSuffix(last_successfully_parsed_idx, format_data_idx); - } - catch (Exception & e) - { - format.throwInvalidFormat(e.message() + " While parsing prefix", last_successfully_parsed_idx); - } -} - -/// Asserts delimiters and skips fields in prefix or suffix. -/// tryReadPrefixOrSuffix(...) is used in checkForSuffix() to avoid throwing an exception after read of each row -/// (most likely false will be returned on first call of checkString(...)) -template -ReturnType TemplateRowInputFormat::tryReadPrefixOrSuffix(size_t & input_part_beg, size_t input_part_end) -{ - static constexpr bool throw_exception = std::is_same_v; - - skipSpaces(); - if constexpr (throw_exception) - assertString(format.delimiters[input_part_beg], *buf); - else - { - if (likely(!checkString(format.delimiters[input_part_beg], *buf))) - return ReturnType(false); - } - - while (input_part_beg < input_part_end) - { - skipSpaces(); - if constexpr (throw_exception) - skipField(format.escaping_rules[input_part_beg]); - else - { - try - { - skipField(format.escaping_rules[input_part_beg]); - } - catch (const Exception & e) - { - if (e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF && - e.code() != ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE && - e.code() != ErrorCodes::CANNOT_PARSE_QUOTED_STRING) - throw; - /// If it's parsing error, then suffix is not found - return ReturnType(false); - } - } - ++input_part_beg; - - skipSpaces(); - if constexpr (throw_exception) - assertString(format.delimiters[input_part_beg], *buf); - else - { - if (likely(!checkString(format.delimiters[input_part_beg], *buf))) - return ReturnType(false); - } - } - - if constexpr (!throw_exception) - return ReturnType(true); + format_reader->readPrefix(); } bool TemplateRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & extra) @@ -165,9 +84,7 @@ bool TemplateRowInputFormat::readRow(MutableColumns & columns, RowReadExtension if (unlikely(end_of_stream)) return false; - skipSpaces(); - - if (unlikely(checkForSuffix())) + if (unlikely(format_reader->checkForSuffix())) { end_of_stream = true; return false; @@ -176,27 +93,24 @@ bool TemplateRowInputFormat::readRow(MutableColumns & columns, RowReadExtension updateDiagnosticInfo(); if (likely(row_num != 1)) - assertString(row_between_delimiter, *buf); + format_reader->skipRowBetweenDelimiter(); extra.read_columns.assign(columns.size(), false); for (size_t i = 0; i < row_format.columnsCount(); ++i) { - skipSpaces(); - assertString(row_format.delimiters[i], *buf); - skipSpaces(); + format_reader->skipDelimiter(i); + if (row_format.format_idx_to_column_idx[i]) { size_t col_idx = *row_format.format_idx_to_column_idx[i]; extra.read_columns[col_idx] = deserializeField(data_types[col_idx], serializations[col_idx], *columns[col_idx], i); } else - skipField(row_format.escaping_rules[i]); - + format_reader->skipField(row_format.escaping_rules[i]); } - skipSpaces(); - assertString(row_format.delimiters.back(), *buf); + format_reader->skipRowEndDelimiter(); for (const auto & idx : always_default_columns) data_types[idx]->insertDefaultInto(*columns[idx]); @@ -219,65 +133,21 @@ bool TemplateRowInputFormat::deserializeField(const DataTypePtr & type, catch (Exception & e) { if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) - throwUnexpectedEof(); + throwUnexpectedEof(row_num); throw; } } -void TemplateRowInputFormat::skipField(TemplateRowInputFormat::EscapingRule escaping_rule) -{ - try - { - skipFieldByEscapingRule(*buf, escaping_rule, settings); - } - catch (Exception & e) - { - if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) - throwUnexpectedEof(); - throw; - } -} - -/// Returns true if all rows have been read i.e. there are only suffix and spaces (if ignore_spaces == true) before EOF. -/// Otherwise returns false -bool TemplateRowInputFormat::checkForSuffix() -{ - PeekableReadBufferCheckpoint checkpoint{*buf}; - bool suffix_found = false; - size_t last_successfully_parsed_idx = format_data_idx + 1; - try - { - suffix_found = tryReadPrefixOrSuffix(last_successfully_parsed_idx, format.columnsCount()); - } - catch (const Exception & e) - { - if (e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF && - e.code() != ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE && - e.code() != ErrorCodes::CANNOT_PARSE_QUOTED_STRING) - throw; - } - - if (unlikely(suffix_found)) - { - skipSpaces(); - if (buf->eof()) - return true; - } - - buf->rollbackToCheckpoint(); - return false; -} - bool TemplateRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) { out << "Suffix does not match: "; - size_t last_successfully_parsed_idx = format_data_idx + 1; + size_t last_successfully_parsed_idx = format_reader->getFormatDataIdx() + 1; const ReadBuffer::Position row_begin_pos = buf->position(); bool caught = false; try { PeekableReadBufferCheckpoint checkpoint{*buf, true}; - tryReadPrefixOrSuffix(last_successfully_parsed_idx, format.columnsCount()); + format_reader->tryReadPrefixOrSuffix(last_successfully_parsed_idx, format.columnsCount()); } catch (Exception & e) { @@ -309,7 +179,7 @@ bool TemplateRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & col if (!parseDelimiterWithDiagnosticInfo(out, *buf, row_format.delimiters[i], "delimiter before field " + std::to_string(i), ignore_spaces)) return false; - skipSpaces(); + format_reader->skipSpaces(); if (row_format.format_idx_to_column_idx[i]) { const auto & header = getPort().getHeader(); @@ -364,7 +234,7 @@ void TemplateRowInputFormat::tryDeserializeField(const DataTypePtr & type, IColu if (index) deserializeField(type, serializations[*index], column, file_column); else - skipField(row_format.escaping_rules[file_column]); + format_reader->skipField(row_format.escaping_rules[file_column]); } bool TemplateRowInputFormat::isGarbageAfterField(size_t, ReadBuffer::Position) @@ -387,13 +257,6 @@ void TemplateRowInputFormat::syncAfterError() /// It will cause another parsing error. } -void TemplateRowInputFormat::throwUnexpectedEof() -{ - throw ParsingException("Unexpected EOF while parsing row " + std::to_string(row_num) + ". " - "Maybe last row has wrong format or input doesn't contain specified suffix before EOF.", - ErrorCodes::CANNOT_READ_ALL_DATA); -} - void TemplateRowInputFormat::resetParser() { RowInputFormatWithDiagnosticInfo::resetParser(); @@ -407,6 +270,268 @@ void TemplateRowInputFormat::setReadBuffer(ReadBuffer & in_) IInputFormat::setReadBuffer(*buf); } +TemplateFormatReader::TemplateFormatReader( + PeekableReadBuffer & buf_, + bool ignore_spaces_, + const ParsedTemplateFormatString & format_, + const ParsedTemplateFormatString & row_format_, + std::string row_between_delimiter_, + const FormatSettings & format_settings_) + : buf(&buf_) + , ignore_spaces(ignore_spaces_) + , format(format_) + , row_format(row_format_) + , row_between_delimiter(row_between_delimiter_) + , format_settings(format_settings_) +{ + /// Validate format string for result set + bool has_data = false; + for (size_t i = 0; i < format.columnsCount(); ++i) + { + if (format.format_idx_to_column_idx[i]) + { + if (*format.format_idx_to_column_idx[i] != 0) + format.throwInvalidFormat("Invalid input part", i); + if (has_data) + format.throwInvalidFormat("${data} can occur only once", i); + if (format.escaping_rules[i] != EscapingRule::None) + format.throwInvalidFormat("${data} must have empty or None deserialization type", i); + has_data = true; + format_data_idx = i; + } + else + { + if (format.escaping_rules[i] == EscapingRule::XML) + format.throwInvalidFormat("XML deserialization is not supported", i); + } + } + + /// Validate format string for rows + for (size_t i = 0; i < row_format.columnsCount(); ++i) + { + if (row_format.escaping_rules[i] == EscapingRule::XML) + row_format.throwInvalidFormat("XML deserialization is not supported", i); + } +} + +void TemplateFormatReader::readPrefix() +{ + size_t last_successfully_parsed_idx = 0; + try + { + tryReadPrefixOrSuffix(last_successfully_parsed_idx, format_data_idx); + } + catch (Exception & e) + { + format.throwInvalidFormat(e.message() + " While parsing prefix", last_successfully_parsed_idx); + } +} + +void TemplateFormatReader::skipField(EscapingRule escaping_rule) +{ + try + { + skipFieldByEscapingRule(*buf, escaping_rule, format_settings); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) + throwUnexpectedEof(row_num); + throw; + } +} + +/// Asserts delimiters and skips fields in prefix or suffix. +/// tryReadPrefixOrSuffix(...) is used in checkForSuffix() to avoid throwing an exception after read of each row +/// (most likely false will be returned on first call of checkString(...)) +template +ReturnType TemplateFormatReader::tryReadPrefixOrSuffix(size_t & input_part_beg, size_t input_part_end) +{ + static constexpr bool throw_exception = std::is_same_v; + + skipSpaces(); + if constexpr (throw_exception) + assertString(format.delimiters[input_part_beg], *buf); + else + { + if (likely(!checkString(format.delimiters[input_part_beg], *buf))) + return ReturnType(false); + } + + while (input_part_beg < input_part_end) + { + skipSpaces(); + if constexpr (throw_exception) + skipField(format.escaping_rules[input_part_beg]); + else + { + try + { + skipField(format.escaping_rules[input_part_beg]); + } + catch (const Exception & e) + { + if (e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF && + e.code() != ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE && + e.code() != ErrorCodes::CANNOT_PARSE_QUOTED_STRING) + throw; + /// If it's parsing error, then suffix is not found + return ReturnType(false); + } + } + ++input_part_beg; + + skipSpaces(); + if constexpr (throw_exception) + assertString(format.delimiters[input_part_beg], *buf); + else + { + if (likely(!checkString(format.delimiters[input_part_beg], *buf))) + return ReturnType(false); + } + } + + if constexpr (!throw_exception) + return ReturnType(true); +} + +/// Returns true if all rows have been read i.e. there are only suffix and spaces (if ignore_spaces == true) before EOF. +/// Otherwise returns false +bool TemplateFormatReader::checkForSuffix() +{ + PeekableReadBufferCheckpoint checkpoint{*buf}; + bool suffix_found = false; + size_t last_successfully_parsed_idx = format_data_idx + 1; + try + { + suffix_found = tryReadPrefixOrSuffix(last_successfully_parsed_idx, format.columnsCount()); + } + catch (const Exception & e) + { + if (e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF && + e.code() != ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE && + e.code() != ErrorCodes::CANNOT_PARSE_QUOTED_STRING) + throw; + } + + if (unlikely(suffix_found)) + { + skipSpaces(); + if (buf->eof()) + return true; + } + + buf->rollbackToCheckpoint(); + return false; +} + +void TemplateFormatReader::skipDelimiter(size_t index) +{ + skipSpaces(); + assertString(row_format.delimiters[index], *buf); + skipSpaces(); +} + +void TemplateFormatReader::skipRowEndDelimiter() +{ + ++row_num; + skipSpaces(); + assertString(row_format.delimiters.back(), *buf); + skipSpaces(); +} + +void TemplateFormatReader::skipRowBetweenDelimiter() +{ + skipSpaces(); + assertString(row_between_delimiter, *buf); + skipSpaces(); +} + +TemplateSchemaReader::TemplateSchemaReader( + ReadBuffer & in_, + bool ignore_spaces_, + const ParsedTemplateFormatString & format_, + const ParsedTemplateFormatString & row_format_, + std::string row_between_delimiter, + const FormatSettings & format_settings_, + ContextPtr context_) + : IRowSchemaReader(buf, format_settings_.max_rows_to_read_for_schema_inference) + , buf(in_) + , format(format_) + , row_format(row_format_) + , format_settings(format_settings_) + , context(context_) + , format_reader(buf, ignore_spaces_, format, row_format, row_between_delimiter, format_settings) +{ + setColumnNames(row_format.column_names); +} + +DataTypes TemplateSchemaReader::readRowAndGetDataTypes() +{ + if (first_row) + format_reader.readPrefix(); + + if (format_reader.checkForSuffix()) + return {}; + + if (first_row) + first_row = false; + else + format_reader.skipRowBetweenDelimiter(); + + DataTypes data_types; + data_types.reserve(row_format.columnsCount()); + String field; + for (size_t i = 0; i != row_format.columnsCount(); ++i) + { + format_reader.skipDelimiter(i); + if (row_format.escaping_rules[i] == FormatSettings::EscapingRule::CSV) + format_settings.csv.delimiter = row_format.delimiters[i + 1].empty() ? format_settings.csv.delimiter : row_format.delimiters[i + 1].front(); + + field = readFieldByEscapingRule(buf, row_format.escaping_rules[i], format_settings); + data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, row_format.escaping_rules[i], context)); + } + + format_reader.skipRowEndDelimiter(); + return data_types; +} + +static ParsedTemplateFormatString fillResultSetFormat(const FormatSettings & settings) +{ + ParsedTemplateFormatString resultset_format; + if (settings.template_settings.resultset_format.empty()) + { + /// Default format string: "${data}" + resultset_format.delimiters.resize(2); + resultset_format.escaping_rules.emplace_back(ParsedTemplateFormatString::EscapingRule::None); + resultset_format.format_idx_to_column_idx.emplace_back(0); + resultset_format.column_names.emplace_back("data"); + } + else + { + /// Read format string from file + resultset_format = ParsedTemplateFormatString( + FormatSchemaInfo(settings.template_settings.resultset_format, "Template", false, + settings.schema.is_server, settings.schema.format_schema_path), + [&](const String & partName) -> std::optional + { + if (partName == "data") + return 0; + throw Exception("Unknown input part " + partName, + ErrorCodes::SYNTAX_ERROR); + }); + } + return resultset_format; +} + +static ParsedTemplateFormatString fillRowFormat(const FormatSettings & settings, ParsedTemplateFormatString::ColumnIdxGetter idx_getter, bool allow_indexes) +{ + return ParsedTemplateFormatString( + FormatSchemaInfo( + settings.template_settings.row_format, "Template", false, settings.schema.is_server, settings.schema.format_schema_path), + idx_getter, allow_indexes); +} + void registerInputFormatTemplate(FormatFactory & factory) { for (bool ignore_spaces : {false, true}) @@ -417,39 +542,34 @@ void registerInputFormatTemplate(FormatFactory & factory) IRowInputFormat::Params params, const FormatSettings & settings) { - ParsedTemplateFormatString resultset_format; - if (settings.template_settings.resultset_format.empty()) + auto idx_getter = [&](const String & colName) -> std::optional { - /// Default format string: "${data}" - resultset_format.delimiters.resize(2); - resultset_format.escaping_rules.emplace_back(ParsedTemplateFormatString::EscapingRule::None); - resultset_format.format_idx_to_column_idx.emplace_back(0); - resultset_format.column_names.emplace_back("data"); - } - else - { - /// Read format string from file - resultset_format = ParsedTemplateFormatString( - FormatSchemaInfo(settings.template_settings.resultset_format, "Template", false, - settings.schema.is_server, settings.schema.format_schema_path), - [&](const String & partName) -> std::optional - { - if (partName == "data") - return 0; - throw Exception("Unknown input part " + partName, - ErrorCodes::SYNTAX_ERROR); - }); - } + return sample.getPositionByName(colName); + }; - ParsedTemplateFormatString row_format = ParsedTemplateFormatString( - FormatSchemaInfo(settings.template_settings.row_format, "Template", false, - settings.schema.is_server, settings.schema.format_schema_path), - [&](const String & colName) -> std::optional - { - return sample.getPositionByName(colName); - }); + return std::make_shared( + sample, + buf, + params, + settings, + ignore_spaces, + fillResultSetFormat(settings), + fillRowFormat(settings, idx_getter, true), + settings.template_settings.row_between_delimiter); + }); + } +} - return std::make_shared(sample, buf, params, settings, ignore_spaces, resultset_format, row_format, settings.template_settings.row_between_delimiter); +void registerTemplateSchemaReader(FormatFactory & factory) +{ + for (bool ignore_spaces : {false, true}) + { + factory.registerSchemaReader(ignore_spaces ? "TemplateIgnoreSpaces" : "Template", [ignore_spaces](ReadBuffer & buf, const FormatSettings & settings, ContextPtr context) + { + size_t index = 0; + auto idx_getter = [&](const String &) -> std::optional { return index++; }; + auto row_format = fillRowFormat(settings, idx_getter, false); + return std::make_shared(buf, ignore_spaces, fillResultSetFormat(settings), row_format, settings.template_settings.row_between_delimiter, settings, context); }); } } diff --git a/src/Processors/Formats/Impl/TemplateRowInputFormat.h b/src/Processors/Formats/Impl/TemplateRowInputFormat.h index 61cd97413bf..755ad6cb39b 100644 --- a/src/Processors/Formats/Impl/TemplateRowInputFormat.h +++ b/src/Processors/Formats/Impl/TemplateRowInputFormat.h @@ -2,15 +2,19 @@ #include #include +#include #include #include #include #include +#include namespace DB { +class TemplateFormatReader; + class TemplateRowInputFormat : public RowInputFormatWithDiagnosticInfo { using EscapingRule = FormatSettings::EscapingRule; @@ -40,14 +44,6 @@ private: bool deserializeField(const DataTypePtr & type, const SerializationPtr & serialization, IColumn & column, size_t file_column); - void skipField(EscapingRule escaping_rule); - inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(*buf); } - - template - ReturnType tryReadPrefixOrSuffix(size_t & input_part_beg, size_t input_part_end); - bool checkForSuffix(); - [[noreturn]] void throwUnexpectedEof(); - bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override; void tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) override; @@ -63,12 +59,76 @@ private: const ParsedTemplateFormatString format; const ParsedTemplateFormatString row_format; - size_t format_data_idx; bool end_of_stream = false; std::vector always_default_columns; const char default_csv_delimiter; const std::string row_between_delimiter; + + std::unique_ptr format_reader; +}; + +class TemplateFormatReader +{ + using EscapingRule = FormatSettings::EscapingRule; + +public: + TemplateFormatReader( + PeekableReadBuffer & buf_, + bool ignore_spaces_, + const ParsedTemplateFormatString & format_, + const ParsedTemplateFormatString & row_format_, + std::string row_between_delimiter, + const FormatSettings & format_settings_); + + void readPrefix(); + void skipField(EscapingRule escaping_rule); + inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(*buf); } + + template + ReturnType tryReadPrefixOrSuffix(size_t & input_part_beg, size_t input_part_end); + bool checkForSuffix(); + + void setReadBuffer(PeekableReadBuffer & buf_) { buf = &buf_; } + + void skipDelimiter(size_t index); + void skipRowEndDelimiter(); + void skipRowBetweenDelimiter(); + + size_t getFormatDataIdx() const { return format_data_idx; } + +private: + PeekableReadBuffer * buf; + bool ignore_spaces; + const ParsedTemplateFormatString & format; + const ParsedTemplateFormatString & row_format; + const std::string row_between_delimiter; + const FormatSettings & format_settings; + size_t format_data_idx; + size_t row_num; +}; + +class TemplateSchemaReader : public IRowSchemaReader +{ +public: + TemplateSchemaReader(ReadBuffer & in_, + bool ignore_spaces_, + const ParsedTemplateFormatString & format_, + const ParsedTemplateFormatString & row_format_, + std::string row_between_delimiter, + const FormatSettings & format_settings_, + ContextPtr context_); + + DataTypes readRowAndGetDataTypes() override; + +private: + PeekableReadBuffer buf; + const ParsedTemplateFormatString format; + const ParsedTemplateFormatString row_format; + FormatSettings format_settings; + ContextPtr context; + TemplateFormatReader format_reader; + bool first_row = true; }; bool parseDelimiterWithDiagnosticInfo(WriteBuffer & out, ReadBuffer & buf, const String & delimiter, const String & description, bool skip_spaces); diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index adf6d2e8a25..b58be3f5526 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -15,6 +16,7 @@ #include #include +#include namespace DB { @@ -286,6 +288,50 @@ namespace } } +/// Can be used in fileSegmentationEngine for parallel parsing of Values +static bool skipToNextRow(PeekableReadBuffer * buf, size_t min_chunk_bytes, int balance) +{ + skipWhitespaceIfAny(*buf); + if (buf->eof() || *buf->position() == ';') + return false; + bool quoted = false; + + size_t chunk_begin_buf_count = buf->count(); + while (!buf->eof() && (balance || buf->count() - chunk_begin_buf_count < min_chunk_bytes)) + { + buf->position() = find_first_symbols<'\\', '\'', ')', '('>(buf->position(), buf->buffer().end()); + if (buf->position() == buf->buffer().end()) + continue; + if (*buf->position() == '\\') + { + ++buf->position(); + if (!buf->eof()) + ++buf->position(); + } + else if (*buf->position() == '\'') + { + quoted ^= true; + ++buf->position(); + } + else if (*buf->position() == ')') + { + ++buf->position(); + if (!quoted) + --balance; + } + else if (*buf->position() == '(') + { + ++buf->position(); + if (!quoted) + ++balance; + } + } + + if (!buf->eof() && *buf->position() == ',') + ++buf->position(); + return true; +} + bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx) { const Block & header = getPort().getHeader(); @@ -293,7 +339,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx auto settings = context->getSettingsRef(); /// We need continuous memory containing the expression to use Lexer - skipToNextRow(0, 1); + skipToNextRow(buf.get(), 0, 1); buf->makeContinuousMemoryFromCheckpointToPos(); buf->rollbackToCheckpoint(); @@ -437,50 +483,6 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx return true; } -/// Can be used in fileSegmentationEngine for parallel parsing of Values -bool ValuesBlockInputFormat::skipToNextRow(size_t min_chunk_bytes, int balance) -{ - skipWhitespaceIfAny(*buf); - if (buf->eof() || *buf->position() == ';') - return false; - bool quoted = false; - - size_t chunk_begin_buf_count = buf->count(); - while (!buf->eof() && (balance || buf->count() - chunk_begin_buf_count < min_chunk_bytes)) - { - buf->position() = find_first_symbols<'\\', '\'', ')', '('>(buf->position(), buf->buffer().end()); - if (buf->position() == buf->buffer().end()) - continue; - if (*buf->position() == '\\') - { - ++buf->position(); - if (!buf->eof()) - ++buf->position(); - } - else if (*buf->position() == '\'') - { - quoted ^= true; - ++buf->position(); - } - else if (*buf->position() == ')') - { - ++buf->position(); - if (!quoted) - --balance; - } - else if (*buf->position() == '(') - { - ++buf->position(); - if (!quoted) - ++balance; - } - } - - if (!buf->eof() && *buf->position() == ',') - ++buf->position(); - return true; -} - void ValuesBlockInputFormat::assertDelimiterAfterValue(size_t column_idx) { if (unlikely(!checkDelimiterAfterValue(column_idx))) @@ -559,6 +561,63 @@ void ValuesBlockInputFormat::setReadBuffer(ReadBuffer & in_) IInputFormat::setReadBuffer(*buf); } +ValuesSchemaReader::ValuesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_, ContextPtr context_) + : IRowSchemaReader(buf, format_settings_.max_rows_to_read_for_schema_inference), buf(in_), context(context_) +{ +} + +DataTypes ValuesSchemaReader::readRowAndGetDataTypes() +{ + if (first_row) + { + skipBOMIfExists(buf); + first_row = false; + } + + skipWhitespaceIfAny(buf); + if (buf.eof()) + return {}; + + assertChar('(', buf); + PeekableReadBufferCheckpoint checkpoint(buf); + skipToNextRow(&buf, 0, 1); + buf.makeContinuousMemoryFromCheckpointToPos(); + buf.rollbackToCheckpoint(); + + Tokens tokens(buf.position(), buf.buffer().end()); + IParser::Pos token_iterator(tokens, context->getSettingsRef().max_parser_depth); + + DataTypes data_types; + bool finish = false; + while (!finish) + { + Expected expected; + ASTPtr ast; + + bool parsed = parser.parse(token_iterator, ast, expected); + /// Consider delimiter after value (',' or ')') as part of expression + parsed &= token_iterator->type == TokenType::Comma || token_iterator->type == TokenType::ClosingRoundBracket; + + if (!parsed) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Cannot parse expression here: {}, token: {}", + String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position())), String(token_iterator.get().begin, token_iterator.get().end)); + + std::pair result = evaluateConstantExpression(ast, context); + data_types.push_back(generalizeDataType(result.second)); + + if (token_iterator->type == TokenType::ClosingRoundBracket) + finish = true; + ++token_iterator; + buf.position() = const_cast(token_iterator->begin); + } + + skipWhitespaceIfAny(buf); + if (!buf.eof() && *buf.position() == ',') + ++buf.position(); + + return data_types; +} + void registerInputFormatValues(FormatFactory & factory) { factory.registerInputFormat("Values", []( @@ -571,4 +630,12 @@ void registerInputFormatValues(FormatFactory & factory) }); } +void registerValuesSchemaReader(FormatFactory & factory) +{ + factory.registerSchemaReader("Values", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr context) + { + return std::make_shared(buf, settings, context); + }); +} + } diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index 5bbd4bea5ba..e1521955472 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -7,6 +7,7 @@ #include #include #include +#include #include namespace DB @@ -68,8 +69,6 @@ private: void readPrefix(); void readSuffix(); - bool skipToNextRow(size_t min_chunk_bytes = 0, int balance = 0); - std::unique_ptr buf; const RowInputFormatParams params; @@ -95,4 +94,18 @@ private: BlockMissingValues block_missing_values; }; +class ValuesSchemaReader : public IRowSchemaReader +{ +public: + ValuesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, ContextPtr context_); + +private: + DataTypes readRowAndGetDataTypes() override; + + PeekableReadBuffer buf; + ContextPtr context; + ParserExpression parser; + bool first_row = true; +}; + } diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index 87fa5ec1c4a..7720b01dc74 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -1,5 +1,7 @@ #include +#include #include +#include #include #include @@ -9,6 +11,7 @@ namespace DB namespace ErrorCodes { extern const int INCORRECT_DATA; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes( @@ -17,8 +20,13 @@ RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes( const Params & params_, bool with_names_, bool with_types_, - const FormatSettings & format_settings_) - : RowInputFormatWithDiagnosticInfo(header_, in_, params_), format_settings(format_settings_), with_names(with_names_), with_types(with_types_) + const FormatSettings & format_settings_, + std::unique_ptr format_reader_) + : RowInputFormatWithDiagnosticInfo(header_, in_, params_) + , format_settings(format_settings_) + , with_names(with_names_) + , with_types(with_types_) + , format_reader(std::move(format_reader_)) { const auto & sample = getPort().getHeader(); size_t num_columns = sample.columns(); @@ -88,7 +96,7 @@ void RowInputFormatWithNamesAndTypes::readPrefix() } /// Skip prefix before names and types. - skipPrefixBeforeHeader(); + format_reader->skipPrefixBeforeHeader(); /// This is a bit of abstraction leakage, but we need it in parallel parsing: /// we check if this InputFormat is working with the "real" beginning of the data. @@ -97,7 +105,7 @@ void RowInputFormatWithNamesAndTypes::readPrefix() if (format_settings.with_names_use_header) { std::vector read_columns(data_types.size(), false); - auto column_names = readNames(); + auto column_names = format_reader->readNames(); for (const auto & name : column_names) addInputColumn(name, read_columns); @@ -110,7 +118,7 @@ void RowInputFormatWithNamesAndTypes::readPrefix() else { setupAllColumnsByTableSchema(); - skipNames(); + format_reader->skipNames(); } } else if (!column_mapping->is_set) @@ -119,10 +127,10 @@ void RowInputFormatWithNamesAndTypes::readPrefix() if (with_types) { /// Skip delimiter between names and types. - skipRowBetweenDelimiter(); + format_reader->skipRowBetweenDelimiter(); if (format_settings.with_types_use_header) { - auto types = readTypes(); + auto types = format_reader->readTypes(); if (types.size() != column_mapping->column_indexes_for_input_fields.size()) throw Exception( ErrorCodes::INCORRECT_DATA, @@ -143,7 +151,7 @@ void RowInputFormatWithNamesAndTypes::readPrefix() } } else - skipTypes(); + format_reader->skipTypes(); } } @@ -161,7 +169,7 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE if (unlikely(end_of_stream)) return false; - if (unlikely(checkForSuffix())) + if (unlikely(format_reader->checkForSuffix())) { end_of_stream = true; return false; @@ -170,9 +178,9 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE updateDiagnosticInfo(); if (likely(row_num != 1 || (getCurrentUnitNumber() == 0 && (with_names || with_types)))) - skipRowBetweenDelimiter(); + format_reader->skipRowBetweenDelimiter(); - skipRowStartDelimiter(); + format_reader->skipRowStartDelimiter(); ext.read_columns.resize(data_types.size()); for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column) @@ -180,20 +188,20 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE const auto & column_index = column_mapping->column_indexes_for_input_fields[file_column]; const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size(); if (column_index) - ext.read_columns[*column_index] = readField( + ext.read_columns[*column_index] = format_reader->readField( *columns[*column_index], data_types[*column_index], serializations[*column_index], is_last_file_column, column_mapping->names_of_columns[file_column]); else - skipField(file_column); + format_reader->skipField(file_column); if (!is_last_file_column) - skipFieldDelimiter(); + format_reader->skipFieldDelimiter(); } - skipRowEndDelimiter(); + format_reader->skipRowEndDelimiter(); insertDefaultsForNotSeenColumns(columns, ext); @@ -218,13 +226,13 @@ void RowInputFormatWithNamesAndTypes::tryDeserializeField(const DataTypePtr & ty const auto & index = column_mapping->column_indexes_for_input_fields[file_column]; if (index) { - checkNullValueForNonNullable(type); + format_reader->checkNullValueForNonNullable(type); const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size(); - readField(column, type, serializations[*index], is_last_file_column, column_mapping->names_of_columns[file_column]); + format_reader->readField(column, type, serializations[*index], is_last_file_column, column_mapping->names_of_columns[file_column]); } else { - skipField(file_column); + format_reader->skipField(file_column); } } @@ -236,13 +244,13 @@ bool RowInputFormatWithNamesAndTypes::parseRowAndPrintDiagnosticInfo(MutableColu return false; } - if (!tryParseSuffixWithDiagnosticInfo(out)) + if (!format_reader->tryParseSuffixWithDiagnosticInfo(out)) return false; - if (likely(row_num != 1) && !parseRowBetweenDelimiterWithDiagnosticInfo(out)) + if (likely(row_num != 1) && !format_reader->parseRowBetweenDelimiterWithDiagnosticInfo(out)) return false; - if (!parseRowStartWithDiagnosticInfo(out)) + if (!format_reader->parseRowStartWithDiagnosticInfo(out)) return false; for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column) @@ -266,22 +274,68 @@ bool RowInputFormatWithNamesAndTypes::parseRowAndPrintDiagnosticInfo(MutableColu /// Delimiters if (file_column + 1 != column_mapping->column_indexes_for_input_fields.size()) { - if (!parseFieldDelimiterWithDiagnosticInfo(out)) + if (!format_reader->parseFieldDelimiterWithDiagnosticInfo(out)) return false; } } - return parseRowEndWithDiagnosticInfo(out); + return format_reader->parseRowEndWithDiagnosticInfo(out); } - -void registerFileSegmentationEngineForFormatWithNamesAndTypes( - FormatFactory & factory, const String & base_format_name, FormatFactory::FileSegmentationEngine segmentation_engine) +bool RowInputFormatWithNamesAndTypes::isGarbageAfterField(size_t index, ReadBuffer::Position pos) { - factory.registerFileSegmentationEngine(base_format_name, segmentation_engine); - factory.registerFileSegmentationEngine(base_format_name + "WithNames", segmentation_engine); - factory.registerFileSegmentationEngine(base_format_name + "WithNamesAndTypes", segmentation_engine); + return format_reader->isGarbageAfterField(index, pos); } +void RowInputFormatWithNamesAndTypes::setReadBuffer(ReadBuffer & in_) +{ + format_reader->setReadBuffer(in_); + IInputFormat::setReadBuffer(in_); +} + +FormatWithNamesAndTypesSchemaReader::FormatWithNamesAndTypesSchemaReader( + ReadBuffer & in_, + size_t max_rows_to_read_, + bool with_names_, + bool with_types_, + FormatWithNamesAndTypesReader * format_reader_, + DataTypePtr default_type_) + : IRowSchemaReader(in_, max_rows_to_read_, default_type_), with_names(with_names_), with_types(with_types_), format_reader(format_reader_) +{ +} + +NamesAndTypesList FormatWithNamesAndTypesSchemaReader::readSchema() +{ + if (with_names || with_types) + skipBOMIfExists(in); + + format_reader->skipPrefixBeforeHeader(); + + Names names; + if (with_names) + names = format_reader->readNames(); + + if (with_types) + { + format_reader->skipRowBetweenDelimiter(); + std::vector data_type_names = format_reader->readTypes(); + if (data_type_names.size() != names.size()) + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "The number of column names {} differs with the number of types {}", names.size(), data_type_names.size()); + + NamesAndTypesList result; + for (size_t i = 0; i != data_type_names.size(); ++i) + result.emplace_back(names[i], DataTypeFactory::instance().get(data_type_names[i])); + return result; + } + + if (!names.empty()) + setColumnNames(names); + + /// We should determine types by reading rows with data. Use the implementation from IRowSchemaReader. + return IRowSchemaReader::readSchema(); +} } + diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index cd7cd34d7e6..25ffc8d6de2 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -1,12 +1,15 @@ #pragma once #include +#include #include #include namespace DB { +class FormatWithNamesAndTypesReader; + /// Base class for input formats with -WithNames and -WithNamesAndTypes suffixes. /// It accepts 2 parameters in constructor - with_names and with_types and implements /// input format depending on them: @@ -20,7 +23,7 @@ namespace DB /// then reads/skips types. So you can this invariant. class RowInputFormatWithNamesAndTypes : public RowInputFormatWithDiagnosticInfo { -public: +protected: /** with_names - in the first line the header with column names * with_types - in the second line the header with column names */ @@ -28,44 +31,14 @@ public: const Block & header_, ReadBuffer & in_, const Params & params_, - bool with_names_, bool with_types_, const FormatSettings & format_settings_); + bool with_names_, + bool with_types_, + const FormatSettings & format_settings_, + std::unique_ptr format_reader_); void resetParser() override; - -protected: - /// Read single field from input. Return false if there was no real value and we inserted default value. - virtual bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) = 0; - - /// Skip single field, it's used to skip unknown columns. - virtual void skipField(size_t file_column) = 0; - /// Skip the whole row with names. - virtual void skipNames() = 0; - /// Skip the whole row with types. - virtual void skipTypes() = 0; - - /// Skip delimiters, if any. - virtual void skipPrefixBeforeHeader() {} - virtual void skipRowStartDelimiter() {} - virtual void skipFieldDelimiter() {} - virtual void skipRowEndDelimiter() {} - virtual void skipRowBetweenDelimiter() {} - - /// Check suffix. - virtual bool checkForSuffix() { return in->eof(); } - - /// Methods for parsing with diagnostic info. - virtual void checkNullValueForNonNullable(DataTypePtr) {} - virtual bool parseRowStartWithDiagnosticInfo(WriteBuffer &) { return true; } - virtual bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer &) { return true; } - virtual bool parseRowEndWithDiagnosticInfo(WriteBuffer &) { return true;} - virtual bool parseRowBetweenDelimiterWithDiagnosticInfo(WriteBuffer &) { return true;} - virtual bool tryParseSuffixWithDiagnosticInfo(WriteBuffer &) { return true; } - bool isGarbageAfterField(size_t, ReadBuffer::Position) override {return false; } - - /// Read row with names and return the list of them. - virtual std::vector readNames() = 0; - /// Read row with types and return the list of them. - virtual std::vector readTypes() = 0; + bool isGarbageAfterField(size_t index, ReadBuffer::Position pos) override; + void setReadBuffer(ReadBuffer & in_) override; const FormatSettings format_settings; DataTypes data_types; @@ -84,10 +57,90 @@ private: bool with_names; bool with_types; + std::unique_ptr format_reader; std::unordered_map column_indexes_by_names; }; -void registerFileSegmentationEngineForFormatWithNamesAndTypes( - FormatFactory & factory, const String & base_format_name, FormatFactory::FileSegmentationEngine segmentation_engine); +/// Base class for parsing data in input formats with -WithNames and -WithNamesAndTypes suffixes. +/// Used for reading/skipping names/types/delimiters in specific format. +class FormatWithNamesAndTypesReader +{ +public: + explicit FormatWithNamesAndTypesReader(ReadBuffer & in_, const FormatSettings & format_settings_) : in(&in_), format_settings(format_settings_) {} + + /// Read single field from input. Return false if there was no real value and we inserted default value. + virtual bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) = 0; + + /// Methods for parsing with diagnostic info. + virtual void checkNullValueForNonNullable(DataTypePtr) {} + virtual bool parseRowStartWithDiagnosticInfo(WriteBuffer &) { return true; } + virtual bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer &) { return true; } + virtual bool parseRowEndWithDiagnosticInfo(WriteBuffer &) { return true;} + virtual bool parseRowBetweenDelimiterWithDiagnosticInfo(WriteBuffer &) { return true;} + virtual bool tryParseSuffixWithDiagnosticInfo(WriteBuffer &) { return true; } + virtual bool isGarbageAfterField(size_t, ReadBuffer::Position) { return false; } + + /// Read row with names and return the list of them. + virtual std::vector readNames() = 0; + /// Read row with types and return the list of them. + virtual std::vector readTypes() = 0; + + /// Skip single field, it's used to skip unknown columns. + virtual void skipField(size_t file_column) = 0; + /// Skip the whole row with names. + virtual void skipNames() = 0; + /// Skip the whole row with types. + virtual void skipTypes() = 0; + + /// Skip delimiters, if any. + virtual void skipPrefixBeforeHeader() {} + virtual void skipRowStartDelimiter() {} + virtual void skipFieldDelimiter() {} + virtual void skipRowEndDelimiter() {} + virtual void skipRowBetweenDelimiter() {} + + /// Check suffix. + virtual bool checkForSuffix() { return in->eof(); } + + const FormatSettings & getFormatSettings() const { return format_settings; } + + virtual void setReadBuffer(ReadBuffer & in_) { in = &in_; } + + virtual ~FormatWithNamesAndTypesReader() = default; + +protected: + ReadBuffer * in; + const FormatSettings format_settings; +}; + +/// Base class for schema inference for formats with -WithNames and -WithNamesAndTypes suffixes. +/// For formats with -WithNamesAndTypes suffix the schema will be determined by first two rows. +/// For formats with -WithNames suffix the names of columns will be determined by the first row +/// and types of columns by the rows with data. +/// For formats without suffixes default column names will be used +/// and types will be determined by the rows with data. +class FormatWithNamesAndTypesSchemaReader : public IRowSchemaReader +{ +public: + FormatWithNamesAndTypesSchemaReader( + ReadBuffer & in, + size_t max_rows_to_read_, + bool with_names_, + bool with_types_, + FormatWithNamesAndTypesReader * format_reader_, + DataTypePtr default_type_ = nullptr); + + NamesAndTypesList readSchema() override; + +protected: + virtual DataTypes readRowAndGetDataTypes() override = 0; + + bool with_names; + bool with_types; + +private: + FormatWithNamesAndTypesReader * format_reader; +}; } + diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index 502f8b800e3..f33fd938092 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -88,7 +88,7 @@ struct URLBasedDataSourceConfiguration String url; String format; String compression_method = "auto"; - String structure; + String structure = "auto"; std::vector> headers; String http_method; diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 2105228abf6..3d4b38afa1c 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -14,7 +14,6 @@ #include #include -#include #include #include @@ -29,6 +28,8 @@ #include #include + +#include #include #include @@ -51,6 +52,69 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ACCESS_DENIED; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; +} +namespace +{ + /* Recursive directory listing with matched paths as a result. + * Have the same method in StorageFile. + */ + Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match) + { + const size_t first_glob = for_match.find_first_of("*?{"); + + const size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); + const String suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' + const String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' + + const size_t next_slash = suffix_with_globs.find('/', 1); + re2::RE2 matcher(makeRegexpPatternFromGlobs(suffix_with_globs.substr(0, next_slash))); + + HDFSFileInfo ls; + ls.file_info = hdfsListDirectory(fs.get(), prefix_without_globs.data(), &ls.length); + Strings result; + for (int i = 0; i < ls.length; ++i) + { + const String full_path = String(ls.file_info[i].mName); + const size_t last_slash = full_path.rfind('/'); + const String file_name = full_path.substr(last_slash); + const bool looking_for_directory = next_slash != std::string::npos; + const bool is_directory = ls.file_info[i].mKind == 'D'; + /// Condition with type of current file_info means what kind of path is it in current iteration of ls + if (!is_directory && !looking_for_directory) + { + if (re2::RE2::FullMatch(file_name, matcher)) + { + result.push_back(String(ls.file_info[i].mName)); + } + } + else if (is_directory && looking_for_directory) + { + if (re2::RE2::FullMatch(file_name, matcher)) + { + Strings result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, suffix_with_globs.substr(next_slash)); + /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. + std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); + } + } + } + + return result; + } + + std::pair getPathFromUriAndUriWithoutPath(const String & uri) + { + const size_t begin_of_path = uri.find('/', uri.find("//") + 2); + return {uri.substr(begin_of_path), uri.substr(0, begin_of_path)}; + } + + std::vector getPathsList(const String & path_from_uri, const String & uri_without_path, ContextPtr context) + { + HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context->getGlobalContext()->getConfigRef()); + HDFSFSPtr fs = createHDFSFS(builder.get()); + + return LSWithRegexpMatching("/", fs, path_from_uri); + } } static Strings listFilesWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match); @@ -79,12 +143,45 @@ StorageHDFS::StorageHDFS( checkHDFSURL(uri); StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); + + if (columns_.empty()) + { + auto columns = getTableStructureFromData(format_name, uri, compression_method, context_); + storage_metadata.setColumns(columns); + } + else + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); } +ColumnsDescription StorageHDFS::getTableStructureFromData( + const String & format, + const String & uri, + const String & compression_method, + ContextPtr ctx) +{ + auto read_buffer_creator = [&]() + { + const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri); + auto paths = getPathsList(path_from_uri, uri, ctx); + if (paths.empty()) + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file, because there are no files in HDFS with provided path. You must " + "specify table structure manually", + format); + + auto compression = chooseCompressionMethod(paths[0], compression_method); + return wrapReadBufferWithCompressionMethod( + std::make_unique(uri_without_path, paths[0], ctx->getGlobalContext()->getConfigRef()), compression); + }; + + return readSchemaFromFormat(format, std::nullopt, read_buffer_creator, ctx); +} + class HDFSSource::DisclosedGlobIterator::Impl { public: @@ -339,51 +436,6 @@ private: }; -/* Recursive directory listing with matched paths as a result. - * Have the same method in StorageFile. - */ -Strings listFilesWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match) -{ - const size_t first_glob = for_match.find_first_of("*?{"); - - const size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); - const String suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' - const String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' - - const size_t next_slash = suffix_with_globs.find('/', 1); - re2::RE2 matcher(makeRegexpPatternFromGlobs(suffix_with_globs.substr(0, next_slash))); - - HDFSFileInfo ls; - ls.file_info = hdfsListDirectory(fs.get(), prefix_without_globs.data(), &ls.length); - Strings result; - for (int i = 0; i < ls.length; ++i) - { - const String full_path = String(ls.file_info[i].mName); - const size_t last_slash = full_path.rfind('/'); - const String file_name = full_path.substr(last_slash); - const bool looking_for_directory = next_slash != std::string::npos; - const bool is_directory = ls.file_info[i].mKind == 'D'; - /// Condition with type of current file_info means what kind of path is it in current iteration of ls - if (!is_directory && !looking_for_directory) - { - if (re2::RE2::FullMatch(file_name, matcher)) - { - result.push_back(String(ls.file_info[i].mName)); - } - } - else if (is_directory && looking_for_directory) - { - if (re2::RE2::FullMatch(file_name, matcher)) - { - Strings result_part = listFilesWithRegexpMatching(fs::path(full_path) / "", fs, suffix_with_globs.substr(next_slash)); - /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. - std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); - } - } - } - return result; -} - bool StorageHDFS::isColumnOriented() const { return format_name != "Distributed" && FormatFactory::instance().checkIfFormatIsColumnOriented(format_name); @@ -400,6 +452,7 @@ Pipe StorageHDFS::read( { bool need_path_column = false; bool need_file_column = false; + for (const auto & column : column_names) { if (column == "_path") @@ -528,6 +581,7 @@ void registerStorageHDFS(StorageFactory & factory) }, { .supports_sort_order = true, // for partition by + .supports_schema_inference = true, .source_access_type = AccessType::HDFS, }); } diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 3e2f7a43127..9e845d8fd74 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -31,7 +31,7 @@ public: size_t max_block_size, unsigned num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override; void truncate( const ASTPtr & query, @@ -49,6 +49,12 @@ public: /// format to read only them. Note: this hack cannot be done with ordinary formats like TSV. bool isColumnOriented() const; + static ColumnsDescription getTableStructureFromData( + const String & format, + const String & uri, + const String & compression_method, + ContextPtr ctx); + protected: friend class HDFSSource; StorageHDFS( diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1b7be8ca98d..6597c28360d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -224,7 +224,6 @@ MergeTreeData::MergeTreeData( { try { - checkPartitionKeyAndInitMinMax(metadata_.partition_key); setProperties(metadata_, metadata_, attach); if (minmax_idx_date_column_pos == -1) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index cb52c8b86c0..fc3eff7459b 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -35,6 +35,7 @@ namespace ErrorCodes extern const int NO_ELEMENTS_IN_CONFIG; extern const int UNKNOWN_STORAGE; extern const int NO_REPLICA_NAME_GIVEN; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } @@ -258,6 +259,34 @@ If you use the Replicated version of engines, see https://clickhouse.com/docs/en return help; } +static ColumnsDescription getColumnsDescriptionFromZookeeper(const String & raw_zookeeper_path, ContextMutablePtr context) +{ + String zookeeper_name = zkutil::extractZooKeeperName(raw_zookeeper_path); + String zookeeper_path = zkutil::extractZooKeeperPath(raw_zookeeper_path, true); + + if (!context->hasZooKeeper() && !context->hasAuxiliaryZooKeeper(zookeeper_name)) + throw Exception{ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot get replica structure without zookeeper, you must specify the structure manually"}; + + zkutil::ZooKeeperPtr zookeeper; + try + { + if (zookeeper_name == StorageReplicatedMergeTree::getDefaultZooKeeperName()) + zookeeper = context->getZooKeeper(); + else + zookeeper = context->getAuxiliaryZooKeeper(zookeeper_name); + } + catch (...) + { + throw Exception{ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot get replica structure from zookeeper, because cannot get zookeeper: {}. You must specify structure manually", getCurrentExceptionMessage(false)}; + } + + if (!zookeeper->exists(zookeeper_path + "/replicas")) + throw Exception{ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot get replica structure, because there no other replicas in zookeeper. You must specify the structure manually"}; + + Coordination::Stat columns_stat; + return ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_path) / "columns", &columns_stat)); +} + static StoragePtr create(const StorageFactory::Arguments & args) { @@ -638,7 +667,14 @@ static StoragePtr create(const StorageFactory::Arguments & args) String date_column_name; StorageInMemoryMetadata metadata; - metadata.setColumns(args.columns); + + ColumnsDescription columns; + if (args.columns.empty() && replicated) + columns = getColumnsDescriptionFromZookeeper(zookeeper_path, args.getContext()); + else + columns = args.columns; + + metadata.setColumns(columns); metadata.setComment(args.comment); std::unique_ptr storage_settings; @@ -705,12 +741,12 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.query.columns_list && args.query.columns_list->indices) for (auto & index : args.query.columns_list->indices->children) - metadata.secondary_indices.push_back(IndexDescription::getIndexFromAST(index, args.columns, args.getContext())); + metadata.secondary_indices.push_back(IndexDescription::getIndexFromAST(index, columns, args.getContext())); if (args.query.columns_list && args.query.columns_list->projections) for (auto & projection_ast : args.query.columns_list->projections->children) { - auto projection = ProjectionDescription::getProjectionFromAST(projection_ast, args.columns, args.getContext()); + auto projection = ProjectionDescription::getProjectionFromAST(projection_ast, columns, args.getContext()); metadata.projections.add(std::move(projection)); } @@ -720,10 +756,10 @@ static StoragePtr create(const StorageFactory::Arguments & args) constraints.push_back(constraint); metadata.constraints = ConstraintsDescription(constraints); - auto column_ttl_asts = args.columns.getColumnTTLs(); + auto column_ttl_asts = columns.getColumnTTLs(); for (const auto & [name, ast] : column_ttl_asts) { - auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, args.columns, args.getContext(), metadata.primary_key); + auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, columns, args.getContext(), metadata.primary_key); metadata.column_ttls_by_name[name] = new_ttl_entry; } @@ -850,6 +886,7 @@ void registerStorageMergeTree(StorageFactory & factory) features.supports_replication = true; features.supports_deduplication = true; + features.supports_schema_inference = true; factory.registerStorage("ReplicatedMergeTree", create, features); factory.registerStorage("ReplicatedCollapsingMergeTree", create, features); diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 87a8ea2315d..0cc401aa93c 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -126,7 +126,13 @@ StorageBuffer::StorageBuffer( , bg_pool(getContext()->getBufferFlushSchedulePool()) { StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); + if (columns_.empty()) + { + auto dest_table = DatabaseCatalog::instance().getTable(destination_id, context_); + storage_metadata.setColumns(dest_table->getInMemoryMetadataPtr()->getColumns()); + } + else + storage_metadata.setColumns(columns_); storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); @@ -1167,6 +1173,7 @@ void registerStorageBuffer(StorageFactory & factory) }, { .supports_parallel_insert = true, + .supports_schema_inference = true, }); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index ddf363e3957..19869b77106 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include @@ -24,7 +25,6 @@ #include #include -#include #include #include #include @@ -42,7 +42,6 @@ #include #include #include -#include #include #include #include @@ -63,7 +62,6 @@ #include #include -#include #include #include @@ -71,8 +69,6 @@ #include #include -#include - #include #include #include @@ -329,7 +325,16 @@ StorageDistributed::StorageDistributed( , rng(randomSeed()) { StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); + if (columns_.empty()) + { + StorageID id = StorageID::createEmpty(); + id.table_name = remote_table; + id.database_name = remote_database; + storage_metadata.setColumns(getStructureOfRemoteTable(*getCluster(), id, getContext(), remote_table_function_ptr)); + } + else + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); @@ -1398,6 +1403,7 @@ void registerStorageDistributed(StorageFactory & factory) { .supports_settings = true, .supports_parallel_insert = true, + .supports_schema_inference = true, .source_access_type = AccessType::REMOTE, }); } diff --git a/src/Storages/StorageFactory.h b/src/Storages/StorageFactory.h index 20db1a44897..6ffa6327176 100644 --- a/src/Storages/StorageFactory.h +++ b/src/Storages/StorageFactory.h @@ -66,6 +66,7 @@ public: bool supports_deduplication = false; /// See also IStorage::supportsParallelInsert() bool supports_parallel_insert = false; + bool supports_schema_inference = false; AccessType source_access_type = AccessType::NONE; }; @@ -98,6 +99,7 @@ public: .supports_replication = false, .supports_deduplication = false, .supports_parallel_insert = false, + .supports_schema_inference = false, .source_access_type = AccessType::NONE, }); @@ -126,6 +128,12 @@ public: AccessType getSourceAccessType(const String & table_engine) const; + bool checkIfStorageSupportsSchemaInterface(const String & storage_name) + { + if (storages.contains(storage_name)) + return storages[storage_name].features.supports_schema_inference; + return false; + } private: Storages storages; }; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 13a70af2ada..a479f982c70 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -15,8 +15,9 @@ #include #include -#include #include +#include +#include #include #include @@ -38,6 +39,7 @@ #include #include #include +#include #include #include #include @@ -63,6 +65,7 @@ namespace ErrorCodes extern const int INCOMPATIBLE_COLUMNS; extern const int CANNOT_STAT; extern const int LOGICAL_ERROR; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } namespace @@ -135,6 +138,56 @@ void checkCreationIsAllowed(ContextPtr context_global, const std::string & db_di throw Exception("File must not be a directory", ErrorCodes::INCORRECT_FILE_NAME); } +std::unique_ptr createReadBuffer( + const String & current_path, + bool use_table_fd, + const String & storage_name, + int table_fd, + const String & compression_method, + ContextPtr context) +{ + std::unique_ptr nested_buffer; + CompressionMethod method; + + struct stat file_stat{}; + + if (use_table_fd) + { + /// Check if file descriptor allows random reads (and reading it twice). + if (0 != fstat(table_fd, &file_stat)) + throwFromErrno("Cannot stat table file descriptor, inside " + storage_name, ErrorCodes::CANNOT_STAT); + + if (S_ISREG(file_stat.st_mode)) + nested_buffer = std::make_unique(table_fd); + else + nested_buffer = std::make_unique(table_fd); + + method = chooseCompressionMethod("", compression_method); + } + else + { + /// Check if file descriptor allows random reads (and reading it twice). + if (0 != stat(current_path.c_str(), &file_stat)) + throwFromErrno("Cannot stat file " + current_path, ErrorCodes::CANNOT_STAT); + + if (S_ISREG(file_stat.st_mode)) + nested_buffer = std::make_unique(current_path, context->getSettingsRef().max_read_buffer_size); + else + nested_buffer = std::make_unique(current_path, context->getSettingsRef().max_read_buffer_size); + + method = chooseCompressionMethod(current_path, compression_method); + } + + /// For clickhouse-local add progress callback to display progress bar. + if (context->getApplicationType() == Context::ApplicationType::LOCAL) + { + auto & in = static_cast(*nested_buffer); + in.setProgressCallback(context); + } + + return wrapReadBufferWithCompressionMethod(std::move(nested_buffer), method); +} + } Strings StorageFile::getPathsList(const String & table_path, const String & user_files_path, ContextPtr context, size_t & total_bytes_to_read) @@ -164,6 +217,42 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user return paths; } + +ColumnsDescription StorageFile::getTableStructureFromData( + const String & format, + const std::vector & paths, + const String & compression_method, + const std::optional & format_settings, + ContextPtr context) +{ + if (format == "Distributed") + { + if (paths.empty()) + throw Exception( + "Cannot get table structure from file, because no files match specified name", ErrorCodes::INCORRECT_FILE_NAME); + + auto source = StorageDistributedDirectoryMonitor::createSourceFromFile(paths[0]); + return ColumnsDescription(source->getOutputs().front().getHeader().getNamesAndTypesList()); + } + + auto read_buffer_creator = [&]() + { + String path; + auto it = std::find_if(paths.begin(), paths.end(), [](const String & p){ return std::filesystem::exists(p); }); + if (it == paths.end()) + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file, because there are no files with provided path. You must specify " + "table structure manually", + format); + + path = *it; + return createReadBuffer(path, false, "File", -1, compression_method, context); + }; + + return readSchemaFromFormat(format, format_settings, read_buffer_creator, context); +} + bool StorageFile::isColumnOriented() const { return format_name != "Distributed" && FormatFactory::instance().checkIfFormatIsColumnOriented(format_name); @@ -182,10 +271,13 @@ StorageFile::StorageFile(int table_fd_, CommonArguments args) throw Exception("Using file descriptor as source of storage isn't allowed for server daemons", ErrorCodes::DATABASE_ACCESS_DENIED); if (args.format_name == "Distributed") throw Exception("Distributed format is allowed only with explicit file path", ErrorCodes::INCORRECT_FILE_NAME); + if (args.columns.empty()) + throw Exception("Automatic schema inference is not allowed when using file descriptor as source of storage", ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE); is_db_table = false; use_table_fd = true; table_fd = table_fd_; + setStorageMetadata(args); } StorageFile::StorageFile(const std::string & table_path_, const std::string & user_files_path, CommonArguments args) @@ -194,22 +286,7 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us is_db_table = false; paths = getPathsList(table_path_, user_files_path, args.getContext(), total_bytes_to_read); path_for_partitioned_write = table_path_; - - if (args.format_name == "Distributed") - { - if (paths.empty()) - throw Exception("Cannot get table structure from file, because no files match specified name", ErrorCodes::INCORRECT_FILE_NAME); - - auto & first_path = paths[0]; - Block header = StorageDistributedDirectoryMonitor::createSourceFromFile(first_path)->getOutputs().front().getHeader(); - - StorageInMemoryMetadata storage_metadata; - auto columns = ColumnsDescription(header.getNamesAndTypesList()); - if (!args.columns.empty() && columns != args.columns) - throw Exception("Table structure and file structure are different", ErrorCodes::INCOMPATIBLE_COLUMNS); - storage_metadata.setColumns(columns); - setInMemoryMetadata(storage_metadata); - } + setStorageMetadata(args); } StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArguments args) @@ -225,6 +302,8 @@ StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArgu paths = {getTablePath(table_dir_path, format_name)}; if (fs::exists(paths[0])) total_bytes_to_read = fs::file_size(paths[0]); + + setStorageMetadata(args); } StorageFile::StorageFile(CommonArguments args) @@ -233,9 +312,21 @@ StorageFile::StorageFile(CommonArguments args) , format_settings(args.format_settings) , compression_method(args.compression_method) , base_path(args.getContext()->getPath()) +{ +} + +void StorageFile::setStorageMetadata(CommonArguments args) { StorageInMemoryMetadata storage_metadata; - if (args.format_name != "Distributed") + + if (args.format_name == "Distributed" || args.columns.empty()) + { + auto columns = getTableStructureFromData(format_name, paths, compression_method, format_settings, args.getContext()); + if (!args.columns.empty() && args.columns != columns) + throw Exception("Table structure and file structure are different", ErrorCodes::INCOMPATIBLE_COLUMNS); + storage_metadata.setColumns(columns); + } + else storage_metadata.setColumns(args.columns); storage_metadata.setConstraints(args.constraints); @@ -350,46 +441,7 @@ public: } } - std::unique_ptr nested_buffer; - CompressionMethod method; - - struct stat file_stat{}; - - if (storage->use_table_fd) - { - /// Check if file descriptor allows random reads (and reading it twice). - if (0 != fstat(storage->table_fd, &file_stat)) - throwFromErrno("Cannot stat table file descriptor, inside " + storage->getName(), ErrorCodes::CANNOT_STAT); - - if (S_ISREG(file_stat.st_mode)) - nested_buffer = std::make_unique(storage->table_fd); - else - nested_buffer = std::make_unique(storage->table_fd); - - method = chooseCompressionMethod("", storage->compression_method); - } - else - { - /// Check if file descriptor allows random reads (and reading it twice). - if (0 != stat(current_path.c_str(), &file_stat)) - throwFromErrno("Cannot stat file " + current_path, ErrorCodes::CANNOT_STAT); - - if (S_ISREG(file_stat.st_mode)) - nested_buffer = std::make_unique(current_path, context->getSettingsRef().max_read_buffer_size); - else - nested_buffer = std::make_unique(current_path, context->getSettingsRef().max_read_buffer_size); - - method = chooseCompressionMethod(current_path, storage->compression_method); - } - - /// For clickhouse-local add progress callback to display progress bar. - if (context->getApplicationType() == Context::ApplicationType::LOCAL) - { - auto & in = static_cast(*nested_buffer); - in.setProgressCallback(context); - } - - read_buf = wrapReadBufferWithCompressionMethod(std::move(nested_buffer), method); + read_buf = createReadBuffer(current_path, storage->use_table_fd, storage->getName(), storage->table_fd, storage->compression_method, context); auto get_block_for_format = [&]() -> Block { @@ -853,7 +905,8 @@ void registerStorageFile(StorageFactory & factory) { StorageFactory::StorageFeatures storage_features{ .supports_settings = true, - .source_access_type = AccessType::FILE + .supports_schema_inference = true, + .source_access_type = AccessType::FILE, }; factory.registerStorage( diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index f48d1c285da..6b015976589 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -1,6 +1,7 @@ #pragma once #include + #include #include @@ -70,6 +71,13 @@ public: bool supportsPartitionBy() const override { return true; } + static ColumnsDescription getTableStructureFromData( + const String & format, + const std::vector & paths, + const String & compression_method, + const std::optional & format_settings, + ContextPtr context); + protected: friend class StorageFileSource; friend class StorageFileSink; @@ -86,6 +94,8 @@ protected: private: explicit StorageFile(CommonArguments args); + void setStorageMetadata(CommonArguments args); + std::string format_name; // We use format settings from global context + CREATE query for File table // function -- in this case, format_settings is set. diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index f82f9d21217..bdb7ddb744a 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -43,6 +43,7 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int SAMPLING_NOT_SUPPORTED; extern const int ALTER_OF_COLUMN_IS_FORBIDDEN; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } StorageMerge::StorageMerge( @@ -61,7 +62,7 @@ StorageMerge::StorageMerge( , database_is_regexp(database_is_regexp_) { StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); + storage_metadata.setColumns(columns_.empty() ? getColumnsDescriptionFromSourceTables() : columns_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); } @@ -82,11 +83,19 @@ StorageMerge::StorageMerge( , database_is_regexp(database_is_regexp_) { StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); + storage_metadata.setColumns(columns_.empty() ? getColumnsDescriptionFromSourceTables() : columns_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); } +ColumnsDescription StorageMerge::getColumnsDescriptionFromSourceTables() const +{ + auto table = getFirstTable([](auto && t) { return t; }); + if (!table) + throw Exception{ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "There are no tables satisfied provided regexp, you must specify table structure manually"}; + return table->getInMemoryMetadataPtr()->getColumns(); +} + template StoragePtr StorageMerge::getFirstTable(F && predicate) const { @@ -762,7 +771,6 @@ void StorageMerge::convertingSourceStream( IStorage::ColumnSizeByName StorageMerge::getColumnSizes() const { - auto first_materialized_mysql = getFirstTable([](const StoragePtr & table) { return table && table->getName() == "MaterializedMySQL"; }); if (!first_materialized_mysql) return {}; @@ -816,6 +824,9 @@ void registerStorageMerge(StorageFactory & factory) return StorageMerge::create( args.table_id, args.columns, args.comment, source_database_name_or_regexp, is_regexp, table_name_regexp, args.getContext()); + }, + { + .supports_schema_inference = true }); } diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 56adeab9279..ad3075efd08 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -132,6 +132,8 @@ protected: static SelectQueryInfo getModifiedQueryInfo( const SelectQueryInfo & query_info, ContextPtr modified_context, const StorageID & current_storage_id, bool is_merge_engine); + + ColumnsDescription getColumnsDescriptionFromSourceTables() const; }; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a1f82e14868..ce5576bd809 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3,7 +3,6 @@ #include "Common/hex.h" #include #include -#include #include #include #include @@ -20,7 +19,6 @@ #include #include #include -#include #include #include #include @@ -35,7 +33,6 @@ #include -#include #include #include @@ -45,7 +42,6 @@ #include #include #include -#include #include #include @@ -68,7 +64,6 @@ #include -#include #include #include @@ -194,56 +189,6 @@ zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeper() const return res; } -static std::string normalizeZooKeeperPath(std::string zookeeper_path, bool check_starts_with_slash, Poco::Logger * log = nullptr) -{ - if (!zookeeper_path.empty() && zookeeper_path.back() == '/') - zookeeper_path.resize(zookeeper_path.size() - 1); - /// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it. - if (!zookeeper_path.empty() && zookeeper_path.front() != '/') - { - /// Do not allow this for new tables, print warning for tables created in old versions - if (check_starts_with_slash) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path must starts with '/', got '{}'", zookeeper_path); - if (log) - LOG_WARNING(log, "ZooKeeper path ('{}') does not start with '/'. It will not be supported in future releases"); - zookeeper_path = "/" + zookeeper_path; - } - - return zookeeper_path; -} - -static String extractZooKeeperName(const String & path) -{ - static constexpr auto default_zookeeper_name = "default"; - if (path.empty()) - throw Exception("ZooKeeper path should not be empty", ErrorCodes::BAD_ARGUMENTS); - if (path[0] == '/') - return default_zookeeper_name; - auto pos = path.find(":/"); - if (pos != String::npos && pos < path.find('/')) - { - auto zookeeper_name = path.substr(0, pos); - if (zookeeper_name.empty()) - throw Exception("Zookeeper path should start with '/' or ':/'", ErrorCodes::BAD_ARGUMENTS); - return zookeeper_name; - } - return default_zookeeper_name; -} - -static String extractZooKeeperPath(const String & path, bool check_starts_with_slash, Poco::Logger * log = nullptr) -{ - if (path.empty()) - throw Exception("ZooKeeper path should not be empty", ErrorCodes::BAD_ARGUMENTS); - if (path[0] == '/') - return normalizeZooKeeperPath(path, check_starts_with_slash, log); - auto pos = path.find(":/"); - if (pos != String::npos && pos < path.find('/')) - { - return normalizeZooKeeperPath(path.substr(pos + 1, String::npos), check_starts_with_slash, log); - } - return normalizeZooKeeperPath(path, check_starts_with_slash, log); -} - static MergeTreePartInfo makeDummyDropRangeForMovePartitionOrAttachPartitionFrom(const String & partition_id) { /// NOTE We don't have special log entry type for MOVE PARTITION/ATTACH PARTITION FROM, @@ -287,8 +232,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( true, /// require_part_metadata attach, [this] (const std::string & name) { enqueuePartForCheck(name); }) - , zookeeper_name(extractZooKeeperName(zookeeper_path_)) - , zookeeper_path(extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ !attach, log)) + , zookeeper_name(zkutil::extractZooKeeperName(zookeeper_path_)) + , zookeeper_path(zkutil::extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ !attach, log)) , replica_name(replica_name_) , replica_path(fs::path(zookeeper_path) / "replicas" / replica_name_) , reader(*this) @@ -5561,8 +5506,8 @@ void StorageReplicatedMergeTree::fetchPartition( info.table_id = getStorageID(); info.table_id.uuid = UUIDHelpers::Nil; auto expand_from = query_context->getMacros()->expand(from_, info); - String auxiliary_zookeeper_name = extractZooKeeperName(expand_from); - String from = extractZooKeeperPath(expand_from, /* check_starts_with_slash */ true); + String auxiliary_zookeeper_name = zkutil::extractZooKeeperName(expand_from); + String from = zkutil::extractZooKeeperPath(expand_from, /* check_starts_with_slash */ true); if (from.empty()) throw Exception("ZooKeeper path should not be empty", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -6638,7 +6583,7 @@ void StorageReplicatedMergeTree::movePartitionToShard( if (!move_part) throw Exception("MOVE PARTITION TO SHARD is not supported, use MOVE PART instead", ErrorCodes::NOT_IMPLEMENTED); - if (normalizeZooKeeperPath(zookeeper_path, /* check_starts_with_slash */ true) == normalizeZooKeeperPath(to, /* check_starts_with_slash */ true)) + if (zkutil::normalizeZooKeeperPath(zookeeper_path, /* check_starts_with_slash */ true) == zkutil::normalizeZooKeeperPath(to, /* check_starts_with_slash */ true)) throw Exception("Source and destination are the same", ErrorCodes::BAD_ARGUMENTS); auto zookeeper = getZooKeeper(); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 6861d89f070..b2721210344 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -263,6 +263,8 @@ public: bool createEmptyPartInsteadOfLost(zkutil::ZooKeeperPtr zookeeper, const String & lost_part_name); + static const String getDefaultZooKeeperName() { return default_zookeeper_name; } + private: std::atomic_bool are_restoring_replica {false}; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 3a03ac3906c..48609dda029 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -28,6 +28,7 @@ #include #include +#include #include #include @@ -70,6 +71,7 @@ namespace ErrorCodes extern const int S3_ERROR; extern const int UNEXPECTED_EXPRESSION; extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } class IOutputFormat; @@ -480,13 +482,39 @@ StorageS3::StorageS3( { context_->getGlobalContext()->getRemoteHostFilter().checkURL(uri_.uri); StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); + + updateClientAndAuthSettings(context_, client_auth); + if (columns_.empty()) + { + auto columns = getTableStructureFromDataImpl(format_name, client_auth, max_single_read_retries_, compression_method, distributed_processing_, format_settings, context_); + storage_metadata.setColumns(columns); + } + else + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - updateClientAndAuthSettings(context_, client_auth); } +std::shared_ptr StorageS3::createFileIterator(const ClientAuthentication & client_auth, bool distributed_processing, ContextPtr local_context) +{ + std::shared_ptr iterator_wrapper{nullptr}; + if (distributed_processing) + { + return std::make_shared( + [callback = local_context->getReadTaskCallback()]() -> String { + return callback(); + }); + } + + /// Iterate through disclosed globs and make a source for each file + auto glob_iterator = std::make_shared(*client_auth.client, client_auth.uri); + return std::make_shared([glob_iterator]() + { + return glob_iterator->next(); + }); +} Pipe StorageS3::read( const Names & column_names, @@ -510,23 +538,7 @@ Pipe StorageS3::read( need_file_column = true; } - std::shared_ptr iterator_wrapper{nullptr}; - if (distributed_processing) - { - iterator_wrapper = std::make_shared( - [callback = local_context->getReadTaskCallback()]() -> String { - return callback(); - }); - } - else - { - /// Iterate through disclosed globs and make a source for each file - auto glob_iterator = std::make_shared(*client_auth.client, client_auth.uri); - iterator_wrapper = std::make_shared([glob_iterator]() - { - return glob_iterator->next(); - }); - } + std::shared_ptr iterator_wrapper = createFileIterator(client_auth, distributed_processing, local_context); for (size_t i = 0; i < num_streams; ++i) { @@ -707,6 +719,51 @@ StorageS3Configuration StorageS3::getConfiguration(ASTs & engine_args, ContextPt return configuration; } +ColumnsDescription StorageS3::getTableStructureFromData( + const String & format, + const S3::URI & uri, + const String & access_key_id, + const String & secret_access_key, + UInt64 max_connections, + UInt64 max_single_read_retries, + const String & compression_method, + bool distributed_processing, + const std::optional & format_settings, + ContextPtr ctx) +{ + ClientAuthentication client_auth{uri, access_key_id, secret_access_key, max_connections, {}, {}}; + updateClientAndAuthSettings(ctx, client_auth); + return getTableStructureFromDataImpl(format, client_auth, max_single_read_retries, compression_method, distributed_processing, format_settings, ctx); +} + +ColumnsDescription StorageS3::getTableStructureFromDataImpl( + const String & format, + const ClientAuthentication & client_auth, + UInt64 max_single_read_retries, + const String & compression_method, + bool distributed_processing, + const std::optional & format_settings, + ContextPtr ctx) +{ + auto read_buffer_creator = [&]() + { + auto file_iterator = createFileIterator(client_auth, distributed_processing, ctx); + String current_key = (*file_iterator)(); + if (current_key.empty()) + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file, because there are no files with provided path in S3. You must specify " + "table structure manually", + format); + + return wrapReadBufferWithCompressionMethod( + std::make_unique(client_auth.client, client_auth.uri.bucket, current_key, max_single_read_retries, ctx->getReadSettings(), DBMS_DEFAULT_BUFFER_SIZE), + chooseCompressionMethod(current_key, compression_method)); + }; + + return readSchemaFromFormat(format, format_settings, read_buffer_creator, ctx); +} + void registerStorageS3Impl(const String & name, StorageFactory & factory) { diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 248238379dc..0690040915d 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -147,8 +147,19 @@ public: static StorageS3Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context); -private: + static ColumnsDescription getTableStructureFromData( + const String & format, + const S3::URI & uri, + const String & access_key_id, + const String & secret_access_key, + UInt64 max_connections, + UInt64 max_single_read_retries, + const String & compression_method, + bool distributed_processing, + const std::optional & format_settings, + ContextPtr ctx); +private: friend class StorageS3Cluster; friend class TableFunctionS3Cluster; @@ -175,6 +186,17 @@ private: ASTPtr partition_by; static void updateClientAndAuthSettings(ContextPtr, ClientAuthentication &); + + static std::shared_ptr createFileIterator(const ClientAuthentication & client_auth, bool distributed_processing, ContextPtr local_context); + + static ColumnsDescription getTableStructureFromDataImpl( + const String & format, + const ClientAuthentication & client_auth, + UInt64 max_single_read_retries, + const String & compression_method, + bool distributed_processing, + const std::optional & format_settings, + ContextPtr ctx); }; } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 0eec77ac8e7..471b460d349 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -13,8 +13,9 @@ #include #include -#include +#include #include +#include #include #include @@ -40,7 +41,7 @@ namespace ErrorCodes IStorageURLBase::IStorageURLBase( const String & uri_, - ContextPtr /*context_*/, + ContextPtr context_, const StorageID & table_id_, const String & format_name_, const std::optional & format_settings_, @@ -61,12 +62,48 @@ IStorageURLBase::IStorageURLBase( , partition_by(partition_by_) { StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); + if (columns_.empty()) + { + auto columns = getTableStructureFromData(format_name, uri, compression_method, headers, format_settings, context_); + storage_metadata.setColumns(columns); + } + else + storage_metadata.setColumns(columns_); storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); } +ColumnsDescription IStorageURLBase::getTableStructureFromData( + const String & format, + const String & uri, + const String & compression_method, + const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers, + const std::optional & format_settings, + ContextPtr context) +{ + auto read_buffer_creator = [&]() + { + auto parsed_uri = Poco::URI(uri); + return wrapReadBufferWithCompressionMethod( + std::make_unique( + parsed_uri, + Poco::Net::HTTPRequest::HTTP_GET, + nullptr, + ConnectionTimeouts::getHTTPTimeouts(context), + Poco::Net::HTTPBasicCredentials{}, + context->getSettingsRef().max_http_get_redirects, + DBMS_DEFAULT_BUFFER_SIZE, + context->getReadSettings(), + headers, + ReadWriteBufferFromHTTP::Range{}, + context->getRemoteHostFilter()), + chooseCompressionMethod(parsed_uri.getPath(), compression_method)); + }; + + return readSchemaFromFormat(format, format_settings, read_buffer_creator, context); +} + namespace { ReadWriteBufferFromHTTP::HTTPHeaderEntries getHeaders( @@ -642,6 +679,7 @@ void registerStorageURL(StorageFactory & factory) }, { .supports_settings = true, + .supports_schema_inference = true, .source_access_type = AccessType::URL, }); } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index cf72352a183..790f01135d3 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -41,6 +41,14 @@ public: bool supportsPartitionBy() const override { return true; } + static ColumnsDescription getTableStructureFromData( + const String & format, + const String & uri, + const String & compression_method, + const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers, + const std::optional & format_settings, + ContextPtr context); + protected: IStorageURLBase( const String & uri_, diff --git a/src/TableFunctions/ITableFunction.cpp b/src/TableFunctions/ITableFunction.cpp index fa7f6e52220..42b24abdbbe 100644 --- a/src/TableFunctions/ITableFunction.cpp +++ b/src/TableFunctions/ITableFunction.cpp @@ -15,25 +15,23 @@ namespace DB { StoragePtr ITableFunction::execute(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, - ColumnsDescription cached_columns) const + ColumnsDescription cached_columns, bool use_global_context) const { ProfileEvents::increment(ProfileEvents::TableFunctionExecute); context->checkAccess(AccessType::CREATE_TEMPORARY_TABLE | StorageFactory::instance().getSourceAccessType(getStorageTypeName())); + auto context_to_use = use_global_context ? context->getGlobalContext() : context; + if (cached_columns.empty()) return executeImpl(ast_function, context, table_name, std::move(cached_columns)); - /// We have table structure, so it's CREATE AS table_function(). - /// We should use global context here because there will be no query context on server startup - /// and because storage lifetime is bigger than query context lifetime. - auto global_context = context->getGlobalContext(); if (hasStaticStructure() && cached_columns == getActualTableStructure(context)) - return executeImpl(ast_function, global_context, table_name, std::move(cached_columns)); + return executeImpl(ast_function, context_to_use, table_name, std::move(cached_columns)); auto this_table_function = shared_from_this(); auto get_storage = [=]() -> StoragePtr { - return this_table_function->executeImpl(ast_function, global_context, table_name, cached_columns); + return this_table_function->executeImpl(ast_function, context_to_use, table_name, cached_columns); }; /// It will request actual table structure and create underlying storage lazily diff --git a/src/TableFunctions/ITableFunction.h b/src/TableFunctions/ITableFunction.h index 56147ffd598..93cf5057e88 100644 --- a/src/TableFunctions/ITableFunction.h +++ b/src/TableFunctions/ITableFunction.h @@ -54,7 +54,7 @@ public: /// Create storage according to the query. StoragePtr - execute(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns_ = {}) const; + execute(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns_ = {}, bool use_global_context = false) const; virtual ~ITableFunction() = default; diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index 699ad698bd8..4395c318983 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -1,4 +1,3 @@ -#include #include #include @@ -6,16 +5,16 @@ #include #include -#include #include #include -#include #include #include +#include + namespace DB { @@ -23,10 +22,27 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int INCORRECT_FILE_NAME; extern const int BAD_ARGUMENTS; } +namespace +{ + void checkIfFormatSupportsAutoStructure(const String & name, const String & format) + { + if (name == "file" && format == "Distributed") + return; + + if (FormatFactory::instance().checkIfFormatHasAnySchemaReader(format)) + return; + + throw Exception( + "Table function '" + name + + "' allows automatic structure determination only for formats that support schema inference and for Distributed format in table function " + "'file'", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } +} + void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, ContextPtr context) { /// Parse args @@ -46,21 +62,23 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context filename = args[0]->as().value.safeGet(); format = args[1]->as().value.safeGet(); - if (args.size() == 2 && getName() == "file") + if (args.size() == 2) { - if (format == "Distributed") - return; - throw Exception("Table function '" + getName() + "' allows 2 arguments only for Distributed format.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + checkIfFormatSupportsAutoStructure(getName(), format); + return; } if (args.size() != 3 && args.size() != 4) - throw Exception("Table function '" + getName() + "' requires 3 or 4 arguments: filename, format, structure and compression method (default auto).", + throw Exception("Table function '" + getName() + "' requires 2, 3 or 4 arguments: filename, format, structure (default auto) and compression method (default auto)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); structure = args[2]->as().value.safeGet(); + if (structure == "auto") + checkIfFormatSupportsAutoStructure(getName(), format); + if (structure.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Table structure is empty for table function '{}'", + "Table structure is empty for table function '{}'. If you want to use automatic schema inference, use 'auto'", ast_function->formatForErrorMessage()); if (args.size() == 4) @@ -69,25 +87,12 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { - auto columns = getActualTableStructure(context); + ColumnsDescription columns; + if (structure != "auto") + columns = parseColumnsListFromString(structure, context); StoragePtr storage = getStorage(filename, format, columns, context, table_name, compression_method); storage->startup(); return storage; } -ColumnsDescription ITableFunctionFileLike::getActualTableStructure(ContextPtr context) const -{ - if (structure.empty()) - { - assert(getName() == "file" && format == "Distributed"); - size_t total_bytes_to_read = 0; - Strings paths = StorageFile::getPathsList(filename, context->getUserFilesPath(), context, total_bytes_to_read); - if (paths.empty()) - throw Exception("Cannot get table structure from file, because no files match specified name", ErrorCodes::INCORRECT_FILE_NAME); - auto source = StorageDistributedDirectoryMonitor::createSourceFromFile(paths[0]); - return ColumnsDescription{source->getOutputs().front().getHeader().getNamesAndTypesList()}; - } - return parseColumnsListFromString(structure, context); -} - } diff --git a/src/TableFunctions/ITableFunctionFileLike.h b/src/TableFunctions/ITableFunctionFileLike.h index 2069f02b0dd..2ceafdee229 100644 --- a/src/TableFunctions/ITableFunctionFileLike.h +++ b/src/TableFunctions/ITableFunctionFileLike.h @@ -8,7 +8,7 @@ class ColumnsDescription; class Context; /* - * function(source, format, structure) - creates a temporary storage from formatted source + * function(source, format, structure[, compression_method]) - creates a temporary storage from formatted source */ class ITableFunctionFileLike : public ITableFunction { @@ -18,7 +18,7 @@ protected: String filename; String format; - String structure; + String structure = "auto"; String compression_method = "auto"; private: @@ -28,8 +28,7 @@ private: const String & source, const String & format, const ColumnsDescription & columns, ContextPtr global_context, const std::string & table_name, const String & compression_method) const = 0; - ColumnsDescription getActualTableStructure(ContextPtr context) const override; - - bool hasStaticStructure() const override { return true; } + bool hasStaticStructure() const override { return structure != "auto"; } }; + } diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index d8bdb3b45c4..71aba5494e8 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -1,4 +1,5 @@ #include +#include #include "registerTableFunctions.h" #include @@ -9,11 +10,13 @@ namespace DB { + StoragePtr TableFunctionFile::getStorage(const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, const std::string & table_name, const std::string & compression_method_) const { + LOG_DEBUG(&Poco::Logger::get("TableFunctionFile"), "getStorage"); // For `file` table function, we are going to use format settings from the // query context. StorageFile::CommonArguments args{ @@ -30,8 +33,21 @@ StoragePtr TableFunctionFile::getStorage(const String & source, return StorageFile::create(source, global_context->getUserFilesPath(), args); } +ColumnsDescription TableFunctionFile::getActualTableStructure(ContextPtr context) const +{ + if (structure == "auto") + { + size_t total_bytes_to_read = 0; + Strings paths = StorageFile::getPathsList(filename, context->getUserFilesPath(), context, total_bytes_to_read); + return StorageFile::getTableStructureFromData(format, paths, compression_method, std::nullopt, context); + } + + return parseColumnsListFromString(structure, context); +} + void registerTableFunctionFile(TableFunctionFactory & factory) { factory.registerFunction(); } + } diff --git a/src/TableFunctions/TableFunctionFile.h b/src/TableFunctions/TableFunctionFile.h index 460656a7218..f26e4a9c06d 100644 --- a/src/TableFunctions/TableFunctionFile.h +++ b/src/TableFunctions/TableFunctionFile.h @@ -6,7 +6,7 @@ namespace DB { -/* file(path, format, structure) - creates a temporary storage from file +/* file(path, format[, structure, compression]) - creates a temporary storage from file * * The file must be in the clickhouse data directory. * The relative path begins with the clickhouse data directory. @@ -20,9 +20,13 @@ public: return name; } + ColumnsDescription getActualTableStructure(ContextPtr context) const override; + private: StoragePtr getStorage( const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, const std::string & table_name, const std::string & compression_method_) const override; const char * getStorageTypeName() const override { return "File"; } -};} +}; + +} diff --git a/src/TableFunctions/TableFunctionHDFS.cpp b/src/TableFunctions/TableFunctionHDFS.cpp index 245674b0e06..b626f563977 100644 --- a/src/TableFunctions/TableFunctionHDFS.cpp +++ b/src/TableFunctions/TableFunctionHDFS.cpp @@ -6,9 +6,11 @@ #include #include #include +#include namespace DB { + StoragePtr TableFunctionHDFS::getStorage( const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, const std::string & table_name, const String & compression_method_) const @@ -24,12 +26,18 @@ StoragePtr TableFunctionHDFS::getStorage( compression_method_); } +ColumnsDescription TableFunctionHDFS::getActualTableStructure(ContextPtr context) const +{ + if (structure == "auto") + return StorageHDFS::getTableStructureFromData(format, filename, compression_method, context); + + return parseColumnsListFromString(structure, context); +} -#if USE_HDFS void registerTableFunctionHDFS(TableFunctionFactory & factory) { factory.registerFunction(); } -#endif + } #endif diff --git a/src/TableFunctions/TableFunctionHDFS.h b/src/TableFunctions/TableFunctionHDFS.h index 70bdc67efc8..74139818209 100644 --- a/src/TableFunctions/TableFunctionHDFS.h +++ b/src/TableFunctions/TableFunctionHDFS.h @@ -12,7 +12,7 @@ namespace DB class Context; -/* hdfs(URI, format, structure) - creates a temporary storage from hdfs files +/* hdfs(URI, format[, structure, compression]) - creates a temporary storage from hdfs files * */ class TableFunctionHDFS : public ITableFunctionFileLike @@ -24,6 +24,8 @@ public: return name; } + ColumnsDescription getActualTableStructure(ContextPtr context) const override; + private: StoragePtr getStorage( const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, diff --git a/src/TableFunctions/TableFunctionRemote.h b/src/TableFunctions/TableFunctionRemote.h index 845c36182dc..976397ddc45 100644 --- a/src/TableFunctions/TableFunctionRemote.h +++ b/src/TableFunctions/TableFunctionRemote.h @@ -27,6 +27,7 @@ public: bool needStructureConversion() const override { return false; } private: + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override; const char * getStorageTypeName() const override { return "Distributed"; } diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index e26c282c622..c4be01c6b5c 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include "registerTableFunctions.h" @@ -28,6 +29,7 @@ void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr con const auto message = fmt::format( "The signature of table function {} could be the following:\n" \ + " - url, format\n" \ " - url, format, structure\n" \ " - url, format, structure, compression_method\n" \ " - url, access_key_id, secret_access_key, format, structure\n" \ @@ -69,17 +71,32 @@ void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr con /// Size -> argument indexes static auto size_to_args = std::map> { + {2, {{"format", 1}}}, {3, {{"format", 1}, {"structure", 2}}}, - {4, {{"format", 1}, {"structure", 2}, {"compression_method", 3}}}, {5, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}}}, {6, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}, {"compression_method", 5}}} }; + std::map args_to_idx; + /// For 4 arguments we support 2 possible variants: + /// s3(source, format, structure, compression_method) and s3(source, access_key_id, access_key_id, format) + /// We can distinguish them by looking at the 4-th argument: check if it's a format name or not. + if (args.size() == 4) + { + auto last_arg = args[3]->as().value.safeGet(); + if (FormatFactory::instance().getAllFormats().contains(last_arg)) + args_to_idx = {{"access_key_id", 1}, {"access_key_id", 2}, {"format", 3}}; + else + args_to_idx = {{"format", 1}, {"structure", 2}, {"compression_method", 3}}; + } + else + { + args_to_idx = size_to_args[args.size()]; + } + /// This argument is always the first configuration.url = args[0]->as().value.safeGet(); - auto & args_to_idx = size_to_args[args.size()]; - if (args_to_idx.contains("format")) configuration.format = args[args_to_idx["format"]]->as().value.safeGet(); @@ -101,6 +118,21 @@ void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr con ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context) const { + if (s3_configuration->structure == "auto") + { + return StorageS3::getTableStructureFromData( + s3_configuration->format, + S3::URI(Poco::URI(s3_configuration->url)), + s3_configuration->access_key_id, + s3_configuration->secret_access_key, + context->getSettingsRef().s3_max_connections, + context->getSettingsRef().s3_max_single_read_retries, + s3_configuration->compression_method, + false, + std::nullopt, + context); + } + return parseColumnsListFromString(s3_configuration->structure, context); } @@ -113,6 +145,10 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context UInt64 max_single_part_upload_size = context->getSettingsRef().s3_max_single_part_upload_size; UInt64 max_connections = context->getSettingsRef().s3_max_connections; + ColumnsDescription columns; + if (s3_configuration->structure != "auto") + columns = parseColumnsListFromString(s3_configuration->structure, context); + StoragePtr storage = StorageS3::create( s3_uri, s3_configuration->access_key_id, diff --git a/src/TableFunctions/TableFunctionS3.h b/src/TableFunctions/TableFunctionS3.h index 8d4c1391236..374e653072e 100644 --- a/src/TableFunctions/TableFunctionS3.h +++ b/src/TableFunctions/TableFunctionS3.h @@ -13,7 +13,7 @@ namespace DB class Context; -/* s3(source, [access_key_id, secret_access_key,] format, structure) - creates a temporary storage for a file in S3 +/* s3(source, [access_key_id, secret_access_key,] format, structure[, compression]) - creates a temporary storage for a file in S3 */ class TableFunctionS3 : public ITableFunction { @@ -23,7 +23,7 @@ public: { return name; } - bool hasStaticStructure() const override { return true; } + bool hasStaticStructure() const override { return s3_configuration->structure != "auto"; } protected: StoragePtr executeImpl( diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index c3ea30f800f..7c4d7b4a444 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -2,11 +2,11 @@ #include "registerTableFunctions.h" #include -#include #include #include #include #include +#include #include @@ -59,20 +59,10 @@ void TableFunctionURL::parseArguments(const ASTPtr & ast_function, ContextPtr co } } - StoragePtr TableFunctionURL::getStorage( const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, const std::string & table_name, const String & compression_method_) const { - ReadWriteBufferFromHTTP::HTTPHeaderEntries headers; - for (const auto & [header, value] : configuration.headers) - { - auto value_literal = value.safeGet(); - if (header == "Range") - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Range headers are not allowed"); - headers.emplace_back(std::make_pair(header, value_literal)); - } - return StorageURL::create( source, StorageID(getDatabaseName(), table_name), @@ -83,10 +73,31 @@ StoragePtr TableFunctionURL::getStorage( String{}, global_context, compression_method_, - headers, + getHeaders(), configuration.http_method); } +ReadWriteBufferFromHTTP::HTTPHeaderEntries TableFunctionURL::getHeaders() const +{ + ReadWriteBufferFromHTTP::HTTPHeaderEntries headers; + for (const auto & [header, value] : configuration.headers) + { + auto value_literal = value.safeGet(); + if (header == "Range") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Range headers are not allowed"); + headers.emplace_back(std::make_pair(header, value_literal)); + } + return headers; +} + +ColumnsDescription TableFunctionURL::getActualTableStructure(ContextPtr context) const +{ + if (structure == "auto") + return StorageURL::getTableStructureFromData(format, filename, compression_method, getHeaders(), std::nullopt, context); + + return parseColumnsListFromString(structure, context); +} + void registerTableFunctionURL(TableFunctionFactory & factory) { factory.registerFunction(); diff --git a/src/TableFunctions/TableFunctionURL.h b/src/TableFunctions/TableFunctionURL.h index 9425112acb2..798a37dc478 100644 --- a/src/TableFunctions/TableFunctionURL.h +++ b/src/TableFunctions/TableFunctionURL.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -9,7 +10,7 @@ namespace DB class Context; -/* url(source, format, structure) - creates a temporary storage from url +/* url(source, format[, structure, compression]) - creates a temporary storage from url */ class TableFunctionURL : public ITableFunctionFileLike { @@ -20,6 +21,8 @@ public: return name; } + ColumnsDescription getActualTableStructure(ContextPtr context) const override; + protected: void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; @@ -29,6 +32,8 @@ private: const std::string & table_name, const String & compression_method_) const override; const char * getStorageTypeName() const override { return "URL"; } + ReadWriteBufferFromHTTP::HTTPHeaderEntries getHeaders() const; + URLBasedDataSourceConfiguration configuration; }; From dd994aa7610e7bf280a618507b937ede098bb49e Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 16 Dec 2021 21:48:38 +0300 Subject: [PATCH 0553/1260] Add some tests and some fixes --- src/Formats/CapnProtoUtils.cpp | 5 +- src/Formats/EscapingRuleUtils.cpp | 6 +- src/Formats/JSONEachRowUtils.cpp | 20 + src/Formats/ProtobufSerializer.cpp | 9 +- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 23 +- .../Impl/CustomSeparatedRowInputFormat.cpp | 2 +- .../Impl/JSONCompactEachRowRowInputFormat.cpp | 1 + .../Formats/Impl/ORCBlockOutputFormat.cpp | 2 + .../Formats/Impl/RegexpRowInputFormat.cpp | 9 +- .../Formats/Impl/RegexpRowInputFormat.h | 3 +- .../00825_protobuf_format_enum_mapping.sh | 6 +- ...protobuf_format_enum_mapping.yaodzJ.binary | Bin 0 -> 11 bytes .../queries/0_stateless/01903_http_fields.sh | 3 + .../02149_external_schema_inference.reference | 168 +++++++ .../02149_external_schema_inference.sh | 91 ++++ .../02149_schema_inference.reference | 160 +++++++ .../0_stateless/02149_schema_inference.sh | 242 ++++++++++ ...ma_inference_create_table_syntax.reference | 50 ++ ...49_schema_inference_create_table_syntax.sh | 45 ++ ...ma_inference_formats_with_schema.reference | 448 ++++++++++++++++++ ...49_schema_inference_formats_with_schema.sh | 65 +++ .../00825_protobuf_format_enum_mapping.proto | 4 +- 22 files changed, 1340 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/00825_protobuf_format_enum_mapping.yaodzJ.binary create mode 100644 tests/queries/0_stateless/02149_external_schema_inference.reference create mode 100755 tests/queries/0_stateless/02149_external_schema_inference.sh create mode 100644 tests/queries/0_stateless/02149_schema_inference.reference create mode 100755 tests/queries/0_stateless/02149_schema_inference.sh create mode 100644 tests/queries/0_stateless/02149_schema_inference_create_table_syntax.reference create mode 100755 tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh create mode 100644 tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference create mode 100755 tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index a76428a45d2..bed46a97c1b 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -489,8 +489,6 @@ static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type) case capnp::schema::Type::STRUCT: { auto struct_schema = capnp_type.asStruct(); - if (checkIfStructContainsUnnamedUnion(struct_schema)) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Unnamed union is not supported"); /// Check if it can be Nullable. if (checkIfStructIsNamedUnion(struct_schema)) @@ -506,6 +504,9 @@ static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type) return std::make_shared(nested_type); } + if (checkIfStructContainsUnnamedUnion(struct_schema)) + throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Unnamed union is not supported"); + /// Treat Struct as Tuple. DataTypes nested_types; Names nested_names; diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 4c7f5aa35f9..3049168351f 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -290,8 +290,10 @@ DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSe case FormatSettings::EscapingRule::JSON: { Poco::JSON::Parser parser; - Poco::Dynamic::Var var = parser.parse(field); - return getDataTypeFromJSONField(var); + String json = "{\"field\" : " + field + "}"; + auto var = parser.parse(json); + Poco::JSON::Object::Ptr object = var.extract(); + return getDataTypeFromJSONField(object->get("field")); } case FormatSettings::EscapingRule::CSV: { diff --git a/src/Formats/JSONEachRowUtils.cpp b/src/Formats/JSONEachRowUtils.cpp index df4ee77e586..b7672388491 100644 --- a/src/Formats/JSONEachRowUtils.cpp +++ b/src/Formats/JSONEachRowUtils.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -217,6 +218,25 @@ DataTypePtr getDataTypeFromJSONField(const Poco::Dynamic::Var & field) return std::make_shared(nested_data_types.back()); } + if (field.type() == typeid(Poco::JSON::Object::Ptr)) + { + Poco::JSON::Object::Ptr object = field.extract(); + auto names = object->getNames(); + DataTypePtr value_type; + for (const auto & name : names) + { + auto type = getDataTypeFromJSONField(object->get(name)); + if (!type) + return nullptr; + + if (value_type && value_type->getName() != type->getName()) + return nullptr; + + value_type = type; + } + return std::make_shared(std::make_shared(), value_type); + } + throw Exception{ErrorCodes::INCORRECT_DATA, "Unexpected JSON type {}", field.type().name()}; } diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index 72ac94865e2..b59db12a16c 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -3019,10 +3019,8 @@ namespace { std::vector column_names_used; column_names_used.reserve(used_column_indices_in_nested.size()); - for (size_t i : used_column_indices_in_nested) column_names_used.emplace_back(nested_column_names[i]); - auto field_serializer = std::make_unique( std::move(column_names_used), field_descriptor, std::move(nested_message_serializer), get_root_desc_function); transformColumnIndices(used_column_indices_in_nested, nested_column_indices); @@ -3247,6 +3245,13 @@ namespace NameAndTypePair getNameAndDataTypeFromField(const google::protobuf::FieldDescriptor * field_descriptor, bool allow_repeat = true) { + if (allow_repeat && field_descriptor->is_map()) + { + auto name_and_type = getNameAndDataTypeFromField(field_descriptor, false); + const auto * tuple_type = assert_cast(name_and_type.type.get()); + return {name_and_type.name, std::make_shared(tuple_type->getElements())}; + } + if (allow_repeat && field_descriptor->is_repeated()) { auto name_and_type = getNameAndDataTypeFromField(field_descriptor, false); diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 95d37d8422e..aa181ea0b8b 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -239,10 +239,8 @@ static ColumnWithTypeAndName readColumnWithTimestampData(std::shared_ptr -static ColumnWithTypeAndName readColumnWithDecimalData(std::shared_ptr & arrow_column, const String & column_name) +static ColumnWithTypeAndName readColumnWithDecimalDataImpl(std::shared_ptr & arrow_column, const String & column_name, DataTypePtr internal_type) { - const auto * arrow_decimal_type = static_cast(arrow_column->type().get()); - auto internal_type = std::make_shared>(arrow_decimal_type->precision(), arrow_decimal_type->scale()); auto internal_column = internal_type->createColumn(); auto & column = assert_cast &>(*internal_column); auto & column_data = column.getData(); @@ -259,6 +257,21 @@ static ColumnWithTypeAndName readColumnWithDecimalData(std::shared_ptr +static ColumnWithTypeAndName readColumnWithDecimalData(std::shared_ptr & arrow_column, const String & column_name) +{ + const auto * arrow_decimal_type = static_cast(arrow_column->type().get()); + size_t precision = arrow_decimal_type->precision(); + auto internal_type = createDecimal(precision, arrow_decimal_type->scale()); + if (precision <= DecimalUtils::max_precision) + return readColumnWithDecimalDataImpl(arrow_column, column_name, internal_type); + else if (precision <= DecimalUtils::max_precision) + return readColumnWithDecimalDataImpl(arrow_column, column_name, internal_type); + else if (precision <= DecimalUtils::max_precision) + return readColumnWithDecimalDataImpl(arrow_column, column_name, internal_type); + return readColumnWithDecimalDataImpl(arrow_column, column_name, internal_type); +} + /// Creates a null bytemap from arrow's null bitmap static ColumnPtr readByteMapFromArrowColumn(std::shared_ptr & arrow_column) { @@ -373,9 +386,9 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( case arrow::Type::TIMESTAMP: return readColumnWithTimestampData(arrow_column, column_name); case arrow::Type::DECIMAL128: - return readColumnWithDecimalData(arrow_column, column_name); + return readColumnWithDecimalData(arrow_column, column_name); case arrow::Type::DECIMAL256: - return readColumnWithDecimalData(arrow_column, column_name); + return readColumnWithDecimalData(arrow_column, column_name); case arrow::Type::MAP: { auto arrow_nested_column = getNestedArrowColumn(arrow_column); diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp index 0366b8ff2e3..d2e0d6e21a9 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp @@ -84,7 +84,7 @@ void CustomSeparatedRowInputFormat::syncAfterError() void CustomSeparatedRowInputFormat::setReadBuffer(ReadBuffer & in_) { buf = std::make_unique(in_); - RowInputFormatWithNamesAndTypes::setReadBuffer(in_); + RowInputFormatWithNamesAndTypes::setReadBuffer(*buf); } CustomSeparatedFormatReader::CustomSeparatedFormatReader( diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index da6adced0a7..263702ad20f 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -188,6 +188,7 @@ JSONCompactEachRowRowSchemaReader::JSONCompactEachRowRowSchemaReader(ReadBuffer DataTypes JSONCompactEachRowRowSchemaReader::readRowAndGetDataTypes() { + skipWhitespaceIfAny(in); if (in.eof()) return {}; diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 4c8f6ab2c54..651b9545c81 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -87,6 +87,7 @@ ORC_UNIQUE_PTR ORCBlockOutputFormat::getORCType(const DataTypePtr & t { return orc::createPrimitiveType(orc::TypeKind::DOUBLE); } + case TypeIndex::Date32: [[fallthrough]]; case TypeIndex::Date: { return orc::createPrimitiveType(orc::TypeKind::DATE); @@ -292,6 +293,7 @@ void ORCBlockOutputFormat::writeColumn( writeNumbers(orc_column, column, null_bytemap, [](const UInt16 & value){ return value; }); break; } + case TypeIndex::Date32: [[fallthrough]]; case TypeIndex::Int32: { writeNumbers(orc_column, column, null_bytemap, [](const Int32 & value){ return value; }); diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index 23b8c8212c0..90db6f6f0ec 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -124,7 +124,7 @@ void RegexpRowInputFormat::setReadBuffer(ReadBuffer & in_) IInputFormat::setReadBuffer(*buf); } -RegexpSchemaReader::RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) +RegexpSchemaReader::RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_, ContextPtr context_) : IRowSchemaReader( buf, format_settings_.max_rows_to_read_for_schema_inference, @@ -132,6 +132,7 @@ RegexpSchemaReader::RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & , format_settings(format_settings_) , field_extractor(format_settings) , buf(in_) + , context(context_) { } @@ -147,7 +148,7 @@ DataTypes RegexpSchemaReader::readRowAndGetDataTypes() for (size_t i = 0; i != field_extractor.getMatchedFieldsSize(); ++i) { String field(field_extractor.getField(i)); - data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, format_settings.regexp.escaping_rule)); + data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, format_settings.regexp.escaping_rule, context)); } return data_types; @@ -206,9 +207,9 @@ void registerFileSegmentationEngineRegexp(FormatFactory & factory) void registerRegexpSchemaReader(FormatFactory & factory) { - factory.registerSchemaReader("Regexp", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr) + factory.registerSchemaReader("Regexp", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr context) { - return std::make_shared(buf, settings); + return std::make_shared(buf, settings, context); }); } diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.h b/src/Processors/Formats/Impl/RegexpRowInputFormat.h index acd7b0d0934..dffd2f82e02 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.h +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.h @@ -75,7 +75,7 @@ private: class RegexpSchemaReader : public IRowSchemaReader { public: - RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings); + RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, ContextPtr context_); private: DataTypes readRowAndGetDataTypes() override; @@ -84,6 +84,7 @@ private: const FormatSettings format_settings; RegexpFieldExtractor field_extractor; PeekableReadBuffer buf; + ContextPtr context; }; } diff --git a/tests/queries/0_stateless/00825_protobuf_format_enum_mapping.sh b/tests/queries/0_stateless/00825_protobuf_format_enum_mapping.sh index 2731e4bcce3..8d9e2689e26 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_enum_mapping.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_enum_mapping.sh @@ -25,15 +25,15 @@ SELECT * FROM enum_mapping_protobuf_00825; EOF BINARY_FILE_PATH=$(mktemp "$CURDIR/00825_protobuf_format_enum_mapping.XXXXXX.binary") -$CLICKHOUSE_CLIENT --query "SELECT * FROM enum_mapping_protobuf_00825 FORMAT Protobuf SETTINGS format_schema = '$SCHEMADIR/00825_protobuf_format_enum_mapping:Message'" > "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "SELECT * FROM enum_mapping_protobuf_00825 FORMAT Protobuf SETTINGS format_schema = '$SCHEMADIR/00825_protobuf_format_enum_mapping:EnumMessage'" > "$BINARY_FILE_PATH" # Check the output in the protobuf format echo -$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/00825_protobuf_format_enum_mapping:Message" --input "$BINARY_FILE_PATH" +$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/00825_protobuf_format_enum_mapping:EnumMessage" --input "$BINARY_FILE_PATH" # Check the input in the protobuf format (now the table contains the same data twice). echo -$CLICKHOUSE_CLIENT --query "INSERT INTO enum_mapping_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_enum_mapping:Message'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO enum_mapping_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_enum_mapping:EnumMessage'" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM enum_mapping_protobuf_00825" rm "$BINARY_FILE_PATH" diff --git a/tests/queries/0_stateless/00825_protobuf_format_enum_mapping.yaodzJ.binary b/tests/queries/0_stateless/00825_protobuf_format_enum_mapping.yaodzJ.binary new file mode 100644 index 0000000000000000000000000000000000000000..4b7b97a300feafdcad4775a6a33acfa138393a9d GIT binary patch literal 11 ScmZSLU}WM*VPIh5U<3dIF#v@C literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/01903_http_fields.sh b/tests/queries/0_stateless/01903_http_fields.sh index 0f2b0df13d9..3dab080c823 100755 --- a/tests/queries/0_stateless/01903_http_fields.sh +++ b/tests/queries/0_stateless/01903_http_fields.sh @@ -4,6 +4,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +echo $CLICKHOUSE_CURL +echo $CLICKHOUSE_URL + DEFAULT_MAX_NAME_SIZE=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.settings WHERE name='http_max_field_name_size'") DEFAULT_MAX_VALUE_SIZE=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.settings WHERE name='http_max_field_value_size'") diff --git a/tests/queries/0_stateless/02149_external_schema_inference.reference b/tests/queries/0_stateless/02149_external_schema_inference.reference new file mode 100644 index 00000000000..875659c7fb6 --- /dev/null +++ b/tests/queries/0_stateless/02149_external_schema_inference.reference @@ -0,0 +1,168 @@ +Protobuf + +a_b_c Array(Array(Array(Int32))) + +a String +b_c Array(Array(Float64)) + +x Enum8(\'FIRST\' = 0, \'SECOND\' = 1, \'TEN\' = 10, \'HUNDRED\' = 100) + +a Map(String, UInt32) + +x_y_z Array(Array(Int32)) + +uuid String +name String +surname String +gender Enum8(\'female\' = 0, \'male\' = 1) +birthDate UInt32 +photo String +phoneNumber String +isOnline UInt8 +visitTime UInt32 +age UInt32 +zodiacSign Enum8(\'aries\' = 0, \'taurus\' = 1, \'gemini\' = 2, \'cancer\' = 3, \'leo\' = 4, \'virgo\' = 5, \'libra\' = 6, \'scorpius\' = 7, \'sagittarius\' = 8, \'capricorn\' = 9, \'aquarius\' = 10, \'pisces\' = 11) +songs Array(String) +color Array(UInt32) +hometown String +location Array(Float32) +pi Float64 +lotteryWin Float64 +someRatio Float32 +temperature Float32 +randomBigNumber Int64 +measureUnits Array(Tuple(unit String, coef Float32)) +nestiness_a_b_c Tuple(d UInt32, e Array(UInt32)) + +location Array(Int32) +pi Float32 +uuid String +newFieldBool UInt8 +name String +gender Enum8(\'male\' = 0, \'female\' = 1) +zodiacSign Int32 +birthDate Int64 +age String +isOnline Enum8(\'offline\' = 0, \'online\' = 1) +someRatio Float64 +visitTime UInt64 +newMessage Tuple(empty Array(Tuple()), z Float32) +randomBigNumber Int64 +newFieldInt Array(Int32) +color Array(Float32) +lotteryWin UInt64 +surname String +phoneNumber UInt64 +temperature Int32 +newFieldStr String +measureUnits_unit Array(String) +measureUnits_coef Array(Float32) +nestiness_a_b_c_d UInt32 +nestiness_a_b_c_e Array(UInt32) + +uuid String +name String +surname String +gender String +birthDate String +phoneNumber String +isOnline String +visitTime String +age String +zodiacSign String +songs Array(String) +color Array(String) +hometown String +location Array(String) +pi String +lotteryWin String +someRatio String +temperature String +randomBigNumber String +measureUnits Tuple(unit Array(String), coef Array(String)) +nestiness_a_b_c Tuple(d String, e Array(String)) + +uuid String +name String +surname String +gender Enum8(\'female\' = 0, \'male\' = 1) +birthDate UInt32 +photo String +phoneNumber String +isOnline UInt8 +visitTime UInt32 +age UInt32 +zodiacSign Enum8(\'aries\' = 0, \'taurus\' = 1, \'gemini\' = 2, \'cancer\' = 3, \'leo\' = 4, \'virgo\' = 5, \'libra\' = 6, \'scorpius\' = 7, \'sagittarius\' = 8, \'capricorn\' = 9, \'aquarius\' = 10, \'pisces\' = 11) +songs Array(String) +color Array(UInt32) +hometown String +location Array(Float32) +pi Float64 +lotteryWin Float64 +someRatio Float32 +temperature Float32 +randomBigNumber Int64 +measureunits Tuple(coef Array(Float32), unit Array(String)) +nestiness_a_b_c Tuple(d UInt32, e Array(UInt32)) +newFieldStr String +newFieldInt Int32 +newBool UInt8 + +identifier String +modules Array(Tuple(module_id UInt32, supply UInt32, temp UInt32, nodes Array(Tuple(node_id UInt32, opening_time UInt32, closing_time UInt32, current UInt32, coords_y Float32)))) + +Capnproto + +value Enum8(\'one\' = 0, \'two\' = 1, \'tHrEe\' = 2) + +value UInt64 +list1 Array(UInt64) +list2 Array(Array(Array(UInt64))) + +lc1 String +lc2 Nullable(String) +lc3 Array(Nullable(String)) + +value UInt64 +nested Tuple(a Tuple(b UInt64, c Array(Array(UInt64))), d Array(Tuple(e Array(Array(Tuple(f UInt64, g UInt64))), h Array(Tuple(k Array(UInt64)))))) + +nested Tuple(value Array(UInt64), array Array(Array(UInt64)), tuple Array(Tuple(one UInt64, two UInt64))) + +a Tuple(b UInt64, c Tuple(d UInt64, e Tuple(f UInt64))) + +nullable Nullable(UInt64) +array Array(Nullable(UInt64)) +tuple Tuple(nullable Nullable(UInt64)) + +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +float32 Float32 +float64 Float64 +string String +fixed String +data String +date UInt16 +datetime UInt32 +datetime64 Int64 + +value UInt64 +tuple1 Tuple(one UInt64, two Tuple(three UInt64, four UInt64)) +tuple2 Tuple(nested1 Tuple(nested2 Tuple(x UInt64))) + +RawBLOB + +raw_blob String + +LineAsString + +line String + +JSONAsString + +json String diff --git a/tests/queries/0_stateless/02149_external_schema_inference.sh b/tests/queries/0_stateless/02149_external_schema_inference.sh new file mode 100755 index 00000000000..9c878236746 --- /dev/null +++ b/tests/queries/0_stateless/02149_external_schema_inference.sh @@ -0,0 +1,91 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +FILE_NAME=test_02149.data +DATA_FILE=$USER_FILES_PATH/$FILE_NAME + +touch $DATA_FILE + +SCHEMADIR=$(clickhouse-client --query "select * from file('$FILE_NAME', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") +CLIENT_SCHEMADIR=$CURDIR/format_schemas +SERVER_SCHEMADIR=test_02149 +mkdir -p $SCHEMADIR/$SERVER_SCHEMADIR +cp -r $CLIENT_SCHEMADIR/* $SCHEMADIR/$SERVER_SCHEMADIR/ + +echo -e "Protobuf\n" +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Protobuf') settings format_schema='$SERVER_SCHEMADIR/00825_protobuf_format_array_3dim:ABC'" + +echo +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Protobuf') settings format_schema='$SERVER_SCHEMADIR/00825_protobuf_format_array_of_arrays:AA'" + +echo +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Protobuf') settings format_schema='$SERVER_SCHEMADIR/00825_protobuf_format_enum_mapping.proto:EnumMessage'" + +echo +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Protobuf') settings format_schema='$SERVER_SCHEMADIR/00825_protobuf_format_map:Message'" + +echo +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Protobuf') settings format_schema='$SERVER_SCHEMADIR/00825_protobuf_format_nested_in_nested:MessageType'" + +echo +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Protobuf') settings format_schema='$SERVER_SCHEMADIR/00825_protobuf_format_persons:Person'" + +echo +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Protobuf') settings format_schema='$SERVER_SCHEMADIR/00825_protobuf_format_persons:AltPerson'" + +echo +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Protobuf') settings format_schema='$SERVER_SCHEMADIR/00825_protobuf_format_persons:StrPerson'" + +echo +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Protobuf') settings format_schema='$SERVER_SCHEMADIR/00825_protobuf_format_persons_syntax2:Syntax2Person'" + +echo +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Protobuf') settings format_schema='$SERVER_SCHEMADIR/00825_protobuf_format_skipped_column_in_nested:UpdateMessage'" + + +echo -e "\nCapnproto\n" +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CapnProto') settings format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message'" + +echo +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CapnProto') settings format_schema='$SERVER_SCHEMADIR/02030_capnp_lists:Message'" + +echo +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CapnProto') settings format_schema='$SERVER_SCHEMADIR/02030_capnp_low_cardinality:Message'" + +echo +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CapnProto') settings format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_lists_and_tuples:Message'" + +echo +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CapnProto') settings format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_table:Message'" + +echo +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CapnProto') settings format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_tuples:Message'" + +echo +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CapnProto') settings format_schema='$SERVER_SCHEMADIR/02030_capnp_nullable:Message'" + +echo +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CapnProto') settings format_schema='$SERVER_SCHEMADIR/02030_capnp_simple_types:Message'" + +echo +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CapnProto') settings format_schema='$SERVER_SCHEMADIR/02030_capnp_tuples:Message'" + +echo -e "\nRawBLOB\n" +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'RawBLOB')" + +echo -e "\nLineAsString\n" +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'LineAsString')" + +echo -e "\nJSONAsString\n" +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONAsString')" + + + +rm -rf $SCHEMADIR/$SERVER_SCHEMADIR +rm $DATA_FILE diff --git a/tests/queries/0_stateless/02149_schema_inference.reference b/tests/queries/0_stateless/02149_schema_inference.reference new file mode 100644 index 00000000000..1bcd4642f26 --- /dev/null +++ b/tests/queries/0_stateless/02149_schema_inference.reference @@ -0,0 +1,160 @@ +TSV +column_1 Nullable(String) +column_2 Nullable(String) +column_3 Nullable(String) +column_4 Nullable(String) +42 Some string [1, 2, 3, 4] (1, 2, 3) +42 abcd [] (4, 5, 6) +TSVWithNames +number Nullable(String) +string Nullable(String) +array Nullable(String) +tuple Nullable(String) +42 Some string [1, 2, 3, 4] (1, 2, 3) +42 abcd [] (4, 5, 6) +CSV +column_1 Nullable(Float64) +column_2 Nullable(String) +column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +column_4 Array(Nullable(Float64)) +\N Some string [([1,2.3],'String'),([],NULL)] [1,NULL,3] +42 \N [([1,2.3],'String'),([3],'abcd')] [4,5,6] +column_1 Nullable(String) +column_2 Nullable(String) +42 String +String 42 +column_1 Nullable(String) +column_2 Nullable(String) +\N [NULL, NULL] +\N [] +CSVWithNames +a Nullable(Float64) +b Nullable(String) +c Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +d Array(Nullable(Float64)) +\N Some string [([1,2.3],'String'),([],NULL)] [1,NULL,3] +42 \N [([1,2.3],'String'),([3],'abcd')] [4,5,6] +JSONCompactEachRow +column_1 Nullable(Float64) +column_2 Array(Tuple(Nullable(Float64), Nullable(String))) +column_3 Map(String, Nullable(Float64)) +column_4 Nullable(UInt8) +42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} 1 +column_1 Nullable(Float64) +column_2 Array(Tuple(Nullable(Float64), Nullable(String))) +column_3 Map(String, Nullable(Float64)) +column_4 Nullable(UInt8) +\N [(1,'String'),(2,NULL)] {'key':NULL,'key2':24} \N +32 [(2,'String 2'),(3,'hello')] {'key3':4242,'key4':2424} 1 +JSONCompactEachRowWithNames +a Nullable(Float64) +b Array(Tuple(Nullable(Float64), Nullable(String))) +c Map(String, Nullable(Float64)) +d Nullable(UInt8) +42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} 1 +JSONEachRow +d Nullable(UInt8) +b Array(Tuple(Nullable(Float64), Nullable(String))) +c Map(String, Nullable(Float64)) +a Nullable(Float64) +1 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} 42.42 +d Nullable(UInt8) +b Array(Tuple(Nullable(Float64), Nullable(String))) +c Map(String, Nullable(Float64)) +a Nullable(Float64) +\N [(1,'String'),(2,NULL)] {'key':NULL,'key2':24} \N +1 [(2,'String 2'),(3,'hello')] {'key3':4242,'key4':2424} 32 +b Nullable(String) +c Array(Nullable(Float64)) +a Nullable(Float64) +s1 [] 1 +\N [2] 2 +\N [] \N +\N [] \N +\N [3] \N +TSKV +b Nullable(String) +c Nullable(String) +a Nullable(String) +s1 \N 1 +} [2] 2 +\N \N \N +\N \N \N +\N [3] \N +Values +column_1 Nullable(Float64) +column_2 Nullable(String) +column_3 Array(Nullable(Float64)) +column_4 Tuple(Nullable(Float64), Nullable(String)) +column_5 Tuple(Array(Nullable(Float64)), Array(Tuple(Nullable(Float64), Nullable(String)))) +42.42 Some string [1,2,3] (1,'2') ([1,2],[(3,'4'),(5,'6')]) +column_1 Nullable(Float64) +column_2 Nullable(String) +column_3 Array(Nullable(Float64)) +column_4 Tuple(Nullable(Float64), Nullable(Float64)) +column_5 Tuple(Array(Nullable(Float64)), Array(Tuple(Nullable(Float64), Nullable(String)))) +42.42 \N [1,NULL,3] (1,NULL) ([1,2],[(3,'4'),(5,'6')]) +\N Some string [10] (1,2) ([],[]) +Regexp +column_1 Nullable(String) +column_2 Nullable(String) +column_3 Nullable(String) +42 Some string 1 [([1, 2, 3], String 1), ([], String 1)] +2 Some string 2 [([4, 5, 6], String 2), ([], String 2)] +312 Some string 3 [([1, 2, 3], String 2), ([], String 2)] +column_1 Nullable(Float64) +column_2 Nullable(String) +column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +42 Some string 1 [([1,2,3],'String 1'),([],'String 1')] +3 Some string 2 [([3,5,1],'String 2'),([],'String 2')] +244 Some string 3 [([],'String 3'),([],'String 3')] +column_1 Nullable(Float64) +column_2 Nullable(String) +column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +42 Some string 1 [([1,2,3],'String 1'),([],'String 1')] +2 Some string 2 [([],'String 2'),([],'String 2')] +43 Some string 3 [([1,5,3],'String 3'),([],'String 3')] +column_1 Nullable(Float64) +column_2 Nullable(String) +column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] +52 Some string 2 [([],'String 2'),([1],'String 2')] +24 Some string 3 [([1,2,3],'String 3'),([1],'String 3')] +CustomSeparated +column_1 Nullable(Float64) +column_2 Nullable(String) +column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +42.42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] +42 Some string 2 [([],'String 2'),([],'String 2')] +\N Some string 3 [([1,2,3],'String 3'),([1],'String 3')] +column_1 Nullable(Float64) +column_2 Nullable(String) +column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +42.42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] +42 Some string 2 [([],'String 2'),([],'String 2')] +\N Some string 3 [([1,2,3],'String 3'),([1],'String 3')] +column_1 Nullable(Float64) +column_2 Nullable(String) +column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +42.42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] +42 Some string 2 [([],'String 2'),([],'String 2')] +\N Some string 3 [([1,2,3],'String 3'),([1],'String 3')] +Template +column_1 Nullable(Float64) +column_2 Nullable(String) +column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +42.42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] +42 Some string 2 [([],'String 2'),([],'String 2')] +\N Some string 3 [([1,2,3],'String 3'),([1],'String 3')] +column_1 Nullable(Float64) +column_2 Nullable(String) +column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +42.42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] +42 Some string 2 [([],'String 2'),([],'String 2')] +\N Some string 3 [([1,2,3],'String 3'),([1],'String 3')] +column_1 Nullable(Float64) +column_2 Nullable(String) +column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +42.42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] +42 Some string 2 [([],'String 2'),([],'String 2')] +\N Some string 3 [([1,2,3],'String 3'),([1],'String 3')] diff --git a/tests/queries/0_stateless/02149_schema_inference.sh b/tests/queries/0_stateless/02149_schema_inference.sh new file mode 100755 index 00000000000..cc6c8d7e276 --- /dev/null +++ b/tests/queries/0_stateless/02149_schema_inference.sh @@ -0,0 +1,242 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +FILE_NAME=test_02149.data +DATA_FILE=$USER_FILES_PATH/$FILE_NAME + +touch $DATA_FILE + +SCHEMADIR=$(clickhouse-client --query "select * from file('$FILE_NAME', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") + +echo "TSV" + +echo -e "42\tSome string\t[1, 2, 3, 4]\t(1, 2, 3) +42\tabcd\t[]\t(4, 5, 6)" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSV')" + +echo "TSVWithNames" + +echo -e "number\tstring\tarray\ttuple +42\tSome string\t[1, 2, 3, 4]\t(1, 2, 3) +42\tabcd\t[]\t(4, 5, 6)" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSVWithNames')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSVWithNames')" + +echo "CSV" + +echo -e "\N,\"Some string\",\"[([1, 2.3], 'String'), ([], NULL)]\",\"[1, NULL, 3]\" +42,\N,\"[([1, 2.3], 'String'), ([3.], 'abcd')]\",\"[4, 5, 6]\"" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" + +echo -e "42,\"String\" +\"String\",42" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" + +echo -e "\N,\"[NULL, NULL]\" +\N,[]" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSV')" + +echo "CSVWithNames" + +echo -e "a,b,c,d +\N,\"Some string\",\"[([1, 2.3], 'String'), ([], NULL)]\",\"[1, NULL, 3]\" +42,\N,\"[([1, 2.3], 'String'), ([3.], 'abcd')]\",\"[4, 5, 6]\"" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CSVWithNames')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CSVWithNames')" + +echo "JSONCompactEachRow" + +echo -e "[42.42, [[1, \"String\"], [2, \"abcd\"]], {\"key\" : 42, \"key2\" : 24}, true]" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONCompactEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONCompactEachRow')" + +echo -e "[null, [[1, \"String\"], [2, null]], {\"key\" : null, \"key2\" : 24}, null] +[32, [[2, \"String 2\"], [3, \"hello\"]], {\"key3\" : 4242, \"key4\" : 2424}, true]" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONCompactEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONCompactEachRow')" + +echo "JSONCompactEachRowWithNames" + +echo -e "[\"a\", \"b\", \"c\", \"d\"] +[42.42, [[1, \"String\"], [2, \"abcd\"]], {\"key\" : 42, \"key2\" : 24}, true]" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONCompactEachRowWithNames')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONCompactEachRowWithNames')" + + +echo "JSONEachRow" +echo -e '{"a" : 42.42, "b" : [[1, "String"], [2, "abcd"]], "c" : {"key" : 42, "key2" : 24}, "d" : true}' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONEachRow')" + +echo -e '{"a" : null, "b" : [[1, "String"], [2, null]], "c" : {"key" : null, "key2" : 24}, "d" : null} +{"a" : 32, "b" : [[2, "String 2"], [3, "hello"]], "c" : {"key3" : 4242, "key4" : 2424}, "d" : true}' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONEachRow')" + +echo -e '{"a" : 1, "b" : "s1", "c" : null} +{"c" : [2], "a" : 2, "b" : null} +{} +{"a" : null} +{"c" : [3], "a" : null}' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONEachRow')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'JSONEachRow')" + + +echo "TSKV" + +echo -e 'a=1\tb=s1\tc=\N +c=[2]\ta=2\tb=\N} + +a=\N +c=[3]\ta=\N' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'TSKV')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'TSKV')" + + +echo "Values" + +echo -e "(42.42, 'Some string', [1, 2, 3], (1, '2'), ([1, 2], [(3, '4'), (5, '6')]))" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Values')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Values')" + +echo -e "(42.42, NULL, [1, NULL, 3], (1, NULL), ([1, 2], [(3, '4'), (5, '6')])), (NULL, 'Some string', [10], (1, 2), ([], []))" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Values')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Values')" + + +echo "Regexp" + +REGEXP="^Line: value_1=(.+?), value_2=(.+?), value_3=(.+?)" + +echo "Line: value_1=42, value_2=Some string 1, value_3=[([1, 2, 3], String 1), ([], String 1)] +Line: value_1=2, value_2=Some string 2, value_3=[([4, 5, 6], String 2), ([], String 2)] +Line: value_1=312, value_2=Some string 3, value_3=[([1, 2, 3], String 2), ([], String 2)]" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Regexp') settings format_regexp='$REGEXP', format_regexp_escaping_rule='Escaped'" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Regexp') settings format_regexp='$REGEXP', format_regexp_escaping_rule='Escaped'" + + +echo "Line: value_1=42, value_2=\"Some string 1\", value_3=\"[([1, 2, 3], 'String 1'), ([], 'String 1')]\" +Line: value_1=3, value_2=\"Some string 2\", value_3=\"[([3, 5, 1], 'String 2'), ([], 'String 2')]\" +Line: value_1=244, value_2=\"Some string 3\", value_3=\"[([], 'String 3'), ([], 'String 3')]\"" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Regexp') settings format_regexp='$REGEXP', format_regexp_escaping_rule='CSV'" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Regexp') settings format_regexp='$REGEXP', format_regexp_escaping_rule='CSV'" + + +echo "Line: value_1=42, value_2='Some string 1', value_3=[([1, 2, 3], 'String 1'), ([], 'String 1')] +Line: value_1=2, value_2='Some string 2', value_3=[([], 'String 2'), ([], 'String 2')] +Line: value_1=43, value_2='Some string 3', value_3=[([1, 5, 3], 'String 3'), ([], 'String 3')]" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Regexp') settings format_regexp='$REGEXP', format_regexp_escaping_rule='Quoted'" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Regexp') settings format_regexp='$REGEXP', format_regexp_escaping_rule='Quoted'" + + +echo "Line: value_1=42, value_2=\"Some string 1\", value_3=[[[1, 2, 3], \"String 1\"], [[1], \"String 1\"]] +Line: value_1=52, value_2=\"Some string 2\", value_3=[[[], \"String 2\"], [[1], \"String 2\"]] +Line: value_1=24, value_2=\"Some string 3\", value_3=[[[1, 2, 3], \"String 3\"], [[1], \"String 3\"]]" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Regexp') settings format_regexp='$REGEXP', format_regexp_escaping_rule='JSON'" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Regexp') settings format_regexp='$REGEXP', format_regexp_escaping_rule='JSON'" + + +echo "CustomSeparated" + +CUSTOM_SETTINGS="SETTINGS format_custom_row_before_delimiter='', format_custom_row_after_delimiter='\n', format_custom_row_between_delimiter='\n', format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n', format_custom_field_delimiter=''" + +echo -e " +42.42\"Some string 1\"\"[([1, 2, 3], 'String 1'), ([1], 'String 1')]\" + +42\"Some string 2\"\"[([], 'String 2'), ([], 'String 2')]\" + +\N\"Some string 3\"\"[([1, 2, 3], 'String 3'), ([1], 'String 3')]\" +" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CustomSeparated') $CUSTOM_SETTINGS, format_custom_escaping_rule='CSV'" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CustomSeparated') $CUSTOM_SETTINGS, format_custom_escaping_rule='CSV'" + +echo -e " +42.42'Some string 1'[([1, 2, 3], 'String 1'), ([1], 'String 1')] + +42'Some string 2'[([], 'String 2'), ([], 'String 2')] + +NULL'Some string 3'[([1, 2, 3], 'String 3'), ([1], 'String 3')] +" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CustomSeparated') $CUSTOM_SETTINGS, format_custom_escaping_rule='Quoted'" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CustomSeparated') $CUSTOM_SETTINGS, format_custom_escaping_rule='Quoted'" + +echo -e " +42.42\"Some string 1\"[[[1, 2, 3], \"String 1\"], [[1], \"String 1\"]] + +42\"Some string 2\"[[[], \"String 2\"], [[], \"String 2\"]] + +null\"Some string 3\"[[[1, 2, 3], \"String 3\"], [[1], \"String 3\"]] +" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'CustomSeparated') $CUSTOM_SETTINGS, format_custom_escaping_rule='JSON'" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'CustomSeparated') $CUSTOM_SETTINGS, format_custom_escaping_rule='JSON'" + + +echo "Template" + +echo -e " +\${data}" > $SCHEMADIR/resultset_format_02149 + +echo -e "\${column_1:CSV}\${column_2:CSV}\${column_3:CSV}" > $SCHEMADIR/row_format_02149 + +TEMPLATE_SETTINGS="SETTINGS format_template_rows_between_delimiter='\n', format_template_row='row_format_02149', format_template_resultset='resultset_format_02149'" + +echo -e " +42.42\"Some string 1\"\"[([1, 2, 3], 'String 1'), ([1], 'String 1')]\" + +42\"Some string 2\"\"[([], 'String 2'), ([], 'String 2')]\" + +\N\"Some string 3\"\"[([1, 2, 3], 'String 3'), ([1], 'String 3')]\" +" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Template') $TEMPLATE_SETTINGS" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Template') $TEMPLATE_SETTINGS" + +echo -e "\${column_1:Quoted}\${column_2:Quoted}\${column_3:Quoted}" > $SCHEMADIR/row_format_02149 + +echo -e " +42.42'Some string 1'[([1, 2, 3], 'String 1'), ([1], 'String 1')] + +42'Some string 2'[([], 'String 2'), ([], 'String 2')] + +NULL'Some string 3'[([1, 2, 3], 'String 3'), ([1], 'String 3')] +" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Template') $TEMPLATE_SETTINGS" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Template') $TEMPLATE_SETTINGS" + +echo -e "\${column_1:JSON}\${column_2:JSON}\${column_3:JSON}" > $SCHEMADIR/row_format_02149 + +echo -e " +42.42\"Some string 1\"[[[1, 2, 3], \"String 1\"], [[1], \"String 1\"]] + +42\"Some string 2\"[[[], \"String 2\"], [[], \"String 2\"]] + +null\"Some string 3\"[[[1, 2, 3], \"String 3\"], [[1], \"String 3\"]] +" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Template') $TEMPLATE_SETTINGS" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Template') $TEMPLATE_SETTINGS" + +rm $SCHEMADIR/resultset_format_02149 $SCHEMADIR/row_format_02149 +rm $DATA_FILE + diff --git a/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.reference b/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.reference new file mode 100644 index 00000000000..fd2f8adf54f --- /dev/null +++ b/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.reference @@ -0,0 +1,50 @@ +0 Str: 0 [0,1] +1 Str: 1 [1,2] +2 Str: 2 [2,3] +3 Str: 3 [3,4] +4 Str: 4 [4,5] +5 Str: 5 [5,6] +6 Str: 6 [6,7] +7 Str: 7 [7,8] +8 Str: 8 [8,9] +9 Str: 9 [9,10] +0 Str: 0 [0,1] +1 Str: 1 [1,2] +2 Str: 2 [2,3] +3 Str: 3 [3,4] +4 Str: 4 [4,5] +5 Str: 5 [5,6] +6 Str: 6 [6,7] +7 Str: 7 [7,8] +8 Str: 8 [8,9] +9 Str: 9 [9,10] +0 0 [0,1] +1 1 [1,2] +2 2 [2,3] +3 3 [3,4] +4 4 [4,5] +5 5 [5,6] +6 6 [6,7] +7 7 [7,8] +8 8 [8,9] +9 9 [9,10] +0 0 [0,1] +1 1 [1,2] +2 2 [2,3] +3 3 [3,4] +4 4 [4,5] +5 5 [5,6] +6 6 [6,7] +7 7 [7,8] +8 8 [8,9] +9 9 [9,10] +0 0 [0,1] +1 1 [1,2] +2 2 [2,3] +3 3 [3,4] +4 4 [4,5] +5 5 [5,6] +6 6 [6,7] +7 7 [7,8] +8 8 [8,9] +9 9 [9,10] diff --git a/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh b/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh new file mode 100755 index 00000000000..fa05e40ab5a --- /dev/null +++ b/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh @@ -0,0 +1,45 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +mkdir $USER_FILES_PATH/test_02149 +FILE_NAME=test_02149/data.Parquet +DATA_FILE=$USER_FILES_PATH/$FILE_NAME + +$CLICKHOUSE_CLIENT -q "select number as num, concat('Str: ', toString(number)) as str, [number, number + 1] as arr from numbers(10) format Parquet" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "drop table if exists test_02149" +$CLICKHOUSE_CLIENT -q "create table test_02149 engine=File('Parquet', '$FILE_NAME')" +$CLICKHOUSE_CLIENT -q "select * from test_02149" +$CLICKHOUSE_CLIENT -q "drop table test_02149" + +$CLICKHOUSE_CLIENT -q "attach table test_02149 from 'test_02149' engine=File('Parquet')" +$CLICKHOUSE_CLIENT -q "select * from test_02149" +$CLICKHOUSE_CLIENT -q "drop table test_02149" + +$CLICKHOUSE_CLIENT -q "create table test_02149 (x UInt32, s String, a Array(UInt32)) engine=Memory" +$CLICKHOUSE_CLIENT -q "insert into test_02149 select number, toString(number), [number, number + 1] from numbers(10)" + +$CLICKHOUSE_CLIENT -q "drop table if exists test_merge" +$CLICKHOUSE_CLIENT -q "create table test_merge engine=Merge(currentDatabase(), 'test_02149')" +$CLICKHOUSE_CLIENT -q "select * from test_merge" +$CLICKHOUSE_CLIENT -q "drop table test_merge" + +$CLICKHOUSE_CLIENT -q "drop table if exists test_distributed" +$CLICKHOUSE_CLIENT -q "create table test_distributed engine=Distributed(test_shard_localhost, currentDatabase(), 'test_02149')" +$CLICKHOUSE_CLIENT -q "select * from test_distributed" +$CLICKHOUSE_CLIENT -q "drop table test_distributed" + +$CLICKHOUSE_CLIENT -q "drop table if exists test_buffer" +$CLICKHOUSE_CLIENT -q "create table test_buffer engine=Buffer(currentDatabase(), 'test_02149', 16, 10, 100, 10000, 1000000, 10000000, 100000000)" +$CLICKHOUSE_CLIENT -q "select * from test_buffer" +$CLICKHOUSE_CLIENT -q "drop table test_buffer" + +rm -rf $USER_FILES_PATH/test_02149 + + diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference new file mode 100644 index 00000000000..79308cc5a75 --- /dev/null +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference @@ -0,0 +1,448 @@ +Arrow +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date UInt16 +datetime UInt32 +date32 Date32 +0 0 1970-01-01 +1 1 1970-01-02 +str String +fixed_string String +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(`tuple.0` UInt64, `tuple.1` String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(`nested1.0` Array(UInt64), `nested1.1` Map(String, UInt64))) +nested2 Tuple(`nested2.0` Tuple(`nested2.0.0` Array(Array(UInt64)), `nested2.0.1` Map(UInt64, Array(Tuple(`nested2.0.1.0` UInt64, `nested2.0.1.1` String)))), `nested2.1` UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +ArrowStream +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date UInt16 +datetime UInt32 +date32 Date32 +0 0 1970-01-01 +1 1 1970-01-02 +str String +fixed_string String +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(`tuple.0` UInt64, `tuple.1` String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(`nested1.0` Array(UInt64), `nested1.1` Map(String, UInt64))) +nested2 Tuple(`nested2.0` Tuple(`nested2.0.0` Array(Array(UInt64)), `nested2.0.1` Map(UInt64, Array(Tuple(`nested2.0.1.0` UInt64, `nested2.0.1.1` String)))), `nested2.1` UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +Parquet +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 Int64 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date UInt16 +datetime Int64 +date32 Date32 +0 0 1970-01-01 +1 1 1970-01-02 +str String +fixed_string String +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(`tuple.0` UInt64, `tuple.1` String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(`nested1.0` Array(UInt64), `nested1.1` Map(String, UInt64))) +nested2 Tuple(`nested2.0` Tuple(`nested2.0.0` Array(Array(UInt64)), `nested2.0.1` Map(UInt64, Array(Tuple(`nested2.0.1.0` UInt64, `nested2.0.1.1` String)))), `nested2.1` UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +ORC +int8 Int8 +uint8 Int8 +int16 Int16 +uint16 Int16 +int32 Int32 +uint32 Int32 +int64 Int64 +uint64 Int64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Date32 +datetime UInt32 +date32 Date32 +1970-01-01 0 1970-01-01 +1970-01-02 1 1970-01-02 +str String +fixed_string String +Str: 0 100 +Str: 1 200 +array Array(Int64) +tuple Tuple(`tuple.0` Int64, `tuple.1` String) +map Map(String, Int64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(`nested1.0` Array(Int64), `nested1.1` Map(String, Int64))) +nested2 Tuple(`nested2.0` Tuple(`nested2.0.0` Array(Array(Int64)), `nested2.0.1` Map(Int64, Array(Tuple(`nested2.0.1.0` Int64, `nested2.0.1.1` String)))), `nested2.1` Int8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +Native +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Date +datetime DateTime +date32 Date32 +1970-01-01 1970-01-01 03:00:00 1970-01-01 +1970-01-02 1970-01-01 03:00:01 1970-01-02 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(UInt64, String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) +nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +TSVWithNamesAndTypes +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Date +datetime DateTime +date32 Date32 +1970-01-01 1970-01-01 03:00:00 1970-01-01 +1970-01-02 1970-01-01 03:00:01 1970-01-02 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(UInt64, String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) +nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +TSVRawWithNamesAndTypes +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Date +datetime DateTime +date32 Date32 +1970-01-01 1970-01-01 03:00:00 1970-01-01 +1970-01-02 1970-01-01 03:00:01 1970-01-02 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(UInt64, String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) +nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +CSVWithNamesAndTypes +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Date +datetime DateTime +date32 Date32 +1970-01-01 1970-01-01 03:00:00 1970-01-01 +1970-01-02 1970-01-01 03:00:01 1970-01-02 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(UInt64, String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) +nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +JSONCompactEachRowWithNamesAndTypes +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Date +datetime DateTime +date32 Date32 +1970-01-01 1970-01-01 03:00:00 1970-01-01 +1970-01-02 1970-01-01 03:00:01 1970-01-02 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(UInt64, String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) +nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +JSONCompactStringsEachRowWithNamesAndTypes +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Date +datetime DateTime +date32 Date32 +1970-01-01 1970-01-01 03:00:00 1970-01-01 +1970-01-02 1970-01-01 03:00:01 1970-01-02 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(UInt64, String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) +nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +RowBinaryWithNamesAndTypes +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Date +datetime DateTime +date32 Date32 +1970-01-01 1970-01-01 03:00:00 1970-01-01 +1970-01-02 1970-01-01 03:00:01 1970-01-02 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(UInt64, String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) +nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +CustomSeparatedWithNamesAndTypes +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Date +datetime DateTime +date32 Date32 +1970-01-01 1970-01-01 03:00:00 1970-01-01 +1970-01-02 1970-01-01 03:00:01 1970-01-02 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(UInt64, String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) +nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +Avro +CustomSeparatedWithNamesAndTypes +int8 Int32 +uint8 Int32 +int16 Int32 +uint16 Int32 +int32 Int32 +uint32 Int32 +int64 Int64 +uint64 Int64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +0 0 +1.2 0.7692307692307692 +date Int32 +datetime Int32 +0 0 +1 1 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(Int64) +nested Array(Array(Array(Int64))) +[0,1] [[[0],[1]]] +[1,2] [[[1],[2]]] diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh new file mode 100755 index 00000000000..14572fd904c --- /dev/null +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh @@ -0,0 +1,65 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +FILE_NAME=test_02149.data +DATA_FILE=$USER_FILES_PATH/$FILE_NAME + +for format in Arrow ArrowStream Parquet ORC Native TSVWithNamesAndTypes TSVRawWithNamesAndTypes CSVWithNamesAndTypes JSONCompactEachRowWithNamesAndTypes JSONCompactStringsEachRowWithNamesAndTypes RowBinaryWithNamesAndTypes CustomSeparatedWithNamesAndTypes +do + echo $format + $CLICKHOUSE_CLIENT -q "select toInt8(-number) as int8, toUInt8(number) as uint8, toInt16(-number) as int16, toUInt16(number) as uint16, toInt32(-number) as int32, toUInt32(number) as uint32, toInt64(-number) as int64, toUInt64(number) as uint64 from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', '$format')" + $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', '$format')" + + $CLICKHOUSE_CLIENT -q "select toFloat32(number * 1.2) as float32, toFloat64(number / 1.3) as float64, toDecimal32(number / 0.3, 5) as decimal32, toDecimal64(number / 0.003, 5) as decimal64 from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', '$format')" + $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', '$format')" + + $CLICKHOUSE_CLIENT -q "select toDate(number) as date, toDateTime(number) as datetime, toDate32(number) as date32 from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', '$format')" + $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', '$format')" + + $CLICKHOUSE_CLIENT -q "select concat('Str: ', toString(number)) as str, toFixedString(toString((number + 1) * 100 % 1000), 3) as fixed_string from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', '$format')" + $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', '$format')" + + $CLICKHOUSE_CLIENT -q "select [number, number + 1] as array, (number, toString(number)) as tuple, map(toString(number), number) as map from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', '$format')" + $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', '$format')" + + $CLICKHOUSE_CLIENT -q "select [([number, number + 1], map('42', number)), ([], map()), ([42], map('42', 42))] as nested1, (([[number], [number + 1], []], map(number, [(number, '42'), (number + 1, '42')])), 42) as nested2 from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', '$format')" + $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', '$format')" +done + +echo "Avro" + +echo $format +$CLICKHOUSE_CLIENT -q "select toInt8(-number) as int8, toUInt8(number) as uint8, toInt16(-number) as int16, toUInt16(number) as uint16, toInt32(-number) as int32, toUInt32(number) as uint32, toInt64(-number) as int64, toUInt64(number) as uint64 from numbers(2) format Avro" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Avro')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Avro')" + +$CLICKHOUSE_CLIENT -q "select toFloat32(number * 1.2) as float32, toFloat64(number / 1.3) as float64 from numbers(2) format Avro" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Avro')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Avro')" + +$CLICKHOUSE_CLIENT -q "select toDate(number) as date, toDateTime(number) as datetime from numbers(2) format Avro" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Avro')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Avro')" + +$CLICKHOUSE_CLIENT -q "select concat('Str: ', toString(number)) as str, toFixedString(toString((number + 1) * 100 % 1000), 3) as fixed_string from numbers(2) format Avro" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Avro')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Avro')" + +$CLICKHOUSE_CLIENT -q "select [number, number + 1] as array, [[[number], [number + 1]]] as nested from numbers(2) format Avro" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Avro')" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Avro')" + +rm $DATA_FILE + diff --git a/tests/queries/0_stateless/format_schemas/00825_protobuf_format_enum_mapping.proto b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_enum_mapping.proto index ba558dbbadb..048a689d021 100644 --- a/tests/queries/0_stateless/format_schemas/00825_protobuf_format_enum_mapping.proto +++ b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_enum_mapping.proto @@ -1,6 +1,6 @@ syntax = "proto3"; -message Message +message EnumMessage { enum Enum { @@ -10,4 +10,4 @@ message Message HUNDRED = 100; }; Enum x = 1; -}; \ No newline at end of file +}; From aaf9f85c67706996f656292a3cf1fc4656277ccb Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 17 Dec 2021 18:34:13 +0300 Subject: [PATCH 0554/1260] Add more tests and fixes --- src/Formats/ReadSchemaUtils.cpp | 2 +- src/Formats/ReadSchemaUtils.h | 2 +- .../Formats/Impl/MsgPackRowInputFormat.cpp | 7 ++- src/Storages/StorageS3.cpp | 5 +- tests/integration/test_storage_hdfs/test.py | 53 ++++++++++++------- tests/integration/test_storage_s3/test.py | 20 +++++++ .../0_stateless/00646_url_engine.python | 23 ++++++-- .../queries/0_stateless/01903_http_fields.sh | 3 -- .../02149_schema_inference.reference | 10 ++++ .../0_stateless/02149_schema_inference.sh | 13 ++++- ...49_schema_inference_create_table_syntax.sh | 2 +- ...49_schema_inference_formats_with_schema.sh | 2 +- tests/queries/0_stateless/tmp.sh | 23 ++++++++ 13 files changed, 128 insertions(+), 37 deletions(-) create mode 100755 tests/queries/0_stateless/tmp.sh diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index 967cc3de54e..37067eae64f 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -94,7 +94,7 @@ DataTypePtr generalizeDataType(DataTypePtr type) if (which.isMap()) { const auto * map_type = assert_cast(type.get()); - auto key_type = generalizeDataType(map_type->getKeyType()); + auto key_type = removeNullable(generalizeDataType(map_type->getKeyType())); auto value_type = generalizeDataType(map_type->getValueType()); return key_type && value_type ? std::make_shared(key_type, value_type) : nullptr; } diff --git a/src/Formats/ReadSchemaUtils.h b/src/Formats/ReadSchemaUtils.h index 3fba6ad9f70..fb43acc3cd6 100644 --- a/src/Formats/ReadSchemaUtils.h +++ b/src/Formats/ReadSchemaUtils.h @@ -22,7 +22,7 @@ ColumnsDescription readSchemaFromFormat(const String & format_name, const std::o /// - Type -> Nullable(type) /// - Array(Type) -> Array(Nullable(Type)) /// - Tuple(Type1, ..., TypeN) -> Tuple(Nullable(Type1), ..., Nullable(TypeN)) -/// - Map(KeyType, ValueType) -> Map(Nullable(KeyType), Nullable(ValueType)) +/// - Map(KeyType, ValueType) -> Map(KeyType, Nullable(ValueType)) /// - LowCardinality(Type) -> LowCardinality(Nullable(Type)) /// If type is Nothing or one of the nested types is Nothing, return nullptr. DataTypePtr generalizeDataType(DataTypePtr type); diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index e4f727857ba..8b0c3d2eea4 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -418,8 +418,7 @@ DataTypePtr MsgPackSchemaReader::getDataType(const msgpack::object & object) { switch (object.type) { - case msgpack::type::object_type::POSITIVE_INTEGER: - return makeNullable(std::make_shared()); + case msgpack::type::object_type::POSITIVE_INTEGER: [[fallthrough]]; case msgpack::type::object_type::NEGATIVE_INTEGER: return makeNullable(std::make_shared()); case msgpack::type::object_type::FLOAT32: @@ -447,8 +446,8 @@ DataTypePtr MsgPackSchemaReader::getDataType(const msgpack::object & object) msgpack::object_map object_map = object.via.map; if (object_map.size) { - auto key_type = getDataType(object_map.ptr[0].key); - auto value_type = getDataType(object_map.ptr[1].val); + auto key_type = removeNullable(getDataType(object_map.ptr[0].key)); + auto value_type = getDataType(object_map.ptr[0].val); if (key_type && value_type) return std::make_shared(key_type, value_type); } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 48609dda029..3d988472b54 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -3,7 +3,6 @@ #if USE_AWS_S3 -#include #include #include @@ -25,7 +24,6 @@ #include #include -#include #include #include @@ -757,7 +755,7 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( format); return wrapReadBufferWithCompressionMethod( - std::make_unique(client_auth.client, client_auth.uri.bucket, current_key, max_single_read_retries, ctx->getReadSettings(), DBMS_DEFAULT_BUFFER_SIZE), + std::make_unique(client_auth.client, client_auth.uri.bucket, current_key, max_single_read_retries, ctx->getReadSettings()), chooseCompressionMethod(current_key, compression_method)); }; @@ -832,6 +830,7 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) { .supports_settings = true, .supports_sort_order = true, // for partition by + .supports_schema_inference = true, .source_access_type = AccessType::S3, }); } diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 33ce94a7a29..52c0d9dc3a4 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -16,7 +16,7 @@ def started_cluster(): finally: cluster.shutdown() -def test_read_write_storage(started_cluster): +def _test_read_write_storage(started_cluster): hdfs_api = started_cluster.hdfs_api node1.query("drop table if exists SimpleHDFSStorage SYNC") node1.query( @@ -26,7 +26,7 @@ def test_read_write_storage(started_cluster): assert node1.query("select * from SimpleHDFSStorage") == "1\tMark\t72.53\n" -def test_read_write_storage_with_globs(started_cluster): +def _test_read_write_storage_with_globs(started_cluster): hdfs_api = started_cluster.hdfs_api node1.query( @@ -69,7 +69,7 @@ def test_read_write_storage_with_globs(started_cluster): assert "in readonly mode" in str(ex) -def test_read_write_table(started_cluster): +def _test_read_write_table(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -82,7 +82,7 @@ def test_read_write_table(started_cluster): "select * from hdfs('hdfs://hdfs1:9000/simple_table_function', 'TSV', 'id UInt64, text String, number Float64')") == data -def test_write_table(started_cluster): +def _test_write_table(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -95,7 +95,7 @@ def test_write_table(started_cluster): assert node1.query("select * from OtherHDFSStorage order by id") == result -def test_bad_hdfs_uri(started_cluster): +def _test_bad_hdfs_uri(started_cluster): try: node1.query( "create table BadStorage1 (id UInt32, name String, weight Float64) ENGINE = HDFS('hads:hgsdfs100500:9000/other_storage', 'TSV')") @@ -117,7 +117,7 @@ def test_bad_hdfs_uri(started_cluster): assert "Unable to open HDFS file" in str(ex) @pytest.mark.timeout(800) -def test_globs_in_read_table(started_cluster): +def _test_globs_in_read_table(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -151,7 +151,7 @@ def test_globs_in_read_table(started_cluster): files_amount) -def test_read_write_gzip_table(started_cluster): +def _test_read_write_gzip_table(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -164,7 +164,7 @@ def test_read_write_gzip_table(started_cluster): "select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64')") == data -def test_read_write_gzip_table_with_parameter_gzip(started_cluster): +def _test_read_write_gzip_table_with_parameter_gzip(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -177,7 +177,7 @@ def test_read_write_gzip_table_with_parameter_gzip(started_cluster): "select * from hdfs('hdfs://hdfs1:9000/simple_table_function', 'TSV', 'id UInt64, text String, number Float64', 'gzip')") == data -def test_read_write_table_with_parameter_none(started_cluster): +def _test_read_write_table_with_parameter_none(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -190,7 +190,7 @@ def test_read_write_table_with_parameter_none(started_cluster): "select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64', 'none')") == data -def test_read_write_gzip_table_with_parameter_auto_gz(started_cluster): +def _test_read_write_gzip_table_with_parameter_auto_gz(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -203,7 +203,7 @@ def test_read_write_gzip_table_with_parameter_auto_gz(started_cluster): "select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64', 'auto')") == data -def test_write_gz_storage(started_cluster): +def _test_write_gz_storage(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -214,7 +214,7 @@ def test_write_gz_storage(started_cluster): assert node1.query("select * from GZHDFSStorage") == "1\tMark\t72.53\n" -def test_write_gzip_storage(started_cluster): +def _test_write_gzip_storage(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -225,7 +225,7 @@ def test_write_gzip_storage(started_cluster): assert node1.query("select * from GZIPHDFSStorage") == "1\tMark\t72.53\n" -def test_virtual_columns(started_cluster): +def _test_virtual_columns(started_cluster): hdfs_api = started_cluster.hdfs_api node1.query("create table virtual_cols (id UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/file*', 'TSV')") @@ -236,7 +236,7 @@ def test_virtual_columns(started_cluster): assert node1.query("select id, _file as file_name, _path as file_path from virtual_cols order by id") == expected -def test_read_files_with_spaces(started_cluster): +def _test_read_files_with_spaces(started_cluster): hdfs_api = started_cluster.hdfs_api fs = HdfsClient(hosts=started_cluster.hdfs_ip) @@ -256,7 +256,7 @@ def test_read_files_with_spaces(started_cluster): -def test_truncate_table(started_cluster): +def _test_truncate_table(started_cluster): hdfs_api = started_cluster.hdfs_api node1.query( "create table test_truncate (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/tr', 'TSV')") @@ -268,7 +268,7 @@ def test_truncate_table(started_cluster): node1.query("drop table test_truncate") -def test_partition_by(started_cluster): +def _test_partition_by(started_cluster): hdfs_api = started_cluster.hdfs_api table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -296,7 +296,7 @@ def test_partition_by(started_cluster): assert(result.strip() == "1\t2\t3") -def test_seekable_formats(started_cluster): +def _test_seekable_formats(started_cluster): hdfs_api = started_cluster.hdfs_api table_function = f"hdfs('hdfs://hdfs1:9000/parquet', 'Parquet', 'a Int32, b String')" @@ -310,7 +310,8 @@ def test_seekable_formats(started_cluster): result = node1.query(f"SELECT count() FROM {table_function}") assert(int(result) == 5000000) -def test_read_table_with_default(started_cluster): + +def _test_read_table_with_default(started_cluster): hdfs_api = started_cluster.hdfs_api data = "n\n100\n" @@ -322,6 +323,22 @@ def test_read_table_with_default(started_cluster): "select * from hdfs('hdfs://hdfs1:9000/simple_table_function', 'TSVWithNames', 'n UInt32, m UInt32 DEFAULT n * 2') FORMAT TSVWithNames") == output +def test_schema_inference(started_cluster): + node1.query(f"insert into table function hdfs('hdfs://hdfs1:9000/parquet', 'Parquet', 'a Int32, b String') SELECT number, randomString(100) FROM numbers(5000000)") + + result = node1.query(f"desc hdfs('hdfs://hdfs1:9000/parquet', 'Parquet')") + assert result == "a\tInt32\t\t\t\t\t\nb\tString\t\t\t\t\t\n" + + result = node1.query(f"select count(*) from hdfs('hdfs://hdfs1:9000/parquet', 'Parquet')") + assert(int(result) == 5000000) + + node1.query(f"create table schema_inference engine=HDFS('hdfs://hdfs1:9000/parquet', 'Parquet')") + result = node1.query(f"desc schema_inference") + assert result == "a\tInt32\t\t\t\t\t\nb\tString\t\t\t\t\t\n" + + result = node1.query(f"select count(*) from schema_inference") + assert(int(result) == 5000000) + def test_hdfsCluster(started_cluster): hdfs_api = started_cluster.hdfs_api diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index f3c4b1dd0cf..17482447222 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -865,3 +865,23 @@ def test_insert_with_path_with_globs(started_cluster): table_function_3 = f"s3('http://minio1:9001/root/test_parquet*', 'minio', 'minio123', 'Parquet', 'a Int32, b String')" instance.query_and_get_error(f"insert into table function {table_function_3} SELECT number, randomString(100) FROM numbers(500)") + + +def test_s3_schema_inference(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] + + instance.query(f"insert into table function s3(s3_parquet, structure='a Int32, b String', format='Parquet') select number, randomString(100) from numbers(5000000)") + result = instance.query(f"desc s3(s3_parquet, format='Parquet')") + assert result == "a\tInt32\t\t\t\t\t\nb\tString\t\t\t\t\t\n" + + result = instance.query(f"select count(*) from s3(s3_parquet, format='Parquet')") + assert(int(result) == 5000000) + + instance.query(f"create table schema_inference engine=S3(s3_parquet, format='Parquet')") + result = instance.query(f"desc schema_inference") + assert result == "a\tInt32\t\t\t\t\t\nb\tString\t\t\t\t\t\n" + + result = instance.query(f"select count(*) from schema_inference") + assert(int(result) == 5000000) + diff --git a/tests/queries/0_stateless/00646_url_engine.python b/tests/queries/0_stateless/00646_url_engine.python index 85ae3e776ed..ee67f7063e7 100644 --- a/tests/queries/0_stateless/00646_url_engine.python +++ b/tests/queries/0_stateless/00646_url_engine.python @@ -145,17 +145,24 @@ def test_select(table_name="", schema="str String,numuint UInt32,numint Int32,do if table_name: get_ch_answer("drop table if exists {}".format(table_name)) - get_ch_answer("create table {} ({}) engine=URL('{}', 'CSV')".format(table_name, schema, HTTP_SERVER_URL_STR)) + if schema is None: + get_ch_answer("create table {} engine=URL('{}', 'CSV')".format(table_name, HTTP_SERVER_URL_STR)) + else: + get_ch_answer("create table {} ({}) engine=URL('{}', 'CSV')".format(table_name, schema, HTTP_SERVER_URL_STR)) for i in range(len(requests)): tbl = table_name if not tbl: - tbl = "url('{addr}', 'CSV', '{schema}')".format(addr=HTTP_SERVER_URL_STR, schema=schema) + if schema is None: + tbl = "url('{addr}', 'CSV')".format(addr=HTTP_SERVER_URL_STR) + else: + tbl = "url('{addr}', 'CSV', '{schema}')".format(addr=HTTP_SERVER_URL_STR, schema=schema) check_answers(requests[i].format(tbl=tbl), answers[i]) if table_name: get_ch_answer("drop table if exists {}".format(table_name)) + def test_insert(table_name="", schema="str String,numuint UInt32,numint Int32,double Float64", requests_insert=[], requests_select=[], answers=[]): with open(CSV_DATA, 'w') as f: # flush test file f.write('') @@ -189,6 +196,12 @@ def main(): "select str,numuint,numint,double from {tbl} limit 1": test_data.split("\n")[0].replace(',', '\t'), } + select_only_requests_without_schema = { + "select * from {tbl}" : test_data.replace(',', '\t'), + "select column_2, count(*) from {tbl} group by column_2" : "2\t2", + "select * from {tbl} limit 1": test_data.split("\n")[0].replace(',', '\t'), + } + insert_requests = [ "insert into {tbl} values('Hello',10,-2,7.7)('World',10,-5,7.7)", "insert into {tbl} select 'Buy', number, 9-number, 9.9 from system.numbers limit 10", @@ -200,12 +213,16 @@ def main(): 'select double, count(*) from {tbl} group by double': "7.7\t2\n9.9\t10" } - t = start_server(len(select_only_requests) * 2 + (len(insert_requests) + len(select_requests)) * 2) + t = start_server(len(select_only_requests) * 2 + len(select_only_requests_without_schema) * 2 + 4 + (len(insert_requests) + len(select_requests)) * 2) t.start() # test table with url engine test_select(table_name="test_table_select", requests=list(select_only_requests.keys()), answers=list(select_only_requests.values()), test_data=test_data) # test table function url test_select(requests=list(select_only_requests.keys()), answers=list(select_only_requests.values()), test_data=test_data) + # test table with url engine without schema + test_select(table_name="test_table_select", schema=None, requests=list(select_only_requests_without_schema.keys()), answers=list(select_only_requests_without_schema.values()), test_data=test_data) + # test table function url without schema + test_select(schema=None, requests=list(select_only_requests_without_schema.keys()), answers=list(select_only_requests_without_schema.values()), test_data=test_data) #test insert into table with url engine test_insert(table_name="test_table_insert", requests_insert=insert_requests, requests_select=list(select_requests.keys()), answers=list(select_requests.values())) #test insert into table function url diff --git a/tests/queries/0_stateless/01903_http_fields.sh b/tests/queries/0_stateless/01903_http_fields.sh index 3dab080c823..0f2b0df13d9 100755 --- a/tests/queries/0_stateless/01903_http_fields.sh +++ b/tests/queries/0_stateless/01903_http_fields.sh @@ -4,9 +4,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -echo $CLICKHOUSE_CURL -echo $CLICKHOUSE_URL - DEFAULT_MAX_NAME_SIZE=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.settings WHERE name='http_max_field_name_size'") DEFAULT_MAX_VALUE_SIZE=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.settings WHERE name='http_max_field_value_size'") diff --git a/tests/queries/0_stateless/02149_schema_inference.reference b/tests/queries/0_stateless/02149_schema_inference.reference index 1bcd4642f26..77f1ab707f5 100644 --- a/tests/queries/0_stateless/02149_schema_inference.reference +++ b/tests/queries/0_stateless/02149_schema_inference.reference @@ -158,3 +158,13 @@ column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) 42.42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] 42 Some string 2 [([],'String 2'),([],'String 2')] \N Some string 3 [([1,2,3],'String 3'),([1],'String 3')] +MsgPack +column_1 Nullable(Int64) +column_2 Nullable(Int64) +column_3 Nullable(Float32) +column_4 Nullable(String) +column_5 Array(Array(Nullable(Int64))) +column_6 Map(Int64, Array(Nullable(Int64))) +\N 0 0 Str: 0 [[0,1],[0]] {0:[0,1]} +1 \N 1 Str: 1 [[1,2],[1]] {1:[1,2]} +\N 2 2 Str: 2 [[2,3],[2]] {2:[2,3]} diff --git a/tests/queries/0_stateless/02149_schema_inference.sh b/tests/queries/0_stateless/02149_schema_inference.sh index cc6c8d7e276..f146bc20ac9 100755 --- a/tests/queries/0_stateless/02149_schema_inference.sh +++ b/tests/queries/0_stateless/02149_schema_inference.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -12,7 +12,7 @@ DATA_FILE=$USER_FILES_PATH/$FILE_NAME touch $DATA_FILE -SCHEMADIR=$(clickhouse-client --query "select * from file('$FILE_NAME', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") +SCHEMADIR=$(clickhouse-client --query "select * from file('$FILE_NAME', 'Template', 'val1 char') settings format_template_row='nonexist'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist)") echo "TSV" @@ -237,6 +237,15 @@ echo -e " $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Template') $TEMPLATE_SETTINGS" $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Template') $TEMPLATE_SETTINGS" + +echo "MsgPack" + +$CLICKHOUSE_CLIENT -q "select toInt32(number % 2 ? number : NULL) as int, toUInt64(number % 2 ? NULL : number) as uint, toFloat32(number) as float, concat('Str: ', toString(number)) as str, [[number, number + 1], [number]] as arr, map(number, [number, number + 1]) as map from numbers(3) format MsgPack" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'MsgPack') settings input_format_msgpack_number_of_columns=6" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'MsgPack') settings input_format_msgpack_number_of_columns=6" + + rm $SCHEMADIR/resultset_format_02149 $SCHEMADIR/row_format_02149 rm $DATA_FILE diff --git a/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh b/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh index fa05e40ab5a..fa675bc9676 100755 --- a/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh +++ b/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh index 14572fd904c..1ae4c764cb5 100755 --- a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/tmp.sh b/tests/queries/0_stateless/tmp.sh new file mode 100755 index 00000000000..def77c9b3f8 --- /dev/null +++ b/tests/queries/0_stateless/tmp.sh @@ -0,0 +1,23 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +FILE_NAME=test_02149.data +DATA_FILE=$USER_FILES_PATH/$FILE_NAME + +echo "MsgPack" + +$CLICKHOUSE_CLIENT -q "select toInt32(number % 2 ? number : NULL) as int, toUInt64(number % 2 ? NULL : number) as uint, toFloat32(number) as float, concat('Str: ', toString(number)) as str, [[number, number + 1], [number]] as arr, map(number, [number, number + 1]) as map from numbers(3) format MsgPack" > $DATA_FILE + +$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'MsgPack') settings input_format_msgpack_number_of_columns=6" +$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'MsgPack') settings input_format_msgpack_number_of_columns=6" + + +rm $SCHEMADIR/resultset_format_02149 $SCHEMADIR/row_format_02149 +rm $DATA_FILE + From 3d38e466b1a5d12da7b630ef979460fff4c7a59c Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 17 Dec 2021 19:25:27 +0300 Subject: [PATCH 0555/1260] Fix tests --- .../02149_external_schema_inference.sh | 2 +- tests/queries/0_stateless/tmp.sh | 23 ------------------- 2 files changed, 1 insertion(+), 24 deletions(-) delete mode 100755 tests/queries/0_stateless/tmp.sh diff --git a/tests/queries/0_stateless/02149_external_schema_inference.sh b/tests/queries/0_stateless/02149_external_schema_inference.sh index 9c878236746..fc142bd4c6b 100755 --- a/tests/queries/0_stateless/02149_external_schema_inference.sh +++ b/tests/queries/0_stateless/02149_external_schema_inference.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/tmp.sh b/tests/queries/0_stateless/tmp.sh deleted file mode 100755 index def77c9b3f8..00000000000 --- a/tests/queries/0_stateless/tmp.sh +++ /dev/null @@ -1,23 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-parallel - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - - -USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -FILE_NAME=test_02149.data -DATA_FILE=$USER_FILES_PATH/$FILE_NAME - -echo "MsgPack" - -$CLICKHOUSE_CLIENT -q "select toInt32(number % 2 ? number : NULL) as int, toUInt64(number % 2 ? NULL : number) as uint, toFloat32(number) as float, concat('Str: ', toString(number)) as str, [[number, number + 1], [number]] as arr, map(number, [number, number + 1]) as map from numbers(3) format MsgPack" > $DATA_FILE - -$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'MsgPack') settings input_format_msgpack_number_of_columns=6" -$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'MsgPack') settings input_format_msgpack_number_of_columns=6" - - -rm $SCHEMADIR/resultset_format_02149 $SCHEMADIR/row_format_02149 -rm $DATA_FILE - From 74f09d64769e7404aabb377b819764b61cce2dce Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 20 Dec 2021 17:39:15 +0300 Subject: [PATCH 0556/1260] Fix tests --- .../Impl/TabSeparatedRowOutputFormat.cpp | 5 ++- tests/integration/test_storage_hdfs/test.py | 36 ++++++++-------- .../configs/named_collections.xml | 5 +++ tests/integration/test_storage_s3/test.py | 10 ++--- .../0_stateless/02149_schema_inference.sh | 2 +- ...ma_inference_create_table_syntax.reference | 10 ----- ...49_schema_inference_create_table_syntax.sh | 8 +--- ...ma_inference_formats_with_schema.reference | 42 +++++++++---------- ...49_schema_inference_formats_with_schema.sh | 2 +- 9 files changed, 56 insertions(+), 64 deletions(-) diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index 5d87f5a0b14..37bd8daa502 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -22,7 +22,10 @@ void TabSeparatedRowOutputFormat::writeLine(const std::vector & values) { for (size_t i = 0; i < values.size(); ++i) { - writeEscapedString(values[i], out); + if (is_raw) + writeString(values[i], out); + else + writeEscapedString(values[i], out); if (i + 1 == values.size()) writeRowEndDelimiter(); else diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 52c0d9dc3a4..bd9e572f4bd 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -16,7 +16,7 @@ def started_cluster(): finally: cluster.shutdown() -def _test_read_write_storage(started_cluster): +def test_read_write_storage(started_cluster): hdfs_api = started_cluster.hdfs_api node1.query("drop table if exists SimpleHDFSStorage SYNC") node1.query( @@ -26,7 +26,7 @@ def _test_read_write_storage(started_cluster): assert node1.query("select * from SimpleHDFSStorage") == "1\tMark\t72.53\n" -def _test_read_write_storage_with_globs(started_cluster): +def test_read_write_storage_with_globs(started_cluster): hdfs_api = started_cluster.hdfs_api node1.query( @@ -69,7 +69,7 @@ def _test_read_write_storage_with_globs(started_cluster): assert "in readonly mode" in str(ex) -def _test_read_write_table(started_cluster): +def test_read_write_table(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -82,7 +82,7 @@ def _test_read_write_table(started_cluster): "select * from hdfs('hdfs://hdfs1:9000/simple_table_function', 'TSV', 'id UInt64, text String, number Float64')") == data -def _test_write_table(started_cluster): +def test_write_table(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -95,7 +95,7 @@ def _test_write_table(started_cluster): assert node1.query("select * from OtherHDFSStorage order by id") == result -def _test_bad_hdfs_uri(started_cluster): +def test_bad_hdfs_uri(started_cluster): try: node1.query( "create table BadStorage1 (id UInt32, name String, weight Float64) ENGINE = HDFS('hads:hgsdfs100500:9000/other_storage', 'TSV')") @@ -117,7 +117,7 @@ def _test_bad_hdfs_uri(started_cluster): assert "Unable to open HDFS file" in str(ex) @pytest.mark.timeout(800) -def _test_globs_in_read_table(started_cluster): +def test_globs_in_read_table(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -151,7 +151,7 @@ def _test_globs_in_read_table(started_cluster): files_amount) -def _test_read_write_gzip_table(started_cluster): +def test_read_write_gzip_table(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -164,7 +164,7 @@ def _test_read_write_gzip_table(started_cluster): "select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64')") == data -def _test_read_write_gzip_table_with_parameter_gzip(started_cluster): +def test_read_write_gzip_table_with_parameter_gzip(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -177,7 +177,7 @@ def _test_read_write_gzip_table_with_parameter_gzip(started_cluster): "select * from hdfs('hdfs://hdfs1:9000/simple_table_function', 'TSV', 'id UInt64, text String, number Float64', 'gzip')") == data -def _test_read_write_table_with_parameter_none(started_cluster): +def test_read_write_table_with_parameter_none(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -190,7 +190,7 @@ def _test_read_write_table_with_parameter_none(started_cluster): "select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64', 'none')") == data -def _test_read_write_gzip_table_with_parameter_auto_gz(started_cluster): +def test_read_write_gzip_table_with_parameter_auto_gz(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -203,7 +203,7 @@ def _test_read_write_gzip_table_with_parameter_auto_gz(started_cluster): "select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64', 'auto')") == data -def _test_write_gz_storage(started_cluster): +def test_write_gz_storage(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -214,7 +214,7 @@ def _test_write_gz_storage(started_cluster): assert node1.query("select * from GZHDFSStorage") == "1\tMark\t72.53\n" -def _test_write_gzip_storage(started_cluster): +def test_write_gzip_storage(started_cluster): hdfs_api = started_cluster.hdfs_api @@ -225,7 +225,7 @@ def _test_write_gzip_storage(started_cluster): assert node1.query("select * from GZIPHDFSStorage") == "1\tMark\t72.53\n" -def _test_virtual_columns(started_cluster): +def test_virtual_columns(started_cluster): hdfs_api = started_cluster.hdfs_api node1.query("create table virtual_cols (id UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/file*', 'TSV')") @@ -236,7 +236,7 @@ def _test_virtual_columns(started_cluster): assert node1.query("select id, _file as file_name, _path as file_path from virtual_cols order by id") == expected -def _test_read_files_with_spaces(started_cluster): +def test_read_files_with_spaces(started_cluster): hdfs_api = started_cluster.hdfs_api fs = HdfsClient(hosts=started_cluster.hdfs_ip) @@ -256,7 +256,7 @@ def _test_read_files_with_spaces(started_cluster): -def _test_truncate_table(started_cluster): +def test_truncate_table(started_cluster): hdfs_api = started_cluster.hdfs_api node1.query( "create table test_truncate (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/tr', 'TSV')") @@ -268,7 +268,7 @@ def _test_truncate_table(started_cluster): node1.query("drop table test_truncate") -def _test_partition_by(started_cluster): +def test_partition_by(started_cluster): hdfs_api = started_cluster.hdfs_api table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -296,7 +296,7 @@ def _test_partition_by(started_cluster): assert(result.strip() == "1\t2\t3") -def _test_seekable_formats(started_cluster): +def test_seekable_formats(started_cluster): hdfs_api = started_cluster.hdfs_api table_function = f"hdfs('hdfs://hdfs1:9000/parquet', 'Parquet', 'a Int32, b String')" @@ -311,7 +311,7 @@ def _test_seekable_formats(started_cluster): assert(int(result) == 5000000) -def _test_read_table_with_default(started_cluster): +def test_read_table_with_default(started_cluster): hdfs_api = started_cluster.hdfs_api data = "n\n100\n" diff --git a/tests/integration/test_storage_s3/configs/named_collections.xml b/tests/integration/test_storage_s3/configs/named_collections.xml index dfcbeeb2d4a..efadedc1bde 100644 --- a/tests/integration/test_storage_s3/configs/named_collections.xml +++ b/tests/integration/test_storage_s3/configs/named_collections.xml @@ -15,5 +15,10 @@ minio minio123 + + http://minio1:9001/root/test_native + minio + minio123 + diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 17482447222..04c6a19b0dc 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -815,7 +815,6 @@ def test_seekable_formats(started_cluster): result = instance.query(f"SELECT count() FROM {table_function}") assert(int(result) == 5000000) - instance.query("SYSTEM FLUSH LOGS") result = instance.query(f"SELECT formatReadableSize(memory_usage) FROM system.query_log WHERE startsWith(query, 'SELECT count() FROM s3') AND memory_usage > 0 ORDER BY event_time desc") print(result[:3]) assert(int(result[:3]) < 200) @@ -839,7 +838,6 @@ def test_seekable_formats_url(started_cluster): result = instance.query(f"SELECT count() FROM {table_function}") assert(int(result) == 5000000) - instance.query("SYSTEM FLUSH LOGS") result = instance.query(f"SELECT formatReadableSize(memory_usage) FROM system.query_log WHERE startsWith(query, 'SELECT count() FROM url') AND memory_usage > 0 ORDER BY event_time desc") print(result[:3]) assert(int(result[:3]) < 200) @@ -871,14 +869,14 @@ def test_s3_schema_inference(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] - instance.query(f"insert into table function s3(s3_parquet, structure='a Int32, b String', format='Parquet') select number, randomString(100) from numbers(5000000)") - result = instance.query(f"desc s3(s3_parquet, format='Parquet')") + instance.query(f"insert into table function s3(s3_native, structure='a Int32, b String', format='Native') select number, randomString(100) from numbers(5000000)") + result = instance.query(f"desc s3(s3_native, format='Native')") assert result == "a\tInt32\t\t\t\t\t\nb\tString\t\t\t\t\t\n" - result = instance.query(f"select count(*) from s3(s3_parquet, format='Parquet')") + result = instance.query(f"select count(*) from s3(s3_native, format='Native')") assert(int(result) == 5000000) - instance.query(f"create table schema_inference engine=S3(s3_parquet, format='Parquet')") + instance.query(f"create table schema_inference engine=S3(s3_native, format='Native')") result = instance.query(f"desc schema_inference") assert result == "a\tInt32\t\t\t\t\t\nb\tString\t\t\t\t\t\n" diff --git a/tests/queries/0_stateless/02149_schema_inference.sh b/tests/queries/0_stateless/02149_schema_inference.sh index f146bc20ac9..1ccec240627 100755 --- a/tests/queries/0_stateless/02149_schema_inference.sh +++ b/tests/queries/0_stateless/02149_schema_inference.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') FILE_NAME=test_02149.data -DATA_FILE=$USER_FILES_PATH/$FILE_NAME +DATA_FILE=${USER_FILES_PATH:?}/$FILE_NAME touch $DATA_FILE diff --git a/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.reference b/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.reference index fd2f8adf54f..dae12318ce0 100644 --- a/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.reference +++ b/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.reference @@ -8,16 +8,6 @@ 7 Str: 7 [7,8] 8 Str: 8 [8,9] 9 Str: 9 [9,10] -0 Str: 0 [0,1] -1 Str: 1 [1,2] -2 Str: 2 [2,3] -3 Str: 3 [3,4] -4 Str: 4 [4,5] -5 Str: 5 [5,6] -6 Str: 6 [6,7] -7 Str: 7 [7,8] -8 Str: 8 [8,9] -9 Str: 9 [9,10] 0 0 [0,1] 1 1 [1,2] 2 2 [2,3] diff --git a/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh b/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh index fa675bc9676..7b15ef59eab 100755 --- a/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh +++ b/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh @@ -11,17 +11,14 @@ mkdir $USER_FILES_PATH/test_02149 FILE_NAME=test_02149/data.Parquet DATA_FILE=$USER_FILES_PATH/$FILE_NAME -$CLICKHOUSE_CLIENT -q "select number as num, concat('Str: ', toString(number)) as str, [number, number + 1] as arr from numbers(10) format Parquet" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "insert into table function file('$FILE_NAME', 'Parquet', 'num UInt64, str String, arr Array(UInt64)') select number as num, concat('Str: ', toString(number)) as str, [number, number + 1] as arr from numbers(10)" + $CLICKHOUSE_CLIENT -q "drop table if exists test_02149" $CLICKHOUSE_CLIENT -q "create table test_02149 engine=File('Parquet', '$FILE_NAME')" $CLICKHOUSE_CLIENT -q "select * from test_02149" $CLICKHOUSE_CLIENT -q "drop table test_02149" -$CLICKHOUSE_CLIENT -q "attach table test_02149 from 'test_02149' engine=File('Parquet')" -$CLICKHOUSE_CLIENT -q "select * from test_02149" -$CLICKHOUSE_CLIENT -q "drop table test_02149" - $CLICKHOUSE_CLIENT -q "create table test_02149 (x UInt32, s String, a Array(UInt32)) engine=Memory" $CLICKHOUSE_CLIENT -q "insert into test_02149 select number, toString(number), [number, number + 1] from numbers(10)" @@ -42,4 +39,3 @@ $CLICKHOUSE_CLIENT -q "drop table test_buffer" rm -rf $USER_FILES_PATH/test_02149 - diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference index 79308cc5a75..98c1ad43c80 100644 --- a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference @@ -191,10 +191,10 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date Date -datetime DateTime +datetime DateTime(\'UTC\') date32 Date32 -1970-01-01 1970-01-01 03:00:00 1970-01-01 -1970-01-02 1970-01-01 03:00:01 1970-01-02 +1970-01-01 1970-01-01 00:00:00 1970-01-01 +1970-01-02 1970-01-01 00:00:01 1970-01-02 str String fixed_string FixedString(3) Str: 0 100 @@ -226,10 +226,10 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date Date -datetime DateTime +datetime DateTime(\'UTC\') date32 Date32 -1970-01-01 1970-01-01 03:00:00 1970-01-01 -1970-01-02 1970-01-01 03:00:01 1970-01-02 +1970-01-01 1970-01-01 00:00:00 1970-01-01 +1970-01-02 1970-01-01 00:00:01 1970-01-02 str String fixed_string FixedString(3) Str: 0 100 @@ -261,10 +261,10 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date Date -datetime DateTime +datetime DateTime(\'UTC\') date32 Date32 -1970-01-01 1970-01-01 03:00:00 1970-01-01 -1970-01-02 1970-01-01 03:00:01 1970-01-02 +1970-01-01 1970-01-01 00:00:00 1970-01-01 +1970-01-02 1970-01-01 00:00:01 1970-01-02 str String fixed_string FixedString(3) Str: 0 100 @@ -296,10 +296,10 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date Date -datetime DateTime +datetime DateTime(\'UTC\') date32 Date32 -1970-01-01 1970-01-01 03:00:00 1970-01-01 -1970-01-02 1970-01-01 03:00:01 1970-01-02 +1970-01-01 1970-01-01 00:00:00 1970-01-01 +1970-01-02 1970-01-01 00:00:01 1970-01-02 str String fixed_string FixedString(3) Str: 0 100 @@ -331,10 +331,10 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date Date -datetime DateTime +datetime DateTime(\'UTC\') date32 Date32 -1970-01-01 1970-01-01 03:00:00 1970-01-01 -1970-01-02 1970-01-01 03:00:01 1970-01-02 +1970-01-01 1970-01-01 00:00:00 1970-01-01 +1970-01-02 1970-01-01 00:00:01 1970-01-02 str String fixed_string FixedString(3) Str: 0 100 @@ -366,10 +366,10 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date Date -datetime DateTime +datetime DateTime(\'UTC\') date32 Date32 -1970-01-01 1970-01-01 03:00:00 1970-01-01 -1970-01-02 1970-01-01 03:00:01 1970-01-02 +1970-01-01 1970-01-01 00:00:00 1970-01-01 +1970-01-02 1970-01-01 00:00:01 1970-01-02 str String fixed_string FixedString(3) Str: 0 100 @@ -401,10 +401,10 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date Date -datetime DateTime +datetime DateTime(\'UTC\') date32 Date32 -1970-01-01 1970-01-01 03:00:00 1970-01-01 -1970-01-02 1970-01-01 03:00:01 1970-01-02 +1970-01-01 1970-01-01 00:00:00 1970-01-01 +1970-01-02 1970-01-01 00:00:01 1970-01-02 str String fixed_string FixedString(3) Str: 0 100 diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh index 1ae4c764cb5..4863f8c949e 100755 --- a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh @@ -21,7 +21,7 @@ do $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', '$format')" $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', '$format')" - $CLICKHOUSE_CLIENT -q "select toDate(number) as date, toDateTime(number) as datetime, toDate32(number) as date32 from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_CLIENT -q "select toDate(number) as date, toDateTime(number, 'UTC') as datetime, toDate32(number) as date32 from numbers(2) format $format" > $DATA_FILE $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', '$format')" $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', '$format')" From 8a65c265e94ee33115606bdd6d16394731e63615 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 20 Dec 2021 17:51:28 +0300 Subject: [PATCH 0557/1260] Fix shellcheck --- tests/queries/0_stateless/02149_external_schema_inference.sh | 2 +- .../0_stateless/02149_schema_inference_create_table_syntax.sh | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02149_external_schema_inference.sh b/tests/queries/0_stateless/02149_external_schema_inference.sh index fc142bd4c6b..df2b9a43565 100755 --- a/tests/queries/0_stateless/02149_external_schema_inference.sh +++ b/tests/queries/0_stateless/02149_external_schema_inference.sh @@ -87,5 +87,5 @@ $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'JSONAsString')" -rm -rf $SCHEMADIR/$SERVER_SCHEMADIR +rm -rf ${SCHEMADIR:?}/$SERVER_SCHEMADIR rm $DATA_FILE diff --git a/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh b/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh index 7b15ef59eab..b2bf15f3ff9 100755 --- a/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh +++ b/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh @@ -9,7 +9,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') mkdir $USER_FILES_PATH/test_02149 FILE_NAME=test_02149/data.Parquet -DATA_FILE=$USER_FILES_PATH/$FILE_NAME $CLICKHOUSE_CLIENT -q "insert into table function file('$FILE_NAME', 'Parquet', 'num UInt64, str String, arr Array(UInt64)') select number as num, concat('Str: ', toString(number)) as str, [number, number + 1] as arr from numbers(10)" @@ -37,5 +36,5 @@ $CLICKHOUSE_CLIENT -q "create table test_buffer engine=Buffer(currentDatabase(), $CLICKHOUSE_CLIENT -q "select * from test_buffer" $CLICKHOUSE_CLIENT -q "drop table test_buffer" -rm -rf $USER_FILES_PATH/test_02149 +rm -rf ${USER_FILES_PATH:?}/test_02149 From 9f30c17777387a278ffcc5b479ec291c219683c8 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 21 Dec 2021 00:00:40 +0300 Subject: [PATCH 0558/1260] Fix tests --- tests/integration/test_storage_hdfs/test.py | 8 ++++---- .../02149_schema_inference_create_table_syntax.sh | 4 ++-- .../02149_schema_inference_formats_with_schema.sh | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index bd9e572f4bd..f317fb5429a 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -324,15 +324,15 @@ def test_read_table_with_default(started_cluster): def test_schema_inference(started_cluster): - node1.query(f"insert into table function hdfs('hdfs://hdfs1:9000/parquet', 'Parquet', 'a Int32, b String') SELECT number, randomString(100) FROM numbers(5000000)") + node1.query(f"insert into table function hdfs('hdfs://hdfs1:9000/native', 'Native', 'a Int32, b String') SELECT number, randomString(100) FROM numbers(5000000)") - result = node1.query(f"desc hdfs('hdfs://hdfs1:9000/parquet', 'Parquet')") + result = node1.query(f"desc hdfs('hdfs://hdfs1:9000/native', 'Native')") assert result == "a\tInt32\t\t\t\t\t\nb\tString\t\t\t\t\t\n" - result = node1.query(f"select count(*) from hdfs('hdfs://hdfs1:9000/parquet', 'Parquet')") + result = node1.query(f"select count(*) from hdfs('hdfs://hdfs1:9000/native', 'Native')") assert(int(result) == 5000000) - node1.query(f"create table schema_inference engine=HDFS('hdfs://hdfs1:9000/parquet', 'Parquet')") + node1.query(f"create table schema_inference engine=HDFS('hdfs://hdfs1:9000/native', 'Native')") result = node1.query(f"desc schema_inference") assert result == "a\tInt32\t\t\t\t\t\nb\tString\t\t\t\t\t\n" diff --git a/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh b/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh index b2bf15f3ff9..f00f2531dd0 100755 --- a/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh +++ b/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh @@ -9,9 +9,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') mkdir $USER_FILES_PATH/test_02149 FILE_NAME=test_02149/data.Parquet +DATA_FILE=$USER_FILES_PATH/$FILE_NAME -$CLICKHOUSE_CLIENT -q "insert into table function file('$FILE_NAME', 'Parquet', 'num UInt64, str String, arr Array(UInt64)') select number as num, concat('Str: ', toString(number)) as str, [number, number + 1] as arr from numbers(10)" - +$CLICKHOUSE_CLIENT -q "select number as num, concat('Str: ', toString(number)) as str, [number, number + 1] as arr from numbers(10) format Parquet" > $DATA_FILE $CLICKHOUSE_CLIENT -q "drop table if exists test_02149" $CLICKHOUSE_CLIENT -q "create table test_02149 engine=File('Parquet', '$FILE_NAME')" diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh index 4863f8c949e..68cfe3894f2 100755 --- a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh @@ -49,7 +49,7 @@ $CLICKHOUSE_CLIENT -q "select toFloat32(number * 1.2) as float32, toFloat64(numb $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Avro')" $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Avro')" -$CLICKHOUSE_CLIENT -q "select toDate(number) as date, toDateTime(number) as datetime from numbers(2) format Avro" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "select toDate(number) as date, toDateTime(number, 'UTC') as datetime from numbers(2) format Avro" > $DATA_FILE $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Avro')" $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Avro')" From 622a013b7b0c9baa63833db4d6befce524ce4691 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 21 Dec 2021 03:33:35 +0300 Subject: [PATCH 0559/1260] Fix tests --- ...ma_inference_formats_with_schema.reference | 65 ++++++++----------- ...49_schema_inference_formats_with_schema.sh | 4 +- 2 files changed, 28 insertions(+), 41 deletions(-) diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference index 98c1ad43c80..d3d2d86d696 100644 --- a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference @@ -16,10 +16,9 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date UInt16 -datetime UInt32 date32 Date32 -0 0 1970-01-01 -1 1 1970-01-02 +0 1970-01-01 +1 1970-01-02 str String fixed_string String Str: 0 100 @@ -51,10 +50,9 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date UInt16 -datetime UInt32 date32 Date32 -0 0 1970-01-01 -1 1 1970-01-02 +0 1970-01-01 +1 1970-01-02 str String fixed_string String Str: 0 100 @@ -86,10 +84,9 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date UInt16 -datetime Int64 date32 Date32 -0 0 1970-01-01 -1 1 1970-01-02 +0 1970-01-01 +1 1970-01-02 str String fixed_string String Str: 0 100 @@ -121,10 +118,9 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date Date32 -datetime UInt32 date32 Date32 -1970-01-01 0 1970-01-01 -1970-01-02 1 1970-01-02 +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 str String fixed_string String Str: 0 100 @@ -156,10 +152,9 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date Date -datetime DateTime date32 Date32 -1970-01-01 1970-01-01 03:00:00 1970-01-01 -1970-01-02 1970-01-01 03:00:01 1970-01-02 +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 str String fixed_string FixedString(3) Str: 0 100 @@ -191,10 +186,9 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date Date -datetime DateTime(\'UTC\') date32 Date32 -1970-01-01 1970-01-01 00:00:00 1970-01-01 -1970-01-02 1970-01-01 00:00:01 1970-01-02 +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 str String fixed_string FixedString(3) Str: 0 100 @@ -226,10 +220,9 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date Date -datetime DateTime(\'UTC\') date32 Date32 -1970-01-01 1970-01-01 00:00:00 1970-01-01 -1970-01-02 1970-01-01 00:00:01 1970-01-02 +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 str String fixed_string FixedString(3) Str: 0 100 @@ -261,10 +254,9 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date Date -datetime DateTime(\'UTC\') date32 Date32 -1970-01-01 1970-01-01 00:00:00 1970-01-01 -1970-01-02 1970-01-01 00:00:01 1970-01-02 +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 str String fixed_string FixedString(3) Str: 0 100 @@ -296,10 +288,9 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date Date -datetime DateTime(\'UTC\') date32 Date32 -1970-01-01 1970-01-01 00:00:00 1970-01-01 -1970-01-02 1970-01-01 00:00:01 1970-01-02 +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 str String fixed_string FixedString(3) Str: 0 100 @@ -331,10 +322,9 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date Date -datetime DateTime(\'UTC\') date32 Date32 -1970-01-01 1970-01-01 00:00:00 1970-01-01 -1970-01-02 1970-01-01 00:00:01 1970-01-02 +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 str String fixed_string FixedString(3) Str: 0 100 @@ -366,10 +356,9 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date Date -datetime DateTime(\'UTC\') date32 Date32 -1970-01-01 1970-01-01 00:00:00 1970-01-01 -1970-01-02 1970-01-01 00:00:01 1970-01-02 +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 str String fixed_string FixedString(3) Str: 0 100 @@ -401,10 +390,9 @@ decimal64 Decimal(18, 5) 0 0 0 0 1.2 0.7692307692307692 3.33333 333.33333 date Date -datetime DateTime(\'UTC\') date32 Date32 -1970-01-01 1970-01-01 00:00:00 1970-01-01 -1970-01-02 1970-01-01 00:00:01 1970-01-02 +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 str String fixed_string FixedString(3) Str: 0 100 @@ -435,9 +423,8 @@ float64 Float64 0 0 1.2 0.7692307692307692 date Int32 -datetime Int32 -0 0 -1 1 +0 +1 str String fixed_string FixedString(3) Str: 0 100 diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh index 68cfe3894f2..d263ef63681 100755 --- a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh @@ -21,7 +21,7 @@ do $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', '$format')" $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', '$format')" - $CLICKHOUSE_CLIENT -q "select toDate(number) as date, toDateTime(number, 'UTC') as datetime, toDate32(number) as date32 from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_CLIENT -q "select toDate(number) as date, toDate32(number) as date32 from numbers(2) format $format" > $DATA_FILE $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', '$format')" $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', '$format')" @@ -49,7 +49,7 @@ $CLICKHOUSE_CLIENT -q "select toFloat32(number * 1.2) as float32, toFloat64(numb $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Avro')" $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Avro')" -$CLICKHOUSE_CLIENT -q "select toDate(number) as date, toDateTime(number, 'UTC') as datetime from numbers(2) format Avro" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "select toDate(number) as date from numbers(2) format Avro" > $DATA_FILE $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Avro')" $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Avro')" From bc5f428f975d36cfeef41e1712caaebe6d96156a Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 21 Dec 2021 03:38:39 +0300 Subject: [PATCH 0560/1260] Fix tests --- tests/integration/test_storage_s3/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 04c6a19b0dc..3d85cc328b5 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -815,6 +815,7 @@ def test_seekable_formats(started_cluster): result = instance.query(f"SELECT count() FROM {table_function}") assert(int(result) == 5000000) + instance.query("SYSTEM FLUSH LOGS") result = instance.query(f"SELECT formatReadableSize(memory_usage) FROM system.query_log WHERE startsWith(query, 'SELECT count() FROM s3') AND memory_usage > 0 ORDER BY event_time desc") print(result[:3]) assert(int(result[:3]) < 200) @@ -838,6 +839,7 @@ def test_seekable_formats_url(started_cluster): result = instance.query(f"SELECT count() FROM {table_function}") assert(int(result) == 5000000) + instance.query("SYSTEM FLUSH LOGS") result = instance.query(f"SELECT formatReadableSize(memory_usage) FROM system.query_log WHERE startsWith(query, 'SELECT count() FROM url') AND memory_usage > 0 ORDER BY event_time desc") print(result[:3]) assert(int(result[:3]) < 200) From e3dbfe6bf649d18474d8cb05ff3942093b268153 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 23 Dec 2021 19:44:24 +0300 Subject: [PATCH 0561/1260] Better test url engine --- tests/integration/test_storage_s3/test.py | 88 +++++++++++++------ .../0_stateless/00646_url_engine.python | 22 +---- 2 files changed, 62 insertions(+), 48 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 3d85cc328b5..885a37f875c 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -126,7 +126,7 @@ def run_query(instance, query, stdin=None, settings=None): pytest.param("'wrongid','wrongkey',", False, 'xz', id="xz"), pytest.param("'wrongid','wrongkey',", False, 'zstd', id="zstd") ]) -def test_put(started_cluster, maybe_auth, positive, compression): +def _test_put(started_cluster, maybe_auth, positive, compression): # type: (ClickHouseCluster) -> None bucket = started_cluster.minio_bucket if not maybe_auth else started_cluster.minio_restricted_bucket @@ -148,7 +148,7 @@ def test_put(started_cluster, maybe_auth, positive, compression): assert values_csv == get_s3_file_content(started_cluster, bucket, filename) -def test_partition_by(started_cluster): +def _test_partition_by(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -173,7 +173,7 @@ def test_partition_by(started_cluster): assert "78,43,45\n" == get_s3_file_content(started_cluster, bucket, "test2_45.csv") -def test_partition_by_string_column(started_cluster): +def _test_partition_by_string_column(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "col_num UInt32, col_str String" @@ -191,7 +191,7 @@ def test_partition_by_string_column(started_cluster): assert '78,"你好"\n' == get_s3_file_content(started_cluster, bucket, "test_你好.csv") -def test_partition_by_const_column(started_cluster): +def _test_partition_by_const_column(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -212,7 +212,7 @@ def test_partition_by_const_column(started_cluster): "space", "plus" ]) -def test_get_file_with_special(started_cluster, special): +def _test_get_file_with_special(started_cluster, special): symbol = {"space": " ", "plus": "+"}[special] urlsafe_symbol = {"space": "%20", "plus": "%2B"}[special] auth = "'minio','minio123'," @@ -239,7 +239,7 @@ def test_get_file_with_special(started_cluster, special): "plus", "plus2" ]) -def test_get_path_with_special(started_cluster, special): +def _test_get_path_with_special(started_cluster, special): symbol = {"space": "%20", "plus": "%2B", "plus2": "%2B"}[special] safe_symbol = {"space": "%20", "plus": "+", "plus2": "%2B"}[special] auth = "'minio','minio123'," @@ -253,7 +253,7 @@ def test_get_path_with_special(started_cluster, special): @pytest.mark.parametrize("auth", [ pytest.param("'minio','minio123',", id="minio") ]) -def test_empty_put(started_cluster, auth): +def _test_empty_put(started_cluster, auth): # type: (ClickHouseCluster, str) -> None bucket = started_cluster.minio_bucket @@ -291,7 +291,7 @@ def test_empty_put(started_cluster, auth): pytest.param("'minio','minio123',", True, id="auth_positive"), pytest.param("'wrongid','wrongkey',", False, id="negative"), ]) -def test_put_csv(started_cluster, maybe_auth, positive): +def _test_put_csv(started_cluster, maybe_auth, positive): # type: (ClickHouseCluster, bool, str) -> None bucket = started_cluster.minio_bucket if not maybe_auth else started_cluster.minio_restricted_bucket @@ -313,7 +313,7 @@ def test_put_csv(started_cluster, maybe_auth, positive): # Test put and get with S3 server redirect. -def test_put_get_with_redirect(started_cluster): +def _test_put_get_with_redirect(started_cluster): # type: (ClickHouseCluster) -> None bucket = started_cluster.minio_bucket @@ -340,7 +340,7 @@ def test_put_get_with_redirect(started_cluster): # Test put with restricted S3 server redirect. -def test_put_with_zero_redirect(started_cluster): +def _test_put_with_zero_redirect(started_cluster): # type: (ClickHouseCluster) -> None bucket = started_cluster.minio_bucket @@ -367,7 +367,7 @@ def test_put_with_zero_redirect(started_cluster): assert exception_raised -def test_put_get_with_globs(started_cluster): +def _test_put_get_with_globs(started_cluster): # type: (ClickHouseCluster) -> None unique_prefix = random.randint(1,10000) bucket = started_cluster.minio_bucket @@ -399,7 +399,7 @@ def test_put_get_with_globs(started_cluster): pytest.param("'wrongid','wrongkey'", False, id="negative"), # ("'minio','minio123',",True), Redirect with credentials not working with nginx. ]) -def test_multipart_put(started_cluster, maybe_auth, positive): +def _test_multipart_put(started_cluster, maybe_auth, positive): # type: (ClickHouseCluster) -> None bucket = started_cluster.minio_bucket if not maybe_auth else started_cluster.minio_restricted_bucket @@ -439,7 +439,7 @@ def test_multipart_put(started_cluster, maybe_auth, positive): assert csv_data == get_s3_file_content(started_cluster, bucket, filename) -def test_remote_host_filter(started_cluster): +def _test_remote_host_filter(started_cluster): instance = started_cluster.instances["restricted_dummy"] format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -457,7 +457,7 @@ def test_remote_host_filter(started_cluster): pytest.param("''", id="1_argument"), pytest.param("'','','','','',''", id="6_arguments"), ]) -def test_wrong_s3_syntax(started_cluster, s3_storage_args): +def _test_wrong_s3_syntax(started_cluster, s3_storage_args): instance = started_cluster.instances["dummy"] # type: ClickHouseInstance expected_err_msg = "Code: 42" # NUMBER_OF_ARGUMENTS_DOESNT_MATCH @@ -466,7 +466,7 @@ def test_wrong_s3_syntax(started_cluster, s3_storage_args): # https://en.wikipedia.org/wiki/One_Thousand_and_One_Nights -def test_s3_glob_scheherazade(started_cluster): +def _test_s3_glob_scheherazade(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -535,7 +535,7 @@ def replace_config(old, new): config.close() -def test_custom_auth_headers(started_cluster): +def _test_custom_auth_headers(started_cluster): table_format = "column1 UInt32, column2 UInt32, column3 UInt32" filename = "test.csv" get_query = "select * from s3('http://resolver:8080/{bucket}/{file}', 'CSV', '{table_format}')".format( @@ -566,7 +566,7 @@ def test_custom_auth_headers(started_cluster): instance.query("DROP TABLE test") -def test_custom_auth_headers_exclusion(started_cluster): +def _test_custom_auth_headers_exclusion(started_cluster): table_format = "column1 UInt32, column2 UInt32, column3 UInt32" filename = "test.csv" get_query = f"SELECT * FROM s3('http://resolver:8080/{started_cluster.minio_restricted_bucket}/restricteddirectory/{filename}', 'CSV', '{table_format}')" @@ -580,7 +580,7 @@ def test_custom_auth_headers_exclusion(started_cluster): assert 'Forbidden Error' in ei.value.stderr -def test_infinite_redirect(started_cluster): +def _test_infinite_redirect(started_cluster): bucket = "redirected" table_format = "column1 UInt32, column2 UInt32, column3 UInt32" filename = "test.csv" @@ -598,7 +598,7 @@ def test_infinite_redirect(started_cluster): pytest.param("bin", "gzip", id="bin"), pytest.param("gz", "auto", id="gz"), ]) -def test_storage_s3_get_gzip(started_cluster, extension, method): +def _test_storage_s3_get_gzip(started_cluster, extension, method): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] filename = f"test_get_gzip.{extension}" @@ -638,7 +638,7 @@ def test_storage_s3_get_gzip(started_cluster, extension, method): run_query(instance, f"DROP TABLE {name}") -def test_storage_s3_get_unstable(started_cluster): +def _test_storage_s3_get_unstable(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] table_format = "column1 Int64, column2 Int64, column3 Int64, column4 Int64" @@ -647,7 +647,7 @@ def test_storage_s3_get_unstable(started_cluster): assert result.splitlines() == ["500001,500000,0"] -def test_storage_s3_put_uncompressed(started_cluster): +def _test_storage_s3_put_uncompressed(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] filename = "test_put_uncompressed.bin" @@ -684,7 +684,7 @@ def test_storage_s3_put_uncompressed(started_cluster): pytest.param("bin", "gzip", id="bin"), pytest.param("gz", "auto", id="gz") ]) -def test_storage_s3_put_gzip(started_cluster, extension, method): +def _test_storage_s3_put_gzip(started_cluster, extension, method): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] filename = f"test_put_gzip.{extension}" @@ -721,7 +721,7 @@ def test_storage_s3_put_gzip(started_cluster, extension, method): assert sum([ int(i.split(',')[1]) for i in uncompressed_content.splitlines() ]) == 708 -def test_truncate_table(started_cluster): +def _test_truncate_table(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] # type: ClickHouseInstance name = "truncate" @@ -745,7 +745,7 @@ def test_truncate_table(started_cluster): assert instance.query("SELECT * FROM {}".format(name)) == "" -def test_predefined_connection_configuration(started_cluster): +def _test_predefined_connection_configuration(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] # type: ClickHouseInstance name = "test_table" @@ -762,7 +762,7 @@ def test_predefined_connection_configuration(started_cluster): result = "" -def test_url_reconnect_in_the_middle(started_cluster): +def _test_url_reconnect_in_the_middle(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] table_format = "id String, data String" @@ -783,7 +783,7 @@ def test_url_reconnect_in_the_middle(started_cluster): f"""select sum(cityHash64(x)) from (select toUInt64(id) + sleep(0.1) as x from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{filename}', 'TSV', '{table_format}') settings http_max_tries = 10, http_retry_max_backoff_ms=2000, http_send_timeout=1, http_receive_timeout=1)""") - assert(int(result), 3914219105369203805) + assert(int(result) == 3914219105369203805) thread = threading.Thread(target=select) thread.start() @@ -796,10 +796,10 @@ def test_url_reconnect_in_the_middle(started_cluster): thread.join() - assert(int(result), 3914219105369203805) + assert(int(result) == 3914219105369203805) -def test_seekable_formats(started_cluster): +def _test_seekable_formats(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] # type: ClickHouseInstance @@ -821,7 +821,7 @@ def test_seekable_formats(started_cluster): assert(int(result[:3]) < 200) -def test_seekable_formats_url(started_cluster): +def _test_seekable_formats_url(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] @@ -885,3 +885,33 @@ def test_s3_schema_inference(started_cluster): result = instance.query(f"select count(*) from schema_inference") assert(int(result) == 5000000) + + table_function = f"url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_native', 'Native')" + result = instance.query(f"desc {table_function}") + assert result == "a\tInt32\t\t\t\t\t\nb\tString\t\t\t\t\t\n" + + result = instance.query(f"select count(*) from {table_function}") + assert(int(result) == 5000000) + + instance.query(f"create table schema_inference_2 engine=URL('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_native', 'Native')") + result = instance.query(f"desc schema_inference_2") + assert result == "a\tInt32\t\t\t\t\t\nb\tString\t\t\t\t\t\n" + + result = instance.query(f"select count(*) from schema_inference_2") + assert(int(result) == 5000000) + + +def test_empty_file(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] + + name = "empty" + url = f'http://{started_cluster.minio_ip}:{MINIO_INTERNAL_PORT}/{bucket}/{name}' + + minio = started_cluster.minio_client + minio.put_object(bucket, name, io.BytesIO(b""), 0) + + table_function = f"s3('{url}', 'CSV', 'id Int32')" + result = instance.query(f"SELECT count() FROM {table_function}") + assert(int(result) == 0) + diff --git a/tests/queries/0_stateless/00646_url_engine.python b/tests/queries/0_stateless/00646_url_engine.python index ee67f7063e7..4f47e819328 100644 --- a/tests/queries/0_stateless/00646_url_engine.python +++ b/tests/queries/0_stateless/00646_url_engine.python @@ -145,18 +145,12 @@ def test_select(table_name="", schema="str String,numuint UInt32,numint Int32,do if table_name: get_ch_answer("drop table if exists {}".format(table_name)) - if schema is None: - get_ch_answer("create table {} engine=URL('{}', 'CSV')".format(table_name, HTTP_SERVER_URL_STR)) - else: - get_ch_answer("create table {} ({}) engine=URL('{}', 'CSV')".format(table_name, schema, HTTP_SERVER_URL_STR)) + get_ch_answer("create table {} ({}) engine=URL('{}', 'CSV')".format(table_name, schema, HTTP_SERVER_URL_STR)) for i in range(len(requests)): tbl = table_name if not tbl: - if schema is None: - tbl = "url('{addr}', 'CSV')".format(addr=HTTP_SERVER_URL_STR) - else: - tbl = "url('{addr}', 'CSV', '{schema}')".format(addr=HTTP_SERVER_URL_STR, schema=schema) + tbl = "url('{addr}', 'CSV', '{schema}')".format(addr=HTTP_SERVER_URL_STR, schema=schema) check_answers(requests[i].format(tbl=tbl), answers[i]) if table_name: @@ -196,12 +190,6 @@ def main(): "select str,numuint,numint,double from {tbl} limit 1": test_data.split("\n")[0].replace(',', '\t'), } - select_only_requests_without_schema = { - "select * from {tbl}" : test_data.replace(',', '\t'), - "select column_2, count(*) from {tbl} group by column_2" : "2\t2", - "select * from {tbl} limit 1": test_data.split("\n")[0].replace(',', '\t'), - } - insert_requests = [ "insert into {tbl} values('Hello',10,-2,7.7)('World',10,-5,7.7)", "insert into {tbl} select 'Buy', number, 9-number, 9.9 from system.numbers limit 10", @@ -213,16 +201,12 @@ def main(): 'select double, count(*) from {tbl} group by double': "7.7\t2\n9.9\t10" } - t = start_server(len(select_only_requests) * 2 + len(select_only_requests_without_schema) * 2 + 4 + (len(insert_requests) + len(select_requests)) * 2) + t = start_server(len(select_only_requests) * 2 + (len(insert_requests) + len(select_requests)) * 2) t.start() # test table with url engine test_select(table_name="test_table_select", requests=list(select_only_requests.keys()), answers=list(select_only_requests.values()), test_data=test_data) # test table function url test_select(requests=list(select_only_requests.keys()), answers=list(select_only_requests.values()), test_data=test_data) - # test table with url engine without schema - test_select(table_name="test_table_select", schema=None, requests=list(select_only_requests_without_schema.keys()), answers=list(select_only_requests_without_schema.values()), test_data=test_data) - # test table function url without schema - test_select(schema=None, requests=list(select_only_requests_without_schema.keys()), answers=list(select_only_requests_without_schema.values()), test_data=test_data) #test insert into table with url engine test_insert(table_name="test_table_insert", requests_insert=insert_requests, requests_select=list(select_requests.keys()), answers=list(select_requests.values())) #test insert into table function url From 26abf7aa626331edd04349b3ee5dc24350448f35 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 27 Dec 2021 20:30:40 +0300 Subject: [PATCH 0562/1260] Remove code duplication, use simdjson and rapidjson instead of Poco --- src/CMakeLists.txt | 8 + src/Formats/EscapingRuleUtils.cpp | 8 +- src/Formats/JSONEachRowUtils.cpp | 216 ++++++++---------- src/Formats/JSONEachRowUtils.h | 7 +- src/Formats/config_formats.h.in | 3 +- src/Processors/Formats/ISchemaReader.cpp | 4 +- .../Formats/Impl/ORCBlockInputFormat.cpp | 15 +- src/Storages/HDFS/StorageHDFS.cpp | 13 +- .../02149_schema_inference.reference | 114 ++++----- 9 files changed, 181 insertions(+), 207 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 7124961821e..0fe66314114 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -526,6 +526,14 @@ if (USE_BZIP2) target_include_directories (clickhouse_common_io SYSTEM BEFORE PRIVATE ${BZIP2_INCLUDE_DIR}) endif() +if(USE_SIMDJSON) + dbms_target_link_libraries(PRIVATE simdjson) +endif() + +if(USE_RAPIDJSON) + dbms_target_include_directories(SYSTEM PRIVATE ${RAPIDJSON_INCLUDE_DIR}) +endif() + dbms_target_link_libraries(PUBLIC consistent-hashing) include ("${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake") diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 3049168351f..0a7747fc864 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -288,13 +288,7 @@ DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSe return parsed ? type : nullptr; } case FormatSettings::EscapingRule::JSON: - { - Poco::JSON::Parser parser; - String json = "{\"field\" : " + field + "}"; - auto var = parser.parse(json); - Poco::JSON::Object::Ptr object = var.extract(); - return getDataTypeFromJSONField(object->get("field")); - } + return getDataTypeFromJSONField(field); case FormatSettings::EscapingRule::CSV: { if (field.empty() || field == format_settings.csv.null_representation) diff --git a/src/Formats/JSONEachRowUtils.cpp b/src/Formats/JSONEachRowUtils.cpp index b7672388491..0ee00c126df 100644 --- a/src/Formats/JSONEachRowUtils.cpp +++ b/src/Formats/JSONEachRowUtils.cpp @@ -9,10 +9,14 @@ #include #include #include -#include +#include +#include +#include #include +#include + namespace DB { namespace ErrorCodes @@ -34,7 +38,7 @@ static std::pair fileSegmentationEngineJSONEachRowImpl(ReadBuffer while (loadAtPosition(in, memory, pos) && (balance || memory.size() + static_cast(pos - in.position()) < min_chunk_size || number_of_rows < min_rows)) { const auto current_object_size = memory.size() + static_cast(pos - in.position()); - if (current_object_size > 10 * min_chunk_size) + if (min_chunk_size != 0 && current_object_size > 10 * min_chunk_size) throw ParsingException("Size of JSON object is extremely large. Expected not greater than " + std::to_string(min_chunk_size) + " bytes, but current is " + std::to_string(current_object_size) + " bytes per row. Increase the value setting 'min_chunk_bytes_for_parallel_parsing' or check your data manually, most likely JSON is malformed", ErrorCodes::INCORRECT_DATA); @@ -103,87 +107,20 @@ static std::pair fileSegmentationEngineJSONEachRowImpl(ReadBuffer template static String readJSONEachRowLineIntoStringImpl(ReadBuffer & in) { - skipWhitespaceIfAny(in); - - if (in.eof()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read JSON object: unexpected end of file"); - - char * pos = in.position(); - if (*pos != opening_bracket) - throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read JSONEachRow line: {} expected, {} got", opening_bracket, *in.position()); - ++pos; - Memory memory; - size_t balance = 1; - bool quotes = false; - while (loadAtPosition(in, memory, pos) && balance) - { - if (quotes) - { - pos = find_first_symbols<'\\', '"'>(pos, in.buffer().end()); - - if (pos == in.buffer().end()) - continue; - - if (*pos == '\\') - { - ++pos; - if (loadAtPosition(in, memory, pos)) - ++pos; - } - else if (*pos == '"') - { - ++pos; - quotes = false; - } - } - else - { - pos = find_first_symbols(pos, in.buffer().end()); - - if (pos == in.buffer().end()) - continue; - - else if (*pos == opening_bracket) - { - ++balance; - ++pos; - } - else if (*pos == closing_bracket) - { - --balance; - ++pos; - } - else if (*pos == '\\') - { - ++pos; - if (loadAtPosition(in, memory, pos)) - ++pos; - } - else if (*pos == '"') - { - quotes = true; - ++pos; - } - } - } - - if (balance) - throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read JSON object: unexpected end of file"); - - saveUpToPosition(in, memory, pos); + fileSegmentationEngineJSONEachRowImpl(in, memory, 0, 1); return String(memory.data(), memory.size()); } -DataTypePtr getDataTypeFromJSONField(const Poco::Dynamic::Var & field) +DataTypePtr getDataTypeFromJSONFieldImpl(const SimdJSONParser::Element & field) { - if (field.isEmpty()) + if (field.isNull()) return nullptr; - if (field.isBoolean()) + if (field.isBool()) return makeNullable(std::make_shared()); - if (field.isNumeric()) + if (field.isInt64() || field.isUInt64() || field.isDouble()) return makeNullable(std::make_shared()); if (field.isString()) @@ -191,18 +128,18 @@ DataTypePtr getDataTypeFromJSONField(const Poco::Dynamic::Var & field) if (field.isArray()) { - Poco::JSON::Array::Ptr array = field.extract(); + auto array = field.getArray(); /// Return nullptr in case of empty array because we cannot determine nested type. - if (array->size() == 0) + if (array.size() == 0) return nullptr; DataTypes nested_data_types; /// If this array contains fields with different types we will treat it as Tuple. bool is_tuple = false; - for (size_t i = 0; i != array->size(); ++i) + for (const auto element : array) { - auto type = getDataTypeFromJSONField(array->get(i)); + auto type = getDataTypeFromJSONFieldImpl(element); if (!type) return nullptr; @@ -218,14 +155,13 @@ DataTypePtr getDataTypeFromJSONField(const Poco::Dynamic::Var & field) return std::make_shared(nested_data_types.back()); } - if (field.type() == typeid(Poco::JSON::Object::Ptr)) + if (field.isObject()) { - Poco::JSON::Object::Ptr object = field.extract(); - auto names = object->getNames(); + auto object = field.getObject(); DataTypePtr value_type; - for (const auto & name : names) + for (const auto key_value_pair : object) { - auto type = getDataTypeFromJSONField(object->get(name)); + auto type = getDataTypeFromJSONFieldImpl(key_value_pair.second); if (!type) return nullptr; @@ -237,23 +173,45 @@ DataTypePtr getDataTypeFromJSONField(const Poco::Dynamic::Var & field) return std::make_shared(std::make_shared(), value_type); } - throw Exception{ErrorCodes::INCORRECT_DATA, "Unexpected JSON type {}", field.type().name()}; + throw Exception{ErrorCodes::INCORRECT_DATA, "Unexpected JSON type"}; } -using JSONEachRowFieldExtractor = std::function(const Poco::Dynamic::Var &)>; - -template -static DataTypes determineColumnDataTypesFromJSONEachRowDataImpl(ReadBuffer & in, bool /*json_strings*/, JSONEachRowFieldExtractor extractor) +auto getJSONParserAndElement() { - Poco::JSON::Parser parser; - DataTypes data_types; +#if USE_SIMDJSON + return std::pair(); +#elif USE_RAPIDJSON + return std::pair(); +#else + return std::pair(); +#endif +} +DataTypePtr getDataTypeFromJSONField(const String & field) +{ + auto [parser, element] = getJSONParserAndElement(); + bool parsed = parser.parse(field, element); + if (!parsed) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot parse JSON object"); + + return getDataTypeFromJSONFieldImpl(element); +} + +template +static DataTypes determineColumnDataTypesFromJSONEachRowDataImpl(ReadBuffer & in, bool /*json_strings*/, Extractor & extractor) +{ String line = readJSONEachRowLineIntoStringImpl(in); - auto var = parser.parse(line); - std::vector fields = extractor(var); + auto [parser, element] = getJSONParserAndElement(); + bool parsed = parser.parse(line, element); + if (!parsed) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot parse JSON object"); + + auto fields = extractor.extract(element); + + DataTypes data_types; data_types.reserve(fields.size()); for (const auto & field : fields) - data_types.push_back(getDataTypeFromJSONField(field)); + data_types.push_back(getDataTypeFromJSONFieldImpl(field)); /// TODO: For JSONStringsEachRow/JSONCompactStringsEach all types will be strings. /// Should we try to parse data inside strings somehow in this case? @@ -271,43 +229,57 @@ std::pair fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in return fileSegmentationEngineJSONEachRowImpl<'[', ']'>(in, memory, min_chunk_size, min_rows); } +struct JSONEachRowFieldsExtractor +{ + template + std::vector extract(const Element & element) + { + /// {..., "" : , ...} + auto object = element.getObject(); + std::vector fields; + fields.reserve(object.size()); + column_names.reserve(object.size()); + for (const auto & key_value_pair : object) + { + column_names.emplace_back(key_value_pair.first); + fields.push_back(key_value_pair.second); + } + + return fields; + } + + std::vector column_names; +}; + std::unordered_map readRowAndGetNamesAndDataTypesForJSONEachRow(ReadBuffer & in, bool json_strings) { - std::vector column_names; - - /// {..., "" : , ...} - auto extractor = [&](const Poco::Dynamic::Var & var) - { - Poco::JSON::Object::Ptr object = var.extract(); - column_names = object->getNames(); - - std::vector fields; - for (size_t i = 0; i != object->size(); ++i) - fields.push_back(object->get(column_names[i])); - return fields; - }; - - auto data_types = determineColumnDataTypesFromJSONEachRowDataImpl<'{', '}'>(in, json_strings, extractor); + JSONEachRowFieldsExtractor extractor; + auto data_types = determineColumnDataTypesFromJSONEachRowDataImpl(in, json_strings, extractor); std::unordered_map result; - for (size_t i = 0; i != column_names.size(); ++i) - result[column_names[i]] = data_types[i]; + for (size_t i = 0; i != extractor.column_names.size(); ++i) + result[extractor.column_names[i]] = data_types[i]; return result; } +struct JSONCompactEachRowFieldsExtractor +{ + template + std::vector extract(const Element & element) + { + /// [..., , ...] + auto array = element.getArray(); + std::vector fields; + fields.reserve(array.size()); + for (size_t i = 0; i != array.size(); ++i) + fields.push_back(array[i]); + return fields; + } +}; + DataTypes readRowAndGetDataTypesForJSONCompactEachRow(ReadBuffer & in, bool json_strings) { - /// [..., , ...] - auto extractor = [](const Poco::Dynamic::Var & var) - { - Poco::JSON::Array::Ptr array = var.extract(); - std::vector fields; - fields.reserve(array->size()); - for (size_t i = 0; i != array->size(); ++i) - fields.push_back(array->get(i)); - return fields; - }; - - return determineColumnDataTypesFromJSONEachRowDataImpl<'[', ']'>(in, json_strings, extractor); + JSONCompactEachRowFieldsExtractor extractor; + return determineColumnDataTypesFromJSONEachRowDataImpl(in, json_strings, extractor); } diff --git a/src/Formats/JSONEachRowUtils.h b/src/Formats/JSONEachRowUtils.h index 68dc737a6df..9f2dcaa349b 100644 --- a/src/Formats/JSONEachRowUtils.h +++ b/src/Formats/JSONEachRowUtils.h @@ -1,9 +1,10 @@ #pragma once +#include "config_formats.h" + #include #include #include -#include #include namespace DB @@ -13,10 +14,10 @@ std::pair fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::M std::pair fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, size_t min_rows); -/// Convert JSON type to ClickHouse type. Make the result type always Nullable. +/// Parse JSON from string and convert it's type to ClickHouse type. Make the result type always Nullable. /// JSON array with different nested types is treated as Tuple. /// If cannot convert (for example when field contains null), return nullptr. -DataTypePtr getDataTypeFromJSONField(const Poco::Dynamic::Var & field); +DataTypePtr getDataTypeFromJSONField(const String & field); /// Read row in JSONEachRow format and try to determine type for each field. /// Return map {column_name : type}. diff --git a/src/Formats/config_formats.h.in b/src/Formats/config_formats.h.in index f6497b4830b..ab2420fcac4 100644 --- a/src/Formats/config_formats.h.in +++ b/src/Formats/config_formats.h.in @@ -10,4 +10,5 @@ #cmakedefine01 USE_ARROW #cmakedefine01 USE_PROTOBUF #cmakedefine01 USE_MSGPACK - +#cmakedefine01 USE_SIMDJSON +#cmakedefine01 USE_RAPIDJSON diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 73ad57e71e6..096e39a2893 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -55,12 +55,12 @@ NamesAndTypesList IRowSchemaReader::readSchema() if (data_types.empty()) throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot read rows from the data"); - /// If column names weren't set, use default names 'column_1', 'column_2', ... + /// If column names weren't set, use default names 'c1', 'c2', ... if (column_names.empty()) { column_names.reserve(data_types.size()); for (size_t i = 0; i != data_types.size(); ++i) - column_names.push_back("column_" + std::to_string(i + 1)); + column_names.push_back("c" + std::to_string(i + 1)); } /// If column names were set, check that the number of names match the number of types. else if (column_names.size() != data_types.size()) diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 44f6a44e492..9a787e5a614 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -88,9 +88,13 @@ static size_t countIndicesForType(std::shared_ptr type) } static void getFileReaderAndSchema( - ReadBuffer & in, std::unique_ptr & file_reader, std::shared_ptr & schema, const FormatSettings & format_settings) + ReadBuffer & in, + std::unique_ptr & file_reader, + std::shared_ptr & schema, + const FormatSettings & format_settings, + std::atomic & is_stopped) { - auto arrow_file = asArrowFile(*in, format_settings, is_stopped); + auto arrow_file = asArrowFile(in, format_settings, is_stopped); if (is_stopped) return; @@ -108,7 +112,9 @@ static void getFileReaderAndSchema( void ORCBlockInputFormat::prepareReader() { std::shared_ptr schema; - getFileReaderAndSchema(*in, file_reader, schema, format_settings); + getFileReaderAndSchema(*in, file_reader, schema, format_settings, is_stopped); + if (is_stopped) + return; arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), "ORC", format_settings.orc.import_nested); @@ -143,7 +149,8 @@ NamesAndTypesList ORCSchemaReader::readSchema() { std::unique_ptr file_reader; std::shared_ptr schema; - getFileReaderAndSchema(in, file_reader, schema, format_settings); + std::atomic is_stopped = 0; + getFileReaderAndSchema(in, file_reader, schema, format_settings, is_stopped); auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader(*schema, "ORC"); return header.getNamesAndTypesList(); } diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 3d4b38afa1c..d22fea92375 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -117,9 +117,6 @@ namespace } } -static Strings listFilesWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match); - - StorageHDFS::StorageHDFS( const String & uri_, const StorageID & table_id_, @@ -187,14 +184,8 @@ class HDFSSource::DisclosedGlobIterator::Impl public: Impl(ContextPtr context_, const String & uri) { - const size_t begin_of_path = uri.find('/', uri.find("//") + 2); - const String path_from_uri = uri.substr(begin_of_path); - const String uri_without_path = uri.substr(0, begin_of_path); /// ends without '/' - - HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context_->getGlobalContext()->getConfigRef()); - HDFSFSPtr fs = createHDFSFS(builder.get()); - - uris = listFilesWithRegexpMatching("/", fs, path_from_uri); + const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri); + uris = getPathsList(path_from_uri, uri_without_path, context_); for (auto & elem : uris) elem = uri_without_path + elem; uris_iter = uris.begin(); diff --git a/tests/queries/0_stateless/02149_schema_inference.reference b/tests/queries/0_stateless/02149_schema_inference.reference index 77f1ab707f5..f46e3bee101 100644 --- a/tests/queries/0_stateless/02149_schema_inference.reference +++ b/tests/queries/0_stateless/02149_schema_inference.reference @@ -1,8 +1,8 @@ TSV -column_1 Nullable(String) -column_2 Nullable(String) -column_3 Nullable(String) -column_4 Nullable(String) +c1 Nullable(String) +c2 Nullable(String) +c3 Nullable(String) +c4 Nullable(String) 42 Some string [1, 2, 3, 4] (1, 2, 3) 42 abcd [] (4, 5, 6) TSVWithNames @@ -13,18 +13,18 @@ tuple Nullable(String) 42 Some string [1, 2, 3, 4] (1, 2, 3) 42 abcd [] (4, 5, 6) CSV -column_1 Nullable(Float64) -column_2 Nullable(String) -column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) -column_4 Array(Nullable(Float64)) +c1 Nullable(Float64) +c2 Nullable(String) +c3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +c4 Array(Nullable(Float64)) \N Some string [([1,2.3],'String'),([],NULL)] [1,NULL,3] 42 \N [([1,2.3],'String'),([3],'abcd')] [4,5,6] -column_1 Nullable(String) -column_2 Nullable(String) +c1 Nullable(String) +c2 Nullable(String) 42 String String 42 -column_1 Nullable(String) -column_2 Nullable(String) +c1 Nullable(String) +c2 Nullable(String) \N [NULL, NULL] \N [] CSVWithNames @@ -35,15 +35,15 @@ d Array(Nullable(Float64)) \N Some string [([1,2.3],'String'),([],NULL)] [1,NULL,3] 42 \N [([1,2.3],'String'),([3],'abcd')] [4,5,6] JSONCompactEachRow -column_1 Nullable(Float64) -column_2 Array(Tuple(Nullable(Float64), Nullable(String))) -column_3 Map(String, Nullable(Float64)) -column_4 Nullable(UInt8) +c1 Nullable(Float64) +c2 Array(Tuple(Nullable(Float64), Nullable(String))) +c3 Map(String, Nullable(Float64)) +c4 Nullable(UInt8) 42.42 [(1,'String'),(2,'abcd')] {'key':42,'key2':24} 1 -column_1 Nullable(Float64) -column_2 Array(Tuple(Nullable(Float64), Nullable(String))) -column_3 Map(String, Nullable(Float64)) -column_4 Nullable(UInt8) +c1 Nullable(Float64) +c2 Array(Tuple(Nullable(Float64), Nullable(String))) +c3 Map(String, Nullable(Float64)) +c4 Nullable(UInt8) \N [(1,'String'),(2,NULL)] {'key':NULL,'key2':24} \N 32 [(2,'String 2'),(3,'hello')] {'key3':4242,'key4':2424} 1 JSONCompactEachRowWithNames @@ -82,60 +82,60 @@ s1 \N 1 \N \N \N \N [3] \N Values -column_1 Nullable(Float64) -column_2 Nullable(String) -column_3 Array(Nullable(Float64)) -column_4 Tuple(Nullable(Float64), Nullable(String)) -column_5 Tuple(Array(Nullable(Float64)), Array(Tuple(Nullable(Float64), Nullable(String)))) +c1 Nullable(Float64) +c2 Nullable(String) +c3 Array(Nullable(Float64)) +c4 Tuple(Nullable(Float64), Nullable(String)) +c5 Tuple(Array(Nullable(Float64)), Array(Tuple(Nullable(Float64), Nullable(String)))) 42.42 Some string [1,2,3] (1,'2') ([1,2],[(3,'4'),(5,'6')]) -column_1 Nullable(Float64) -column_2 Nullable(String) -column_3 Array(Nullable(Float64)) -column_4 Tuple(Nullable(Float64), Nullable(Float64)) -column_5 Tuple(Array(Nullable(Float64)), Array(Tuple(Nullable(Float64), Nullable(String)))) +c1 Nullable(Float64) +c2 Nullable(String) +c3 Array(Nullable(Float64)) +c4 Tuple(Nullable(Float64), Nullable(Float64)) +c5 Tuple(Array(Nullable(Float64)), Array(Tuple(Nullable(Float64), Nullable(String)))) 42.42 \N [1,NULL,3] (1,NULL) ([1,2],[(3,'4'),(5,'6')]) \N Some string [10] (1,2) ([],[]) Regexp -column_1 Nullable(String) -column_2 Nullable(String) -column_3 Nullable(String) +c1 Nullable(String) +c2 Nullable(String) +c3 Nullable(String) 42 Some string 1 [([1, 2, 3], String 1), ([], String 1)] 2 Some string 2 [([4, 5, 6], String 2), ([], String 2)] 312 Some string 3 [([1, 2, 3], String 2), ([], String 2)] -column_1 Nullable(Float64) -column_2 Nullable(String) -column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +c1 Nullable(Float64) +c2 Nullable(String) +c3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) 42 Some string 1 [([1,2,3],'String 1'),([],'String 1')] 3 Some string 2 [([3,5,1],'String 2'),([],'String 2')] 244 Some string 3 [([],'String 3'),([],'String 3')] -column_1 Nullable(Float64) -column_2 Nullable(String) -column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +c1 Nullable(Float64) +c2 Nullable(String) +c3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) 42 Some string 1 [([1,2,3],'String 1'),([],'String 1')] 2 Some string 2 [([],'String 2'),([],'String 2')] 43 Some string 3 [([1,5,3],'String 3'),([],'String 3')] -column_1 Nullable(Float64) -column_2 Nullable(String) -column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +c1 Nullable(Float64) +c2 Nullable(String) +c3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) 42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] 52 Some string 2 [([],'String 2'),([1],'String 2')] 24 Some string 3 [([1,2,3],'String 3'),([1],'String 3')] CustomSeparated -column_1 Nullable(Float64) -column_2 Nullable(String) -column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +c1 Nullable(Float64) +c2 Nullable(String) +c3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) 42.42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] 42 Some string 2 [([],'String 2'),([],'String 2')] \N Some string 3 [([1,2,3],'String 3'),([1],'String 3')] -column_1 Nullable(Float64) -column_2 Nullable(String) -column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +c1 Nullable(Float64) +c2 Nullable(String) +c3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) 42.42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] 42 Some string 2 [([],'String 2'),([],'String 2')] \N Some string 3 [([1,2,3],'String 3'),([1],'String 3')] -column_1 Nullable(Float64) -column_2 Nullable(String) -column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) +c1 Nullable(Float64) +c2 Nullable(String) +c3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) 42.42 Some string 1 [([1,2,3],'String 1'),([1],'String 1')] 42 Some string 2 [([],'String 2'),([],'String 2')] \N Some string 3 [([1,2,3],'String 3'),([1],'String 3')] @@ -159,12 +159,12 @@ column_3 Array(Tuple(Array(Nullable(Float64)), Nullable(String))) 42 Some string 2 [([],'String 2'),([],'String 2')] \N Some string 3 [([1,2,3],'String 3'),([1],'String 3')] MsgPack -column_1 Nullable(Int64) -column_2 Nullable(Int64) -column_3 Nullable(Float32) -column_4 Nullable(String) -column_5 Array(Array(Nullable(Int64))) -column_6 Map(Int64, Array(Nullable(Int64))) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Float32) +c4 Nullable(String) +c5 Array(Array(Nullable(Int64))) +c6 Map(Int64, Array(Nullable(Int64))) \N 0 0 Str: 0 [[0,1],[0]] {0:[0,1]} 1 \N 1 Str: 1 [[1,2],[1]] {1:[1,2]} \N 2 2 Str: 2 [[2,3],[2]] {2:[2,3]} From d718a2e22097ab9b2934385f6de747a9f4038d59 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 27 Dec 2021 20:34:24 +0300 Subject: [PATCH 0563/1260] Clean up --- src/Formats/JSONEachRowUtils.cpp | 2 -- src/Formats/JSONEachRowUtils.h | 2 -- src/Formats/config_formats.h.in | 2 -- 3 files changed, 6 deletions(-) diff --git a/src/Formats/JSONEachRowUtils.cpp b/src/Formats/JSONEachRowUtils.cpp index 0ee00c126df..e384a3537c1 100644 --- a/src/Formats/JSONEachRowUtils.cpp +++ b/src/Formats/JSONEachRowUtils.cpp @@ -15,8 +15,6 @@ #include -#include - namespace DB { namespace ErrorCodes diff --git a/src/Formats/JSONEachRowUtils.h b/src/Formats/JSONEachRowUtils.h index 9f2dcaa349b..6f71baa8b40 100644 --- a/src/Formats/JSONEachRowUtils.h +++ b/src/Formats/JSONEachRowUtils.h @@ -1,7 +1,5 @@ #pragma once -#include "config_formats.h" - #include #include #include diff --git a/src/Formats/config_formats.h.in b/src/Formats/config_formats.h.in index ab2420fcac4..427abc7d1ce 100644 --- a/src/Formats/config_formats.h.in +++ b/src/Formats/config_formats.h.in @@ -10,5 +10,3 @@ #cmakedefine01 USE_ARROW #cmakedefine01 USE_PROTOBUF #cmakedefine01 USE_MSGPACK -#cmakedefine01 USE_SIMDJSON -#cmakedefine01 USE_RAPIDJSON From 8436638a8953fa5e86066f2166593863aa65820a Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 27 Dec 2021 20:48:40 +0300 Subject: [PATCH 0564/1260] Fix style --- src/Storages/HDFS/StorageHDFS.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index d22fea92375..f22f6f66ced 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -443,7 +443,7 @@ Pipe StorageHDFS::read( { bool need_path_column = false; bool need_file_column = false; - + for (const auto & column : column_names) { if (column == "_path") From 78b522fd515df4ee5be7cd61ae16621f240c2ce4 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 27 Dec 2021 21:12:10 +0300 Subject: [PATCH 0565/1260] Fix fasttest build --- src/Formats/JSONEachRowUtils.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Formats/JSONEachRowUtils.cpp b/src/Formats/JSONEachRowUtils.cpp index e384a3537c1..c63b8453634 100644 --- a/src/Formats/JSONEachRowUtils.cpp +++ b/src/Formats/JSONEachRowUtils.cpp @@ -110,7 +110,8 @@ static String readJSONEachRowLineIntoStringImpl(ReadBuffer & in) return String(memory.data(), memory.size()); } -DataTypePtr getDataTypeFromJSONFieldImpl(const SimdJSONParser::Element & field) +template +DataTypePtr getDataTypeFromJSONFieldImpl(const Element & field) { if (field.isNull()) return nullptr; From cb0ed7fcb728d473b642a6076c390152b93523c6 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 28 Dec 2021 09:40:36 +0300 Subject: [PATCH 0566/1260] Fix typo --- src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index 8b0c3d2eea4..c56af536e15 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -375,8 +375,8 @@ void MsgPackRowInputFormat::setReadBuffer(ReadBuffer & in_) IInputFormat::setReadBuffer(in_); } -MsgPackSchemaReader::MsgPackSchemaReader(ReadBuffer & in_, const FormatSettings & forma_settings_) - : IRowSchemaReader(buf, forma_settings_.max_rows_to_read_for_schema_inference), buf(in_), number_of_columns(forma_settings_.msgpack.number_of_columns) +MsgPackSchemaReader::MsgPackSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) + : IRowSchemaReader(buf, format_settings_.max_rows_to_read_for_schema_inference), buf(in_), number_of_columns(format_settings_.msgpack.number_of_columns) { if (!number_of_columns) throw Exception(ErrorCodes::BAD_ARGUMENTS, "You must specify setting input_format_msgpack_number_of_columns to extract table schema from MsgPack data"); From 364b4f5d36a00290560e3649f368359712a29b12 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 28 Dec 2021 20:57:16 +0300 Subject: [PATCH 0567/1260] Fix special build --- src/Processors/Formats/Impl/CapnProtoRowInputFormat.h | 2 +- src/Processors/Formats/Impl/NativeFormat.cpp | 2 +- src/Processors/Formats/Impl/ProtobufRowInputFormat.h | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h index 3db4f3de0d4..053de14d1a4 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h @@ -42,7 +42,7 @@ private: class CapnProtoSchemaReader : public IExternalSchemaReader { public: - CapnProtoSchemaReader(const FormatSettings & format_settings_); + explicit CapnProtoSchemaReader(const FormatSettings & format_settings_); NamesAndTypesList readSchema() override; diff --git a/src/Processors/Formats/Impl/NativeFormat.cpp b/src/Processors/Formats/Impl/NativeFormat.cpp index d7c70e1c37b..19e2ede6b65 100644 --- a/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/src/Processors/Formats/Impl/NativeFormat.cpp @@ -87,7 +87,7 @@ private: class NativeSchemaReader : public ISchemaReader { public: - NativeSchemaReader(ReadBuffer & in_) : ISchemaReader(in_) {} + explicit NativeSchemaReader(ReadBuffer & in_) : ISchemaReader(in_) {} NamesAndTypesList readSchema() override { diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h index 15305977619..d7d16d36ddf 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h @@ -47,7 +47,7 @@ private: class ProtobufSchemaReader : public IExternalSchemaReader { public: - ProtobufSchemaReader(const FormatSettings & format_settings); + explicit ProtobufSchemaReader(const FormatSettings & format_settings); NamesAndTypesList readSchema() override; From e62a8de46bae87ef5c43db341ee43d4ecfc357ee Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Dec 2021 12:43:12 +0300 Subject: [PATCH 0568/1260] Update docs/ru/development/developer-instruction.md --- docs/ru/development/developer-instruction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index eb224adeb0b..bdd007bcf7b 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -40,7 +40,7 @@ ClickHouse не работает и не собирается на 32-битны Выполните в терминале: - git clone --recursive git@github.com:your_github_username/ClickHouse.git + git clone git@github.com:your_github_username/ClickHouse.git cd ClickHouse Замените первое вхождение слова `ClickHouse` в команде для git на имя вашего аккаунта на GitHub. From e90003f05081c5691cd5dc2bc0a5e14fcc74adf5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Dec 2021 12:43:19 +0300 Subject: [PATCH 0569/1260] Update docs/ru/operations/clickhouse-keeper.md --- docs/ru/operations/clickhouse-keeper.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/clickhouse-keeper.md b/docs/ru/operations/clickhouse-keeper.md index 05784b2b942..51e34261b60 100644 --- a/docs/ru/operations/clickhouse-keeper.md +++ b/docs/ru/operations/clickhouse-keeper.md @@ -116,7 +116,7 @@ ClickHouse Keeper также поддерживает 4-х буквенные к У 4-х буквенных команд есть параметр для настройки разрешенного списка `four_letter_word_white_list`, который имеет значение по умолчанию "conf,cons,crst,envi,ruok,srst,srvr,stat, wchc,wchs,dirs,mntr,isro". -Вы можете отправлять команды в ClickHouse Keeper через telnet или nc через клиентский порт. +Вы можете отправлять команды в ClickHouse Keeper через telnet или nc на порт для клиента. ``` echo mntr | nc localhost 9181 From 4dd890ed17b6b06a73ded70ffe7ab7ff29071a6e Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Dec 2021 12:43:24 +0300 Subject: [PATCH 0570/1260] Update docs/ru/development/developer-instruction.md --- docs/ru/development/developer-instruction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index bdd007bcf7b..6e30e0052e5 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -68,7 +68,7 @@ ClickHouse не работает и не собирается на 32-битны Вы также можете клонировать репозиторий по протоколу https: - git clone --recursive https://github.com/ClickHouse/ClickHouse.git + git clone https://github.com/ClickHouse/ClickHouse.git Этот вариант не подходит для отправки изменений на сервер. Вы можете временно его использовать, а затем добавить ssh ключи и заменить адрес репозитория с помощью команды `git remote`. From a2e330e271de2952576b52042f427031f83638d2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Dec 2021 12:43:29 +0300 Subject: [PATCH 0571/1260] Update docs/ru/operations/clickhouse-keeper.md --- docs/ru/operations/clickhouse-keeper.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/clickhouse-keeper.md b/docs/ru/operations/clickhouse-keeper.md index 51e34261b60..41f5ea10f3e 100644 --- a/docs/ru/operations/clickhouse-keeper.md +++ b/docs/ru/operations/clickhouse-keeper.md @@ -152,7 +152,7 @@ zk_followers 0 zk_synced_followers 0 ``` -- `srvr`: Выводит полную информацию о сервере. +- `srvr`: Выводит информацию о сервере: его версию, роль участника кворума и т.п. ``` ClickHouse Keeper version: v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 From 73af23e323628ab7d7bc50009cc0c859cf62bfb4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Dec 2021 12:43:34 +0300 Subject: [PATCH 0572/1260] Update docs/ru/operations/clickhouse-keeper.md --- docs/ru/operations/clickhouse-keeper.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/clickhouse-keeper.md b/docs/ru/operations/clickhouse-keeper.md index 41f5ea10f3e..93264bf3f18 100644 --- a/docs/ru/operations/clickhouse-keeper.md +++ b/docs/ru/operations/clickhouse-keeper.md @@ -265,7 +265,7 @@ log_dir_size: 3875 rw ``` -- `wchs`: Показывает краткую информацию о наблюдениях за сервером. +- `wchs`: Показывает краткую информацию о количестве отслеживаемых путей (watches) на сервере. ``` 1 connections watching 1 paths From bc5c4057fdd9c1bf508913d49febe82478e6b297 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Dec 2021 12:43:41 +0300 Subject: [PATCH 0573/1260] Update docs/ru/operations/clickhouse-keeper.md --- docs/ru/operations/clickhouse-keeper.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/clickhouse-keeper.md b/docs/ru/operations/clickhouse-keeper.md index 93264bf3f18..27d0a5fd41f 100644 --- a/docs/ru/operations/clickhouse-keeper.md +++ b/docs/ru/operations/clickhouse-keeper.md @@ -272,7 +272,7 @@ rw Total watches:1 ``` -- `wchc`: Показывает подробную информацию о наблюдениях за сервером в разбивке по сессиям. При этом выводится список сессий (подключений) с соответствующими наблюдениями (путями). Обратите внимание, что в зависимости от количества наблюдений эта операция может быть дорогостоящей (т. е. повлиять на производительность сервера), используйте ее осторожно. +- `wchc`: Показывает подробную информацию об отслеживаемых путях (watches) на сервере в разбивке по сессиям. При этом выводится список сессий (подключений) с соответствующими отслеживаемыми путями. Обратите внимание, что в зависимости от количества отслеживаемых путей эта операция может быть дорогостоящей (т. е. повлиять на производительность сервера), используйте ее осторожно. ``` 0x0000000000000001 From ace477b6e6c3ea5731e4e48b1203f11e46d5a749 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Dec 2021 12:43:48 +0300 Subject: [PATCH 0574/1260] Update docs/ru/operations/clickhouse-keeper.md --- docs/ru/operations/clickhouse-keeper.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/clickhouse-keeper.md b/docs/ru/operations/clickhouse-keeper.md index 27d0a5fd41f..a41428459a8 100644 --- a/docs/ru/operations/clickhouse-keeper.md +++ b/docs/ru/operations/clickhouse-keeper.md @@ -279,7 +279,7 @@ Total watches:1 /clickhouse/task_queue/ddl ``` -- `wchp`: Показывает подробную информацию о наблюдениях за сервером в разбивке по пути. При этом выводится список путей (узлов) с соответствующими сессиями. Обратите внимание, что в зависимости от количества наблюдений эта операция может быть дорогостоящей (т. е. повлиять на производительность сервера), используйте ее осторожно. +- `wchp`: Показывает подробную информацию об отслеживаемых путях (watches) на сервере в разбивке по пути. При этом выводится список путей (узлов) с соответствующими сессиями. Обратите внимание, что в зависимости от количества отселживаемых путей (watches) эта операция может быть дорогостоящей (т. е. повлиять на производительность сервера), используйте ее осторожно. ``` /clickhouse/task_queue/ddl From dcdaa31c7d7e92f6e3584fa088362dffbeac7d53 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Dec 2021 12:43:54 +0300 Subject: [PATCH 0575/1260] Update docs/ru/operations/clickhouse-keeper.md --- docs/ru/operations/clickhouse-keeper.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/clickhouse-keeper.md b/docs/ru/operations/clickhouse-keeper.md index a41428459a8..2f3f3c0f63c 100644 --- a/docs/ru/operations/clickhouse-keeper.md +++ b/docs/ru/operations/clickhouse-keeper.md @@ -286,7 +286,7 @@ Total watches:1 0x0000000000000001 ``` -- `dump`: Выводит список незавершенных сеансов и эфемерных узлов. Это работает только с репликой-лидером. +- `dump`: Выводит список незавершенных сеансов и эфемерных узлов. Команда работает только на лидере. ``` Sessions dump (2): From 8690cf082ff2e505151e73837b8b8b4383cddfe9 Mon Sep 17 00:00:00 2001 From: zhoubintao Date: Wed, 29 Dec 2021 17:48:39 +0800 Subject: [PATCH 0576/1260] remove redundant judge to make heavy map iter operation faster --- src/Interpreters/RowRefs.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/RowRefs.h b/src/Interpreters/RowRefs.h index 047146d569c..987fd197d9d 100644 --- a/src/Interpreters/RowRefs.h +++ b/src/Interpreters/RowRefs.h @@ -103,7 +103,7 @@ struct RowRefList : RowRef } } - bool ok() const { return first || (batch && position < batch->size); } + bool ok() const { return first || batch; } private: const RowRefList * root; From 28835757f9cdb0228c8cc448972c37a2e90e1af7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Dec 2021 12:56:32 +0300 Subject: [PATCH 0577/1260] Update docs/ru/operations/settings/settings.md --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 860ac900ba5..d36e27991f8 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1649,7 +1649,7 @@ ClickHouse генерирует исключение: ## insert_quorum_parallel {#settings-insert_quorum_parallel} -Включает и выключает параллелизм для кворума запросов `INSERT`. Когда включена, можно выполнить дополнительные запросы `INSERT` в то время, пока предыдущие запросы еще не завершены. Когда выключена, дополнительные записи в ту же таблицу будут отклонены. +Включает и выключает параллелизм для кворумных вставок (`INSERT`-запросы). Когда опция включена, возможно выполнять несколько кворумных `INSERT`-запросов одновременно, при этом запросы не дожидаются окончания друг друга . Когда опция выключена, одновременные записи с кворумом в одну и ту же таблицу будут отклонены (будет выполнена только одна из них). Возможные значения: From 475fdb7beb7a62f5afcd09445187fc75aa24b8d7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Dec 2021 12:56:46 +0300 Subject: [PATCH 0578/1260] Update docs/ru/operations/settings/settings.md --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index d36e27991f8..ee030065ae8 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1666,7 +1666,7 @@ ClickHouse генерирует исключение: ## select_sequential_consistency {#settings-select_sequential_consistency} -Включает или выключает последовательную консистентность для запросов `SELECT`. Необходимо, чтобы `insert_quorum_parallel` была выключена (по умолчанию включена). +Включает или выключает последовательную консистентность для запросов `SELECT`. Необходимо, чтобы `insert_quorum_parallel` была выключена (по умолчанию включена), а опция `insert_quorum` включена. Возможные значения: From 396cbbd6b77eede3e760bdc3d2caed13550c0663 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Dec 2021 12:57:08 +0300 Subject: [PATCH 0579/1260] Update docs/ru/operations/settings/settings.md --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index ee030065ae8..0c4621d6e6d 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1627,7 +1627,7 @@ ClickHouse генерирует исключение: - Если количество доступных реплик на момент запроса меньше `insert_quorum`. - При попытке записать данные в момент, когда предыдущий блок ещё не вставлен в `insert_quorum` реплик. Эта ситуация может возникнуть, если пользователь вызвал `INSERT` прежде, чем завершился предыдущий с `insert_quorum`. -- При выключенной `insert_quorum_parallel` и при попытке записать данные в момент, когда предыдущий блок еще не вставлен в `insert_quorum` реплик. Эта ситуация может возникнуть при попытке пользователя выполнить очередной запрос `INSERT` к той же таблице, прежде чем завершится предыдущий с `insert_quorum`. +- При выключенной `insert_quorum_parallel` и при попытке записать данные в момент, когда предыдущий блок еще не вставлен в `insert_quorum` реплик (несколько параллельных `INSERT`-запросов). Эта ситуация может возникнуть при попытке пользователя выполнить очередной запрос `INSERT` к той же таблице, прежде чем завершится предыдущий с `insert_quorum`. См. также: From 2279dd0c7f6008b8a3e50c2aeb151cbacf66edc5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 29 Dec 2021 13:14:21 +0300 Subject: [PATCH 0580/1260] Reuse check_rabbitmq_is_available --- tests/integration/test_storage_rabbitmq/test.py | 13 +------------ 1 file changed, 1 insertion(+), 12 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index c5074156d88..a3d99159cb2 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -35,22 +35,11 @@ def rabbitmq_check_result(result, check=False, ref_file='test_rabbitmq_json.refe else: return TSV(result) == TSV(reference) -def check_rabbitmq_is_available(rabbitmq_id): - p = subprocess.Popen(('docker', - 'exec', - '-i', - rabbitmq_id, - 'rabbitmqctl', - 'await_startup'), - stdout=subprocess.PIPE) - p.communicate() - return p.returncode == 0 - def wait_rabbitmq_to_start(rabbitmq_docker_id, timeout=180): start = time.time() while time.time() - start < timeout: try: - if check_rabbitmq_is_available(rabbitmq_docker_id): + if instance.cluster.check_rabbitmq_is_available(rabbitmq_docker_id): logging.debug("RabbitMQ is available") return time.sleep(0.5) From 6b6a82f3b9f707fc856393b12fe8fa521ac0e24f Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 29 Dec 2021 18:18:38 +0800 Subject: [PATCH 0581/1260] add MappedHolder to get cache values --- src/Common/LRUResourceCache.h | 210 ++++++++++------ src/Common/tests/gtest_lru_resource_cache.cpp | 233 +++++++++--------- 2 files changed, 242 insertions(+), 201 deletions(-) diff --git a/src/Common/LRUResourceCache.h b/src/Common/LRUResourceCache.h index 221ba225f2c..d33280add25 100644 --- a/src/Common/LRUResourceCache.h +++ b/src/Common/LRUResourceCache.h @@ -32,102 +32,59 @@ template < class LRUResourceCache { public: - LRUResourceCache(size_t max_weight_, size_t max_element_size_ = 0) : max_weight(max_weight_), max_element_size(max_element_size_) { } - ~LRUResourceCache() = default; using Key = TKey; using Mapped = TMapped; using MappedPtr = std::shared_ptr; - - // - load_func : when key is not exists in cache, load_func is called to generate a new key - // - return: is null when there is no more space for the new value or the old value is in used. - template - MappedPtr acquire(const Key & key, LoadFunc && load_func) + class MappedHolder { - InsertToken * insert_token = nullptr; + public: + ~MappedHolder() { - std::lock_guard lock(mutex); - auto it = cells.find(key); - if (it != cells.end()) - { - hits++; - it->second.reference_count += 1; - queue.splice(queue.end(), queue, it->second.queue_iterator); - return it->second.value; - } - misses++; - insert_token = acquireInsertToken(key); + cache->release(key); + } - Cell * cell_ptr = nullptr; + Mapped & value() { - std::lock_guard lock(insert_token->mutex); - if (!insert_token->value) - { - insert_token->value = load_func(); - std::lock_guard cell_lock(mutex); - cell_ptr = insert_value(key, insert_token->value); - if (cell_ptr) - { - cell_ptr->reference_count += 1; - } - else - { - insert_token->value = nullptr; - } - } + return *(val.get()); + } + static bool tryRemove(std::unique_ptr * holder_ptr) + { + auto & holder = *holder_ptr; + auto cache = holder->cache; + auto key = holder->key; + *holder_ptr = nullptr; + return cache->tryRemove(key); } - std::lock_guard lock(mutex); - releaseInsertToken(key); - if (cell_ptr) + MappedHolder(LRUResourceCache * cache_, const Key & key_, MappedPtr value_) : cache(cache_), key(key_), val(value_) { - return cell_ptr->value; } - return nullptr; - } + protected: + LRUResourceCache * cache; + Key key; + MappedPtr val; + }; + using MappedHolderPtr = std::unique_ptr; - MappedPtr acquire(const Key & key) + // use get() or getOrSet() to access the elements + MappedHolderPtr get(const Key & key) { - std::lock_guard lock(mutex); - auto it = cells.find(key); - if (it == cells.end()) - { - misses++; + auto mappedptr = getImpl(key); + if (!mappedptr) return nullptr; - } - hits++; - it->second.reference_count += 1; - queue.splice(queue.end(), queue, it->second.queue_iterator); - return it->second.value; + return std::make_unique(this, key, mappedptr); + } + template + MappedHolderPtr getOrSet(const Key & key, LoadFunc && load_func) + { + auto mappedptr = getImpl(key, load_func); + if (!mappedptr) + return nullptr; + return std::make_unique(this, key, mappedptr); } - // mark a reference is released - void release(const Key & key) - { - std::lock_guard lock(mutex); - auto it = cells.find(key); - if (it == cells.end() || it->second.reference_count == 0) - { - LOG_ERROR(&Poco::Logger::get("LRUResourceCache"), "try to release an invalid element"); - abort(); - } - it->second.reference_count -= 1; - } - - // If you want to update a value, call tryRemove() at first and then call acquire() with load_func. - bool tryRemove(const Key & key) - { - std::lock_guard guard(mutex); - auto it = cells.find(key); - if (it == cells.end()) - return true; - auto & cell = it->second; - if (cell.reference_count) - return false; - queue.erase(cell.queue_iterator); - current_weight -= cell.weight; - cells.erase(it); - return true; - } + LRUResourceCache(size_t max_weight_, size_t max_element_size_ = 0) : max_weight(max_weight_), max_element_size(max_element_size_) { } + ~LRUResourceCache() = default; size_t weight() { @@ -181,6 +138,81 @@ private: std::atomic hits{0}; std::atomic misses{0}; std::atomic evict_count{0}; + + // - load_func : when key is not exists in cache, load_func is called to generate a new key + // - return: is null when there is no more space for the new value or the old value is in used. + template + MappedPtr getImpl(const Key & key, LoadFunc && load_func) + { + InsertToken * insert_token = nullptr; + { + std::lock_guard lock(mutex); + auto it = cells.find(key); + if (it != cells.end()) + { + hits++; + it->second.reference_count += 1; + queue.splice(queue.end(), queue, it->second.queue_iterator); + return it->second.value; + } + misses++; + insert_token = acquireInsertToken(key); + } + Cell * cell_ptr = nullptr; + { + std::lock_guard lock(insert_token->mutex); + if (!insert_token->value) + { + insert_token->value = load_func(); + std::lock_guard cell_lock(mutex); + cell_ptr = set(key, insert_token->value); + if (cell_ptr) + { + cell_ptr->reference_count += 1; + } + else + { + insert_token->value = nullptr; + } + } + } + + std::lock_guard lock(mutex); + releaseInsertToken(key); + if (cell_ptr) + { + return cell_ptr->value; + } + return nullptr; + } + + MappedPtr getImpl(const Key & key) + { + std::lock_guard lock(mutex); + auto it = cells.find(key); + if (it == cells.end()) + { + misses++; + return nullptr; + } + hits++; + it->second.reference_count += 1; + queue.splice(queue.end(), queue, it->second.queue_iterator); + return it->second.value; + } + + // mark a reference is released + void release(const Key & key) + { + std::lock_guard lock(mutex); + auto it = cells.find(key); + if (it == cells.end() || it->second.reference_count == 0) + { + LOG_ERROR(&Poco::Logger::get("LRUResourceCache"), "try to release an invalid element"); + abort(); + } + it->second.reference_count -= 1; + } InsertToken * acquireInsertToken(const Key & key) { @@ -201,7 +233,7 @@ private: } // key mustn't be in the cache - Cell * insert_value(const Key & insert_key, MappedPtr value) + Cell * set(const Key & insert_key, MappedPtr value) { auto weight = value ? weight_function(*value) : 0; auto queue_size = cells.size() + 1; @@ -251,5 +283,21 @@ private: new_cell.queue_iterator = queue.insert(queue.end(), insert_key); return &new_cell; } + + // If you want to update a value, call tryRemove() at first and then call acquire() with load_func. + bool tryRemove(const Key & key) + { + std::lock_guard guard(mutex); + auto it = cells.find(key); + if (it == cells.end()) + return true; + auto & cell = it->second; + if (cell.reference_count) + return false; + queue.erase(cell.queue_iterator); + current_weight -= cell.weight; + cells.erase(it); + return true; + } }; } diff --git a/src/Common/tests/gtest_lru_resource_cache.cpp b/src/Common/tests/gtest_lru_resource_cache.cpp index f87855bf51d..0d58fd571a2 100644 --- a/src/Common/tests/gtest_lru_resource_cache.cpp +++ b/src/Common/tests/gtest_lru_resource_cache.cpp @@ -3,21 +3,21 @@ #include #include -TEST(LRUResourceCache, acquire) +TEST(LRUResourceCache, get) { using MyCache = DB::LRUResourceCache; auto mcache = MyCache(10, 10); int x = 10; auto load_int = [&] { return std::make_shared(x); }; - auto val = mcache.acquire(1, load_int); + auto holder1 = mcache.getOrSet(1, load_int); x = 11; - val = mcache.acquire(2, load_int); - ASSERT_TRUE(val != nullptr); - ASSERT_TRUE(*val == 11); + auto holder2 = mcache.getOrSet(2, load_int); + ASSERT_TRUE(holder2 != nullptr); + ASSERT_TRUE(holder2->value() == 11); - val = mcache.acquire(1); - ASSERT_TRUE(val != nullptr); - ASSERT_TRUE(*val == 10); + auto holder3 = mcache.get(1); + ASSERT_TRUE(holder3 != nullptr); + ASSERT_TRUE(holder3->value() == 10); } TEST(LRUResourceCache, remove) @@ -26,27 +26,20 @@ TEST(LRUResourceCache, remove) auto mcache = MyCache(10, 10); int x = 10; auto load_int = [&] { return std::make_shared(x); }; - auto val = mcache.acquire(1, load_int); - x = 11; - val = mcache.acquire(2, load_int); + auto holder0 = mcache.getOrSet(1, load_int); + auto holder1 = mcache.getOrSet(1, load_int); - auto succ = mcache.tryRemove(3); - ASSERT_TRUE(succ); - - succ = mcache.tryRemove(1); + auto succ = MyCache::MappedHolder::tryRemove(&holder0); ASSERT_TRUE(!succ); - val = mcache.acquire(1); - ASSERT_TRUE(val != nullptr); - ASSERT_TRUE(*val == 10); + holder0 = mcache.get(1); + ASSERT_TRUE(holder0 != nullptr); + ASSERT_TRUE(holder0->value() == 10); - mcache.release(1); - succ = mcache.tryRemove(1); - ASSERT_TRUE(!succ); - mcache.release(1); - succ = mcache.tryRemove(1); + holder0 = nullptr; + succ = MyCache::MappedHolder::tryRemove(&holder1); ASSERT_TRUE(succ); - val = mcache.acquire(1); - ASSERT_TRUE(val == nullptr); + holder1 = mcache.get(1); + ASSERT_TRUE(holder1 == nullptr); } struct MyWeight @@ -60,27 +53,27 @@ TEST(LRUResourceCache, evict_on_weight) auto mcache = MyCache(5, 10); int x = 2; auto load_int = [&] { return std::make_shared(x); }; - auto val = mcache.acquire(1, load_int); - mcache.release(1); + auto holder1 = mcache.getOrSet(1, load_int); + holder1 = nullptr; - val = mcache.acquire(2, load_int); - mcache.release(2); + auto holder2 = mcache.getOrSet(2, load_int); + holder2 = nullptr; x = 3; - val = mcache.acquire(3, load_int); - ASSERT_TRUE(val != nullptr); + auto holder3 = mcache.getOrSet(3, load_int); + ASSERT_TRUE(holder3 != nullptr); auto w = mcache.weight(); ASSERT_EQ(w, 5); auto n = mcache.size(); ASSERT_EQ(n, 2); - val = mcache.acquire(1); - ASSERT_TRUE(val == nullptr); - val = mcache.acquire(2); - ASSERT_TRUE(val != nullptr); - val = mcache.acquire(3); - ASSERT_TRUE(val != nullptr); + holder1 = mcache.get(1); + ASSERT_TRUE(holder1 == nullptr); + holder2 = mcache.get(2); + ASSERT_TRUE(holder2 != nullptr); + holder3 = mcache.get(3); + ASSERT_TRUE(holder3 != nullptr); } TEST(LRUResourceCache, evict_on_weight_v2) @@ -89,30 +82,30 @@ TEST(LRUResourceCache, evict_on_weight_v2) auto mcache = MyCache(5, 10); int x = 2; auto load_int = [&] { return std::make_shared(x); }; - auto val = mcache.acquire(1, load_int); - mcache.release(1); + auto holder1 = mcache.getOrSet(1, load_int); + holder1 = nullptr; - val = mcache.acquire(2, load_int); - mcache.release(2); + auto holder2 = mcache.getOrSet(2, load_int); + holder2 = nullptr; - val = mcache.acquire(1); - mcache.release(1); + holder1 = mcache.get(1); + holder1 = nullptr; x = 3; - val = mcache.acquire(3, load_int); - ASSERT_TRUE(val != nullptr); + auto holder3 = mcache.getOrSet(3, load_int); + ASSERT_TRUE(holder3 != nullptr); auto w = mcache.weight(); ASSERT_EQ(w, 5); auto n = mcache.size(); ASSERT_EQ(n, 2); - val = mcache.acquire(1); - ASSERT_TRUE(val != nullptr); - val = mcache.acquire(2); - ASSERT_TRUE(val == nullptr); - val = mcache.acquire(3); - ASSERT_TRUE(val != nullptr); + holder1 = mcache.get(1); + ASSERT_TRUE(holder1 != nullptr); + holder2 = mcache.get(2); + ASSERT_TRUE(holder2 == nullptr); + holder3 = mcache.get(3); + ASSERT_TRUE(holder3 != nullptr); } TEST(LRUResourceCache, evict_on_weight_v3) @@ -121,30 +114,30 @@ TEST(LRUResourceCache, evict_on_weight_v3) auto mcache = MyCache(5, 10); int x = 2; auto load_int = [&] { return std::make_shared(x); }; - auto val = mcache.acquire(1, load_int); - mcache.release(1); + auto holder1 = mcache.getOrSet(1, load_int); + holder1 = nullptr; - val = mcache.acquire(2, load_int); - mcache.release(2); + auto holder2 = mcache.getOrSet(2, load_int); + holder2 = nullptr; - val = mcache.acquire(1, load_int); - mcache.release(1); + holder1 = mcache.getOrSet(1, load_int); + holder1 = nullptr; x = 3; - val = mcache.acquire(3, load_int); - ASSERT_TRUE(val != nullptr); + auto holder3 = mcache.getOrSet(3, load_int); + ASSERT_TRUE(holder3 != nullptr); auto w = mcache.weight(); ASSERT_EQ(w, 5); auto n = mcache.size(); ASSERT_EQ(n, 2); - val = mcache.acquire(1); - ASSERT_TRUE(val != nullptr); - val = mcache.acquire(2); - ASSERT_TRUE(val == nullptr); - val = mcache.acquire(3); - ASSERT_TRUE(val != nullptr); + holder1 = mcache.get(1); + ASSERT_TRUE(holder1 != nullptr); + holder2 = mcache.get(2); + ASSERT_TRUE(holder2 == nullptr); + holder3 = mcache.get(3); + ASSERT_TRUE(holder3 != nullptr); } TEST(LRUResourceCache, evict_on_size) @@ -153,27 +146,27 @@ TEST(LRUResourceCache, evict_on_size) auto mcache = MyCache(5, 2); int x = 2; auto load_int = [&] { return std::make_shared(x); }; - auto val = mcache.acquire(1, load_int); - mcache.release(1); + auto holder1 = mcache.getOrSet(1, load_int); + holder1 = nullptr; - val = mcache.acquire(2, load_int); - mcache.release(2); + auto holder2 = mcache.getOrSet(2, load_int); + holder2 = nullptr; x = 3; - val = mcache.acquire(3, load_int); - ASSERT_TRUE(val != nullptr); + auto holder3 = mcache.getOrSet(3, load_int); + ASSERT_TRUE(holder3 != nullptr); auto n = mcache.size(); ASSERT_EQ(n, 2); auto w = mcache.weight(); ASSERT_EQ(w, 2); - val = mcache.acquire(1); - ASSERT_TRUE(val == nullptr); - val = mcache.acquire(2); - ASSERT_TRUE(val != nullptr); - val = mcache.acquire(3); - ASSERT_TRUE(val != nullptr); + holder1 = mcache.get(1); + ASSERT_TRUE(holder1 == nullptr); + holder2 = mcache.get(2); + ASSERT_TRUE(holder2 != nullptr); + holder3 = mcache.get(3); + ASSERT_TRUE(holder3 != nullptr); } TEST(LRUResourceCache, not_evict_used_element) @@ -182,95 +175,95 @@ TEST(LRUResourceCache, not_evict_used_element) auto mcache = MyCache(7, 10); int x = 2; auto load_int = [&] { return std::make_shared(x); }; - auto val = mcache.acquire(1, load_int); + auto holder1 = mcache.getOrSet(1, load_int); - val = mcache.acquire(2, load_int); - mcache.release(2); + auto holder2 = mcache.getOrSet(2, load_int); + holder2 = nullptr; - val = mcache.acquire(3, load_int); - mcache.release(3); + auto holder3 = mcache.getOrSet(3, load_int); + holder3 = nullptr; x = 3; - val = mcache.acquire(4, load_int); - ASSERT_TRUE(val != nullptr); + auto holder4 = mcache.getOrSet(4, load_int); + ASSERT_TRUE(holder4 != nullptr); auto n = mcache.size(); ASSERT_EQ(n, 3); auto w = mcache.weight(); ASSERT_EQ(w, 7); - val = mcache.acquire(1); - ASSERT_TRUE(val != nullptr); - val = mcache.acquire(2); - ASSERT_TRUE(val == nullptr); - val = mcache.acquire(3); - ASSERT_TRUE(val != nullptr); - val = mcache.acquire(4); - ASSERT_TRUE(val != nullptr); + holder1 = mcache.get(1); + ASSERT_TRUE(holder1 != nullptr); + holder2 = mcache.get(2); + ASSERT_TRUE(holder2 == nullptr); + holder3 = mcache.get(3); + ASSERT_TRUE(holder3 != nullptr); + holder4 = mcache.get(4); + ASSERT_TRUE(holder4 != nullptr); } -TEST(LRUResourceCache, acquire_fail) +TEST(LRUResourceCache, get_fail) { using MyCache = DB::LRUResourceCache; auto mcache = MyCache(5, 10); int x = 2; auto load_int = [&] { return std::make_shared(x); }; - auto val = mcache.acquire(1, load_int); - val = mcache.acquire(2, load_int); - val = mcache.acquire(3, load_int); - ASSERT_TRUE(val == nullptr); + auto holder1 = mcache.getOrSet(1, load_int); + auto holder2 = mcache.getOrSet(2, load_int); + auto holder3 = mcache.getOrSet(3, load_int); + ASSERT_TRUE(holder3 == nullptr); auto n = mcache.size(); ASSERT_EQ(n, 2); auto w = mcache.weight(); ASSERT_EQ(w, 4); - val = mcache.acquire(1); - ASSERT_TRUE(val != nullptr); - val = mcache.acquire(2); - ASSERT_TRUE(val != nullptr); - val = mcache.acquire(3); - ASSERT_TRUE(val == nullptr); + holder1 = mcache.get(1); + ASSERT_TRUE(holder1 != nullptr); + holder2 = mcache.get(2); + ASSERT_TRUE(holder2 != nullptr); + holder3 = mcache.get(3); + ASSERT_TRUE(holder3 == nullptr); } -TEST(LRUResourceCache, dup_acquire) +TEST(LRUResourceCache, dup_get) { using MyCache = DB::LRUResourceCache; auto mcache = MyCache(20, 10); int x = 2; auto load_int = [&] { return std::make_shared(x); }; - auto val = mcache.acquire(1, load_int); - mcache.release(1); + auto holder1 = mcache.getOrSet(1, load_int); + holder1 = nullptr; x = 11; - val = mcache.acquire(1, load_int); - ASSERT_TRUE(val != nullptr); + holder1 = mcache.getOrSet(1, load_int); + ASSERT_TRUE(holder1 != nullptr); auto n = mcache.size(); ASSERT_EQ(n, 1); auto w = mcache.weight(); ASSERT_EQ(w, 2); - val = mcache.acquire(1); - ASSERT_TRUE(val != nullptr); - ASSERT_TRUE(*val == 2); + holder1 = mcache.get(1); + ASSERT_TRUE(holder1 != nullptr); + ASSERT_TRUE(holder1->value() == 2); } -TEST(LRUResourceCache, re_acquire) +TEST(LRUResourceCache, re_get) { using MyCache = DB::LRUResourceCache; auto mcache = MyCache(20, 10); int x = 2; auto load_int = [&] { return std::make_shared(x); }; - auto val = mcache.acquire(1, load_int); - mcache.release(1); - mcache.tryRemove(1); + auto holder1 = mcache.getOrSet(1, load_int); + MyCache::MappedHolder::tryRemove(&holder1); + x = 11; - val = mcache.acquire(1, load_int); - ASSERT_TRUE(val != nullptr); + holder1 = mcache.getOrSet(1, load_int); + ASSERT_TRUE(holder1 != nullptr); auto n = mcache.size(); ASSERT_EQ(n, 1); auto w = mcache.weight(); ASSERT_EQ(w, 11); - val = mcache.acquire(1); - ASSERT_TRUE(val != nullptr); - ASSERT_TRUE(*val == 11); + holder1 = mcache.get(1); + ASSERT_TRUE(holder1 != nullptr); + ASSERT_TRUE(holder1->value() == 11); } From a41273826448b6b278d5d6d2214de78005b42be5 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 29 Dec 2021 18:27:52 +0800 Subject: [PATCH 0582/1260] format code style --- src/Common/LRUResourceCache.h | 22 +++++++--------------- 1 file changed, 7 insertions(+), 15 deletions(-) diff --git a/src/Common/LRUResourceCache.h b/src/Common/LRUResourceCache.h index d33280add25..3dd078bec5a 100644 --- a/src/Common/LRUResourceCache.h +++ b/src/Common/LRUResourceCache.h @@ -38,15 +38,8 @@ public: class MappedHolder { public: - ~MappedHolder() - { - cache->release(key); - - } - Mapped & value() - { - return *(val.get()); - } + ~MappedHolder() { cache->release(key); } + Mapped & value() { return *(val.get()); } static bool tryRemove(std::unique_ptr * holder_ptr) { auto & holder = *holder_ptr; @@ -56,9 +49,8 @@ public: return cache->tryRemove(key); } - MappedHolder(LRUResourceCache * cache_, const Key & key_, MappedPtr value_) : cache(cache_), key(key_), val(value_) - { - } + MappedHolder(LRUResourceCache * cache_, const Key & key_, MappedPtr value_) : cache(cache_), key(key_), val(value_) { } + protected: LRUResourceCache * cache; Key key; @@ -74,7 +66,7 @@ public: return nullptr; return std::make_unique(this, key, mappedptr); } - template + template MappedHolderPtr getOrSet(const Key & key, LoadFunc && load_func) { auto mappedptr = getImpl(key, load_func); @@ -138,7 +130,7 @@ private: std::atomic hits{0}; std::atomic misses{0}; std::atomic evict_count{0}; - + // - load_func : when key is not exists in cache, load_func is called to generate a new key // - return: is null when there is no more space for the new value or the old value is in used. template @@ -283,7 +275,7 @@ private: new_cell.queue_iterator = queue.insert(queue.end(), insert_key); return &new_cell; } - + // If you want to update a value, call tryRemove() at first and then call acquire() with load_func. bool tryRemove(const Key & key) { From 154e7b03c0587228d3a4add37684d0559178a38d Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 29 Dec 2021 13:34:54 +0300 Subject: [PATCH 0583/1260] Disable FunctionConvertFromString::canBeExecutedOnDefaultArguments --- src/Functions/FunctionsConversion.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 8018fa8e726..62e62b5f5dc 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1835,6 +1835,8 @@ public: size_t getNumberOfArguments() const override { return 0; } bool useDefaultImplementationForConstants() const override { return true; } + bool canBeExecutedOnDefaultArguments() const override { return false; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override From b78f5c70b5a3a6296b9f39b8d17d9d3fbbbbb4a2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 29 Dec 2021 13:41:13 +0300 Subject: [PATCH 0584/1260] Add test parse_date_lowcard_default_throw, issue #31979 --- .../02155_parse_date_lowcard_default_throw.reference | 1 + .../0_stateless/02155_parse_date_lowcard_default_throw.sql | 4 ++++ 2 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/02155_parse_date_lowcard_default_throw.reference create mode 100644 tests/queries/0_stateless/02155_parse_date_lowcard_default_throw.sql diff --git a/tests/queries/0_stateless/02155_parse_date_lowcard_default_throw.reference b/tests/queries/0_stateless/02155_parse_date_lowcard_default_throw.reference new file mode 100644 index 00000000000..e599dcc71e5 --- /dev/null +++ b/tests/queries/0_stateless/02155_parse_date_lowcard_default_throw.reference @@ -0,0 +1 @@ +2016-07-15 00:00:00 diff --git a/tests/queries/0_stateless/02155_parse_date_lowcard_default_throw.sql b/tests/queries/0_stateless/02155_parse_date_lowcard_default_throw.sql new file mode 100644 index 00000000000..c2a261c77bc --- /dev/null +++ b/tests/queries/0_stateless/02155_parse_date_lowcard_default_throw.sql @@ -0,0 +1,4 @@ +SELECT parseDateTimeBestEffort(q0.date_field) AS parsed_date +FROM (SELECT 1 AS pk1) AS t1 +INNER JOIN ( SELECT 1 AS pk1, toLowCardinality('15-JUL-16') AS date_field ) AS q0 +ON q0.pk1 = t1.pk1; From 3ec0401e5c9f387c2b8d4ad39a22dfdeef22f6f3 Mon Sep 17 00:00:00 2001 From: benbiti Date: Mon, 2 Aug 2021 20:25:15 +0800 Subject: [PATCH 0585/1260] asynchronous_metrics.md in zh --- .../system-tables/asynchronous_metrics.md | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/docs/zh/operations/system-tables/asynchronous_metrics.md b/docs/zh/operations/system-tables/asynchronous_metrics.md index d6d2682c9a1..0303c408497 100644 --- a/docs/zh/operations/system-tables/asynchronous_metrics.md +++ b/docs/zh/operations/system-tables/asynchronous_metrics.md @@ -1,8 +1,3 @@ ---- -machine_translated: true -machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 ---- - # system.asynchronous_metrics {#system_tables-asynchronous_metrics} 包含在后台定期计算的指标。 例如,在使用的RAM量。 @@ -33,8 +28,8 @@ SELECT * FROM system.asynchronous_metrics LIMIT 10 └─────────────────────────────────────────┴────────────┘ ``` -**另请参阅** -- [监测](../../operations/monitoring.md) — ClickHouse监控的基本概念。 -- [系统。指标](../../operations/system-tables/metrics.md#system_tables-metrics) — 包含即时计算的指标。 -- [系统。活动](../../operations/system-tables/events.md#system_tables-events) — 包含出现的事件的次数。 -- [系统。metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — 包含`system.metrics` 和 `system.events`表中的指标的历史值。 +**参见** +- [监控](../../operations/monitoring.md) — ClickHouse监控的基本概念。 +- [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) — 包含即时计算的指标。 +- [system.events](../../operations/system-tables/events.md#system_tables-events) — 包含已发生的事件数。 +- [system.metric_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — 包含 `system.metrics` 和 `system.events` 表中的指标的历史值。 From c8a79d61e8bd2ed2cfae643d185d556d28f2b284 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 29 Dec 2021 15:06:27 +0300 Subject: [PATCH 0586/1260] Simplify case in test parse_date_lowcard_default_throw --- .../0_stateless/02155_parse_date_lowcard_default_throw.sql | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02155_parse_date_lowcard_default_throw.sql b/tests/queries/0_stateless/02155_parse_date_lowcard_default_throw.sql index c2a261c77bc..703cf1fed7a 100644 --- a/tests/queries/0_stateless/02155_parse_date_lowcard_default_throw.sql +++ b/tests/queries/0_stateless/02155_parse_date_lowcard_default_throw.sql @@ -1,4 +1 @@ -SELECT parseDateTimeBestEffort(q0.date_field) AS parsed_date -FROM (SELECT 1 AS pk1) AS t1 -INNER JOIN ( SELECT 1 AS pk1, toLowCardinality('15-JUL-16') AS date_field ) AS q0 -ON q0.pk1 = t1.pk1; +SELECT parseDateTimeBestEffort(toLowCardinality(materialize('15-JUL-16'))); From 754e258ab4e36b74a214c3ed280a52b3946803e7 Mon Sep 17 00:00:00 2001 From: benbiti Date: Wed, 29 Dec 2021 20:10:49 +0800 Subject: [PATCH 0587/1260] asynchronous_metric_log.md --- .../system-tables/asynchronous_metric_log.md | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/docs/zh/operations/system-tables/asynchronous_metric_log.md b/docs/zh/operations/system-tables/asynchronous_metric_log.md index 592fb99c5ef..ba37713ac44 100644 --- a/docs/zh/operations/system-tables/asynchronous_metric_log.md +++ b/docs/zh/operations/system-tables/asynchronous_metric_log.md @@ -1,18 +1,13 @@ ---- -machine_translated: true -machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 ---- - ## system.asynchronous_metric_log {#system-tables-async-log} -包含每分钟记录一次的 `system.asynchronous_metrics`历史值. 默认开启. +包含每分钟记录一次的 `system.asynchronous_metrics`历史值。默认开启。 列: -- `event_date` ([Date](../../sql-reference/data-types/date.md)) — 事件日期. -- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — 事件时间. -- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — 事件时间(微秒). -- `name` ([String](../../sql-reference/data-types/string.md)) — 指标名. -- `value` ([Float64](../../sql-reference/data-types/float.md)) — 指标值. +- `event_date` ([Date](../../sql-reference/data-types/date.md)) — 事件日期。 +- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — 事件时间。 +- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — 事件时间(微秒)。 +- `name` ([String](../../sql-reference/data-types/string.md)) — 指标名。 +- `value` ([Float64](../../sql-reference/data-types/float.md)) — 指标值。 **示例** ``` sql From d35e5f831969c1647798bc19ca42a7b3dc4f0787 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 29 Dec 2021 20:59:01 +0800 Subject: [PATCH 0588/1260] add lower bound session timeout to keeper --- docs/en/operations/clickhouse-keeper.md | 3 ++- programs/keeper/keeper_config.xml | 3 ++- src/Common/ZooKeeper/ZooKeeperConstants.h | 4 +++- src/Coordination/CoordinationSettings.cpp | 6 ++++-- src/Coordination/CoordinationSettings.h | 3 ++- src/Server/KeeperTCPHandler.cpp | 14 ++++++++++---- src/Server/KeeperTCPHandler.h | 2 ++ tests/config/config.d/keeper_port.xml | 3 ++- tests/integration/helpers/keeper_config1.xml | 2 +- tests/integration/helpers/keeper_config2.xml | 2 +- tests/integration/helpers/keeper_config3.xml | 2 +- .../test_keeper_auth/configs/keeper_config.xml | 2 +- .../configs/enable_keeper.xml | 2 +- .../configs/enable_keeper.xml | 2 +- .../configs/enable_keeper1.xml | 3 ++- .../configs/enable_keeper2.xml | 3 ++- .../configs/enable_keeper3.xml | 3 ++- .../test_keeper_four_word_command/test.py | 3 ++- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_secure_keeper1.xml | 2 +- .../configs/enable_secure_keeper2.xml | 2 +- .../configs/enable_secure_keeper3.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper3.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper3.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper_three_nodes_1.xml | 2 +- .../configs/enable_keeper_three_nodes_2.xml | 2 +- .../configs/enable_keeper_three_nodes_3.xml | 2 +- .../configs/enable_keeper_two_nodes_1.xml | 2 +- .../configs/enable_keeper_two_nodes_2.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper3.xml | 2 +- .../configs/enable_keeper_node4_1.xml | 2 +- .../configs/enable_keeper_node4_2.xml | 2 +- .../configs/enable_keeper_node4_4.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper3.xml | 2 +- .../configs/enable_keeper_two_nodes_1.xml | 2 +- .../configs/enable_keeper_two_nodes_2.xml | 2 +- .../configs/enable_single_keeper1.xml | 2 +- .../configs/enable_keeper.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper3.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper_two_nodes_1.xml | 2 +- .../configs/enable_keeper_two_nodes_2.xml | 2 +- .../configs/enable_keeper_two_nodes_3.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper3.xml | 2 +- .../configs/enable_secure_keeper.xml | 2 +- .../configs/keeper_config1.xml | 2 +- .../configs/keeper_config2.xml | 2 +- .../configs/keeper_config3.xml | 2 +- .../configs/enable_keeper.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper3.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper3.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper3.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/keeper_config.xml | 2 +- 77 files changed, 100 insertions(+), 80 deletions(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 6738f77cff9..624e7fdad28 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -36,7 +36,8 @@ Other common parameters are inherited from the ClickHouse server config (`listen Internal coordination settings are located in `.` section: - `operation_timeout_ms` — Timeout for a single client operation (ms) (default: 10000). -- `session_timeout_ms` — Timeout for client session (ms) (default: 30000). +- `min_session_timeout_ms` — Min timeout for client session (ms) (default: 10000). +- `max_session_timeout_ms` — Max timeout for client session (ms) (default: 30000). - `dead_session_check_period_ms` — How often ClickHouse Keeper check dead sessions and remove them (ms) (default: 500). - `heart_beat_interval_ms` — How often a ClickHouse Keeper leader will send heartbeats to followers (ms) (default: 500). - `election_timeout_lower_bound_ms` — If the follower didn't receive heartbeats from the leader in this interval, then it can initiate leader election (default: 1000). diff --git a/programs/keeper/keeper_config.xml b/programs/keeper/keeper_config.xml index 8b4d4274e6a..5dc550e5245 100644 --- a/programs/keeper/keeper_config.xml +++ b/programs/keeper/keeper_config.xml @@ -38,7 +38,8 @@ 10000 - 30000 + 10000 + 30000 information diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.h b/src/Common/ZooKeeper/ZooKeeperConstants.h index ed7afd83628..425fc9941c9 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.h +++ b/src/Common/ZooKeeper/ZooKeeperConstants.h @@ -47,7 +47,9 @@ static constexpr int32_t PASSWORD_LENGTH = 16; /// ZooKeeper has 1 MB node size and serialization limit by default, /// but it can be raised up, so we have a slightly larger limit on our side. static constexpr int32_t MAX_STRING_OR_ARRAY_SIZE = 1 << 28; /// 256 MiB -static constexpr int32_t DEFAULT_SESSION_TIMEOUT_MS = 30000; +static constexpr int32_t DEFAULT_SESSION_TIMEOUT_MS = 10000; +static constexpr int32_t DEFAULT_MIN_SESSION_TIMEOUT_MS = 10000; +static constexpr int32_t DEFAULT_MAX_SESSION_TIMEOUT_MS = 30000; static constexpr int32_t DEFAULT_OPERATION_TIMEOUT_MS = 10000; } diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index bb160c98402..31ccfdbc8e3 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -94,8 +94,10 @@ void KeeperConfigurationAndSettings::dump(WriteBufferFromOwnString & buf) const writeText("max_requests_batch_size=", buf); write_int(coordination_settings->max_requests_batch_size); - writeText("session_timeout_ms=", buf); - write_int(uint64_t(coordination_settings->session_timeout_ms)); + writeText("min_session_timeout_ms=", buf); + write_int(uint64_t(coordination_settings->min_session_timeout_ms)); + writeText("max_session_timeout_ms=", buf); + write_int(uint64_t(coordination_settings->max_session_timeout_ms)); writeText("operation_timeout_ms=", buf); write_int(uint64_t(coordination_settings->operation_timeout_ms)); writeText("dead_session_check_period_ms=", buf); diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 9cf566f95df..f653d64017e 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -19,7 +19,8 @@ struct Settings; #define LIST_OF_COORDINATION_SETTINGS(M) \ - M(Milliseconds, session_timeout_ms, Coordination::DEFAULT_SESSION_TIMEOUT_MS, "Default client session timeout", 0) \ + M(Milliseconds, min_session_timeout_ms, Coordination::DEFAULT_MIN_SESSION_TIMEOUT_MS, "Min client session timeout", 0) \ + M(Milliseconds, max_session_timeout_ms, Coordination::DEFAULT_MAX_SESSION_TIMEOUT_MS, "Max client session timeout", 0) \ M(Milliseconds, operation_timeout_ms, Coordination::DEFAULT_OPERATION_TIMEOUT_MS, "Default client operation timeout", 0) \ M(Milliseconds, dead_session_check_period_ms, 500, "How often leader will check sessions to consider them dead and remove", 0) \ M(Milliseconds, heart_beat_interval_ms, 500, "Heartbeat interval between quorum nodes", 0) \ diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 0c5d7d93689..3d66113c92f 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -212,10 +212,14 @@ KeeperTCPHandler::KeeperTCPHandler(IServer & server_, const Poco::Net::StreamSoc 0, global_context->getConfigRef().getUInt( "keeper_server.coordination_settings.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000) - , session_timeout( + , min_session_timeout( 0, global_context->getConfigRef().getUInt( - "keeper_server.coordination_settings.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000) + "keeper_server.coordination_settings.min_session_timeout_ms", Coordination::DEFAULT_MIN_SESSION_TIMEOUT_MS) * 1000) + , max_session_timeout( + 0, + global_context->getConfigRef().getUInt( + "keeper_server.coordination_settings.max_session_timeout_ms", Coordination::DEFAULT_MAX_SESSION_TIMEOUT_MS) * 1000) , poll_wrapper(std::make_unique(socket_)) , responses(std::make_unique(std::numeric_limits::max())) , last_op(std::make_unique(EMPTY_LAST_OP)) @@ -320,8 +324,10 @@ void KeeperTCPHandler::runImpl() int32_t handshake_length = header; auto client_timeout = receiveHandshake(handshake_length); - if (client_timeout != 0) - session_timeout = std::min(client_timeout, session_timeout); + if (client_timeout == 0) + client_timeout = Coordination::DEFAULT_SESSION_TIMEOUT_MS; + session_timeout = std::max(client_timeout, min_session_timeout); + session_timeout = std::min(client_timeout, max_session_timeout); } catch (const Exception & e) /// Typical for an incorrect username, password, or address. { diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index f98b269b8be..7953dfd2cbe 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -63,6 +63,8 @@ private: ContextPtr global_context; std::shared_ptr keeper_dispatcher; Poco::Timespan operation_timeout; + Poco::Timespan min_session_timeout; + Poco::Timespan max_session_timeout; Poco::Timespan session_timeout; int64_t session_id{-1}; Stopwatch session_stopwatch; diff --git a/tests/config/config.d/keeper_port.xml b/tests/config/config.d/keeper_port.xml index 21163ece190..567179774ce 100644 --- a/tests/config/config.d/keeper_port.xml +++ b/tests/config/config.d/keeper_port.xml @@ -5,7 +5,8 @@ 10000 - 30000 + 10000 + 30000 false 240000 diff --git a/tests/integration/helpers/keeper_config1.xml b/tests/integration/helpers/keeper_config1.xml index 4b7e43fbc2a..27660460b40 100644 --- a/tests/integration/helpers/keeper_config1.xml +++ b/tests/integration/helpers/keeper_config1.xml @@ -15,7 +15,7 @@ 10000 - 30000 + 30000 trace false diff --git a/tests/integration/helpers/keeper_config2.xml b/tests/integration/helpers/keeper_config2.xml index e34149469ed..9734fb9e8ec 100644 --- a/tests/integration/helpers/keeper_config2.xml +++ b/tests/integration/helpers/keeper_config2.xml @@ -15,7 +15,7 @@ 10000 - 30000 + 30000 trace false diff --git a/tests/integration/helpers/keeper_config3.xml b/tests/integration/helpers/keeper_config3.xml index 8001cea44d6..2a536e764f3 100644 --- a/tests/integration/helpers/keeper_config3.xml +++ b/tests/integration/helpers/keeper_config3.xml @@ -15,7 +15,7 @@ 10000 - 30000 + 30000 trace false diff --git a/tests/integration/test_keeper_auth/configs/keeper_config.xml b/tests/integration/test_keeper_auth/configs/keeper_config.xml index 926cdd3a0f9..eb871498bbe 100644 --- a/tests/integration/test_keeper_auth/configs/keeper_config.xml +++ b/tests/integration/test_keeper_auth/configs/keeper_config.xml @@ -8,7 +8,7 @@ 5000 - 10000 + 10000 trace 75 diff --git a/tests/integration/test_keeper_back_to_back/configs/enable_keeper.xml b/tests/integration/test_keeper_back_to_back/configs/enable_keeper.xml index 9649117e4b1..ca29dc7d74d 100644 --- a/tests/integration/test_keeper_back_to_back/configs/enable_keeper.xml +++ b/tests/integration/test_keeper_back_to_back/configs/enable_keeper.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace false diff --git a/tests/integration/test_keeper_clickhouse_hard_restart/configs/enable_keeper.xml b/tests/integration/test_keeper_clickhouse_hard_restart/configs/enable_keeper.xml index c1d38a1de52..ac78962c78f 100644 --- a/tests/integration/test_keeper_clickhouse_hard_restart/configs/enable_keeper.xml +++ b/tests/integration/test_keeper_clickhouse_hard_restart/configs/enable_keeper.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml b/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml index fcb3553bb98..580d3bd3e03 100644 --- a/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml @@ -8,7 +8,8 @@ 5000 - 30000 + 30000 + 1000 75 trace diff --git a/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml b/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml index 65011cd2637..35b378670b7 100644 --- a/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml @@ -8,7 +8,8 @@ 5000 - 30000 + 30000 + 1000 75 trace diff --git a/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml b/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml index a0d8c99f3d8..e896dd4b454 100644 --- a/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml @@ -8,7 +8,8 @@ 5000 - 30000 + 30000 + 1000 75 trace diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 6ea12342044..0824270a73b 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -285,7 +285,8 @@ def test_cmd_conf(started_cluster): assert result["log_storage_path"] == "/var/lib/clickhouse/coordination/log" assert result["snapshot_storage_path"] == "/var/lib/clickhouse/coordination/snapshots" - assert result["session_timeout_ms"] == "30000" + assert result["max_session_timeout_ms"] == "30000" + assert result["min_session_timeout_ms"] == "1000" assert result["operation_timeout_ms"] == "5000" assert result["dead_session_check_period_ms"] == "500" assert result["heart_beat_interval_ms"] == "500" diff --git a/tests/integration/test_keeper_incorrect_config/configs/enable_keeper1.xml b/tests/integration/test_keeper_incorrect_config/configs/enable_keeper1.xml index c1d38a1de52..ac78962c78f 100644 --- a/tests/integration/test_keeper_incorrect_config/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_incorrect_config/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper1.xml b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper1.xml index 986b503ebe3..540a5bdf8f0 100644 --- a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper1.xml +++ b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper2.xml b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper2.xml index 652b1992f46..752a7171e45 100644 --- a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper2.xml +++ b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper3.xml b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper3.xml index 6507f97473b..365dce589b1 100644 --- a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper3.xml +++ b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper1.xml b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper1.xml index 17455ed12f5..9b93bb6197d 100644 --- a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper2.xml b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper2.xml index 03a23984cc2..23faef0ba14 100644 --- a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper3.xml b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper3.xml index a3196ac3061..c7a89a4a6b2 100644 --- a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_multinode_simple/configs/enable_keeper1.xml b/tests/integration/test_keeper_multinode_simple/configs/enable_keeper1.xml index 17455ed12f5..9b93bb6197d 100644 --- a/tests/integration/test_keeper_multinode_simple/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_multinode_simple/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_multinode_simple/configs/enable_keeper2.xml b/tests/integration/test_keeper_multinode_simple/configs/enable_keeper2.xml index 03a23984cc2..23faef0ba14 100644 --- a/tests/integration/test_keeper_multinode_simple/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_multinode_simple/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_multinode_simple/configs/enable_keeper3.xml b/tests/integration/test_keeper_multinode_simple/configs/enable_keeper3.xml index a3196ac3061..c7a89a4a6b2 100644 --- a/tests/integration/test_keeper_multinode_simple/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_multinode_simple/configs/enable_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper1.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper1.xml index c1d38a1de52..ac78962c78f 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_1.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_1.xml index d2717283a8d..83dd49b2744 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_1.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_2.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_2.xml index 5924ee1c2dc..c9847d93f14 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_2.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_3.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_3.xml index d261e4f67f3..83f80395dec 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_3.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_1.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_1.xml index 697986638d7..3d1147830de 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_1.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_2.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_2.xml index 967940e1e2b..307f356f38e 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_2.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper1.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper1.xml index 1e57d42016d..f2035082d05 100644 --- a/tests/integration/test_keeper_nodes_move/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper2.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper2.xml index 98422b41c9b..eb1c0574d7a 100644 --- a/tests/integration/test_keeper_nodes_move/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper3.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper3.xml index 43800bd2dfb..f4e114af2e5 100644 --- a/tests/integration/test_keeper_nodes_move/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_1.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_1.xml index 0d7544f9a5b..185150606c7 100644 --- a/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_1.xml +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_2.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_2.xml index 65feae85e3e..de2fbdca8c0 100644 --- a/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_2.xml +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_4.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_4.xml index 2499de4fe86..5ef9fde8a0b 100644 --- a/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_4.xml +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_4.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper1.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper1.xml index 1e57d42016d..f2035082d05 100644 --- a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper2.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper2.xml index 98422b41c9b..eb1c0574d7a 100644 --- a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper3.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper3.xml index 43800bd2dfb..f4e114af2e5 100644 --- a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_1.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_1.xml index 697986638d7..3d1147830de 100644 --- a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_1.xml +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_2.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_2.xml index 967940e1e2b..307f356f38e 100644 --- a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_2.xml +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_single_keeper1.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_single_keeper1.xml index c1d38a1de52..ac78962c78f 100644 --- a/tests/integration/test_keeper_nodes_remove/configs/enable_single_keeper1.xml +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_single_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_persistent_log/configs/enable_keeper.xml b/tests/integration/test_keeper_persistent_log/configs/enable_keeper.xml index d6166a83d2f..58a7d1260e3 100644 --- a/tests/integration/test_keeper_persistent_log/configs/enable_keeper.xml +++ b/tests/integration/test_keeper_persistent_log/configs/enable_keeper.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper1.xml b/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper1.xml index 06b70c1b5d0..ac8a1773f81 100644 --- a/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper2.xml b/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper2.xml index f13f8b902b4..7ad000018e0 100644 --- a/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper3.xml b/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper3.xml index a69cabf8c54..c33b31c75e3 100644 --- a/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_remove_leader/configs/enable_keeper1.xml b/tests/integration/test_keeper_remove_leader/configs/enable_keeper1.xml index 1e57d42016d..f2035082d05 100644 --- a/tests/integration/test_keeper_remove_leader/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_remove_leader/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_remove_leader/configs/enable_keeper2.xml b/tests/integration/test_keeper_remove_leader/configs/enable_keeper2.xml index 98422b41c9b..eb1c0574d7a 100644 --- a/tests/integration/test_keeper_remove_leader/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_remove_leader/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_1.xml b/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_1.xml index d51e420f733..ecf7f56118f 100644 --- a/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_1.xml +++ b/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_2.xml b/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_2.xml index 3f1ee1e01a8..3974d5fcb06 100644 --- a/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_2.xml +++ b/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_3.xml b/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_3.xml index a99bd5d5296..b460ab2df45 100644 --- a/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_3.xml +++ b/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper1.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper1.xml index 4ea543e6f31..3eff33fc221 100644 --- a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace 100 10 diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper2.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper2.xml index 4bf3083c1fa..e6453a1ab01 100644 --- a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace 100 10 diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper3.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper3.xml index b9e2a2d0422..837a459040f 100644 --- a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace 100 10 diff --git a/tests/integration/test_keeper_secure_client/configs/enable_secure_keeper.xml b/tests/integration/test_keeper_secure_client/configs/enable_secure_keeper.xml index 2c144269bcc..a21851aca2b 100644 --- a/tests/integration/test_keeper_secure_client/configs/enable_secure_keeper.xml +++ b/tests/integration/test_keeper_secure_client/configs/enable_secure_keeper.xml @@ -8,7 +8,7 @@ 10000 - 30000 + 30000 trace false diff --git a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config1.xml b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config1.xml index ed5909d01eb..d605cd92ae3 100644 --- a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config1.xml +++ b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config1.xml @@ -9,7 +9,7 @@ 75 5 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config2.xml b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config2.xml index 896853a3713..17a5ef859b1 100644 --- a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config2.xml +++ b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config2.xml @@ -9,7 +9,7 @@ 75 5 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config3.xml b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config3.xml index 8d1d5c73dd8..22a212cced1 100644 --- a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config3.xml +++ b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config3.xml @@ -9,7 +9,7 @@ 75 5 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_snapshots/configs/enable_keeper.xml b/tests/integration/test_keeper_snapshots/configs/enable_keeper.xml index a3217b34501..00a90409d24 100644 --- a/tests/integration/test_keeper_snapshots/configs/enable_keeper.xml +++ b/tests/integration/test_keeper_snapshots/configs/enable_keeper.xml @@ -9,7 +9,7 @@ 10 5 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper1.xml b/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper1.xml index 27d0d38f596..0cbcadcbb6e 100644 --- a/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper1.xml @@ -9,7 +9,7 @@ 10 5 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper2.xml b/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper2.xml index dedbd312544..87f460c8d04 100644 --- a/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper2.xml @@ -9,7 +9,7 @@ 10 5 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper3.xml b/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper3.xml index fd36458b5eb..596be35a119 100644 --- a/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper3.xml @@ -9,7 +9,7 @@ 10 5 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper1.xml b/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper1.xml index 1e57d42016d..f2035082d05 100644 --- a/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper2.xml b/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper2.xml index 98422b41c9b..eb1c0574d7a 100644 --- a/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper3.xml b/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper3.xml index 43800bd2dfb..f4e114af2e5 100644 --- a/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper1.xml b/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper1.xml index d2159348588..e6637613380 100644 --- a/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper2.xml b/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper2.xml index 0d9aeb191d9..041148b7ef4 100644 --- a/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper1.xml b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper1.xml index d2717283a8d..83dd49b2744 100644 --- a/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper2.xml b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper2.xml index 5924ee1c2dc..c9847d93f14 100644 --- a/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper3.xml b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper3.xml index d261e4f67f3..83f80395dec 100644 --- a/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_two_nodes_cluster/configs/enable_keeper1.xml b/tests/integration/test_keeper_two_nodes_cluster/configs/enable_keeper1.xml index 7c2e283e89f..757ba52fa4f 100644 --- a/tests/integration/test_keeper_two_nodes_cluster/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_two_nodes_cluster/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_two_nodes_cluster/configs/enable_keeper2.xml b/tests/integration/test_keeper_two_nodes_cluster/configs/enable_keeper2.xml index 618e6a04aec..927354cb6af 100644 --- a/tests/integration/test_keeper_two_nodes_cluster/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_two_nodes_cluster/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_zookeeper_converter/configs/keeper_config.xml b/tests/integration/test_keeper_zookeeper_converter/configs/keeper_config.xml index 9b50f2c6c41..9f6927a6c11 100644 --- a/tests/integration/test_keeper_zookeeper_converter/configs/keeper_config.xml +++ b/tests/integration/test_keeper_zookeeper_converter/configs/keeper_config.xml @@ -7,7 +7,7 @@ 30000 - 600000 + 600000 trace 75 From ad7f206840a81caeb8a0fb3a9fbd5c4661474fbd Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 29 Dec 2021 21:05:27 +0800 Subject: [PATCH 0589/1260] change max_session_timeout_ms default value from 30000 to 100000 --- docs/en/operations/clickhouse-keeper.md | 2 +- src/Common/ZooKeeper/ZooKeeperConstants.h | 2 +- tests/config/config.d/keeper_port.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 624e7fdad28..b07867fb240 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -37,7 +37,7 @@ Internal coordination settings are located in `. 10000 10000 - 30000 + 100000 false 240000 From e75e4b77e7dd6f863ede1ca3b7944510f9a49c5e Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 28 Dec 2021 14:57:19 -0400 Subject: [PATCH 0590/1260] test for #24410 --- .../0_stateless/02155_create_table_w_timezone.reference | 0 .../queries/0_stateless/02155_create_table_w_timezone.sql | 8 ++++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/02155_create_table_w_timezone.reference create mode 100644 tests/queries/0_stateless/02155_create_table_w_timezone.sql diff --git a/tests/queries/0_stateless/02155_create_table_w_timezone.reference b/tests/queries/0_stateless/02155_create_table_w_timezone.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02155_create_table_w_timezone.sql b/tests/queries/0_stateless/02155_create_table_w_timezone.sql new file mode 100644 index 00000000000..cd187cc85a4 --- /dev/null +++ b/tests/queries/0_stateless/02155_create_table_w_timezone.sql @@ -0,0 +1,8 @@ +create table 02155_t64_tz ( a DateTime64(9, America/Chicago)) Engine = Memory; -- { serverError 62 } +create table 02155_t_tz ( a DateTime(America/Chicago)) Engine = Memory; -- { serverError 62 } + +create table 02155_t64_tz ( a DateTime64(9, 'America/Chicago')) Engine = Memory; +create table 02155_t_tz ( a DateTime('America/Chicago')) Engine = Memory; + +drop table 02155_t64_tz; +drop table 02155_t_tz; From d19de98f0742575131082a72b6d2c4b50c6c5609 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 28 Dec 2021 20:09:55 -0400 Subject: [PATCH 0591/1260] Update 02155_create_table_w_timezone.sql --- .../0_stateless/02155_create_table_w_timezone.sql | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02155_create_table_w_timezone.sql b/tests/queries/0_stateless/02155_create_table_w_timezone.sql index cd187cc85a4..0b72122ce39 100644 --- a/tests/queries/0_stateless/02155_create_table_w_timezone.sql +++ b/tests/queries/0_stateless/02155_create_table_w_timezone.sql @@ -1,8 +1,8 @@ -create table 02155_t64_tz ( a DateTime64(9, America/Chicago)) Engine = Memory; -- { serverError 62 } -create table 02155_t_tz ( a DateTime(America/Chicago)) Engine = Memory; -- { serverError 62 } +create table t02155_t64_tz ( a DateTime64(9, America/Chicago)) Engine = Memory; -- { clientError 62 } +create table t02155_t_tz ( a DateTime(America/Chicago)) Engine = Memory; -- { clientError 62 } -create table 02155_t64_tz ( a DateTime64(9, 'America/Chicago')) Engine = Memory; -create table 02155_t_tz ( a DateTime('America/Chicago')) Engine = Memory; +create table t02155_t64_tz ( a DateTime64(9, 'America/Chicago')) Engine = Memory; +create table t02155_t_tz ( a DateTime('America/Chicago')) Engine = Memory; -drop table 02155_t64_tz; -drop table 02155_t_tz; +drop table t02155_t64_tz; +drop table t02155_t_tz; From 7e6cd4adbbf11aceedfd07ed843425dfc0fc6cd8 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 29 Dec 2021 21:25:55 +0800 Subject: [PATCH 0592/1260] add tests --- .../configs/keeper_config.xml | 27 +++++++ tests/integration/test_keeper_session/test.py | 71 +++++++++++++++++++ 2 files changed, 98 insertions(+) create mode 100644 tests/integration/test_keeper_session/configs/keeper_config.xml create mode 100644 tests/integration/test_keeper_session/test.py diff --git a/tests/integration/test_keeper_session/configs/keeper_config.xml b/tests/integration/test_keeper_session/configs/keeper_config.xml new file mode 100644 index 00000000000..87ba2064dc9 --- /dev/null +++ b/tests/integration/test_keeper_session/configs/keeper_config.xml @@ -0,0 +1,27 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + * + + + 5000 + 10000 + 5000 + 75 + trace + + + + + 1 + node1 + 9234 + true + 3 + + + + diff --git a/tests/integration/test_keeper_session/test.py b/tests/integration/test_keeper_session/test.py new file mode 100644 index 00000000000..2e552492762 --- /dev/null +++ b/tests/integration/test_keeper_session/test.py @@ -0,0 +1,71 @@ +import pytest +from helpers.cluster import ClickHouseCluster +import time + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', main_configs=['configs/keeper_config.xml'], stay_alive=True) + +from kazoo.client import KazooClient, KazooState + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def destroy_zk_client(zk): + try: + if zk: + zk.stop() + zk.close() + except: + pass + + +def wait_node(node): + for _ in range(100): + zk = None + try: + zk = get_fake_zk(node.name, timeout=30.0) + print("node", node.name, "ready") + break + except Exception as ex: + time.sleep(0.2) + print("Waiting until", node.name, "will be ready, exception", ex) + finally: + destroy_zk_client(zk) + else: + raise Exception("Can't wait node", node.name, "to become ready") + + +def wait_nodes(): + for n in [node1]: + wait_node(n) + + +def get_fake_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout) + _fake_zk_instance.start() + return _fake_zk_instance + + +def test_session_timeout(started_cluster): + zk = None + try: + wait_nodes() + + zk1 = get_fake_zk(node1.name, timeout=1.0) + assert zk1._session_timeout == 5000 + + zk1 = get_fake_zk(node1.name, timeout=8.0) + assert zk1._session_timeout == 8000 + + zk1 = get_fake_zk(node1.name, timeout=20.0) + assert zk1._session_timeout == 10000 + finally: + destroy_zk_client(zk) From 9169a8467145c64a01e5b1a2b9264b21f6848475 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 29 Dec 2021 13:32:49 +0100 Subject: [PATCH 0593/1260] Add enum QueryKind for IAST class --- src/Parsers/ASTAlterQuery.h | 1 + src/Parsers/ASTCreateQuery.h | 1 + src/Parsers/ASTDropQuery.h | 1 + src/Parsers/ASTInsertQuery.h | 1 + src/Parsers/ASTRenameQuery.h | 1 + src/Parsers/ASTSelectIntersectExceptQuery.h | 1 + src/Parsers/ASTSelectQuery.h | 1 + src/Parsers/ASTSelectWithUnionQuery.h | 1 + src/Parsers/ASTSystemQuery.h | 1 + src/Parsers/Access/ASTGrantQuery.h | 1 + src/Parsers/IAST.h | 18 +++++++++++++++++- 11 files changed, 27 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index b0980c20f5e..f757f00c100 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -247,6 +247,7 @@ public: } const char * getQueryKindString() const override { return "Alter"; } + virtual QueryKind getQueryKind() const override { return QueryKind::Alter; } protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index 93fced7dba5..461e5b61f2f 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -114,6 +114,7 @@ public: bool isView() const { return is_ordinary_view || is_materialized_view || is_live_view || is_window_view; } const char * getQueryKindString() const override { return "Create"; } + virtual QueryKind getQueryKind() const override { return QueryKind::Create; } protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; diff --git a/src/Parsers/ASTDropQuery.h b/src/Parsers/ASTDropQuery.h index 6e5fd5854d8..24f7c706062 100644 --- a/src/Parsers/ASTDropQuery.h +++ b/src/Parsers/ASTDropQuery.h @@ -46,6 +46,7 @@ public: } const char * getQueryKindString() const override { return "Drop"; } + virtual QueryKind getQueryKind() const override { return QueryKind::Drop; } protected: void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; diff --git a/src/Parsers/ASTInsertQuery.h b/src/Parsers/ASTInsertQuery.h index d539ad5fdb3..f52a3603f2a 100644 --- a/src/Parsers/ASTInsertQuery.h +++ b/src/Parsers/ASTInsertQuery.h @@ -67,6 +67,7 @@ public: } const char * getQueryKindString() const override { return "Insert"; } + virtual QueryKind getQueryKind() const override { return QueryKind::Insert; } protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; diff --git a/src/Parsers/ASTRenameQuery.h b/src/Parsers/ASTRenameQuery.h index dafc166f672..a2ee6f7e622 100644 --- a/src/Parsers/ASTRenameQuery.h +++ b/src/Parsers/ASTRenameQuery.h @@ -66,6 +66,7 @@ public: } const char * getQueryKindString() const override { return "Rename"; } + virtual QueryKind getQueryKind() const override { return QueryKind::Rename; } protected: void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override diff --git a/src/Parsers/ASTSelectIntersectExceptQuery.h b/src/Parsers/ASTSelectIntersectExceptQuery.h index 86475fcba5c..2e0bcd2c885 100644 --- a/src/Parsers/ASTSelectIntersectExceptQuery.h +++ b/src/Parsers/ASTSelectIntersectExceptQuery.h @@ -23,6 +23,7 @@ public: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; const char * getQueryKindString() const override { return "SelectIntersectExcept"; } + virtual QueryKind getQueryKind() const override { return QueryKind::SelectIntersectExcept; } ASTs getListOfSelects() const; diff --git a/src/Parsers/ASTSelectQuery.h b/src/Parsers/ASTSelectQuery.h index 2b004e9e040..70f2db28503 100644 --- a/src/Parsers/ASTSelectQuery.h +++ b/src/Parsers/ASTSelectQuery.h @@ -136,6 +136,7 @@ public: void setFinal(); const char * getQueryKindString() const override { return "Select"; } + virtual QueryKind getQueryKind() const override { return QueryKind::Select; } protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; diff --git a/src/Parsers/ASTSelectWithUnionQuery.h b/src/Parsers/ASTSelectWithUnionQuery.h index 3fc8f9171c0..7df03e6aab9 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.h +++ b/src/Parsers/ASTSelectWithUnionQuery.h @@ -18,6 +18,7 @@ public: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; const char * getQueryKindString() const override { return "Select"; } + virtual QueryKind getQueryKind() const override { return QueryKind::Select; } SelectUnionMode union_mode; diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 16f8a3c118a..a4c86866e7b 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -108,6 +108,7 @@ public: } const char * getQueryKindString() const override { return "System"; } + virtual QueryKind getQueryKind() const override { return QueryKind::System; } protected: diff --git a/src/Parsers/Access/ASTGrantQuery.h b/src/Parsers/Access/ASTGrantQuery.h index f8ea9b478fe..44d6de6a662 100644 --- a/src/Parsers/Access/ASTGrantQuery.h +++ b/src/Parsers/Access/ASTGrantQuery.h @@ -35,5 +35,6 @@ public: void replaceCurrentUserTag(const String & current_user_name) const; ASTPtr getRewrittenASTWithoutOnCluster(const std::string &) const override { return removeOnCluster(clone()); } const char * getQueryKindString() const override { return is_revoke ? "Revoke" : "Grant"; } + virtual QueryKind getQueryKind() const override { return is_revoke ? QueryKind::Revoke : QueryKind::Grant; } }; } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 2f30a1f7bee..5806f652593 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -248,7 +248,23 @@ public: // Return query_kind string representation of this AST query. virtual const char * getQueryKindString() const { return ""; } -public: + enum QueryKind + { + None, + Alter, + Create, + Drop, + Grant, + Insert, + Rename, + Revoke, + SelectIntersectExcept, + Select, + System, + }; + /// Return QueryKind of this AST query. + virtual QueryKind getQueryKind() const { return QueryKind::None; } + /// For syntax highlighting. static const char * hilite_keyword; static const char * hilite_identifier; From 67fc9d7129f607f9551bb5ab9944d8bc23712ddc Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Dec 2021 18:35:54 +0300 Subject: [PATCH 0594/1260] Fix clang-tidy --- src/Processors/Transforms/buildPushingToViewsChain.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index fdd0150536a..17075e2b318 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -596,7 +596,7 @@ void PushingToLiveViewSink::consume(Chunk chunk) { Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); StorageLiveView::writeIntoLiveView(live_view, getHeader().cloneWithColumns(chunk.detachColumns()), context); - auto process = context->getProcessListElement(); + auto * process = context->getProcessListElement(); if (process) process->updateProgressIn(local_progress); ProfileEvents::increment(ProfileEvents::SelectedRows, local_progress.read_rows); @@ -619,7 +619,7 @@ void PushingToWindowViewSink::consume(Chunk chunk) Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); StorageWindowView::writeIntoWindowView( window_view, getHeader().cloneWithColumns(chunk.detachColumns()), context); - auto process = context->getProcessListElement(); + auto * process = context->getProcessListElement(); if (process) process->updateProgressIn(local_progress); ProfileEvents::increment(ProfileEvents::SelectedRows, local_progress.read_rows); From 34145c47da9787ffb4da119cb10f006d9f70844c Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Dec 2021 18:36:42 +0300 Subject: [PATCH 0595/1260] Fix clang tidy --- src/Formats/FormatFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index cf2cdd6c547..467fdfddafe 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -238,7 +238,7 @@ InputFormatPtr FormatFactory::getInputFormat( static void addExistingProgressToOutputFormat(OutputFormatPtr format, ContextPtr context) { - auto element_id = context->getProcessListElement(); + auto * element_id = context->getProcessListElement(); if (element_id) { /// While preparing the query there might have been progress (for example in subscalar subqueries) so add it here From a882e64644b2456f024c441bfe2d7cce52e91ae3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 29 Dec 2021 15:25:56 +0100 Subject: [PATCH 0596/1260] Use IAST::QueryKind instead of strings in QueryKindAmount --- src/Interpreters/ProcessList.cpp | 59 +++++++++++++++++--------------- src/Interpreters/ProcessList.h | 19 ++++++---- 2 files changed, 44 insertions(+), 34 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index e7e52142fc8..5b3e294917b 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -11,6 +11,7 @@ #include #include #include +#include "Parsers/IAST.h" #include #include #include @@ -86,15 +87,14 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as throw Exception("Too many simultaneous queries. Maximum: " + toString(max_size), ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES); } - String query_kind{ast->getQueryKindString()}; if (!is_unlimited_query) { - auto amount = getQueryKindAmount(query_kind); - if (max_insert_queries_amount && query_kind == "Insert" && amount >= max_insert_queries_amount) + QueryAmount amount = getQueryKindAmount(ast->getQueryKind()); + if (max_insert_queries_amount && ast->getQueryKind() == IAST::QueryKind::Insert && amount >= max_insert_queries_amount) throw Exception(ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, "Too many simultaneous insert queries. Maximum: {}, current: {}", max_insert_queries_amount, amount); - if (max_select_queries_amount && query_kind == "Select" && amount >= max_select_queries_amount) + if (max_select_queries_amount && ast->getQueryKind() == IAST::QueryKind::Select && amount >= max_select_queries_amount) throw Exception(ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, "Too many simultaneous select queries. Maximum: {}, current: {}", max_select_queries_amount, amount); @@ -190,9 +190,9 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as } auto process_it = processes.emplace(processes.end(), - query_context, query_, client_info, priorities.insert(settings.priority), query_kind); + query_context, query_, client_info, priorities.insert(settings.priority), ast->getQueryKind()); - increaseQueryKindAmount(query_kind); + increaseQueryKindAmount(ast->getQueryKind()); res = std::make_shared(*this, process_it); @@ -258,7 +258,7 @@ ProcessListEntry::~ProcessListEntry() String user = it->getClientInfo().current_user; String query_id = it->getClientInfo().current_query_id; - String query_kind = it->query_kind; + IAST::QueryKind query_kind = it->query_kind; const QueryStatus * process_list_element_ptr = &*it; @@ -306,7 +306,7 @@ ProcessListEntry::~ProcessListEntry() QueryStatus::QueryStatus( - ContextPtr context_, const String & query_, const ClientInfo & client_info_, QueryPriorities::Handle && priority_handle_, const String & query_kind_) + ContextPtr context_, const String & query_, const ClientInfo & client_info_, QueryPriorities::Handle && priority_handle_, const IAST::QueryKind & query_kind_) : WithContext(context_) , query(query_) , client_info(client_info_) @@ -505,33 +505,38 @@ ProcessList::UserInfo ProcessList::getUserInfo(bool get_profile_events) const return per_user_infos; } -void ProcessList::increaseQueryKindAmount(const String & query_kind) +void ProcessList::increaseQueryKindAmount(const IAST::QueryKind & query_kind) const { - auto found = query_kind_amounts.find(query_kind); - if (found == query_kind_amounts.end()) - query_kind_amounts[query_kind] = 1; - else - found->second += 1; + if (query_kind == IAST::QueryKind::Insert) + query_kind_amounts->insert++; + else if (query_kind == IAST::QueryKind::Select) + query_kind_amounts->select++; } -void ProcessList::decreaseQueryKindAmount(const String & query_kind) +void ProcessList::decreaseQueryKindAmount(const IAST::QueryKind & query_kind) const { - auto found = query_kind_amounts.find(query_kind); - /// TODO: we could just rebuild the map, as we have saved all query_kind. - if (found == query_kind_amounts.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong query kind amount: decrease before increase on '{}'", query_kind); - else if (found->second == 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong query kind amount: decrease to negative on '{}'", query_kind, found->second); - else - found->second -= 1; + if (!(query_kind == IAST::QueryKind::Insert || query_kind == IAST::QueryKind::Select)) + return; + QueryAmount amount = getQueryKindAmount(query_kind); + if (amount == 0) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong query kind amount: decrease to negative on '{}'", query_kind, amount); + } + if (query_kind == IAST::QueryKind::Insert) + query_kind_amounts->insert--; + else if (query_kind == IAST::QueryKind::Select) + query_kind_amounts->select--; } -ProcessList::QueryAmount ProcessList::getQueryKindAmount(const String & query_kind) + +ProcessList::QueryAmount ProcessList::getQueryKindAmount(const IAST::QueryKind & query_kind) const { - auto found = query_kind_amounts.find(query_kind); - if (found == query_kind_amounts.end()) + if (query_kind == IAST::QueryKind::Insert) + return query_kind_amounts->insert; + else if (query_kind == IAST::QueryKind::Select) + return query_kind_amounts->select; + else return 0; - return found->second; } } diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index ada24c03275..8a1d26147a5 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -118,7 +119,7 @@ protected: ProcessListForUser * user_process_list = nullptr; - String query_kind; + IAST::QueryKind query_kind; public: @@ -127,7 +128,7 @@ public: const String & query_, const ClientInfo & client_info_, QueryPriorities::Handle && priority_handle_, - const String & query_kind_ + const IAST::QueryKind & query_kind_ ); ~QueryStatus(); @@ -270,7 +271,11 @@ public: /// User -> queries using UserToQueries = std::unordered_map; - using QueryKindToAmount = std::unordered_map; + struct QueryKindAmounts + { + QueryAmount insert; + QueryAmount select; + }; protected: friend class ProcessListEntry; @@ -301,11 +306,11 @@ protected: size_t max_select_queries_amount = 0; /// amount of queries by query kind. - QueryKindToAmount query_kind_amounts; + QueryKindAmounts * query_kind_amounts = new QueryKindAmounts{0, 0}; - void increaseQueryKindAmount(const String & query_kind); - void decreaseQueryKindAmount(const String & query_kind); - QueryAmount getQueryKindAmount(const String & query_kind); + void increaseQueryKindAmount(const IAST::QueryKind & query_kind) const; + void decreaseQueryKindAmount(const IAST::QueryKind & query_kind) const; + QueryAmount getQueryKindAmount(const IAST::QueryKind & query_kind) const; public: using EntryPtr = std::shared_ptr; From c1297d24313dbdda21eba811d31b15fc7b21b942 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 29 Dec 2021 19:57:40 +0300 Subject: [PATCH 0597/1260] Compatibility --- src/Client/ClientBase.cpp | 91 ++++++++++++++++++++------------------- 1 file changed, 47 insertions(+), 44 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 8d7cdd1577d..fb2a58978c6 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -800,56 +800,59 @@ void ClientBase::onProfileEvents(Block & block) if (rows == 0) return; - const auto & array_thread_id = typeid_cast(*block.getByName("thread_id").column).getData(); - const auto & names = typeid_cast(*block.getByName("name").column); - const auto & host_names = typeid_cast(*block.getByName("host_name").column); - const auto & array_values = typeid_cast(*block.getByName("value").column).getData(); - - const auto * user_time_name = ProfileEvents::getName(ProfileEvents::UserTimeMicroseconds); - const auto * system_time_name = ProfileEvents::getName(ProfileEvents::SystemTimeMicroseconds); - - HostToThreadTimesMap thread_times; - for (size_t i = 0; i < rows; ++i) + if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS) { - auto thread_id = array_thread_id[i]; - auto host_name = host_names.getDataAt(i).toString(); - if (thread_id != 0) - progress_indication.addThreadIdToList(host_name, thread_id); - auto event_name = names.getDataAt(i); - auto value = array_values[i]; - if (event_name == user_time_name) - { - thread_times[host_name][thread_id].user_ms = value; - } - else if (event_name == system_time_name) - { - thread_times[host_name][thread_id].system_ms = value; - } - else if (event_name == MemoryTracker::USAGE_EVENT_NAME) - { - thread_times[host_name][thread_id].memory_usage = value; - } - } - auto elapsed_time = profile_events.watch.elapsedMicroseconds(); - progress_indication.updateThreadEventData(thread_times, elapsed_time); + const auto & array_thread_id = typeid_cast(*block.getByName("thread_id").column).getData(); + const auto & names = typeid_cast(*block.getByName("name").column); + const auto & host_names = typeid_cast(*block.getByName("host_name").column); + const auto & array_values = typeid_cast(*block.getByName("value").column).getData(); - if (profile_events.print) - { - if (profile_events.watch.elapsedMilliseconds() >= profile_events.delay_ms) - { - initLogsOutputStream(); - progress_indication.clearProgressOutput(); - logs_out_stream->writeProfileEvents(block); - logs_out_stream->flush(); + const auto * user_time_name = ProfileEvents::getName(ProfileEvents::UserTimeMicroseconds); + const auto * system_time_name = ProfileEvents::getName(ProfileEvents::SystemTimeMicroseconds); - profile_events.last_block = {}; - } - else + HostToThreadTimesMap thread_times; + for (size_t i = 0; i < rows; ++i) { - incrementProfileEventsBlock(profile_events.last_block, block); + auto thread_id = array_thread_id[i]; + auto host_name = host_names.getDataAt(i).toString(); + if (thread_id != 0) + progress_indication.addThreadIdToList(host_name, thread_id); + auto event_name = names.getDataAt(i); + auto value = array_values[i]; + if (event_name == user_time_name) + { + thread_times[host_name][thread_id].user_ms = value; + } + else if (event_name == system_time_name) + { + thread_times[host_name][thread_id].system_ms = value; + } + else if (event_name == MemoryTracker::USAGE_EVENT_NAME) + { + thread_times[host_name][thread_id].memory_usage = value; + } } + auto elapsed_time = profile_events.watch.elapsedMicroseconds(); + progress_indication.updateThreadEventData(thread_times, elapsed_time); + + if (profile_events.print) + { + if (profile_events.watch.elapsedMilliseconds() >= profile_events.delay_ms) + { + initLogsOutputStream(); + progress_indication.clearProgressOutput(); + logs_out_stream->writeProfileEvents(block); + logs_out_stream->flush(); + + profile_events.last_block = {}; + } + else + { + incrementProfileEventsBlock(profile_events.last_block, block); + } + } + profile_events.watch.restart(); } - profile_events.watch.restart(); } From 90d7dc0e04d5dbb806c56ea09de4eedf188b9979 Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Wed, 29 Dec 2021 10:19:45 -0700 Subject: [PATCH 0598/1260] Updated adopters: 7 new adopters plus some minor editing --- docs/en/introduction/adopters.md | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 87c5a6f7aec..8f6e9a3de52 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -60,8 +60,10 @@ toc_title: Adopters | Exness | Trading | Metrics, Logging | — | — | [Talk in Russian, May 2019](https://youtu.be/_rpU-TvSfZ8?t=3215) | | EventBunker.io | Serverless Data Processing | — | — | — | [Tweet, April 2021](https://twitter.com/Halil_D_/status/1379839133472985091) | | FastNetMon | DDoS Protection | Main Product | | — | [Official website](https://fastnetmon.com/docs-fnm-advanced/fastnetmon-advanced-traffic-persistency/) | +| Flipkart | e-Commerce | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=239) | | FunCorp | Games | | — | 14 bn records/day as of Jan 2021 | [Article](https://www.altinity.com/blog/migrating-from-redshift-to-clickhouse) | +| Futurra Group | Analytics | — | — | — | [Article in Russian, December 2021](https://dou.ua/forums/topic/35587/) | | Geniee | Ad network | Main product | — | — | [Blog post in Japanese, July 2017](https://tech.geniee.co.jp/entry/2017/07/20/160100) | | Genotek | Bioinformatics | Main product | — | — | [Video, August 2020](https://youtu.be/v3KyZbz9lEE) | | Gigapipe | Managed ClickHouse | Main product | — | — | [Official website](https://gigapipe.com/) | @@ -70,6 +72,7 @@ toc_title: Adopters | Grouparoo | Data Warehouse Integrations | Main product | — | — | [Official Website, November 2021](https://www.grouparoo.com/integrations) | | HUYA | Video Streaming | Analytics | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/7.%20ClickHouse万亿数据分析实践%20李本旺(sundy-li)%20虎牙.pdf) | | Hydrolix | Cloud data platform | Main product | — | — | [Documentation](https://docs.hydrolix.io/guide/query) | +| ICA | FinTech | Risk Management | — | — | [Blog Post in English, Sep 2020](https://altinity.com/blog/clickhouse-vs-redshift-performance-for-fintech-risk-management?utm_campaign=ClickHouse%20vs%20RedShift&utm_content=143520807&utm_medium=social&utm_source=twitter&hss_channel=tw-3894792263) | | Idealista | Real Estate | Analytics | — | — | [Blog Post in English, April 2019](https://clickhouse.com/blog/en/clickhouse-meetup-in-madrid-on-april-2-2019) | | Infobaleen | AI markting tool | Analytics | — | — | [Official site](https://infobaleen.com) | @@ -81,14 +84,18 @@ toc_title: Adopters | Ippon Technologies | Technology Consulting | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=205) | | Ivi | Online Cinema | Analytics, Monitoring | — | — | [Article in Russian, Jan 2018](https://habr.com/en/company/ivi/blog/347408/) | | Jinshuju 金数据 | BI Analytics | Main product | — | — | [Slides in Chinese, October 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/3.%20金数据数据架构调整方案Public.pdf) | -| Jitsu | Cloud Software | Data Pipeline | — | — | [Documentation](https://jitsu.com/docs/destinations-configuration/clickhouse-destination), [Hacker News](https://news.ycombinator.com/item?id=29106082) | +| Jitsu | Cloud Software | Data Pipeline | — | — | [Documentation](https://jitsu.com/docs/destinations-configuration/clickhouse-destination), [Hacker News post](https://news.ycombinator.com/item?id=29106082) | +| kakaocorp | Internet company | — | — | — | [if(kakao)2020](https://tv.kakao.com/channel/3693125/cliplink/414129353), [if(kakao)2021](https://if.kakao.com/session/24) | | Kodiak Data | Clouds | Main product | — | — | [Slides in Engish, April 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup13/kodiak_data.pdf) | | Kontur | Software Development | Metrics | — | — | [Talk in Russian, November 2018](https://www.youtube.com/watch?v=U4u4Bd0FtrY) | | Kuaishou | Video | — | — | — | [ClickHouse Meetup, October 2018](https://clickhouse.com/blog/en/2018/clickhouse-community-meetup-in-beijing-on-october-28-2018/) | | KGK Global | Vehicle monitoring | — | — | — | [Press release, June 2021](https://zoom.cnews.ru/news/item/530921) | +| LANCOM Systems | Network Solutions | Traffic analysis | - | - | [ClickHouse Operator for Kubernetes](https://www.lancom-systems.com/), [Hacker News post] (https://news.ycombinator.com/item?id=29413660) | | Lawrence Berkeley National Laboratory | Research | Traffic analysis | 5 servers | 55 TiB | [Slides in English, April 2019](https://www.smitasin.com/presentations/2019-04-17_DOE-NSM.pdf) | +| LifeStreet | Ad network | Main product | 75 servers (3 replicas) | 5.27 PiB | [Blog post in Russian, February 2017](https://habr.com/en/post/322620/) | +| Lookforsale | E-Commerce | — | — | — | [Job Posting, December 2021](https://telegram.me/javascript_jobs/587318) | | Mail.ru Cloud Solutions | Cloud services | Main product | — | — | [Article in Russian](https://mcs.mail.ru/help/db-create/clickhouse#) | | MAXILECT | Ad Tech, Blockchain, ML, AI | — | — | — | [Job advertisement, 2021](https://www.linkedin.com/feed/update/urn:li:activity:6780842017229430784/) | | Marilyn | Advertising | Statistics | — | — | [Talk in Russian, June 2017](https://www.youtube.com/watch?v=iXlIgx2khwc) | @@ -106,6 +113,7 @@ toc_title: Adopters | Ok.ru | Social Network | — | 72 servers | 810 TB compressed, 50bn rows/day, 1.5 TB/day | [SmartData conference, October 2021](https://assets.ctfassets.net/oxjq45e8ilak/4JPHkbJenLgZhBGGyyonFP/57472ec6987003ec4078d0941740703b/____________________ClickHouse_______________________.pdf) | | Omnicomm | Transportation Monitoring | — | — | — | [Facebook post, October 2021](https://www.facebook.com/OmnicommTeam/posts/2824479777774500) | | OneAPM | Monitoring and Data Analysis | Main product | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/8.%20clickhouse在OneAPM的应用%20杜龙.pdf) | +| Open Targets | Genome Research | Genome Search | — | — | [Tweet, October 2021](https://twitter.com/OpenTargets/status/1452570865342758913?s=20), [Blog](https://blog.opentargets.org/graphql/) | | OZON | E-commerce | — | — | — | [Official website](https://job.ozon.ru/vacancy/razrabotchik-clickhouse-ekspluatatsiya-40991870/) | | Panelbear | Analytics | Monitoring and Analytics | — | — | [Tech Stack, November 2020](https://panelbear.com/blog/tech-stack/) | @@ -118,6 +126,7 @@ toc_title: Adopters | PRANA | Industrial predictive analytics | Main product | — | — | [News (russian), Feb 2021](https://habr.com/en/news/t/541392/) | | QINGCLOUD | Cloud services | Main product | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/4.%20Cloud%20%2B%20TSDB%20for%20ClickHouse%20张健%20QingCloud.pdf) | | Qrator | DDoS protection | Main product | — | — | [Blog Post, March 2019](https://blog.qrator.net/en/clickhouse-ddos-mitigation_37/) | +| R-Vision | Information Security | — | — | — | [Article in Russian, December 2021](https://www.anti-malware.ru/reviews/R-Vision-SENSE-15) | | Raiffeisenbank | Banking | Analytics | — | — | [Lecture in Russian, December 2020](https://cs.hse.ru/announcements/421965599.html) | | Rambler | Internet services | Analytics | — | — | [Talk in Russian, April 2018](https://medium.com/@ramblertop/разработка-api-clickhouse-для-рамблер-топ-100-f4c7e56f3141) | | Replica | Urban Planning | Analytics | — | — | [Job advertisement](https://boards.greenhouse.io/replica/jobs/5547732002?gh_jid=5547732002) | @@ -153,6 +162,7 @@ toc_title: Adopters | Tinybird | Real-time Data Products | Data processing | — | — | [Official website](https://www.tinybird.co/) | | Traffic Stars | AD network | — | 300 servers in Europe/US | 1.8 PiB, 700 000 insert rps (as of 2021) | [Slides in Russian, May 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup15/lightning/ninja.pdf) | | Uber | Taxi | Logging | — | — | [Slides, February 2020](https://presentations.clickhouse.com/meetup40/uber.pdf) | +| UseTech | Software Development | — | — | — | [Job Posting, December 2021](https://vk.com/wall136266658_2418) | | UTMSTAT | Analytics | Main product | — | — | [Blog post, June 2020](https://vc.ru/tribuna/133956-striming-dannyh-iz-servisa-skvoznoy-analitiki-v-clickhouse) | | Vercel | Traffic and Performance Analytics | — | — | — | Direct reference, October 2021 | | VKontakte | Social Network | Statistics, Logging | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/3_vk.pdf) | @@ -168,7 +178,8 @@ toc_title: Adopters | Yandex Cloud | Public Cloud | Main product | — | — | [Talk in Russian, December 2019](https://www.youtube.com/watch?v=pgnak9e_E0o) | | Yandex DataLens | Business Intelligence | Main product | — | — | [Slides in Russian, December 2019](https://presentations.clickhouse.com/meetup38/datalens.pdf) | | Yandex Market | e-Commerce | Metrics, Logging | — | — | [Talk in Russian, January 2019](https://youtu.be/_l1qP0DyBcA?t=478) | -| Yandex Metrica | Web analytics | Main product | 630 servers in one cluster, 360 servers in another cluster, 1862 servers in one department | 133 PiB / 8.31 PiB / 120 trillion records | [Slides, February 2020](https://presentations.clickhouse.com/meetup40/introduction/#13) | +| Yandex Metrica | Web analytics | Macin product | 630 servers in one cluster, 360 servers in another cluster, 1862 servers in one department | 133 PiB / 8.31 PiB / 120 trillion records | [Slides, February 2020](https://presentations.clickhouse.com/meetup40/introduction/#13) | +| Yotascale | Cloud | Data pipeline | — | 2 bn records/day | [LinkedIn (Accomplishments)](https://www.linkedin.com/in/adilsaleem/) | | Your Analytics | Product Analytics | Main Product | — | - | [Tweet, November 2021](https://twitter.com/mikenikles/status/1459737241165565953) | | Zagrava Trading | — | — | — | — | [Job offer, May 2021](https://twitter.com/datastackjobs/status/1394707267082063874) | @@ -178,9 +189,5 @@ toc_title: Adopters | Цифровой Рабочий | Industrial IoT, Analytics | — | — | — | [Blog post in Russian, March 2021](https://habr.com/en/company/croc/blog/548018/) | | ООО «МПЗ Богородский» | Agriculture | — | — | — | [Article in Russian, November 2020](https://cloud.yandex.ru/cases/okraina) | | ДомКлик | Real Estate | — | — | — | [Article in Russian, October 2021](https://habr.com/ru/company/domclick/blog/585936/) | -| Futurra Group | Analytics | — | — | — | [Article in Russian, December 2021](https://dou.ua/forums/topic/35587/) | -| UseTech | Software Development | — | — | — | [Job Posting, December 2021](https://vk.com/wall136266658_2418) | -| Lookforsale | E-Commerce | — | — | — | [Job Posting, December 2021](https://telegram.me/javascript_jobs/587318) | -| R-Vision | Information Security | — | — | — | [Article in Russian, December 2021](https://www.anti-malware.ru/reviews/R-Vision-SENSE-15) | [Original article](https://clickhouse.com/docs/en/introduction/adopters/) From 7c6f7f67328ffe72e46ac00c034d05c54f0576bc Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 29 Dec 2021 04:33:39 +0300 Subject: [PATCH 0599/1260] support 'optimize_move_to_prewhere' with storage 'Merge' --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Storages/IStorage.h | 3 ++ src/Storages/StorageDistributed.h | 4 ++ src/Storages/StorageMerge.cpp | 31 +++++++++++++-- src/Storages/StorageMerge.h | 7 +++- .../02156_storage_merge_prewhere.reference | 13 +++++++ .../02156_storage_merge_prewhere.sql | 38 +++++++++++++++++++ 7 files changed, 92 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02156_storage_merge_prewhere.reference create mode 100644 tests/queries/0_stateless/02156_storage_merge_prewhere.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 5c0322ac1d9..7049dfc03f6 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -401,7 +401,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( view = nullptr; } - if (try_move_to_prewhere && storage && storage->supportsPrewhere() && query.where() && !query.prewhere()) + if (try_move_to_prewhere && storage && storage->canMoveConditionsToPrewhere() && query.where() && !query.prewhere()) { /// PREWHERE optimization: transfer some condition from WHERE to PREWHERE if enabled and viable if (const auto & column_sizes = storage->getColumnSizes(); !column_sizes.empty()) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index bcbc771815b..6342c3f6b47 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -135,6 +135,9 @@ public: /// Returns true if the storage supports queries with the PREWHERE section. virtual bool supportsPrewhere() const { return false; } + /// Returns true if the storage supports optimization of moving conditions to PREWHERE section. + virtual bool canMoveConditionsToPrewhere() const { return supportsPrewhere(); } + /// Returns true if the storage replicates SELECT, INSERT and ALTER commands among replicas. virtual bool supportsReplication() const { return false; } diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 06fa8905639..e47e0fddd6c 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -53,6 +53,10 @@ public: bool supportsSubcolumns() const override { return true; } StoragePolicyPtr getStoragePolicy() const override; + /// Do not apply moving to PREWHERE optimization for distributed tables, + /// because we can't be sure that underlying table supports PREWHERE. + bool canMoveConditionsToPrewhere() const override { return false; } + bool isRemote() const override { return true; } QueryProcessingStage::Enum diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index f82f9d21217..75a37385218 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -107,6 +107,15 @@ StoragePtr StorageMerge::getFirstTable(F && predicate) const return {}; } +template +void StorageMerge::forEachTable(F && func) const +{ + getFirstTable([&func](const auto & table) + { + func(table); + return false; + }); +} bool StorageMerge::isRemote() const { @@ -114,6 +123,16 @@ bool StorageMerge::isRemote() const return first_remote_table != nullptr; } +bool StorageMerge::canMoveConditionsToPrewhere() const +{ + /// NOTE: This check is used during query analysis as condition for applying + /// "move to PREWHERE" optimization. However, it contains a logical race: + /// If new table that matches regexp for current storage and doesn't support PREWHERE + /// will appear after this check and before calling "read" method, the optimized query may fail. + /// Since it's quite rare case, we just ignore this possibility. + + return getFirstTable([](const auto & table) { return !table->canMoveConditionsToPrewhere(); }) == nullptr; +} bool StorageMerge::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, ContextPtr query_context, const StorageMetadataPtr & /*metadata_snapshot*/) const { @@ -762,11 +781,15 @@ void StorageMerge::convertingSourceStream( IStorage::ColumnSizeByName StorageMerge::getColumnSizes() const { + ColumnSizeByName column_sizes; - auto first_materialized_mysql = getFirstTable([](const StoragePtr & table) { return table && table->getName() == "MaterializedMySQL"; }); - if (!first_materialized_mysql) - return {}; - return first_materialized_mysql->getColumnSizes(); + forEachTable([&](const auto & table) + { + for (const auto & [name, size] : table->getColumnSizes()) + column_sizes[name].add(size); + }); + + return column_sizes; } diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 56adeab9279..bd9262a98a3 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -22,10 +22,12 @@ public: /// The check is delayed to the read method. It checks the support of the tables used. bool supportsSampling() const override { return true; } - bool supportsPrewhere() const override { return true; } bool supportsFinal() const override { return true; } bool supportsIndexForIn() const override { return true; } bool supportsSubcolumns() const override { return true; } + bool supportsPrewhere() const override { return true; } + + bool canMoveConditionsToPrewhere() const override; QueryProcessingStage::Enum getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageMetadataPtr &, SelectQueryInfo &) const override; @@ -75,6 +77,9 @@ private: template StoragePtr getFirstTable(F && predicate) const; + template + void forEachTable(F && func) const; + DatabaseTablesIteratorPtr getDatabaseIterator(const String & database_name, ContextPtr context) const; DatabaseTablesIterators getDatabaseIterators(ContextPtr context) const; diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference new file mode 100644 index 00000000000..6df60403ae0 --- /dev/null +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference @@ -0,0 +1,13 @@ +SELECT count() +FROM t_02156_merge1 +PREWHERE k = 1 +WHERE (k = 1) AND notEmpty(v) +2 +SELECT count() +FROM t_02156_merge2 +WHERE (k = 1) AND notEmpty(v) +2 +SELECT count() +FROM t_02156_merge3 +WHERE (k = 1) AND notEmpty(v) +2 diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.sql b/tests/queries/0_stateless/02156_storage_merge_prewhere.sql new file mode 100644 index 00000000000..e033005e014 --- /dev/null +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.sql @@ -0,0 +1,38 @@ +DROP TABLE IF EXISTS t_02156_mt1; +DROP TABLE IF EXISTS t_02156_mt2; +DROP TABLE IF EXISTS t_02156_log; +DROP TABLE IF EXISTS t_02156_dist; +DROP TABLE IF EXISTS t_02156_merge1; +DROP TABLE IF EXISTS t_02156_merge2; +DROP TABLE IF EXISTS t_02156_merge3; + +CREATE TABLE t_02156_mt1 (k UInt32, v String) ENGINE = MergeTree ORDER BY k; +CREATE TABLE t_02156_mt2 (k UInt32, v String) ENGINE = MergeTree ORDER BY k; +CREATE TABLE t_02156_log (k UInt32, v String) ENGINE = Log; + +CREATE TABLE t_02156_dist (k UInt32, v String) ENGINE = Distributed(test_shard_localhost, currentDatabase(), t_02156_mt1); + +CREATE TABLE t_02156_merge1 (k UInt32, v String) ENGINE = Merge(currentDatabase(), 't_02156_mt1|t_02156_mt2'); +CREATE TABLE t_02156_merge2 (k UInt32, v String) ENGINE = Merge(currentDatabase(), 't_02156_mt1|t_02156_log'); +CREATE TABLE t_02156_merge3 (k UInt32, v String) ENGINE = Merge(currentDatabase(), 't_02156_mt2|t_02156_dist'); + +INSERT INTO t_02156_mt1 SELECT number, toString(number) FROM numbers(10000); +INSERT INTO t_02156_mt2 SELECT number, toString(number) FROM numbers(10000); +INSERT INTO t_02156_log SELECT number, toString(number) FROM numbers(10000); + +EXPLAIN SYNTAX SELECT count() FROM t_02156_merge1 WHERE k = 1 AND notEmpty(v); +SELECT count() FROM t_02156_merge1 WHERE k = 1 AND notEmpty(v); + +EXPLAIN SYNTAX SELECT count() FROM t_02156_merge2 WHERE k = 1 AND notEmpty(v); +SELECT count() FROM t_02156_merge2 WHERE k = 1 AND notEmpty(v); + +EXPLAIN SYNTAX SELECT count() FROM t_02156_merge3 WHERE k = 1 AND notEmpty(v); +SELECT count() FROM t_02156_merge3 WHERE k = 1 AND notEmpty(v); + +DROP TABLE IF EXISTS t_02156_mt1; +DROP TABLE IF EXISTS t_02156_mt2; +DROP TABLE IF EXISTS t_02156_log; +DROP TABLE IF EXISTS t_02156_dist; +DROP TABLE IF EXISTS t_02156_merge1; +DROP TABLE IF EXISTS t_02156_merge2; +DROP TABLE IF EXISTS t_02156_merge3; From 7067fafd76205b2b05c240c7b7517bac68856c0a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 29 Dec 2021 22:09:07 +0300 Subject: [PATCH 0600/1260] Reimplement #33054 --- programs/benchmark/Benchmark.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 1c276a83768..35ffb97b8e2 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -342,6 +342,9 @@ private: } } + /// Now we don't block the Ctrl+C signal and second signal will terminate the program without waiting. + interrupt_listener.unblock(); + pool.wait(); total_watch.stop(); @@ -586,7 +589,6 @@ public: #ifndef __clang__ #pragma GCC optimize("-fno-var-tracking-assignments") #endif -#pragma GCC diagnostic ignored "-Wmissing-declarations" int mainEntryClickHouseBenchmark(int argc, char ** argv) { From 8b331cd9bb197c1372b55822b43e612e916d35b1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Dec 2021 22:41:47 +0300 Subject: [PATCH 0601/1260] Remove method from IStorage --- src/Storages/IStorage.h | 3 --- src/Storages/StorageReplicatedMergeTree.h | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index f1fd25d6d12..bcbc771815b 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -138,9 +138,6 @@ public: /// Returns true if the storage replicates SELECT, INSERT and ALTER commands among replicas. virtual bool supportsReplication() const { return false; } - /// Returns replica name for replicated storage - virtual String getReplicaName() const { return ""; } - /// Returns true if the storage supports parallel insert. virtual bool supportsParallelInsert() const { return false; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index e931efee735..efff3d6a2d7 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -253,7 +253,7 @@ public: /// Get best replica having this partition on a same type remote disk String getSharedDataReplica(const IMergeTreeDataPart & part, DiskType disk_type) const; - inline String getReplicaName() const override { return replica_name; } + inline String getReplicaName() const { return replica_name; } /// Restores table metadata if ZooKeeper lost it. /// Used only on restarted readonly replicas (not checked). All active (Committed) parts are moved to detached/ From 3ad438525a8200163023e64a70775685bced1f3b Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Dec 2021 18:35:54 +0300 Subject: [PATCH 0602/1260] Fix clang-tidy --- src/Processors/Transforms/buildPushingToViewsChain.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index fdd0150536a..17075e2b318 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -596,7 +596,7 @@ void PushingToLiveViewSink::consume(Chunk chunk) { Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); StorageLiveView::writeIntoLiveView(live_view, getHeader().cloneWithColumns(chunk.detachColumns()), context); - auto process = context->getProcessListElement(); + auto * process = context->getProcessListElement(); if (process) process->updateProgressIn(local_progress); ProfileEvents::increment(ProfileEvents::SelectedRows, local_progress.read_rows); @@ -619,7 +619,7 @@ void PushingToWindowViewSink::consume(Chunk chunk) Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); StorageWindowView::writeIntoWindowView( window_view, getHeader().cloneWithColumns(chunk.detachColumns()), context); - auto process = context->getProcessListElement(); + auto * process = context->getProcessListElement(); if (process) process->updateProgressIn(local_progress); ProfileEvents::increment(ProfileEvents::SelectedRows, local_progress.read_rows); From 903f3035798dd0f45341ac8e49354df6085c9246 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Dec 2021 18:36:42 +0300 Subject: [PATCH 0603/1260] Fix clang tidy --- src/Formats/FormatFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index a0a5550627d..2068de0d01c 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -236,7 +236,7 @@ InputFormatPtr FormatFactory::getInputFormat( static void addExistingProgressToOutputFormat(OutputFormatPtr format, ContextPtr context) { - auto element_id = context->getProcessListElement(); + auto * element_id = context->getProcessListElement(); if (element_id) { /// While preparing the query there might have been progress (for example in subscalar subqueries) so add it here From 4a1819c4d2f275e362ec56dbae4c4255cbeaa9ee Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Dec 2021 22:44:41 +0300 Subject: [PATCH 0604/1260] Fix jepsen check --- tests/ci/keeper_jepsen_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/keeper_jepsen_check.py b/tests/ci/keeper_jepsen_check.py index 2c2b8b4783f..b7acc92b0f3 100644 --- a/tests/ci/keeper_jepsen_check.py +++ b/tests/ci/keeper_jepsen_check.py @@ -122,7 +122,7 @@ if __name__ == "__main__": logging.info("Start at PR number %s, commit sha %s labels %s", pr_info.number, pr_info.sha, pr_info.labels) - if pr_info.number != 0 and 'jepsen-test' not in pr_info.labels(): + if pr_info.number != 0 and 'jepsen-test' not in pr_info.labels: logging.info("Not jepsen test label in labels list, skipping") sys.exit(0) From 1868570920fdbea34c0caa793c791a9f805c521c Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Wed, 29 Dec 2021 12:27:39 +0000 Subject: [PATCH 0605/1260] Add test for broken connection --- tests/integration/test_odbc_interaction/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 001a46e1237..8d3a8773bc4 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -338,6 +338,8 @@ def test_postgres_odbc_hashed_dictionary_with_schema(started_cluster): cursor.execute("truncate table clickhouse.test_table") cursor.execute("insert into clickhouse.test_table values(1, 1, 'hello'),(2, 2, 'world')") node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_hashed") + node1.exec_in_container(["ss", "-K", "dport", "postgresql"], privileged=True, user='root') + node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_hashed") assert_eq_with_retry(node1, "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(1))", "hello") assert_eq_with_retry(node1, "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(2))", "world") From e879aca58bbccb668eb746df46efa6f530127ab6 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 29 Dec 2021 23:21:47 +0300 Subject: [PATCH 0606/1260] Update run_check.py --- tests/ci/run_check.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 692cda18f20..2085a5c1e2b 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -68,6 +68,7 @@ TRUSTED_CONTRIBUTORS = {e.lower() for e in [ "YiuRULE", "zlobober", # Developer of YT "ilejn", # Arenadata, responsible for Kerberized Kafka + "thomoco", # ClickHouse ]} From 2e6c63dbcabb9f9bc577a3a49abe6d57e12982c4 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Wed, 29 Dec 2021 20:25:29 +0000 Subject: [PATCH 0607/1260] Fix launcher not being defined for ccache < 4.0 and slightly cleanup --- cmake/find/ccache.cmake | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/cmake/find/ccache.cmake b/cmake/find/ccache.cmake index 95ec3d8a034..9acc0423f67 100644 --- a/cmake/find/ccache.cmake +++ b/cmake/find/ccache.cmake @@ -31,6 +31,7 @@ if (CCACHE_FOUND AND NOT COMPILER_MATCHES_CCACHE) if (CCACHE_VERSION VERSION_GREATER "3.2.0" OR NOT CMAKE_CXX_COMPILER_ID STREQUAL "Clang") message(STATUS "Using ${CCACHE_FOUND} ${CCACHE_VERSION}") + set(LAUNCHER ${CCACHE_FOUND}) # debian (debhelpers) set SOURCE_DATE_EPOCH environment variable, that is # filled from the debian/changelog or current time. @@ -39,13 +40,8 @@ if (CCACHE_FOUND AND NOT COMPILER_MATCHES_CCACHE) # of the manifest, which do not allow to use previous cache, # - 4.2+ ccache ignores SOURCE_DATE_EPOCH for every file w/o __DATE__/__TIME__ # - # So for: - # - 4.2+ does not require any sloppiness - # - 4.0+ will ignore SOURCE_DATE_EPOCH environment variable. - if (CCACHE_VERSION VERSION_GREATER_EQUAL "4.2") - message(STATUS "ccache is 4.2+ no quirks for SOURCE_DATE_EPOCH required") - set(LAUNCHER ${CCACHE_FOUND}) - elseif (CCACHE_VERSION VERSION_GREATER_EQUAL "4.0") + # Exclude SOURCE_DATE_EPOCH env for ccache versions between [4.0, 4.2). + if (CCACHE_VERSION VERSION_GREATER_EQUAL "4.0" AND CCACHE_VERSION VERSION_LESS "4.2") message(STATUS "Ignore SOURCE_DATE_EPOCH for ccache") set(LAUNCHER env -u SOURCE_DATE_EPOCH ${CCACHE_FOUND}) endif() From fd1e9ffa664a4eb7f9bdd3184308d17a069b9d4e Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Wed, 29 Dec 2021 20:31:24 +0000 Subject: [PATCH 0608/1260] Return early if azure blob storage is not used, otherwise it might with fail with irrelevant errors --- cmake/find/blob_storage.cmake | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/cmake/find/blob_storage.cmake b/cmake/find/blob_storage.cmake index 74a907da7db..ec1b97f4695 100644 --- a/cmake/find/blob_storage.cmake +++ b/cmake/find/blob_storage.cmake @@ -1,14 +1,16 @@ option (ENABLE_AZURE_BLOB_STORAGE "Enable Azure blob storage" ${ENABLE_LIBRARIES}) -option(USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY - "Set to FALSE to use system Azure SDK instead of bundled (OFF currently not implemented)" - ON) - if (ENABLE_AZURE_BLOB_STORAGE) set(USE_AZURE_BLOB_STORAGE 1) set(AZURE_BLOB_STORAGE_LIBRARY azure_sdk) +else() + return() endif() +option(USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY + "Set to FALSE to use system Azure SDK instead of bundled (OFF currently not implemented)" + ON) + if ((NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/azure/sdk" OR NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/azure/cmake-modules") AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) From b4b7acabc5bf15dfc82221459668cacb3da49ede Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Wed, 29 Dec 2021 13:44:51 -0700 Subject: [PATCH 0609/1260] Updated adopters: remove smart quotes --- docs/en/introduction/adopters.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 8f6e9a3de52..c2660653907 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -60,7 +60,7 @@ toc_title: Adopters | Exness | Trading | Metrics, Logging | — | — | [Talk in Russian, May 2019](https://youtu.be/_rpU-TvSfZ8?t=3215) | | EventBunker.io | Serverless Data Processing | — | — | — | [Tweet, April 2021](https://twitter.com/Halil_D_/status/1379839133472985091) | | FastNetMon | DDoS Protection | Main Product | | — | [Official website](https://fastnetmon.com/docs-fnm-advanced/fastnetmon-advanced-traffic-persistency/) | -| Firebolt | Analytics | Main product | - | - | [YouTube Tech Talk](https://www.youtube.com/watch?v=9rW9uEJ15tU) | | Flipkart | e-Commerce | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=239) | | FunCorp | Games | | — | 14 bn records/day as of Jan 2021 | [Article](https://www.altinity.com/blog/migrating-from-redshift-to-clickhouse) | | Futurra Group | Analytics | — | — | — | [Article in Russian, December 2021](https://dou.ua/forums/topic/35587/) | @@ -72,7 +72,7 @@ toc_title: Adopters | Grouparoo | Data Warehouse Integrations | Main product | — | — | [Official Website, November 2021](https://www.grouparoo.com/integrations) | | HUYA | Video Streaming | Analytics | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/7.%20ClickHouse万亿数据分析实践%20李本旺(sundy-li)%20虎牙.pdf) | | Hydrolix | Cloud data platform | Main product | — | — | [Documentation](https://docs.hydrolix.io/guide/query) | -| Hystax | Cloud Operations | Observability Analytics | - | - | [Blog](https://hystax.com/clickhouse-for-real-time-cost-saving-analytics-how-to-stop-hammering-screws-and-use-an-electric-screwdriver/) | | ICA | FinTech | Risk Management | — | — | [Blog Post in English, Sep 2020](https://altinity.com/blog/clickhouse-vs-redshift-performance-for-fintech-risk-management?utm_campaign=ClickHouse%20vs%20RedShift&utm_content=143520807&utm_medium=social&utm_source=twitter&hss_channel=tw-3894792263) | | Idealista | Real Estate | Analytics | — | — | [Blog Post in English, April 2019](https://clickhouse.com/blog/en/clickhouse-meetup-in-madrid-on-april-2-2019) | | Infobaleen | AI markting tool | Analytics | — | — | [Official site](https://infobaleen.com) | @@ -85,7 +85,7 @@ toc_title: Adopters | Ivi | Online Cinema | Analytics, Monitoring | — | — | [Article in Russian, Jan 2018](https://habr.com/en/company/ivi/blog/347408/) | | Jinshuju 金数据 | BI Analytics | Main product | — | — | [Slides in Chinese, October 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/3.%20金数据数据架构调整方案Public.pdf) | | Jitsu | Cloud Software | Data Pipeline | — | — | [Documentation](https://jitsu.com/docs/destinations-configuration/clickhouse-destination), [Hacker News post](https://news.ycombinator.com/item?id=29106082) | -| JuiceFS | Storage | Shopping Cart | - | - | [Blog](https://juicefs.com/blog/en/posts/shopee-clickhouse-with-juicefs/) | | kakaocorp | Internet company | — | — | — | [if(kakao)2020](https://tv.kakao.com/channel/3693125/cliplink/414129353), [if(kakao)2021](https://if.kakao.com/session/24) | | Kodiak Data | Clouds | Main product | — | — | [Slides in Engish, April 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup13/kodiak_data.pdf) | | Kontur | Software Development | Metrics | — | — | [Talk in Russian, November 2018](https://www.youtube.com/watch?v=U4u4Bd0FtrY) | @@ -93,7 +93,7 @@ toc_title: Adopters | KGK Global | Vehicle monitoring | — | — | — | [Press release, June 2021](https://zoom.cnews.ru/news/item/530921) | | LANCOM Systems | Network Solutions | Traffic analysis | - | - | [ClickHouse Operator for Kubernetes](https://www.lancom-systems.com/), [Hacker News post] (https://news.ycombinator.com/item?id=29413660) | | Lawrence Berkeley National Laboratory | Research | Traffic analysis | 5 servers | 55 TiB | [Slides in English, April 2019](https://www.smitasin.com/presentations/2019-04-17_DOE-NSM.pdf) | -| Lever | Talent Management | Recruiting | - | - | [Hacker News post](https://news.ycombinator.com/item?id=29558544) | | LifeStreet | Ad network | Main product | 75 servers (3 replicas) | 5.27 PiB | [Blog post in Russian, February 2017](https://habr.com/en/post/322620/) | | Lookforsale | E-Commerce | — | — | — | [Job Posting, December 2021](https://telegram.me/javascript_jobs/587318) | | Mail.ru Cloud Solutions | Cloud services | Main product | — | — | [Article in Russian](https://mcs.mail.ru/help/db-create/clickhouse#) | @@ -113,7 +113,7 @@ toc_title: Adopters | Ok.ru | Social Network | — | 72 servers | 810 TB compressed, 50bn rows/day, 1.5 TB/day | [SmartData conference, October 2021](https://assets.ctfassets.net/oxjq45e8ilak/4JPHkbJenLgZhBGGyyonFP/57472ec6987003ec4078d0941740703b/____________________ClickHouse_______________________.pdf) | | Omnicomm | Transportation Monitoring | — | — | — | [Facebook post, October 2021](https://www.facebook.com/OmnicommTeam/posts/2824479777774500) | | OneAPM | Monitoring and Data Analysis | Main product | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/8.%20clickhouse在OneAPM的应用%20杜龙.pdf) | -| Opensee | Financial Analytics | Main product | - | - | [Blog](https://opensee.io/news/from-moscow-to-wall-street-the-remarkable-journey-of-clickhouse/) | | Open Targets | Genome Research | Genome Search | — | — | [Tweet, October 2021](https://twitter.com/OpenTargets/status/1452570865342758913?s=20), [Blog](https://blog.opentargets.org/graphql/) | | OZON | E-commerce | — | — | — | [Official website](https://job.ozon.ru/vacancy/razrabotchik-clickhouse-ekspluatatsiya-40991870/) | | Panelbear | Analytics | Monitoring and Analytics | — | — | [Tech Stack, November 2020](https://panelbear.com/blog/tech-stack/) | @@ -179,7 +179,7 @@ toc_title: Adopters | Yandex DataLens | Business Intelligence | Main product | — | — | [Slides in Russian, December 2019](https://presentations.clickhouse.com/meetup38/datalens.pdf) | | Yandex Market | e-Commerce | Metrics, Logging | — | — | [Talk in Russian, January 2019](https://youtu.be/_l1qP0DyBcA?t=478) | | Yandex Metrica | Web analytics | Macin product | 630 servers in one cluster, 360 servers in another cluster, 1862 servers in one department | 133 PiB / 8.31 PiB / 120 trillion records | [Slides, February 2020](https://presentations.clickhouse.com/meetup40/introduction/#13) | -| | Analytics | Main product | - | - | [Integration](https://www.yellowfinbi.com/campaign/yellowfin-9-whats-new#el-30219e0e) | | Yotascale | Cloud | Data pipeline | — | 2 bn records/day | [LinkedIn (Accomplishments)](https://www.linkedin.com/in/adilsaleem/) | | Your Analytics | Product Analytics | Main Product | — | - | [Tweet, November 2021](https://twitter.com/mikenikles/status/1459737241165565953) | | Zagrava Trading | — | — | — | — | [Job offer, May 2021](https://twitter.com/datastackjobs/status/1394707267082063874) | From 88383b67152a63b21ee80a1332e1b1eb04d23609 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 30 Dec 2021 10:08:33 +0800 Subject: [PATCH 0610/1260] some change --- src/Common/LRUResourceCache.h | 84 +++++++++++-------- src/Common/tests/gtest_lru_resource_cache.cpp | 19 +++-- 2 files changed, 58 insertions(+), 45 deletions(-) diff --git a/src/Common/LRUResourceCache.h b/src/Common/LRUResourceCache.h index 3dd078bec5a..192beb5b803 100644 --- a/src/Common/LRUResourceCache.h +++ b/src/Common/LRUResourceCache.h @@ -39,16 +39,7 @@ public: { public: ~MappedHolder() { cache->release(key); } - Mapped & value() { return *(val.get()); } - static bool tryRemove(std::unique_ptr * holder_ptr) - { - auto & holder = *holder_ptr; - auto cache = holder->cache; - auto key = holder->key; - *holder_ptr = nullptr; - return cache->tryRemove(key); - } - + Mapped & value() { return *val.get(); } MappedHolder(LRUResourceCache * cache_, const Key & key_, MappedPtr value_) : cache(cache_), key(key_), val(value_) { } protected: @@ -61,10 +52,10 @@ public: // use get() or getOrSet() to access the elements MappedHolderPtr get(const Key & key) { - auto mappedptr = getImpl(key); - if (!mappedptr) + auto mapped_ptr = getImpl(key); + if (!mapped_ptr) return nullptr; - return std::make_unique(this, key, mappedptr); + return std::make_unique(this, key, mapped_ptr); } template MappedHolderPtr getOrSet(const Key & key, LoadFunc && load_func) @@ -75,6 +66,24 @@ public: return std::make_unique(this, key, mappedptr); } + // If the key's reference_count = 0, delete it immediately. otherwise, mark it expired, and delete in release + void tryRemove(const Key & key) + { + std::lock_guard lock(mutex); + auto it = cells.find(key); + if (it == cells.end()) + return; + auto & cell = it->second; + if (cell.reference_count == 0) + { + queue.erase(cell.queue_iterator); + current_weight -= cell.weight; + cells.erase(it); + } + else + cell.expired = true; + } + LRUResourceCache(size_t max_weight_, size_t max_element_size_ = 0) : max_weight(max_weight_), max_element_size(max_element_size_) { } ~LRUResourceCache() = default; @@ -109,6 +118,7 @@ private: size_t weight = 0; LRUQueueIterator queue_iterator; size_t reference_count = 0; + bool expired = false; }; using Cells = std::unordered_map; @@ -140,12 +150,23 @@ private: { std::lock_guard lock(mutex); auto it = cells.find(key); - if (it != cells.end()) + if (it != cells.end() && !it->second.expired) { - hits++; - it->second.reference_count += 1; - queue.splice(queue.end(), queue, it->second.queue_iterator); - return it->second.value; + if (!it->second.expired) + { + hits++; + it->second.reference_count += 1; + queue.splice(queue.end(), queue, it->second.queue_iterator); + return it->second.value; + } + else if (it->second.reference_count > 0) + return nullptr; + else + { + // should not reach here + LOG_ERROR(&Poco::Logger::get("LRUResourceCache"), "element is in invalid status."); + abort(); + } } misses++; insert_token = acquireInsertToken(key); @@ -182,7 +203,7 @@ private: { std::lock_guard lock(mutex); auto it = cells.find(key); - if (it == cells.end()) + if (it == cells.end() || it->second.expired) { misses++; return nullptr; @@ -203,7 +224,14 @@ private: LOG_ERROR(&Poco::Logger::get("LRUResourceCache"), "try to release an invalid element"); abort(); } - it->second.reference_count -= 1; + auto & cell = it->second; + cell.reference_count -= 1; + if (cell.expired && cell.reference_count == 0) + { + queue.erase(cell.queue_iterator); + current_weight -= cell.weight; + cells.erase(it); + } } InsertToken * acquireInsertToken(const Key & key) @@ -275,21 +303,5 @@ private: new_cell.queue_iterator = queue.insert(queue.end(), insert_key); return &new_cell; } - - // If you want to update a value, call tryRemove() at first and then call acquire() with load_func. - bool tryRemove(const Key & key) - { - std::lock_guard guard(mutex); - auto it = cells.find(key); - if (it == cells.end()) - return true; - auto & cell = it->second; - if (cell.reference_count) - return false; - queue.erase(cell.queue_iterator); - current_weight -= cell.weight; - cells.erase(it); - return true; - } }; } diff --git a/src/Common/tests/gtest_lru_resource_cache.cpp b/src/Common/tests/gtest_lru_resource_cache.cpp index 0d58fd571a2..0eb0ec95476 100644 --- a/src/Common/tests/gtest_lru_resource_cache.cpp +++ b/src/Common/tests/gtest_lru_resource_cache.cpp @@ -29,17 +29,16 @@ TEST(LRUResourceCache, remove) auto holder0 = mcache.getOrSet(1, load_int); auto holder1 = mcache.getOrSet(1, load_int); - auto succ = MyCache::MappedHolder::tryRemove(&holder0); - ASSERT_TRUE(!succ); + mcache.tryRemove(1); holder0 = mcache.get(1); - ASSERT_TRUE(holder0 != nullptr); - ASSERT_TRUE(holder0->value() == 10); + ASSERT_TRUE(holder0 == nullptr); + auto n = mcache.size(); + ASSERT_TRUE(n == 1); holder0 = nullptr; - succ = MyCache::MappedHolder::tryRemove(&holder1); - ASSERT_TRUE(succ); - holder1 = mcache.get(1); - ASSERT_TRUE(holder1 == nullptr); + holder1 = nullptr; + n = mcache.size(); + ASSERT_TRUE(n == 0); } struct MyWeight @@ -253,9 +252,10 @@ TEST(LRUResourceCache, re_get) int x = 2; auto load_int = [&] { return std::make_shared(x); }; auto holder1 = mcache.getOrSet(1, load_int); - MyCache::MappedHolder::tryRemove(&holder1); + mcache.tryRemove(1); x = 11; + holder1 = nullptr; holder1 = mcache.getOrSet(1, load_int); ASSERT_TRUE(holder1 != nullptr); @@ -267,3 +267,4 @@ TEST(LRUResourceCache, re_get) ASSERT_TRUE(holder1 != nullptr); ASSERT_TRUE(holder1->value() == 11); } + From f3f0ff0f2307479fd19cda3af9a521f97c7afe20 Mon Sep 17 00:00:00 2001 From: OnePiece <54787696+zhongyuankai@users.noreply.github.com> Date: Thu, 30 Dec 2021 10:12:37 +0800 Subject: [PATCH 0611/1260] Update StorageSystemTables.cpp --- src/Storages/System/StorageSystemTables.cpp | 34 ++++++++++++++++++++- 1 file changed, 33 insertions(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index ac52f0afb32..a6109373b6c 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -88,6 +88,26 @@ static ColumnPtr getFilteredDatabases(const SelectQueryInfo & query_info, Contex return block.getByPosition(0).column; } +static ColumnPtr getFilteredTables(const ASTPtr & query, ColumnPtr & filtered_databases_column, const Context & context) +{ + MutableColumnPtr column = ColumnString::create(); + + size_t database_idx = 0; + for (; database_idx < filtered_databases_column->size(); ++database_idx) + { + auto database_name = filtered_databases_column->getDataAt(database_idx).toString(); + DatabasePtr database = DatabaseCatalog::instance().getDatabase(database_name); + DatabaseTablesIteratorPtr table_it = database->getTablesIterator(context); + + for (; table_it->isValid(); table_it->next()) + column->insert(table_it->name()); + } + + Block block {ColumnWithTypeAndName(std::move(column), std::make_shared(), "name")}; + VirtualColumnUtils::filterBlockWithQuery(query, block, context); + return block.getByPosition(0).column; +} + /// Avoid heavy operation on tables if we only queried columns that we can get without table object. /// Otherwise it will require table initialization for Lazy database. static bool needLockStructure(const DatabasePtr & database, const Block & header) @@ -112,12 +132,19 @@ public: Block header, UInt64 max_block_size_, ColumnPtr databases_, + ColumnPtr tables_, ContextPtr context_) : SourceWithProgress(std::move(header)) , columns_mask(std::move(columns_mask_)) , max_block_size(max_block_size_) , databases(std::move(databases_)) - , context(Context::createCopy(context_)) {} + , context(Context::createCopy(context_)) + { + size_t size = tables_->size(); + tables.reserve(size); + for (size_t idx = 0; idx < size; ++idx) + tables.insert(tables_->getDataAt(idx).toString()); + } String getName() const override { return "Tables"; } @@ -239,6 +266,9 @@ protected: for (; rows_count < max_block_size && tables_it->isValid(); tables_it->next()) { auto table_name = tables_it->name(); + if (!tables.contains(table_name)) + continue; + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) continue; @@ -514,6 +544,7 @@ private: std::vector columns_mask; UInt64 max_block_size; ColumnPtr databases; + NameSet tables; size_t database_idx = 0; DatabaseTablesIteratorPtr tables_it; ContextPtr context; @@ -552,6 +583,7 @@ Pipe StorageSystemTables::read( } ColumnPtr filtered_databases_column = getFilteredDatabases(query_info, context); + ColumnPtr filtered_tables_column = getFilteredTables(query_info.query, filtered_databases_column, context); return Pipe(std::make_shared( std::move(columns_mask), std::move(res_block), max_block_size, std::move(filtered_databases_column), context)); From 54d78bde0dfe484c43571611e92cd4aa034f7a33 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 30 Dec 2021 10:15:56 +0800 Subject: [PATCH 0612/1260] update comments --- src/Common/LRUResourceCache.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/LRUResourceCache.h b/src/Common/LRUResourceCache.h index 192beb5b803..8117e37b844 100644 --- a/src/Common/LRUResourceCache.h +++ b/src/Common/LRUResourceCache.h @@ -141,7 +141,7 @@ private: std::atomic misses{0}; std::atomic evict_count{0}; - // - load_func : when key is not exists in cache, load_func is called to generate a new key + // - load_func : when key is not exists in cache, load_func is called to generate a new value // - return: is null when there is no more space for the new value or the old value is in used. template MappedPtr getImpl(const Key & key, LoadFunc && load_func) From cec282b9bea8cca9c60d55c40e312893fbfc3439 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 30 Dec 2021 03:19:37 +0000 Subject: [PATCH 0613/1260] Fix docs typos --- docs/ru/sql-reference/functions/nlp-functions.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/sql-reference/functions/nlp-functions.md b/docs/ru/sql-reference/functions/nlp-functions.md index 250403ab127..992a7d6ccf3 100644 --- a/docs/ru/sql-reference/functions/nlp-functions.md +++ b/docs/ru/sql-reference/functions/nlp-functions.md @@ -3,10 +3,10 @@ toc_priority: 67 toc_title: NLP --- -# [экспериментально] Функции для работы с ествественным языком {#nlp-functions} +# [экспериментально] Функции для работы с естественным языком {#nlp-functions} !!! warning "Предупреждение" - Сейчас использование функций для работы с ествественным языком является экспериментальной возможностью. Чтобы использовать данные функции, включите настройку `allow_experimental_nlp_functions = 1`. + Сейчас использование функций для работы с естественным языком является экспериментальной возможностью. Чтобы использовать данные функции, включите настройку `allow_experimental_nlp_functions = 1`. ## stem {#stem} @@ -84,7 +84,7 @@ SELECT lemmatize('en', 'wolves'); Находит синонимы к заданному слову. Представлены два типа расширений словарей: `plain` и `wordnet`. -Для работы расширения типа `plain` необходимо указать путь до простого текстового файла, где каждая строка соотвествует одному набору синонимов. Слова в данной строке должны быть разделены с помощью пробела или знака табуляции. +Для работы расширения типа `plain` необходимо указать путь до простого текстового файла, где каждая строка соответствует одному набору синонимов. Слова в данной строке должны быть разделены с помощью пробела или знака табуляции. Для работы расширения типа `plain` необходимо указать путь до WordNet тезауруса. Тезаурус должен содержать WordNet sense index. From db08e36472dc9488520aa0c8ea39f7ebb42f1b61 Mon Sep 17 00:00:00 2001 From: "Chun-Sheng, Li" Date: Thu, 30 Dec 2021 11:21:57 +0800 Subject: [PATCH 0614/1260] Adding & improve cURL compressed data examples --- docs/en/interfaces/http.md | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 38e729fde0b..202ee70ae6f 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -186,7 +186,7 @@ $ echo "SELECT 1" | gzip -c | \ ``` ``` bash -# Receiving compressed data from the server +# Receiving compressed data archive from the server $ curl -vsS "http://localhost:8123/?enable_http_compression=1" \ -H 'Accept-Encoding: gzip' --output result.gz -d 'SELECT number FROM system.numbers LIMIT 3' $ zcat result.gz @@ -195,6 +195,15 @@ $ zcat result.gz 2 ``` +```bash +# Receiving compressed data from the server and using the gunzip to receive decompressed data +$ curl -sS "http://default:openstack@localhost:8123/?enable_http_compression=1" \ + -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 3' | gunzip - +0 +1 +2 +``` + ## Default Database {#default-database} You can use the ‘database’ URL parameter or the ‘X-ClickHouse-Database’ header to specify the default database. From 38605d6be3a3ad19952624ffca4c94885cefe930 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 30 Dec 2021 12:20:14 +0800 Subject: [PATCH 0615/1260] rename test cases --- src/Common/tests/gtest_lru_resource_cache.cpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Common/tests/gtest_lru_resource_cache.cpp b/src/Common/tests/gtest_lru_resource_cache.cpp index 0eb0ec95476..fcdd2902522 100644 --- a/src/Common/tests/gtest_lru_resource_cache.cpp +++ b/src/Common/tests/gtest_lru_resource_cache.cpp @@ -46,7 +46,7 @@ struct MyWeight size_t operator()(const int & x) const { return static_cast(x); } }; -TEST(LRUResourceCache, evict_on_weight) +TEST(LRUResourceCache, evictOnWweight) { using MyCache = DB::LRUResourceCache; auto mcache = MyCache(5, 10); @@ -75,7 +75,7 @@ TEST(LRUResourceCache, evict_on_weight) ASSERT_TRUE(holder3 != nullptr); } -TEST(LRUResourceCache, evict_on_weight_v2) +TEST(LRUResourceCache, evictOnWeightV2) { using MyCache = DB::LRUResourceCache; auto mcache = MyCache(5, 10); @@ -107,7 +107,7 @@ TEST(LRUResourceCache, evict_on_weight_v2) ASSERT_TRUE(holder3 != nullptr); } -TEST(LRUResourceCache, evict_on_weight_v3) +TEST(LRUResourceCache, evictOnWeightV3) { using MyCache = DB::LRUResourceCache; auto mcache = MyCache(5, 10); @@ -139,7 +139,7 @@ TEST(LRUResourceCache, evict_on_weight_v3) ASSERT_TRUE(holder3 != nullptr); } -TEST(LRUResourceCache, evict_on_size) +TEST(LRUResourceCache, evictOnSize) { using MyCache = DB::LRUResourceCache; auto mcache = MyCache(5, 2); @@ -168,7 +168,7 @@ TEST(LRUResourceCache, evict_on_size) ASSERT_TRUE(holder3 != nullptr); } -TEST(LRUResourceCache, not_evict_used_element) +TEST(LRUResourceCache, notEvictUsedElement) { using MyCache = DB::LRUResourceCache; auto mcache = MyCache(7, 10); @@ -201,7 +201,7 @@ TEST(LRUResourceCache, not_evict_used_element) ASSERT_TRUE(holder4 != nullptr); } -TEST(LRUResourceCache, get_fail) +TEST(LRUResourceCache, getFail) { using MyCache = DB::LRUResourceCache; auto mcache = MyCache(5, 10); @@ -224,7 +224,7 @@ TEST(LRUResourceCache, get_fail) ASSERT_TRUE(holder3 == nullptr); } -TEST(LRUResourceCache, dup_get) +TEST(LRUResourceCache, dupGet) { using MyCache = DB::LRUResourceCache; auto mcache = MyCache(20, 10); @@ -245,7 +245,7 @@ TEST(LRUResourceCache, dup_get) ASSERT_TRUE(holder1->value() == 2); } -TEST(LRUResourceCache, re_get) +TEST(LRUResourceCache, reGet) { using MyCache = DB::LRUResourceCache; auto mcache = MyCache(20, 10); From 79e15e84d529bc3ba40153feb2a9524248e2e914 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 30 Dec 2021 04:47:34 +0000 Subject: [PATCH 0616/1260] Refactor --- src/IO/BrotliReadBuffer.cpp | 6 +++--- src/IO/BrotliReadBuffer.h | 2 +- src/IO/Bzip2ReadBuffer.cpp | 8 ++++---- src/IO/Bzip2ReadBuffer.h | 2 +- src/IO/LZMAInflatingReadBuffer.cpp | 6 +++--- src/IO/LZMAInflatingReadBuffer.h | 2 +- src/IO/Lz4InflatingReadBuffer.cpp | 4 ++-- src/IO/Lz4InflatingReadBuffer.h | 2 +- src/IO/ZlibInflatingReadBuffer.cpp | 6 +++--- src/IO/ZlibInflatingReadBuffer.h | 2 +- src/IO/ZstdInflatingReadBuffer.cpp | 4 ++-- src/IO/ZstdInflatingReadBuffer.h | 2 +- 12 files changed, 23 insertions(+), 23 deletions(-) diff --git a/src/IO/BrotliReadBuffer.cpp b/src/IO/BrotliReadBuffer.cpp index b66bbf45054..77069746153 100644 --- a/src/IO/BrotliReadBuffer.cpp +++ b/src/IO/BrotliReadBuffer.cpp @@ -39,7 +39,7 @@ BrotliReadBuffer::BrotliReadBuffer(std::unique_ptr in_, size_t buf_s , in_data(nullptr) , out_capacity(0) , out_data(nullptr) - , eof(false) + , eof_flag(false) { } @@ -47,7 +47,7 @@ BrotliReadBuffer::~BrotliReadBuffer() = default; bool BrotliReadBuffer::nextImpl() { - if (eof) + if (eof_flag) return false; if (!in_available) @@ -74,7 +74,7 @@ bool BrotliReadBuffer::nextImpl() { if (in->eof()) { - eof = true; + eof_flag = true; return !working_buffer.empty(); } else diff --git a/src/IO/BrotliReadBuffer.h b/src/IO/BrotliReadBuffer.h index 0fa999d1de5..44a7dc7ddbd 100644 --- a/src/IO/BrotliReadBuffer.h +++ b/src/IO/BrotliReadBuffer.h @@ -32,7 +32,7 @@ private: size_t out_capacity; uint8_t * out_data; - bool eof; + bool eof_flag; }; } diff --git a/src/IO/Bzip2ReadBuffer.cpp b/src/IO/Bzip2ReadBuffer.cpp index df9a8d5b369..c2060612757 100644 --- a/src/IO/Bzip2ReadBuffer.cpp +++ b/src/IO/Bzip2ReadBuffer.cpp @@ -42,7 +42,7 @@ Bzip2ReadBuffer::Bzip2ReadBuffer(std::unique_ptr in_, size_t buf_siz : BufferWithOwnMemory(buf_size, existing_memory, alignment) , in(std::move(in_)) , bz(std::make_unique()) - , eof(false) + , eof_flag(false) { } @@ -50,7 +50,7 @@ Bzip2ReadBuffer::~Bzip2ReadBuffer() = default; bool Bzip2ReadBuffer::nextImpl() { - if (eof) + if (eof_flag) return false; if (!bz->stream.avail_in) @@ -72,7 +72,7 @@ bool Bzip2ReadBuffer::nextImpl() { if (in->eof()) { - eof = true; + eof_flag = true; return !working_buffer.empty(); } else @@ -91,7 +91,7 @@ bool Bzip2ReadBuffer::nextImpl() if (in->eof()) { - eof = true; + eof_flag = true; throw Exception(ErrorCodes::UNEXPECTED_END_OF_FILE, "Unexpected end of bzip2 archive"); } diff --git a/src/IO/Bzip2ReadBuffer.h b/src/IO/Bzip2ReadBuffer.h index dc113800683..de1e61ee388 100644 --- a/src/IO/Bzip2ReadBuffer.h +++ b/src/IO/Bzip2ReadBuffer.h @@ -26,7 +26,7 @@ private: class Bzip2StateWrapper; std::unique_ptr bz; - bool eof; + bool eof_flag; }; } diff --git a/src/IO/LZMAInflatingReadBuffer.cpp b/src/IO/LZMAInflatingReadBuffer.cpp index f2df6bdca6a..80da7421fc3 100644 --- a/src/IO/LZMAInflatingReadBuffer.cpp +++ b/src/IO/LZMAInflatingReadBuffer.cpp @@ -7,7 +7,7 @@ namespace ErrorCodes extern const int LZMA_STREAM_DECODER_FAILED; } LZMAInflatingReadBuffer::LZMAInflatingReadBuffer(std::unique_ptr in_, size_t buf_size, char * existing_memory, size_t alignment) - : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)), eof(false) + : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)), eof_flag(false) { lstr = LZMA_STREAM_INIT; lstr.allocator = nullptr; @@ -36,7 +36,7 @@ LZMAInflatingReadBuffer::~LZMAInflatingReadBuffer() bool LZMAInflatingReadBuffer::nextImpl() { - if (eof) + if (eof_flag) return false; lzma_action action = LZMA_RUN; @@ -64,7 +64,7 @@ bool LZMAInflatingReadBuffer::nextImpl() { if (in->eof()) { - eof = true; + eof_flag = true; return !working_buffer.empty(); } else diff --git a/src/IO/LZMAInflatingReadBuffer.h b/src/IO/LZMAInflatingReadBuffer.h index 18922f64516..2d676eeeeb3 100644 --- a/src/IO/LZMAInflatingReadBuffer.h +++ b/src/IO/LZMAInflatingReadBuffer.h @@ -25,7 +25,7 @@ private: std::unique_ptr in; lzma_stream lstr; - bool eof; + bool eof_flag; }; } diff --git a/src/IO/Lz4InflatingReadBuffer.cpp b/src/IO/Lz4InflatingReadBuffer.cpp index 22bce94cad2..61e912d440c 100644 --- a/src/IO/Lz4InflatingReadBuffer.cpp +++ b/src/IO/Lz4InflatingReadBuffer.cpp @@ -32,7 +32,7 @@ Lz4InflatingReadBuffer::~Lz4InflatingReadBuffer() bool Lz4InflatingReadBuffer::nextImpl() { - if (eof) + if (eof_flag) return false; if (!in_available) @@ -66,7 +66,7 @@ bool Lz4InflatingReadBuffer::nextImpl() if (in->eof()) { - eof = true; + eof_flag = true; return !working_buffer.empty(); } diff --git a/src/IO/Lz4InflatingReadBuffer.h b/src/IO/Lz4InflatingReadBuffer.h index 0462d85adf7..d4d81f8765c 100644 --- a/src/IO/Lz4InflatingReadBuffer.h +++ b/src/IO/Lz4InflatingReadBuffer.h @@ -35,7 +35,7 @@ private: size_t in_available; size_t out_available; - bool eof = false; + bool eof_flag = false; }; } diff --git a/src/IO/ZlibInflatingReadBuffer.cpp b/src/IO/ZlibInflatingReadBuffer.cpp index 472399dea3d..28426e920ef 100644 --- a/src/IO/ZlibInflatingReadBuffer.cpp +++ b/src/IO/ZlibInflatingReadBuffer.cpp @@ -16,7 +16,7 @@ ZlibInflatingReadBuffer::ZlibInflatingReadBuffer( size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment) , in(std::move(in_)) - , eof(false) + , eof_flag(false) { zstr.zalloc = nullptr; zstr.zfree = nullptr; @@ -54,7 +54,7 @@ bool ZlibInflatingReadBuffer::nextImpl() do { /// if we already found eof, we shouldn't do anything - if (eof) + if (eof_flag) return false; /// if there is no available bytes in zstr, move ptr to next available data @@ -83,7 +83,7 @@ bool ZlibInflatingReadBuffer::nextImpl() /// * false if there is no data in working buffer if (in->eof()) { - eof = true; + eof_flag = true; return !working_buffer.empty(); } /// If it is not end of file, we need to reset zstr and return true, because we still have some data to read diff --git a/src/IO/ZlibInflatingReadBuffer.h b/src/IO/ZlibInflatingReadBuffer.h index b8c141e9b9b..905ab0cd3fc 100644 --- a/src/IO/ZlibInflatingReadBuffer.h +++ b/src/IO/ZlibInflatingReadBuffer.h @@ -33,7 +33,7 @@ private: std::unique_ptr in; z_stream zstr; - bool eof; + bool eof_flag; }; } diff --git a/src/IO/ZstdInflatingReadBuffer.cpp b/src/IO/ZstdInflatingReadBuffer.cpp index ce89f09f955..6f244dc5a75 100644 --- a/src/IO/ZstdInflatingReadBuffer.cpp +++ b/src/IO/ZstdInflatingReadBuffer.cpp @@ -31,7 +31,7 @@ bool ZstdInflatingReadBuffer::nextImpl() do { // If it is known that end of file was reached, return false - if (eof) + if (eof_flag) return false; /// If end was reached, get next part @@ -64,7 +64,7 @@ bool ZstdInflatingReadBuffer::nextImpl() /// If end of file is reached, fill eof variable and return true if there is some data in buffer, otherwise return false if (in->eof()) { - eof = true; + eof_flag = true; return !working_buffer.empty(); } /// It is possible, that input buffer is not at eof yet, but nothing was decompressed in current iteration. diff --git a/src/IO/ZstdInflatingReadBuffer.h b/src/IO/ZstdInflatingReadBuffer.h index e6e2dad0ad5..ec80b860e0e 100644 --- a/src/IO/ZstdInflatingReadBuffer.h +++ b/src/IO/ZstdInflatingReadBuffer.h @@ -31,7 +31,7 @@ private: ZSTD_DCtx * dctx; ZSTD_inBuffer input; ZSTD_outBuffer output; - bool eof = false; + bool eof_flag = false; }; } From b14f485f8554c076988f176589af5943698e955f Mon Sep 17 00:00:00 2001 From: OnePiece <54787696+zhongyuankai@users.noreply.github.com> Date: Thu, 30 Dec 2021 13:13:41 +0800 Subject: [PATCH 0617/1260] Update StorageSystemTables.cpp --- src/Storages/System/StorageSystemTables.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index a6109373b6c..d3f18861d01 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -88,7 +88,7 @@ static ColumnPtr getFilteredDatabases(const SelectQueryInfo & query_info, Contex return block.getByPosition(0).column; } -static ColumnPtr getFilteredTables(const ASTPtr & query, ColumnPtr & filtered_databases_column, const Context & context) +static ColumnPtr getFilteredTables(const ASTPtr & query, ColumnPtr & filtered_databases_column, ContextPtr context) { MutableColumnPtr column = ColumnString::create(); From 0c714e8c7e8616b22e4f819775be8c01c254392f Mon Sep 17 00:00:00 2001 From: OnePiece <54787696+zhongyuankai@users.noreply.github.com> Date: Thu, 30 Dec 2021 13:37:27 +0800 Subject: [PATCH 0618/1260] Update StorageSystemTables.cpp --- src/Storages/System/StorageSystemTables.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index d3f18861d01..3ec1849b73a 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -138,12 +138,12 @@ public: , columns_mask(std::move(columns_mask_)) , max_block_size(max_block_size_) , databases(std::move(databases_)) - , context(Context::createCopy(context_)) + , context(Context::createCopy(context_)) { size_t size = tables_->size(); tables.reserve(size); for (size_t idx = 0; idx < size; ++idx) - tables.insert(tables_->getDataAt(idx).toString()); + tables.insert(tables_->getDataAt(idx).toString()); } String getName() const override { return "Tables"; } @@ -586,7 +586,7 @@ Pipe StorageSystemTables::read( ColumnPtr filtered_tables_column = getFilteredTables(query_info.query, filtered_databases_column, context); return Pipe(std::make_shared( - std::move(columns_mask), std::move(res_block), max_block_size, std::move(filtered_databases_column), context)); + std::move(columns_mask), std::move(res_block), max_block_size, std::move(filtered_databases_column), std::move(filtered_tables_column), context)); } } From 71f45f981ebf0b3989c884d961aa9e5f7cecc0ca Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 30 Dec 2021 10:13:38 +0400 Subject: [PATCH 0619/1260] Revert "Better cmake script for azure blob" --- cmake/find/blob_storage.cmake | 37 ++++++++++++++++++----------------- 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/cmake/find/blob_storage.cmake b/cmake/find/blob_storage.cmake index 271b1e595a6..74a907da7db 100644 --- a/cmake/find/blob_storage.cmake +++ b/cmake/find/blob_storage.cmake @@ -7,23 +7,24 @@ option(USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY if (ENABLE_AZURE_BLOB_STORAGE) set(USE_AZURE_BLOB_STORAGE 1) set(AZURE_BLOB_STORAGE_LIBRARY azure_sdk) - - if ((NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/azure/sdk" - OR NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/azure/cmake-modules") - AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) - message (WARNING "submodule contrib/azure is missing. to fix try run: \n git submodule update --init") - set(USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY OFF) - set(USE_AZURE_BLOB_STORAGE 0) - endif () - - if (NOT USE_INTERNAL_SSL_LIBRARY AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) - message (FATAL_ERROR "Currently Blob Storage support can be built only with internal SSL library") - endif() - - if (NOT USE_INTERNAL_CURL AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) - message (FATAL_ERROR "Currently Blob Storage support can be built only with internal curl library") - endif() - endif() -message (STATUS "Using Azure Blob Storage - ${USE_AZURE_BLOB_STORAGE}") +if ((NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/azure/sdk" + OR NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/azure/cmake-modules") + AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) + message (WARNING "submodule contrib/azure is missing. to fix try run: \n git submodule update --init") + set(USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY OFF) + set(USE_AZURE_BLOB_STORAGE 0) +endif () + +if (NOT USE_INTERNAL_SSL_LIBRARY AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) + message (FATAL_ERROR "Currently Blob Storage support can be built only with internal SSL library") +endif() + +if (NOT USE_INTERNAL_CURL AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) + message (FATAL_ERROR "Currently Blob Storage support can be built only with internal curl library") +endif() + +if (USE_AZURE_BLOB_STORAGE) + message (STATUS "Using Azure Blob Storage - ${USE_AZURE_BLOB_STORAGE}") +endif() From e8c9079bb89a32d7bed5d890c096c7147aa420ce Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 30 Dec 2021 14:34:08 +0800 Subject: [PATCH 0620/1260] rename test cases --- src/Common/tests/gtest_lru_cache.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/tests/gtest_lru_cache.cpp b/src/Common/tests/gtest_lru_cache.cpp index 8a8b2ff3de8..7694a76ea72 100644 --- a/src/Common/tests/gtest_lru_cache.cpp +++ b/src/Common/tests/gtest_lru_cache.cpp @@ -47,7 +47,7 @@ struct ValueWeight size_t operator()(const size_t & x) const { return x; } }; -TEST(LRUCache, evict_on_size) +TEST(LRUCache, evictOnSize) { using SimpleLRUCache = DB::LRUCache; auto lru_cache = SimpleLRUCache(20, 3); @@ -63,7 +63,7 @@ TEST(LRUCache, evict_on_size) ASSERT_TRUE(value == nullptr); } -TEST(LRUCache, evict_on_weight) +TEST(LRUCache, evictOnWeight) { using SimpleLRUCache = DB::LRUCache, ValueWeight>; auto lru_cache = SimpleLRUCache(10, 10); From 52ad7a45381a17359baa93e2f636c0673085a7a5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 30 Dec 2021 10:25:10 +0300 Subject: [PATCH 0621/1260] Fix test_storage_kafka failures by adjusting retention.ms This test uses predefined timestamps, and default retention.ms is too small: kafka1_1 | [2021-12-28 21:40:21,842] INFO Created log for partition virt2_0-0 in /var/lib/kafka/data with properties {compression.type -> producer, message.format.version -> 2.2-IV1, file.delete.delay.ms -> 60000, max.message.bytes -> 1000012, min.compaction.lag.ms -> 0, message.timestamp.type -> CreateTime, message.downconversion.enable -> true, min.insync.replicas -> 1, segment.jitter.ms -> 0, preallocate -> false, min.cleanable.dirty.ratio -> 0.5, index.interval.bytes -> 4096, unclean.leader.election.enable -> false, retention.bytes -> -1, delete.retention.ms -> 86400000, cleanup.policy -> [delete], flush.ms -> 9223372036854775807, segment.ms -> 604800000, segment.bytes -> 1073741824, retention.ms -> 604800000, message.timestamp.difference.max.ms -> 9223372036854775807, segment.index.bytes -> 10485760, flush.messages -> 9223372036854775807}. (kafka.log.LogManager) kafka1_1 | [2021-12-28 21:40:24,540] INFO [Log partition=virt2_0-0, dir=/var/lib/kafka/data] Found deletable segments with base offsets [0] due to retention time 604800000ms breach (kafka.log.Log) v2: fix tests with 0 timestamp_ms and this should be all tests that was left: $ fgrep 'Found deletable segments with base offsets' test_storage_kafka/_instances_0/docker.log kafka1_1 | [2021-12-29 09:46:15,610] INFO [Log partition=avro1-0, dir=/var/lib/kafka/data] Found deletable segments with base offsets [0] due to retention time 604800000ms breach (kafka.log.Log) kafka1_1 | [2021-12-29 09:51:15,609] INFO [Log partition=virt1-0, dir=/var/lib/kafka/data] Found deletable segments with base offsets [0] due to retention time 604800000ms breach (kafka.log.Log) --- tests/integration/test_storage_kafka/test.py | 29 ++++++++++++++++++-- 1 file changed, 27 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 1ee7f3cf125..ce4ec3df867 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -1498,6 +1498,13 @@ def test_kafka_flush_on_big_message(kafka_cluster): def test_kafka_virtual_columns(kafka_cluster): + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) + topic_config = { + # default retention, since predefined timestamp_ms is used. + 'retention.ms': '-1', + } + kafka_create_topic(admin_client, "virt1", config=topic_config) + instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) ENGINE = Kafka @@ -1530,6 +1537,13 @@ def test_kafka_virtual_columns(kafka_cluster): def test_kafka_virtual_columns_with_materialized_view(kafka_cluster): + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) + topic_config = { + # default retention, since predefined timestamp_ms is used. + 'retention.ms': '-1', + } + kafka_create_topic(admin_client, "virt2", config=topic_config) + instance.query(''' DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.consumer; @@ -1738,8 +1752,12 @@ def test_kafka_commit_on_block_write(kafka_cluster): def test_kafka_virtual_columns2(kafka_cluster): admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) - kafka_create_topic(admin_client, "virt2_0", num_partitions=2) - kafka_create_topic(admin_client, "virt2_1", num_partitions=2) + topic_config = { + # default retention, since predefined timestamp_ms is used. + 'retention.ms': '-1', + } + kafka_create_topic(admin_client, "virt2_0", num_partitions=2, config=topic_config) + kafka_create_topic(admin_client, "virt2_1", num_partitions=2, config=topic_config) instance.query(''' CREATE TABLE test.kafka (value UInt64) @@ -1867,6 +1885,13 @@ def test_kafka_produce_key_timestamp(kafka_cluster): def test_kafka_insert_avro(kafka_cluster): + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) + topic_config = { + # default retention, since predefined timestamp_ms is used. + 'retention.ms': '-1', + } + kafka_create_topic(admin_client, "avro1", config=topic_config) + instance.query(''' DROP TABLE IF EXISTS test.kafka; CREATE TABLE test.kafka (key UInt64, value UInt64, _timestamp DateTime('UTC')) From 19b5394be855c33bf22baea58a909f56c1a2c9da Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 30 Dec 2021 15:49:48 +0800 Subject: [PATCH 0622/1260] fix tests --- programs/keeper/keeper_config.xml | 2 +- src/Server/KeeperTCPHandler.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/keeper/keeper_config.xml b/programs/keeper/keeper_config.xml index 5dc550e5245..0eb8480a0b5 100644 --- a/programs/keeper/keeper_config.xml +++ b/programs/keeper/keeper_config.xml @@ -39,7 +39,7 @@ 10000 10000 - 30000 + 100000 information diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 3d66113c92f..46c18b4dab8 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -327,7 +327,7 @@ void KeeperTCPHandler::runImpl() if (client_timeout == 0) client_timeout = Coordination::DEFAULT_SESSION_TIMEOUT_MS; session_timeout = std::max(client_timeout, min_session_timeout); - session_timeout = std::min(client_timeout, max_session_timeout); + session_timeout = std::min(session_timeout, max_session_timeout); } catch (const Exception & e) /// Typical for an incorrect username, password, or address. { From 6fbb9f5e87749251cf73a54ce540163f6403bfd2 Mon Sep 17 00:00:00 2001 From: Stig Bakken Date: Thu, 16 Dec 2021 09:26:37 +0100 Subject: [PATCH 0623/1260] Implement `EXPLAIN TABLE OVERRIDE` for pre-validating overrides. Example: ```sql EXPLAIN TABLE OVERRIDE mysql('127.0.0.1:3306', 'db', 'table', 'user', 'pw') PARTITION BY tuple(toYYYYMM(created), id % 8) ``` Validations done: * check that storage overrides do not reference unknown or nullable columns * check that default specifier is not modified for columns --- src/Common/ErrorCodes.cpp | 1 + src/Interpreters/InterpreterExplainQuery.cpp | 214 +++++++++++-------- src/Interpreters/TableOverrideUtils.cpp | 174 +++++++++++++++ src/Interpreters/TableOverrideUtils.h | 38 ++++ src/Parsers/ASTExplainQuery.h | 37 +++- src/Parsers/ASTTableOverrides.cpp | 17 +- src/Parsers/ASTTableOverrides.h | 3 +- src/Parsers/ParserCreateQuery.cpp | 23 +- src/Parsers/ParserCreateQuery.h | 4 + src/Parsers/ParserExplainQuery.cpp | 14 ++ 10 files changed, 413 insertions(+), 112 deletions(-) create mode 100644 src/Interpreters/TableOverrideUtils.cpp create mode 100644 src/Interpreters/TableOverrideUtils.h diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 16f85fcae61..ef2be3b2164 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -605,6 +605,7 @@ M(634, MONGODB_ERROR) \ M(635, CANNOT_POLL) \ M(636, CANNOT_EXTRACT_TABLE_STRUCTURE) \ + M(637, INVALID_TABLE_OVERRIDE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index e3a6812124f..fdb35637a9a 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -7,10 +7,12 @@ #include #include #include +#include #include #include #include #include +#include #include #include @@ -250,117 +252,141 @@ QueryPipeline InterpreterExplainQuery::executeImpl() WriteBufferFromOwnString buf; bool single_line = false; + bool insert_buf = true; - if (ast.getKind() == ASTExplainQuery::ParsedAST) + switch (ast.getKind()) { - if (ast.getSettings()) - throw Exception("Settings are not supported for EXPLAIN AST query.", ErrorCodes::UNKNOWN_SETTING); - - dumpAST(*ast.getExplainedQuery(), buf); - } - else if (ast.getKind() == ASTExplainQuery::AnalyzedSyntax) - { - if (ast.getSettings()) - throw Exception("Settings are not supported for EXPLAIN SYNTAX query.", ErrorCodes::UNKNOWN_SETTING); - - ExplainAnalyzedSyntaxVisitor::Data data(getContext()); - ExplainAnalyzedSyntaxVisitor(data).visit(query); - - ast.getExplainedQuery()->format(IAST::FormatSettings(buf, false)); - } - else if (ast.getKind() == ASTExplainQuery::QueryPlan) - { - if (!dynamic_cast(ast.getExplainedQuery().get())) - throw Exception("Only SELECT is supported for EXPLAIN query", ErrorCodes::INCORRECT_QUERY); - - auto settings = checkAndGetSettings(ast.getSettings()); - QueryPlan plan; - - InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); - interpreter.buildQueryPlan(plan); - - if (settings.optimize) - plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext())); - - if (settings.json) + case ASTExplainQuery::ParsedAST: { - /// Add extra layers to make plan look more like from postgres. - auto plan_map = std::make_unique(); - plan_map->add("Plan", plan.explainPlan(settings.query_plan_options)); - auto plan_array = std::make_unique(); - plan_array->add(std::move(plan_map)); + if (ast.getSettings()) + throw Exception("Settings are not supported for EXPLAIN AST query.", ErrorCodes::UNKNOWN_SETTING); - auto format_settings = getFormatSettings(getContext()); - format_settings.json.quote_64bit_integers = false; - - JSONBuilder::FormatSettings json_format_settings{.settings = format_settings}; - JSONBuilder::FormatContext format_context{.out = buf}; - - plan_array->format(json_format_settings, format_context); - - single_line = true; + dumpAST(*ast.getExplainedQuery(), buf); + break; } - else - plan.explainPlan(buf, settings.query_plan_options); - } - else if (ast.getKind() == ASTExplainQuery::QueryPipeline) - { - if (dynamic_cast(ast.getExplainedQuery().get())) + case ASTExplainQuery::AnalyzedSyntax: { - auto settings = checkAndGetSettings(ast.getSettings()); + if (ast.getSettings()) + throw Exception("Settings are not supported for EXPLAIN SYNTAX query.", ErrorCodes::UNKNOWN_SETTING); + + ExplainAnalyzedSyntaxVisitor::Data data(getContext()); + ExplainAnalyzedSyntaxVisitor(data).visit(query); + + ast.getExplainedQuery()->format(IAST::FormatSettings(buf, false)); + break; + } + case ASTExplainQuery::QueryPlan: + { + if (!dynamic_cast(ast.getExplainedQuery().get())) + throw Exception("Only SELECT is supported for EXPLAIN query", ErrorCodes::INCORRECT_QUERY); + + auto settings = checkAndGetSettings(ast.getSettings()); QueryPlan plan; InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); interpreter.buildQueryPlan(plan); - auto pipeline = plan.buildQueryPipeline( + + if (settings.optimize) + plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext())); + + if (settings.json) + { + /// Add extra layers to make plan look more like from postgres. + auto plan_map = std::make_unique(); + plan_map->add("Plan", plan.explainPlan(settings.query_plan_options)); + auto plan_array = std::make_unique(); + plan_array->add(std::move(plan_map)); + + auto format_settings = getFormatSettings(getContext()); + format_settings.json.quote_64bit_integers = false; + + JSONBuilder::FormatSettings json_format_settings{.settings = format_settings}; + JSONBuilder::FormatContext format_context{.out = buf}; + + plan_array->format(json_format_settings, format_context); + + single_line = true; + } + else + plan.explainPlan(buf, settings.query_plan_options); + break; + } + case ASTExplainQuery::QueryPipeline: + { + if (dynamic_cast(ast.getExplainedQuery().get())) + { + auto settings = checkAndGetSettings(ast.getSettings()); + QueryPlan plan; + + InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); + interpreter.buildQueryPlan(plan); + auto pipeline = plan.buildQueryPipeline( + QueryPlanOptimizationSettings::fromContext(getContext()), + BuildQueryPipelineSettings::fromContext(getContext())); + + if (settings.graph) + { + /// Pipe holds QueryPlan, should not go out-of-scope + auto pipe = QueryPipelineBuilder::getPipe(std::move(*pipeline)); + const auto & processors = pipe.getProcessors(); + + if (settings.compact) + printPipelineCompact(processors, buf, settings.query_pipeline_options.header); + else + printPipeline(processors, buf); + } + else + { + plan.explainPipeline(buf, settings.query_pipeline_options); + } + } + else if (dynamic_cast(ast.getExplainedQuery().get())) + { + InterpreterInsertQuery insert(ast.getExplainedQuery(), getContext()); + auto io = insert.execute(); + printPipeline(io.pipeline.getProcessors(), buf); + } + else + throw Exception("Only SELECT and INSERT is supported for EXPLAIN PIPELINE query", ErrorCodes::INCORRECT_QUERY); + break; + } + case ASTExplainQuery::QueryEstimates: + { + if (!dynamic_cast(ast.getExplainedQuery().get())) + throw Exception("Only SELECT is supported for EXPLAIN ESTIMATE query", ErrorCodes::INCORRECT_QUERY); + + auto settings = checkAndGetSettings(ast.getSettings()); + QueryPlan plan; + + InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); + interpreter.buildQueryPlan(plan); + // collect the selected marks, rows, parts during build query pipeline. + plan.buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(getContext()), BuildQueryPipelineSettings::fromContext(getContext())); - if (settings.graph) - { - /// Pipe holds QueryPlan, should not go out-of-scope - auto pipe = QueryPipelineBuilder::getPipe(std::move(*pipeline)); - const auto & processors = pipe.getProcessors(); - - if (settings.compact) - printPipelineCompact(processors, buf, settings.query_pipeline_options.header); - else - printPipeline(processors, buf); - } - else - { - plan.explainPipeline(buf, settings.query_pipeline_options); - } + if (settings.optimize) + plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext())); + plan.explainEstimate(res_columns); + insert_buf = false; + break; } - else if (dynamic_cast(ast.getExplainedQuery().get())) + case ASTExplainQuery::TableOverride: { - InterpreterInsertQuery insert(ast.getExplainedQuery(), getContext()); - auto io = insert.execute(); - printPipeline(io.pipeline.getProcessors(), buf); + if (auto * table_function = ast.getTableFunction()->as(); !table_function || table_function->name != "mysql") + { + throw Exception(ErrorCodes::INCORRECT_QUERY, "EXPLAIN TABLE OVERRIDE is not supported for the {}() table function", table_function->name); + } + auto storage = getContext()->getQueryContext()->executeTableFunction(ast.getTableFunction()); + auto metadata_snapshot = storage->getInMemoryMetadata(); + TableOverrideAnalyzer::Result override_info; + TableOverrideAnalyzer override_analyzer(ast.getTableOverride()); + override_analyzer.analyze(metadata_snapshot, override_info); + override_info.appendTo(buf); + break; } - else - throw Exception("Only SELECT and INSERT is supported for EXPLAIN PIPELINE query", ErrorCodes::INCORRECT_QUERY); } - else if (ast.getKind() == ASTExplainQuery::QueryEstimates) - { - if (!dynamic_cast(ast.getExplainedQuery().get())) - throw Exception("Only SELECT is supported for EXPLAIN ESTIMATE query", ErrorCodes::INCORRECT_QUERY); - - auto settings = checkAndGetSettings(ast.getSettings()); - QueryPlan plan; - - InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); - interpreter.buildQueryPlan(plan); - // collect the selected marks, rows, parts during build query pipeline. - plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(getContext()), - BuildQueryPipelineSettings::fromContext(getContext())); - - if (settings.optimize) - plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext())); - plan.explainEstimate(res_columns); - } - if (ast.getKind() != ASTExplainQuery::QueryEstimates) + if (insert_buf) { if (single_line) res_columns[0]->insertData(buf.str().data(), buf.str().size()); diff --git a/src/Interpreters/TableOverrideUtils.cpp b/src/Interpreters/TableOverrideUtils.cpp new file mode 100644 index 00000000000..922dd6af25b --- /dev/null +++ b/src/Interpreters/TableOverrideUtils.cpp @@ -0,0 +1,174 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INVALID_TABLE_OVERRIDE; +} + +namespace +{ + +class MaybeNullableColumnsMatcher +{ +public: + using Visitor = ConstInDepthNodeVisitor; + using Data = RequiredSourceColumnsData; + + static bool needChildVisit(const ASTPtr & node, const ASTPtr & child) + { + if (const auto * f = node->as(); f && f->name == "assumeNotNull") + return false; + return RequiredSourceColumnsMatcher::needChildVisit(node, child); + } + + static void visit(const ASTPtr & ast, Data & data) + { + RequiredSourceColumnsMatcher::visit(ast, data); + } +}; + +using MaybeNullableColumnsVisitor = MaybeNullableColumnsMatcher::Visitor; + +} + +static void checkRequiredColumns(const IAST * ast, const NameToTypeMap & existing_types, NamesAndTypes & used_columns, const String & what, bool allow_nulls = false) +{ + if (!ast) + return; + RequiredSourceColumnsData columns_data; + RequiredSourceColumnsVisitor(columns_data).visit(ast->clone()); + auto required_columns = columns_data.requiredColumns(); + for (const auto & column : required_columns) + { + auto type = existing_types.find(column); + if (type == existing_types.end()) + throw Exception(ErrorCodes::INVALID_TABLE_OVERRIDE, "{} override refers to unknown column {}", what, backQuote(column)); + } + if (!allow_nulls) + { + RequiredSourceColumnsData nullable_data; + MaybeNullableColumnsVisitor(nullable_data).visit(ast->clone()); + for (const auto & column : nullable_data.requiredColumns()) + { + if (existing_types.find(column)->second->isNullable()) + throw Exception( + ErrorCodes::INVALID_TABLE_OVERRIDE, + "{} override refers to nullable column {} (use assumeNotNull() if the column does not in fact contain NULL values)", + what, + backQuote(column)); + } + } + for (const auto & col : required_columns) + { + used_columns.push_back({col, existing_types.find(col)->second}); + } +} + +void TableOverrideAnalyzer::analyze(const StorageInMemoryMetadata & metadata, Result & result) const +{ + for (const auto & column : metadata.columns) + result.existing_types[column.name] = column.type; + checkRequiredColumns(override->storage->order_by, result.existing_types, result.order_by_columns, "ORDER BY"); + checkRequiredColumns(override->storage->primary_key, result.existing_types, result.primary_key_columns, "PRIMARY KEY"); + checkRequiredColumns(override->storage->partition_by, result.existing_types, result.partition_by_columns, "PARTITION BY"); + checkRequiredColumns(override->storage->sample_by, result.existing_types, result.sample_by_columns, "SAMPLE BY"); + checkRequiredColumns(override->storage->ttl_table, result.existing_types, result.ttl_columns, "TTL"); + if (override->columns && override->columns->columns) + { + for (const auto & column_ast : override->columns->columns->children) + { + auto * override_column = column_ast->as(); + auto override_type = DataTypeFactory::instance().get(override_column->type); + auto found = metadata.columns.tryGetColumnOrSubcolumn(ColumnsDescription::GetFlags::All, override_column->name); + std::optional override_default_kind; + if (!override_column->default_specifier.empty()) + override_default_kind = columnDefaultKindFromString(override_column->default_specifier); + if (found) + { + std::optional existing_default_kind; + if (auto col_default = metadata.columns.getDefault(found->name)) + existing_default_kind = col_default->kind; + if (existing_default_kind != override_default_kind) + throw Exception(ErrorCodes::INVALID_TABLE_OVERRIDE, "column {}: modifying default specifier is not allowed", backQuote(override_column->name)); + result.modified_columns.push_back({found->name, override_type}); + /// TODO: validate that the original type can be converted to the overridden type + } + else + { + if (override_default_kind && *override_default_kind == ColumnDefaultKind::Alias) + result.added_columns.push_back({override_column->name, override_type}); + else + throw Exception(ErrorCodes::INVALID_TABLE_OVERRIDE, "column {}: can only add ALIAS columns", backQuote(override_column->name)); + } + /// TODO: validate default and materialized expressions (use checkRequiredColumns, allowing nulls) + } + } +} + +void TableOverrideAnalyzer::Result::appendTo(WriteBuffer & ostr) +{ + const auto & format_names = [&](const NamesAndTypes & names) -> String + { + WriteBufferFromOwnString buf; + bool first = true; + for (const auto & name : names) + { + if (!first) + buf << ", "; + first = false; + buf << backQuote(name.name) << " "; + auto old_type = existing_types.find(name.name); + if (old_type != existing_types.end() && old_type->second != name.type) + buf << old_type->second->getName() << " -> "; + buf << name.type->getName(); + } + return buf.str(); + }; + if (!modified_columns.empty()) + { + ostr << "Modified columns: " << format_names(modified_columns) << "\n"; + } + if (!added_columns.empty()) + { + ostr << "Added columns: " << format_names(added_columns) << "\n"; + } + if (!order_by_columns.empty()) + { + ostr << "ORDER BY uses columns: " << format_names(order_by_columns) << "\n"; + } + if (!primary_key_columns.empty()) + { + ostr << "PRIMARY KEY uses columns: " << format_names(primary_key_columns) << "\n"; + } + if (!partition_by_columns.empty()) + { + ostr << "PARTITION BY uses columns: " << format_names(partition_by_columns) << "\n"; + } + if (!sample_by_columns.empty()) + { + ostr << "SAMPLE BY uses columns: " << format_names(sample_by_columns) << "\n"; + } + if (!ttl_columns.empty()) + { + ostr << "TTL uses columns: " << format_names(ttl_columns) << "\n"; + } +} + +} diff --git a/src/Interpreters/TableOverrideUtils.h b/src/Interpreters/TableOverrideUtils.h new file mode 100644 index 00000000000..810ffecd573 --- /dev/null +++ b/src/Interpreters/TableOverrideUtils.h @@ -0,0 +1,38 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +struct StorageInMemoryMetadata; + +using NameToTypeMap = std::map; + +struct TableOverrideAnalyzer +{ + struct Result + { + NameToTypeMap existing_types; + NamesAndTypes order_by_columns; + NamesAndTypes primary_key_columns; + NamesAndTypes partition_by_columns; + NamesAndTypes sample_by_columns; + NamesAndTypes ttl_columns; + NamesAndTypes added_columns; + NamesAndTypes modified_columns; + + void appendTo(WriteBuffer &); + }; + + ASTTableOverride * override; + + explicit TableOverrideAnalyzer(ASTPtr ast) : override(assert_cast(ast.get())) { } + + void analyze(const StorageInMemoryMetadata & metadata, Result & result) const; +}; + +} diff --git a/src/Parsers/ASTExplainQuery.h b/src/Parsers/ASTExplainQuery.h index 5c50a8cd82e..abed9803a7b 100644 --- a/src/Parsers/ASTExplainQuery.h +++ b/src/Parsers/ASTExplainQuery.h @@ -18,6 +18,7 @@ public: QueryPlan, /// 'EXPLAIN SELECT ...' QueryPipeline, /// 'EXPLAIN PIPELINE ...' QueryEstimates, /// 'EXPLAIN ESTIMATE ...' + TableOverride, /// 'EXPLAIN TABLE OVERRIDE ...' }; explicit ASTExplainQuery(ExplainKind kind_) : kind(kind_) {} @@ -45,8 +46,22 @@ public: ast_settings = std::move(settings_); } + void setTableFunction(ASTPtr table_function_) + { + children.emplace_back(table_function_); + table_function = std::move(table_function_); + } + + void setTableOverride(ASTPtr table_override_) + { + children.emplace_back(table_override_); + table_override = std::move(table_override_); + } + const ASTPtr & getExplainedQuery() const { return query; } const ASTPtr & getSettings() const { return ast_settings; } + const ASTPtr & getTableFunction() const { return table_function; } + const ASTPtr & getTableOverride() const { return table_override; } protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override @@ -59,8 +74,21 @@ protected: ast_settings->formatImpl(settings, state, frame); } - settings.ostr << settings.nl_or_ws; - query->formatImpl(settings, state, frame); + if (query) + { + settings.ostr << settings.nl_or_ws; + query->formatImpl(settings, state, frame); + } + if (table_function) + { + settings.ostr << settings.nl_or_ws; + table_function->formatImpl(settings, state, frame); + } + if (table_override) + { + settings.ostr << settings.nl_or_ws; + table_override->formatImpl(settings, state, frame); + } } private: @@ -69,6 +97,10 @@ private: ASTPtr query; ASTPtr ast_settings; + /// Used by EXPLAIN TABLE OVERRIDE + ASTPtr table_function; + ASTPtr table_override; + static String toString(ExplainKind kind) { switch (kind) @@ -78,6 +110,7 @@ private: case QueryPlan: return "EXPLAIN"; case QueryPipeline: return "EXPLAIN PIPELINE"; case QueryEstimates: return "EXPLAIN ESTIMATE"; + case TableOverride: return "EXPLAIN TABLE OVERRIDE"; } __builtin_unreachable(); diff --git a/src/Parsers/ASTTableOverrides.cpp b/src/Parsers/ASTTableOverrides.cpp index d2625bf19b4..8fc21db218f 100644 --- a/src/Parsers/ASTTableOverrides.cpp +++ b/src/Parsers/ASTTableOverrides.cpp @@ -31,13 +31,19 @@ void ASTTableOverride::formatImpl(const FormatSettings & settings_, FormatState String hl_keyword = settings.hilite ? hilite_keyword : ""; String hl_none = settings.hilite ? hilite_none : ""; - settings.ostr << hl_keyword << "TABLE OVERRIDE " << hl_none; - ASTIdentifier(table_name).formatImpl(settings, state, frame); + if (is_standalone) + { + settings.ostr << hl_keyword << "TABLE OVERRIDE " << hl_none; + ASTIdentifier(table_name).formatImpl(settings, state, frame); + } if (!columns && (!storage || storage->children.empty())) return; auto override_frame = frame; - ++override_frame.indent; - settings.ostr << nl_or_ws << '(' << nl_or_nothing; + if (is_standalone) + { + ++override_frame.indent; + settings.ostr << nl_or_ws << '(' << nl_or_nothing; + } String indent_str = settings.one_line ? "" : String(4 * override_frame.indent, ' '); size_t override_elems = 0; if (columns) @@ -68,7 +74,8 @@ void ASTTableOverride::formatImpl(const FormatSettings & settings_, FormatState format_storage_elem(storage->ttl_table, "TTL"); } - settings.ostr << nl_or_nothing << ')'; + if (is_standalone) + settings.ostr << nl_or_nothing << ')'; } ASTPtr ASTTableOverrideList::clone() const diff --git a/src/Parsers/ASTTableOverrides.h b/src/Parsers/ASTTableOverrides.h index 62e96b16b01..c0603f7a8e0 100644 --- a/src/Parsers/ASTTableOverrides.h +++ b/src/Parsers/ASTTableOverrides.h @@ -15,7 +15,7 @@ class ASTStorage; /// Storage and column overrides for a single table, for example: /// -/// TABLE OVERRIDE `foo` PARTITION BY toYYYYMM(`createtime`) +/// TABLE OVERRIDE `foo` (PARTITION BY toYYYYMM(`createtime`)) /// class ASTTableOverride : public IAST { @@ -23,6 +23,7 @@ public: String table_name; ASTColumns * columns = nullptr; ASTStorage * storage = nullptr; + bool is_standalone = true; String getID(char) const override { return "TableOverride " + table_name; } ASTPtr clone() const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 7f47e1efb49..6d295a0d516 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -969,14 +969,15 @@ bool ParserTableOverrideDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expecte ASTPtr sample_by; ASTPtr ttl_table; - if (!s_table_override.ignore(pos, expected)) - return false; - - if (!table_name_p.parse(pos, table_name, expected)) - return false; - - if (!lparen_p.ignore(pos, expected)) - return false; + if (is_standalone) + { + if (!s_table_override.ignore(pos, expected)) + return false; + if (!table_name_p.parse(pos, table_name, expected)) + return false; + if (!lparen_p.ignore(pos, expected)) + return false; + } while (true) { @@ -1034,7 +1035,7 @@ bool ParserTableOverrideDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expecte break; } - if (!rparen_p.ignore(pos, expected)) + if (is_standalone && !rparen_p.ignore(pos, expected)) return false; auto storage = std::make_shared(); @@ -1045,7 +1046,9 @@ bool ParserTableOverrideDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expecte storage->set(storage->ttl_table, ttl_table); auto res = std::make_shared(); - res->table_name = table_name->as()->name(); + if (table_name) + res->table_name = table_name->as()->name(); + res->is_standalone = is_standalone; res->set(res->storage, storage); if (columns) res->set(res->columns, columns); diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 33aafb40d83..615121eae58 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -389,6 +389,10 @@ protected: class ParserTableOverrideDeclaration : public IParserBase { +public: + const bool is_standalone; + ParserTableOverrideDeclaration(bool is_standalone_ = true) : is_standalone(is_standalone_) { } + protected: const char * getName() const override { return "table override declaration"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; diff --git a/src/Parsers/ParserExplainQuery.cpp b/src/Parsers/ParserExplainQuery.cpp index ffaab0f2b6d..e072f6a14d7 100644 --- a/src/Parsers/ParserExplainQuery.cpp +++ b/src/Parsers/ParserExplainQuery.cpp @@ -21,6 +21,7 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_pipeline("PIPELINE"); ParserKeyword s_plan("PLAN"); ParserKeyword s_estimates("ESTIMATE"); + ParserKeyword s_table_override("TABLE OVERRIDE"); if (s_explain.ignore(pos, expected)) { @@ -36,6 +37,8 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected kind = ASTExplainQuery::ExplainKind::QueryPlan; //-V1048 else if (s_estimates.ignore(pos, expected)) kind = ASTExplainQuery::ExplainKind::QueryEstimates; //-V1048 + else if (s_table_override.ignore(pos, expected)) + kind = ASTExplainQuery::ExplainKind::TableOverride; } else return false; @@ -65,6 +68,17 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected else return false; } + else if (kind == ASTExplainQuery::ExplainKind::TableOverride) + { + ASTPtr table_function; + if (!ParserFunction(true, true).parse(pos, table_function, expected)) + return false; + ASTPtr table_override; + if (!ParserTableOverrideDeclaration(false).parse(pos, table_override, expected)) + return false; + explain_query->setTableFunction(table_function); + explain_query->setTableOverride(table_override); + } else if (select_p.parse(pos, query, expected) || create_p.parse(pos, query, expected) || insert_p.parse(pos, query, expected)) From 6b4a16977ed53e9eaabaf01fe36fdbf72dbb971e Mon Sep 17 00:00:00 2001 From: Stig Bakken Date: Thu, 16 Dec 2021 13:23:03 +0100 Subject: [PATCH 0624/1260] Add some docs for EXPLAIN TABLE OVERRIDE --- docs/en/sql-reference/statements/explain.md | 40 ++++++++++++++++++++- 1 file changed, 39 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/explain.md b/docs/en/sql-reference/statements/explain.md index 2d129f1bc60..9c74c069f02 100644 --- a/docs/en/sql-reference/statements/explain.md +++ b/docs/en/sql-reference/statements/explain.md @@ -10,7 +10,12 @@ Shows the execution plan of a statement. Syntax: ```sql -EXPLAIN [AST | SYNTAX | PLAN | PIPELINE] [setting = value, ...] SELECT ... [FORMAT ...] +EXPLAIN [AST | SYNTAX | PLAN | PIPELINE | TABLE OVERRIDE] [setting = value, ...] + [ + SELECT ... | + tableFunction(...) [COLUMNS (...)] [ORDER BY ...] [PARTITION BY ...] [PRIMARY KEY] [SAMPLE BY ...] [TTL ...] + ] + [FORMAT ...] ``` Example: @@ -412,4 +417,37 @@ Result: └──────────┴───────┴───────┴──────┴───────┘ ``` +### EXPLAIN TABLE OVERRIDE {#explain-table-override} + +Shows the result of a table override on a table schema accessed through a table function. +Also does some validation, throwing an exception if the override would have caused some kind of failure. + +**Example** + +Assume you have a remote MySQL table like this: + +```sql +CREATE TABLE db.tbl ( + id INT PRIMARY KEY, + created DATETIME DEFAULT now() +) +``` + +```sql +EXPLAIN TABLE OVERRIDE mysql('127.0.0.1:3306', 'db', 'tbl', 'root', 'clickhouse') +PARTITION BY toYYYYMM(assumeNotNull(created)) +``` + +Result: + +```text +┌─explain─────────────────────────────────────────────────┐ +│ PARTITION BY uses columns: `created` Nullable(DateTime) │ +└─────────────────────────────────────────────────────────┘ +``` + +!!! note "Note" + The validation is not complete, so a successfull query does not guarantee that the override would + not cause issues. + [Оriginal article](https://clickhouse.com/docs/en/sql-reference/statements/explain/) From 90068405fcffbe9f2ca3fdb4c734e81605aaaddf Mon Sep 17 00:00:00 2001 From: "Chun-Sheng, Li" Date: Thu, 30 Dec 2021 16:19:05 +0800 Subject: [PATCH 0625/1260] Remove unused sample authentication credentials --- docs/en/interfaces/http.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 202ee70ae6f..a49143bf599 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -197,7 +197,7 @@ $ zcat result.gz ```bash # Receiving compressed data from the server and using the gunzip to receive decompressed data -$ curl -sS "http://default:openstack@localhost:8123/?enable_http_compression=1" \ +$ curl -sS "http://localhost:8123/?enable_http_compression=1" \ -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 3' | gunzip - 0 1 From 569ce62e8d75c31c65bc5c2996da576d4383be52 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 30 Dec 2021 17:18:51 +0800 Subject: [PATCH 0626/1260] use session_timeout as session timeout uper bound --- docs/en/operations/clickhouse-keeper.md | 2 +- programs/keeper/keeper_config.xml | 2 +- src/Common/ZooKeeper/ZooKeeperConstants.h | 2 +- src/Coordination/CoordinationSettings.cpp | 4 ++-- src/Coordination/CoordinationSettings.h | 2 +- src/Server/KeeperTCPHandler.cpp | 2 +- tests/config/config.d/keeper_port.xml | 2 +- tests/integration/helpers/keeper_config1.xml | 2 +- tests/integration/helpers/keeper_config2.xml | 2 +- tests/integration/helpers/keeper_config3.xml | 2 +- tests/integration/test_keeper_auth/configs/keeper_config.xml | 2 +- .../test_keeper_back_to_back/configs/enable_keeper.xml | 2 +- .../configs/enable_keeper.xml | 2 +- .../test_keeper_four_word_command/configs/enable_keeper1.xml | 3 +-- .../test_keeper_four_word_command/configs/enable_keeper2.xml | 3 +-- .../test_keeper_four_word_command/configs/enable_keeper3.xml | 3 +-- tests/integration/test_keeper_four_word_command/test.py | 4 ++-- .../test_keeper_incorrect_config/configs/enable_keeper1.xml | 2 +- .../configs/enable_secure_keeper1.xml | 2 +- .../configs/enable_secure_keeper2.xml | 2 +- .../configs/enable_secure_keeper3.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper3.xml | 2 +- .../test_keeper_multinode_simple/configs/enable_keeper1.xml | 2 +- .../test_keeper_multinode_simple/configs/enable_keeper2.xml | 2 +- .../test_keeper_multinode_simple/configs/enable_keeper3.xml | 2 +- .../test_keeper_nodes_add/configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper_three_nodes_1.xml | 2 +- .../configs/enable_keeper_three_nodes_2.xml | 2 +- .../configs/enable_keeper_three_nodes_3.xml | 2 +- .../configs/enable_keeper_two_nodes_1.xml | 2 +- .../configs/enable_keeper_two_nodes_2.xml | 2 +- .../test_keeper_nodes_move/configs/enable_keeper1.xml | 2 +- .../test_keeper_nodes_move/configs/enable_keeper2.xml | 2 +- .../test_keeper_nodes_move/configs/enable_keeper3.xml | 2 +- .../test_keeper_nodes_move/configs/enable_keeper_node4_1.xml | 2 +- .../test_keeper_nodes_move/configs/enable_keeper_node4_2.xml | 2 +- .../test_keeper_nodes_move/configs/enable_keeper_node4_4.xml | 2 +- .../test_keeper_nodes_remove/configs/enable_keeper1.xml | 2 +- .../test_keeper_nodes_remove/configs/enable_keeper2.xml | 2 +- .../test_keeper_nodes_remove/configs/enable_keeper3.xml | 2 +- .../configs/enable_keeper_two_nodes_1.xml | 2 +- .../configs/enable_keeper_two_nodes_2.xml | 2 +- .../configs/enable_single_keeper1.xml | 2 +- .../test_keeper_persistent_log/configs/enable_keeper.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper3.xml | 2 +- .../test_keeper_remove_leader/configs/enable_keeper1.xml | 2 +- .../test_keeper_remove_leader/configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper_two_nodes_1.xml | 2 +- .../configs/enable_keeper_two_nodes_2.xml | 2 +- .../configs/enable_keeper_two_nodes_3.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper3.xml | 2 +- .../configs/enable_secure_keeper.xml | 2 +- .../integration/test_keeper_session/configs/keeper_config.xml | 2 +- .../configs/keeper_config1.xml | 2 +- .../configs/keeper_config2.xml | 2 +- .../configs/keeper_config3.xml | 2 +- .../test_keeper_snapshots/configs/enable_keeper.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper3.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper3.xml | 2 +- .../test_keeper_three_nodes_start/configs/enable_keeper1.xml | 2 +- .../test_keeper_three_nodes_start/configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper1.xml | 2 +- .../configs/enable_keeper2.xml | 2 +- .../configs/enable_keeper3.xml | 2 +- .../test_keeper_two_nodes_cluster/configs/enable_keeper1.xml | 2 +- .../test_keeper_two_nodes_cluster/configs/enable_keeper2.xml | 2 +- .../test_keeper_zookeeper_converter/configs/keeper_config.xml | 2 +- 77 files changed, 79 insertions(+), 82 deletions(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index b07867fb240..4e50752b1f9 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -37,7 +37,7 @@ Internal coordination settings are located in `. 10000 10000 - 100000 + 100000 information diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.h b/src/Common/ZooKeeper/ZooKeeperConstants.h index 1b0a014cef1..bfbcf6da9b4 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.h +++ b/src/Common/ZooKeeper/ZooKeeperConstants.h @@ -47,7 +47,7 @@ static constexpr int32_t PASSWORD_LENGTH = 16; /// ZooKeeper has 1 MB node size and serialization limit by default, /// but it can be raised up, so we have a slightly larger limit on our side. static constexpr int32_t MAX_STRING_OR_ARRAY_SIZE = 1 << 28; /// 256 MiB -static constexpr int32_t DEFAULT_SESSION_TIMEOUT_MS = 10000; +static constexpr int32_t DEFAULT_SESSION_TIMEOUT_MS = 30000; static constexpr int32_t DEFAULT_MIN_SESSION_TIMEOUT_MS = 10000; static constexpr int32_t DEFAULT_MAX_SESSION_TIMEOUT_MS = 100000; static constexpr int32_t DEFAULT_OPERATION_TIMEOUT_MS = 10000; diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 31ccfdbc8e3..2d2ae4409a3 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -96,8 +96,8 @@ void KeeperConfigurationAndSettings::dump(WriteBufferFromOwnString & buf) const write_int(coordination_settings->max_requests_batch_size); writeText("min_session_timeout_ms=", buf); write_int(uint64_t(coordination_settings->min_session_timeout_ms)); - writeText("max_session_timeout_ms=", buf); - write_int(uint64_t(coordination_settings->max_session_timeout_ms)); + writeText("session_timeout_ms=", buf); + write_int(uint64_t(coordination_settings->session_timeout_ms)); writeText("operation_timeout_ms=", buf); write_int(uint64_t(coordination_settings->operation_timeout_ms)); writeText("dead_session_check_period_ms=", buf); diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index f653d64017e..4159c2ad994 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -20,7 +20,7 @@ struct Settings; #define LIST_OF_COORDINATION_SETTINGS(M) \ M(Milliseconds, min_session_timeout_ms, Coordination::DEFAULT_MIN_SESSION_TIMEOUT_MS, "Min client session timeout", 0) \ - M(Milliseconds, max_session_timeout_ms, Coordination::DEFAULT_MAX_SESSION_TIMEOUT_MS, "Max client session timeout", 0) \ + M(Milliseconds, session_timeout_ms, Coordination::DEFAULT_MAX_SESSION_TIMEOUT_MS, "Max client session timeout", 0) \ M(Milliseconds, operation_timeout_ms, Coordination::DEFAULT_OPERATION_TIMEOUT_MS, "Default client operation timeout", 0) \ M(Milliseconds, dead_session_check_period_ms, 500, "How often leader will check sessions to consider them dead and remove", 0) \ M(Milliseconds, heart_beat_interval_ms, 500, "Heartbeat interval between quorum nodes", 0) \ diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 46c18b4dab8..e1fe15f17df 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -219,7 +219,7 @@ KeeperTCPHandler::KeeperTCPHandler(IServer & server_, const Poco::Net::StreamSoc , max_session_timeout( 0, global_context->getConfigRef().getUInt( - "keeper_server.coordination_settings.max_session_timeout_ms", Coordination::DEFAULT_MAX_SESSION_TIMEOUT_MS) * 1000) + "keeper_server.coordination_settings.session_timeout_ms", Coordination::DEFAULT_MAX_SESSION_TIMEOUT_MS) * 1000) , poll_wrapper(std::make_unique(socket_)) , responses(std::make_unique(std::numeric_limits::max())) , last_op(std::make_unique(EMPTY_LAST_OP)) diff --git a/tests/config/config.d/keeper_port.xml b/tests/config/config.d/keeper_port.xml index c763c927a51..8cea9044dd0 100644 --- a/tests/config/config.d/keeper_port.xml +++ b/tests/config/config.d/keeper_port.xml @@ -5,8 +5,8 @@ 10000 + 100000 10000 - 100000 false 240000 diff --git a/tests/integration/helpers/keeper_config1.xml b/tests/integration/helpers/keeper_config1.xml index 27660460b40..4b7e43fbc2a 100644 --- a/tests/integration/helpers/keeper_config1.xml +++ b/tests/integration/helpers/keeper_config1.xml @@ -15,7 +15,7 @@ 10000 - 30000 + 30000 trace false diff --git a/tests/integration/helpers/keeper_config2.xml b/tests/integration/helpers/keeper_config2.xml index 9734fb9e8ec..e34149469ed 100644 --- a/tests/integration/helpers/keeper_config2.xml +++ b/tests/integration/helpers/keeper_config2.xml @@ -15,7 +15,7 @@ 10000 - 30000 + 30000 trace false diff --git a/tests/integration/helpers/keeper_config3.xml b/tests/integration/helpers/keeper_config3.xml index 2a536e764f3..8001cea44d6 100644 --- a/tests/integration/helpers/keeper_config3.xml +++ b/tests/integration/helpers/keeper_config3.xml @@ -15,7 +15,7 @@ 10000 - 30000 + 30000 trace false diff --git a/tests/integration/test_keeper_auth/configs/keeper_config.xml b/tests/integration/test_keeper_auth/configs/keeper_config.xml index eb871498bbe..926cdd3a0f9 100644 --- a/tests/integration/test_keeper_auth/configs/keeper_config.xml +++ b/tests/integration/test_keeper_auth/configs/keeper_config.xml @@ -8,7 +8,7 @@ 5000 - 10000 + 10000 trace 75 diff --git a/tests/integration/test_keeper_back_to_back/configs/enable_keeper.xml b/tests/integration/test_keeper_back_to_back/configs/enable_keeper.xml index ca29dc7d74d..9649117e4b1 100644 --- a/tests/integration/test_keeper_back_to_back/configs/enable_keeper.xml +++ b/tests/integration/test_keeper_back_to_back/configs/enable_keeper.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace false diff --git a/tests/integration/test_keeper_clickhouse_hard_restart/configs/enable_keeper.xml b/tests/integration/test_keeper_clickhouse_hard_restart/configs/enable_keeper.xml index ac78962c78f..c1d38a1de52 100644 --- a/tests/integration/test_keeper_clickhouse_hard_restart/configs/enable_keeper.xml +++ b/tests/integration/test_keeper_clickhouse_hard_restart/configs/enable_keeper.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml b/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml index 580d3bd3e03..fcb3553bb98 100644 --- a/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_four_word_command/configs/enable_keeper1.xml @@ -8,8 +8,7 @@ 5000 - 30000 - 1000 + 30000 75 trace diff --git a/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml b/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml index 35b378670b7..65011cd2637 100644 --- a/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_four_word_command/configs/enable_keeper2.xml @@ -8,8 +8,7 @@ 5000 - 30000 - 1000 + 30000 75 trace diff --git a/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml b/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml index e896dd4b454..a0d8c99f3d8 100644 --- a/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_four_word_command/configs/enable_keeper3.xml @@ -8,8 +8,7 @@ 5000 - 30000 - 1000 + 30000 75 trace diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 0824270a73b..7a464a67c81 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -285,8 +285,8 @@ def test_cmd_conf(started_cluster): assert result["log_storage_path"] == "/var/lib/clickhouse/coordination/log" assert result["snapshot_storage_path"] == "/var/lib/clickhouse/coordination/snapshots" - assert result["max_session_timeout_ms"] == "30000" - assert result["min_session_timeout_ms"] == "1000" + assert result["session_timeout_ms"] == "30000" + assert result["min_session_timeout_ms"] == "10000" assert result["operation_timeout_ms"] == "5000" assert result["dead_session_check_period_ms"] == "500" assert result["heart_beat_interval_ms"] == "500" diff --git a/tests/integration/test_keeper_incorrect_config/configs/enable_keeper1.xml b/tests/integration/test_keeper_incorrect_config/configs/enable_keeper1.xml index ac78962c78f..c1d38a1de52 100644 --- a/tests/integration/test_keeper_incorrect_config/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_incorrect_config/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper1.xml b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper1.xml index 540a5bdf8f0..986b503ebe3 100644 --- a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper1.xml +++ b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper2.xml b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper2.xml index 752a7171e45..652b1992f46 100644 --- a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper2.xml +++ b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper3.xml b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper3.xml index 365dce589b1..6507f97473b 100644 --- a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper3.xml +++ b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper1.xml b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper1.xml index 9b93bb6197d..17455ed12f5 100644 --- a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper2.xml b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper2.xml index 23faef0ba14..03a23984cc2 100644 --- a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper3.xml b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper3.xml index c7a89a4a6b2..a3196ac3061 100644 --- a/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_multinode_blocade_leader/configs/enable_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_multinode_simple/configs/enable_keeper1.xml b/tests/integration/test_keeper_multinode_simple/configs/enable_keeper1.xml index 9b93bb6197d..17455ed12f5 100644 --- a/tests/integration/test_keeper_multinode_simple/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_multinode_simple/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_multinode_simple/configs/enable_keeper2.xml b/tests/integration/test_keeper_multinode_simple/configs/enable_keeper2.xml index 23faef0ba14..03a23984cc2 100644 --- a/tests/integration/test_keeper_multinode_simple/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_multinode_simple/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_multinode_simple/configs/enable_keeper3.xml b/tests/integration/test_keeper_multinode_simple/configs/enable_keeper3.xml index c7a89a4a6b2..a3196ac3061 100644 --- a/tests/integration/test_keeper_multinode_simple/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_multinode_simple/configs/enable_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper1.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper1.xml index ac78962c78f..c1d38a1de52 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_1.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_1.xml index 83dd49b2744..d2717283a8d 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_1.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_2.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_2.xml index c9847d93f14..5924ee1c2dc 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_2.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_3.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_3.xml index 83f80395dec..d261e4f67f3 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_3.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_1.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_1.xml index 3d1147830de..697986638d7 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_1.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_2.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_2.xml index 307f356f38e..967940e1e2b 100644 --- a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_2.xml +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper1.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper1.xml index f2035082d05..1e57d42016d 100644 --- a/tests/integration/test_keeper_nodes_move/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper2.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper2.xml index eb1c0574d7a..98422b41c9b 100644 --- a/tests/integration/test_keeper_nodes_move/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper3.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper3.xml index f4e114af2e5..43800bd2dfb 100644 --- a/tests/integration/test_keeper_nodes_move/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_1.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_1.xml index 185150606c7..0d7544f9a5b 100644 --- a/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_1.xml +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_2.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_2.xml index de2fbdca8c0..65feae85e3e 100644 --- a/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_2.xml +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_4.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_4.xml index 5ef9fde8a0b..2499de4fe86 100644 --- a/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_4.xml +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_4.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper1.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper1.xml index f2035082d05..1e57d42016d 100644 --- a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper2.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper2.xml index eb1c0574d7a..98422b41c9b 100644 --- a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper3.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper3.xml index f4e114af2e5..43800bd2dfb 100644 --- a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_1.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_1.xml index 3d1147830de..697986638d7 100644 --- a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_1.xml +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_2.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_2.xml index 307f356f38e..967940e1e2b 100644 --- a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_2.xml +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_single_keeper1.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_single_keeper1.xml index ac78962c78f..c1d38a1de52 100644 --- a/tests/integration/test_keeper_nodes_remove/configs/enable_single_keeper1.xml +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_single_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_persistent_log/configs/enable_keeper.xml b/tests/integration/test_keeper_persistent_log/configs/enable_keeper.xml index 58a7d1260e3..d6166a83d2f 100644 --- a/tests/integration/test_keeper_persistent_log/configs/enable_keeper.xml +++ b/tests/integration/test_keeper_persistent_log/configs/enable_keeper.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper1.xml b/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper1.xml index ac8a1773f81..06b70c1b5d0 100644 --- a/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper2.xml b/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper2.xml index 7ad000018e0..f13f8b902b4 100644 --- a/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper3.xml b/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper3.xml index c33b31c75e3..a69cabf8c54 100644 --- a/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_persistent_log_multinode/configs/enable_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_remove_leader/configs/enable_keeper1.xml b/tests/integration/test_keeper_remove_leader/configs/enable_keeper1.xml index f2035082d05..1e57d42016d 100644 --- a/tests/integration/test_keeper_remove_leader/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_remove_leader/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_remove_leader/configs/enable_keeper2.xml b/tests/integration/test_keeper_remove_leader/configs/enable_keeper2.xml index eb1c0574d7a..98422b41c9b 100644 --- a/tests/integration/test_keeper_remove_leader/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_remove_leader/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_1.xml b/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_1.xml index ecf7f56118f..d51e420f733 100644 --- a/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_1.xml +++ b/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_2.xml b/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_2.xml index 3974d5fcb06..3f1ee1e01a8 100644 --- a/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_2.xml +++ b/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_3.xml b/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_3.xml index b460ab2df45..a99bd5d5296 100644 --- a/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_3.xml +++ b/tests/integration/test_keeper_remove_leader/configs/enable_keeper_two_nodes_3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper1.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper1.xml index 3eff33fc221..4ea543e6f31 100644 --- a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace 100 10 diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper2.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper2.xml index e6453a1ab01..4bf3083c1fa 100644 --- a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace 100 10 diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper3.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper3.xml index 837a459040f..b9e2a2d0422 100644 --- a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace 100 10 diff --git a/tests/integration/test_keeper_secure_client/configs/enable_secure_keeper.xml b/tests/integration/test_keeper_secure_client/configs/enable_secure_keeper.xml index a21851aca2b..2c144269bcc 100644 --- a/tests/integration/test_keeper_secure_client/configs/enable_secure_keeper.xml +++ b/tests/integration/test_keeper_secure_client/configs/enable_secure_keeper.xml @@ -8,7 +8,7 @@ 10000 - 30000 + 30000 trace false diff --git a/tests/integration/test_keeper_session/configs/keeper_config.xml b/tests/integration/test_keeper_session/configs/keeper_config.xml index 87ba2064dc9..ed0bb52bd51 100644 --- a/tests/integration/test_keeper_session/configs/keeper_config.xml +++ b/tests/integration/test_keeper_session/configs/keeper_config.xml @@ -8,7 +8,7 @@ 5000 - 10000 + 10000 5000 75 trace diff --git a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config1.xml b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config1.xml index d605cd92ae3..ed5909d01eb 100644 --- a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config1.xml +++ b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config1.xml @@ -9,7 +9,7 @@ 75 5 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config2.xml b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config2.xml index 17a5ef859b1..896853a3713 100644 --- a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config2.xml +++ b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config2.xml @@ -9,7 +9,7 @@ 75 5 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config3.xml b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config3.xml index 22a212cced1..8d1d5c73dd8 100644 --- a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config3.xml +++ b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config3.xml @@ -9,7 +9,7 @@ 75 5 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_snapshots/configs/enable_keeper.xml b/tests/integration/test_keeper_snapshots/configs/enable_keeper.xml index 00a90409d24..a3217b34501 100644 --- a/tests/integration/test_keeper_snapshots/configs/enable_keeper.xml +++ b/tests/integration/test_keeper_snapshots/configs/enable_keeper.xml @@ -9,7 +9,7 @@ 10 5 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper1.xml b/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper1.xml index 0cbcadcbb6e..27d0d38f596 100644 --- a/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper1.xml @@ -9,7 +9,7 @@ 10 5 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper2.xml b/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper2.xml index 87f460c8d04..dedbd312544 100644 --- a/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper2.xml @@ -9,7 +9,7 @@ 10 5 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper3.xml b/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper3.xml index 596be35a119..fd36458b5eb 100644 --- a/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_snapshots_multinode/configs/enable_keeper3.xml @@ -9,7 +9,7 @@ 10 5 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper1.xml b/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper1.xml index f2035082d05..1e57d42016d 100644 --- a/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper2.xml b/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper2.xml index eb1c0574d7a..98422b41c9b 100644 --- a/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper3.xml b/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper3.xml index f4e114af2e5..43800bd2dfb 100644 --- a/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_start_as_follower_multinode/configs/enable_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper1.xml b/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper1.xml index e6637613380..d2159348588 100644 --- a/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper2.xml b/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper2.xml index 041148b7ef4..0d9aeb191d9 100644 --- a/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper1.xml b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper1.xml index 83dd49b2744..d2717283a8d 100644 --- a/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper2.xml b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper2.xml index c9847d93f14..5924ee1c2dc 100644 --- a/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper3.xml b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper3.xml index 83f80395dec..d261e4f67f3 100644 --- a/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper3.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 trace diff --git a/tests/integration/test_keeper_two_nodes_cluster/configs/enable_keeper1.xml b/tests/integration/test_keeper_two_nodes_cluster/configs/enable_keeper1.xml index 757ba52fa4f..7c2e283e89f 100644 --- a/tests/integration/test_keeper_two_nodes_cluster/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_two_nodes_cluster/configs/enable_keeper1.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_two_nodes_cluster/configs/enable_keeper2.xml b/tests/integration/test_keeper_two_nodes_cluster/configs/enable_keeper2.xml index 927354cb6af..618e6a04aec 100644 --- a/tests/integration/test_keeper_two_nodes_cluster/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_two_nodes_cluster/configs/enable_keeper2.xml @@ -7,7 +7,7 @@ 5000 - 10000 + 10000 75 trace diff --git a/tests/integration/test_keeper_zookeeper_converter/configs/keeper_config.xml b/tests/integration/test_keeper_zookeeper_converter/configs/keeper_config.xml index 9f6927a6c11..9b50f2c6c41 100644 --- a/tests/integration/test_keeper_zookeeper_converter/configs/keeper_config.xml +++ b/tests/integration/test_keeper_zookeeper_converter/configs/keeper_config.xml @@ -7,7 +7,7 @@ 30000 - 600000 + 600000 trace 75 From ae7e5691b83ac05444e722dbfda97965673e6081 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 30 Dec 2021 12:55:50 +0300 Subject: [PATCH 0627/1260] Get rid of IAST::getQueryKindString --- src/Interpreters/executeQuery.cpp | 4 +++- src/Parsers/ASTAlterQuery.h | 1 - src/Parsers/ASTCreateQuery.h | 1 - src/Parsers/ASTDropQuery.h | 1 - src/Parsers/ASTInsertQuery.h | 1 - src/Parsers/ASTRenameQuery.h | 1 - src/Parsers/ASTSelectIntersectExceptQuery.h | 1 - src/Parsers/ASTSelectQuery.h | 1 - src/Parsers/ASTSelectWithUnionQuery.h | 1 - src/Parsers/ASTSystemQuery.h | 1 - src/Parsers/Access/ASTGrantQuery.h | 1 - src/Parsers/IAST.h | 3 --- 12 files changed, 3 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 7dcfc4b95b3..9770d1a988f 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -57,6 +57,8 @@ #include #include +#include + #include @@ -271,7 +273,7 @@ static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr // Try log query_kind if ast is valid if (ast) { - elem.query_kind = ast->getQueryKindString(); + elem.query_kind = magic_enum::enum_name(ast->getQueryKind()); if (settings.log_formatted_queries) elem.formatted_query = queryToString(ast); } diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index f757f00c100..5c7870af319 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -246,7 +246,6 @@ public: return removeOnCluster(clone(), new_database); } - const char * getQueryKindString() const override { return "Alter"; } virtual QueryKind getQueryKind() const override { return QueryKind::Alter; } protected: diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index 461e5b61f2f..66a3f949741 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -113,7 +113,6 @@ public: bool isView() const { return is_ordinary_view || is_materialized_view || is_live_view || is_window_view; } - const char * getQueryKindString() const override { return "Create"; } virtual QueryKind getQueryKind() const override { return QueryKind::Create; } protected: diff --git a/src/Parsers/ASTDropQuery.h b/src/Parsers/ASTDropQuery.h index 24f7c706062..2e67eaf3692 100644 --- a/src/Parsers/ASTDropQuery.h +++ b/src/Parsers/ASTDropQuery.h @@ -45,7 +45,6 @@ public: return removeOnCluster(clone(), new_database); } - const char * getQueryKindString() const override { return "Drop"; } virtual QueryKind getQueryKind() const override { return QueryKind::Drop; } protected: diff --git a/src/Parsers/ASTInsertQuery.h b/src/Parsers/ASTInsertQuery.h index f52a3603f2a..db9262ea794 100644 --- a/src/Parsers/ASTInsertQuery.h +++ b/src/Parsers/ASTInsertQuery.h @@ -66,7 +66,6 @@ public: return res; } - const char * getQueryKindString() const override { return "Insert"; } virtual QueryKind getQueryKind() const override { return QueryKind::Insert; } protected: diff --git a/src/Parsers/ASTRenameQuery.h b/src/Parsers/ASTRenameQuery.h index a2ee6f7e622..01ab0df9774 100644 --- a/src/Parsers/ASTRenameQuery.h +++ b/src/Parsers/ASTRenameQuery.h @@ -65,7 +65,6 @@ public: return query_ptr; } - const char * getQueryKindString() const override { return "Rename"; } virtual QueryKind getQueryKind() const override { return QueryKind::Rename; } protected: diff --git a/src/Parsers/ASTSelectIntersectExceptQuery.h b/src/Parsers/ASTSelectIntersectExceptQuery.h index 2e0bcd2c885..fa574b46c8d 100644 --- a/src/Parsers/ASTSelectIntersectExceptQuery.h +++ b/src/Parsers/ASTSelectIntersectExceptQuery.h @@ -22,7 +22,6 @@ public: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - const char * getQueryKindString() const override { return "SelectIntersectExcept"; } virtual QueryKind getQueryKind() const override { return QueryKind::SelectIntersectExcept; } ASTs getListOfSelects() const; diff --git a/src/Parsers/ASTSelectQuery.h b/src/Parsers/ASTSelectQuery.h index 70f2db28503..1c631783fdb 100644 --- a/src/Parsers/ASTSelectQuery.h +++ b/src/Parsers/ASTSelectQuery.h @@ -135,7 +135,6 @@ public: void setFinal(); - const char * getQueryKindString() const override { return "Select"; } virtual QueryKind getQueryKind() const override { return QueryKind::Select; } protected: diff --git a/src/Parsers/ASTSelectWithUnionQuery.h b/src/Parsers/ASTSelectWithUnionQuery.h index 7df03e6aab9..bd45dd7fc05 100644 --- a/src/Parsers/ASTSelectWithUnionQuery.h +++ b/src/Parsers/ASTSelectWithUnionQuery.h @@ -17,7 +17,6 @@ public: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - const char * getQueryKindString() const override { return "Select"; } virtual QueryKind getQueryKind() const override { return QueryKind::Select; } SelectUnionMode union_mode; diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index a4c86866e7b..22488e35e12 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -107,7 +107,6 @@ public: return removeOnCluster(clone(), new_database); } - const char * getQueryKindString() const override { return "System"; } virtual QueryKind getQueryKind() const override { return QueryKind::System; } protected: diff --git a/src/Parsers/Access/ASTGrantQuery.h b/src/Parsers/Access/ASTGrantQuery.h index 44d6de6a662..934d619fc36 100644 --- a/src/Parsers/Access/ASTGrantQuery.h +++ b/src/Parsers/Access/ASTGrantQuery.h @@ -34,7 +34,6 @@ public: void replaceEmptyDatabase(const String & current_database); void replaceCurrentUserTag(const String & current_user_name) const; ASTPtr getRewrittenASTWithoutOnCluster(const std::string &) const override { return removeOnCluster(clone()); } - const char * getQueryKindString() const override { return is_revoke ? "Revoke" : "Grant"; } virtual QueryKind getQueryKind() const override { return is_revoke ? QueryKind::Revoke : QueryKind::Grant; } }; } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 5806f652593..ed3c54624ba 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -245,9 +245,6 @@ public: void cloneChildren(); - // Return query_kind string representation of this AST query. - virtual const char * getQueryKindString() const { return ""; } - enum QueryKind { None, From 91e1ac437e44e11dbed74e93c0c7ae57778000e4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 30 Dec 2021 12:57:38 +0300 Subject: [PATCH 0628/1260] Tiny improvements --- src/Storages/StorageReplicatedMergeTree.cpp | 56 +++++++++++---------- src/Storages/StorageReplicatedMergeTree.h | 9 ++-- 2 files changed, 34 insertions(+), 31 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b8d7ca09ae0..fb8a307a866 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7042,35 +7042,35 @@ String StorageReplicatedMergeTree::getTableSharedID() const void StorageReplicatedMergeTree::createTableSharedID() { - if (table_shared_id == UUIDHelpers::Nil) + if (table_shared_id != UUIDHelpers::Nil) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table shared id already initialized"); + + zkutil::ZooKeeperPtr zookeeper = getZooKeeper(); + String zookeeper_table_id_path = fs::path(zookeeper_path) / "table_shared_id"; + String id; + if (!zookeeper->tryGet(zookeeper_table_id_path, id)) { - zkutil::ZooKeeperPtr zookeeper = getZooKeeper(); - String zookeeper_table_id_path = fs::path(zookeeper_path) / "table_shared_id"; - String id; - if (!zookeeper->tryGet(zookeeper_table_id_path, id)) - { - UUID table_id_candidate; - auto storage_id = getStorageID(); - if (storage_id.uuid != UUIDHelpers::Nil) - table_id_candidate = storage_id.uuid; - else - table_id_candidate = UUIDHelpers::generateV4(); + UUID table_id_candidate; + auto storage_id = getStorageID(); + if (storage_id.uuid != UUIDHelpers::Nil) + table_id_candidate = storage_id.uuid; + else + table_id_candidate = UUIDHelpers::generateV4(); - id = toString(table_id_candidate); + id = toString(table_id_candidate); - auto code = zookeeper->tryCreate(zookeeper_table_id_path, id, zkutil::CreateMode::Persistent); - if (code == Coordination::Error::ZNODEEXISTS) - { /// Other replica create node early - id = zookeeper->get(zookeeper_table_id_path); - } - else if (code != Coordination::Error::ZOK) - { - throw zkutil::KeeperException(code, zookeeper_table_id_path); - } + auto code = zookeeper->tryCreate(zookeeper_table_id_path, id, zkutil::CreateMode::Persistent); + if (code == Coordination::Error::ZNODEEXISTS) + { /// Other replica create node early + id = zookeeper->get(zookeeper_table_id_path); + } + else if (code != Coordination::Error::ZOK) + { + throw zkutil::KeeperException(code, zookeeper_table_id_path); } - - table_shared_id = parseFromString(id); } + + table_shared_id = parseFromString(id); } @@ -7123,12 +7123,12 @@ bool StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & par if (ref_count > 0) /// Keep part shard info for frozen backups return false; - return unlockSharedDataById(part.getUniqueId(), getTableSharedID(), name, replica_name, disk, zookeeper, *getSettings(), log, + return unlockSharedDataByID(part.getUniqueId(), getTableSharedID(), name, replica_name, disk, zookeeper, *getSettings(), log, zookeeper_path); } -bool StorageReplicatedMergeTree::unlockSharedDataById(String id, const String & table_uuid, const String & part_name, +bool StorageReplicatedMergeTree::unlockSharedDataByID(String id, const String & table_uuid, const String & part_name, const String & replica_name_, DiskPtr disk, zkutil::ZooKeeperPtr zookeeper_ptr, const MergeTreeSettings & settings, Poco::Logger * logger, const String & zookeeper_path_old) { @@ -7552,6 +7552,8 @@ void StorageReplicatedMergeTree::createZeroCopyLockNode(const zkutil::ZooKeeperP namespace { +/// Special metadata used during freeze table. Required for zero-copy +/// replication. struct FreezeMetaData { public: @@ -7673,7 +7675,7 @@ bool StorageReplicatedMergeTree::removeSharedDetachedPart(DiskPtr disk, const St if (ref_count == 0) { String id = disk->getUniqueId(checksums); - keep_shared = !StorageReplicatedMergeTree::unlockSharedDataById(id, table_uuid, part_name, + keep_shared = !StorageReplicatedMergeTree::unlockSharedDataByID(id, table_uuid, part_name, detached_replica_name, disk, zookeeper, getContext()->getReplicatedMergeTreeSettings(), log, detached_zookeeper_path); } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 5d8b92ec217..8527439f2c9 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -243,7 +243,7 @@ public: /// Unlock shared data part in zookeeper by part id /// Return true if data unlocked /// Return false if data is still used by another node - static bool unlockSharedDataById(String id, const String & table_uuid, const String & part_name, const String & replica_name_, + static bool unlockSharedDataByID(String id, const String & table_uuid, const String & part_name, const String & replica_name_, DiskPtr disk, zkutil::ZooKeeperPtr zookeeper_, const MergeTreeSettings & settings, Poco::Logger * logger, const String & zookeeper_path_old); @@ -410,6 +410,9 @@ private: ThrottlerPtr replicated_fetches_throttler; ThrottlerPtr replicated_sends_throttler; + /// Global ID, synced via ZooKeeper between replicas + UUID table_shared_id; + template void foreachCommittedParts(Func && func, bool select_sequential_consistency) const; @@ -749,6 +752,7 @@ private: bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid, const String & zookeeper_name, const String & replica_name, const String & zookeeper_path); + /// Create freeze metadata for table and save in zookeeper. Required only if zero-copy replication enabled. void createAndStoreFreezeMetadata(DiskPtr disk, DataPartPtr part, String backup_part_path) const override; // Create table id if needed @@ -770,9 +774,6 @@ protected: std::unique_ptr settings_, bool has_force_restore_data_flag, bool allow_renaming_); - - /// Global ID, synced via ZooKeeper between replicas - UUID table_shared_id; }; String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info); From 77084f53496d4a4eb8ff7b0a31ebe2935158cd29 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 30 Dec 2021 18:43:12 +0800 Subject: [PATCH 0629/1260] add token holder --- src/Common/LRUResourceCache.h | 114 +++++++++++++++++++++++++++------- 1 file changed, 90 insertions(+), 24 deletions(-) diff --git a/src/Common/LRUResourceCache.h b/src/Common/LRUResourceCache.h index 8117e37b844..76280497310 100644 --- a/src/Common/LRUResourceCache.h +++ b/src/Common/LRUResourceCache.h @@ -60,10 +60,10 @@ public: template MappedHolderPtr getOrSet(const Key & key, LoadFunc && load_func) { - auto mappedptr = getImpl(key, load_func); - if (!mappedptr) + auto mapped_ptr = getImpl(key, load_func); + if (!mapped_ptr) return nullptr; - return std::make_unique(this, key, mappedptr); + return std::make_unique(this, key, mapped_ptr); } // If the key's reference_count = 0, delete it immediately. otherwise, mark it expired, and delete in release @@ -128,14 +128,70 @@ private: size_t max_weight = 0; size_t max_element_size = 0; + /// Represents pending insertion attempt. struct InsertToken { + explicit InsertToken(LRUResourceCache & cache_) : cache(cache_) { } + std::mutex mutex; - MappedPtr value; - size_t reference_count = 0; + bool cleaned_up = false; /// Protected by the token mutex + MappedPtr value; /// Protected by the token mutex + + LRUResourceCache & cache; + size_t refcount = 0; /// Protected by the cache mutex }; - using InsertTokens = std::unordered_map; - InsertTokens insert_tokens; + + using InsertTokenById = std::unordered_map, HashFunction>; + + /// This class is responsible for removing used insert tokens from the insert_tokens map. + /// Among several concurrent threads the first successful one is responsible for removal. But if they all + /// fail, then the last one is responsible. + struct InsertTokenHolder + { + const Key * key = nullptr; + std::shared_ptr token; + bool cleaned_up = false; + + InsertTokenHolder() = default; + + void + acquire(const Key * key_, const std::shared_ptr & token_, [[maybe_unused]] std::lock_guard & cache_lock) + { + key = key_; + token = token_; + ++token->refcount; + } + + void cleanup([[maybe_unused]] std::lock_guard & token_lock, [[maybe_unused]] std::lock_guard & cache_lock) + { + token->cache.insert_tokens.erase(*key); + token->cleaned_up = true; + cleaned_up = true; + } + + ~InsertTokenHolder() + { + if (!token) + return; + + if (cleaned_up) + return; + + std::lock_guard token_lock(token->mutex); + + if (token->cleaned_up) + return; + + std::lock_guard cache_lock(token->cache.mutex); + + --token->refcount; + if (token->refcount == 0) + cleanup(token_lock, cache_lock); + } + }; + + friend struct InsertTokenHolder; + InsertTokenById insert_tokens; WeightFunction weight_function; std::atomic hits{0}; std::atomic misses{0}; @@ -146,7 +202,7 @@ private: template MappedPtr getImpl(const Key & key, LoadFunc && load_func) { - InsertToken * insert_token = nullptr; + InsertTokenHolder token_holder; { std::lock_guard lock(mutex); auto it = cells.find(key); @@ -169,31 +225,41 @@ private: } } misses++; - insert_token = acquireInsertToken(key); + auto & token = insert_tokens[key]; + if (!token) + token = std::make_shared(*this); + token_holder.acquire(&key, token, lock); } + + auto * token = token_holder.token.get(); + std::lock_guard token_lock(token->mutex); + token_holder.cleaned_up = token->cleaned_up; + + if (!token->value) + token->value = load_func(); + + std::lock_guard lock(mutex); + auto token_it = insert_tokens.find(key); Cell * cell_ptr = nullptr; + if (token_it != insert_tokens.end() && token_it->second.get() == token) { - std::lock_guard lock(insert_token->mutex); - if (!insert_token->value) + cell_ptr = set(key, token->value); + } + else + { + auto cell_it = cells.find(key); + if (cell_it != cells.end() && !cell_it->second.expired) { - insert_token->value = load_func(); - std::lock_guard cell_lock(mutex); - cell_ptr = set(key, insert_token->value); - if (cell_ptr) - { - cell_ptr->reference_count += 1; - } - else - { - insert_token->value = nullptr; - } + cell_ptr = &cell_it->second; } } - std::lock_guard lock(mutex); - releaseInsertToken(key); + if (!token->cleaned_up) + token_holder.cleanup(token_lock, lock); + if (cell_ptr) { + cell_ptr->reference_count++; return cell_ptr->value; } return nullptr; From bfc705c09835b6120465b183a3b5c23379377666 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 29 Dec 2021 13:02:18 +0300 Subject: [PATCH 0630/1260] Better --- .../ClickHouseDictionarySource.cpp | 10 ++++++-- src/Dictionaries/MongoDBDictionarySource.cpp | 6 ++++- src/Dictionaries/MySQLDictionarySource.cpp | 17 ++++++++++++- .../PostgreSQLDictionarySource.cpp | 8 +++++-- .../ExternalDataSourceConfiguration.cpp | 24 ++++++++++--------- .../ExternalDataSourceConfiguration.h | 7 ++++-- 6 files changed, 53 insertions(+), 19 deletions(-) diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 6abd5f317e2..a99bee60cfd 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -28,6 +28,10 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +static const std::unordered_set dictionary_allowed_keys = { + "host", "port", "user", "password", "db", "database", "table", + "update_field", "update_tag", "invalidate_query", "query", "where", "name", "secure"}; + namespace { constexpr size_t MAX_CONNECTIONS = 16; @@ -235,9 +239,11 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) std::string db = config.getString(settings_config_prefix + ".db", default_database); std::string table = config.getString(settings_config_prefix + ".table", ""); UInt16 port = static_cast(config.getUInt(settings_config_prefix + ".port", default_port)); + auto has_config_key = [](const String & key) { return dictionary_allowed_keys.contains(key); }; - auto named_collection = created_from_ddl ? - getExternalDataSourceConfiguration(config, settings_config_prefix, global_context) : std::nullopt; + auto named_collection = created_from_ddl + ? getExternalDataSourceConfiguration(config, settings_config_prefix, global_context, has_config_key) + : std::nullopt; if (named_collection) { diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index 1e8be726941..7c720691c3c 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -8,6 +8,9 @@ namespace DB { +static const std::unordered_set dictionary_allowed_keys = { + "host", "port", "user", "password", "db", "database", "uri", "collection", "name", "method"}; + void registerDictionarySourceMongoDB(DictionarySourceFactory & factory) { auto create_mongo_db_dictionary = []( @@ -21,7 +24,8 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory) { const auto config_prefix = root_config_prefix + ".mongodb"; ExternalDataSourceConfiguration configuration; - auto named_collection = getExternalDataSourceConfiguration(config, config_prefix, context); + auto has_config_key = [](const String & key) { return dictionary_allowed_keys.contains(key); }; + auto named_collection = getExternalDataSourceConfiguration(config, config_prefix, context, has_config_key); if (named_collection) { configuration = *named_collection; diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 0bf5cc3cae0..5bfb6273e8d 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -30,6 +30,18 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; } +static const std::unordered_set dictionary_allowed_keys = { + "host", "port", "user", "password", + "db", "database", "table", "schema", + "update_field", "invalidate_query", "priority", + "update_tag", "dont_check_update_time", + "query", "where", "name" /* name_collection */, "socket", + "share_connection", "fail_on_connection_loss", "close_connection", + "ssl_ca", "ssl_cert", "ssl_key", + "enable_local_infile", "opt_reconnect", + "connect_timeout", "mysql_connect_timeout", + "mysql_rw_timeout", "rw_timeout"}; + void registerDictionarySourceMysql(DictionarySourceFactory & factory) { auto create_table_source = [=]([[maybe_unused]] const DictionaryStructure & dict_struct, @@ -48,8 +60,11 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) auto settings_config_prefix = config_prefix + ".mysql"; std::shared_ptr pool; + auto has_config_key = [](const String & key) { return dictionary_allowed_keys.contains(key) || key.starts_with("replica"); }; StorageMySQLConfiguration configuration; - auto named_collection = created_from_ddl ? getExternalDataSourceConfiguration(config, settings_config_prefix, global_context) : std::nullopt; + auto named_collection = created_from_ddl + ? getExternalDataSourceConfiguration(config, settings_config_prefix, global_context, has_config_key) + : std::nullopt; if (named_collection) { configuration.set(*named_collection); diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 0ac84b35048..9af3ea06838 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -28,6 +28,10 @@ namespace ErrorCodes static const UInt64 max_block_size = 8192; +static const std::unordered_set dictionary_allowed_keys = { + "host", "port", "user", "password", "db", "database", "table", "schema", + "update_field", "update_tag", "invalidate_query", "query", "where", "name", "priority"}; + namespace { ExternalQueryBuilder makeExternalQueryBuilder(const DictionaryStructure & dict_struct, const String & schema, const String & table, const String & query, const String & where) @@ -185,8 +189,8 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) { #if USE_LIBPQXX const auto settings_config_prefix = config_prefix + ".postgresql"; - - auto configuration = getExternalDataSourceConfigurationByPriority(config, settings_config_prefix, context); + auto has_config_key = [](const String & key) { return dictionary_allowed_keys.contains(key) || key.starts_with("replica"); }; + auto configuration = getExternalDataSourceConfigurationByPriority(config, settings_config_prefix, context, has_config_key); auto pool = std::make_shared( configuration.replicas_configurations, context->getSettingsRef().postgresql_connection_pool_size, diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 5a93b688da6..265587d2b1a 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -16,6 +16,8 @@ #include #endif +#include + namespace DB { @@ -166,22 +168,22 @@ std::optional getExternalDataSourceConfiguration(const } static void validateConfigKeys( - const Poco::Util::AbstractConfiguration & dict_config, const String & config_prefix, const std::unordered_set & allowed_keys) + const Poco::Util::AbstractConfiguration & dict_config, const String & config_prefix, HasConfigKeyFunc has_config_key_func) { Poco::Util::AbstractConfiguration::Keys config_keys; dict_config.keys(config_prefix, config_keys); for (const auto & config_key : config_keys) { - if (allowed_keys.contains(config_key) || config_key.starts_with("replica")) - continue; - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected key `{}` in dictionary source configuration", config_key); + if (!has_config_key_func(config_key)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected key `{}` in dictionary source configuration", config_key); } } std::optional getExternalDataSourceConfiguration( - const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context) + const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, + ContextPtr context, HasConfigKeyFunc has_config_key) { - validateConfigKeys(dict_config, dict_config_prefix, dictionary_allowed_keys); + validateConfigKeys(dict_config, dict_config_prefix, has_config_key); ExternalDataSourceConfiguration configuration; auto collection_name = dict_config.getString(dict_config_prefix + ".name", ""); @@ -189,7 +191,7 @@ std::optional getExternalDataSourceConfiguratio { const auto & config = context->getConfigRef(); const auto & collection_prefix = fmt::format("named_collections.{}", collection_name); - validateConfigKeys(dict_config, collection_prefix, dictionary_allowed_keys); + validateConfigKeys(dict_config, collection_prefix, has_config_key); if (!config.has(collection_prefix)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", collection_name); @@ -215,12 +217,12 @@ std::optional getExternalDataSourceConfiguratio ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority( - const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context) + const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context, HasConfigKeyFunc has_config_key) { - validateConfigKeys(dict_config, dict_config_prefix, dictionary_allowed_keys); + validateConfigKeys(dict_config, dict_config_prefix, has_config_key); ExternalDataSourceConfiguration common_configuration; - auto named_collection = getExternalDataSourceConfiguration(dict_config, dict_config_prefix, context); + auto named_collection = getExternalDataSourceConfiguration(dict_config, dict_config_prefix, context, has_config_key); if (named_collection) { common_configuration = *named_collection; @@ -255,7 +257,7 @@ ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority( { ExternalDataSourceConfiguration replica_configuration(common_configuration); String replica_name = dict_config_prefix + "." + config_key; - validateConfigKeys(dict_config, replica_name, {"host", "port", "user", "password", "priority"}); + validateConfigKeys(dict_config, replica_name, has_config_key); size_t priority = dict_config.getInt(replica_name + ".priority", 0); replica_configuration.host = dict_config.getString(replica_name + ".host", common_configuration.host); diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index 502f8b800e3..d73caf8b548 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -64,8 +64,11 @@ struct ExternalDataSourceConfig */ std::optional getExternalDataSourceConfiguration(const ASTs & args, ContextPtr context, bool is_database_engine = false, bool throw_on_no_collection = true); +using HasConfigKeyFunc = std::function; + std::optional getExternalDataSourceConfiguration( - const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context); + const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, + ContextPtr context, HasConfigKeyFunc has_config_key); /// Highest priority is 0, the bigger the number in map, the less the priority. @@ -80,7 +83,7 @@ struct ExternalDataSourcesByPriority }; ExternalDataSourcesByPriority -getExternalDataSourceConfigurationByPriority(const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context); +getExternalDataSourceConfigurationByPriority(const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context, HasConfigKeyFunc has_config_key); struct URLBasedDataSourceConfiguration From 2686a189dee12de061795a778ec02c03d021486b Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 30 Dec 2021 13:54:39 +0300 Subject: [PATCH 0631/1260] Apply suggestion for code review, changes in StorageSystemTables getFilteredTables Co-authored-by: Azat Khuzhin --- src/Storages/System/StorageSystemTables.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 3ec1849b73a..24e3fe4f7a9 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -88,18 +88,18 @@ static ColumnPtr getFilteredDatabases(const SelectQueryInfo & query_info, Contex return block.getByPosition(0).column; } -static ColumnPtr getFilteredTables(const ASTPtr & query, ColumnPtr & filtered_databases_column, ContextPtr context) +static ColumnPtr getFilteredTables(const ASTPtr & query, const ColumnPtr & filtered_databases_column, ContextPtr context) { MutableColumnPtr column = ColumnString::create(); - size_t database_idx = 0; - for (; database_idx < filtered_databases_column->size(); ++database_idx) + for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx) { - auto database_name = filtered_databases_column->getDataAt(database_idx).toString(); - DatabasePtr database = DatabaseCatalog::instance().getDatabase(database_name); - DatabaseTablesIteratorPtr table_it = database->getTablesIterator(context); + const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString(); + DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name); + if (!database) + continue; - for (; table_it->isValid(); table_it->next()) + for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next()) column->insert(table_it->name()); } From e6528ca0eff25af7a2af102deaf6c6ac53bbecb0 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 30 Dec 2021 20:38:58 +0800 Subject: [PATCH 0632/1260] fix a bug in LRUCache::remove() --- src/Common/LRUCache.h | 1 + src/Common/tests/gtest_lru_resource_cache.cpp | 32 +++++++++---------- 2 files changed, 17 insertions(+), 16 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 1058003a327..480a03ab399 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -72,6 +72,7 @@ public: return; auto & cell = it->second; current_size -= cell.size; + queue.erase(cell.queue_iterator); cells.erase(it); } diff --git a/src/Common/tests/gtest_lru_resource_cache.cpp b/src/Common/tests/gtest_lru_resource_cache.cpp index fcdd2902522..f88eded531e 100644 --- a/src/Common/tests/gtest_lru_resource_cache.cpp +++ b/src/Common/tests/gtest_lru_resource_cache.cpp @@ -35,8 +35,8 @@ TEST(LRUResourceCache, remove) auto n = mcache.size(); ASSERT_TRUE(n == 1); - holder0 = nullptr; - holder1 = nullptr; + holder0.reset(); + holder1.reset(); n = mcache.size(); ASSERT_TRUE(n == 0); } @@ -53,10 +53,10 @@ TEST(LRUResourceCache, evictOnWweight) int x = 2; auto load_int = [&] { return std::make_shared(x); }; auto holder1 = mcache.getOrSet(1, load_int); - holder1 = nullptr; + holder1.reset(); auto holder2 = mcache.getOrSet(2, load_int); - holder2 = nullptr; + holder2.reset(); x = 3; auto holder3 = mcache.getOrSet(3, load_int); @@ -82,13 +82,13 @@ TEST(LRUResourceCache, evictOnWeightV2) int x = 2; auto load_int = [&] { return std::make_shared(x); }; auto holder1 = mcache.getOrSet(1, load_int); - holder1 = nullptr; + holder1.reset(); auto holder2 = mcache.getOrSet(2, load_int); - holder2 = nullptr; + holder2.reset(); holder1 = mcache.get(1); - holder1 = nullptr; + holder1.reset(); x = 3; auto holder3 = mcache.getOrSet(3, load_int); @@ -114,13 +114,13 @@ TEST(LRUResourceCache, evictOnWeightV3) int x = 2; auto load_int = [&] { return std::make_shared(x); }; auto holder1 = mcache.getOrSet(1, load_int); - holder1 = nullptr; + holder1.reset(); auto holder2 = mcache.getOrSet(2, load_int); - holder2 = nullptr; + holder2.reset(); holder1 = mcache.getOrSet(1, load_int); - holder1 = nullptr; + holder1.reset(); x = 3; auto holder3 = mcache.getOrSet(3, load_int); @@ -146,10 +146,10 @@ TEST(LRUResourceCache, evictOnSize) int x = 2; auto load_int = [&] { return std::make_shared(x); }; auto holder1 = mcache.getOrSet(1, load_int); - holder1 = nullptr; + holder1.reset(); auto holder2 = mcache.getOrSet(2, load_int); - holder2 = nullptr; + holder2.reset(); x = 3; auto holder3 = mcache.getOrSet(3, load_int); @@ -177,10 +177,10 @@ TEST(LRUResourceCache, notEvictUsedElement) auto holder1 = mcache.getOrSet(1, load_int); auto holder2 = mcache.getOrSet(2, load_int); - holder2 = nullptr; + holder2.reset(); auto holder3 = mcache.getOrSet(3, load_int); - holder3 = nullptr; + holder3.reset(); x = 3; auto holder4 = mcache.getOrSet(4, load_int); @@ -231,7 +231,7 @@ TEST(LRUResourceCache, dupGet) int x = 2; auto load_int = [&] { return std::make_shared(x); }; auto holder1 = mcache.getOrSet(1, load_int); - holder1 = nullptr; + holder1.reset(); x = 11; holder1 = mcache.getOrSet(1, load_int); ASSERT_TRUE(holder1 != nullptr); @@ -255,7 +255,7 @@ TEST(LRUResourceCache, reGet) mcache.tryRemove(1); x = 11; - holder1 = nullptr; + holder1.reset(); holder1 = mcache.getOrSet(1, load_int); ASSERT_TRUE(holder1 != nullptr); From 9c32723dcf4c141883017e148a6f6f4b015aebf4 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 30 Dec 2021 21:48:54 +0800 Subject: [PATCH 0633/1260] fix a bug in LRUResourceCache::getImpl() --- src/Common/LRUResourceCache.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/LRUResourceCache.h b/src/Common/LRUResourceCache.h index 76280497310..b872d649a57 100644 --- a/src/Common/LRUResourceCache.h +++ b/src/Common/LRUResourceCache.h @@ -259,6 +259,7 @@ private: if (cell_ptr) { + queue.splice(queue.end(), queue, cell_ptr->queue_iterator); cell_ptr->reference_count++; return cell_ptr->value; } From c28fcd5a2fec7e54273eab02194dd58fe82a499e Mon Sep 17 00:00:00 2001 From: cmsxbc Date: Thu, 30 Dec 2021 23:34:11 +0800 Subject: [PATCH 0634/1260] support `` for cluster --- src/Common/Config/CMakeLists.txt | 1 + src/Common/Config/ConfigHelper.cpp | 23 ++++++++++++ src/Common/Config/ConfigHelper.h | 16 ++++++++ src/Common/tests/gtest_config_helper.cpp | 37 +++++++++++++++++++ src/Interpreters/Cluster.cpp | 3 +- .../test_config_xml_full/configs/config.xml | 18 +++++++++ .../test_config_yaml_full/configs/config.yaml | 6 +++ 7 files changed, 103 insertions(+), 1 deletion(-) create mode 100644 src/Common/Config/ConfigHelper.cpp create mode 100644 src/Common/Config/ConfigHelper.h create mode 100644 src/Common/tests/gtest_config_helper.cpp diff --git a/src/Common/Config/CMakeLists.txt b/src/Common/Config/CMakeLists.txt index 3da44be2af6..4d72960f727 100644 --- a/src/Common/Config/CMakeLists.txt +++ b/src/Common/Config/CMakeLists.txt @@ -4,6 +4,7 @@ set (SRCS configReadClient.cpp ConfigReloader.cpp YAMLParser.cpp + ConfigHelper.cpp ) add_library(clickhouse_common_config ${SRCS}) diff --git a/src/Common/Config/ConfigHelper.cpp b/src/Common/Config/ConfigHelper.cpp new file mode 100644 index 00000000000..e7248753d03 --- /dev/null +++ b/src/Common/Config/ConfigHelper.cpp @@ -0,0 +1,23 @@ +#include +#include + +namespace DB +{ + +namespace ConfigHelper +{ + +bool getBool(const Poco::Util::AbstractConfiguration & config, const std::string & key, bool default_, bool empty_as) +{ + if (!config.has(key)) + return default_; + Poco::Util::AbstractConfiguration::Keys sub_keys; + config.keys(key, sub_keys); + if (sub_keys.size() == 0 && config.getString(key).size() == 0) + return empty_as; + return config.getBool(key, default_); +} + +} + +} diff --git a/src/Common/Config/ConfigHelper.h b/src/Common/Config/ConfigHelper.h new file mode 100644 index 00000000000..cc747992eb0 --- /dev/null +++ b/src/Common/Config/ConfigHelper.h @@ -0,0 +1,16 @@ +#pragma once + +namespace Poco +{ + namespace Util + { + class AbstractConfiguration; + } +} + +namespace DB::ConfigHelper +{ + +bool getBool(const Poco::Util::AbstractConfiguration & config, const std::string & key, bool default_, bool empty_as); + +} diff --git a/src/Common/tests/gtest_config_helper.cpp b/src/Common/tests/gtest_config_helper.cpp new file mode 100644 index 00000000000..18753eeb792 --- /dev/null +++ b/src/Common/tests/gtest_config_helper.cpp @@ -0,0 +1,37 @@ +#include +#include +#include +#include + +#include + + +using namespace DB; + +TEST(Common, ConfigHelper_getBool) +{ + std::string xml(R"CONFIG( + 0 + 1 + Yes + + + + 1 + 1 + Yes1 +)CONFIG"); + + Poco::XML::DOMParser dom_parser; + Poco::AutoPtr document = dom_parser.parseString(xml); + Poco::AutoPtr config = new Poco::Util::XMLConfiguration(document); + EXPECT_EQ(ConfigHelper::getBool(*config, "zero_as_false", false, true), false); + EXPECT_EQ(ConfigHelper::getBool(*config, "one_as_true", false, true), true); + EXPECT_EQ(ConfigHelper::getBool(*config, "yes_as_true", false, true), true); + EXPECT_EQ(ConfigHelper::getBool(*config, "empty_as_true_1", false, true), true); + EXPECT_EQ(ConfigHelper::getBool(*config, "empty_as_true_2", false, true), true); + ASSERT_THROW(ConfigHelper::getBool(*config, "has_empty_child_1", false, true), Poco::Exception); + EXPECT_EQ(ConfigHelper::getBool(*config, "has_empty_child_2", false, true), true); + EXPECT_EQ(ConfigHelper::getBool(*config, "has_child_1", false, true), true); + ASSERT_THROW(ConfigHelper::getBool(*config, "has_child_2", false, true), Poco::Exception); +} diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index b7b6b84439b..05972f2ee50 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -101,7 +102,7 @@ Cluster::Address::Address( user = config.getString(config_prefix + ".user", "default"); password = config.getString(config_prefix + ".password", ""); default_database = config.getString(config_prefix + ".default_database", ""); - secure = config.getBool(config_prefix + ".secure", false) ? Protocol::Secure::Enable : Protocol::Secure::Disable; + secure = ConfigHelper::getBool(config, config_prefix + ".secure", false, /* empty_as */true) ? Protocol::Secure::Enable : Protocol::Secure::Disable; priority = config.getInt(config_prefix + ".priority", 1); const char * port_type = secure == Protocol::Secure::Enable ? "tcp_port_secure" : "tcp_port"; is_local = isLocal(config.getInt(port_type, 0)); diff --git a/tests/integration/test_config_xml_full/configs/config.xml b/tests/integration/test_config_xml_full/configs/config.xml index c277ff7341f..76eceedbcea 100644 --- a/tests/integration/test_config_xml_full/configs/config.xml +++ b/tests/integration/test_config_xml_full/configs/config.xml @@ -639,6 +639,24 @@ + + + + localhost + 9440 + + + + + + + + localhost + 9440 + + + + diff --git a/tests/integration/test_config_yaml_full/configs/config.yaml b/tests/integration/test_config_yaml_full/configs/config.yaml index 5958d463d21..21cf439f7ec 100644 --- a/tests/integration/test_config_yaml_full/configs/config.yaml +++ b/tests/integration/test_config_yaml_full/configs/config.yaml @@ -100,6 +100,12 @@ remote_servers: host: localhost port: 9440 secure: 1 + test_shard_localhost_secure_empty_tag: + shard: + replica: + host: localhost + port: 9440 + secure: test_unavailable_shard: shard: - replica: From 08b3273c408d6d3753875fedfdb2c44792ab9a2d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 30 Dec 2021 18:53:30 +0300 Subject: [PATCH 0635/1260] Update http.md --- docs/en/interfaces/http.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 4fc1d709b36..0be996817d6 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -9,7 +9,7 @@ The HTTP interface lets you use ClickHouse on any platform from any programming By default, `clickhouse-server` listens for HTTP on port 8123 (this can be changed in the config). -By default, `cURL` command is not available on user operating systems. Please refer this [documentation](https://curl.se/download.html) to install it before running the examples. +Sometimes, `curl` command is not available on user operating systems. On Ubuntu or Debian, run `sudo apt install curl`. Please refer this [documentation](https://curl.se/download.html) to install it before running the examples. If you make a `GET /` request without parameters, it returns 200 response code and the string which defined in [http_server_default_response](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-http_server_default_response) default value “Ok.” (with a line feed at the end) From f0e82e2cb64cfdcceb8077249786da2c555066cb Mon Sep 17 00:00:00 2001 From: cmsxbc Date: Fri, 31 Dec 2021 00:47:01 +0800 Subject: [PATCH 0636/1260] add comment for ConfigHelper::getBool --- src/Common/Config/ConfigHelper.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/Config/ConfigHelper.h b/src/Common/Config/ConfigHelper.h index cc747992eb0..62271bbaf0a 100644 --- a/src/Common/Config/ConfigHelper.h +++ b/src/Common/Config/ConfigHelper.h @@ -11,6 +11,8 @@ namespace Poco namespace DB::ConfigHelper { +/// The behavior is like `config.getBool(key, default_)`, +/// except when the tag is empty (aka. self-closing), `empty_as` will be used instead of throwing Poco::Exception. bool getBool(const Poco::Util::AbstractConfiguration & config, const std::string & key, bool default_, bool empty_as); } From f1d398ae4b5e538d7d0ce718555f3032d54e4190 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 30 Dec 2021 20:38:54 +0300 Subject: [PATCH 0637/1260] Add LineAsString output format --- src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp | 2 ++ .../0_stateless/02157_line_as_string_output_format.reference | 3 +++ .../queries/0_stateless/02157_line_as_string_output_format.sql | 2 ++ 3 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/02157_line_as_string_output_format.reference create mode 100644 tests/queries/0_stateless/02157_line_as_string_output_format.sql diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index 5d87f5a0b14..12159149335 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -95,6 +95,8 @@ void registerOutputFormatTabSeparated(FormatFactory & factory) registerWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func); registerWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func); + if (is_raw) + registerWithNamesAndTypes("LineAsString", register_func); } } diff --git a/tests/queries/0_stateless/02157_line_as_string_output_format.reference b/tests/queries/0_stateless/02157_line_as_string_output_format.reference new file mode 100644 index 00000000000..43d109b4ad8 --- /dev/null +++ b/tests/queries/0_stateless/02157_line_as_string_output_format.reference @@ -0,0 +1,3 @@ +10 2000-01-11 [0,1,2,3,4,5,6,7,8,9]  (10,'2000-01-11') +256 2000-09-13 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122,123,124,125,126,127,128,129,130,131,132,133,134,135,136,137,138,139,140,141,142,143,144,145,146,147,148,149,150,151,152,153,154,155,156,157,158,159,160,161,162,163,164,165,166,167,168,169,170,171,172,173,174,175,176,177,178,179,180,181,182,183,184,185,186,187,188,189,190,191,192,193,194,195,196,197,198,199,200,201,202,203,204,205,206,207,208,209,210,211,212,213,214,215,216,217,218,219,220,221,222,223,224,225,226,227,228,229,230,231,232,233,234,235,236,237,238,239,240,241,242,243,244,245,246,247,248,249,250,251,252,253,254,255]  +  !"#$%&'()*+,-./0123456789:;<=>?@ABCDEFGHIJKLMNOPQRSTUVWXYZ[\]^_`abcdefghijklmnopqrstuvwxyz{|}~ (256,'2000-09-13') diff --git a/tests/queries/0_stateless/02157_line_as_string_output_format.sql b/tests/queries/0_stateless/02157_line_as_string_output_format.sql new file mode 100644 index 00000000000..637795e9367 --- /dev/null +++ b/tests/queries/0_stateless/02157_line_as_string_output_format.sql @@ -0,0 +1,2 @@ +SET output_format_write_statistics = 0; +SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT LineAsString; From 2eec4304673fbbf0050b27c529d358bbf23ac217 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 30 Dec 2021 21:06:22 +0300 Subject: [PATCH 0638/1260] Update 02157_line_as_string_output_format.sql --- tests/queries/0_stateless/02157_line_as_string_output_format.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02157_line_as_string_output_format.sql b/tests/queries/0_stateless/02157_line_as_string_output_format.sql index 637795e9367..438eb82cba3 100644 --- a/tests/queries/0_stateless/02157_line_as_string_output_format.sql +++ b/tests/queries/0_stateless/02157_line_as_string_output_format.sql @@ -1,2 +1 @@ -SET output_format_write_statistics = 0; SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT LineAsString; From d547fddf22def7bdec11a1769b7fa0ab2590d59e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Dec 2021 21:37:21 +0300 Subject: [PATCH 0639/1260] Add missing access control to some system commands --- src/Access/Common/AccessType.h | 1 + src/Interpreters/InterpreterSystemQuery.cpp | 3 +++ 2 files changed, 4 insertions(+) diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index cb6c326cb84..4472e975878 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -160,6 +160,7 @@ enum class AccessType M(SYSTEM_FLUSH_DISTRIBUTED, "FLUSH DISTRIBUTED", TABLE, SYSTEM_FLUSH) \ M(SYSTEM_FLUSH_LOGS, "FLUSH LOGS", GLOBAL, SYSTEM_FLUSH) \ M(SYSTEM_FLUSH, "", GROUP, SYSTEM) \ + M(SYSTEM_THREAD_FUZZER, "SYSTEM START THREAD FUZZER, SYSTEM STOP THREAD FUZZER, START THREAD FUZZER, STOP THREAD FUZZER", GLOBAL, SYSTEM) \ M(SYSTEM, "", GROUP, ALL) /* allows to execute SYSTEM {SHUTDOWN|RELOAD CONFIG|...} */ \ \ M(dictGet, "dictHas, dictGetHierarchy, dictIsIn", DICTIONARY, ALL) /* allows to execute functions dictGet(), dictHas(), dictGetHierarchy(), dictIsIn() */\ diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index b39ededaa91..123ff6ba2ca 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -250,6 +250,7 @@ BlockIO InterpreterSystemQuery::execute() } case Type::SUSPEND: { + getContext()->checkAccess(AccessType::SYSTEM_SHUTDOWN); auto command = fmt::format("kill -STOP {0} && sleep {1} && kill -CONT {0}", getpid(), query.seconds); LOG_DEBUG(log, "Will run {}", command); auto res = ShellCommand::execute(command); @@ -453,9 +454,11 @@ BlockIO InterpreterSystemQuery::execute() case Type::START_LISTEN_QUERIES: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{} is not supported yet", query.type); case Type::STOP_THREAD_FUZZER: + getContext()->checkAccess(AccessType::SYSTEM_THREAD_FUZZER); ThreadFuzzer::stop(); break; case Type::START_THREAD_FUZZER: + getContext()->checkAccess(AccessType::SYSTEM_THREAD_FUZZER); ThreadFuzzer::start(); break; default: From ae2813a2d51a22c588663e21c29cb86a7199b72f Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 30 Dec 2021 23:29:06 +0300 Subject: [PATCH 0640/1260] Fix --- src/Common/LRUResourceCache.h | 62 ++++++++++++++++++++++------------- 1 file changed, 40 insertions(+), 22 deletions(-) diff --git a/src/Common/LRUResourceCache.h b/src/Common/LRUResourceCache.h index b872d649a57..e1a28e7ab60 100644 --- a/src/Common/LRUResourceCache.h +++ b/src/Common/LRUResourceCache.h @@ -17,15 +17,14 @@ struct TrivailLRUResourceCacheWeightFunction size_t operator()(const T &) const { return 1; } }; -/* - * A resource cache with key index. There is only one instance for every key which is not like the normal resource pool. - * Resource cache has max weight capacity and keys size limitation. If the limitation is exceeded, keys would be evicted - * by LRU policy. +/** + * Similar to implementation in LRUCache.h, but with the difference that keys can + * only be evicted when they are releasable. Release state is controlled by this implementation. + * get() and getOrSet() methods return a Holder to actual value, which does release() in destructor. * - * acquire and release must be used in pair. + * Warning (!): This implementation is in development, not to be used. */ -template < - typename TKey, +template , typename HashFunction = std::hash> @@ -35,21 +34,28 @@ public: using Key = TKey; using Mapped = TMapped; using MappedPtr = std::shared_ptr; + class MappedHolder { public: + MappedHolder(LRUResourceCache * cache_, const Key & key_, MappedPtr value_) + : cache(cache_), key(key_), val(value_) {} + ~MappedHolder() { cache->release(key); } - Mapped & value() { return *val.get(); } - MappedHolder(LRUResourceCache * cache_, const Key & key_, MappedPtr value_) : cache(cache_), key(key_), val(value_) { } + + Mapped & value() { return *val; } protected: LRUResourceCache * cache; Key key; MappedPtr val; }; + using MappedHolderPtr = std::unique_ptr; - // use get() or getOrSet() to access the elements + explicit LRUResourceCache(size_t max_weight_, size_t max_element_size_ = 0) + : max_weight(max_weight_), max_element_size(max_element_size_) {} + MappedHolderPtr get(const Key & key) { auto mapped_ptr = getImpl(key); @@ -57,6 +63,7 @@ public: return nullptr; return std::make_unique(this, key, mapped_ptr); } + template MappedHolderPtr getOrSet(const Key & key, LoadFunc && load_func) { @@ -66,7 +73,8 @@ public: return std::make_unique(this, key, mapped_ptr); } - // If the key's reference_count = 0, delete it immediately. otherwise, mark it expired, and delete in release + // If the key's reference_count = 0, delete it immediately. + // Otherwise, mark it expired (not visible to get()), and delete when refcount is 0. void tryRemove(const Key & key) { std::lock_guard lock(mutex); @@ -84,9 +92,6 @@ public: cell.expired = true; } - LRUResourceCache(size_t max_weight_, size_t max_element_size_ = 0) : max_weight(max_weight_), max_element_size(max_element_size_) { } - ~LRUResourceCache() = default; - size_t weight() { std::lock_guard lock(mutex); @@ -197,8 +202,7 @@ private: std::atomic misses{0}; std::atomic evict_count{0}; - // - load_func : when key is not exists in cache, load_func is called to generate a new value - // - return: is null when there is no more space for the new value or the old value is in used. + /// Returns nullptr when there is no more space for the new value or the old value is in used. template MappedPtr getImpl(const Key & key, LoadFunc && load_func) { @@ -210,7 +214,7 @@ private: { if (!it->second.expired) { - hits++; + ++hits; it->second.reference_count += 1; queue.splice(queue.end(), queue, it->second.queue_iterator); return it->second.value; @@ -224,7 +228,7 @@ private: abort(); } } - misses++; + ++misses; auto & token = insert_tokens[key]; if (!token) token = std::make_shared(*this); @@ -269,13 +273,15 @@ private: MappedPtr getImpl(const Key & key) { std::lock_guard lock(mutex); + auto it = cells.find(key); if (it == cells.end() || it->second.expired) { - misses++; + ++misses; return nullptr; } - hits++; + + ++hits; it->second.reference_count += 1; queue.splice(queue.end(), queue, it->second.queue_iterator); return it->second.value; @@ -285,12 +291,14 @@ private: void release(const Key & key) { std::lock_guard lock(mutex); + auto it = cells.find(key); if (it == cells.end() || it->second.reference_count == 0) { LOG_ERROR(&Poco::Logger::get("LRUResourceCache"), "try to release an invalid element"); abort(); } + auto & cell = it->second; cell.reference_count -= 1; if (cell.expired && cell.reference_count == 0) @@ -325,20 +333,25 @@ private: auto weight = value ? weight_function(*value) : 0; auto queue_size = cells.size() + 1; auto loss_weight = 0; + auto is_overflow = [&] { return current_weight + weight - loss_weight > max_weight || (max_element_size != 0 && queue_size > max_element_size); }; + auto key_it = queue.begin(); std::unordered_set to_release_keys; + while (is_overflow() && queue_size > 1 && key_it != queue.end()) { const Key & key = *key_it; + auto cell_it = cells.find(key); if (cell_it == cells.end()) { LOG_ERROR(&Poco::Logger::get("LRUResourceCache"), "LRUResourceCache became inconsistent. There must be a bug in it."); abort(); } + auto & cell = cell_it->second; if (cell.reference_count == 0) { @@ -346,22 +359,27 @@ private: queue_size -= 1; to_release_keys.insert(key); } - key_it++; + + ++key_it; } + if (is_overflow()) return nullptr; + if (loss_weight > current_weight + weight) { LOG_ERROR(&Poco::Logger::get("LRUResourceCache"), "LRUResourceCache became inconsistent. There must be a bug in it."); abort(); } + for (auto & key : to_release_keys) { auto & cell = cells[key]; queue.erase(cell.queue_iterator); cells.erase(key); - evict_count++; + ++evict_count; } + current_weight = current_weight + weight - loss_weight; auto & new_cell = cells[insert_key]; From a3bbdffe3d4926c928dd788b8867255d33a4f88f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Dec 2021 23:39:43 +0300 Subject: [PATCH 0641/1260] Update test --- tests/queries/0_stateless/01271_show_privileges.reference | 1 + .../0_stateless/02117_show_create_table_system.reference | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index d347f149230..cc237a40a3f 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -114,6 +114,7 @@ SYSTEM RESTORE REPLICA ['RESTORE REPLICA'] TABLE SYSTEM SYSTEM FLUSH DISTRIBUTED ['FLUSH DISTRIBUTED'] TABLE SYSTEM FLUSH SYSTEM FLUSH LOGS ['FLUSH LOGS'] GLOBAL SYSTEM FLUSH SYSTEM FLUSH [] \N SYSTEM +SYSTEM THREAD FUZZER ['SYSTEM START THREAD FUZZER','SYSTEM STOP THREAD FUZZER','START THREAD FUZZER','STOP THREAD FUZZER'] GLOBAL SYSTEM SYSTEM [] \N ALL dictGet ['dictHas','dictGetHierarchy','dictIsIn'] DICTIONARY ALL addressToLine [] GLOBAL INTROSPECTION diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 35de7f8e82c..b52a8712087 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -20,7 +20,7 @@ CREATE TABLE system.errors\n(\n `name` String,\n `code` Int32,\n `value CREATE TABLE system.events\n(\n `event` String,\n `value` UInt64,\n `description` String\n)\nENGINE = SystemEvents()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.formats\n(\n `name` String,\n `is_input` UInt8,\n `is_output` UInt8\n)\nENGINE = SystemFormats()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.functions\n(\n `name` String,\n `is_aggregate` UInt8,\n `case_insensitive` UInt8,\n `alias_to` String,\n `create_query` String,\n `origin` Enum8(\'System\' = 0, \'SQLUserDefined\' = 1, \'ExecutableUserDefined\' = 2)\n)\nENGINE = SystemFunctions()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.grants\n(\n `user_name` Nullable(String),\n `role_name` Nullable(String),\n `access_type` Enum8(\'SQLITE\' = -128, \'ODBC\' = -127, \'JDBC\' = -126, \'HDFS\' = -125, \'S3\' = -124, \'SOURCES\' = -123, \'ALL\' = -122, \'NONE\' = -121, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM\' = 116, \'dictGet\' = 117, \'addressToLine\' = 118, \'addressToSymbol\' = 119, \'demangle\' = 120, \'INTROSPECTION\' = 121, \'FILE\' = 122, \'URL\' = 123, \'REMOTE\' = 124, \'MONGO\' = 125, \'MYSQL\' = 126, \'POSTGRES\' = 127),\n `database` Nullable(String),\n `table` Nullable(String),\n `column` Nullable(String),\n `is_partial_revoke` UInt8,\n `grant_option` UInt8\n)\nENGINE = SystemGrants()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.grants\n(\n `user_name` Nullable(String),\n `role_name` Nullable(String),\n `access_type` Enum8(\'POSTGRES\' = -128, \'SQLITE\' = -127, \'ODBC\' = -126, \'JDBC\' = -125, \'HDFS\' = -124, \'S3\' = -123, \'SOURCES\' = -122, \'ALL\' = -121, \'NONE\' = -120, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM THREAD FUZZER\' = 116, \'SYSTEM\' = 117, \'dictGet\' = 118, \'addressToLine\' = 119, \'addressToSymbol\' = 120, \'demangle\' = 121, \'INTROSPECTION\' = 122, \'FILE\' = 123, \'URL\' = 124, \'REMOTE\' = 125, \'MONGO\' = 126, \'MYSQL\' = 127),\n `database` Nullable(String),\n `table` Nullable(String),\n `column` Nullable(String),\n `is_partial_revoke` UInt8,\n `grant_option` UInt8\n)\nENGINE = SystemGrants()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.graphite_retentions\n(\n `config_name` String,\n `regexp` String,\n `function` String,\n `age` UInt64,\n `precision` UInt64,\n `priority` UInt16,\n `is_default` UInt8,\n `Tables.database` Array(String),\n `Tables.table` Array(String)\n)\nENGINE = SystemGraphite()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.licenses\n(\n `library_name` String,\n `license_type` String,\n `license_path` String,\n `license_text` String\n)\nENGINE = SystemLicenses()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.macros\n(\n `macro` String,\n `substitution` String\n)\nENGINE = SystemMacros()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' @@ -35,7 +35,7 @@ CREATE TABLE system.one\n(\n `dummy` UInt8\n)\nENGINE = SystemOne()\nCOMMENT CREATE TABLE system.part_moves_between_shards\n(\n `database` String,\n `table` String,\n `task_name` String,\n `task_uuid` UUID,\n `create_time` DateTime,\n `part_name` String,\n `part_uuid` UUID,\n `to_shard` String,\n `dst_part_name` String,\n `update_time` DateTime,\n `state` String,\n `rollback` UInt8,\n `num_tries` UInt32,\n `last_exception` String\n)\nENGINE = SystemShardMoves()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.parts\n(\n `partition` String,\n `name` String,\n `uuid` UUID,\n `part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `secondary_indices_compressed_bytes` UInt64,\n `secondary_indices_uncompressed_bytes` UInt64,\n `secondary_indices_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `is_frozen` UInt8,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `hash_of_all_files` String,\n `hash_of_uncompressed_files` String,\n `uncompressed_hash_of_compressed_files` String,\n `delete_ttl_info_min` DateTime,\n `delete_ttl_info_max` DateTime,\n `move_ttl_info.expression` Array(String),\n `move_ttl_info.min` Array(DateTime),\n `move_ttl_info.max` Array(DateTime),\n `default_compression_codec` String,\n `recompression_ttl_info.expression` Array(String),\n `recompression_ttl_info.min` Array(DateTime),\n `recompression_ttl_info.max` Array(DateTime),\n `group_by_ttl_info.expression` Array(String),\n `group_by_ttl_info.min` Array(DateTime),\n `group_by_ttl_info.max` Array(DateTime),\n `rows_where_ttl_info.expression` Array(String),\n `rows_where_ttl_info.min` Array(DateTime),\n `rows_where_ttl_info.max` Array(DateTime),\n `projections` Array(String),\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.parts_columns\n(\n `partition` String,\n `name` String,\n `uuid` UUID,\n `part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `column` String,\n `type` String,\n `column_position` UInt64,\n `default_kind` String,\n `default_expression` String,\n `column_bytes_on_disk` UInt64,\n `column_data_compressed_bytes` UInt64,\n `column_data_uncompressed_bytes` UInt64,\n `column_marks_bytes` UInt64,\n `serialization_kind` String,\n `subcolumns.names` Array(String),\n `subcolumns.types` Array(String),\n `subcolumns.serializations` Array(String),\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemPartsColumns()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.privileges\n(\n `privilege` Enum8(\'SQLITE\' = -128, \'ODBC\' = -127, \'JDBC\' = -126, \'HDFS\' = -125, \'S3\' = -124, \'SOURCES\' = -123, \'ALL\' = -122, \'NONE\' = -121, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM\' = 116, \'dictGet\' = 117, \'addressToLine\' = 118, \'addressToSymbol\' = 119, \'demangle\' = 120, \'INTROSPECTION\' = 121, \'FILE\' = 122, \'URL\' = 123, \'REMOTE\' = 124, \'MONGO\' = 125, \'MYSQL\' = 126, \'POSTGRES\' = 127),\n `aliases` Array(String),\n `level` Nullable(Enum8(\'GLOBAL\' = 0, \'DATABASE\' = 1, \'TABLE\' = 2, \'DICTIONARY\' = 3, \'VIEW\' = 4, \'COLUMN\' = 5)),\n `parent_group` Nullable(Enum8(\'SQLITE\' = -128, \'ODBC\' = -127, \'JDBC\' = -126, \'HDFS\' = -125, \'S3\' = -124, \'SOURCES\' = -123, \'ALL\' = -122, \'NONE\' = -121, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM\' = 116, \'dictGet\' = 117, \'addressToLine\' = 118, \'addressToSymbol\' = 119, \'demangle\' = 120, \'INTROSPECTION\' = 121, \'FILE\' = 122, \'URL\' = 123, \'REMOTE\' = 124, \'MONGO\' = 125, \'MYSQL\' = 126, \'POSTGRES\' = 127))\n)\nENGINE = SystemPrivileges()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.privileges\n(\n `privilege` Enum8(\'POSTGRES\' = -128, \'SQLITE\' = -127, \'ODBC\' = -126, \'JDBC\' = -125, \'HDFS\' = -124, \'S3\' = -123, \'SOURCES\' = -122, \'ALL\' = -121, \'NONE\' = -120, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM THREAD FUZZER\' = 116, \'SYSTEM\' = 117, \'dictGet\' = 118, \'addressToLine\' = 119, \'addressToSymbol\' = 120, \'demangle\' = 121, \'INTROSPECTION\' = 122, \'FILE\' = 123, \'URL\' = 124, \'REMOTE\' = 125, \'MONGO\' = 126, \'MYSQL\' = 127),\n `aliases` Array(String),\n `level` Nullable(Enum8(\'GLOBAL\' = 0, \'DATABASE\' = 1, \'TABLE\' = 2, \'DICTIONARY\' = 3, \'VIEW\' = 4, \'COLUMN\' = 5)),\n `parent_group` Nullable(Enum8(\'POSTGRES\' = -128, \'SQLITE\' = -127, \'ODBC\' = -126, \'JDBC\' = -125, \'HDFS\' = -124, \'S3\' = -123, \'SOURCES\' = -122, \'ALL\' = -121, \'NONE\' = -120, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM THREAD FUZZER\' = 116, \'SYSTEM\' = 117, \'dictGet\' = 118, \'addressToLine\' = 119, \'addressToSymbol\' = 120, \'demangle\' = 121, \'INTROSPECTION\' = 122, \'FILE\' = 123, \'URL\' = 124, \'REMOTE\' = 125, \'MONGO\' = 126, \'MYSQL\' = 127))\n)\nENGINE = SystemPrivileges()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.processes\n(\n `is_initial_query` UInt8,\n `user` String,\n `query_id` String,\n `address` IPv6,\n `port` UInt16,\n `initial_user` String,\n `initial_query_id` String,\n `initial_address` IPv6,\n `initial_port` UInt16,\n `interface` UInt8,\n `os_user` String,\n `client_hostname` String,\n `client_name` String,\n `client_revision` UInt64,\n `client_version_major` UInt64,\n `client_version_minor` UInt64,\n `client_version_patch` UInt64,\n `http_method` UInt8,\n `http_user_agent` String,\n `http_referer` String,\n `forwarded_for` String,\n `quota_key` String,\n `elapsed` Float64,\n `is_cancelled` UInt8,\n `read_rows` UInt64,\n `read_bytes` UInt64,\n `total_rows_approx` UInt64,\n `written_rows` UInt64,\n `written_bytes` UInt64,\n `memory_usage` Int64,\n `peak_memory_usage` Int64,\n `query` String,\n `thread_ids` Array(UInt64),\n `ProfileEvents` Map(String, UInt64),\n `Settings` Map(String, String),\n `current_database` String,\n `ProfileEvents.Names` Array(String),\n `ProfileEvents.Values` Array(UInt64),\n `Settings.Names` Array(String),\n `Settings.Values` Array(String)\n)\nENGINE = SystemProcesses()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.projection_parts\n(\n `partition` String,\n `name` String,\n `part_type` String,\n `parent_name` String,\n `parent_uuid` UUID,\n `parent_part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `parent_marks` UInt64,\n `parent_rows` UInt64,\n `parent_bytes_on_disk` UInt64,\n `parent_data_compressed_bytes` UInt64,\n `parent_data_uncompressed_bytes` UInt64,\n `parent_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `is_frozen` UInt8,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `hash_of_all_files` String,\n `hash_of_uncompressed_files` String,\n `uncompressed_hash_of_compressed_files` String,\n `delete_ttl_info_min` DateTime,\n `delete_ttl_info_max` DateTime,\n `move_ttl_info.expression` Array(String),\n `move_ttl_info.min` Array(DateTime),\n `move_ttl_info.max` Array(DateTime),\n `default_compression_codec` String,\n `recompression_ttl_info.expression` Array(String),\n `recompression_ttl_info.min` Array(DateTime),\n `recompression_ttl_info.max` Array(DateTime),\n `group_by_ttl_info.expression` Array(String),\n `group_by_ttl_info.min` Array(DateTime),\n `group_by_ttl_info.max` Array(DateTime),\n `rows_where_ttl_info.expression` Array(String),\n `rows_where_ttl_info.min` Array(DateTime),\n `rows_where_ttl_info.max` Array(DateTime),\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemProjectionParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.projection_parts_columns\n(\n `partition` String,\n `name` String,\n `part_type` String,\n `parent_name` String,\n `parent_uuid` UUID,\n `parent_part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `parent_marks` UInt64,\n `parent_rows` UInt64,\n `parent_bytes_on_disk` UInt64,\n `parent_data_compressed_bytes` UInt64,\n `parent_data_uncompressed_bytes` UInt64,\n `parent_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `column` String,\n `type` String,\n `column_position` UInt64,\n `default_kind` String,\n `default_expression` String,\n `column_bytes_on_disk` UInt64,\n `column_data_compressed_bytes` UInt64,\n `column_data_uncompressed_bytes` UInt64,\n `column_marks_bytes` UInt64,\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemProjectionPartsColumns()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' From fbe768054478f5305c2a65375f70ac59e61c2072 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Dec 2021 23:45:02 +0300 Subject: [PATCH 0642/1260] Add a test --- .../0_stateless/02157_readonly_system_suspend.reference | 1 + .../queries/0_stateless/02157_readonly_system_suspend.sh | 9 +++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/02157_readonly_system_suspend.reference create mode 100755 tests/queries/0_stateless/02157_readonly_system_suspend.sh diff --git a/tests/queries/0_stateless/02157_readonly_system_suspend.reference b/tests/queries/0_stateless/02157_readonly_system_suspend.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02157_readonly_system_suspend.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02157_readonly_system_suspend.sh b/tests/queries/0_stateless/02157_readonly_system_suspend.sh new file mode 100755 index 00000000000..77fe7b5f291 --- /dev/null +++ b/tests/queries/0_stateless/02157_readonly_system_suspend.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash + +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --readonly 1 --query "SYSTEM SUSPEND FOR 1 SECOND" 2>&1 | grep -c -F 'Code: 164' From 187c43eba8d5b722df8bd7a8af2aa21e565cd003 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 30 Dec 2021 17:27:22 +0300 Subject: [PATCH 0643/1260] rename Committed state to Active --- src/Common/CurrentMetrics.cpp | 6 +- src/Storages/MergeTree/DataPartsExchange.cpp | 4 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 18 +++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 14 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 74 +++++++++---------- src/Storages/MergeTree/MergeTreeData.h | 10 +-- .../ReplicatedMergeMutateTaskBase.cpp | 6 +- .../ReplicatedMergeTreePartCheckThread.cpp | 6 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- .../ReplicatedMergeTreeRestartingThread.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 14 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 38 +++++----- src/Storages/StorageReplicatedMergeTree.h | 6 +- src/Storages/System/StorageSystemParts.cpp | 2 +- .../System/StorageSystemPartsBase.cpp | 4 +- .../System/StorageSystemPartsColumns.cpp | 2 +- .../System/StorageSystemProjectionParts.cpp | 2 +- .../StorageSystemProjectionPartsColumns.cpp | 2 +- .../01600_parts_states_metrics_long.sh | 2 +- .../01660_system_parts_smoke.reference | 10 +-- 21 files changed, 117 insertions(+), 109 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 5c9ba177b78..896168253cf 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -63,8 +63,10 @@ M(MaxDDLEntryID, "Max processed DDL entry of DDLWorker.") \ M(MaxPushedDDLEntryID, "Max DDL entry of DDLWorker that pushed to zookeeper.") \ M(PartsTemporary, "The part is generating now, it is not in data_parts list.") \ - M(PartsPreCommitted, "The part is in data_parts, but not used for SELECTs.") \ - M(PartsCommitted, "Active data part, used by current and upcoming SELECTs.") \ + M(PartsPreCommitted, "Deprecated. See PartsPreActive.") \ + M(PartsCommitted, "Deprecated. See PartsActive.") \ + M(PartsPreActive, "The part is in data_parts, but not used for SELECTs.") \ + M(PartsActive, "Active data part, used by current and upcoming SELECTs.") \ M(PartsOutdated, "Not active data part, but could be used by only current SELECTs, could be deleted after SELECTs finishes.") \ M(PartsDeleting, "Not active data part with identity refcounter, it is deleting right now by a cleaner.") \ M(PartsDeleteOnDestroy, "Part was moved to another disk and should be deleted in own destructor.") \ diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 2855e21356d..2a964aecd4e 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -361,10 +361,10 @@ void Service::sendPartFromDiskRemoteMeta(const MergeTreeData::DataPartPtr & part MergeTreeData::DataPartPtr Service::findPart(const String & name) { - /// It is important to include PreCommitted and Outdated parts here because remote replicas cannot reliably + /// It is important to include PreActive and Outdated parts here because remote replicas cannot reliably /// determine the local state of the part, so queries for the parts in these states are completely normal. auto part = data.getPartIfExists( - name, {MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}); + name, {MergeTreeDataPartState::PreActive, MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); if (part) return part; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 83328594363..49e7b23968a 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -28,6 +28,8 @@ namespace CurrentMetrics extern const Metric PartsTemporary; extern const Metric PartsPreCommitted; extern const Metric PartsCommitted; + extern const Metric PartsPreActive; + extern const Metric PartsActive; extern const Metric PartsOutdated; extern const Metric PartsDeleting; extern const Metric PartsDeleteOnDestroy; @@ -189,10 +191,12 @@ static void incrementStateMetric(IMergeTreeDataPart::State state) case IMergeTreeDataPart::State::Temporary: CurrentMetrics::add(CurrentMetrics::PartsTemporary); return; - case IMergeTreeDataPart::State::PreCommitted: + case IMergeTreeDataPart::State::PreActive: + CurrentMetrics::add(CurrentMetrics::PartsPreActive); CurrentMetrics::add(CurrentMetrics::PartsPreCommitted); return; - case IMergeTreeDataPart::State::Committed: + case IMergeTreeDataPart::State::Active: + CurrentMetrics::add(CurrentMetrics::PartsActive); CurrentMetrics::add(CurrentMetrics::PartsCommitted); return; case IMergeTreeDataPart::State::Outdated: @@ -214,10 +218,12 @@ static void decrementStateMetric(IMergeTreeDataPart::State state) case IMergeTreeDataPart::State::Temporary: CurrentMetrics::sub(CurrentMetrics::PartsTemporary); return; - case IMergeTreeDataPart::State::PreCommitted: + case IMergeTreeDataPart::State::PreActive: + CurrentMetrics::sub(CurrentMetrics::PartsPreActive); CurrentMetrics::sub(CurrentMetrics::PartsPreCommitted); return; - case IMergeTreeDataPart::State::Committed: + case IMergeTreeDataPart::State::Active: + CurrentMetrics::sub(CurrentMetrics::PartsActive); CurrentMetrics::sub(CurrentMetrics::PartsCommitted); return; case IMergeTreeDataPart::State::Outdated: @@ -286,7 +292,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( , parent_part(parent_part_) { if (parent_part) - state = State::Committed; + state = State::Active; incrementStateMetric(state); incrementTypeMetric(part_type); @@ -311,7 +317,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( , parent_part(parent_part_) { if (parent_part) - state = State::Committed; + state = State::Active; incrementStateMetric(state); incrementTypeMetric(part_type); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index ab08ca1c33a..d9e5737e703 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -218,19 +218,19 @@ public: * Part state should be modified under data_parts mutex. * * Possible state transitions: - * Temporary -> Precommitted: we are trying to commit a fetched, inserted or merged part to active set - * Precommitted -> Outdated: we could not add a part to active set and are doing a rollback (for example it is duplicated part) - * Precommitted -> Committed: we successfully committed a part to active dataset - * Precommitted -> Outdated: a part was replaced by a covering part or DROP PARTITION + * Temporary -> PreActive: we are trying to add a fetched, inserted or merged part to active set + * PreActive -> Outdated: we could not add a part to active set and are doing a rollback (for example it is duplicated part) + * PreActive -> Active: we successfully added a part to active dataset + * PreActive -> Outdated: a part was replaced by a covering part or DROP PARTITION * Outdated -> Deleting: a cleaner selected this part for deletion * Deleting -> Outdated: if an ZooKeeper error occurred during the deletion, we will retry deletion - * Committed -> DeleteOnDestroy: if part was moved to another disk + * Active -> DeleteOnDestroy: if part was moved to another disk */ enum class State { Temporary, /// the part is generating now, it is not in data_parts list - PreCommitted, /// the part is in data_parts, but not used for SELECTs - Committed, /// active data part, used by current and upcoming SELECTs + PreActive, /// the part is in data_parts, but not used for SELECTs + Active, /// active data part, used by current and upcoming SELECTs Outdated, /// not active data part, but could be used by only current SELECTs, could be deleted after SELECTs finishes Deleting, /// not active data part with identity refcounter, it is deleting right now by a cleaner DeleteOnDestroy, /// part was moved to another disk and should be deleted in own destructor diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6597c28360d..660864b895f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1037,8 +1037,8 @@ void MergeTreeData::loadDataPartsFromDisk( has_adaptive_parts.store(true, std::memory_order_relaxed); part->modification_time = part_disk_ptr->getLastModified(fs::path(relative_data_path) / part_name).epochTime(); - /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later - part->setState(DataPartState::Committed); + /// Assume that all parts are Active, covered parts will be detected and marked as Outdated later + part->setState(DataPartState::Active); std::lock_guard loading_lock(mutex); auto [it, inserted] = data_parts_indexes.insert(part); @@ -1130,12 +1130,12 @@ void MergeTreeData::loadDataPartsFromWAL( { for (auto & part : parts_from_wal) { - if (getActiveContainingPart(part->info, DataPartState::Committed, part_lock)) + if (getActiveContainingPart(part->info, DataPartState::Active, part_lock)) continue; part->modification_time = time(nullptr); - /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later - part->setState(DataPartState::Committed); + /// Assume that all parts are Active, covered parts will be detected and marked as Outdated later + part->setState(DataPartState::Active); auto [it, inserted] = data_parts_indexes.insert(part); if (!inserted) @@ -1291,9 +1291,9 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) removePartContributionToDataVolume(*it); }; - (*prev_jt)->assertState({DataPartState::Committed}); + (*prev_jt)->assertState({DataPartState::Active}); - while (curr_jt != data_parts_by_state_and_info.end() && (*curr_jt)->getState() == DataPartState::Committed) + while (curr_jt != data_parts_by_state_and_info.end() && (*curr_jt)->getState() == DataPartState::Active) { /// Don't consider data parts belonging to different partitions. if ((*curr_jt)->info.partition_id != (*prev_jt)->info.partition_id) @@ -2367,8 +2367,8 @@ MergeTreeData::DataPartsVector MergeTreeData::getActivePartsToReplace( DataPartsLock & /* data_parts_lock */) const { /// Parts contained in the part are consecutive in data_parts, intersecting the insertion place for the part itself. - auto it_middle = data_parts_by_state_and_info.lower_bound(DataPartStateAndInfo{DataPartState::Committed, new_part_info}); - auto committed_parts_range = getDataPartsStateRange(DataPartState::Committed); + auto it_middle = data_parts_by_state_and_info.lower_bound(DataPartStateAndInfo{DataPartState::Active, new_part_info}); + auto committed_parts_range = getDataPartsStateRange(DataPartState::Active); /// Go to the left. DataPartIteratorByStateAndInfo begin = it_middle; @@ -2522,7 +2522,7 @@ bool MergeTreeData::renameTempPartAndReplace( part->name = part_name; part->info = part_info; part->is_temp = false; - part->setState(DataPartState::PreCommitted); + part->setState(DataPartState::PreActive); part->renameTo(part_name, true); auto part_it = data_parts_indexes.insert(part).first; @@ -2549,7 +2549,7 @@ bool MergeTreeData::renameTempPartAndReplace( decreaseDataVolume(reduce_bytes, reduce_rows, reduce_parts); - modifyPartState(part_it, DataPartState::Committed); + modifyPartState(part_it, DataPartState::Active); addPartContributionToColumnAndSecondaryIndexSizes(part); addPartContributionToDataVolume(part); } @@ -2591,13 +2591,13 @@ void MergeTreeData::removePartsFromWorkingSet(const MergeTreeData::DataPartsVect for (const DataPartPtr & part : remove) { - if (part->getState() == IMergeTreeDataPart::State::Committed) + if (part->getState() == IMergeTreeDataPart::State::Active) { removePartContributionToColumnAndSecondaryIndexSizes(part); removePartContributionToDataVolume(part); } - if (part->getState() == IMergeTreeDataPart::State::Committed || clear_without_timeout) + if (part->getState() == IMergeTreeDataPart::State::Active || clear_without_timeout) part->remove_time.store(remove_time, std::memory_order_relaxed); if (part->getState() != IMergeTreeDataPart::State::Outdated) @@ -2633,7 +2633,7 @@ void MergeTreeData::removePartsFromWorkingSet(const DataPartsVector & remove, bo if (!data_parts_by_info.count(part->info)) throw Exception("Part " + part->getNameWithState() + " not found in data_parts", ErrorCodes::LOGICAL_ERROR); - part->assertState({DataPartState::PreCommitted, DataPartState::Committed, DataPartState::Outdated}); + part->assertState({DataPartState::PreActive, DataPartState::Active, DataPartState::Outdated}); } removePartsFromWorkingSet(remove, clear_without_timeout, lock); @@ -2731,7 +2731,7 @@ restore_covered) /// What if part_to_detach is a reference to *it_part? Make a new owner just in case. DataPartPtr part = *it_part; - if (part->getState() == DataPartState::Committed) + if (part->getState() == DataPartState::Active) { removePartContributionToDataVolume(part); removePartContributionToColumnAndSecondaryIndexSizes(part); @@ -2758,7 +2758,7 @@ restore_covered) auto is_appropriate_state = [] (DataPartState state) { - return state == DataPartState::Committed || state == DataPartState::Outdated; + return state == DataPartState::Active || state == DataPartState::Outdated; }; auto update_error = [&] (DataPartIteratorByInfo it) @@ -2780,11 +2780,11 @@ restore_covered) if ((*it)->info.min_block != part->info.min_block) update_error(it); - if ((*it)->getState() != DataPartState::Committed) + if ((*it)->getState() != DataPartState::Active) { addPartContributionToColumnAndSecondaryIndexSizes(*it); addPartContributionToDataVolume(*it); - modifyPartState(it, DataPartState::Committed); // iterator is not invalidated here + modifyPartState(it, DataPartState::Active); // iterator is not invalidated here } pos = (*it)->info.max_block + 1; @@ -2811,11 +2811,11 @@ restore_covered) if ((*it)->info.min_block > pos) update_error(it); - if ((*it)->getState() != DataPartState::Committed) + if ((*it)->getState() != DataPartState::Active) { addPartContributionToColumnAndSecondaryIndexSizes(*it); addPartContributionToDataVolume(*it); - modifyPartState(it, DataPartState::Committed); + modifyPartState(it, DataPartState::Active); } pos = (*it)->info.max_block + 1; @@ -2929,7 +2929,7 @@ size_t MergeTreeData::getMaxPartsCountForPartitionWithState(DataPartState state) size_t MergeTreeData::getMaxPartsCountForPartition() const { - return getMaxPartsCountForPartitionWithState(DataPartState::Committed); + return getMaxPartsCountForPartitionWithState(DataPartState::Active); } @@ -2944,7 +2944,7 @@ std::optional MergeTreeData::getMinPartDataVersion() const auto lock = lockParts(); std::optional result; - for (const auto & part : getDataPartsStateRange(DataPartState::Committed)) + for (const auto & part : getDataPartsStateRange(DataPartState::Active)) { if (!result || *result > part->info.getDataVersion()) result = part->info.getDataVersion(); @@ -3050,7 +3050,7 @@ MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart( void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy) { auto lock = lockParts(); - for (auto original_active_part : getDataPartsStateRange(DataPartState::Committed)) // NOLINT (copy is intended) + for (auto original_active_part : getDataPartsStateRange(DataPartState::Active)) // NOLINT (copy is intended) { if (part_copy->name == original_active_part->name) { @@ -3075,7 +3075,7 @@ void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy) data_parts_indexes.erase(active_part_it); auto part_it = data_parts_indexes.insert(part_copy).first; - modifyPartState(part_it, DataPartState::Committed); + modifyPartState(part_it, DataPartState::Active); removePartContributionToDataVolume(original_active_part); addPartContributionToDataVolume(part_copy); @@ -3100,7 +3100,7 @@ void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy) MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(const MergeTreePartInfo & part_info) const { auto lock = lockParts(); - return getActiveContainingPart(part_info, DataPartState::Committed, lock); + return getActiveContainingPart(part_info, DataPartState::Active, lock); } MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(const String & part_name) const @@ -3170,7 +3170,7 @@ void MergeTreeData::calculateColumnAndSecondaryIndexSizesImpl() column_sizes.clear(); /// Take into account only committed parts - auto committed_parts_range = getDataPartsStateRange(DataPartState::Committed); + auto committed_parts_range = getDataPartsStateRange(DataPartState::Active); for (const auto & part : committed_parts_range) addPartContributionToColumnAndSecondaryIndexSizes(part); } @@ -3265,7 +3265,7 @@ void MergeTreeData::checkAlterPartitionIsPossible( void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition) { const String partition_id = getPartitionIDFromQuery(partition, getContext()); - auto parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + auto parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id); UInt64 partition_size = 0; @@ -3278,7 +3278,7 @@ void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition) void MergeTreeData::checkPartCanBeDropped(const String & part_name) { - auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Committed}); + auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active}); if (!part) throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in committed state", part_name); @@ -3304,7 +3304,7 @@ void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String & throw Exception("Part " + partition_id + " is not exists or not active", ErrorCodes::NO_SUCH_DATA_PART); } else - parts = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + parts = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id); auto disk = getStoragePolicy()->getDiskByName(name); if (!disk) @@ -3349,7 +3349,7 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String throw Exception("Part " + partition_id + " is not exists or not active", ErrorCodes::NO_SUCH_DATA_PART); } else - parts = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + parts = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id); auto volume = getStoragePolicy()->getVolumeByName(name); if (!volume) @@ -3531,7 +3531,7 @@ BackupEntries MergeTreeData::backup(const ASTs & partitions, ContextPtr local_co if (partitions.empty()) data_parts = getDataPartsVector(); else - data_parts = getDataPartsVectorInPartitions(MergeTreeDataPartState::Committed, getPartitionIDsFromQuery(partitions, local_context)); + data_parts = getDataPartsVectorInPartitions(MergeTreeDataPartState::Active, getPartitionIDsFromQuery(partitions, local_context)); return backupDataParts(data_parts); } @@ -4169,20 +4169,20 @@ MergeTreeData::DataParts MergeTreeData::getDataParts(const DataPartStates & affo MergeTreeData::DataParts MergeTreeData::getDataParts() const { - return getDataParts({DataPartState::Committed}); + return getDataParts({DataPartState::Active}); } MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector() const { - return getDataPartsVector({DataPartState::Committed}); + return getDataPartsVector({DataPartState::Active}); } MergeTreeData::DataPartPtr MergeTreeData::getAnyPartInPartition( const String & partition_id, DataPartsLock & /*data_parts_lock*/) const { - auto it = data_parts_by_state_and_info.lower_bound(DataPartStateAndPartitionID{DataPartState::Committed, partition_id}); + auto it = data_parts_by_state_and_info.lower_bound(DataPartStateAndPartitionID{DataPartState::Active, partition_id}); - if (it != data_parts_by_state_and_info.end() && (*it)->getState() == DataPartState::Committed && (*it)->info.partition_id == partition_id) + if (it != data_parts_by_state_and_info.end() && (*it)->getState() == DataPartState::Active && (*it)->info.partition_id == partition_id) return *it; return nullptr; @@ -4275,7 +4275,7 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData: add_rows += part->rows_count; ++add_parts; - data.modifyPartState(part, DataPartState::Committed); + data.modifyPartState(part, DataPartState::Active); data.addPartContributionToColumnAndSecondaryIndexSizes(part); } } @@ -5722,7 +5722,7 @@ ReservationPtr MergeTreeData::balancedReservation( for (const auto & part : covered_parts) submerging_big_parts_from_partition.insert(part->name); - for (const auto & part : getDataPartsStateRange(MergeTreeData::DataPartState::Committed)) + for (const auto & part : getDataPartsStateRange(MergeTreeData::DataPartState::Active)) { if (part->isStoredOnDisk() && part->getBytesOnDisk() >= min_bytes_to_rebalance_partition_over_jbod && part_info.partition_id == part->info.partition_id) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 380c2f4f4c5..31e1269f3a1 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -233,7 +233,7 @@ public: const VolumePtr & volume, const String & relative_path, const IMergeTreeDataPart * parent_part = nullptr) const; /// Auxiliary object to add a set of parts into the working set in two steps: - /// * First, as PreCommitted parts (the parts are ready, but not yet in the active set). + /// * First, as PreActive parts (the parts are ready, but not yet in the active set). /// * Next, if commit() is called, the parts are added to the active set and the parts that are /// covered by them are marked Outdated. /// If neither commit() nor rollback() was called, the destructor rollbacks the operation. @@ -452,7 +452,7 @@ public: MutableDataPartsVector tryLoadPartsToAttach(const ASTPtr & partition, bool attach_part, ContextPtr context, PartsTemporaryRename & renamed_parts); - /// Returns Committed parts + /// Returns Active parts DataParts getDataParts() const; DataPartsVector getDataPartsVector() const; @@ -494,7 +494,7 @@ public: /// Renames temporary part to a permanent part and adds it to the parts set. /// It is assumed that the part does not intersect with existing parts. /// If increment != nullptr, part index is determining using increment. Otherwise part index remains unchanged. - /// If out_transaction != nullptr, adds the part in the PreCommitted state (the part will be added to the + /// If out_transaction != nullptr, adds the part in the PreActive state (the part will be added to the /// active set later with out_transaction->commit()). /// Else, commits the part immediately. /// Returns true if part was added. Returns false if part is covered by bigger part. @@ -518,7 +518,7 @@ public: void removePartsFromWorkingSetImmediatelyAndSetTemporaryState(const DataPartsVector & remove); /// Removes parts from the working set parts. - /// Parts in add must already be in data_parts with PreCommitted, Committed, or Outdated states. + /// Parts in add must already be in data_parts with PreActive, Active, or Outdated states. /// If clear_without_timeout is true, the parts will be deleted at once, or during the next call to /// clearOldParts (ignoring old_parts_lifetime). void removePartsFromWorkingSet(const DataPartsVector & remove, bool clear_without_timeout, DataPartsLock * acquired_lock = nullptr); @@ -1049,7 +1049,7 @@ protected: /// If there is no part in the partition with ID `partition_id`, returns empty ptr. Should be called under the lock. DataPartPtr getAnyPartInPartition(const String & partition_id, DataPartsLock & data_parts_lock) const; - /// Return parts in the Committed set that are covered by the new_part_info or the part that covers it. + /// Return parts in the Active set that are covered by the new_part_info or the part that covers it. /// Will check that the new part doesn't already exist and that it doesn't intersect existing part. DataPartsVector getActivePartsToReplace( const MergeTreePartInfo & new_part_info, diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index 5fe7de70a20..db5ca15ce8a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -217,9 +217,9 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() ReplicatedMergeMutateTaskBase::CheckExistingPartResult ReplicatedMergeMutateTaskBase::checkExistingPart() { /// If we already have this part or a part covering it, we do not need to do anything. - /// The part may be still in the PreCommitted -> Committed transition so we first search - /// among PreCommitted parts to definitely find the desired part if it exists. - MergeTreeData::DataPartPtr existing_part = storage.getPartIfExists(entry.new_part_name, {MergeTreeDataPartState::PreCommitted}); + /// The part may be still in the PreActive -> Active transition so we first search + /// among PreActive parts to definitely find the desired part if it exists. + MergeTreeData::DataPartPtr existing_part = storage.getPartIfExists(entry.new_part_name, {MergeTreeDataPartState::PreActive}); if (!existing_part) existing_part = storage.getActiveContainingPart(entry.new_part_name); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 8d3cb146990..8fcaee66007 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -272,10 +272,10 @@ std::pair ReplicatedMergeTreePartCheckThread::findLo /// but checker thread will remove part from zookeeper and queue fetch. bool exists_in_zookeeper = zookeeper->exists(part_path); - /// If the part is still in the PreCommitted -> Committed transition, it is not lost + /// If the part is still in the PreActive -> Active transition, it is not lost /// and there is no need to go searching for it on other replicas. To definitely find the needed part - /// if it exists (or a part containing it) we first search among the PreCommitted parts. - auto part = storage.getPartIfExists(part_name, {MergeTreeDataPartState::PreCommitted}); + /// if it exists (or a part containing it) we first search among the PreActive parts. + auto part = storage.getPartIfExists(part_name, {MergeTreeDataPartState::PreActive}); if (!part) part = storage.getActiveContainingPart(part_name); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index b3da3d47684..1432728d00a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1177,7 +1177,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( return false; } - auto part = data.getPartIfExists(name, {MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}); + auto part = data.getPartIfExists(name, {MergeTreeDataPartState::PreActive, MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); if (part) { if (auto part_in_memory = asInMemoryPart(part)) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 0cc6955ff72..7a5b82979bd 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -253,7 +253,7 @@ void ReplicatedMergeTreeRestartingThread::removeFailedQuorumParts() for (const auto & part_name : failed_parts) { auto part = storage.getPartIfExists( - part_name, {MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}); + part_name, {MergeTreeDataPartState::PreActive, MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); if (part) { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 1ce748640dc..29d5cc0a8f5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -370,7 +370,7 @@ void ReplicatedMergeTreeSink::commitPart( block_id, existing_part_name); /// If it does not exist, we will write a new part with existing name. - /// Note that it may also appear on filesystem right now in PreCommitted state due to concurrent inserts of the same data. + /// Note that it may also appear on filesystem right now in PreActive state due to concurrent inserts of the same data. /// It will be checked when we will try to rename directory. part->name = existing_part_name; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 03ac27d0e46..11815d9ceef 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -229,7 +229,7 @@ std::optional StorageMergeTree::totalRows(const Settings &) const std::optional StorageMergeTree::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr local_context) const { - auto parts = getDataPartsVector({DataPartState::Committed}); + auto parts = getDataPartsVector({DataPartState::Active}); return totalRowsByPartitionPredicateImpl(query_info, local_context, parts); } @@ -1294,7 +1294,7 @@ MergeTreeDataPartPtr StorageMergeTree::outdatePart(const String & part_name, boo { /// Forcefully stop merges and make part outdated auto merge_blocker = stopMergesAndWait(); - auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Committed}); + auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active}); if (!part) throw Exception("Part " + part_name + " not found, won't try to drop it.", ErrorCodes::NO_SUCH_DATA_PART); removePartsFromWorkingSet({part}, true); @@ -1306,7 +1306,7 @@ MergeTreeDataPartPtr StorageMergeTree::outdatePart(const String & part_name, boo /// Wait merges selector std::unique_lock lock(currently_processing_in_background_mutex); - auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Committed}); + auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active}); /// It's okay, part was already removed if (!part) return nullptr; @@ -1344,7 +1344,7 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont /// This protects against "revival" of data for a removed partition after completion of merge. auto merge_blocker = stopMergesAndWait(); String partition_id = getPartitionIDFromQuery(partition, local_context); - parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id); /// TODO should we throw an exception if parts_to_remove is empty? removePartsFromWorkingSet(parts_to_remove, true); @@ -1426,7 +1426,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, my_metadata_snapshot); String partition_id = getPartitionIDFromQuery(partition, local_context); - DataPartsVector src_parts = src_data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + DataPartsVector src_parts = src_data.getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id); MutableDataPartsVector dst_parts; static const String TMP_PREFIX = "tmp_replace_from_"; @@ -1511,7 +1511,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const MergeTreeData & src_data = dest_table_storage->checkStructureAndGetMergeTreeData(*this, metadata_snapshot, dest_metadata_snapshot); String partition_id = getPartitionIDFromQuery(partition, local_context); - DataPartsVector src_parts = src_data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + DataPartsVector src_parts = src_data.getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id); MutableDataPartsVector dst_parts; static const String TMP_PREFIX = "tmp_move_from_"; @@ -1591,7 +1591,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ if (const auto & check_query = query->as(); check_query.partition) { String partition_id = getPartitionIDFromQuery(check_query.partition, local_context); - data_parts = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + data_parts = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id); } else data_parts = getDataPartsVector(); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ce5576bd809..004847f4082 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1120,8 +1120,8 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) /// Parts in ZK. NameSet expected_parts(expected_parts_vec.begin(), expected_parts_vec.end()); - /// There are no PreCommitted parts at startup. - auto parts = getDataParts({MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}); + /// There are no PreActive parts at startup. + auto parts = getDataParts({MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); /** Local parts that are not in ZK. * In very rare cases they may cover missing parts @@ -1460,9 +1460,9 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) if (is_get_or_attach || entry.type == LogEntry::MERGE_PARTS || entry.type == LogEntry::MUTATE_PART) { /// If we already have this part or a part covering it, we do not need to do anything. - /// The part may be still in the PreCommitted -> Committed transition so we first search - /// among PreCommitted parts to definitely find the desired part if it exists. - DataPartPtr existing_part = getPartIfExists(entry.new_part_name, {MergeTreeDataPartState::PreCommitted}); + /// The part may be still in the PreActive -> Active transition so we first search + /// among PreActive parts to definitely find the desired part if it exists. + DataPartPtr existing_part = getPartIfExists(entry.new_part_name, {MergeTreeDataPartState::PreActive}); if (!existing_part) existing_part = getActiveContainingPart(entry.new_part_name); @@ -1903,7 +1903,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) for (const PartDescriptionPtr & part_desc : all_parts) { - if (!getActiveContainingPart(part_desc->new_part_info, MergeTreeDataPartState::Committed, data_parts_lock)) + if (!getActiveContainingPart(part_desc->new_part_info, MergeTreeDataPartState::Active, data_parts_lock)) parts_to_add.emplace_back(part_desc); } @@ -1961,7 +1961,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); DataPartStates valid_states{ - MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}; + MergeTreeDataPartState::PreActive, MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}; size_t num_clonable_parts = 0; for (PartDescriptionPtr & part_desc : parts_to_add) @@ -3287,7 +3287,7 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n /// It's quite dangerous, so clone covered parts to detached. auto broken_part_info = MergeTreePartInfo::fromPartName(part_name, format_version); - auto partition_range = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, broken_part_info.partition_id); + auto partition_range = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, broken_part_info.partition_id); for (const auto & part : partition_range) { if (!broken_part_info.contains(part->info)) @@ -4219,7 +4219,7 @@ Pipe StorageReplicatedMergeTree::read( template -void StorageReplicatedMergeTree::foreachCommittedParts(Func && func, bool select_sequential_consistency) const +void StorageReplicatedMergeTree::foreachActiveParts(Func && func, bool select_sequential_consistency) const { std::optional max_added_blocks = {}; @@ -4230,7 +4230,7 @@ void StorageReplicatedMergeTree::foreachCommittedParts(Func && func, bool select max_added_blocks = getMaxAddedBlocks(); auto lock = lockParts(); - for (const auto & part : getDataPartsStateRange(DataPartState::Committed)) + for (const auto & part : getDataPartsStateRange(DataPartState::Active)) { if (part->isEmpty()) continue; @@ -4249,21 +4249,21 @@ void StorageReplicatedMergeTree::foreachCommittedParts(Func && func, bool select std::optional StorageReplicatedMergeTree::totalRows(const Settings & settings) const { UInt64 res = 0; - foreachCommittedParts([&res](auto & part) { res += part->rows_count; }, settings.select_sequential_consistency); + foreachActiveParts([&res](auto & part) { res += part->rows_count; }, settings.select_sequential_consistency); return res; } std::optional StorageReplicatedMergeTree::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr local_context) const { DataPartsVector parts; - foreachCommittedParts([&](auto & part) { parts.push_back(part); }, local_context->getSettingsRef().select_sequential_consistency); + foreachActiveParts([&](auto & part) { parts.push_back(part); }, local_context->getSettingsRef().select_sequential_consistency); return totalRowsByPartitionPredicateImpl(query_info, local_context, parts); } std::optional StorageReplicatedMergeTree::totalBytes(const Settings & settings) const { UInt64 res = 0; - foreachCommittedParts([&res](auto & part) { res += part->getBytesOnDisk(); }, settings.select_sequential_consistency); + foreachActiveParts([&res](auto & part) { res += part->getBytesOnDisk(); }, settings.select_sequential_consistency); return res; } @@ -4808,12 +4808,12 @@ void StorageReplicatedMergeTree::restoreMetadataInZooKeeper() const DataPartsVector all_parts = getAllDataPartsVector(); Strings active_parts_names; - /// Why all parts (not only Committed) are moved to detached/: + /// Why all parts (not only Active) are moved to detached/: /// After ZK metadata restoration ZK resets sequential counters (including block number counters), so one may /// potentially encounter a situation that a part we want to attach already exists. for (const auto & part : all_parts) { - if (part->getState() == DataPartState::Committed) + if (part->getState() == DataPartState::Active) active_parts_names.push_back(part->name); forgetPartAndMoveToDetached(part); @@ -6161,7 +6161,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( String partition_id = getPartitionIDFromQuery(partition, query_context); /// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet. - DataPartsVector src_all_parts = src_data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + DataPartsVector src_all_parts = src_data.getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id); LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size()); @@ -6591,7 +6591,7 @@ void StorageReplicatedMergeTree::movePartitionToShard( String part_name = partition->as().value.safeGet(); auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); - auto part = getPartIfExists(part_info, {MergeTreeDataPartState::Committed}); + auto part = getPartIfExists(part_info, {MergeTreeDataPartState::Active}); if (!part) throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "Part {} not found locally", part_name); @@ -6809,7 +6809,7 @@ bool StorageReplicatedMergeTree::dropPartImpl( { ReplicatedMergeTreeMergePredicate merge_pred = queue.getMergePredicate(zookeeper); - auto part = getPartIfExists(part_info, {MergeTreeDataPartState::Committed}); + auto part = getPartIfExists(part_info, {MergeTreeDataPartState::Active}); if (!part) { @@ -6984,7 +6984,7 @@ CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, Context if (const auto & check_query = query->as(); check_query.partition) { String partition_id = getPartitionIDFromQuery(check_query.partition, local_context); - data_parts = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); + data_parts = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id); } else data_parts = getDataPartsVector(); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index b2721210344..772ec335d68 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -245,7 +245,7 @@ public: inline String getReplicaName() const { return replica_name; } /// Restores table metadata if ZooKeeper lost it. - /// Used only on restarted readonly replicas (not checked). All active (Committed) parts are moved to detached/ + /// Used only on restarted readonly replicas (not checked). All active (Active) parts are moved to detached/ /// folder and attached. Parts in all other states are just moved to detached/ folder. void restoreMetadataInZooKeeper(); @@ -394,7 +394,7 @@ private: ThrottlerPtr replicated_sends_throttler; template - void foreachCommittedParts(Func && func, bool select_sequential_consistency) const; + void foreachActiveParts(Func && func, bool select_sequential_consistency) const; /** Creates the minimum set of nodes in ZooKeeper and create first replica. * Returns true if was created, false if exists. @@ -438,7 +438,7 @@ private: String getChecksumsForZooKeeper(const MergeTreeDataPartChecksums & checksums) const; - /// Accepts a PreCommitted part, atomically checks its checksums with ones on other replicas and commit the part + /// Accepts a PreActive part, atomically checks its checksums with ones on other replicas and commit the part DataPartsVector checkPartChecksumsAndCommit(Transaction & transaction, const DataPartPtr & part); bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const override; diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 6826082ef1d..2efb337b302 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -117,7 +117,7 @@ void StorageSystemParts::processNextStorage( if (columns_mask[src_index++]) columns[res_index++]->insert(part->getTypeName()); if (columns_mask[src_index++]) - columns[res_index++]->insert(part_state == State::Committed); + columns[res_index++]->insert(part_state == State::Active); if (columns_mask[src_index++]) columns[res_index++]->insert(part->getMarksCount()); if (columns_mask[src_index++]) diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index c730d5a95c9..6c8159ca720 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -57,12 +57,12 @@ StoragesInfo::getParts(MergeTreeData::DataPartStateVector & state, bool has_stat { /// If has_state_column is requested, return all states. if (!has_state_column) - return data->getDataPartsVector({State::Committed, State::Outdated}, &state, require_projection_parts); + return data->getDataPartsVector({State::Active, State::Outdated}, &state, require_projection_parts); return data->getAllDataPartsVector(&state, require_projection_parts); } - return data->getDataPartsVector({State::Committed}, &state, require_projection_parts); + return data->getDataPartsVector({State::Active}, &state, require_projection_parts); } StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context) diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index f1b3a13c332..f5e9b82c136 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -132,7 +132,7 @@ void StorageSystemPartsColumns::processNextStorage( if (columns_mask[src_index++]) columns[res_index++]->insert(part->getTypeName()); if (columns_mask[src_index++]) - columns[res_index++]->insert(part_state == State::Committed); + columns[res_index++]->insert(part_state == State::Active); if (columns_mask[src_index++]) columns[res_index++]->insert(part->getMarksCount()); diff --git a/src/Storages/System/StorageSystemProjectionParts.cpp b/src/Storages/System/StorageSystemProjectionParts.cpp index 378437bd4ec..d15acc97cb1 100644 --- a/src/Storages/System/StorageSystemProjectionParts.cpp +++ b/src/Storages/System/StorageSystemProjectionParts.cpp @@ -125,7 +125,7 @@ void StorageSystemProjectionParts::processNextStorage( if (columns_mask[src_index++]) columns[res_index++]->insert(parent_part->getTypeName()); if (columns_mask[src_index++]) - columns[res_index++]->insert(part_state == State::Committed); + columns[res_index++]->insert(part_state == State::Active); if (columns_mask[src_index++]) columns[res_index++]->insert(part->getMarksCount()); if (columns_mask[src_index++]) diff --git a/src/Storages/System/StorageSystemProjectionPartsColumns.cpp b/src/Storages/System/StorageSystemProjectionPartsColumns.cpp index f6490177014..29c877733d8 100644 --- a/src/Storages/System/StorageSystemProjectionPartsColumns.cpp +++ b/src/Storages/System/StorageSystemProjectionPartsColumns.cpp @@ -146,7 +146,7 @@ void StorageSystemProjectionPartsColumns::processNextStorage( if (columns_mask[src_index++]) columns[res_index++]->insert(parent_part->getTypeName()); if (columns_mask[src_index++]) - columns[res_index++]->insert(part_state == State::Committed); + columns[res_index++]->insert(part_state == State::Active); if (columns_mask[src_index++]) columns[res_index++]->insert(part->getMarksCount()); if (columns_mask[src_index++]) diff --git a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh index 9c0d28fdd91..f47d0863e69 100755 --- a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh +++ b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # NOTE: database = $CLICKHOUSE_DATABASE is unwanted verify_sql="SELECT - (SELECT sumIf(value, metric = 'PartsCommitted'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics) + (SELECT sumIf(value, metric = 'PartsActive'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics) = (SELECT sum(active), sum(NOT active) FROM system.parts)" # The query is not atomic - it can compare states between system.parts and system.metrics from different points in time. diff --git a/tests/queries/0_stateless/01660_system_parts_smoke.reference b/tests/queries/0_stateless/01660_system_parts_smoke.reference index f21fab8e539..36550f31bd0 100644 --- a/tests/queries/0_stateless/01660_system_parts_smoke.reference +++ b/tests/queries/0_stateless/01660_system_parts_smoke.reference @@ -1,13 +1,13 @@ # two parts -Committed -Committed -all_1_1_0 Committed -all_2_2_0 Committed +Active +Active +all_1_1_0 Active +all_2_2_0 Active all_1_1_0 1 all_2_2_0 1 # optimize +1 Active 2 Outdated -1 Committed # truncate Outdated Outdated From 8fe29eb665045b7aa6ff7464fa32213797339053 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 30 Dec 2021 23:48:32 +0300 Subject: [PATCH 0644/1260] Update 02157_line_as_string_output_format.sql --- .../queries/0_stateless/02157_line_as_string_output_format.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02157_line_as_string_output_format.sql b/tests/queries/0_stateless/02157_line_as_string_output_format.sql index 438eb82cba3..dbc76651f02 100644 --- a/tests/queries/0_stateless/02157_line_as_string_output_format.sql +++ b/tests/queries/0_stateless/02157_line_as_string_output_format.sql @@ -1 +1 @@ -SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT LineAsString; +SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT LineAsString; From 9fcfc5c1889878b3d7fec7f730112a8099238a69 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 30 Dec 2021 23:49:36 +0300 Subject: [PATCH 0645/1260] Update 02157_line_as_string_output_format.sql --- .../queries/0_stateless/02157_line_as_string_output_format.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02157_line_as_string_output_format.sql b/tests/queries/0_stateless/02157_line_as_string_output_format.sql index dbc76651f02..f1c567cf41d 100644 --- a/tests/queries/0_stateless/02157_line_as_string_output_format.sql +++ b/tests/queries/0_stateless/02157_line_as_string_output_format.sql @@ -1 +1 @@ -SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT LineAsString; +SELECT 'Hello \\ World' FORMAT LineAsString; From 764631f2d4ecf486f2c5b91787c44c13a7b4d2dd Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 30 Dec 2021 23:49:53 +0300 Subject: [PATCH 0646/1260] Update 02157_line_as_string_output_format.reference --- .../0_stateless/02157_line_as_string_output_format.reference | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02157_line_as_string_output_format.reference b/tests/queries/0_stateless/02157_line_as_string_output_format.reference index 43d109b4ad8..196aafcda30 100644 --- a/tests/queries/0_stateless/02157_line_as_string_output_format.reference +++ b/tests/queries/0_stateless/02157_line_as_string_output_format.reference @@ -1,3 +1 @@ -10 2000-01-11 [0,1,2,3,4,5,6,7,8,9]  (10,'2000-01-11') -256 2000-09-13 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122,123,124,125,126,127,128,129,130,131,132,133,134,135,136,137,138,139,140,141,142,143,144,145,146,147,148,149,150,151,152,153,154,155,156,157,158,159,160,161,162,163,164,165,166,167,168,169,170,171,172,173,174,175,176,177,178,179,180,181,182,183,184,185,186,187,188,189,190,191,192,193,194,195,196,197,198,199,200,201,202,203,204,205,206,207,208,209,210,211,212,213,214,215,216,217,218,219,220,221,222,223,224,225,226,227,228,229,230,231,232,233,234,235,236,237,238,239,240,241,242,243,244,245,246,247,248,249,250,251,252,253,254,255]  -  !"#$%&'()*+,-./0123456789:;<=>?@ABCDEFGHIJKLMNOPQRSTUVWXYZ[\]^_`abcdefghijklmnopqrstuvwxyz{|}~ (256,'2000-09-13') +Hello \ World From fccc4dade1ba24fd75ff66202f23262097675fde Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 30 Dec 2021 22:18:22 +0100 Subject: [PATCH 0647/1260] Fix (?) encoding issue in #33331 --- tests/ci/pr_info.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 812834824b7..4b0c97d7183 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -140,16 +140,15 @@ class PRInfo: if not self.diff_url: raise Exception("Diff URL cannot be find for event") + response = requests.get(self.diff_url) + response.raise_for_status() if 'commits' in self.event and self.number == 0: - response = requests.get(self.diff_url) - response.raise_for_status() diff = response.json() if 'files' in diff: self.changed_files = [f['filename'] for f in diff['files']] else: - diff = urllib.request.urlopen(self.diff_url) - diff_object = PatchSet(diff, diff.headers.get_charsets()[0]) + diff_object = PatchSet(response.text) self.changed_files = {f.path for f in diff_object} def get_dict(self): From 5df89e978e879cb1879d6ecdf1373160326d2002 Mon Sep 17 00:00:00 2001 From: "Anselmo D. Adams" <96303263+anselmodadams@users.noreply.github.com> Date: Thu, 30 Dec 2021 19:33:31 -0300 Subject: [PATCH 0648/1260] Update dictionary.md Fixed order of DICTIONARY term in create statement --- docs/en/sql-reference/statements/create/dictionary.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/dictionary.md b/docs/en/sql-reference/statements/create/dictionary.md index 61428cce126..9e7aae293eb 100644 --- a/docs/en/sql-reference/statements/create/dictionary.md +++ b/docs/en/sql-reference/statements/create/dictionary.md @@ -10,7 +10,7 @@ Creates a new [external dictionary](../../../sql-reference/dictionaries/external **Syntax** ``` sql -CREATE DICTIONARY [OR REPLACE][IF NOT EXISTS] [db.]dictionary_name [ON CLUSTER cluster] +CREATE [OR REPLACE] [IF NOT EXISTS] DICTIONARY [db.]dictionary_name [ON CLUSTER cluster] ( key1 type1 [DEFAULT|EXPRESSION expr1] [IS_OBJECT_ID], key2 type2 [DEFAULT|EXPRESSION expr2], From 883b83acb0eca239bc02fc52381d46ad82f5b2cd Mon Sep 17 00:00:00 2001 From: "Anselmo D. Adams" <96303263+anselmodadams@users.noreply.github.com> Date: Thu, 30 Dec 2021 19:43:54 -0300 Subject: [PATCH 0649/1260] Update dictionary.md --- docs/en/sql-reference/statements/create/dictionary.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/dictionary.md b/docs/en/sql-reference/statements/create/dictionary.md index 9e7aae293eb..86ab8f977b0 100644 --- a/docs/en/sql-reference/statements/create/dictionary.md +++ b/docs/en/sql-reference/statements/create/dictionary.md @@ -10,7 +10,7 @@ Creates a new [external dictionary](../../../sql-reference/dictionaries/external **Syntax** ``` sql -CREATE [OR REPLACE] [IF NOT EXISTS] DICTIONARY [db.]dictionary_name [ON CLUSTER cluster] +CREATE [OR REPLACE] DICTIONARY [IF NOT EXISTS] [db.]dictionary_name [ON CLUSTER cluster] ( key1 type1 [DEFAULT|EXPRESSION expr1] [IS_OBJECT_ID], key2 type2 [DEFAULT|EXPRESSION expr2], From fd9f1193d8ae4dd7b6df4e1b7b881b8f969cd369 Mon Sep 17 00:00:00 2001 From: cmsxbc Date: Fri, 31 Dec 2021 08:30:21 +0800 Subject: [PATCH 0650/1260] use .empty() instead of .size() == 0 --- src/Common/Config/ConfigHelper.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/Config/ConfigHelper.cpp b/src/Common/Config/ConfigHelper.cpp index e7248753d03..69fe42de86c 100644 --- a/src/Common/Config/ConfigHelper.cpp +++ b/src/Common/Config/ConfigHelper.cpp @@ -13,7 +13,7 @@ bool getBool(const Poco::Util::AbstractConfiguration & config, const std::string return default_; Poco::Util::AbstractConfiguration::Keys sub_keys; config.keys(key, sub_keys); - if (sub_keys.size() == 0 && config.getString(key).size() == 0) + if (sub_keys.empty() && config.getString(key).empty()) return empty_as; return config.getBool(key, default_); } From 4c36657926ccc609541432408bda8522dac1a119 Mon Sep 17 00:00:00 2001 From: SuperDJY Date: Fri, 31 Dec 2021 10:17:17 +0800 Subject: [PATCH 0651/1260] link clickhouse_common_config to unit_test_dbms --- src/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 0fe66314114..b51fad2ca25 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -567,6 +567,7 @@ if (ENABLE_TESTS AND USE_GTEST) clickhouse_storages_system dbms clickhouse_common_zookeeper + clickhouse_common_config string_utils) add_check(unit_tests_dbms) From 4322fd4bdb7f948ce86802b522b445c1b9448174 Mon Sep 17 00:00:00 2001 From: SuperDJY Date: Fri, 31 Dec 2021 10:19:03 +0800 Subject: [PATCH 0652/1260] avoid underscore in gtest name --- src/Common/tests/gtest_config_helper.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/tests/gtest_config_helper.cpp b/src/Common/tests/gtest_config_helper.cpp index 18753eeb792..59a6cfa0ae0 100644 --- a/src/Common/tests/gtest_config_helper.cpp +++ b/src/Common/tests/gtest_config_helper.cpp @@ -8,7 +8,7 @@ using namespace DB; -TEST(Common, ConfigHelper_getBool) +TEST(Common, ConfigHelperGetBool) { std::string xml(R"CONFIG( 0 From b347e8356c07818ff58aabd4c9fcb8de52a6b0fd Mon Sep 17 00:00:00 2001 From: benbiti Date: Fri, 31 Dec 2021 10:46:07 +0800 Subject: [PATCH 0653/1260] [Docs-zh]update zh docs note tag --- docs/zh/sql-reference/aggregate-functions/reference/corr.md | 2 +- .../sql-reference/aggregate-functions/reference/covarpop.md | 2 +- .../sql-reference/aggregate-functions/reference/covarsamp.md | 2 +- .../aggregate-functions/reference/quantiletiming.md | 2 +- .../aggregate-functions/reference/quantiletimingweighted.md | 2 +- .../sql-reference/aggregate-functions/reference/stddevpop.md | 4 ++-- .../sql-reference/aggregate-functions/reference/stddevsamp.md | 2 +- docs/zh/sql-reference/aggregate-functions/reference/varpop.md | 2 +- .../zh/sql-reference/aggregate-functions/reference/varsamp.md | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/docs/zh/sql-reference/aggregate-functions/reference/corr.md b/docs/zh/sql-reference/aggregate-functions/reference/corr.md index 5ab49f75023..5352aed5fc4 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/corr.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/corr.md @@ -12,4 +12,4 @@ toc_priority: 107 计算Pearson相关系数: `Σ((x - x̅)(y - y̅)) / sqrt(Σ((x - x̅)^2) * Σ((y - y̅)^2))`。 !!! note "注" -该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `corrStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 + 该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `corrStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 diff --git a/docs/zh/sql-reference/aggregate-functions/reference/covarpop.md b/docs/zh/sql-reference/aggregate-functions/reference/covarpop.md index c6f43c6b9e9..4b961a22795 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/covarpop.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/covarpop.md @@ -12,4 +12,4 @@ covarPop(x, y) 计算 `Σ((x - x̅)(y - y̅)) / n` 的值。 !!! note "注" -该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `covarPopStable` 函数。 它的工作速度较慢,但提供了较低的计算错误。 + 该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `covarPopStable` 函数。 它的工作速度较慢,但提供了较低的计算错误。 diff --git a/docs/zh/sql-reference/aggregate-functions/reference/covarsamp.md b/docs/zh/sql-reference/aggregate-functions/reference/covarsamp.md index 5ef5104504b..bed522bbbfa 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/covarsamp.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/covarsamp.md @@ -14,4 +14,4 @@ covarSamp(x, y) 返回Float64。 当 `n <= 1`, 返回 +∞。 !!! note "注" -该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `covarSampStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 + 该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `covarSampStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 diff --git a/docs/zh/sql-reference/aggregate-functions/reference/quantiletiming.md b/docs/zh/sql-reference/aggregate-functions/reference/quantiletiming.md index a193b60338a..72bd797279f 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/quantiletiming.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/quantiletiming.md @@ -46,7 +46,7 @@ quantileTiming(level)(expr) 类型: `Float32`。 !!! note "注" -如果没有值传递给函数(当使用 `quantileTimingIf`), [NaN](../../../sql-reference/data-types/float.md#data_type-float-nan-inf)被返回。 这样做的目的是将这些案例与导致零的案例区分开来。 参见 [ORDER BY clause](../../../sql-reference/statements/select/order-by.md#select-order-by) 对于 `NaN` 值排序注意事项。 + 如果没有值传递给函数(当使用 `quantileTimingIf`), [NaN](../../../sql-reference/data-types/float.md#data_type-float-nan-inf)被返回。 这样做的目的是将这些案例与导致零的案例区分开来。 参见 [ORDER BY clause](../../../sql-reference/statements/select/order-by.md#select-order-by) 对于 `NaN` 值排序注意事项。 **示例** diff --git a/docs/zh/sql-reference/aggregate-functions/reference/quantiletimingweighted.md b/docs/zh/sql-reference/aggregate-functions/reference/quantiletimingweighted.md index 7b130dbddbd..3ae1124c9c0 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/quantiletimingweighted.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/quantiletimingweighted.md @@ -48,7 +48,7 @@ quantileTimingWeighted(level)(expr, weight) 类型: `Float32`。 !!! note "注" -如果没有值传递给函数(当使用 `quantileTimingIf`), [NaN](../../../sql-reference/data-types/float.md#data_type-float-nan-inf)被返回。 这样做的目的是将这些案例与导致零的案例区分开来。 参见 [ORDER BY clause](../../../sql-reference/statements/select/order-by.md#select-order-by) 对于 `NaN` 值排序注意事项。 + 如果没有值传递给函数(当使用 `quantileTimingIf`), [NaN](../../../sql-reference/data-types/float.md#data_type-float-nan-inf)被返回。 这样做的目的是将这些案例与导致零的案例区分开来。 参见 [ORDER BY clause](../../../sql-reference/statements/select/order-by.md#select-order-by) 对于 `NaN` 值排序注意事项。 **示例** diff --git a/docs/zh/sql-reference/aggregate-functions/reference/stddevpop.md b/docs/zh/sql-reference/aggregate-functions/reference/stddevpop.md index 378ef4ae7e4..03478bae900 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/stddevpop.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/stddevpop.md @@ -4,7 +4,7 @@ toc_priority: 30 # stddevPop {#stddevpop} -结果等于 [varPop] (../../../sql-reference/aggregate-functions/reference/varpop.md)的平方根。 +结果等于 [varPop](../../../sql-reference/aggregate-functions/reference/varpop.md)的平方根。 !!! note "注" -该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `stddevPopStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 + 该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `stddevPopStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 diff --git a/docs/zh/sql-reference/aggregate-functions/reference/stddevsamp.md b/docs/zh/sql-reference/aggregate-functions/reference/stddevsamp.md index 68a348146a9..d49b9d89fd9 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/stddevsamp.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/stddevsamp.md @@ -7,4 +7,4 @@ toc_priority: 31 结果等于 [varSamp] (../../../sql-reference/aggregate-functions/reference/varsamp.md)的平方根。 !!! note "注" -该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `stddevSampStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 + 该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `stddevSampStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 diff --git a/docs/zh/sql-reference/aggregate-functions/reference/varpop.md b/docs/zh/sql-reference/aggregate-functions/reference/varpop.md index 4dca8efde38..502c1887e38 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/varpop.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/varpop.md @@ -9,4 +9,4 @@ toc_priority: 32 换句话说,计算一组数据的离差。 返回 `Float64`。 !!! note "注" -该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `varPopStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 + 该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `varPopStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 diff --git a/docs/zh/sql-reference/aggregate-functions/reference/varsamp.md b/docs/zh/sql-reference/aggregate-functions/reference/varsamp.md index c83ee7e24d2..73481c329e4 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/varsamp.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/varsamp.md @@ -11,5 +11,5 @@ toc_priority: 33 返回 `Float64`。 当 `n <= 1`,返回 `+∞`。 !!! note "注" -该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `varSampStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 + 该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `varSampStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 From c0072c6db0e4e17681a37c23a8eb1d0d672295b9 Mon Sep 17 00:00:00 2001 From: freedomDR <1640145602@qq.com> Date: Fri, 31 Dec 2021 02:47:52 +0000 Subject: [PATCH 0654/1260] docs setting add shutdown_wait_unfinished_queries shutdown_wait_unfinished config --- docs/en/operations/settings/settings.md | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 510047f4353..8a0fd618d32 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4155,3 +4155,20 @@ Default value: `''`. Sets the character that is interpreted as a suffix after the result set for [CustomSeparated](../../interfaces/formats.md#format-customseparated) data format. Default value: `''`. + +## shutdown_wait_unfinished_queries + +Enables or disables waiting unfinished queries when shutdown server. + +Possible values: + +- 0 — Disabled. +- 1 — Enabled. The wait time equal shutdown_wait_unfinished config. + +Default value: 0. + +## shutdown_wait_unfinished + +The waiting time in seconds for currently handled connections when shutdown server. + +Default Value: 5. From 3761994f68883025d34d1d39da5c3b5526ea85ac Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 31 Dec 2021 07:07:01 +0000 Subject: [PATCH 0655/1260] fix inner table parser in window view --- src/Storages/WindowView/StorageWindowView.cpp | 30 +++++++++++++++---- ...7_window_view_parser_inner_table.reference | 8 +++++ .../01047_window_view_parser_inner_table.sql | 24 +++++++++++++++ 3 files changed, 57 insertions(+), 5 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 25ecc0e16ef..5f5991259fb 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -91,10 +92,6 @@ namespace data.is_hop = t->name == "hop"; auto temp_node = t->clone(); temp_node->setAlias(""); - if (startsWith(t->arguments->children[0]->getColumnName(), "toDateTime")) - throw Exception( - "The first argument of time window function should not be a constant value.", - ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); if (!data.window_function) { data.serialized_window_function = serializeAST(*temp_node); @@ -565,7 +562,13 @@ std::shared_ptr StorageWindowView::getInnerTableCreateQuery( inner_create_query->setDatabase(database_name); inner_create_query->setTable(table_name); - auto inner_select_query = std::static_pointer_cast(inner_query); + Aliases aliases; + QueryAliasesVisitor(aliases).visit(inner_query); + auto inner_query_normalized = inner_query->clone(); + QueryNormalizer::Data normalizer_data(aliases, {}, false, getContext()->getSettingsRef(), false); + QueryNormalizer(normalizer_data).visit(inner_query_normalized); + + auto inner_select_query = std::static_pointer_cast(inner_query_normalized); auto t_sample_block = InterpreterSelectQuery( @@ -582,6 +585,8 @@ std::shared_ptr StorageWindowView::getInnerTableCreateQuery( columns_list->children.push_back(column_window); } + bool has_window_id = false; + for (const auto & column : t_sample_block.getColumnsWithTypeAndName()) { ParserIdentifierWithOptionalParameters parser; @@ -591,8 +596,18 @@ std::shared_ptr StorageWindowView::getInnerTableCreateQuery( column_dec->name = column.name; column_dec->type = ast; columns_list->children.push_back(column_dec); + if(!is_time_column_func_now && !has_window_id) + { + if (startsWith(column.name, "windowID")) + has_window_id = true; + } } + if (!is_time_column_func_now && !has_window_id) + throw Exception( + "The first argument of time window function should not be a constant value.", + ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); + ToIdentifierMatcher::Data query_data; query_data.window_id_name = window_id_name; query_data.window_id_alias = window_id_alias; @@ -634,10 +649,15 @@ std::shared_ptr StorageWindowView::getInnerTableCreateQuery( /// tumble/hop -> windowID func_window_visitor.visit(node); to_identifier_visitor.visit(node); + QueryNormalizer(normalizer_data).visit(node); + node->setAlias(""); new_storage->set(field, node); } }; + for (auto & [alias_name, ast] : aliases) + ast = std::make_shared(ast->getColumnName()); + visit(storage->partition_by, new_storage->partition_by); visit(storage->primary_key, new_storage->primary_key); visit(storage->order_by, new_storage->order_by); diff --git a/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference b/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference index 77f48f2832c..96f7cbb1d69 100644 --- a/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference +++ b/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference @@ -1,8 +1,12 @@ ---TUMBLE--- +||---DEFAULT ENGINE WITH DATA COLUMN ALIAS--- +CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN NAME--- CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +||---DATA COLUMN ALIAS--- +CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY b\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- @@ -10,10 +14,14 @@ CREATE TABLE test_01047.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(t ||---PARTITION--- CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(____timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `windowID(____timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(____timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ---HOP--- +||---DEFAULT ENGINE WITH DATA COLUMN ALIAS--- +CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN NAME--- CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +||---DATA COLUMN ALIAS--- +CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY b\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- diff --git a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql index 777c5ae2a5a..595d93e0771 100644 --- a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql +++ b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql @@ -9,6 +9,12 @@ DROP TABLE IF EXISTS test_01047.mt; CREATE TABLE test_01047.mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); SELECT '---TUMBLE---'; +SELECT '||---DEFAULT ENGINE WITH DATA COLUMN ALIAS---'; +DROP TABLE IF EXISTS test_01047.wv; +DROP TABLE IF EXISTS test_01047.`.inner.wv`; +CREATE WINDOW VIEW test_01047.wv AS SELECT count(a) AS count, b as id FROM test_01047.mt GROUP BY id, tumble(timestamp, INTERVAL '1' SECOND); +SHOW CREATE TABLE test_01047.`.inner.wv`; + SELECT '||---WINDOW COLUMN NAME---'; DROP TABLE IF EXISTS test_01047.wv; DROP TABLE IF EXISTS test_01047.`.inner.wv`; @@ -21,6 +27,12 @@ DROP TABLE IF EXISTS test_01047.`.inner.wv`; CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, tumble(timestamp, INTERVAL '1' SECOND) AS wid FROM test_01047.mt GROUP BY wid; SHOW CREATE TABLE test_01047.`.inner.wv`; +SELECT '||---DATA COLUMN ALIAS---'; +DROP TABLE IF EXISTS test_01047.wv; +DROP TABLE IF EXISTS test_01047.`.inner.wv`; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY id AS SELECT count(a) AS count, b as id FROM test_01047.mt GROUP BY id, tumble(timestamp, INTERVAL '1' SECOND); +SHOW CREATE TABLE test_01047.`.inner.wv`; + SELECT '||---IDENTIFIER---'; DROP TABLE IF EXISTS test_01047.wv; DROP TABLE IF EXISTS test_01047.`.inner.wv`; @@ -41,6 +53,12 @@ SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '---HOP---'; +SELECT '||---DEFAULT ENGINE WITH DATA COLUMN ALIAS---'; +DROP TABLE IF EXISTS test_01047.wv; +DROP TABLE IF EXISTS test_01047.`.inner.wv`; +CREATE WINDOW VIEW test_01047.wv AS SELECT count(a) AS count, b as id FROM test_01047.mt GROUP BY id, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND); +SHOW CREATE TABLE test_01047.`.inner.wv`; + SELECT '||---WINDOW COLUMN NAME---'; DROP TABLE IF EXISTS test_01047.wv; DROP TABLE IF EXISTS test_01047.`.inner.wv`; @@ -53,6 +71,12 @@ DROP TABLE IF EXISTS test_01047.`.inner.wv`; CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM test_01047.mt GROUP BY wid; SHOW CREATE TABLE test_01047.`.inner.wv`; +SELECT '||---DATA COLUMN ALIAS---'; +DROP TABLE IF EXISTS test_01047.wv; +DROP TABLE IF EXISTS test_01047.`.inner.wv`; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY id AS SELECT count(a) AS count, b as id FROM test_01047.mt GROUP BY id, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND); +SHOW CREATE TABLE test_01047.`.inner.wv`; + SELECT '||---IDENTIFIER---'; DROP TABLE IF EXISTS test_01047.wv; DROP TABLE IF EXISTS test_01047.`.inner.wv`; From 0685fd99855bacd0bce02507c00a3bd7709eea61 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 31 Dec 2021 10:58:44 +0300 Subject: [PATCH 0656/1260] clickhouse-local: track memory under --max_memory_usage_in_client option --- programs/client/Client.cpp | 17 -------------- src/Client/ClientBase.cpp | 16 +++++++++++++ .../02003_memory_limit_in_client.expect | 23 ++++++++++++++++++- 3 files changed, 38 insertions(+), 18 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index e01677aaac6..b1e1345cf71 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -25,7 +25,6 @@ #include #include #include -#include "Common/MemoryTracker.h" #include #include @@ -56,11 +55,6 @@ #pragma GCC optimize("-fno-var-tracking-assignments") #endif -namespace CurrentMetrics -{ - extern const Metric MemoryTracking; -} - namespace fs = std::filesystem; @@ -410,16 +404,6 @@ try std::cout << std::fixed << std::setprecision(3); std::cerr << std::fixed << std::setprecision(3); - /// Limit on total memory usage - size_t max_client_memory_usage = config().getInt64("max_memory_usage_in_client", 0 /*default value*/); - - if (max_client_memory_usage != 0) - { - total_memory_tracker.setHardLimit(max_client_memory_usage); - total_memory_tracker.setDescription("(total)"); - total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); - } - registerFormats(); registerFunctions(); registerAggregateFunctions(); @@ -1014,7 +998,6 @@ void Client::addOptions(OptionsDescription & options_description) ("opentelemetry-tracestate", po::value(), "OpenTelemetry tracestate header as described by W3C Trace Context recommendation") ("no-warnings", "disable warnings when client connects to server") - ("max_memory_usage_in_client", po::value(), "sets memory limit in client") ; /// Commandline options related to external tables. diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index fb2a58978c6..0938a9cfee5 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include "Common/Exception.h" @@ -65,6 +66,11 @@ namespace fs = std::filesystem; using namespace std::literals; +namespace CurrentMetrics +{ + extern const Metric MemoryTracking; +} + namespace DB { @@ -1812,6 +1818,7 @@ void ClientBase::init(int argc, char ** argv) ("interactive", "Process queries-file or --query query and start interactive mode") ("pager", po::value(), "Pipe all output into this command (less or similar)") + ("max_memory_usage_in_client", po::value(), "Set memory limit in client/local server") ; addOptions(options_description); @@ -1917,6 +1924,15 @@ void ClientBase::init(int argc, char ** argv) processOptions(options_description, options, external_tables_arguments); argsToConfig(common_arguments, config(), 100); clearPasswordFromCommandLine(argc, argv); + + /// Limit on total memory usage + size_t max_client_memory_usage = config().getInt64("max_memory_usage_in_client", 0 /*default value*/); + if (max_client_memory_usage != 0) + { + total_memory_tracker.setHardLimit(max_client_memory_usage); + total_memory_tracker.setDescription("(total)"); + total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); + } } } diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.expect b/tests/queries/0_stateless/02003_memory_limit_in_client.expect index 29701f49746..a3d6d04110a 100755 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.expect +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.expect @@ -16,6 +16,10 @@ expect_after { } set basedir [file dirname $argv0] + +# +# Check that the query will fail in clickhouse-client +# spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1" expect ":) " @@ -28,7 +32,24 @@ expect ":) " send -- "\4" expect eof -set basedir [file dirname $argv0] +# +# Check that the query will fail in clickhouse-client +# +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1" +expect ":) " + +send -- "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000)\r" +expect "Code: 241" + +expect ":) " + +# Exit. +send -- "\4" +expect eof + +# +# Check that the query will not fail (due to max_untracked_memory) +# spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1" expect ":) " From f1c726a6757e0f2c41be26585ca4f5dc19abe83e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 31 Dec 2021 11:15:35 +0300 Subject: [PATCH 0657/1260] tests/integration/test_storage_kafka: cover Template format --- tests/integration/test_storage_kafka/test.py | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index ce4ec3df867..134d1d62743 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -445,15 +445,14 @@ def test_kafka_formats(kafka_cluster): # /src/Processors/Formats/IRowInputFormat.cpp:0: DB::IRowInputFormat::generate() @ 0x1de72710 in /usr/bin/clickhouse ], }, - # 'Template' : { - # 'data_sample' : [ - # '(id = 0, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)', - # # '(id = 1, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 2, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 3, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 4, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 5, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 6, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 7, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 8, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 9, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 10, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 11, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 12, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 13, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 14, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 15, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)', - # # '(id = 0, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)', - # # '' # tolerates - # ], - # 'extra_settings': ", format_template_row='template_row.format'" - # }, + 'Template' : { + 'data_sample' : [ + '(id = 0, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)', + '(id = 1, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 2, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 3, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 4, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 5, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 6, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 7, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 8, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 9, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 10, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 11, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 12, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 13, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 14, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)\n(id = 15, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)', + '(id = 0, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)', + ], + 'extra_settings': ", format_template_row='template_row.format'" + }, 'Regexp': { 'data_sample': [ '(id = 0, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)', From 63eac4b0cff00365be0231f496d5a4b626aec3be Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 31 Dec 2021 11:24:41 +0300 Subject: [PATCH 0658/1260] tests/integration/test_storage_kafka: cover CustomSeparated --- tests/integration/test_storage_kafka/test.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 134d1d62743..a92dafa0b8a 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -445,6 +445,13 @@ def test_kafka_formats(kafka_cluster): # /src/Processors/Formats/IRowInputFormat.cpp:0: DB::IRowInputFormat::generate() @ 0x1de72710 in /usr/bin/clickhouse ], }, + 'CustomSeparated' : { + 'data_sample' : [ + '0\t0\tAM\t0.5\t1\n', + '1\t0\tAM\t0.5\t1\n2\t0\tAM\t0.5\t1\n3\t0\tAM\t0.5\t1\n4\t0\tAM\t0.5\t1\n5\t0\tAM\t0.5\t1\n6\t0\tAM\t0.5\t1\n7\t0\tAM\t0.5\t1\n8\t0\tAM\t0.5\t1\n9\t0\tAM\t0.5\t1\n10\t0\tAM\t0.5\t1\n11\t0\tAM\t0.5\t1\n12\t0\tAM\t0.5\t1\n13\t0\tAM\t0.5\t1\n14\t0\tAM\t0.5\t1\n15\t0\tAM\t0.5\t1\n', + '0\t0\tAM\t0.5\t1\n', + ], + }, 'Template' : { 'data_sample' : [ '(id = 0, blockNo = 0, val1 = "AM", val2 = 0.5, val3 = 1)', From c2ed92d75dff7c3f340454996555969f001d82b8 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 31 Dec 2021 14:39:28 +0300 Subject: [PATCH 0659/1260] Update pr_info.py --- tests/ci/pr_info.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 4b0c97d7183..48464439dbc 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -1,7 +1,6 @@ #!/usr/bin/env python3 import json import os -import urllib import requests from unidiff import PatchSet From a4558a31359b7ca17c2d5f071e6212a493fb7247 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 31 Dec 2021 20:28:27 +0800 Subject: [PATCH 0660/1260] Fix blob_storage build --- cmake/find/blob_storage.cmake | 43 ++++++++++++++++------------------- 1 file changed, 20 insertions(+), 23 deletions(-) diff --git a/cmake/find/blob_storage.cmake b/cmake/find/blob_storage.cmake index ec1b97f4695..271b1e595a6 100644 --- a/cmake/find/blob_storage.cmake +++ b/cmake/find/blob_storage.cmake @@ -1,32 +1,29 @@ option (ENABLE_AZURE_BLOB_STORAGE "Enable Azure blob storage" ${ENABLE_LIBRARIES}) -if (ENABLE_AZURE_BLOB_STORAGE) - set(USE_AZURE_BLOB_STORAGE 1) - set(AZURE_BLOB_STORAGE_LIBRARY azure_sdk) -else() - return() -endif() - option(USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY "Set to FALSE to use system Azure SDK instead of bundled (OFF currently not implemented)" ON) -if ((NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/azure/sdk" - OR NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/azure/cmake-modules") - AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) - message (WARNING "submodule contrib/azure is missing. to fix try run: \n git submodule update --init") - set(USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY OFF) - set(USE_AZURE_BLOB_STORAGE 0) -endif () +if (ENABLE_AZURE_BLOB_STORAGE) + set(USE_AZURE_BLOB_STORAGE 1) + set(AZURE_BLOB_STORAGE_LIBRARY azure_sdk) + + if ((NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/azure/sdk" + OR NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/azure/cmake-modules") + AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) + message (WARNING "submodule contrib/azure is missing. to fix try run: \n git submodule update --init") + set(USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY OFF) + set(USE_AZURE_BLOB_STORAGE 0) + endif () + + if (NOT USE_INTERNAL_SSL_LIBRARY AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) + message (FATAL_ERROR "Currently Blob Storage support can be built only with internal SSL library") + endif() + + if (NOT USE_INTERNAL_CURL AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) + message (FATAL_ERROR "Currently Blob Storage support can be built only with internal curl library") + endif() -if (NOT USE_INTERNAL_SSL_LIBRARY AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) - message (FATAL_ERROR "Currently Blob Storage support can be built only with internal SSL library") endif() -if (NOT USE_INTERNAL_CURL AND USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) - message (FATAL_ERROR "Currently Blob Storage support can be built only with internal curl library") -endif() - -if (USE_AZURE_BLOB_STORAGE) - message (STATUS "Using Azure Blob Storage - ${USE_AZURE_BLOB_STORAGE}") -endif() +message (STATUS "Using Azure Blob Storage - ${USE_AZURE_BLOB_STORAGE}") From e6f4427a68a31f75c36fbcb712afd8cf3351a269 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 31 Dec 2021 20:53:03 +0800 Subject: [PATCH 0661/1260] Fix --- cmake/find/blob_storage.cmake | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/cmake/find/blob_storage.cmake b/cmake/find/blob_storage.cmake index 271b1e595a6..4ad7296e95e 100644 --- a/cmake/find/blob_storage.cmake +++ b/cmake/find/blob_storage.cmake @@ -1,10 +1,10 @@ option (ENABLE_AZURE_BLOB_STORAGE "Enable Azure blob storage" ${ENABLE_LIBRARIES}) -option(USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY - "Set to FALSE to use system Azure SDK instead of bundled (OFF currently not implemented)" - ON) - if (ENABLE_AZURE_BLOB_STORAGE) + option(USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY + "Set to FALSE to use system Azure SDK instead of bundled (OFF currently not implemented)" + ON) + set(USE_AZURE_BLOB_STORAGE 1) set(AZURE_BLOB_STORAGE_LIBRARY azure_sdk) From fcff7f4dc18438d3c9a8b325daba0b6360724ee0 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Fri, 31 Dec 2021 16:22:02 -0800 Subject: [PATCH 0662/1260] update copyright in docs --- docs/tools/build.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/build.py b/docs/tools/build.py index 785928cf4ab..75278075996 100755 --- a/docs/tools/build.py +++ b/docs/tools/build.py @@ -95,7 +95,7 @@ def build_for_lang(lang, args): site_dir=site_dir, strict=True, theme=theme_cfg, - copyright='©2016–2021 ClickHouse, Inc.', + copyright='©2016–2022 ClickHouse, Inc.', use_directory_urls=True, repo_name='ClickHouse/ClickHouse', repo_url='https://github.com/ClickHouse/ClickHouse/', From 8730dda895ef7b0c83a7a8a9449ba3fa9343c0c1 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 1 Jan 2022 09:16:30 +0800 Subject: [PATCH 0663/1260] fix hivte text --- src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp b/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp index 61e30ddca49..4cbdbc27764 100644 --- a/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/HiveTextRowInputFormat.cpp @@ -26,7 +26,7 @@ HiveTextRowInputFormat::HiveTextRowInputFormat( HiveTextRowInputFormat::HiveTextRowInputFormat( const Block & header_, std::unique_ptr buf_, const Params & params_, const FormatSettings & format_settings_) : CSVRowInputFormat( - header_, *buf_, params_, true, false, format_settings_, std::make_unique(std::move(buf_), format_settings)) + header_, *buf_, params_, true, false, format_settings_, std::make_unique(std::move(buf_), format_settings_)) { } From c8866feecc9ca3ae51cf66142a7a651413717fdd Mon Sep 17 00:00:00 2001 From: benbiti Date: Sat, 1 Jan 2022 10:56:42 +0800 Subject: [PATCH 0664/1260] [Docs-zh]translate create function into zh --- .../statements/create/function.md | 61 ++++++++++++++++++- 1 file changed, 60 insertions(+), 1 deletion(-) mode change 120000 => 100644 docs/zh/sql-reference/statements/create/function.md diff --git a/docs/zh/sql-reference/statements/create/function.md b/docs/zh/sql-reference/statements/create/function.md deleted file mode 120000 index d41429cb260..00000000000 --- a/docs/zh/sql-reference/statements/create/function.md +++ /dev/null @@ -1 +0,0 @@ -../../../../en/sql-reference/statements/create/function.md \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/create/function.md b/docs/zh/sql-reference/statements/create/function.md new file mode 100644 index 00000000000..d57810ac91b --- /dev/null +++ b/docs/zh/sql-reference/statements/create/function.md @@ -0,0 +1,60 @@ +--- +toc_priority: 38 +toc_title: FUNCTION +--- + +# CREATE FUNCTION {#create-function} + +用一个lambda表达式创建用户自定义函数。该表达式必须由函数参数、常数、运算符或其他函数调用组成。 + +**语法** + +```sql +CREATE FUNCTION name AS (parameter0, ...) -> expression +``` + +一个函数可以有任意数量的参数。 + +存在一些限制如下: + +- 函数名在用户自定义函数和系统函数中必须是唯一的。 +- 递归函数是不允许的。 +- 函数所使用的所有变量必须在其参数列表中指定。 + +如果违反了任何限制,就会产生异常。 + +**示例** + +查询: + +```sql +CREATE FUNCTION linear_equation AS (x, k, b) -> k*x + b; +SELECT number, linear_equation(number, 2, 1) FROM numbers(3); +``` + +结果: + +``` text +┌─number─┬─plus(multiply(2, number), 1)─┐ +│ 0 │ 1 │ +│ 1 │ 3 │ +│ 2 │ 5 │ +└────────┴──────────────────────────────┘ +``` + +在下面的查询中,[conditional function](../../../sql-reference/functions/conditional-functions.md)在用户自定义函数中被调用: + +```sql +CREATE FUNCTION parity_str AS (n) -> if(n % 2, 'odd', 'even'); +SELECT number, parity_str(number) FROM numbers(3); +``` + +结果: + +``` text +┌─number─┬─if(modulo(number, 2), 'odd', 'even')─┐ +│ 0 │ even │ +│ 1 │ odd │ +│ 2 │ even │ +└────────┴──────────────────────────────────────┘ +``` From 48c0b41ad5691c97ec704e050ef2893c652b36c9 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 1 Jan 2022 09:05:29 +0000 Subject: [PATCH 0665/1260] Enable binary arithmetic between Decimal and Float fix --- src/Functions/FunctionBinaryArithmetic.h | 45 +++- src/Functions/IsOperation.h | 5 +- src/Functions/plus.cpp | 7 +- ...ary_op_between_float_and_decimal.reference | 215 ++++++++++++++++++ ...55_binary_op_between_float_and_decimal.sql | 96 ++++++++ 5 files changed, 350 insertions(+), 18 deletions(-) create mode 100644 tests/queries/0_stateless/02155_binary_op_between_float_and_decimal.reference create mode 100644 tests/queries/0_stateless/02155_binary_op_between_float_and_decimal.sql diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 8d0ad5db208..acb973b57d8 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -144,7 +145,7 @@ public: /// e.g Decimal - Float64 = Float64 Case::minus && IsDataTypeDecimal && IsFloatingPoint, DataTypeFloat64>, - Case::multiply && IsDataTypeDecimal && IsFloatingPoint, + Case::minus && IsDataTypeDecimal && IsFloatingPoint, DataTypeFloat64>, /// e.g Decimal * Float64 = Float64 @@ -154,9 +155,21 @@ public: DataTypeFloat64>, /// e.g Decimal / Float64 = Float64 - Case::multiply && IsDataTypeDecimal && IsFloatingPoint, + Case::division && IsDataTypeDecimal && IsFloatingPoint, DataTypeFloat64>, - Case::multiply && IsDataTypeDecimal && IsFloatingPoint, + Case::division && IsDataTypeDecimal && IsFloatingPoint, + DataTypeFloat64>, + + /// e.g least(Decimal, Float64) = Float64 + Case::least && IsDataTypeDecimal && IsFloatingPoint, + DataTypeFloat64>, + Case::least && IsDataTypeDecimal && IsFloatingPoint, + DataTypeFloat64>, + + /// e.g greatest(Decimal, Float64) = Float64 + Case::greatest && IsDataTypeDecimal && IsFloatingPoint, + DataTypeFloat64>, + Case::greatest && IsDataTypeDecimal && IsFloatingPoint, DataTypeFloat64>, /// Decimal Real is not supported (traditional DBs convert Decimal Real to Real) @@ -1471,15 +1484,31 @@ public: else // we can't avoid the else because otherwise the compiler may assume the ResultDataType may be Invalid // and that would produce the compile error. { - using T0 = typename LeftDataType::FieldType; - using T1 = typename RightDataType::FieldType; + using T0 = std::conditional_t, Float64, typename LeftDataType::FieldType>; + using T1 = std::conditional_t, Float64, typename RightDataType::FieldType>; using ResultType = typename ResultDataType::FieldType; using ColVecT0 = ColumnVectorOrDecimal; using ColVecT1 = ColumnVectorOrDecimal; using ColVecResult = ColumnVectorOrDecimal; - const auto * const col_left_raw = arguments[0].column.get(); - const auto * const col_right_raw = arguments[1].column.get(); + + const IColumn * col_left_raw; + const IColumn * col_right_raw; + + /// When Decimal op Float32/64, convert both of them into Float64 + if constexpr ((IsDataTypeDecimal || IsDataTypeDecimal)&&IsFloatingPoint) + { + const auto converted_type = std::make_shared(); + auto c0_converted = castColumn(arguments[0], converted_type); + auto c1_converted = castColumn(arguments[1], converted_type); + col_left_raw = c0_converted.get(); + col_right_raw = c1_converted.get(); + } + else + { + col_left_raw = arguments[0].column.get(); + col_right_raw = arguments[1].column.get(); + } const size_t col_left_size = col_left_raw->size(); @@ -1489,7 +1518,7 @@ public: const ColVecT0 * const col_left = checkAndGetColumn(col_left_raw); const ColVecT1 * const col_right = checkAndGetColumn(col_right_raw); - if constexpr (IsDataTypeDecimal || IsDataTypeDecimal) + if constexpr ((IsDataTypeDecimal || IsDataTypeDecimal)&&!IsFloatingPoint) { return executeNumericWithDecimal( left, right, diff --git a/src/Functions/IsOperation.h b/src/Functions/IsOperation.h index 369978fe271..5af8ae77727 100644 --- a/src/Functions/IsOperation.h +++ b/src/Functions/IsOperation.h @@ -57,10 +57,7 @@ struct IsOperation static constexpr bool division = div_floating || div_int || div_int_or_zero; - static constexpr bool allow_decimal = - plus || minus || multiply || - div_floating || div_int || div_int_or_zero || - least || greatest; + static constexpr bool allow_decimal = plus || minus || multiply || division || least || greatest; }; } diff --git a/src/Functions/plus.cpp b/src/Functions/plus.cpp index 96e05b57f12..997cae0dbed 100644 --- a/src/Functions/plus.cpp +++ b/src/Functions/plus.cpp @@ -32,12 +32,7 @@ struct PlusImpl template static inline bool apply(A a, B b, Result & c) { - if constexpr (std::is_same_v) - { - c = static_cast(a) + b; - } - else - return common::addOverflow(static_cast(a), b, c); + return common::addOverflow(static_cast(a), b, c); } #if USE_EMBEDDED_COMPILER diff --git a/tests/queries/0_stateless/02155_binary_op_between_float_and_decimal.reference b/tests/queries/0_stateless/02155_binary_op_between_float_and_decimal.reference new file mode 100644 index 00000000000..861eba18203 --- /dev/null +++ b/tests/queries/0_stateless/02155_binary_op_between_float_and_decimal.reference @@ -0,0 +1,215 @@ +3 +0 +2.25 +1 +3 +0 +2.25 +1 +inf +1 +1.5 + +plus +-4.5 2.5 -2 +-2.5 2.5 0 +2.5 -2.5 0 +4.5 -2.5 2 +-4.5 -2.5 -7 +-2.5 -2.5 -5 +2.5 2.5 5 +4.5 2.5 7 +-45.5 2.5 -43 +-25.5 2.5 -23 +25.5 -2.5 23 +45.5 -2.5 43 +-45.5 -2.5 -48 +-25.5 -2.5 -27.999999999999996 +25.5 2.5 27.999999999999996 +45.5 2.5 48 +-4.5 -2.5 -7 +-2.5 -2.5 -5 +2.5 2.5 5 +4.5 2.5 7 +-4.5 2.5 -2 +2.5 -2.5 0 +-2.5 2.5 0 +4.5 -2.5 2 +-45.5 -2.5 -48 +-25.5 -2.5 -28 +25.5 2.5 28 +45.5 2.5 48 +-45.5 2.5 -43 +-25.5 2.5 -22.999999999999996 +25.5 -2.5 22.999999999999996 +45.5 -2.5 43 + +minus +-4.5 2.5 -7 +-2.5 2.5 -5 +2.5 -2.5 5 +4.5 -2.5 7 +-4.5 -2.5 -2 +-2.5 -2.5 0 +2.5 2.5 0 +4.5 2.5 2 +-45.5 2.5 -48 +-25.5 2.5 -28 +25.5 -2.5 28 +45.5 -2.5 48 +-45.5 -2.5 -43 +-25.5 -2.5 -22.999999999999996 +25.5 2.5 22.999999999999996 +45.5 2.5 43 +-4.5 -2.5 -2 +2.5 2.5 0 +-2.5 -2.5 0 +4.5 2.5 2 +-4.5 2.5 -7 +-2.5 2.5 -5 +2.5 -2.5 5 +4.5 -2.5 7 +-45.5 -2.5 -43 +-25.5 -2.5 -23 +25.5 2.5 23 +45.5 2.5 43 +-45.5 2.5 -48 +-25.5 2.5 -27.999999999999996 +25.5 -2.5 27.999999999999996 +45.5 -2.5 48 + +multiply +4.5 -2.5 -11.25 +-4.5 2.5 -11.25 +-2.5 2.5 -6.25 +2.5 -2.5 -6.25 +-2.5 -2.5 6.25 +2.5 2.5 6.25 +-4.5 -2.5 11.25 +4.5 2.5 11.25 +45.5 -2.5 -113.75 +-45.5 2.5 -113.75 +25.5 -2.5 -63.75 +-25.5 2.5 -63.75 +-25.5 -2.5 63.74999999999999 +25.5 2.5 63.74999999999999 +45.5 2.5 113.75 +-45.5 -2.5 113.75 +-2.5 -2.5 6.25 +2.5 2.5 6.25 +-4.5 -2.5 11.25 +4.5 2.5 11.25 +4.5 -2.5 -11.25 +-4.5 2.5 -11.25 +-2.5 2.5 -6.25 +2.5 -2.5 -6.25 +25.5 2.5 63.75 +-25.5 -2.5 63.75 +-45.5 -2.5 113.75 +45.5 2.5 113.75 +45.5 -2.5 -113.75 +-45.5 2.5 -113.75 +-25.5 2.5 -63.74999999999999 +25.5 -2.5 -63.74999999999999 + +division +-4.5 2.5 -1.8 +4.5 -2.5 -1.8 +-2.5 2.5 -1 +2.5 -2.5 -1 +-2.5 -2.5 1 +2.5 2.5 1 +4.5 2.5 1.8 +-4.5 -2.5 1.8 +-45.5 2.5 -18.2 +45.5 -2.5 -18.2 +25.5 -2.5 -10.2 +-25.5 2.5 -10.2 +25.5 2.5 10.2 +-25.5 -2.5 10.2 +45.5 2.5 18.2 +-45.5 -2.5 18.2 +2.5 2.5 1 +-2.5 -2.5 1 +-4.5 -2.5 1.8 +4.5 2.5 1.8 +-4.5 2.5 -1.8 +4.5 -2.5 -1.8 +-2.5 2.5 -1 +2.5 -2.5 -1 +25.5 2.5 10.2 +-25.5 -2.5 10.2 +-45.5 -2.5 18.2 +45.5 2.5 18.2 +45.5 -2.5 -18.2 +-45.5 2.5 -18.2 +-25.5 2.5 -10.2 +25.5 -2.5 -10.2 + +least +-4.5 2.5 -4.5 +4.5 -2.5 -2.5 +2.5 -2.5 -2.5 +-2.5 2.5 -2.5 +-4.5 -2.5 -4.5 +-2.5 -2.5 -2.5 +4.5 2.5 2.5 +2.5 2.5 2.5 +-45.5 2.5 -45.5 +-25.5 2.5 -25.5 +45.5 -2.5 -2.5 +25.5 -2.5 -2.5 +-45.5 -2.5 -45.5 +-25.5 -2.5 -25.499999999999996 +25.5 2.5 2.5 +45.5 2.5 2.5 +-4.5 -2.5 -4.5 +-2.5 -2.5 -2.5 +4.5 2.5 2.5 +2.5 2.5 2.5 +-4.5 2.5 -4.5 +4.5 -2.5 -2.5 +-2.5 2.5 -2.5 +2.5 -2.5 -2.5 +-45.5 -2.5 -45.5 +-25.5 -2.5 -25.5 +45.5 2.5 2.5 +25.5 2.5 2.5 +-45.5 2.5 -45.5 +-25.5 2.5 -25.499999999999996 +45.5 -2.5 -2.5 +25.5 -2.5 -2.5 + +greatest +-4.5 2.5 2.5 +-2.5 2.5 2.5 +2.5 -2.5 2.5 +4.5 -2.5 4.5 +-4.5 -2.5 -2.5 +-2.5 -2.5 -2.5 +2.5 2.5 2.5 +4.5 2.5 4.5 +-45.5 2.5 2.5 +-25.5 2.5 2.5 +25.5 -2.5 25.5 +45.5 -2.5 45.5 +-45.5 -2.5 -2.5 +-25.5 -2.5 -2.5 +25.5 2.5 25.499999999999996 +45.5 2.5 45.5 +-2.5 -2.5 -2.5 +-4.5 -2.5 -2.5 +2.5 2.5 2.5 +4.5 2.5 4.5 +-4.5 2.5 2.5 +-2.5 2.5 2.5 +2.5 -2.5 2.5 +4.5 -2.5 4.5 +-45.5 -2.5 -2.5 +-25.5 -2.5 -2.5 +25.5 2.5 25.5 +45.5 2.5 45.5 +-45.5 2.5 2.5 +-25.5 2.5 2.5 +25.5 -2.5 25.499999999999996 +45.5 -2.5 45.5 diff --git a/tests/queries/0_stateless/02155_binary_op_between_float_and_decimal.sql b/tests/queries/0_stateless/02155_binary_op_between_float_and_decimal.sql new file mode 100644 index 00000000000..1a7f1b77b9f --- /dev/null +++ b/tests/queries/0_stateless/02155_binary_op_between_float_and_decimal.sql @@ -0,0 +1,96 @@ +SELECT 1.5::Decimal32(5) + 1.5; +SELECT 1.5::Decimal32(5) - 1.5; +SELECT 1.5::Decimal32(5) * 1.5; +SELECT 1.5::Decimal32(5) / 1.5; + +SELECT 1.5 + 1.5::Decimal32(5); +SELECT 1.5 - 1.5::Decimal32(5); +SELECT 1.5 * 1.5::Decimal32(5); +SELECT 1.5 / 1.5::Decimal32(5); + +SELECT 1.0::Decimal32(5) / 0.0; + +SELECT least(1.5, 1.0::Decimal32(5)); +SELECT greatest(1.5, 1.0::Decimal32(5)); + +DROP TABLE IF EXISTS t; +CREATE TABLE t(d1 Decimal32(5), d2 Decimal64(10), d3 Decimal128(20), d4 Decimal256(40), f1 Float32, f2 Float64) ENGINE=Memory; + +INSERT INTO t values (-4.5, 4.5, -45.5, 45.5, 2.5, -2.5); +INSERT INTO t values (4.5, -4.5, 45.5, -45.5, -2.5, 2.5); +INSERT INTO t values (2.5, -2.5, 25.5, -25.5, -2.5, 2.5); +INSERT INTO t values (-2.5, 2.5, -25.5, 25.5, 2.5, -2.5); + +SELECT ''; +SELECT 'plus'; +SELECT d1, f1, d1 + f1 as plus FROM t ORDER BY plus; +SELECT d2, f1, d2 + f1 as plus FROM t ORDER BY plus; +SELECT d3, f1, d3 + f1 as plus FROM t ORDER BY plus; +SELECT d4, f1, d4 + f1 as plus FROM t ORDER BY plus; + +SELECT d1, f2, d1 + f2 as plus FROM t ORDER BY plus; +SELECT d2, f2, d2 + f2 as plus FROM t ORDER BY plus; +SELECT d3, f2, d3 + f2 as plus FROM t ORDER BY plus; +SELECT d4, f2, d4 + f2 as plus FROM t ORDER BY plus; + +SELECT ''; +SELECT 'minus'; +SELECT d1, f1, d1 - f1 as minus FROM t ORDER BY minus; +SELECT d2, f1, d2 - f1 as minus FROM t ORDER BY minus; +SELECT d3, f1, d3 - f1 as minus FROM t ORDER BY minus; +SELECT d4, f1, d4 - f1 as minus FROM t ORDER BY minus; + +SELECT d1, f2, d1 - f2 as minus FROM t ORDER BY minus; +SELECT d2, f2, d2 - f2 as minus FROM t ORDER BY minus; +SELECT d3, f2, d3 - f2 as minus FROM t ORDER BY minus; +SELECT d4, f2, d4 - f2 as minus FROM t ORDER BY minus; + +SELECT ''; +SELECT 'multiply'; +SELECT d1, f1, d1 * f1 as multiply FROM t ORDER BY multiply; +SELECT d2, f1, d2 * f1 as multiply FROM t ORDER BY multiply; +SELECT d3, f1, d3 * f1 as multiply FROM t ORDER BY multiply; +SELECT d4, f1, d4 * f1 as multiply FROM t ORDER BY multiply; + +SELECT d1, f2, d1 * f2 as multiply FROM t ORDER BY multiply; +SELECT d2, f2, d2 * f2 as multiply FROM t ORDER BY multiply; +SELECT d3, f2, d3 * f2 as multiply FROM t ORDER BY multiply; +SELECT d4, f2, d4 * f2 as multiply FROM t ORDER BY multiply; + +SELECT ''; +SELECT 'division'; +SELECT d1, f1, d1 / f1 as division FROM t ORDER BY division; +SELECT d2, f1, d2 / f1 as division FROM t ORDER BY division; +SELECT d3, f1, d3 / f1 as division FROM t ORDER BY division; +SELECT d4, f1, d4 / f1 as division FROM t ORDER BY division; + +SELECT d1, f2, d1 / f2 as division FROM t ORDER BY division; +SELECT d2, f2, d2 / f2 as division FROM t ORDER BY division; +SELECT d3, f2, d3 / f2 as division FROM t ORDER BY division; +SELECT d4, f2, d4 / f2 as division FROM t ORDER BY division; + +SELECT ''; +SELECT 'least'; +SELECT d1, f1, least(d1, f1) as least FROM t ORDER BY least; +SELECT d2, f1, least(d2, f1) as least FROM t ORDER BY least; +SELECT d3, f1, least(d3, f1) as least FROM t ORDER BY least; +SELECT d4, f1, least(d4, f1) as least FROM t ORDER BY least; + +SELECT d1, f2, least(d1, f2) as least FROM t ORDER BY least; +SELECT d2, f2, least(d2, f2) as least FROM t ORDER BY least; +SELECT d3, f2, least(d3, f2) as least FROM t ORDER BY least; +SELECT d4, f2, least(d4, f2) as least FROM t ORDER BY least; + +SELECT ''; +SELECT 'greatest'; +SELECT d1, f1, greatest(d1, f1) as greatest FROM t ORDER BY greatest; +SELECT d2, f1, greatest(d2, f1) as greatest FROM t ORDER BY greatest; +SELECT d3, f1, greatest(d3, f1) as greatest FROM t ORDER BY greatest; +SELECT d4, f1, greatest(d4, f1) as greatest FROM t ORDER BY greatest; + +SELECT d1, f2, greatest(d1, f2) as greatest FROM t ORDER BY greatest; +SELECT d2, f2, greatest(d2, f2) as greatest FROM t ORDER BY greatest; +SELECT d3, f2, greatest(d3, f2) as greatest FROM t ORDER BY greatest; +SELECT d4, f2, greatest(d4, f2) as greatest FROM t ORDER BY greatest; + +DROP TABLE t; From a8578e0ad9d165b1bbc0b720ceca1e72d0f87668 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 1 Jan 2022 09:25:52 +0000 Subject: [PATCH 0666/1260] fix test --- ...ary_op_between_float_and_decimal.reference | 276 +++++++++--------- ...55_binary_op_between_float_and_decimal.sql | 104 +++---- 2 files changed, 190 insertions(+), 190 deletions(-) diff --git a/tests/queries/0_stateless/02155_binary_op_between_float_and_decimal.reference b/tests/queries/0_stateless/02155_binary_op_between_float_and_decimal.reference index 861eba18203..a024d51e285 100644 --- a/tests/queries/0_stateless/02155_binary_op_between_float_and_decimal.reference +++ b/tests/queries/0_stateless/02155_binary_op_between_float_and_decimal.reference @@ -11,205 +11,205 @@ inf 1.5 plus --4.5 2.5 -2 --2.5 2.5 0 +4.5 -3.5 1 2.5 -2.5 0 -4.5 -2.5 2 --4.5 -2.5 -7 +-4.5 2.5 -2 +-2.5 3.5 1 +-4.5 -3.5 -8 -2.5 -2.5 -5 -2.5 2.5 5 4.5 2.5 7 --45.5 2.5 -43 --25.5 2.5 -23 +2.5 3.5 6 +45.5 -3.5 42 25.5 -2.5 23 -45.5 -2.5 43 --45.5 -2.5 -48 --25.5 -2.5 -27.999999999999996 -25.5 2.5 27.999999999999996 -45.5 2.5 48 --4.5 -2.5 -7 --2.5 -2.5 -5 -2.5 2.5 5 -4.5 2.5 7 --4.5 2.5 -2 -2.5 -2.5 0 --2.5 2.5 0 -4.5 -2.5 2 --45.5 -2.5 -48 --25.5 -2.5 -28 -25.5 2.5 28 -45.5 2.5 48 -45.5 2.5 -43 --25.5 2.5 -22.999999999999996 +-25.5 3.5 -22 +-45.5 -3.5 -49 +-25.5 -2.5 -27.999999999999996 +45.5 2.5 48 +25.5 3.5 28.999999999999996 +-4.5 -3.5 -8 +-2.5 -2.5 -5 +4.5 2.5 7 +2.5 3.5 6 +4.5 -3.5 1 +2.5 -2.5 0 +-4.5 2.5 -2 +-2.5 3.5 1 +-45.5 -3.5 -49 +-25.5 -2.5 -28 +45.5 2.5 48 +25.5 3.5 29 +45.5 -3.5 42 25.5 -2.5 22.999999999999996 -45.5 -2.5 43 +-45.5 2.5 -43 +-25.5 3.5 -21.999999999999996 minus --4.5 2.5 -7 --2.5 2.5 -5 +4.5 -3.5 8 2.5 -2.5 5 -4.5 -2.5 7 --4.5 -2.5 -2 +-4.5 2.5 -7 +-2.5 3.5 -6 +-4.5 -3.5 -1 -2.5 -2.5 0 -2.5 2.5 0 4.5 2.5 2 --45.5 2.5 -48 --25.5 2.5 -28 +2.5 3.5 -1 +45.5 -3.5 49 25.5 -2.5 28 -45.5 -2.5 48 --45.5 -2.5 -43 +-45.5 2.5 -48 +-25.5 3.5 -29 +-45.5 -3.5 -42 -25.5 -2.5 -22.999999999999996 -25.5 2.5 22.999999999999996 45.5 2.5 43 --4.5 -2.5 -2 -2.5 2.5 0 +25.5 3.5 21.999999999999996 +-4.5 -3.5 -1 -2.5 -2.5 0 4.5 2.5 2 --4.5 2.5 -7 --2.5 2.5 -5 +2.5 3.5 -1 +4.5 -3.5 8 2.5 -2.5 5 -4.5 -2.5 7 --45.5 -2.5 -43 +-4.5 2.5 -7 +-2.5 3.5 -6 +-45.5 -3.5 -42 -25.5 -2.5 -23 -25.5 2.5 23 45.5 2.5 43 --45.5 2.5 -48 --25.5 2.5 -27.999999999999996 +25.5 3.5 22 +45.5 -3.5 49 25.5 -2.5 27.999999999999996 -45.5 -2.5 48 +-45.5 2.5 -48 +-25.5 3.5 -28.999999999999996 multiply -4.5 -2.5 -11.25 --4.5 2.5 -11.25 --2.5 2.5 -6.25 +4.5 -3.5 -15.75 2.5 -2.5 -6.25 +-4.5 2.5 -11.25 +-2.5 3.5 -8.75 +-4.5 -3.5 15.75 -2.5 -2.5 6.25 -2.5 2.5 6.25 --4.5 -2.5 11.25 4.5 2.5 11.25 -45.5 -2.5 -113.75 --45.5 2.5 -113.75 +2.5 3.5 8.75 +45.5 -3.5 -159.25 25.5 -2.5 -63.75 --25.5 2.5 -63.75 --25.5 -2.5 63.74999999999999 -25.5 2.5 63.74999999999999 -45.5 2.5 113.75 --45.5 -2.5 113.75 --2.5 -2.5 6.25 -2.5 2.5 6.25 --4.5 -2.5 11.25 -4.5 2.5 11.25 -4.5 -2.5 -11.25 --4.5 2.5 -11.25 --2.5 2.5 -6.25 -2.5 -2.5 -6.25 -25.5 2.5 63.75 --25.5 -2.5 63.75 --45.5 -2.5 113.75 -45.5 2.5 113.75 -45.5 -2.5 -113.75 -45.5 2.5 -113.75 --25.5 2.5 -63.74999999999999 +-25.5 3.5 -89.25 +-45.5 -3.5 159.25 +-25.5 -2.5 63.74999999999999 +45.5 2.5 113.75 +25.5 3.5 89.24999999999999 +-4.5 -3.5 15.75 +-2.5 -2.5 6.25 +4.5 2.5 11.25 +2.5 3.5 8.75 +4.5 -3.5 -15.75 +2.5 -2.5 -6.25 +-4.5 2.5 -11.25 +-2.5 3.5 -8.75 +-45.5 -3.5 159.25 +-25.5 -2.5 63.75 +45.5 2.5 113.75 +25.5 3.5 89.25 +45.5 -3.5 -159.25 25.5 -2.5 -63.74999999999999 +-45.5 2.5 -113.75 +-25.5 3.5 -89.24999999999999 division --4.5 2.5 -1.8 -4.5 -2.5 -1.8 --2.5 2.5 -1 +4.5 -3.5 -1.2857142857142858 2.5 -2.5 -1 +-4.5 2.5 -1.8 +-2.5 3.5 -0.7142857142857143 +-4.5 -3.5 1.2857142857142858 -2.5 -2.5 1 -2.5 2.5 1 4.5 2.5 1.8 --4.5 -2.5 1.8 --45.5 2.5 -18.2 -45.5 -2.5 -18.2 +2.5 3.5 0.7142857142857143 +45.5 -3.5 -13 25.5 -2.5 -10.2 --25.5 2.5 -10.2 -25.5 2.5 10.2 +-45.5 2.5 -18.2 +-25.5 3.5 -7.285714285714286 +-45.5 -3.5 13 -25.5 -2.5 10.2 45.5 2.5 18.2 --45.5 -2.5 18.2 -2.5 2.5 1 +25.5 3.5 7.285714285714285 +-4.5 -3.5 1.2857142857142858 -2.5 -2.5 1 --4.5 -2.5 1.8 4.5 2.5 1.8 --4.5 2.5 -1.8 -4.5 -2.5 -1.8 --2.5 2.5 -1 +2.5 3.5 0.7142857142857143 +4.5 -3.5 -1.2857142857142858 2.5 -2.5 -1 -25.5 2.5 10.2 +-4.5 2.5 -1.8 +-2.5 3.5 -0.7142857142857143 +-45.5 -3.5 13 -25.5 -2.5 10.2 --45.5 -2.5 18.2 45.5 2.5 18.2 -45.5 -2.5 -18.2 --45.5 2.5 -18.2 --25.5 2.5 -10.2 +25.5 3.5 7.285714285714286 +45.5 -3.5 -13 25.5 -2.5 -10.2 +-45.5 2.5 -18.2 +-25.5 3.5 -7.285714285714285 least --4.5 2.5 -4.5 -4.5 -2.5 -2.5 +4.5 -3.5 -3.5 2.5 -2.5 -2.5 --2.5 2.5 -2.5 --4.5 -2.5 -4.5 +-4.5 2.5 -4.5 +-2.5 3.5 -2.5 +-4.5 -3.5 -4.5 -2.5 -2.5 -2.5 4.5 2.5 2.5 -2.5 2.5 2.5 --45.5 2.5 -45.5 --25.5 2.5 -25.5 -45.5 -2.5 -2.5 +2.5 3.5 2.5 +45.5 -3.5 -3.5 25.5 -2.5 -2.5 --45.5 -2.5 -45.5 +-45.5 2.5 -45.5 +-25.5 3.5 -25.5 +-45.5 -3.5 -45.5 -25.5 -2.5 -25.499999999999996 -25.5 2.5 2.5 45.5 2.5 2.5 --4.5 -2.5 -4.5 +25.5 3.5 3.5 +-4.5 -3.5 -4.5 -2.5 -2.5 -2.5 4.5 2.5 2.5 -2.5 2.5 2.5 --4.5 2.5 -4.5 -4.5 -2.5 -2.5 --2.5 2.5 -2.5 +2.5 3.5 2.5 +4.5 -3.5 -3.5 2.5 -2.5 -2.5 --45.5 -2.5 -45.5 +-4.5 2.5 -4.5 +-2.5 3.5 -2.5 +-45.5 -3.5 -45.5 -25.5 -2.5 -25.5 45.5 2.5 2.5 -25.5 2.5 2.5 --45.5 2.5 -45.5 --25.5 2.5 -25.499999999999996 -45.5 -2.5 -2.5 +25.5 3.5 3.5 +45.5 -3.5 -3.5 25.5 -2.5 -2.5 +-45.5 2.5 -45.5 +-25.5 3.5 -25.499999999999996 greatest --4.5 2.5 2.5 --2.5 2.5 2.5 +4.5 -3.5 4.5 2.5 -2.5 2.5 -4.5 -2.5 4.5 --4.5 -2.5 -2.5 +-4.5 2.5 2.5 +-2.5 3.5 3.5 +-4.5 -3.5 -3.5 -2.5 -2.5 -2.5 -2.5 2.5 2.5 4.5 2.5 4.5 --45.5 2.5 2.5 --25.5 2.5 2.5 +2.5 3.5 3.5 +45.5 -3.5 45.5 25.5 -2.5 25.5 -45.5 -2.5 45.5 --45.5 -2.5 -2.5 --25.5 -2.5 -2.5 -25.5 2.5 25.499999999999996 -45.5 2.5 45.5 --2.5 -2.5 -2.5 --4.5 -2.5 -2.5 -2.5 2.5 2.5 -4.5 2.5 4.5 --4.5 2.5 2.5 --2.5 2.5 2.5 -2.5 -2.5 2.5 -4.5 -2.5 4.5 --45.5 -2.5 -2.5 --25.5 -2.5 -2.5 -25.5 2.5 25.5 -45.5 2.5 45.5 -45.5 2.5 2.5 --25.5 2.5 2.5 +-25.5 3.5 3.5 +-45.5 -3.5 -3.5 +-25.5 -2.5 -2.5 +45.5 2.5 45.5 +25.5 3.5 25.499999999999996 +-4.5 -3.5 -3.5 +-2.5 -2.5 -2.5 +4.5 2.5 4.5 +2.5 3.5 3.5 +4.5 -3.5 4.5 +2.5 -2.5 2.5 +-4.5 2.5 2.5 +-2.5 3.5 3.5 +-45.5 -3.5 -3.5 +-25.5 -2.5 -2.5 +45.5 2.5 45.5 +25.5 3.5 25.5 +45.5 -3.5 45.5 25.5 -2.5 25.499999999999996 -45.5 -2.5 45.5 +-45.5 2.5 2.5 +-25.5 3.5 3.5 diff --git a/tests/queries/0_stateless/02155_binary_op_between_float_and_decimal.sql b/tests/queries/0_stateless/02155_binary_op_between_float_and_decimal.sql index 1a7f1b77b9f..2e8ac32462e 100644 --- a/tests/queries/0_stateless/02155_binary_op_between_float_and_decimal.sql +++ b/tests/queries/0_stateless/02155_binary_op_between_float_and_decimal.sql @@ -16,81 +16,81 @@ SELECT greatest(1.5, 1.0::Decimal32(5)); DROP TABLE IF EXISTS t; CREATE TABLE t(d1 Decimal32(5), d2 Decimal64(10), d3 Decimal128(20), d4 Decimal256(40), f1 Float32, f2 Float64) ENGINE=Memory; -INSERT INTO t values (-4.5, 4.5, -45.5, 45.5, 2.5, -2.5); -INSERT INTO t values (4.5, -4.5, 45.5, -45.5, -2.5, 2.5); -INSERT INTO t values (2.5, -2.5, 25.5, -25.5, -2.5, 2.5); -INSERT INTO t values (-2.5, 2.5, -25.5, 25.5, 2.5, -2.5); +INSERT INTO t values (-4.5, 4.5, -45.5, 45.5, 2.5, -3.5); +INSERT INTO t values (4.5, -4.5, 45.5, -45.5, -3.5, 2.5); +INSERT INTO t values (2.5, -2.5, 25.5, -25.5, -2.5, 3.5); +INSERT INTO t values (-2.5, 2.5, -25.5, 25.5, 3.5, -2.5); SELECT ''; SELECT 'plus'; -SELECT d1, f1, d1 + f1 as plus FROM t ORDER BY plus; -SELECT d2, f1, d2 + f1 as plus FROM t ORDER BY plus; -SELECT d3, f1, d3 + f1 as plus FROM t ORDER BY plus; -SELECT d4, f1, d4 + f1 as plus FROM t ORDER BY plus; +SELECT d1, f1, d1 + f1 FROM t ORDER BY f1; +SELECT d2, f1, d2 + f1 FROM t ORDER BY f1; +SELECT d3, f1, d3 + f1 FROM t ORDER BY f1; +SELECT d4, f1, d4 + f1 FROM t ORDER BY f1; -SELECT d1, f2, d1 + f2 as plus FROM t ORDER BY plus; -SELECT d2, f2, d2 + f2 as plus FROM t ORDER BY plus; -SELECT d3, f2, d3 + f2 as plus FROM t ORDER BY plus; -SELECT d4, f2, d4 + f2 as plus FROM t ORDER BY plus; +SELECT d1, f2, d1 + f2 FROM t ORDER BY f2; +SELECT d2, f2, d2 + f2 FROM t ORDER BY f2; +SELECT d3, f2, d3 + f2 FROM t ORDER BY f2; +SELECT d4, f2, d4 + f2 FROM t ORDER BY f2; SELECT ''; SELECT 'minus'; -SELECT d1, f1, d1 - f1 as minus FROM t ORDER BY minus; -SELECT d2, f1, d2 - f1 as minus FROM t ORDER BY minus; -SELECT d3, f1, d3 - f1 as minus FROM t ORDER BY minus; -SELECT d4, f1, d4 - f1 as minus FROM t ORDER BY minus; +SELECT d1, f1, d1 - f1 FROM t ORDER BY f1; +SELECT d2, f1, d2 - f1 FROM t ORDER BY f1; +SELECT d3, f1, d3 - f1 FROM t ORDER BY f1; +SELECT d4, f1, d4 - f1 FROM t ORDER BY f1; -SELECT d1, f2, d1 - f2 as minus FROM t ORDER BY minus; -SELECT d2, f2, d2 - f2 as minus FROM t ORDER BY minus; -SELECT d3, f2, d3 - f2 as minus FROM t ORDER BY minus; -SELECT d4, f2, d4 - f2 as minus FROM t ORDER BY minus; +SELECT d1, f2, d1 - f2 FROM t ORDER BY f2; +SELECT d2, f2, d2 - f2 FROM t ORDER BY f2; +SELECT d3, f2, d3 - f2 FROM t ORDER BY f2; +SELECT d4, f2, d4 - f2 FROM t ORDER BY f2; SELECT ''; SELECT 'multiply'; -SELECT d1, f1, d1 * f1 as multiply FROM t ORDER BY multiply; -SELECT d2, f1, d2 * f1 as multiply FROM t ORDER BY multiply; -SELECT d3, f1, d3 * f1 as multiply FROM t ORDER BY multiply; -SELECT d4, f1, d4 * f1 as multiply FROM t ORDER BY multiply; +SELECT d1, f1, d1 * f1 FROM t ORDER BY f1; +SELECT d2, f1, d2 * f1 FROM t ORDER BY f1; +SELECT d3, f1, d3 * f1 FROM t ORDER BY f1; +SELECT d4, f1, d4 * f1 FROM t ORDER BY f1; -SELECT d1, f2, d1 * f2 as multiply FROM t ORDER BY multiply; -SELECT d2, f2, d2 * f2 as multiply FROM t ORDER BY multiply; -SELECT d3, f2, d3 * f2 as multiply FROM t ORDER BY multiply; -SELECT d4, f2, d4 * f2 as multiply FROM t ORDER BY multiply; +SELECT d1, f2, d1 * f2 FROM t ORDER BY f2; +SELECT d2, f2, d2 * f2 FROM t ORDER BY f2; +SELECT d3, f2, d3 * f2 FROM t ORDER BY f2; +SELECT d4, f2, d4 * f2 FROM t ORDER BY f2; SELECT ''; SELECT 'division'; -SELECT d1, f1, d1 / f1 as division FROM t ORDER BY division; -SELECT d2, f1, d2 / f1 as division FROM t ORDER BY division; -SELECT d3, f1, d3 / f1 as division FROM t ORDER BY division; -SELECT d4, f1, d4 / f1 as division FROM t ORDER BY division; +SELECT d1, f1, d1 / f1 FROM t ORDER BY f1; +SELECT d2, f1, d2 / f1 FROM t ORDER BY f1; +SELECT d3, f1, d3 / f1 FROM t ORDER BY f1; +SELECT d4, f1, d4 / f1 FROM t ORDER BY f1; -SELECT d1, f2, d1 / f2 as division FROM t ORDER BY division; -SELECT d2, f2, d2 / f2 as division FROM t ORDER BY division; -SELECT d3, f2, d3 / f2 as division FROM t ORDER BY division; -SELECT d4, f2, d4 / f2 as division FROM t ORDER BY division; +SELECT d1, f2, d1 / f2 FROM t ORDER BY f2; +SELECT d2, f2, d2 / f2 FROM t ORDER BY f2; +SELECT d3, f2, d3 / f2 FROM t ORDER BY f2; +SELECT d4, f2, d4 / f2 FROM t ORDER BY f2; SELECT ''; SELECT 'least'; -SELECT d1, f1, least(d1, f1) as least FROM t ORDER BY least; -SELECT d2, f1, least(d2, f1) as least FROM t ORDER BY least; -SELECT d3, f1, least(d3, f1) as least FROM t ORDER BY least; -SELECT d4, f1, least(d4, f1) as least FROM t ORDER BY least; +SELECT d1, f1, least(d1, f1) FROM t ORDER BY f1; +SELECT d2, f1, least(d2, f1) FROM t ORDER BY f1; +SELECT d3, f1, least(d3, f1) FROM t ORDER BY f1; +SELECT d4, f1, least(d4, f1) FROM t ORDER BY f1; -SELECT d1, f2, least(d1, f2) as least FROM t ORDER BY least; -SELECT d2, f2, least(d2, f2) as least FROM t ORDER BY least; -SELECT d3, f2, least(d3, f2) as least FROM t ORDER BY least; -SELECT d4, f2, least(d4, f2) as least FROM t ORDER BY least; +SELECT d1, f2, least(d1, f2) FROM t ORDER BY f2; +SELECT d2, f2, least(d2, f2) FROM t ORDER BY f2; +SELECT d3, f2, least(d3, f2) FROM t ORDER BY f2; +SELECT d4, f2, least(d4, f2) FROM t ORDER BY f2; SELECT ''; SELECT 'greatest'; -SELECT d1, f1, greatest(d1, f1) as greatest FROM t ORDER BY greatest; -SELECT d2, f1, greatest(d2, f1) as greatest FROM t ORDER BY greatest; -SELECT d3, f1, greatest(d3, f1) as greatest FROM t ORDER BY greatest; -SELECT d4, f1, greatest(d4, f1) as greatest FROM t ORDER BY greatest; +SELECT d1, f1, greatest(d1, f1) FROM t ORDER BY f1; +SELECT d2, f1, greatest(d2, f1) FROM t ORDER BY f1; +SELECT d3, f1, greatest(d3, f1) FROM t ORDER BY f1; +SELECT d4, f1, greatest(d4, f1) FROM t ORDER BY f1; -SELECT d1, f2, greatest(d1, f2) as greatest FROM t ORDER BY greatest; -SELECT d2, f2, greatest(d2, f2) as greatest FROM t ORDER BY greatest; -SELECT d3, f2, greatest(d3, f2) as greatest FROM t ORDER BY greatest; -SELECT d4, f2, greatest(d4, f2) as greatest FROM t ORDER BY greatest; +SELECT d1, f2, greatest(d1, f2) FROM t ORDER BY f2; +SELECT d2, f2, greatest(d2, f2) FROM t ORDER BY f2; +SELECT d3, f2, greatest(d3, f2) FROM t ORDER BY f2; +SELECT d4, f2, greatest(d4, f2) FROM t ORDER BY f2; DROP TABLE t; From a7d0f3e637831a5db69d7e284f9eb5d4e5c3bf3e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 2 Jan 2022 16:21:52 +0300 Subject: [PATCH 0667/1260] UserDefinedExecutableFunction fix exception --- src/Interpreters/UserDefinedExecutableFunctionFactory.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp index 0cffd61eaf6..10cb806028e 100644 --- a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp +++ b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp @@ -98,7 +98,7 @@ public: column_with_type.column = castColumnAccurate(column_to_cast, argument_type); column_with_type.type = argument_type; - column_with_type = column_to_cast; + column_with_type = std::move(column_to_cast); } ColumnWithTypeAndName result(result_type, "result"); @@ -110,7 +110,7 @@ public: ShellCommandSourceConfiguration shell_command_source_configuration; - if (coordinator->getConfiguration().is_executable_pool) + if (coordinator_configuration.is_executable_pool) { shell_command_source_configuration.read_fixed_number_of_rows = true; shell_command_source_configuration.number_of_rows_to_read = input_rows_count; @@ -143,8 +143,8 @@ public: size_t result_column_size = result_column->size(); if (result_column_size != input_rows_count) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Function {} wrong result rows count expected {} actual {}", - getName(), + "Function {}: wrong result, expected {} row(s), actual {}", + quoteString(getName()), input_rows_count, result_column_size); From 4f62155b84ee733875ba1f263d05ee17ba86bd9b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 2 Jan 2022 16:46:11 +0300 Subject: [PATCH 0668/1260] UserDefinedExecutableFunction fix documentation --- docs/en/sql-reference/functions/index.md | 62 +++++++++++++++++++++--- 1 file changed, 54 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/functions/index.md b/docs/en/sql-reference/functions/index.md index 037078ba223..0cdfc86fae3 100644 --- a/docs/en/sql-reference/functions/index.md +++ b/docs/en/sql-reference/functions/index.md @@ -84,18 +84,63 @@ A function configuration contains the following settings: - `command_read_timeout` - timeout for writing data to command stdin in milliseconds. Default value 10000. Optional parameter. - `pool_size` - the size of a command pool. Optional. Default value is `16`. - `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`. -- `execute_direct` - Executable source file will be searched inside `user_scripts` folder and executed directly. Additional arguments can be specified. Example: `test_script arg_1 arg_2`. Default value is true. Optional parameter. +- `execute_direct` - If `execute_direct` = `1`, then `command` will be searched inside user_scripts folder. Additional script arguments can be specified using whitespace separator. Example: `script_name arg1 arg2`. If `execute_direct` = `0`, `command` is passed as argument for `bin/sh -c`. Default value is `1`. Optional parameter. - `lifetime` - the reload interval of a function in seconds. If it is set to `0` then the function is not reloaded. Default value is `0`. Optional parameter. The command must read arguments from `STDIN` and must output the result to `STDOUT`. The command must process arguments iteratively. That is after processing a chunk of arguments it must wait for the next chunk. **Example** -Creating `test_function` using XML configuration: -``` +Creating `test_function` using XML configuration. +File test_function.xml. +```xml executable - test_function + test_function_python + String + + UInt64 + + TabSeparated + test_function.py + + +``` + +Script file inside `user_scripts` folder `test_function.py`. + +```python +#!/usr/bin/python3 + +import sys + +if __name__ == '__main__': + for line in sys.stdin: + print("Value " + line, end='') + sys.stdout.flush() +``` + +Query: + +``` sql +SELECT test_function_python(toUInt64(2)); +``` + +Result: + +``` text +┌─test_function_python(2)─┐ +│ Value 2 │ +└─────────────────────────┘ +``` + +Creating `test_function_sum` manually specifying `execute_direct` to `0` using XML configuration. +File test_function.xml. +```xml + + + executable + test_function_sum UInt64 UInt64 @@ -105,6 +150,7 @@ Creating `test_function` using XML configuration: TabSeparated cd /; clickhouse-local --input-format TabSeparated --output-format TabSeparated --structure 'x UInt64, y UInt64' --query "SELECT x + y FROM table" + 0 ``` @@ -112,15 +158,15 @@ Creating `test_function` using XML configuration: Query: ``` sql -SELECT test_function(toUInt64(2), toUInt64(2)); +SELECT test_function_sum(2, 2); ``` Result: ``` text -┌─test_function(toUInt64(2), toUInt64(2))─┐ -│ 4 │ -└─────────────────────────────────────────┘ +┌─test_function_sum(2, 2)─┐ +│ 4 │ +└─────────────────────────┘ ``` From 8e83f2d154f645242e928e3533d0cf66e7084a5b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 2 Jan 2022 20:02:53 +0300 Subject: [PATCH 0669/1260] Updated documentation --- .../external-dictionaries/external-dicts-dict-sources.md | 4 ++-- docs/en/sql-reference/functions/index.md | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index e606a19af6f..51aa6be1e38 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -126,7 +126,7 @@ Setting fields: - `command_read_timeout` - timeout for reading data from command stdout in milliseconds. Default value 10000. Optional parameter. - `command_read_timeout` - timeout for writing data to command stdin in milliseconds. Default value 10000. Optional parameter. - `implicit_key` — The executable source file can return only values, and the correspondence to the requested keys is determined implicitly — by the order of rows in the result. Default value is false. -- `execute_direct` - Executable source file will be searched inside `user_scripts` folder and executed directly. Arguments are passed using spaces. Example: `test_script arg_1 arg_2`. Default value is false. Optional parameter. +- `execute_direct` - If `execute_direct` = `1`, then `command` will be searched inside user_scripts folder. Additional script arguments can be specified using whitespace separator. Example: `script_name arg1 arg2`. If `execute_direct` = `0`, `command` is passed as argument for `bin/sh -c`. Default value is `0`. Optional parameter. - `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`. That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled, otherwise, the DB user would be able to execute arbitrary binary on ClickHouse node. @@ -161,7 +161,7 @@ Setting fields: - `command_read_timeout` - timeout for reading data from command stdout in milliseconds. Default value 10000. Optional parameter. - `command_read_timeout` - timeout for writing data to command stdin in milliseconds. Default value 10000. Optional parameter. - `implicit_key` — The executable source file can return only values, and the correspondence to the requested keys is determined implicitly — by the order of rows in the result. Default value is false. Optional parameter. -- `execute_direct` - Executable source file will be searched inside `user_scripts` folder and executed directly. Additional arguments can be specified. Example: `test_script arg_1 arg_2`. Default value is false. Optional parameter. +- `execute_direct` - If `execute_direct` = `1`, then `command` will be searched inside user_scripts folder. Additional script arguments can be specified using whitespace separator. Example: `script_name arg1 arg2`. If `execute_direct` = `0`, `command` is passed as argument for `bin/sh -c`. Default value is `1`. Optional parameter. - `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`. That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled, otherwise, the DB user would be able to execute arbitrary binary on ClickHouse node. diff --git a/docs/en/sql-reference/functions/index.md b/docs/en/sql-reference/functions/index.md index 0cdfc86fae3..23b45c4fffc 100644 --- a/docs/en/sql-reference/functions/index.md +++ b/docs/en/sql-reference/functions/index.md @@ -84,7 +84,7 @@ A function configuration contains the following settings: - `command_read_timeout` - timeout for writing data to command stdin in milliseconds. Default value 10000. Optional parameter. - `pool_size` - the size of a command pool. Optional. Default value is `16`. - `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`. -- `execute_direct` - If `execute_direct` = `1`, then `command` will be searched inside user_scripts folder. Additional script arguments can be specified using whitespace separator. Example: `script_name arg1 arg2`. If `execute_direct` = `0`, `command` is passed as argument for `bin/sh -c`. Default value is `1`. Optional parameter. +- `execute_direct` - If `execute_direct` = `1`, then `command` will be searched inside user_scripts folder. Additional script arguments can be specified using whitespace separator. Example: `script_name arg1 arg2`. If `execute_direct` = `0`, `command` is passed as argument for `bin/sh -c`. Default value is `1`. Optional parameter. - `lifetime` - the reload interval of a function in seconds. If it is set to `0` then the function is not reloaded. Default value is `0`. Optional parameter. The command must read arguments from `STDIN` and must output the result to `STDOUT`. The command must process arguments iteratively. That is after processing a chunk of arguments it must wait for the next chunk. From 6de989f5fa6ec77d768c0002bcfc1b93bc871cde Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 30 Dec 2021 23:53:33 +0300 Subject: [PATCH 0670/1260] Allow negative intervals in function intervalLengthSum --- .../AggregateFunctionIntervalLengthSum.h | 21 +++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h b/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h index 5969fca9dcf..5a82d9e6a5c 100644 --- a/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h +++ b/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h @@ -7,6 +7,8 @@ #include #include +#include + #include #include @@ -15,6 +17,7 @@ #include + namespace DB { @@ -25,7 +28,7 @@ namespace ErrorCodes /** * Calculate total length of intervals without intersections. Each interval is the pair of numbers [begin, end]; - * Return UInt64 for integral types (UInt/Int*, Date/DateTime) and return Float64 for Float*. + * Return Int64 for integral types (UInt/Int*, Date/DateTime) and return Float64 for Float*. * * Implementation simply stores intervals sorted by beginning and sums lengths at final. */ @@ -147,7 +150,17 @@ private: /// Check if current interval intersect with next one then add length, otherwise advance interval end if (cur_segment.second < data.segments[i].first) { - res += cur_segment.second - cur_segment.first; + if constexpr (std::is_floating_point_v) + { + res += cur_segment.second - cur_segment.first; + } + else + { + TResult diff; + if (!common::subOverflow(static_cast(cur_segment.second), static_cast(cur_segment.first), diff)) + res += diff; + } + cur_segment = data.segments[i]; } else @@ -171,7 +184,7 @@ public: { if constexpr (std::is_floating_point_v) return std::make_shared(); - return std::make_shared(); + return std::make_shared(); } bool allocatesMemoryInArena() const override { return false; } @@ -212,7 +225,7 @@ public: if constexpr (std::is_floating_point_v) assert_cast(to).getData().push_back(getIntervalLengthSum(this->data(place))); else - assert_cast(to).getData().push_back(getIntervalLengthSum(this->data(place))); + assert_cast(to).getData().push_back(getIntervalLengthSum(this->data(place))); } }; From c409c8a361b6e411ce4734f979ea79be6f08bc87 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 2 Jan 2022 20:20:12 +0300 Subject: [PATCH 0671/1260] Change my mind --- .../AggregateFunctionIntervalLengthSum.h | 25 +++++++++++-------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h b/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h index 5a82d9e6a5c..93e81be93fc 100644 --- a/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h +++ b/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h @@ -26,12 +26,11 @@ namespace ErrorCodes extern const int TOO_LARGE_ARRAY_SIZE; } -/** - * Calculate total length of intervals without intersections. Each interval is the pair of numbers [begin, end]; - * Return Int64 for integral types (UInt/Int*, Date/DateTime) and return Float64 for Float*. - * - * Implementation simply stores intervals sorted by beginning and sums lengths at final. - */ +/** Calculate total length of intervals without intersections. Each interval is the pair of numbers [begin, end]; + * Returns UInt64 for integral types (UInt/Int*, Date/DateTime) and returns Float64 for Float*. + * + * Implementation simply stores intervals sorted by beginning and sums lengths at final. + */ template struct AggregateFunctionIntervalLengthSumData { @@ -46,10 +45,14 @@ struct AggregateFunctionIntervalLengthSumData void add(T begin, T end) { + /// Reversed intervals are counted by absolute value of their length. + if (unlikely(end < begin)) + std::swap(begin, end); + else if (unlikely(begin == end)) + return; + if (sorted && !segments.empty()) - { sorted = segments.back().first <= begin; - } segments.emplace_back(begin, end); } @@ -147,7 +150,7 @@ private: for (size_t i = 1, sz = data.segments.size(); i < sz; ++i) { - /// Check if current interval intersect with next one then add length, otherwise advance interval end + /// Check if current interval intersects with next one then add length, otherwise advance interval end. if (cur_segment.second < data.segments[i].first) { if constexpr (std::is_floating_point_v) @@ -184,7 +187,7 @@ public: { if constexpr (std::is_floating_point_v) return std::make_shared(); - return std::make_shared(); + return std::make_shared(); } bool allocatesMemoryInArena() const override { return false; } @@ -225,7 +228,7 @@ public: if constexpr (std::is_floating_point_v) assert_cast(to).getData().push_back(getIntervalLengthSum(this->data(place))); else - assert_cast(to).getData().push_back(getIntervalLengthSum(this->data(place))); + assert_cast(to).getData().push_back(getIntervalLengthSum(this->data(place))); } }; From 90f5594f80de2c800c91a67803a7d880086de2f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 2 Jan 2022 20:23:35 +0300 Subject: [PATCH 0672/1260] Add a test --- tests/queries/0_stateless/02158_interval_length_sum.reference | 1 + tests/queries/0_stateless/02158_interval_length_sum.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/02158_interval_length_sum.reference create mode 100644 tests/queries/0_stateless/02158_interval_length_sum.sql diff --git a/tests/queries/0_stateless/02158_interval_length_sum.reference b/tests/queries/0_stateless/02158_interval_length_sum.reference new file mode 100644 index 00000000000..b4de3947675 --- /dev/null +++ b/tests/queries/0_stateless/02158_interval_length_sum.reference @@ -0,0 +1 @@ +11 diff --git a/tests/queries/0_stateless/02158_interval_length_sum.sql b/tests/queries/0_stateless/02158_interval_length_sum.sql new file mode 100644 index 00000000000..af22a707caf --- /dev/null +++ b/tests/queries/0_stateless/02158_interval_length_sum.sql @@ -0,0 +1 @@ +SELECT intervalLengthSum(x, y) FROM values('x Int64, y Int64', (0, 10), (5, 5), (5, 6), (1, -1)); From 4a094c2efd48d9377f037172a37a060eecf3a534 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 2 Jan 2022 21:50:41 +0300 Subject: [PATCH 0673/1260] Merging contingency coefficients --- ...ggregateFunctionContingencyCoefficient.cpp | 64 +++--- .../AggregateFunctionCramersV.cpp | 45 +++- .../AggregateFunctionCramersV.h | 207 ------------------ ...AggregateFunctionCramersVBiasCorrected.cpp | 59 +++++ ...ggregateFunctionCramersVBiasCorrection.cpp | 67 ------ .../AggregateFunctionTheilsU.cpp | 69 +++--- src/AggregateFunctions/CrossTab.h | 134 ++++++++++++ .../registerAggregateFunctions.cpp | 8 +- src/Common/HashTable/HashMap.h | 7 + 9 files changed, 300 insertions(+), 360 deletions(-) delete mode 100644 src/AggregateFunctions/AggregateFunctionCramersV.h create mode 100644 src/AggregateFunctions/AggregateFunctionCramersVBiasCorrected.cpp delete mode 100644 src/AggregateFunctions/AggregateFunctionCramersVBiasCorrection.cpp create mode 100644 src/AggregateFunctions/CrossTab.h diff --git a/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp b/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp index 0391fe3c8ee..4d34c14ede6 100644 --- a/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp +++ b/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp @@ -1,61 +1,53 @@ #include -#include +#include #include #include -#include "registerAggregateFunctions.h" #include +#include -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} namespace DB { + namespace { - -struct ContingencyData : public AggregateFunctionCramersVData +struct ContingencyData : CrossTabData { - Float64 get_result() const + static const char * getName() { - if (cur_size < 2){ - throw Exception("Aggregate function contingency coefficient requires at least 2 values in columns", ErrorCodes::BAD_ARGUMENTS); - } + return "contingency"; + } + + Float64 getResult() const + { + if (count < 2) + return std::numeric_limits::quiet_NaN(); + Float64 phi = 0.0; - for (const auto & cell : pairs) { - UInt128 hash_pair = cell.getKey(); - UInt64 count_of_pair_tmp = cell.getMapped(); - Float64 count_of_pair = Float64(count_of_pair_tmp); - UInt64 hash1 = (hash_pair << 64 >> 64); - UInt64 hash2 = (hash_pair >> 64); + for (const auto & [key, value_ab] : count_ab) + { + Float64 value_a = count_a.at(key.items[0]); + Float64 value_b = count_b.at(key.items[1]); - UInt64 count1_tmp = n_i.find(hash1)->getMapped(); - UInt64 count2_tmp = n_j.find(hash2)->getMapped(); - Float64 count1 = static_cast(count1_tmp); - Float64 count2 = Float64(count2_tmp); - - phi += ((count_of_pair * count_of_pair / (count1 * count2) * cur_size) - - 2 * count_of_pair + (count1 * count2 / cur_size)); + phi += value_ab * value_ab / (value_a * value_b) * count - 2 * value_ab + (value_a * value_b) / count; } - phi /= cur_size; - return sqrt(phi / (phi + cur_size)); + phi /= count; + + return sqrt(phi / (phi + count)); } }; - -AggregateFunctionPtr createAggregateFunctionContingencyCoefficient(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) +void registerAggregateFunctionContingency(AggregateFunctionFactory & factory) { - assertNoParameters(name, parameters); - return std::make_shared>(argument_types); + factory.registerFunction(ContingencyData::getName(), + [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) + { + assertNoParameters(name, parameters); + return std::make_shared>(argument_types); + }); } } -void registerAggregateFunctionContingencyCoefficient(AggregateFunctionFactory & factory) -{ - factory.registerFunction("ContingencyCoefficient", createAggregateFunctionContingencyCoefficient); -} - } diff --git a/src/AggregateFunctions/AggregateFunctionCramersV.cpp b/src/AggregateFunctions/AggregateFunctionCramersV.cpp index b04c6a37056..850070f26b2 100644 --- a/src/AggregateFunctions/AggregateFunctionCramersV.cpp +++ b/src/AggregateFunctions/AggregateFunctionCramersV.cpp @@ -1,27 +1,56 @@ #include -#include +#include #include #include -#include "registerAggregateFunctions.h" #include +#include + namespace DB { + namespace { -AggregateFunctionPtr createAggregateFunctionCramersV(const std::string & name, const DataTypes & argument_types, - const Array & parameters, const Settings *) +struct CramersVData : CrossTabData { - assertNoParameters(name, parameters); - return std::make_shared>(argument_types); -} + static const char * getName() + { + return "cramersV"; + } + + Float64 getResult() const + { + if (count < 2) + return std::numeric_limits::quiet_NaN(); + + Float64 phi = 0.0; + for (const auto & [key, value_ab] : count_ab) + { + Float64 value_a = count_a.at(key.items[0]); + Float64 value_b = count_b.at(key.items[1]); + + phi += value_ab * value_ab / (value_a * value_b) * count - 2 * value_ab + (value_a * value_b) / count; + } + + phi /= count; + UInt64 q = std::min(count_a.size(), count_b.size()); + phi /= q - 1; + + return sqrt(phi); + } +}; } void registerAggregateFunctionCramersV(AggregateFunctionFactory & factory) { - factory.registerFunction("CramersV", createAggregateFunctionCramersV); + factory.registerFunction(CramersVData::getName(), + [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) + { + assertNoParameters(name, parameters); + return std::make_shared>(argument_types); + }); } } diff --git a/src/AggregateFunctions/AggregateFunctionCramersV.h b/src/AggregateFunctions/AggregateFunctionCramersV.h deleted file mode 100644 index 383647f8aa9..00000000000 --- a/src/AggregateFunctions/AggregateFunctionCramersV.h +++ /dev/null @@ -1,207 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - - -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - -namespace DB -{ - - - struct AggregateFunctionCramersVData - { - size_t cur_size = 0; - HashMap n_i; - HashMap n_j; - HashMap pairs; - - - void add(UInt64 hash1, UInt64 hash2) - { - cur_size += 1; - n_i[hash1] += 1; - n_j[hash2] += 1; - - UInt128 hash_pair = hash1 | (static_cast(hash2) << 64); - pairs[hash_pair] += 1; - - } - - void merge(const AggregateFunctionCramersVData &other) - { - cur_size += other.cur_size; - - for (const auto& pair : other.n_i) { - UInt64 hash1 = pair.getKey(); - UInt64 count = pair.getMapped(); - n_i[hash1] += count; - } - for (const auto& pair : other.n_j) { - UInt64 hash1 = pair.getKey(); - UInt64 count = pair.getMapped(); - n_j[hash1] += count; - } - for (const auto& pair : other.pairs) { - UInt128 hash1 = pair.getKey(); - UInt64 count = pair.getMapped(); - pairs[hash1] += count; - } - } - - void serialize(WriteBuffer &buf) const - { - writeBinary(cur_size, buf); - n_i.write(buf); - n_j.write(buf); - pairs.write(buf); - } - - void deserialize(ReadBuffer &buf) - { - readBinary(cur_size, buf); - n_i.read(buf); - n_j.read(buf); - pairs.read(buf); - } - - Float64 get_result() const - { - if (cur_size < 2){ - throw Exception("Aggregate function cramer's v requires et least 2 values in columns", ErrorCodes::BAD_ARGUMENTS); - } - Float64 phi = 0.0; - for (const auto & cell : pairs) { - UInt128 hash_pair = cell.getKey(); - UInt64 count_of_pair_tmp = cell.getMapped(); - Float64 count_of_pair = Float64(count_of_pair_tmp); - UInt64 hash1 = (hash_pair << 64 >> 64); - UInt64 hash2 = (hash_pair >> 64); - - UInt64 count1_tmp = n_i.find(hash1)->getMapped(); - UInt64 count2_tmp = n_j.find(hash2)->getMapped(); - Float64 count1 = static_cast(count1_tmp); - Float64 count2 = Float64(count2_tmp); - - phi += ((count_of_pair * count_of_pair / (count1 * count2) * cur_size) - - 2 * count_of_pair + (count1 * count2 / cur_size)); - } - phi /= cur_size; - - UInt64 q = std::min(n_i.size(), n_j.size()); - phi /= (q - 1); - return sqrt(phi); - - } - }; - - template - - class AggregateFunctionCramersV : public - IAggregateFunctionDataHelper< - Data, - AggregateFunctionCramersV - > - { - - public: - AggregateFunctionCramersV( - const DataTypes & arguments - ): - IAggregateFunctionDataHelper< - Data, - AggregateFunctionCramersV - > ({arguments}, {}) - { - // notice: arguments has been in factory - } - - String getName() const override - { - return "CramersV"; - } - - bool allocatesMemoryInArena() const override { return false; } - - DataTypePtr getReturnType() const override - { - return std::make_shared>(); - } - - void add( - AggregateDataPtr __restrict place, - const IColumn ** columns, - size_t row_num, - Arena * - ) const override - { - UInt64 hash1 = UniqVariadicHash::apply(1, columns, row_num); - UInt64 hash2 = UniqVariadicHash::apply(1, columns + 1, row_num); - - this->data(place).add(hash1, hash2); - } - - void merge( - AggregateDataPtr __restrict place, - ConstAggregateDataPtr rhs, Arena * - ) const override - { - this->data(place).merge(this->data(rhs)); - } - - void serialize( - ConstAggregateDataPtr __restrict place, - WriteBuffer & buf - ) const override - { - this->data(place).serialize(buf); - } - - void deserialize( - AggregateDataPtr __restrict place, - ReadBuffer & buf, Arena * - ) const override - { - this->data(place).deserialize(buf); - } - - void insertResultInto( - AggregateDataPtr __restrict place, - IColumn & to, - Arena * - ) const override - { - Float64 result = this->data(place).get_result(); -// std::cerr << "cur_size" << this->data(place).cur_size << '\n'; -// std::cerr << "n_i size" << this->data(place).n_i.size() << '\n'; -// std::cerr << "n_j size" << this->data(place).n_j.size() << '\n'; -// std::cerr << "pair size " << this->data(place).pairs.size() << '\n'; -// std::cerr << "result " << result << '\n'; - - auto & column = static_cast &>(to); - column.getData().push_back(result); - } - - }; - -} diff --git a/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrected.cpp b/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrected.cpp new file mode 100644 index 00000000000..48a3029c399 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrected.cpp @@ -0,0 +1,59 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace +{ + +struct CramersVBiasCorrectedData : CrossTabData +{ + static const char * getName() + { + return "cramersVBiasCorrected"; + } + + Float64 getResult() const + { + if (count < 2) + return std::numeric_limits::quiet_NaN(); + + Float64 phi = 0.0; + for (const auto & [key, value_ab] : count_ab) + { + Float64 value_a = count_a.at(key.items[0]); + Float64 value_b = count_b.at(key.items[1]); + + phi += value_ab * value_ab / (value_a * value_b) * count - 2 * value_ab + (value_a * value_b) / count; + } + + phi /= count; + + Float64 res = std::max(0.0, phi - (static_cast(count_a.size()) - 1) * (static_cast(count_b.size()) - 1) / (count - 1)); + Float64 correction_a = count_a.size() - (static_cast(count_a.size()) - 1) * (static_cast(count_a.size()) - 1) / (count - 1); + Float64 correction_b = count_b.size() - (static_cast(count_b.size()) - 1) * (static_cast(count_b.size()) - 1) / (count - 1); + res /= std::min(correction_a, correction_b) - 1; + + return sqrt(res); + } +}; + +} + +void registerAggregateFunctionCramersVBiasCorrected(AggregateFunctionFactory & factory) +{ + factory.registerFunction(CramersVBiasCorrectedData::getName(), + [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) + { + assertNoParameters(name, parameters); + return std::make_shared>(argument_types); + }); +} + +} diff --git a/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrection.cpp b/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrection.cpp deleted file mode 100644 index c58ca8a59da..00000000000 --- a/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrection.cpp +++ /dev/null @@ -1,67 +0,0 @@ -#include -#include -#include -#include -#include "registerAggregateFunctions.h" -#include - - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -namespace DB -{ -namespace -{ - - -struct BiasCorrectionData : public AggregateFunctionCramersVData -{ - Float64 get_result() const - { - if (cur_size < 2){ - throw Exception("Aggregate function cramer's v bias corrected at least 2 values in columns", ErrorCodes::BAD_ARGUMENTS); - } - Float64 phi = 0.0; - for (const auto & cell : pairs) { - UInt128 hash_pair = cell.getKey(); - UInt64 count_of_pair_tmp = cell.getMapped(); - Float64 count_of_pair = Float64(count_of_pair_tmp); - UInt64 hash1 = (hash_pair << 64 >> 64); - UInt64 hash2 = (hash_pair >> 64); - - UInt64 count1_tmp = n_i.find(hash1)->getMapped(); - UInt64 count2_tmp = n_j.find(hash2)->getMapped(); - Float64 count1 = static_cast(count1_tmp); - Float64 count2 = Float64(count2_tmp); - - phi += ((count_of_pair * count_of_pair / (count1 * count2) * cur_size) - - 2 * count_of_pair + (count1 * count2 / cur_size)); - } - phi /= cur_size; - Float64 answ = std::max(0.0, phi - ((static_cast(n_i.size()) - 1) * (static_cast(n_j.size()) - 1) / (cur_size - 1))); - Float64 k = n_i.size() - (static_cast(n_i.size()) - 1) * (static_cast(n_i.size()) - 1) / (cur_size - 1); - Float64 r = n_j.size() - (static_cast(n_j.size()) - 1) * (static_cast(n_j.size()) - 1) / (cur_size - 1); - Float64 q = std::min(k, r); - answ /= (q - 1); - return sqrt(answ); - } -}; - - -AggregateFunctionPtr createAggregateFunctionCramersVBiasCorrection(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) -{ - assertNoParameters(name, parameters); - return std::make_shared>(argument_types); -} - -} - -void registerAggregateFunctionCramersVBiasCorrection(AggregateFunctionFactory & factory) -{ - factory.registerFunction("CramersVBiasCorrection", createAggregateFunctionCramersVBiasCorrection); -} - -} diff --git a/src/AggregateFunctions/AggregateFunctionTheilsU.cpp b/src/AggregateFunctions/AggregateFunctionTheilsU.cpp index b2eeff3d7c9..3868e57ad6b 100644 --- a/src/AggregateFunctions/AggregateFunctionTheilsU.cpp +++ b/src/AggregateFunctions/AggregateFunctionTheilsU.cpp @@ -1,48 +1,43 @@ #include -#include +#include #include #include -#include "registerAggregateFunctions.h" #include +#include -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} namespace DB { + namespace { - - struct TheilsUData : public AggregateFunctionCramersVData +struct TheilsUData : CrossTabData { - Float64 get_result() const + static const char * getName() { - if (cur_size < 2){ - throw Exception("Aggregate function theil's u requires at least 2 values in columns", ErrorCodes::BAD_ARGUMENTS); - } - Float64 h_x = 0.0; - for (const auto & cell : n_i) { - UInt64 count_x_tmp = cell.getMapped(); - Float64 count_x = Float64(count_x_tmp); - h_x += (count_x / cur_size) * (log(count_x / cur_size)); - } + return "theilsU"; + } + Float64 getResult() const + { + if (count < 2) + return std::numeric_limits::quiet_NaN(); + + Float64 h_a = 0.0; + for (const auto & [key, value] : count_a) + { + Float64 value_float = value; + h_a += (value_float / count) * log(value_float / count); + } Float64 dep = 0.0; - for (const auto & cell : pairs) { - UInt128 hash_pair = cell.getKey(); - UInt64 count_of_pair_tmp = cell.getMapped(); - Float64 count_of_pair = Float64(count_of_pair_tmp); + for (const auto & [key, value] : count_ab) + { + Float64 value_ab = value; + Float64 value_b = count_b.at(key.items[1]); - UInt64 hash2 = (hash_pair >> 64); - - UInt64 count2_tmp = n_j.find(hash2)->getMapped(); - Float64 count2 = Float64 (count2_tmp); - - dep += (count_of_pair / cur_size) * log(count_of_pair / count2); + dep += (value_ab / count) * log(value_ab / value_b); } dep -= h_x; @@ -51,18 +46,16 @@ namespace } }; - -AggregateFunctionPtr createAggregateFunctionTheilsU(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) -{ - assertNoParameters(name, parameters); - return std::make_shared>(argument_types); -} - -} - void registerAggregateFunctionTheilsU(AggregateFunctionFactory & factory) { - factory.registerFunction("TheilsU", createAggregateFunctionTheilsU); + factory.registerFunction(TheilsUData::getName(), + [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) + { + assertNoParameters(name, parameters); + return std::make_shared>(argument_types); + }); +} + } } diff --git a/src/AggregateFunctions/CrossTab.h b/src/AggregateFunctions/CrossTab.h new file mode 100644 index 00000000000..4fc6dc2c21b --- /dev/null +++ b/src/AggregateFunctions/CrossTab.h @@ -0,0 +1,134 @@ +#pragma once + +#include +#include +#include +#include +#include + + +/** Aggregate function that calculates statistics on top of cross-tab: + * - histogram of every argument and every pair of elements. + * These statistics include: + * - Cramer's V; + * - Theil's U; + * - contingency coefficient; + * It can be interpreted as interdependency coefficient between arguments; + * or non-parametric correlation coefficient. + */ +namespace DB +{ + +struct CrossTabData +{ + /// Total count. + UInt64 count = 0; + + /// Count of every value of the first and second argument (values are pre-hashed). + /// Note: non-cryptographic 64bit hash is used, it means that the calculation is approximate. + HashMapWithStackMemory count_a; + HashMapWithStackMemory count_b; + + /// Count of every pair of values. We pack two hashes into UInt128. + HashMapWithStackMemory count_ab; + + + void add(UInt64 hash1, UInt64 hash2) + { + ++count; + ++count_a[hash1]; + ++count_b[hash2]; + + UInt128 hash_pair{hash1, hash2}; + ++count_ab[hash_pair]; + } + + void merge(const CrossTabData & other) + { + count += other.count; + for (const auto & [key, value] : other.count_a) + count_a[key] += value; + for (const auto & [key, value] : other.count_b) + count_b[key] += value; + for (const auto & [key, value] : other.count_ab) + count_ab[key] += value; + } + + void serialize(WriteBuffer &buf) const + { + writeBinary(count, buf); + count_a.write(buf); + count_b.write(buf); + count_ab.write(buf); + } + + void deserialize(ReadBuffer &buf) + { + readBinary(count, buf); + count_a.read(buf); + count_b.read(buf); + count_ab.read(buf); + } +}; + + +template +class AggregateFunctionCrossTab : public IAggregateFunctionDataHelper> +{ +public: + AggregateFunctionCrossTab(const DataTypes & arguments) + : IAggregateFunctionDataHelper>({arguments}, {}) + { + } + + String getName() const override + { + return Data::getName(); + } + + bool allocatesMemoryInArena() const override + { + return false; + } + + DataTypePtr getReturnType() const override + { + return std::make_shared>(); + } + + void add( + AggregateDataPtr __restrict place, + const IColumn ** columns, + size_t row_num, + Arena *) const override + { + UInt64 hash1 = UniqVariadicHash::apply(1, &columns[0], row_num); + UInt64 hash2 = UniqVariadicHash::apply(1, &columns[1], row_num); + + this->data(place).add(hash1, hash2); + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override + { + this->data(place).merge(this->data(rhs)); + } + + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + { + this->data(place).serialize(buf); + } + + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + { + this->data(place).deserialize(buf); + } + + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override + { + Float64 result = this->data(place).getResult(); + auto & column = static_cast &>(to); + column.getData().push_back(result); + } +}; + +} diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index b10f3832e21..c9e46329735 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -50,8 +50,8 @@ void registerAggregateFunctionWelchTTest(AggregateFunctionFactory &); void registerAggregateFunctionStudentTTest(AggregateFunctionFactory &); void registerAggregateFunctionCramersV(AggregateFunctionFactory &); void registerAggregateFunctionTheilsU(AggregateFunctionFactory &); -void registerAggregateFunctionContingencyCoefficient(AggregateFunctionFactory &); -void registerAggregateFunctionCramersVBiasCorrection(AggregateFunctionFactory &); +void registerAggregateFunctionContingency(AggregateFunctionFactory &); +void registerAggregateFunctionCramersVBiasCorrected(AggregateFunctionFactory &); void registerAggregateFunctionSingleValueOrNull(AggregateFunctionFactory &); void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory &); void registerAggregateFunctionExponentialMovingAverage(AggregateFunctionFactory &); @@ -105,8 +105,8 @@ void registerAggregateFunctions() registerAggregateFunctionsBitwise(factory); registerAggregateFunctionCramersV(factory); registerAggregateFunctionTheilsU(factory); - registerAggregateFunctionContingencyCoefficient(factory); - registerAggregateFunctionCramersVBiasCorrection(factory); + registerAggregateFunctionContingency(factory); + registerAggregateFunctionCramersVBiasCorrected(factory); registerAggregateFunctionsBitmap(factory); registerAggregateFunctionsMaxIntersections(factory); registerAggregateFunctionHistogram(factory); diff --git a/src/Common/HashTable/HashMap.h b/src/Common/HashTable/HashMap.h index c5675d4d7c9..e619421b8f7 100644 --- a/src/Common/HashTable/HashMap.h +++ b/src/Common/HashTable/HashMap.h @@ -262,6 +262,13 @@ public: return it->getMapped(); } + + typename Cell::Mapped & ALWAYS_INLINE at(const Key & x) const + { + if (auto it = this->find(x); it != this->end()) + return it->getMapped(); + throw DB::Exception("Cannot find element in HashMap::at method", DB::ErrorCodes::LOGICAL_ERROR); + } }; namespace std From cf66716cd64374beee5e6d4ebeba5867f5b64eca Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 2 Jan 2022 22:42:56 +0300 Subject: [PATCH 0674/1260] Make it less wrong --- ...ggregateFunctionContingencyCoefficient.cpp | 41 ++++++++----------- .../AggregateFunctionCramersV.cpp | 1 - ...AggregateFunctionCramersVBiasCorrected.cpp | 1 - .../AggregateFunctionTheilsU.cpp | 9 ++-- src/AggregateFunctions/CrossTab.h | 4 +- src/Common/HashTable/HashMap.h | 2 +- 6 files changed, 25 insertions(+), 33 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp b/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp index 4d34c14ede6..f5aeb68748f 100644 --- a/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp +++ b/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include @@ -12,31 +11,29 @@ namespace DB namespace { -struct ContingencyData : CrossTabData -{ - static const char * getName() + struct ContingencyData : CrossTabData { - return "contingency"; - } + static const char * getName() { return "contingency"; } - Float64 getResult() const - { - if (count < 2) - return std::numeric_limits::quiet_NaN(); - - Float64 phi = 0.0; - for (const auto & [key, value_ab] : count_ab) + Float64 getResult() const { - Float64 value_a = count_a.at(key.items[0]); - Float64 value_b = count_b.at(key.items[1]); + if (count < 2) + return std::numeric_limits::quiet_NaN(); - phi += value_ab * value_ab / (value_a * value_b) * count - 2 * value_ab + (value_a * value_b) / count; + Float64 phi = 0.0; + for (const auto & [key, value_ab] : count_ab) + { + Float64 value_a = count_a.at(key.items[0]); + Float64 value_b = count_b.at(key.items[1]); + + phi += value_ab * value_ab / (value_a * value_b) * count - 2 * value_ab + (value_a * value_b) / count; + } + phi /= count; + + return sqrt(phi / (phi + count)); } - phi /= count; - - return sqrt(phi / (phi + count)); - } -}; + }; +} void registerAggregateFunctionContingency(AggregateFunctionFactory & factory) { @@ -49,5 +46,3 @@ void registerAggregateFunctionContingency(AggregateFunctionFactory & factory) } } - -} diff --git a/src/AggregateFunctions/AggregateFunctionCramersV.cpp b/src/AggregateFunctions/AggregateFunctionCramersV.cpp index 850070f26b2..9700a1e7fdd 100644 --- a/src/AggregateFunctions/AggregateFunctionCramersV.cpp +++ b/src/AggregateFunctions/AggregateFunctionCramersV.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include diff --git a/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrected.cpp b/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrected.cpp index 48a3029c399..f4d06991fb9 100644 --- a/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrected.cpp +++ b/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrected.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include diff --git a/src/AggregateFunctions/AggregateFunctionTheilsU.cpp b/src/AggregateFunctions/AggregateFunctionTheilsU.cpp index 3868e57ad6b..02a9b0956b1 100644 --- a/src/AggregateFunctions/AggregateFunctionTheilsU.cpp +++ b/src/AggregateFunctions/AggregateFunctionTheilsU.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include @@ -40,12 +39,14 @@ struct TheilsUData : CrossTabData dep += (value_ab / count) * log(value_ab / value_b); } - dep -= h_x; - dep /= h_x; + dep -= h_a; + dep /= h_a; return dep; } }; +} + void registerAggregateFunctionTheilsU(AggregateFunctionFactory & factory) { factory.registerFunction(TheilsUData::getName(), @@ -57,5 +58,3 @@ void registerAggregateFunctionTheilsU(AggregateFunctionFactory & factory) } } - -} diff --git a/src/AggregateFunctions/CrossTab.h b/src/AggregateFunctions/CrossTab.h index 4fc6dc2c21b..d08705d6ae9 100644 --- a/src/AggregateFunctions/CrossTab.h +++ b/src/AggregateFunctions/CrossTab.h @@ -113,12 +113,12 @@ public: this->data(place).merge(this->data(rhs)); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional) const override { this->data(place).serialize(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional, Arena *) const override { this->data(place).deserialize(buf); } diff --git a/src/Common/HashTable/HashMap.h b/src/Common/HashTable/HashMap.h index e619421b8f7..0370344d326 100644 --- a/src/Common/HashTable/HashMap.h +++ b/src/Common/HashTable/HashMap.h @@ -263,7 +263,7 @@ public: return it->getMapped(); } - typename Cell::Mapped & ALWAYS_INLINE at(const Key & x) const + const typename Cell::Mapped & ALWAYS_INLINE at(const Key & x) const { if (auto it = this->find(x); it != this->end()) return it->getMapped(); From 93bd1771cc9d7799c0bfc656a275e9a3d6ee5790 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 2 Jan 2022 23:52:55 +0300 Subject: [PATCH 0675/1260] Fix error --- base/base/wide_integer_impl.h | 5 +++++ src/AggregateFunctions/CrossTab.h | 4 ++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/base/base/wide_integer_impl.h b/base/base/wide_integer_impl.h index b8de5efb859..23d57116d74 100644 --- a/base/base/wide_integer_impl.h +++ b/base/base/wide_integer_impl.h @@ -1020,8 +1020,13 @@ constexpr integer::integer(std::initializer_list il) noexcept { auto it = il.begin(); for (size_t i = 0; i < _impl::item_count; ++i) + { if (it < il.end()) + { items[i] = *it; + ++it; + } + } } } diff --git a/src/AggregateFunctions/CrossTab.h b/src/AggregateFunctions/CrossTab.h index d08705d6ae9..cbb32f7478b 100644 --- a/src/AggregateFunctions/CrossTab.h +++ b/src/AggregateFunctions/CrossTab.h @@ -54,7 +54,7 @@ struct CrossTabData count_ab[key] += value; } - void serialize(WriteBuffer &buf) const + void serialize(WriteBuffer & buf) const { writeBinary(count, buf); count_a.write(buf); @@ -62,7 +62,7 @@ struct CrossTabData count_ab.write(buf); } - void deserialize(ReadBuffer &buf) + void deserialize(ReadBuffer & buf) { readBinary(count, buf); count_a.read(buf); From 420baf54bd89d873311d984d265a6965482a1ccd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 3 Jan 2022 00:30:16 +0300 Subject: [PATCH 0676/1260] Add tests --- .../0_stateless/02158_contingency.reference | 5 +++++ tests/queries/0_stateless/02158_contingency.sql | 5 +++++ .../queries/1_stateful/00169_contingency.reference | 5 +++++ tests/queries/1_stateful/00169_contingency.sql | 14 ++++++++++++++ 4 files changed, 29 insertions(+) create mode 100644 tests/queries/0_stateless/02158_contingency.reference create mode 100644 tests/queries/0_stateless/02158_contingency.sql create mode 100644 tests/queries/1_stateful/00169_contingency.reference create mode 100644 tests/queries/1_stateful/00169_contingency.sql diff --git a/tests/queries/0_stateless/02158_contingency.reference b/tests/queries/0_stateless/02158_contingency.reference new file mode 100644 index 00000000000..ac475c7f204 --- /dev/null +++ b/tests/queries/0_stateless/02158_contingency.reference @@ -0,0 +1,5 @@ +0 0 -0 -0 0 +1 nan -1 -1 0.7 +0.95 0.95 -1 -1 0.23 +0.89 0.87 -0.7 -1 0.14 +0.95 0.89 -1 -0.89 0.23 diff --git a/tests/queries/0_stateless/02158_contingency.sql b/tests/queries/0_stateless/02158_contingency.sql new file mode 100644 index 00000000000..d1e1c76c066 --- /dev/null +++ b/tests/queries/0_stateless/02158_contingency.sql @@ -0,0 +1,5 @@ +SELECT round(cramersV(a, b), 2), round(cramersVBiasCorrected(a, b), 2), round(theilsU(a, b), 2), round(theilsU(b, a), 2), round(contingency(a, b), 2) FROM (SELECT number % 3 AS a, number % 5 AS b FROM numbers(150)); +SELECT round(cramersV(a, b), 2), round(cramersVBiasCorrected(a, b), 2), round(theilsU(a, b), 2), round(theilsU(b, a), 2), round(contingency(a, b), 2) FROM (SELECT number AS a, number + 1 AS b FROM numbers(150)); +SELECT round(cramersV(a, b), 2), round(cramersVBiasCorrected(a, b), 2), round(theilsU(a, b), 2), round(theilsU(b, a), 2), round(contingency(a, b), 2) FROM (SELECT number % 10 AS a, number % 10 AS b FROM numbers(150)); +SELECT round(cramersV(a, b), 2), round(cramersVBiasCorrected(a, b), 2), round(theilsU(a, b), 2), round(theilsU(b, a), 2), round(contingency(a, b), 2) FROM (SELECT number % 10 AS a, number % 5 AS b FROM numbers(150)); +SELECT round(cramersV(a, b), 2), round(cramersVBiasCorrected(a, b), 2), round(theilsU(a, b), 2), round(theilsU(b, a), 2), round(contingency(a, b), 2) FROM (SELECT number % 10 AS a, number % 10 = 0 ? number : a AS b FROM numbers(150)); diff --git a/tests/queries/1_stateful/00169_contingency.reference b/tests/queries/1_stateful/00169_contingency.reference new file mode 100644 index 00000000000..8e881081445 --- /dev/null +++ b/tests/queries/1_stateful/00169_contingency.reference @@ -0,0 +1,5 @@ +1 1 -1 -1 0.09 +0.49 0.49 -0.45 -0.69 0.03 +0.81 0.81 -0.91 -0.85 0.08 +0.96 0.96 -0.9 -0.98 0.14 +0.6 0.6 -0.78 -0.8 0.01 diff --git a/tests/queries/1_stateful/00169_contingency.sql b/tests/queries/1_stateful/00169_contingency.sql new file mode 100644 index 00000000000..cc44bba8509 --- /dev/null +++ b/tests/queries/1_stateful/00169_contingency.sql @@ -0,0 +1,14 @@ +WITH URLDomain AS a, URLDomain AS b +SELECT round(cramersV(a, b), 2), round(cramersVBiasCorrected(a, b), 2), round(theilsU(a, b), 2), round(theilsU(b, a), 2), round(contingency(a, b), 2) FROM test.hits; + +WITH URLDomain AS a, RefererDomain AS b +SELECT round(cramersV(a, b), 2), round(cramersVBiasCorrected(a, b), 2), round(theilsU(a, b), 2), round(theilsU(b, a), 2), round(contingency(a, b), 2) FROM test.hits; + +WITH URLDomain AS a, CounterID AS b +SELECT round(cramersV(a, b), 2), round(cramersVBiasCorrected(a, b), 2), round(theilsU(a, b), 2), round(theilsU(b, a), 2), round(contingency(a, b), 2) FROM test.hits; + +WITH ClientIP AS a, RemoteIP AS b +SELECT round(cramersV(a, b), 2), round(cramersVBiasCorrected(a, b), 2), round(theilsU(a, b), 2), round(theilsU(b, a), 2), round(contingency(a, b), 2) FROM test.hits; + +WITH ResolutionWidth AS a, ResolutionHeight AS b +SELECT round(cramersV(a, b), 2), round(cramersVBiasCorrected(a, b), 2), round(theilsU(a, b), 2), round(theilsU(b, a), 2), round(contingency(a, b), 2) FROM test.hits; From cd08a7ced0010f97ed00dd34c32d43c07fcb43c8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 3 Jan 2022 01:36:55 +0300 Subject: [PATCH 0677/1260] Fix error --- .../AggregateFunctionIntervalLengthSum.h | 38 +++++++++---------- 1 file changed, 17 insertions(+), 21 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h b/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h index 93e81be93fc..b15633b12f5 100644 --- a/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h +++ b/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h @@ -7,8 +7,6 @@ #include #include -#include - #include #include @@ -136,6 +134,11 @@ template class AggregateFunctionIntervalLengthSum final : public IAggregateFunctionDataHelper> { private: + static auto length(typename Data::Segment segment) + { + return segment.second - segment.first; + } + template TResult getIntervalLengthSum(Data & data) const { @@ -146,31 +149,24 @@ private: TResult res = 0; - typename Data::Segment cur_segment = data.segments[0]; + typename Data::Segment curr_segment = data.segments[0]; - for (size_t i = 1, sz = data.segments.size(); i < sz; ++i) + for (size_t i = 1, size = data.segments.size(); i < size; ++i) { + const typename Data::Segment & next_segment = data.segments[i]; + /// Check if current interval intersects with next one then add length, otherwise advance interval end. - if (cur_segment.second < data.segments[i].first) + if (curr_segment.second < next_segment.first) { - if constexpr (std::is_floating_point_v) - { - res += cur_segment.second - cur_segment.first; - } - else - { - TResult diff; - if (!common::subOverflow(static_cast(cur_segment.second), static_cast(cur_segment.first), diff)) - res += diff; - } - - cur_segment = data.segments[i]; + res += length(curr_segment); + curr_segment = next_segment; + } + else if (next_segment.second > curr_segment.second) + { + curr_segment.second = next_segment.second; } - else - cur_segment.second = std::max(cur_segment.second, data.segments[i].second); } - - res += cur_segment.second - cur_segment.first; + res += length(curr_segment); return res; } From 0b7a94c2bed01b5cc212f742e6354f3e2ddf01f8 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 8 Nov 2021 15:19:31 +0800 Subject: [PATCH 0678/1260] Saturate date/datetime to zero --- src/Common/DateLUTImpl.cpp | 14 ++ src/Common/DateLUTImpl.h | 195 +++++++++++++----- src/Functions/CustomWeekTransforms.h | 2 +- src/Functions/DateTimeTransforms.h | 59 +++--- .../FunctionDateOrDateTimeAddInterval.h | 12 +- src/Functions/toStartOfInterval.cpp | 8 +- ...name_with_default_zookeeper_long.reference | 6 +- ...ter_rename_with_default_zookeeper_long.sql | 4 +- .../02096_date_time_1970_saturation.reference | 30 +++ .../02096_date_time_1970_saturation.sql | 31 +++ 10 files changed, 270 insertions(+), 91 deletions(-) create mode 100644 tests/queries/0_stateless/02096_date_time_1970_saturation.reference create mode 100644 tests/queries/0_stateless/02096_date_time_1970_saturation.sql diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index ebf32c4dbd9..869954bb2ae 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -174,6 +174,20 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) { years_months_lut[year_months_lut_index] = first_day_of_last_month; } + + /// Fill saturated LUT. + { + ssize_t day = DATE_LUT_SIZE - 1; + for (; day >= 0; --day) + { + if (lut[day].date >= 0) + lut_saturated[day] = lut[day]; + else + break; + } + for (; day >= 0; --day) + lut_saturated[day] = lut_saturated[day + 1]; + } } diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index e52e6547fa2..c178dc58854 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -61,6 +61,8 @@ private: // has to be a separate type to support overloading // TODO: make sure that any arithmetic on LUTIndex actually results in valid LUTIndex. STRONG_TYPEDEF(UInt32, LUTIndex) + // Same as above but select different function overloads for zero saturation. + STRONG_TYPEDEF(UInt32, LUTIndexWithSaturation) template friend inline LUTIndex operator+(const LUTIndex & index, const T v) @@ -182,6 +184,9 @@ private: /// In comparison to std::vector, plain array is cheaper by one indirection. Values lut[DATE_LUT_SIZE + 1]; + /// Same as above but with dates < 1970-01-01 saturated to 1970-01-01. + Values lut_saturated[DATE_LUT_SIZE + 1]; + /// Year number after DATE_LUT_MIN_YEAR -> LUTIndex in lut for start of year. LUTIndex years_lut[DATE_LUT_YEARS]; @@ -278,19 +283,39 @@ public: auto getOffsetAtStartOfEpoch() const { return offset_at_start_of_epoch; } auto getTimeOffsetAtStartOfLUT() const { return offset_at_start_of_lut; } - auto getDayNumOffsetEpoch() const { return daynum_offset_epoch; } + static auto getDayNumOffsetEpoch() { return daynum_offset_epoch; } /// All functions below are thread-safe; arguments are not checked. - inline ExtendedDayNum toDayNum(ExtendedDayNum d) const + static ExtendedDayNum toDayNum(ExtendedDayNum d) { return d; } - template - inline ExtendedDayNum toDayNum(DateOrTime v) const + static UInt32 saturateMinus(UInt32 x, UInt32 y) { - return ExtendedDayNum{static_cast(toLUTIndex(v).toUnderType() - daynum_offset_epoch)}; + UInt32 res = x - y; + res &= -Int32(res <= x); + return res; + } + + static ExtendedDayNum toDayNum(LUTIndex d) + { + return ExtendedDayNum{static_cast(d.toUnderType() - daynum_offset_epoch)}; + } + + static DayNum toDayNum(LUTIndexWithSaturation d) + { + return DayNum{static_cast(saturateMinus(d.toUnderType(), daynum_offset_epoch))}; + } + + template + inline auto toDayNum(DateOrTime v) const + { + if constexpr (std::is_unsigned_v || std::is_same_v) + return DayNum{static_cast(saturateMinus(toLUTIndex(v).toUnderType(), daynum_offset_epoch))}; + else + return ExtendedDayNum{static_cast(toLUTIndex(v).toUnderType() - daynum_offset_epoch)}; } /// Round down to start of monday. @@ -298,14 +323,20 @@ public: inline Time toFirstDayOfWeek(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); - return lut[i - (lut[i].day_of_week - 1)].date; + if constexpr (std::is_unsigned_v || std::is_same_v) + return lut_saturated[i - (lut[i].day_of_week - 1)].date; + else + return lut[i - (lut[i].day_of_week - 1)].date; } template - inline ExtendedDayNum toFirstDayNumOfWeek(DateOrTime v) const + inline auto toFirstDayNumOfWeek(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); - return toDayNum(i - (lut[i].day_of_week - 1)); + if constexpr (std::is_unsigned_v || std::is_same_v) + return toDayNum(LUTIndexWithSaturation(i - (lut[i].day_of_week - 1))); + else + return toDayNum(LUTIndex(i - (lut[i].day_of_week - 1))); } /// Round down to start of month. @@ -313,21 +344,30 @@ public: inline Time toFirstDayOfMonth(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); - return lut[i - (lut[i].day_of_month - 1)].date; + if constexpr (std::is_unsigned_v || std::is_same_v) + return lut_saturated[i - (lut[i].day_of_month - 1)].date; + else + return lut[i - (lut[i].day_of_month - 1)].date; } template - inline ExtendedDayNum toFirstDayNumOfMonth(DateOrTime v) const + inline auto toFirstDayNumOfMonth(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); - return toDayNum(i - (lut[i].day_of_month - 1)); + if constexpr (std::is_unsigned_v || std::is_same_v) + return toDayNum(LUTIndexWithSaturation(i - (lut[i].day_of_month - 1))); + else + return toDayNum(LUTIndex(i - (lut[i].day_of_month - 1))); } /// Round down to start of quarter. template - inline ExtendedDayNum toFirstDayNumOfQuarter(DateOrTime v) const + inline auto toFirstDayNumOfQuarter(DateOrTime v) const { - return toDayNum(toFirstDayOfQuarterIndex(v)); + if constexpr (std::is_unsigned_v || std::is_same_v) + return toDayNum(LUTIndexWithSaturation(toFirstDayOfQuarterIndex(v))); + else + return toDayNum(LUTIndex(toFirstDayOfQuarterIndex(v))); } template @@ -365,9 +405,12 @@ public: } template - inline ExtendedDayNum toFirstDayNumOfYear(DateOrTime v) const + inline auto toFirstDayNumOfYear(DateOrTime v) const { - return toDayNum(toFirstDayNumOfYearIndex(v)); + if constexpr (std::is_unsigned_v || std::is_same_v) + return toDayNum(LUTIndexWithSaturation(toFirstDayNumOfYearIndex(v))); + else + return toDayNum(LUTIndex(toFirstDayNumOfYearIndex(v))); } inline Time toFirstDayOfNextMonth(Time t) const @@ -514,11 +557,17 @@ public: * because the same calendar day starts/ends at different timestamps in different time zones) */ - inline Time fromDayNum(DayNum d) const { return lut[toLUTIndex(d)].date; } + inline Time fromDayNum(DayNum d) const { return lut_saturated[toLUTIndex(d)].date; } inline Time fromDayNum(ExtendedDayNum d) const { return lut[toLUTIndex(d)].date; } template - inline Time toDate(DateOrTime v) const { return lut[toLUTIndex(v)].date; } + inline Time toDate(DateOrTime v) const + { + if constexpr (std::is_unsigned_v || std::is_same_v) + return lut_saturated[toLUTIndex(v)].date; + else + return lut[toLUTIndex(v)].date; + } template inline unsigned toMonth(DateOrTime v) const { return lut[toLUTIndex(v)].month; } @@ -581,9 +630,12 @@ public: } template - inline ExtendedDayNum toFirstDayNumOfISOYear(DateOrTime v) const + inline auto toFirstDayNumOfISOYear(DateOrTime v) const { - return toDayNum(toFirstDayNumOfISOYearIndex(v)); + if constexpr (std::is_unsigned_v || std::is_same_v) + return toDayNum(LUTIndexWithSaturation(toFirstDayNumOfISOYearIndex(v))); + else + return toDayNum(LUTIndex(toFirstDayNumOfISOYearIndex(v))); } inline Time toFirstDayOfISOYear(Time t) const @@ -596,7 +648,7 @@ public: template inline unsigned toISOWeek(DateOrTime v) const { - return 1 + (toFirstDayNumOfWeek(v) - toFirstDayNumOfISOYear(v)) / 7; + return 1 + (toFirstDayNumOfWeek(v) - toDayNum(toFirstDayNumOfISOYearIndex(v))) / 7; } /* @@ -662,7 +714,7 @@ public: { if (!week_year_mode && ((first_weekday_mode && weekday != 0) || (!first_weekday_mode && weekday >= 4))) return yw; - week_year_mode = 1; + week_year_mode = true; (yw.first)--; first_daynr -= (days = calc_days_in_year(yw.first)); weekday = (weekday + 53 * 7 - days) % 7; @@ -724,7 +776,7 @@ public: /// Get first day of week with week_mode, return Sunday or Monday template - inline ExtendedDayNum toFirstDayNumOfWeek(DateOrTime v, UInt8 week_mode) const + inline auto toFirstDayNumOfWeek(DateOrTime v, UInt8 week_mode) const { bool monday_first_mode = week_mode & static_cast(WeekModeFlag::MONDAY_FIRST); if (monday_first_mode) @@ -733,7 +785,10 @@ public: } else { - return (toDayOfWeek(v) != 7) ? ExtendedDayNum(v - toDayOfWeek(v)) : toDayNum(v); + if constexpr (std::is_unsigned_v || std::is_same_v) + return (toDayOfWeek(v) != 7) ? DayNum(saturateMinus(v, toDayOfWeek(v))) : toDayNum(v); + else + return (toDayOfWeek(v) != 7) ? ExtendedDayNum(v - toDayOfWeek(v)) : toDayNum(v); } } @@ -809,7 +864,7 @@ public: } template - inline ExtendedDayNum toStartOfYearInterval(DateOrTime v, UInt64 years) const + inline auto toStartOfYearInterval(DateOrTime v, UInt64 years) const { if (years == 1) return toFirstDayNumOfYear(v); @@ -822,39 +877,59 @@ public: if (unlikely(year < DATE_LUT_MIN_YEAR)) year = DATE_LUT_MIN_YEAR; - return toDayNum(years_lut[year - DATE_LUT_MIN_YEAR]); + if constexpr (std::is_unsigned_v || std::is_same_v) + return toDayNum(LUTIndexWithSaturation(years_lut[year - DATE_LUT_MIN_YEAR])); + else + return toDayNum(years_lut[year - DATE_LUT_MIN_YEAR]); } - inline ExtendedDayNum toStartOfQuarterInterval(ExtendedDayNum d, UInt64 quarters) const + template || std::is_same_v>> + inline auto toStartOfQuarterInterval(Date d, UInt64 quarters) const { if (quarters == 1) return toFirstDayNumOfQuarter(d); return toStartOfMonthInterval(d, quarters * 3); } - inline ExtendedDayNum toStartOfMonthInterval(ExtendedDayNum d, UInt64 months) const + template || std::is_same_v>> + inline auto toStartOfMonthInterval(Date d, UInt64 months) const { if (months == 1) return toFirstDayNumOfMonth(d); const Values & values = lut[toLUTIndex(d)]; UInt32 month_total_index = (values.year - DATE_LUT_MIN_YEAR) * 12 + values.month - 1; - return toDayNum(years_months_lut[month_total_index / months * months]); + if constexpr (std::is_same_v) + return toDayNum(LUTIndexWithSaturation(years_months_lut[month_total_index / months * months])); + else + return toDayNum(years_months_lut[month_total_index / months * months]); } - inline ExtendedDayNum toStartOfWeekInterval(ExtendedDayNum d, UInt64 weeks) const + template || std::is_same_v>> + inline auto toStartOfWeekInterval(Date d, UInt64 weeks) const { if (weeks == 1) return toFirstDayNumOfWeek(d); UInt64 days = weeks * 7; // January 1st 1970 was Thursday so we need this 4-days offset to make weeks start on Monday. - return ExtendedDayNum(4 + (d - 4) / days * days); + if constexpr (std::is_same_v) + return DayNum(4 + (d - 4) / days * days); + else + return ExtendedDayNum(4 + (d - 4) / days * days); } - inline Time toStartOfDayInterval(ExtendedDayNum d, UInt64 days) const + template || std::is_same_v>> + inline Time toStartOfDayInterval(Date d, UInt64 days) const { if (days == 1) return toDate(d); - return lut[toLUTIndex(ExtendedDayNum(d / days * days))].date; + if constexpr (std::is_same_v) + return lut_saturated[toLUTIndex(ExtendedDayNum(d / days * days))].date; + else + return lut[toLUTIndex(ExtendedDayNum(d / days * days))].date; } inline Time toStartOfHourInterval(Time t, UInt64 hours) const @@ -1140,7 +1215,11 @@ public: /// If resulting month has less deys than source month, then saturation can happen. /// Example: 31 Aug + 1 month = 30 Sep. - inline Time NO_SANITIZE_UNDEFINED addMonths(Time t, Int64 delta) const + template < + typename DateTime, + typename + = std::enable_if_t || std::is_same_v || std::is_same_v>> + inline Time NO_SANITIZE_UNDEFINED addMonths(DateTime t, Int64 delta) const { const auto result_day = addMonthsIndex(t, delta); @@ -1154,20 +1233,28 @@ public: if (time >= lut[result_day].time_at_offset_change()) time -= lut[result_day].amount_of_offset_change(); - return lut[result_day].date + time; + auto res = lut[result_day].date + time; + if constexpr (std::is_same_v) + { + /// Common compiler should generate branchless code for this saturation operation. + return res <= 0 ? 0 : res; + } + else + return res; } - inline ExtendedDayNum NO_SANITIZE_UNDEFINED addMonths(ExtendedDayNum d, Int64 delta) const + template || std::is_same_v>> + inline auto NO_SANITIZE_UNDEFINED addMonths(Date d, Int64 delta) const { - return toDayNum(addMonthsIndex(d, delta)); + if constexpr (std::is_same_v) + return toDayNum(LUTIndexWithSaturation(addMonthsIndex(d, delta))); + else + return toDayNum(addMonthsIndex(d, delta)); } - inline Time NO_SANITIZE_UNDEFINED addQuarters(Time t, Int32 delta) const - { - return addMonths(t, static_cast(delta) * 3); - } - - inline ExtendedDayNum addQuarters(ExtendedDayNum d, Int32 delta) const + template + inline auto addQuarters(DateOrTime d, Int32 delta) const { return addMonths(d, static_cast(delta) * 3); } @@ -1189,7 +1276,11 @@ public: } /// Saturation can occur if 29 Feb is mapped to non-leap year. - inline Time addYears(Time t, Int64 delta) const + template < + typename DateTime, + typename + = std::enable_if_t || std::is_same_v || std::is_same_v>> + inline Time addYears(DateTime t, Int64 delta) const { auto result_day = addYearsIndex(t, delta); @@ -1203,12 +1294,24 @@ public: if (time >= lut[result_day].time_at_offset_change()) time -= lut[result_day].amount_of_offset_change(); - return lut[result_day].date + time; + auto res = lut[result_day].date + time; + if constexpr (std::is_same_v) + { + /// Common compiler should generate branchless code for this saturation operation. + return res <= 0 ? 0 : res; + } + else + return res; } - inline ExtendedDayNum addYears(ExtendedDayNum d, Int64 delta) const + template || std::is_same_v>> + inline auto addYears(Date d, Int64 delta) const { - return toDayNum(addYearsIndex(d, delta)); + if constexpr (std::is_same_v) + return toDayNum(LUTIndexWithSaturation(addYearsIndex(d, delta))); + else + return toDayNum(addYearsIndex(d, delta)); } diff --git a/src/Functions/CustomWeekTransforms.h b/src/Functions/CustomWeekTransforms.h index 5ccb2e06c44..8656f9da927 100644 --- a/src/Functions/CustomWeekTransforms.h +++ b/src/Functions/CustomWeekTransforms.h @@ -76,7 +76,7 @@ struct ToStartOfWeekImpl } static inline UInt16 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone) { - return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d), week_mode); + return time_zone.toFirstDayNumOfWeek(DayNum(d), week_mode); } using FactorTransform = ZeroTransform; diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 08dac9c2ba0..a7f06689820 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -84,7 +84,8 @@ struct ToDate32Impl } static inline Int32 execute(UInt32 t, const DateLUTImpl & time_zone) { - return Int32(time_zone.toDayNum(t)); + /// Don't saturate. + return Int32(time_zone.toDayNum(t)); } static inline Int32 execute(Int32 d, const DateLUTImpl &) { @@ -117,7 +118,7 @@ struct ToStartOfDayImpl } static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toDate(ExtendedDayNum(d)); + return time_zone.toDate(DayNum(d)); } using FactorTransform = ZeroTransform; @@ -143,7 +144,7 @@ struct ToMondayImpl } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d)); + return time_zone.toFirstDayNumOfWeek(DayNum(d)); } using FactorTransform = ZeroTransform; @@ -167,7 +168,7 @@ struct ToStartOfMonthImpl } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toFirstDayNumOfMonth(ExtendedDayNum(d)); + return time_zone.toFirstDayNumOfMonth(DayNum(d)); } using FactorTransform = ZeroTransform; @@ -191,7 +192,7 @@ struct ToStartOfQuarterImpl } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toFirstDayNumOfQuarter(ExtendedDayNum(d)); + return time_zone.toFirstDayNumOfQuarter(DayNum(d)); } using FactorTransform = ZeroTransform; @@ -215,7 +216,7 @@ struct ToStartOfYearImpl } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toFirstDayNumOfYear(ExtendedDayNum(d)); + return time_zone.toFirstDayNumOfYear(DayNum(d)); } using FactorTransform = ZeroTransform; @@ -224,7 +225,7 @@ struct ToStartOfYearImpl struct ToTimeImpl { - /// When transforming to time, the date will be equated to 1970-01-01. + /// When transforming to time, the date will be equated to 1970-01-02. static constexpr auto name = "toTime"; static UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) @@ -456,7 +457,7 @@ struct ToYearImpl } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toYear(ExtendedDayNum(d)); + return time_zone.toYear(DayNum(d)); } using FactorTransform = ZeroTransform; @@ -480,7 +481,7 @@ struct ToQuarterImpl } static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toQuarter(ExtendedDayNum(d)); + return time_zone.toQuarter(DayNum(d)); } using FactorTransform = ToStartOfYearImpl; @@ -504,7 +505,7 @@ struct ToMonthImpl } static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toMonth(ExtendedDayNum(d)); + return time_zone.toMonth(DayNum(d)); } using FactorTransform = ToStartOfYearImpl; @@ -528,7 +529,7 @@ struct ToDayOfMonthImpl } static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toDayOfMonth(ExtendedDayNum(d)); + return time_zone.toDayOfMonth(DayNum(d)); } using FactorTransform = ToStartOfMonthImpl; @@ -552,7 +553,7 @@ struct ToDayOfWeekImpl } static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toDayOfWeek(ExtendedDayNum(d)); + return time_zone.toDayOfWeek(DayNum(d)); } using FactorTransform = ToMondayImpl; @@ -576,7 +577,7 @@ struct ToDayOfYearImpl } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toDayOfYear(ExtendedDayNum(d)); + return time_zone.toDayOfYear(DayNum(d)); } using FactorTransform = ToStartOfYearImpl; @@ -699,7 +700,7 @@ struct ToISOYearImpl } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toISOYear(ExtendedDayNum(d)); + return time_zone.toISOYear(DayNum(d)); } using FactorTransform = ZeroTransform; @@ -723,7 +724,7 @@ struct ToStartOfISOYearImpl } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toFirstDayNumOfISOYear(ExtendedDayNum(d)); + return time_zone.toFirstDayNumOfISOYear(DayNum(d)); } using FactorTransform = ZeroTransform; @@ -747,7 +748,7 @@ struct ToISOWeekImpl } static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toISOWeek(ExtendedDayNum(d)); + return time_zone.toISOWeek(DayNum(d)); } using FactorTransform = ToISOYearImpl; @@ -771,7 +772,7 @@ struct ToRelativeYearNumImpl } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toYear(ExtendedDayNum(d)); + return time_zone.toYear(DayNum(d)); } using FactorTransform = ZeroTransform; @@ -795,7 +796,7 @@ struct ToRelativeQuarterNumImpl } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toRelativeQuarterNum(ExtendedDayNum(d)); + return time_zone.toRelativeQuarterNum(DayNum(d)); } using FactorTransform = ZeroTransform; @@ -819,7 +820,7 @@ struct ToRelativeMonthNumImpl } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toRelativeMonthNum(ExtendedDayNum(d)); + return time_zone.toRelativeMonthNum(DayNum(d)); } using FactorTransform = ZeroTransform; @@ -843,7 +844,7 @@ struct ToRelativeWeekNumImpl } static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toRelativeWeekNum(ExtendedDayNum(d)); + return time_zone.toRelativeWeekNum(DayNum(d)); } using FactorTransform = ZeroTransform; @@ -892,7 +893,7 @@ struct ToRelativeHourNumImpl } static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toRelativeHourNum(ExtendedDayNum(d)); + return time_zone.toRelativeHourNum(DayNum(d)); } using FactorTransform = ZeroTransform; @@ -916,7 +917,7 @@ struct ToRelativeMinuteNumImpl } static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toRelativeMinuteNum(ExtendedDayNum(d)); + return time_zone.toRelativeMinuteNum(DayNum(d)); } using FactorTransform = ZeroTransform; @@ -940,7 +941,7 @@ struct ToRelativeSecondNumImpl } static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.fromDayNum(ExtendedDayNum(d)); + return time_zone.fromDayNum(DayNum(d)); } using FactorTransform = ZeroTransform; @@ -960,11 +961,11 @@ struct ToYYYYMMImpl } static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone) { - return time_zone.toNumYYYYMM(static_cast(d)); + return time_zone.toNumYYYYMM(ExtendedDayNum(d)); } static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toNumYYYYMM(static_cast(d)); + return time_zone.toNumYYYYMM(DayNum(d)); } using FactorTransform = ZeroTransform; @@ -984,11 +985,11 @@ struct ToYYYYMMDDImpl } static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone) { - return time_zone.toNumYYYYMMDD(static_cast(d)); + return time_zone.toNumYYYYMMDD(ExtendedDayNum(d)); } static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toNumYYYYMMDD(static_cast(d)); + return time_zone.toNumYYYYMMDD(DayNum(d)); } using FactorTransform = ZeroTransform; @@ -1008,11 +1009,11 @@ struct ToYYYYMMDDhhmmssImpl } static inline UInt64 execute(Int32 d, const DateLUTImpl & time_zone) { - return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(static_cast(d))); + return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(ExtendedDayNum(d))); } static inline UInt64 execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(static_cast(d))); + return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(DayNum(d))); } using FactorTransform = ZeroTransform; diff --git a/src/Functions/FunctionDateOrDateTimeAddInterval.h b/src/Functions/FunctionDateOrDateTimeAddInterval.h index 4224a74ae8e..8f6b1370935 100644 --- a/src/Functions/FunctionDateOrDateTimeAddInterval.h +++ b/src/Functions/FunctionDateOrDateTimeAddInterval.h @@ -58,7 +58,7 @@ struct AddSecondsImpl } static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone) { - return time_zone.fromDayNum(ExtendedDayNum(d)) + delta; + return time_zone.fromDayNum(DayNum(d)) + delta; } }; @@ -83,7 +83,7 @@ struct AddMinutesImpl } static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone) { - return time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 60; + return time_zone.fromDayNum(DayNum(d)) + delta * 60; } }; @@ -107,7 +107,7 @@ struct AddHoursImpl } static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone) { - return time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 3600; + return time_zone.fromDayNum(DayNum(d)) + delta * 3600; } }; @@ -180,7 +180,7 @@ struct AddMonthsImpl static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone) { - return time_zone.addMonths(ExtendedDayNum(d), delta); + return time_zone.addMonths(DayNum(d), delta); } static inline Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone) @@ -206,7 +206,7 @@ struct AddQuartersImpl static inline UInt16 execute(UInt16 d, Int32 delta, const DateLUTImpl & time_zone) { - return time_zone.addQuarters(ExtendedDayNum(d), delta); + return time_zone.addQuarters(DayNum(d), delta); } static inline Int32 execute(Int32 d, Int32 delta, const DateLUTImpl & time_zone) @@ -232,7 +232,7 @@ struct AddYearsImpl static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone) { - return time_zone.addYears(ExtendedDayNum(d), delta); + return time_zone.addYears(DayNum(d), delta); } static inline Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index f8ea44851b6..09b7931de8d 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -37,7 +37,7 @@ namespace static UInt16 execute(UInt16 d, UInt64 years, const DateLUTImpl & time_zone) { - return time_zone.toStartOfYearInterval(ExtendedDayNum(d), years); + return time_zone.toStartOfYearInterval(DayNum(d), years); } static UInt16 execute(Int32 d, UInt64 years, const DateLUTImpl & time_zone) @@ -63,7 +63,7 @@ namespace static UInt16 execute(UInt16 d, UInt64 quarters, const DateLUTImpl & time_zone) { - return time_zone.toStartOfQuarterInterval(ExtendedDayNum(d), quarters); + return time_zone.toStartOfQuarterInterval(DayNum(d), quarters); } static UInt16 execute(Int32 d, UInt64 quarters, const DateLUTImpl & time_zone) @@ -89,7 +89,7 @@ namespace static UInt16 execute(UInt16 d, UInt64 months, const DateLUTImpl & time_zone) { - return time_zone.toStartOfMonthInterval(ExtendedDayNum(d), months); + return time_zone.toStartOfMonthInterval(DayNum(d), months); } static UInt16 execute(Int32 d, UInt64 months, const DateLUTImpl & time_zone) @@ -115,7 +115,7 @@ namespace static UInt16 execute(UInt16 d, UInt64 weeks, const DateLUTImpl & time_zone) { - return time_zone.toStartOfWeekInterval(ExtendedDayNum(d), weeks); + return time_zone.toStartOfWeekInterval(DayNum(d), weeks); } static UInt16 execute(Int32 d, UInt64 weeks, const DateLUTImpl & time_zone) diff --git a/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper_long.reference b/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper_long.reference index 2a6b00cdddb..968247bd35b 100644 --- a/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper_long.reference +++ b/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper_long.reference @@ -8,10 +8,10 @@ Hello 1 Word 1 date1 date2 value1 value2 2019-10-02 2018-10-02 1 1 -CREATE TABLE default.table_rename_with_ttl\n(\n `date1` Date,\n `date2` Date,\n `value1` String,\n `value2` String TTL date1 + toIntervalMonth(10000)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL date2 + toIntervalMonth(10000)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.table_rename_with_ttl\n(\n `date1` Date,\n `date2` Date,\n `value1` String,\n `value2` String TTL date1 + toIntervalMonth(500)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL date2 + toIntervalMonth(500)\nSETTINGS index_granularity = 8192 renamed_date1 date2 value1 value2 2019-10-02 2018-10-02 1 1 -CREATE TABLE default.table_rename_with_ttl\n(\n `renamed_date1` Date,\n `date2` Date,\n `value1` String,\n `value2` String TTL renamed_date1 + toIntervalMonth(10000)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL date2 + toIntervalMonth(10000)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.table_rename_with_ttl\n(\n `renamed_date1` Date,\n `date2` Date,\n `value1` String,\n `value2` String TTL renamed_date1 + toIntervalMonth(500)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL date2 + toIntervalMonth(500)\nSETTINGS index_granularity = 8192 renamed_date1 renamed_date2 value1 value2 2019-10-02 2018-10-02 1 1 -CREATE TABLE default.table_rename_with_ttl\n(\n `renamed_date1` Date,\n `renamed_date2` Date,\n `value1` String,\n `value2` String TTL renamed_date1 + toIntervalMonth(10000)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL renamed_date2 + toIntervalMonth(10000)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.table_rename_with_ttl\n(\n `renamed_date1` Date,\n `renamed_date2` Date,\n `value1` String,\n `value2` String TTL renamed_date1 + toIntervalMonth(500)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/test_01213/table_rename_with_ttl\', \'1\')\nORDER BY tuple()\nTTL renamed_date2 + toIntervalMonth(500)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper_long.sql b/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper_long.sql index 986947d5979..a831fd18bfe 100644 --- a/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper_long.sql +++ b/tests/queries/0_stateless/01213_alter_rename_with_default_zookeeper_long.sql @@ -38,11 +38,11 @@ CREATE TABLE table_rename_with_ttl date1 Date, date2 Date, value1 String, - value2 String TTL date1 + INTERVAL 10000 MONTH + value2 String TTL date1 + INTERVAL 500 MONTH ) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01213/table_rename_with_ttl', '1') ORDER BY tuple() -TTL date2 + INTERVAL 10000 MONTH; +TTL date2 + INTERVAL 500 MONTH; INSERT INTO table_rename_with_ttl SELECT toDateTime(toDate('2019-10-01') + number % 3, 'Europe/Moscow'), toDateTime(toDate('2018-10-01') + number % 3, 'Europe/Moscow'), toString(number), toString(number) from numbers(9); diff --git a/tests/queries/0_stateless/02096_date_time_1970_saturation.reference b/tests/queries/0_stateless/02096_date_time_1970_saturation.reference new file mode 100644 index 00000000000..3c073b9262e --- /dev/null +++ b/tests/queries/0_stateless/02096_date_time_1970_saturation.reference @@ -0,0 +1,30 @@ +1970-01-01 +1970-01-01 03:00:00 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-02 03:00:00 +1970-01-01 03:00:00 +1970-01-01 03:00:00 +1970-01-01 03:00:00 +1970-01-01 03:00:00 +1970-01-01 03:00:00 +1969-12-31 16:00:00 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-02 16:00:00 +1969-12-31 16:00:00 +1969-12-31 16:00:00 +1969-12-31 16:00:00 +1969-12-31 16:00:00 +1969-12-31 16:00:00 diff --git a/tests/queries/0_stateless/02096_date_time_1970_saturation.sql b/tests/queries/0_stateless/02096_date_time_1970_saturation.sql new file mode 100644 index 00000000000..e0c401443a7 --- /dev/null +++ b/tests/queries/0_stateless/02096_date_time_1970_saturation.sql @@ -0,0 +1,31 @@ +select toDate(0); +select toDateTime(0, 'Europe/Moscow'); +select toMonday(toDate(0)); +select toMonday(toDateTime(0, 'Europe/Moscow')); +select toStartOfWeek(toDate(0)); +select toStartOfWeek(toDateTime(0, 'Europe/Moscow')); +select toStartOfMonth(toDate(0)); +select toStartOfMonth(toDateTime(0, 'Europe/Moscow')); +select toStartOfQuarter(toDate(0)); +select toStartOfQuarter(toDateTime(0, 'Europe/Moscow')); +select toStartOfYear(toDate(0)); +select toStartOfYear(toDateTime(0, 'Europe/Moscow')); +select toTime(toDateTime(0, 'Europe/Moscow')); +select toStartOfMinute(toDateTime(0, 'Europe/Moscow')); +select toStartOfFiveMinute(toDateTime(0, 'Europe/Moscow')); +select toStartOfTenMinutes(toDateTime(0, 'Europe/Moscow')); +select toStartOfFifteenMinutes(toDateTime(0, 'Europe/Moscow')); +select toStartOfHour(toDateTime(0, 'Europe/Moscow')); + +select toDateTime(0, 'America/Los_Angeles'); +select toMonday(toDateTime(0, 'America/Los_Angeles')); +select toStartOfWeek(toDateTime(0, 'America/Los_Angeles')); +select toStartOfMonth(toDateTime(0, 'America/Los_Angeles')); +select toStartOfQuarter(toDateTime(0, 'America/Los_Angeles')); +select toStartOfYear(toDateTime(0, 'America/Los_Angeles')); +select toTime(toDateTime(0, 'America/Los_Angeles'), 'America/Los_Angeles'); +select toStartOfMinute(toDateTime(0, 'America/Los_Angeles')); +select toStartOfFiveMinute(toDateTime(0, 'America/Los_Angeles')); +select toStartOfTenMinutes(toDateTime(0, 'America/Los_Angeles')); +select toStartOfFifteenMinutes(toDateTime(0, 'America/Los_Angeles')); +select toStartOfHour(toDateTime(0, 'America/Los_Angeles')); From cab5f4fe1e3ae21629022cba6659ebe2403ff43c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 3 Jan 2022 02:05:27 +0300 Subject: [PATCH 0679/1260] Merge with master --- src/Functions/FunctionsTimeWindow.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsTimeWindow.h b/src/Functions/FunctionsTimeWindow.h index 6e5d79fd062..313de10702d 100644 --- a/src/Functions/FunctionsTimeWindow.h +++ b/src/Functions/FunctionsTimeWindow.h @@ -48,7 +48,7 @@ struct ToStartOfTransform; template <> \ struct ToStartOfTransform \ { \ - static ExtendedDayNum execute(UInt32 t, UInt64 delta, const DateLUTImpl & time_zone) \ + static auto execute(UInt32 t, UInt64 delta, const DateLUTImpl & time_zone) \ { \ return time_zone.toStartOf##INTERVAL_KIND##Interval(time_zone.toDayNum(t), delta); \ } \ @@ -89,7 +89,7 @@ struct ToStartOfTransform; template <> \ struct AddTime \ { \ - static inline ExtendedDayNum execute(UInt16 d, UInt64 delta, const DateLUTImpl & time_zone) \ + static inline auto execute(UInt16 d, UInt64 delta, const DateLUTImpl & time_zone) \ { \ return time_zone.add##INTERVAL_KIND##s(ExtendedDayNum(d), delta); \ } \ From d77a4eb093d9196127137ca8084d8e9ad6cd79ac Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Jan 2022 02:08:36 +0300 Subject: [PATCH 0680/1260] Update CODEOWNERS --- .github/CODEOWNERS | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index be66f21b838..8b137891791 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -1,3 +1 @@ -docs/* @ClickHouse/docs -docs/zh/* @ClickHouse/docs-zh -website/* @ClickHouse/docs + From 4ebc8918a48325cad1c51619a9d3c33340672316 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 3 Jan 2022 02:11:47 +0300 Subject: [PATCH 0681/1260] Fix style --- src/AggregateFunctions/AggregateFunctionCramersV.cpp | 2 +- src/Common/HashTable/HashMap.h | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionCramersV.cpp b/src/AggregateFunctions/AggregateFunctionCramersV.cpp index 9700a1e7fdd..48dad648d24 100644 --- a/src/AggregateFunctions/AggregateFunctionCramersV.cpp +++ b/src/AggregateFunctions/AggregateFunctionCramersV.cpp @@ -45,7 +45,7 @@ struct CramersVData : CrossTabData void registerAggregateFunctionCramersV(AggregateFunctionFactory & factory) { factory.registerFunction(CramersVData::getName(), - [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) + [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { assertNoParameters(name, parameters); return std::make_shared>(argument_types); diff --git a/src/Common/HashTable/HashMap.h b/src/Common/HashTable/HashMap.h index 0370344d326..236a6d65707 100644 --- a/src/Common/HashTable/HashMap.h +++ b/src/Common/HashTable/HashMap.h @@ -10,6 +10,13 @@ * Also, key in hash table must be of type, that zero bytes is compared equals to zero key. */ +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} +} struct NoInitTag { From 3333c36df307d2fbfba24e4a4a23501889a3383f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Jan 2022 02:45:20 +0300 Subject: [PATCH 0682/1260] Update images.json --- docker/images.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/images.json b/docker/images.json index 3b90f87ffcf..948f4f0f6b1 100644 --- a/docker/images.json +++ b/docker/images.json @@ -109,7 +109,7 @@ "dependent": [] }, "docker/test/integration/dotnet_client": { - "name": "yandex/clickhouse-dotnet-client", + "name": "clickhouse/clickhouse-dotnet-client", "dependent": [] }, "docker/test/integration/mysql_java_client": { From 57926cf3665729554c78132bc730183066e6b6c8 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Jan 2022 02:45:46 +0300 Subject: [PATCH 0683/1260] Update Dockerfile --- docker/test/integration/dotnet_client/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/integration/dotnet_client/Dockerfile b/docker/test/integration/dotnet_client/Dockerfile index bbca5ec58e5..74bd54dcb0a 100644 --- a/docker/test/integration/dotnet_client/Dockerfile +++ b/docker/test/integration/dotnet_client/Dockerfile @@ -3,6 +3,6 @@ FROM mcr.microsoft.com/dotnet/sdk:3.1 WORKDIR /client COPY *.cs *.csproj /client/ -ARG VERSION=3.1.0.379 +ARG VERSION=4.1.0 RUN dotnet add package ClickHouse.Client -v ${VERSION} RUN dotnet run From 817bb0ee6d931a5043f9d5fc0119ede1c3a476ba Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Jan 2022 02:46:08 +0300 Subject: [PATCH 0684/1260] Update docker_compose_dotnet_client.yml --- .../integration/runner/compose/docker_compose_dotnet_client.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/integration/runner/compose/docker_compose_dotnet_client.yml b/docker/test/integration/runner/compose/docker_compose_dotnet_client.yml index 4cc494d094a..07bde6b83e8 100644 --- a/docker/test/integration/runner/compose/docker_compose_dotnet_client.yml +++ b/docker/test/integration/runner/compose/docker_compose_dotnet_client.yml @@ -1,6 +1,6 @@ version: '2.3' services: dotnet1: - image: yandex/clickhouse-dotnet-client:${DOCKER_DOTNET_CLIENT_TAG:-latest} + image: clickhouse/clickhouse-dotnet-client:${DOCKER_DOTNET_CLIENT_TAG:-latest} # to keep container running command: sleep infinity From 7d52ba457619a4e72985a81447f08106bf9c00c3 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Jan 2022 02:54:43 +0300 Subject: [PATCH 0685/1260] Update Dockerfile --- docker/test/integration/dotnet_client/Dockerfile | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/test/integration/dotnet_client/Dockerfile b/docker/test/integration/dotnet_client/Dockerfile index 74bd54dcb0a..8a6920057bc 100644 --- a/docker/test/integration/dotnet_client/Dockerfile +++ b/docker/test/integration/dotnet_client/Dockerfile @@ -5,4 +5,3 @@ COPY *.cs *.csproj /client/ ARG VERSION=4.1.0 RUN dotnet add package ClickHouse.Client -v ${VERSION} -RUN dotnet run From 04a293a6ae40f6606c7cf0a5f3de8666081a32ff Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Jan 2022 02:55:30 +0300 Subject: [PATCH 0686/1260] Update images.json --- docker/images.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/images.json b/docker/images.json index 948f4f0f6b1..117efd86a67 100644 --- a/docker/images.json +++ b/docker/images.json @@ -109,7 +109,7 @@ "dependent": [] }, "docker/test/integration/dotnet_client": { - "name": "clickhouse/clickhouse-dotnet-client", + "name": "clickhouse/dotnet-client", "dependent": [] }, "docker/test/integration/mysql_java_client": { From f1b6bc10748b64442a84008edf58963478980e4d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Jan 2022 02:55:49 +0300 Subject: [PATCH 0687/1260] Update docker_compose_dotnet_client.yml --- .../integration/runner/compose/docker_compose_dotnet_client.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/integration/runner/compose/docker_compose_dotnet_client.yml b/docker/test/integration/runner/compose/docker_compose_dotnet_client.yml index 07bde6b83e8..b63dac51522 100644 --- a/docker/test/integration/runner/compose/docker_compose_dotnet_client.yml +++ b/docker/test/integration/runner/compose/docker_compose_dotnet_client.yml @@ -1,6 +1,6 @@ version: '2.3' services: dotnet1: - image: clickhouse/clickhouse-dotnet-client:${DOCKER_DOTNET_CLIENT_TAG:-latest} + image: clickhouse/dotnet-client:${DOCKER_DOTNET_CLIENT_TAG:-latest} # to keep container running command: sleep infinity From d08b1231d75430f226fc27a6d158d56e5c94a9bb Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Jan 2022 02:56:23 +0300 Subject: [PATCH 0688/1260] Update runner --- tests/integration/runner | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/runner b/tests/integration/runner index e33393c7196..89001918730 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -226,7 +226,7 @@ if __name__ == "__main__": [image, tag] = img_tag.split(":") if image == "clickhouse/mysql-golang-client": env_tags += "-e {}={} ".format("DOCKER_MYSQL_GOLANG_CLIENT_TAG", tag) - elif image == "clickhouse/clickhouse-dotnet-client": + elif image == "clickhouse/dotnet-client": env_tags += "-e {}={} ".format("DOCKER_DOTNET_CLIENT_TAG", tag) elif image == "clickhouse/mysql-java-client": env_tags += "-e {}={} ".format("DOCKER_MYSQL_JAVA_CLIENT_TAG", tag) From f790bcfe03b653cf6553b31111fd1a80a1a22519 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 3 Jan 2022 03:03:26 +0300 Subject: [PATCH 0689/1260] Add something to somewhere --- tests/integration/ci-runner.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 830b8e149f6..16fbb514510 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -228,6 +228,7 @@ class ClickhouseIntegrationTestsRunner: "clickhouse/mysql-java-client", "clickhouse/mysql-js-client", "clickhouse/mysql-php-client", "clickhouse/postgresql-java-client", "clickhouse/integration-test", "clickhouse/kerberos-kdc", + "clickhouse/dotnet-client", "clickhouse/integration-helper", ] From 9678179ba04623c13aba9b66375df8739fddc679 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 3 Jan 2022 03:03:49 +0300 Subject: [PATCH 0690/1260] Fix a typo --- tests/integration/ci-runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 16fbb514510..6058a332c29 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -253,7 +253,7 @@ class ClickhouseIntegrationTestsRunner: logging.info("Executing installation cmd %s", cmd) retcode = subprocess.Popen(cmd, shell=True, stderr=log, stdout=log).wait() if retcode == 0: - logging.info("Instsallation of %s successfull", full_path) + logging.info("Installation of %s successfull", full_path) else: raise Exception("Installation of %s failed", full_path) break From 268842afa8b720bbe758fbb7e02e620995ac6001 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 3 Jan 2022 03:07:25 +0300 Subject: [PATCH 0691/1260] Add a comment --- docker/test/integration/dotnet_client/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/integration/dotnet_client/Dockerfile b/docker/test/integration/dotnet_client/Dockerfile index 8a6920057bc..74eb5c3c4d2 100644 --- a/docker/test/integration/dotnet_client/Dockerfile +++ b/docker/test/integration/dotnet_client/Dockerfile @@ -1,3 +1,4 @@ +# docker build . FROM mcr.microsoft.com/dotnet/sdk:3.1 WORKDIR /client From 2c7a8db58c8ce72fff7234586cf49f798d788e60 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 3 Jan 2022 03:36:23 +0300 Subject: [PATCH 0692/1260] Add test output and compare to reference --- .../test/integration/dotnet_client/Dockerfile | 2 ++ .../test/integration/dotnet_client/Program.cs | 25 +++++++++++++++--- .../dotnet_client/clickhouse.test.csproj | 2 +- .../test_mysql_protocol/dotnet.reference | Bin 0 -> 546 bytes tests/integration/test_mysql_protocol/test.py | 3 +++ 5 files changed, 27 insertions(+), 5 deletions(-) create mode 100644 tests/integration/test_mysql_protocol/dotnet.reference diff --git a/docker/test/integration/dotnet_client/Dockerfile b/docker/test/integration/dotnet_client/Dockerfile index 74eb5c3c4d2..f8d33415175 100644 --- a/docker/test/integration/dotnet_client/Dockerfile +++ b/docker/test/integration/dotnet_client/Dockerfile @@ -1,4 +1,6 @@ # docker build . +# docker run -it --rm --network=host 14f23e59669c dotnet run --host localhost --port 8123 --user default --database default + FROM mcr.microsoft.com/dotnet/sdk:3.1 WORKDIR /client diff --git a/docker/test/integration/dotnet_client/Program.cs b/docker/test/integration/dotnet_client/Program.cs index f8db8637aad..3f640d15e86 100644 --- a/docker/test/integration/dotnet_client/Program.cs +++ b/docker/test/integration/dotnet_client/Program.cs @@ -23,11 +23,28 @@ namespace clickhouse.test command.CommandText = "INSERT INTO test.dotnet_test VALUES({age:Int32}, {name:String})"; await command.ExecuteNonQueryAsync(); - using var result = await connection.ExecuteReaderAsync("SELECT * FROM test.dotnet_test"); - while (result.Read()) + using var result1 = await connection.ExecuteReaderAsync("SELECT * FROM test.dotnet_test"); + while (result1.Read()) { - var values = new object[result.FieldCount]; - result.GetValues(values); + var values = new object[result1.FieldCount]; + result1.GetValues(values); + + foreach (var row in values) + { + Console.WriteLine(row); + } + } + + using var result2 = await connection.ExecuteReaderAsync(selectSql); + while (result2.Read()) + { + var values = new object[result2.FieldCount]; + result2.GetValues(values); + + foreach (var row in values) + { + Console.WriteLine(row); + } } } catch (Exception e) diff --git a/docker/test/integration/dotnet_client/clickhouse.test.csproj b/docker/test/integration/dotnet_client/clickhouse.test.csproj index 2ff2340ca4c..11704487bf6 100644 --- a/docker/test/integration/dotnet_client/clickhouse.test.csproj +++ b/docker/test/integration/dotnet_client/clickhouse.test.csproj @@ -6,7 +6,7 @@ - + diff --git a/tests/integration/test_mysql_protocol/dotnet.reference b/tests/integration/test_mysql_protocol/dotnet.reference new file mode 100644 index 0000000000000000000000000000000000000000..5a04e3f56cc02e359db84266215fdb422a810182 GIT binary patch literal 546 zcmah{!D_=W4DH$AFX$!57V1Q@;>0nGv9aUWrM;EHoF>}>Sqn|N(Z8RZCQx>%BiS!c zFScZ5zJ0{6`NRDFVo-|L=5X$!&GGc(+p3d73T35WQ4aD}!A;%u>;BXnKBahCzYNoCCdD-lpaS?eO))+TKVY9is?5G5g;?jRfV@;b1IT_Sy>Fe!Iit+WD*bVV^Zd+fr^*e5FzSso?Y6^D~ zp7!{=_ou2k__paUQY>&4`Bh|>0cu)B3o{V;k{QGVb%I4vL;!9uuHGivGHGoDxzpMa zZw{0ICK^(T&^SsRa7nyhoSM*#5pc0|B}RZ;JPTPctlSxNRcW0yo}9B@*N$> Date: Mon, 3 Jan 2022 03:44:03 +0300 Subject: [PATCH 0693/1260] Move to separate directory --- .../test_dotnet_client/__init__.py | 0 .../test_dotnet_client/configs/config.xml | 16 +++++++ .../test_dotnet_client/configs/users.xml | 32 +++++++++++++ .../golang.reference | 0 tests/integration/test_dotnet_client/test.py | 47 +++++++++++++++++++ tests/integration/test_mysql_protocol/test.py | 18 ------- 6 files changed, 95 insertions(+), 18 deletions(-) create mode 100644 tests/integration/test_dotnet_client/__init__.py create mode 100644 tests/integration/test_dotnet_client/configs/config.xml create mode 100644 tests/integration/test_dotnet_client/configs/users.xml rename tests/integration/{test_mysql_protocol => test_dotnet_client}/golang.reference (100%) create mode 100644 tests/integration/test_dotnet_client/test.py diff --git a/tests/integration/test_dotnet_client/__init__.py b/tests/integration/test_dotnet_client/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_dotnet_client/configs/config.xml b/tests/integration/test_dotnet_client/configs/config.xml new file mode 100644 index 00000000000..9bcadc43f10 --- /dev/null +++ b/tests/integration/test_dotnet_client/configs/config.xml @@ -0,0 +1,16 @@ + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + + 8123 + 127.0.0.1 + + ./clickhouse/ + users.xml + diff --git a/tests/integration/test_dotnet_client/configs/users.xml b/tests/integration/test_dotnet_client/configs/users.xml new file mode 100644 index 00000000000..1874371871a --- /dev/null +++ b/tests/integration/test_dotnet_client/configs/users.xml @@ -0,0 +1,32 @@ + + + + + + + + + + 123 + + ::/0 + + default + default + + + + + + ::/0 + + default + default + + + + + + + + diff --git a/tests/integration/test_mysql_protocol/golang.reference b/tests/integration/test_dotnet_client/golang.reference similarity index 100% rename from tests/integration/test_mysql_protocol/golang.reference rename to tests/integration/test_dotnet_client/golang.reference diff --git a/tests/integration/test_dotnet_client/test.py b/tests/integration/test_dotnet_client/test.py new file mode 100644 index 00000000000..4cc16ac826e --- /dev/null +++ b/tests/integration/test_dotnet_client/test.py @@ -0,0 +1,47 @@ +# coding: utf-8 + +import datetime +import math +import os +import time + +import logging +import docker +import pytest +from docker.models.containers import Container +from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DOCKER_COMPOSE_PATH = get_docker_compose_path() + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance('node', + user_configs=["configs/users.xml"], env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) + +@pytest.fixture(scope="module") +def started_cluster(): + cluster.start() + try: + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(scope='module') +def dotnet_container(): + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_dotnet_client.yml') + run_and_check( + ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) + yield docker.from_env().containers.get(cluster.project_name + '_dotnet1_1') + + +def test_dotnet_client(started_cluster, dotnet_container): + with open(os.path.join(SCRIPT_DIR, 'dotnet.reference'), 'rb') as fp: + reference = fp.read() + + code, (stdout, stderr) = dotnet_container.exec_run( + 'dotnet run --host {host} --port {port} --user default --password 123 --database default' + .format(host=started_cluster.get_instance_ip('node'), port=8123), demux=True) + + assert code == 0 + assert stdout == reference diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index bfa4f20a04f..0b3f6ea95af 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -38,12 +38,6 @@ def golang_container(): ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=600).containers.get(cluster.project_name + '_golang1_1') -@pytest.fixture(scope='module') -def dotnet_container(): - docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_dotnet_client.yml') - run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.from_env().containers.get(cluster.project_name + '_dotnet1_1') @pytest.fixture(scope='module') def php_container(): @@ -408,18 +402,6 @@ def test_golang_client(started_cluster, golang_container): assert stdout == reference -def test_dotnet_client(started_cluster, dotnet_container): - with open(os.path.join(SCRIPT_DIR, 'dotnet.reference'), 'rb') as fp: - reference = fp.read() - - code, (stdout, stderr) = dotnet_container.exec_run( - 'dotnet run --host {host} --port {port} --user default --password 123 --database default' - .format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True) - - assert code == 0 - assert stdout == reference - - def test_php_client(started_cluster, php_container): # type: (str, Container) -> None code, (stdout, stderr) = php_container.exec_run( From f0d2838a5b425d00882f8a32dc0ae07e7d16133f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Jan 2022 03:52:39 +0300 Subject: [PATCH 0694/1260] Update AggregateFunctionNothing.h --- src/AggregateFunctions/AggregateFunctionNothing.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionNothing.h b/src/AggregateFunctions/AggregateFunctionNothing.h index 34d83ecfff6..22e296a96a4 100644 --- a/src/AggregateFunctions/AggregateFunctionNothing.h +++ b/src/AggregateFunctions/AggregateFunctionNothing.h @@ -28,7 +28,7 @@ public: DataTypePtr getReturnType() const override { - return std::make_shared(std::make_shared()); + return argument_types.front(); } bool allocatesMemoryInArena() const override { return false; } From 141696e146e933f0508231c68a541b8f13c6c154 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Jan 2022 03:53:39 +0300 Subject: [PATCH 0695/1260] Update AggregateFunctionNothing.cpp --- src/AggregateFunctions/AggregateFunctionNothing.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionNothing.cpp b/src/AggregateFunctions/AggregateFunctionNothing.cpp index 4781ea66a73..f5bd117d7c7 100644 --- a/src/AggregateFunctions/AggregateFunctionNothing.cpp +++ b/src/AggregateFunctions/AggregateFunctionNothing.cpp @@ -1,9 +1,7 @@ #include #include #include -#include #include -#include namespace DB @@ -13,7 +11,6 @@ struct Settings; AggregateFunctionPtr createAggregateFunctionNothing(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { assertNoParameters(name, parameters); - return std::make_shared(argument_types, parameters); } From bd002a6d507369fb3e68b0c901127e824774a157 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Jan 2022 03:58:18 +0300 Subject: [PATCH 0696/1260] Update AggregateFunctionNothing.cpp --- .../AggregateFunctionNothing.cpp | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionNothing.cpp b/src/AggregateFunctions/AggregateFunctionNothing.cpp index f5bd117d7c7..ca3d1f58efd 100644 --- a/src/AggregateFunctions/AggregateFunctionNothing.cpp +++ b/src/AggregateFunctions/AggregateFunctionNothing.cpp @@ -6,18 +6,16 @@ namespace DB { -struct Settings; -AggregateFunctionPtr createAggregateFunctionNothing(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) -{ - assertNoParameters(name, parameters); - return std::make_shared(argument_types, parameters); -} +struct Settings; void registerAggregateFunctionNothing(AggregateFunctionFactory & factory) { - AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = false }; - factory.registerFunction("nothing", { createAggregateFunctionNothing, properties }); + factory.registerFunction("nothing", [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) + { + assertNoParameters(name, parameters); + return std::make_shared(argument_types, parameters); + }); } } From fbadbc8cba21c6ab331c56012b6ababe1dc091a3 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Jan 2022 03:59:29 +0300 Subject: [PATCH 0697/1260] Update AggregateFunctionNothing.h --- src/AggregateFunctions/AggregateFunctionNothing.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionNothing.h b/src/AggregateFunctions/AggregateFunctionNothing.h index 22e296a96a4..210ea3dfaca 100644 --- a/src/AggregateFunctions/AggregateFunctionNothing.h +++ b/src/AggregateFunctions/AggregateFunctionNothing.h @@ -28,7 +28,7 @@ public: DataTypePtr getReturnType() const override { - return argument_types.front(); + return argument_types.empty() ? std::make_shared(std::make_shared()) : argument_types.front(); } bool allocatesMemoryInArena() const override { return false; } From 7feed04c9a92b143cd605dd34e3bbd9dad896fc1 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Jan 2022 03:59:45 +0300 Subject: [PATCH 0698/1260] Update AggregateFunctionNothing.h --- src/AggregateFunctions/AggregateFunctionNothing.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionNothing.h b/src/AggregateFunctions/AggregateFunctionNothing.h index 210ea3dfaca..645ea7c3f8a 100644 --- a/src/AggregateFunctions/AggregateFunctionNothing.h +++ b/src/AggregateFunctions/AggregateFunctionNothing.h @@ -64,13 +64,11 @@ public: { } - void serialize(ConstAggregateDataPtr __restrict, WriteBuffer & buf, std::optional) const override { writeChar('\0', buf); } - void deserialize(AggregateDataPtr, ReadBuffer & buf, std::optional, Arena *) const override { [[maybe_unused]] char symbol; From 4b109e53998a06efc4ff20bdbac93a71c4b31405 Mon Sep 17 00:00:00 2001 From: Stig Bakken Date: Mon, 20 Dec 2021 11:12:10 +0100 Subject: [PATCH 0699/1260] Add integration test for EXPLAIN TABLE OVERRIDE --- .../materialize_with_ddl.py | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py index 7265105c8df..1528103e1cb 100644 --- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py @@ -1079,9 +1079,41 @@ def table_overrides(clickhouse_node, mysql_node, service_name): check_query(clickhouse_node, "SELECT count() FROM table_overrides.t1", "1000\n") mysql_node.query("INSERT INTO table_overrides.t1 VALUES(1001, '2021-10-01 00:00:00', 42.0)") check_query(clickhouse_node, "SELECT count() FROM table_overrides.t1", "1001\n") + + explain_with_table_func = f"EXPLAIN TABLE OVERRIDE mysql('{service_name}:3306', 'table_overrides', 't1', 'root', 'clickhouse')" + + for what in ['ORDER BY', 'PRIMARY KEY', 'SAMPLE BY', 'PARTITION BY', 'TTL']: + with pytest.raises(QueryRuntimeException) as exc: + clickhouse_node.query(f"{explain_with_table_func} {what} temperature") + assert f'{what} override refers to nullable column `temperature`' in \ + str(exc.value) + assert f"{what} uses columns: `temperature` Nullable(Float32)" in \ + clickhouse_node.query(f"{explain_with_table_func} {what} assumeNotNull(temperature)") + + for testcase in [ + ('COLUMNS (temperature Nullable(Float32) MATERIALIZED 1.0)', + 'column `temperature`: modifying default specifier is not allowed'), + ('COLUMNS (sensor_id UInt64 ALIAS 42)', + 'column `sensor_id`: modifying default specifier is not allowed') + ]: + with pytest.raises(QueryRuntimeException) as exc: + clickhouse_node.query(f"{explain_with_table_func} {testcase[0]}") + assert testcase[1] in str(exc.value) + + for testcase in [ + ('COLUMNS (temperature Nullable(Float64))', + 'Modified columns: `temperature` Nullable(Float32) -> Nullable(Float64)'), + ('COLUMNS (temp_f Nullable(Float32) ALIAS if(temperature IS NULL, NULL, (temperature * 9.0 / 5.0) + 32),\ + temp_k Nullable(Float32) ALIAS if(temperature IS NULL, NULL, temperature + 273.15))', + 'Added columns: `temp_f` Nullable(Float32), `temp_k` Nullable(Float32)') + ]: + assert testcase[1] in clickhouse_node.query( + f"{explain_with_table_func} {testcase[0]}") + clickhouse_node.query("DROP DATABASE IF EXISTS table_overrides") mysql_node.query("DROP DATABASE IF EXISTS table_overrides") + def materialized_database_support_all_kinds_of_mysql_datatype(clickhouse_node, mysql_node, service_name): mysql_node.query("DROP DATABASE IF EXISTS test_database_datatype") clickhouse_node.query("DROP DATABASE IF EXISTS test_database_datatype") From bc1ac93d4d339b4c7839e98ffcc3554a3ea0ee24 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Jan 2022 14:19:16 +0300 Subject: [PATCH 0700/1260] Update AggregateFunctionNothing.cpp --- src/AggregateFunctions/AggregateFunctionNothing.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionNothing.cpp b/src/AggregateFunctions/AggregateFunctionNothing.cpp index ca3d1f58efd..b476806da08 100644 --- a/src/AggregateFunctions/AggregateFunctionNothing.cpp +++ b/src/AggregateFunctions/AggregateFunctionNothing.cpp @@ -1,7 +1,6 @@ #include #include -#include -#include +#include namespace DB From 6e9684149bec82b0758dfbb08b92619e7414f7dc Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Jan 2022 16:43:58 +0300 Subject: [PATCH 0701/1260] Update wide_integer_impl.h --- base/base/wide_integer_impl.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/base/base/wide_integer_impl.h b/base/base/wide_integer_impl.h index 23d57116d74..84d3928f993 100644 --- a/base/base/wide_integer_impl.h +++ b/base/base/wide_integer_impl.h @@ -1026,6 +1026,8 @@ constexpr integer::integer(std::initializer_list il) noexcept items[i] = *it; ++it; } + else + items[i] = 0; } } } From 8608dbb8e79162df2a3c772386d572b49602ee58 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Jan 2022 16:45:02 +0300 Subject: [PATCH 0702/1260] Fix CLion --- ...ggregateFunctionContingencyCoefficient.cpp | 35 ++++++++++--------- 1 file changed, 18 insertions(+), 17 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp b/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp index f5aeb68748f..52fb7ba98fe 100644 --- a/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp +++ b/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp @@ -11,28 +11,29 @@ namespace DB namespace { - struct ContingencyData : CrossTabData +struct ContingencyData : CrossTabData +{ + static const char * getName() { return "contingency"; } + + Float64 getResult() const { - static const char * getName() { return "contingency"; } + if (count < 2) + return std::numeric_limits::quiet_NaN(); - Float64 getResult() const + Float64 phi = 0.0; + for (const auto & [key, value_ab] : count_ab) { - if (count < 2) - return std::numeric_limits::quiet_NaN(); + Float64 value_a = count_a.at(key.items[0]); + Float64 value_b = count_b.at(key.items[1]); - Float64 phi = 0.0; - for (const auto & [key, value_ab] : count_ab) - { - Float64 value_a = count_a.at(key.items[0]); - Float64 value_b = count_b.at(key.items[1]); - - phi += value_ab * value_ab / (value_a * value_b) * count - 2 * value_ab + (value_a * value_b) / count; - } - phi /= count; - - return sqrt(phi / (phi + count)); + phi += value_ab * value_ab / (value_a * value_b) * count - 2 * value_ab + (value_a * value_b) / count; } - }; + phi /= count; + + return sqrt(phi / (phi + count)); + } +}; + } void registerAggregateFunctionContingency(AggregateFunctionFactory & factory) From 0c5485fd3af9cdc57cc7c0f6f2b39cce1ad9bd44 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Jan 2022 16:45:15 +0300 Subject: [PATCH 0703/1260] Update AggregateFunctionContingencyCoefficient.cpp --- .../AggregateFunctionContingencyCoefficient.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp b/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp index 52fb7ba98fe..77510f361f9 100644 --- a/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp +++ b/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp @@ -39,7 +39,7 @@ struct ContingencyData : CrossTabData void registerAggregateFunctionContingency(AggregateFunctionFactory & factory) { factory.registerFunction(ContingencyData::getName(), - [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) + [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { assertNoParameters(name, parameters); return std::make_shared>(argument_types); From 04a39317c18b3953cd57a9e469503dbfdd86b52d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 3 Jan 2022 21:39:04 +0300 Subject: [PATCH 0704/1260] Add check --- .../AggregateFunctionContingencyCoefficient.cpp | 6 +++++- src/AggregateFunctions/AggregateFunctionCramersV.cpp | 1 + .../AggregateFunctionCramersVBiasCorrected.cpp | 1 + src/AggregateFunctions/AggregateFunctionTheilsU.cpp | 3 ++- 4 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp b/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp index 77510f361f9..fa35c134a73 100644 --- a/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp +++ b/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp @@ -13,7 +13,10 @@ namespace struct ContingencyData : CrossTabData { - static const char * getName() { return "contingency"; } + static const char * getName() + { + return "contingency"; + } Float64 getResult() const { @@ -41,6 +44,7 @@ void registerAggregateFunctionContingency(AggregateFunctionFactory & factory) factory.registerFunction(ContingencyData::getName(), [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { + assertBinary(name, argument_types); assertNoParameters(name, parameters); return std::make_shared>(argument_types); }); diff --git a/src/AggregateFunctions/AggregateFunctionCramersV.cpp b/src/AggregateFunctions/AggregateFunctionCramersV.cpp index 48dad648d24..de070236043 100644 --- a/src/AggregateFunctions/AggregateFunctionCramersV.cpp +++ b/src/AggregateFunctions/AggregateFunctionCramersV.cpp @@ -47,6 +47,7 @@ void registerAggregateFunctionCramersV(AggregateFunctionFactory & factory) factory.registerFunction(CramersVData::getName(), [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { + assertBinary(name, argument_types); assertNoParameters(name, parameters); return std::make_shared>(argument_types); }); diff --git a/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrected.cpp b/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrected.cpp index f4d06991fb9..df0fd774da7 100644 --- a/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrected.cpp +++ b/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrected.cpp @@ -50,6 +50,7 @@ void registerAggregateFunctionCramersVBiasCorrected(AggregateFunctionFactory & f factory.registerFunction(CramersVBiasCorrectedData::getName(), [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { + assertBinary(name, argument_types); assertNoParameters(name, parameters); return std::make_shared>(argument_types); }); diff --git a/src/AggregateFunctions/AggregateFunctionTheilsU.cpp b/src/AggregateFunctions/AggregateFunctionTheilsU.cpp index 02a9b0956b1..96772a0daa8 100644 --- a/src/AggregateFunctions/AggregateFunctionTheilsU.cpp +++ b/src/AggregateFunctions/AggregateFunctionTheilsU.cpp @@ -50,8 +50,9 @@ struct TheilsUData : CrossTabData void registerAggregateFunctionTheilsU(AggregateFunctionFactory & factory) { factory.registerFunction(TheilsUData::getName(), - [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) + [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { + assertBinary(name, argument_types); assertNoParameters(name, parameters); return std::make_shared>(argument_types); }); From ac84015f01976753215abcada08fc1f6ff8274fd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 3 Jan 2022 22:18:57 +0300 Subject: [PATCH 0705/1260] Make at least some sense --- ...ggregateFunctionContingencyCoefficient.cpp | 11 +---- .../AggregateFunctionCramersV.cpp | 14 +------ ...AggregateFunctionCramersVBiasCorrected.cpp | 19 ++++----- src/AggregateFunctions/CrossTab.h | 41 +++++++++++++++++++ 4 files changed, 50 insertions(+), 35 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp b/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp index fa35c134a73..619abbb8a61 100644 --- a/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp +++ b/src/AggregateFunctions/AggregateFunctionContingencyCoefficient.cpp @@ -23,16 +23,7 @@ struct ContingencyData : CrossTabData if (count < 2) return std::numeric_limits::quiet_NaN(); - Float64 phi = 0.0; - for (const auto & [key, value_ab] : count_ab) - { - Float64 value_a = count_a.at(key.items[0]); - Float64 value_b = count_b.at(key.items[1]); - - phi += value_ab * value_ab / (value_a * value_b) * count - 2 * value_ab + (value_a * value_b) / count; - } - phi /= count; - + Float64 phi = getPhiSquared(); return sqrt(phi / (phi + count)); } }; diff --git a/src/AggregateFunctions/AggregateFunctionCramersV.cpp b/src/AggregateFunctions/AggregateFunctionCramersV.cpp index de070236043..07b691141bc 100644 --- a/src/AggregateFunctions/AggregateFunctionCramersV.cpp +++ b/src/AggregateFunctions/AggregateFunctionCramersV.cpp @@ -23,20 +23,8 @@ struct CramersVData : CrossTabData if (count < 2) return std::numeric_limits::quiet_NaN(); - Float64 phi = 0.0; - for (const auto & [key, value_ab] : count_ab) - { - Float64 value_a = count_a.at(key.items[0]); - Float64 value_b = count_b.at(key.items[1]); - - phi += value_ab * value_ab / (value_a * value_b) * count - 2 * value_ab + (value_a * value_b) / count; - } - - phi /= count; UInt64 q = std::min(count_a.size(), count_b.size()); - phi /= q - 1; - - return sqrt(phi); + return sqrt(getPhiSquared() / (q - 1)); } }; diff --git a/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrected.cpp b/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrected.cpp index df0fd774da7..917869dcd9f 100644 --- a/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrected.cpp +++ b/src/AggregateFunctions/AggregateFunctionCramersVBiasCorrected.cpp @@ -23,22 +23,17 @@ struct CramersVBiasCorrectedData : CrossTabData if (count < 2) return std::numeric_limits::quiet_NaN(); - Float64 phi = 0.0; - for (const auto & [key, value_ab] : count_ab) - { - Float64 value_a = count_a.at(key.items[0]); - Float64 value_b = count_b.at(key.items[1]); + Float64 phi = getPhiSquared(); - phi += value_ab * value_ab / (value_a * value_b) * count - 2 * value_ab + (value_a * value_b) / count; - } + Float64 a_size_adjusted = count_a.size() - 1; + Float64 b_size_adjusted = count_b.size() - 1; + Float64 count_adjusted = count - 1; - phi /= count; + Float64 res = std::max(0.0, phi - a_size_adjusted * b_size_adjusted / count_adjusted); + Float64 correction_a = count_a.size() - a_size_adjusted * a_size_adjusted / count_adjusted; + Float64 correction_b = count_b.size() - b_size_adjusted * b_size_adjusted / count_adjusted; - Float64 res = std::max(0.0, phi - (static_cast(count_a.size()) - 1) * (static_cast(count_b.size()) - 1) / (count - 1)); - Float64 correction_a = count_a.size() - (static_cast(count_a.size()) - 1) * (static_cast(count_a.size()) - 1) / (count - 1); - Float64 correction_b = count_b.size() - (static_cast(count_b.size()) - 1) * (static_cast(count_b.size()) - 1) / (count - 1); res /= std::min(correction_a, correction_b) - 1; - return sqrt(res); } }; diff --git a/src/AggregateFunctions/CrossTab.h b/src/AggregateFunctions/CrossTab.h index cbb32f7478b..e01ebcf71ed 100644 --- a/src/AggregateFunctions/CrossTab.h +++ b/src/AggregateFunctions/CrossTab.h @@ -69,6 +69,47 @@ struct CrossTabData count_b.read(buf); count_ab.read(buf); } + + /** See https://en.wikipedia.org/wiki/Cram%C3%A9r%27s_V + * + * φ² is χ² divided by the sample size (count). + * χ² is the sum of squares of the normalized differences between the "expected" and "observed" statistics. + * ("Expected" in the case when one of the hypotheses is true). + * Something resembling the L2 distance. + * + * Note: statisticians use the name χ² for every statistic that has χ² distribution in many various contexts. + * + * Let's suppose that there is no association between the values a and b. + * Then the frequency (e.g. probability) of (a, b) pair is equal to the multiplied frequencies of a and b: + * count_ab / count = (count_a / count) * (count_b / count) + * count_ab = count_a * count_b / count + * + * Let's calculate the difference between the values that are supposed to be equal if there is no association between a and b: + * count_ab - count_a * count_b / count + * + * Let's sum the squares of the differences across all (a, b) pairs. + * Then divide by the second term for normalization: (count_a * count_b / count) + * + * This will be the χ² statistics. + * This statistics is used as a base for many other statistics. + */ + Float64 getPhiSquared() const + { + Float64 chi_squared = 0; + for (const auto & [key, value_ab] : count_ab) + { + Float64 value_a = count_a.at(key.items[0]); + Float64 value_b = count_b.at(key.items[1]); + + Float64 expected_value_ab = (value_a * value_b) / count; + + Float64 chi_squared_elem = value_ab - expected_value_ab; + chi_squared_elem = chi_squared_elem * chi_squared_elem / expected_value_ab; + + chi_squared += chi_squared_elem; + } + return chi_squared / count; + } }; From 8aa26fd04ac041fc7d891d6be6e65ffca76f292c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 3 Jan 2022 22:23:34 +0300 Subject: [PATCH 0706/1260] Fix UBSan --- src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h b/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h index b15633b12f5..84ec5315aae 100644 --- a/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h +++ b/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h @@ -6,6 +6,7 @@ #include #include +#include #include #include @@ -134,7 +135,7 @@ template class AggregateFunctionIntervalLengthSum final : public IAggregateFunctionDataHelper> { private: - static auto length(typename Data::Segment segment) + static NO_SANITIZE_UNDEFINED auto length(typename Data::Segment segment) { return segment.second - segment.first; } From 7bd4d1a8d8db2edb23950cb9705b2e434192e607 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 3 Jan 2022 22:24:03 +0300 Subject: [PATCH 0707/1260] Fix UBSan --- src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h b/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h index 84ec5315aae..443d76f47cb 100644 --- a/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h +++ b/src/AggregateFunctions/AggregateFunctionIntervalLengthSum.h @@ -135,7 +135,7 @@ template class AggregateFunctionIntervalLengthSum final : public IAggregateFunctionDataHelper> { private: - static NO_SANITIZE_UNDEFINED auto length(typename Data::Segment segment) + static auto NO_SANITIZE_UNDEFINED length(typename Data::Segment segment) { return segment.second - segment.first; } From 9114d127e04be4bda99810a255ea7f502ae76dc5 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 3 Jan 2022 15:48:22 -0400 Subject: [PATCH 0708/1260] default values for max_concurrent_queries settings --- .../settings.md | 23 +++++++++++++++---- 1 file changed, 18 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index bd164fa59f9..526aad2809f 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -681,7 +681,9 @@ Queries may be limited by other settings: [max_concurrent_insert_queries](#max-c Possible values: - Positive integer. -- 0 — Disabled. +- 0 — No limit. + +Default value: `100`. **Example** @@ -699,7 +701,9 @@ The maximum number of simultaneously processed insert queries. Possible values: - Positive integer. -- 0 — Disabled. +- 0 — No limit. + +Default value: `0`. **Example** @@ -717,7 +721,9 @@ The maximum number of simultaneously processed select queries. Possible values: - Positive integer. -- 0 — Disabled. +- 0 — No limit. + +Default value: `0`. **Example** @@ -732,7 +738,9 @@ The maximum number of simultaneously processed queries related to MergeTree tabl Possible values: - Positive integer. -- 0 — Disabled. +- 0 — No limit. + +Default value: `0`. **Example** @@ -748,7 +756,12 @@ Example: `max_concurrent_queries_for_all_users` can be set to 99 for all users a Modifying the setting for one query or user does not affect other queries. -Default value: `0` that means no limit. +Possible values: + +- Positive integer. +- 0 — No limit. + +Default value: `0`. **Example** From f764e26c7b7578854491e44ad560d69ad6daed09 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 3 Jan 2022 15:59:52 -0400 Subject: [PATCH 0709/1260] Update settings.md --- .../settings.md | 57 +++++++++++++++++-- 1 file changed, 53 insertions(+), 4 deletions(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 4a2da778a06..d2cc133e0c9 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -673,7 +673,7 @@ ClickHouse поддерживает динамическое изменение ## max_concurrent_queries {#max-concurrent-queries} -Определяет максимальное количество одновременно обрабатываемых запросов, связанных с таблицей семейства `MergeTree`. Запросы также могут быть ограничены настройками: [max_concurrent_queries_for_user](#max-concurrent-queries-for-user), [max_concurrent_queries_for_all_users](#max-concurrent-queries-for-all-users), [min_marks_to_honor_max_concurrent_queries](#min-marks-to-honor-max-concurrent-queries). +Определяет максимальное количество одновременно обрабатываемых запросов, связанных с таблицей семейства `MergeTree`. Запросы также могут быть ограничены настройками: [max_concurrent_insert_queries](#max-concurrent-insert-queries), [max_concurrent_select_queries](#max-concurrent-select-queries), [max_concurrent_queries_for_user](#max-concurrent-queries-for-user), [max_concurrent_queries_for_all_users](#max-concurrent-queries-for-all-users), [min_marks_to_honor_max_concurrent_queries](#min-marks-to-honor-max-concurrent-queries). !!! info "Примечание" Параметры этих настроек могут быть изменены во время выполнения запросов и вступят в силу немедленно. Запросы, которые уже запущены, выполнятся без изменений. @@ -681,7 +681,9 @@ ClickHouse поддерживает динамическое изменение Возможные значения: - Положительное целое число. -- 0 — выключена. +- 0 — нет лимита. + +Значение по умолчанию: `100`. **Пример** @@ -689,6 +691,46 @@ ClickHouse поддерживает динамическое изменение 100 ``` +## max_concurrent_insert_queries {#max-concurrent-insert-queries} + +Определяет максимальное количество одновременных `INSERT` запросов. + +!!! info "Примечание" + Параметры этих настроек могут быть изменены во время выполнения запросов и вступят в силу немедленно. Запросы, которые уже запущены, выполнятся без изменений. + +Возможные значения: + +- Положительное целое число. +- 0 — нет лимита. + +Значение по умолчанию: `0`. + +**Example** + +``` xml +100 +``` + +## max_concurrent_select_queries {#max-concurrent-select-queries} + +Определяет максимальное количество одновременных `SELECT` запросов. + +!!! info "Примечание" + Параметры этих настроек могут быть изменены во время выполнения запросов и вступят в силу немедленно. Запросы, которые уже запущены, выполнятся без изменений. + +Возможные значения: + +- Положительное целое число. +- 0 — нет лимита. + +Значение по умолчанию: `0`. + +**Example** + +``` xml +100 +``` + ## max_concurrent_queries_for_user {#max-concurrent-queries-for-user} Определяет максимальное количество одновременно обрабатываемых запросов, связанных с таблицей семейства `MergeTree`, для пользователя. @@ -696,7 +738,9 @@ ClickHouse поддерживает динамическое изменение Возможные значения: - Положительное целое число. -- 0 — выключена. +- 0 — нет лимита. + +Значение по умолчанию: `0`. **Пример** @@ -712,7 +756,12 @@ ClickHouse поддерживает динамическое изменение Изменение настройки для одного запроса или пользователя не влияет на другие запросы. -Значение по умолчанию: `0` — отсутствие ограничений. +Возможные значения: + +- Положительное целое число. +- 0 — нет лимита. + +Значение по умолчанию: `0`. **Пример** From bb713f1487e27b706b178b46aa8307deeffae39f Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 3 Jan 2022 16:01:18 -0400 Subject: [PATCH 0710/1260] Update settings.md --- .../en/operations/server-configuration-parameters/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 526aad2809f..3c643a447a3 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -693,7 +693,7 @@ Default value: `100`. ## max_concurrent_insert_queries {#max-concurrent-insert-queries} -The maximum number of simultaneously processed insert queries. +The maximum number of simultaneously processed `INSERT` queries. !!! info "Note" These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. @@ -713,7 +713,7 @@ Default value: `0`. ## max_concurrent_select_queries {#max-concurrent-select-queries} -The maximum number of simultaneously processed select queries. +The maximum number of simultaneously processed `SELECT` queries. !!! info "Note" These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. From 7261e121ddb81ebb27d091bcdcecc6b8623d9647 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 3 Jan 2022 16:19:54 -0400 Subject: [PATCH 0711/1260] Update string-search-functions.md --- docs/en/sql-reference/functions/string-search-functions.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index c62603a50b9..a0c0116a058 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -351,8 +351,6 @@ Checks whether the string matches the `pattern` regular expression. A `re2` regu Returns 0 if it does not match, or 1 if it matches. -Note that the backslash symbol (`\`) is used for escaping in the regular expression. The same symbol is used for escaping in string literals. So in order to escape the symbol in a regular expression, you must write two backslashes (\\) in a string literal. - The regular expression works with the string as if it is a set of bytes. The regular expression can’t contain null bytes. For patterns to search for substrings in a string, it is better to use LIKE or ‘position’, since they work much faster. From e80f4008fa0bf5454fc788e6723c1ec29cb4ce67 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 4 Jan 2022 00:50:41 +0300 Subject: [PATCH 0712/1260] Fix idiotic test --- .../00168_parallel_processing_on_replicas_part_1.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh b/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh index d025dae5b2e..699700bcd3e 100755 --- a/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh +++ b/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh @@ -48,6 +48,10 @@ SkipList=( for TESTPATH in "$CURDIR"/*.sql; do TESTNAME=$(basename $TESTPATH) + NUM=$(echo "${TESTNAME}" | grep -o -P '^\d+' | sed 's/^0*//') + if [[ "${NUM}" -ge 168 ]]; then + continue + fi if [[ " ${SkipList[*]} " =~ ${TESTNAME} ]]; then echo "Skipping $TESTNAME " From 05e09e60614898f5720a9631b8839ee615cd678f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 4 Jan 2022 14:30:01 +0800 Subject: [PATCH 0713/1260] fix typo --- src/Storages/Cache/ExternalDataSourceCache.cpp | 2 +- src/Storages/Cache/ExternalDataSourceCache.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp index a168d5e5c3a..5a11e23e2c0 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.cpp +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -81,7 +81,7 @@ off_t RemoteReadBuffer::seek(off_t offset, int whence) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot call seek() in this buffer. It's a bug!"); /* * Need to wait here. For example, the current file has been download at position X, but here we try to seek to - * postition Y (Y > X), it would fail. + * position Y (Y > X), it would fail. */ auto & file_buffer = local_file_holder->file_buffer; local_file_holder->file_cache_controller->waitMoreData(offset, offset + file_buffer->internalBuffer().size()); diff --git a/src/Storages/Cache/ExternalDataSourceCache.h b/src/Storages/Cache/ExternalDataSourceCache.h index 4af7cbc51c8..d5895a5f7c0 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.h +++ b/src/Storages/Cache/ExternalDataSourceCache.h @@ -43,7 +43,7 @@ class RemoteReadBuffer : public BufferWithOwnMemory public: explicit RemoteReadBuffer(size_t buff_size); ~RemoteReadBuffer() override = default; - static std::unique_ptr create(ContextPtr contex, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer, size_t buff_size); + static std::unique_ptr create(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer, size_t buff_size); bool nextImpl() override; off_t seek(off_t off, int whence) override; From 8e675e9e0ad54584199a0016f2da423ccb7ba022 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 4 Jan 2022 15:06:19 +0800 Subject: [PATCH 0714/1260] fix building --- programs/server/Server.cpp | 4 ++-- .../Cache/RemoteFileMetadataFactory.cpp | 20 ++++++------------ .../Cache/RemoteFileMetadataFactory.h | 10 +++++---- .../Cache/registerRemoteFileMetadatas.cpp | 21 +++++++++++++++++++ .../Cache/registerRemoteFileMetadatas.h | 6 ++++++ src/Storages/Hive/StorageHiveMetadata.cpp | 12 ++++++----- src/Storages/Hive/StorageHiveMetadata.h | 10 ++++++++- 7 files changed, 57 insertions(+), 26 deletions(-) create mode 100644 src/Storages/Cache/registerRemoteFileMetadatas.cpp create mode 100644 src/Storages/Cache/registerRemoteFileMetadatas.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index fab6b977114..5fc3f9aa967 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -59,7 +59,7 @@ #include #include #include -#include +#include #include #include #include @@ -527,7 +527,7 @@ int Server::main(const std::vector & /*args*/) registerDictionaries(); registerDisks(); registerFormats(); - registerRemoteFileMatadataCreators(); + registerRemoteFileMetadatas(); CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); diff --git a/src/Storages/Cache/RemoteFileMetadataFactory.cpp b/src/Storages/Cache/RemoteFileMetadataFactory.cpp index 672e48627d4..a123d67088b 100644 --- a/src/Storages/Cache/RemoteFileMetadataFactory.cpp +++ b/src/Storages/Cache/RemoteFileMetadataFactory.cpp @@ -3,7 +3,6 @@ namespace DB { - namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -18,26 +17,19 @@ RemoteFileMetadataFactory & RemoteFileMetadataFactory::instance() IRemoteFileMetadataPtr RemoteFileMetadataFactory::get(const String & name) { - auto it = class_creators.find(name); - if (it == class_creators.end()) + auto it = remote_file_metadatas.find(name); + if (it == remote_file_metadatas.end()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not found metadata class:{}", name); return (it->second)(); } -void RemoteFileMetadataFactory::registerRemoteFileMatadataCreator(const String & name, MetadataCreator creator) +void RemoteFileMetadataFactory::registerRemoteFileMatadata(const String & name, MetadataCreator creator) { - auto it = class_creators.find(name); - if (it != class_creators.end()) + auto it = remote_file_metadatas.find(name); + if (it != remote_file_metadatas.end()) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Metadata class ({}) has already been registered.", name); } - class_creators[name] = creator; -} - -extern void registerStorageHiveMetadataCreator(); - -void registerRemoteFileMatadataCreators() -{ - registerStorageHiveMetadataCreator(); + remote_file_metadatas[name] = creator; } } diff --git a/src/Storages/Cache/RemoteFileMetadataFactory.h b/src/Storages/Cache/RemoteFileMetadataFactory.h index 5a08e469a6b..2cd46ca0682 100644 --- a/src/Storages/Cache/RemoteFileMetadataFactory.h +++ b/src/Storages/Cache/RemoteFileMetadataFactory.h @@ -10,18 +10,20 @@ class RemoteFileMetadataFactory : private boost::noncopyable { public: using MetadataCreator = std::function; + ~RemoteFileMetadataFactory() = default; static RemoteFileMetadataFactory & instance(); + IRemoteFileMetadataPtr get(const String & name); - void registerRemoteFileMatadataCreator(const String &name, MetadataCreator creator); + + void registerRemoteFileMatadata(const String &name, MetadataCreator creator); + protected: RemoteFileMetadataFactory() = default; private: - std::unordered_map class_creators; + std::unordered_map remote_file_metadatas; }; -void registerRemoteFileMatadataCreators(); - } diff --git a/src/Storages/Cache/registerRemoteFileMetadatas.cpp b/src/Storages/Cache/registerRemoteFileMetadatas.cpp new file mode 100644 index 00000000000..225af3026cf --- /dev/null +++ b/src/Storages/Cache/registerRemoteFileMetadatas.cpp @@ -0,0 +1,21 @@ +#include +#include +#include + +namespace DB +{ + +#if USE_HIVE +void registerStorageHiveMetadata(RemoteFileMetadataFactory & factory); +#endif + +void registerRemoteFileMetadatas() +{ + auto & factory = RemoteFileMetadataFactory::instance(); + +#if USE_HIVE + registerStorageHiveMetadata(factory); +#endif +} + +} diff --git a/src/Storages/Cache/registerRemoteFileMetadatas.h b/src/Storages/Cache/registerRemoteFileMetadatas.h new file mode 100644 index 00000000000..a7818b663c9 --- /dev/null +++ b/src/Storages/Cache/registerRemoteFileMetadatas.h @@ -0,0 +1,6 @@ +#pragma once + +namespace DB +{ +void registerRemoteFileMetadatas(); +} diff --git a/src/Storages/Hive/StorageHiveMetadata.cpp b/src/Storages/Hive/StorageHiveMetadata.cpp index 666caf64d34..0aed7c01064 100644 --- a/src/Storages/Hive/StorageHiveMetadata.cpp +++ b/src/Storages/Hive/StorageHiveMetadata.cpp @@ -1,4 +1,7 @@ #include + +#if USE_HIVE + #include #include #include @@ -6,7 +9,6 @@ namespace DB { -StorageHiveMetadata::~StorageHiveMetadata() = default; String StorageHiveMetadata::toString() const { @@ -41,12 +43,12 @@ String StorageHiveMetadata::getVersion() const return std::to_string(last_modification_timestamp); } -void registerStorageHiveMetadataCreator() +void registerStorageHiveMetadata(RemoteFileMetadataFactory & factory) { - auto & factory = RemoteFileMetadataFactory::instance(); auto creator = []() -> IRemoteFileMetadataPtr { return std::make_shared(); }; - factory.registerRemoteFileMatadataCreator("StorageHiveMetadata", creator); + factory.registerRemoteFileMatadata("StorageHiveMetadata", creator); } - } +#endif + diff --git a/src/Storages/Hive/StorageHiveMetadata.h b/src/Storages/Hive/StorageHiveMetadata.h index 3400f1324f6..d385274588f 100644 --- a/src/Storages/Hive/StorageHiveMetadata.h +++ b/src/Storages/Hive/StorageHiveMetadata.h @@ -1,7 +1,13 @@ #pragma once + +#include + +#if USE_HIVE + #include namespace DB { + class StorageHiveMetadata : public IRemoteFileMetadata { public: @@ -20,7 +26,7 @@ public: last_modification_timestamp = last_modification_timestamp_; } - ~StorageHiveMetadata() override; + ~StorageHiveMetadata() override = default; String getName() const override { return "StorageHiveMetadata"; } String getSchema() const { return schema; } @@ -33,4 +39,6 @@ private: String schema; String cluster; }; + } +#endif From 266ca905759fd1e4d3eb3e39fdd970c62f789a03 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 4 Jan 2022 15:17:43 +0800 Subject: [PATCH 0715/1260] fix fast test --- src/Storages/Cache/registerRemoteFileMetadatas.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Cache/registerRemoteFileMetadatas.cpp b/src/Storages/Cache/registerRemoteFileMetadatas.cpp index 225af3026cf..39705b810b7 100644 --- a/src/Storages/Cache/registerRemoteFileMetadatas.cpp +++ b/src/Storages/Cache/registerRemoteFileMetadatas.cpp @@ -11,7 +11,7 @@ void registerStorageHiveMetadata(RemoteFileMetadataFactory & factory); void registerRemoteFileMetadatas() { - auto & factory = RemoteFileMetadataFactory::instance(); + [[maybe_unused]] auto & factory = RemoteFileMetadataFactory::instance(); #if USE_HIVE registerStorageHiveMetadata(factory); From 45c86757bf998db97160be0301a22b43a62dc3a5 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 4 Jan 2022 15:16:24 +0800 Subject: [PATCH 0716/1260] change LRUCache to LRUResourceCache --- src/Common/LRUCache.h | 135 ++++-------------- src/Common/LRUResourceCache.h | 2 +- .../Cache/ExternalDataSourceCache.cpp | 101 +++++++------ src/Storages/Cache/ExternalDataSourceCache.h | 41 +++--- src/Storages/Cache/RemoteCacheController.cpp | 49 ++----- src/Storages/Cache/RemoteCacheController.h | 34 ++--- src/Storages/Cache/RemoteFileCachePolicy.h | 18 +-- src/Storages/Hive/HiveCommon.cpp | 7 +- 8 files changed, 120 insertions(+), 267 deletions(-) diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 9a986568db5..480a03ab399 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -6,11 +6,13 @@ #include #include #include + #include namespace DB { + template struct TrivialWeightFunction { @@ -20,43 +22,19 @@ struct TrivialWeightFunction } }; -template -struct TrivialLRUCacheEvictPolicy -{ - inline bool canRelease(std::shared_ptr) const - { - return true; - } - - inline void release(std::shared_ptr) - { - } -}; - /// Thread-safe cache that evicts entries which are not used for a long time. /// WeightFunction is a functor that takes Mapped as a parameter and returns "weight" (approximate size) /// of that value. /// Cache starts to evict entries when their total weight exceeds max_size. /// Value weight should not change after insertion. -template , - typename WeightFunction = TrivialWeightFunction, - typename EvictPolicy = TrivialLRUCacheEvictPolicy> +template , typename WeightFunction = TrivialWeightFunction> class LRUCache { public: using Key = TKey; using Mapped = TMapped; using MappedPtr = std::shared_ptr; - struct Result - { - MappedPtr value; - bool cache_miss = true; - // set_successful is not trustworthy for getOrSet, because removeOverflow is called right after putting key in cache - bool set_successful = false; - }; /** Initialize LRUCache with max_size and max_elements_size. * max_elements_size == 0 means no elements size restrictions. @@ -82,27 +60,10 @@ public: void set(const Key & key, const MappedPtr & mapped) { std::lock_guard lock(mutex); + setImpl(key, mapped, lock); } - /** - * trySet() will fail (return false) if there is no space left and no keys could be evicted. - * Eviction permission of each key is defined by EvictPolicy. In default policy there is no restriction. - */ - bool trySet(const Key & key, const MappedPtr & mapped) - { - std::lock_guard lock(mutex); - - return setImpl(key, mapped, lock); - } - - template - std::pair getOrSet(const Key & key, LoadFunc && load_func) - { - auto result = getOrTrySet(key, std::move(load_func)); - return std::make_pair(result.value, result.cache_miss); - } - void remove(const Key & key) { std::lock_guard lock(mutex); @@ -117,13 +78,14 @@ public: /// If the value for the key is in the cache, returns it. If it is not, calls load_func() to /// produce it, saves the result in the cache and returns it. - /// Only one of several concurrent threads calling getOrTrySet() will call load_func(), + /// Only one of several concurrent threads calling getOrSet() will call load_func(), /// others will wait for that call to complete and will use its result (this helps prevent cache stampede). /// Exceptions occurring in load_func will be propagated to the caller. Another thread from the /// set of concurrent threads will then try to call its load_func etc. /// + /// Returns std::pair of the cached value and a bool indicating whether the value was produced during this call. template - Result getOrTrySet(const Key &key, LoadFunc && load_func) + std::pair getOrSet(const Key & key, LoadFunc && load_func) { InsertTokenHolder token_holder; { @@ -133,7 +95,7 @@ public: if (val) { ++hits; - return {val, false, false}; + return std::make_pair(val, false); } auto & token = insert_tokens[key]; @@ -153,7 +115,7 @@ public: { /// Another thread already produced the value while we waited for token->mutex. ++hits; - return {token->value, false, false}; + return std::make_pair(token->value, false); } ++misses; @@ -163,39 +125,18 @@ public: /// Insert the new value only if the token is still in present in insert_tokens. /// (The token may be absent because of a concurrent reset() call). - bool is_value_loaded = false; - bool is_value_updated = false; + bool result = false; auto token_it = insert_tokens.find(key); if (token_it != insert_tokens.end() && token_it->second.get() == token) { - // setImpl() may fail, but the final behavior seems not be affected - // next call of getOrTrySet() will still call load_func() - is_value_updated = setImpl(key, token->value, cache_lock); - is_value_loaded = true; + setImpl(key, token->value, cache_lock); + result = true; } if (!token->cleaned_up) token_holder.cleanup(token_lock, cache_lock); - return {token->value, is_value_loaded, is_value_updated}; - } - - /// If key is not in cache or the element can be released, return is true. otherwise, return is false - bool tryRemove(const Key & key) - { - std::lock_guard loc(mutex); - auto it = cells.find(key); - if (it == cells.end()) - return true; - auto & cell = it->second; - if (!evict_policy.canRelease(cell.value)) - return false; - evict_policy.release(cell.value); - - current_size -= cell.size; - cells.erase(it); - queue.erase(cell.queue_iterator); - return true; + return std::make_pair(token->value, result); } void getStats(size_t & out_hits, size_t & out_misses) const @@ -330,7 +271,6 @@ private: std::atomic misses {0}; WeightFunction weight_function; - EvictPolicy evict_policy; MappedPtr getImpl(const Key & key, [[maybe_unused]] std::lock_guard & cache_lock) { @@ -348,7 +288,7 @@ private: return cell.value; } - bool setImpl(const Key & key, const MappedPtr & mapped, [[maybe_unused]] std::lock_guard & cache_lock) + void setImpl(const Key & key, const MappedPtr & mapped, [[maybe_unused]] std::lock_guard & cache_lock) { auto [it, inserted] = cells.emplace(std::piecewise_construct, std::forward_as_tuple(key), @@ -358,14 +298,6 @@ private: if (inserted) { - auto value_weight = mapped ? weight_function(*mapped) : 0; - if (!removeOverflow(value_weight)) - { - // cannot find enough space to put in the new value - cells.erase(it); - return false; - } - try { cell.queue_iterator = queue.insert(queue.end(), key); @@ -378,13 +310,6 @@ private: } else { - if (!evict_policy.canRelease(cell.value)) - { - // the old value is referred by someone, cannot release now - // in default policy, it is always true. - return false; - } - evict_policy.release(cell.value); // release the old value. this action is empty in default policy. current_size -= cell.size; queue.splice(queue.end(), queue, cell.queue_iterator); } @@ -393,18 +318,17 @@ private: cell.size = cell.value ? weight_function(*cell.value) : 0; current_size += cell.size; - return true; + removeOverflow(); } - bool removeOverflow(size_t required_size_to_remove = 0) + void removeOverflow() { size_t current_weight_lost = 0; size_t queue_size = cells.size(); - auto key_it = queue.begin(); - auto is_overflow = [&] { return (current_size + required_size_to_remove > max_size || (max_elements_size != 0 && queue_size > max_elements_size)); }; - while (is_overflow() && (queue_size > 1) && (key_it != queue.end())) + + while ((current_size > max_size || (max_elements_size != 0 && queue_size > max_elements_size)) && (queue_size > 1)) { - const Key & key = *key_it; + const Key & key = queue.front(); auto it = cells.find(key); if (it == cells.end()) @@ -414,23 +338,13 @@ private: } const auto & cell = it->second; - if (evict_policy.canRelease(cell.value))// in default, it is true - { - // always call release() before erasing an element - // in default, it's an empty action - evict_policy.release(cell.value); - current_size -= cell.size; - current_weight_lost += cell.size; + current_size -= cell.size; + current_weight_lost += cell.size; - cells.erase(it); - key_it = queue.erase(key_it); - --queue_size; - } - else - { - key_it++; - } + cells.erase(it); + queue.pop_front(); + --queue_size; } onRemoveOverflowWeightLoss(current_weight_lost); @@ -440,7 +354,6 @@ private: LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); abort(); } - return !is_overflow(); } /// Override this method if you want to track how much weight was lost in removeOverflow method. diff --git a/src/Common/LRUResourceCache.h b/src/Common/LRUResourceCache.h index b872d649a57..43b531c8eb1 100644 --- a/src/Common/LRUResourceCache.h +++ b/src/Common/LRUResourceCache.h @@ -324,7 +324,7 @@ private: { auto weight = value ? weight_function(*value) : 0; auto queue_size = cells.size() + 1; - auto loss_weight = 0; + size_t loss_weight = 0; auto is_overflow = [&] { return current_weight + weight - loss_weight > max_weight || (max_element_size != 0 && queue_size > max_element_size); }; diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp index 5a11e23e2c0..711dfeebcae 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.cpp +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -1,23 +1,23 @@ +#include #include #include -#include #include -#include +#include +#include +#include +#include #include #include -#include +#include #include +#include #include #include #include -#include -#include -#include -#include namespace ProfileEvents { - extern const Event ExternalDataSourceLocalCacheReadBytes; +extern const Event ExternalDataSourceLocalCacheReadBytes; } namespace DB { @@ -27,31 +27,26 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -LocalFileHolder::LocalFileHolder(std::shared_ptr cache_controller):file_cache_controller(cache_controller) +LocalFileHolder::LocalFileHolder(RemoteFileCacheType::MappedHolderPtr cache_controller) : file_cache_controller(std::move(cache_controller)) { - file_buffer = file_cache_controller->allocFile(); + file_buffer = file_cache_controller->value().allocFile(); if (!file_buffer) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Create file readbuffer failed. {}", - file_cache_controller->getLocalPath().string()); - -} - -LocalFileHolder::~LocalFileHolder() -{ - if (file_cache_controller) - file_cache_controller->deallocFile(std::move(file_buffer)); + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Create file readbuffer failed. {}", file_cache_controller->value().getLocalPath().string()); } RemoteReadBuffer::RemoteReadBuffer(size_t buff_size) : BufferWithOwnMemory(buff_size) { } -std::unique_ptr RemoteReadBuffer::create(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer, size_t buff_size) +std::unique_ptr RemoteReadBuffer::create( + ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr read_buffer, size_t buff_size) { auto remote_path = remote_file_metadata->remote_path; auto remote_read_buffer = std::make_unique(buff_size); - std::tie(remote_read_buffer->local_file_holder, read_buffer) = ExternalDataSourceCache::instance().createReader(context, remote_file_metadata, read_buffer); + std::tie(remote_read_buffer->local_file_holder, read_buffer) + = ExternalDataSourceCache::instance().createReader(context, remote_file_metadata, read_buffer); if (remote_read_buffer->local_file_holder == nullptr) return read_buffer; remote_read_buffer->remote_file_size = remote_file_metadata->file_size; @@ -62,14 +57,15 @@ bool RemoteReadBuffer::nextImpl() { auto start_offset = local_file_holder->file_buffer->getPosition(); auto end_offset = start_offset + local_file_holder->file_buffer->internalBuffer().size(); - local_file_holder->file_cache_controller->waitMoreData(start_offset, end_offset); + local_file_holder->file_cache_controller->value().waitMoreData(start_offset, end_offset); auto status = local_file_holder->file_buffer->next(); if (status) { - BufferBase::set(local_file_holder->file_buffer->buffer().begin(), - local_file_holder->file_buffer->buffer().size(), - local_file_holder->file_buffer->offset()); + BufferBase::set( + local_file_holder->file_buffer->buffer().begin(), + local_file_holder->file_buffer->buffer().size(), + local_file_holder->file_buffer->offset()); ProfileEvents::increment(ProfileEvents::ExternalDataSourceLocalCacheReadBytes, local_file_holder->file_buffer->available()); } return status; @@ -84,11 +80,9 @@ off_t RemoteReadBuffer::seek(off_t offset, int whence) * position Y (Y > X), it would fail. */ auto & file_buffer = local_file_holder->file_buffer; - local_file_holder->file_cache_controller->waitMoreData(offset, offset + file_buffer->internalBuffer().size()); + local_file_holder->file_cache_controller->value().waitMoreData(offset, offset + file_buffer->internalBuffer().size()); auto ret = file_buffer->seek(offset, whence); - BufferBase::set(file_buffer->buffer().begin(), - file_buffer->buffer().size(), - file_buffer->offset()); + BufferBase::set(file_buffer->buffer().begin(), file_buffer->buffer().size(), file_buffer->offset()); return ret; } @@ -124,7 +118,8 @@ void ExternalDataSourceCache::recoverTask() invalid_paths.emplace_back(path); continue; } - if (!lru_caches->trySet(path, cache_controller)) + auto cache_load_func = [&] { return cache_controller; }; + if (!lru_caches->getOrSet(path, cache_load_func)) { invalid_paths.emplace_back(path); } @@ -136,9 +131,7 @@ void ExternalDataSourceCache::recoverTask() LOG_INFO(log, "Recovered from directory:{}", root_dir); } -void ExternalDataSourceCache::initOnce( - ContextPtr context, - const String & root_dir_, size_t limit_size_, size_t bytes_read_before_flush_) +void ExternalDataSourceCache::initOnce(ContextPtr context, const String & root_dir_, size_t limit_size_, size_t bytes_read_before_flush_) { std::lock_guard lock(mutex); if (isInitialized()) @@ -149,7 +142,7 @@ void ExternalDataSourceCache::initOnce( log, "Initializing local cache for remote data sources. Local cache root path: {}, cache size limit: {}", root_dir_, limit_size_); root_dir = root_dir_; local_cache_bytes_read_before_flush = bytes_read_before_flush_; - lru_caches = std::make_unique(limit_size_); + lru_caches = std::make_unique(limit_size_); /// create if root_dir not exists if (!fs::exists(fs::path(root_dir))) @@ -157,22 +150,21 @@ void ExternalDataSourceCache::initOnce( fs::create_directories(fs::path(root_dir)); } - recover_task_holder = context->getSchedulePool().createTask("recover local cache metadata for remote files", [this]{ recoverTask(); }); + recover_task_holder = context->getSchedulePool().createTask("recover local cache metadata for remote files", [this] { recoverTask(); }); recover_task_holder->activateAndSchedule(); } String ExternalDataSourceCache::calculateLocalPath(IRemoteFileMetadataPtr metadata) const { // add version into the full_path, and not block to read the new version - String full_path = metadata->getName() + ":" + metadata->remote_path - + ":" + metadata->getVersion(); + String full_path = metadata->getName() + ":" + metadata->remote_path + ":" + metadata->getVersion(); UInt128 hashcode = sipHash128(full_path.c_str(), full_path.size()); String hashcode_str = getHexUIntLowercase(hashcode); return fs::path(root_dir) / hashcode_str.substr(0, 3) / hashcode_str; } -std::pair, std::unique_ptr> -ExternalDataSourceCache::createReader(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr & read_buffer) +std::pair, std::unique_ptr> ExternalDataSourceCache::createReader( + ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr & read_buffer) { // If something is wrong on startup, rollback to read from the original ReadBuffer if (!isInitialized()) @@ -189,36 +181,43 @@ ExternalDataSourceCache::createReader(ContextPtr context, IRemoteFileMetadataPtr if (cache) { // the remote file has been updated, need to redownload - if (!cache->isValid() || cache->isModified(remote_file_metadata)) + if (!cache->value().isValid() || cache->value().isModified(remote_file_metadata)) { LOG_TRACE( log, "Remote file ({}) has been updated. Last saved modification time: {}, actual last modification time: {}", remote_path, - std::to_string(cache->getLastModificationTimestamp()), + std::to_string(cache->value().getLastModificationTimestamp()), std::to_string(last_modification_timestamp)); - cache->markInvalid(); + cache->value().markInvalid(); + cache.reset(); } else { - return {std::make_unique(cache), nullptr}; + return {std::make_unique(std::move(cache)), nullptr}; } } if (!fs::exists(local_path)) fs::create_directories(local_path); - // cache is not found or is invalid - auto new_cache = std::make_shared(remote_file_metadata, local_path, local_cache_bytes_read_before_flush); - if (!lru_caches->trySet(local_path, new_cache)) + // cache is not found or is invalid, try to remove it at first + lru_caches->tryRemove(local_path); + + auto new_cache_controller + = std::make_shared(remote_file_metadata, local_path, local_cache_bytes_read_before_flush); + auto new_cache = lru_caches->getOrSet(local_path, [&] { return new_cache_controller; }); + if (!new_cache) { - LOG_ERROR(log, "Insert the new cache failed. new file size:{}, current total size:{}", - remote_file_metadata->file_size, - lru_caches->weight()); + LOG_ERROR( + log, + "Insert the new cache failed. new file size:{}, current total size:{}", + remote_file_metadata->file_size, + lru_caches->weight()); return {nullptr, std::move(read_buffer)}; } - new_cache->startBackgroundDownload(std::move(read_buffer), context->getSchedulePool()); - return {std::make_unique(new_cache), nullptr}; + new_cache->value().startBackgroundDownload(std::move(read_buffer), context->getSchedulePool()); + return {std::make_unique(std::move(new_cache)), nullptr}; } } diff --git a/src/Storages/Cache/ExternalDataSourceCache.h b/src/Storages/Cache/ExternalDataSourceCache.h index d5895a5f7c0..9e69c9f64fd 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.h +++ b/src/Storages/Cache/ExternalDataSourceCache.h @@ -1,40 +1,42 @@ #pragma once -#include +#include +#include #include -#include #include #include -#include +#include +#include #include -#include -#include -#include -#include -#include #include -#include -#include -#include +#include #include #include #include -#include -#include +#include +#include +#include #include -#include +#include #include #include +#include +#include +#include +#include +#include namespace DB { +using RemoteFileCacheType = LRUResourceCache; + class LocalFileHolder { public: - explicit LocalFileHolder(std::shared_ptr cache_controller); - ~LocalFileHolder(); + explicit LocalFileHolder(RemoteFileCacheType::MappedHolderPtr cache_controller); + ~LocalFileHolder() = default; - std::shared_ptr file_cache_controller; + RemoteFileCacheType::MappedHolderPtr file_cache_controller; std::unique_ptr file_buffer; }; @@ -55,11 +57,10 @@ private: size_t remote_file_size = 0; }; + class ExternalDataSourceCache : private boost::noncopyable { public: - using CacheType = LRUCache, - RemoteFileCacheWeightFunction, RemoteFileCacheEvictPolicy>; ~ExternalDataSourceCache(); // global instance static ExternalDataSourceCache & instance(); @@ -84,7 +85,7 @@ private: std::atomic initialized = false; std::atomic total_size; std::mutex mutex; - std::unique_ptr lru_caches; + std::unique_ptr lru_caches; Poco::Logger * log = &Poco::Logger::get("ExternalDataSourceCache"); diff --git a/src/Storages/Cache/RemoteCacheController.cpp b/src/Storages/Cache/RemoteCacheController.cpp index 0fffaf0d5e2..03c5ef17fe8 100644 --- a/src/Storages/Cache/RemoteCacheController.cpp +++ b/src/Storages/Cache/RemoteCacheController.cpp @@ -1,13 +1,13 @@ -#include -#include -#include +#include +#include #include #include -#include #include +#include +#include +#include #include #include -#include namespace DB { @@ -55,7 +55,11 @@ std::shared_ptr RemoteCacheController::recover(const std: readStringUntilEOF(metadata_content, file_readbuffer); if (!cache_controller->file_metadata_ptr->fromString(metadata_content)) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid metadata file({}) for meta class {}", local_path_.string(), cache_controller->metadata_class); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Invalid metadata file({}) for meta class {}", + local_path_.string(), + cache_controller->metadata_class); } cache_controller->current_offset = fs::file_size(local_path_ / "data.bin"); @@ -65,9 +69,7 @@ std::shared_ptr RemoteCacheController::recover(const std: } RemoteCacheController::RemoteCacheController( - IRemoteFileMetadataPtr file_metadata_, - const std::filesystem::path & local_path_, - size_t cache_bytes_before_flush_) + IRemoteFileMetadataPtr file_metadata_, const std::filesystem::path & local_path_, size_t cache_bytes_before_flush_) : file_metadata_ptr(file_metadata_) , local_path(local_path_) , valid(true) @@ -134,8 +136,7 @@ void RemoteCacheController::startBackgroundDownload(std::unique_ptr data_file_writer = std::make_unique((fs::path(local_path) / "data.bin").string()); flush(true); ReadBufferPtr in_readbuffer(in_readbuffer_.release()); - download_task_holder = thread_pool.createTask("download remote file", - [this, in_readbuffer]{ backgroundDownload(in_readbuffer); }); + download_task_holder = thread_pool.createTask("download remote file", [this, in_readbuffer] { backgroundDownload(in_readbuffer); }); download_task_holder->activateAndSchedule(); } @@ -197,6 +198,7 @@ RemoteCacheController::~RemoteCacheController() { if (download_task_holder) download_task_holder->deactivate(); + close(); } void RemoteCacheController::close() @@ -213,32 +215,7 @@ std::unique_ptr RemoteCacheController::allocFile() //settings.local_fs_method = LocalFSReadMethod::read; auto file_buffer = createReadBufferFromFileBase((local_path / "data.bin").string(), settings); - if (file_buffer) - { - std::lock_guard lock{mutex}; - opened_file_buffer_refs.insert(reinterpret_cast(file_buffer.get())); - } return file_buffer; } -void RemoteCacheController::deallocFile(std::unique_ptr file_buffer) -{ - if (!file_buffer) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Try to release a null file buffer for {}", local_path.string()); - } - auto buffer_ref = reinterpret_cast(file_buffer.get()); - std::lock_guard lock{mutex}; - auto it = opened_file_buffer_refs.find(buffer_ref); - if (it == opened_file_buffer_refs.end()) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Try to deallocate file with invalid handler remote path: {}, local path: {}", - file_metadata_ptr->remote_path, - local_path.string()); - } - opened_file_buffer_refs.erase(it); -} - } diff --git a/src/Storages/Cache/RemoteCacheController.h b/src/Storages/Cache/RemoteCacheController.h index c5dc4a510b6..4399a9e2b90 100644 --- a/src/Storages/Cache/RemoteCacheController.h +++ b/src/Storages/Cache/RemoteCacheController.h @@ -1,17 +1,17 @@ #pragma once -#include -#include +#include #include #include -#include +#include +#include #include -#include -#include #include #include #include -#include +#include #include +#include +#include namespace DB { @@ -22,35 +22,21 @@ public: { TO_DOWNLOAD = 0, DOWNLOADING = 1, - DOWNLOADED = 2, + DOWNLOADED = 2, }; RemoteCacheController( - IRemoteFileMetadataPtr file_metadata_, - const std::filesystem::path & local_path_, - size_t cache_bytes_before_flush_); + IRemoteFileMetadataPtr file_metadata_, const std::filesystem::path & local_path_, size_t cache_bytes_before_flush_); ~RemoteCacheController(); // recover from local disk - static std::shared_ptr - recover(const std::filesystem::path & local_path); + static std::shared_ptr recover(const std::filesystem::path & local_path); /** * Called by LocalCachedFileReader, must be used in pair * The second value of the return tuple is the local_path to store file. */ std::unique_ptr allocFile(); - void deallocFile(std::unique_ptr buffer); - - /** - * when allocFile be called, count++. deallocFile be called, count--. - * the local file could be deleted only count==0 - */ - inline bool closable() - { - std::lock_guard lock{mutex}; - return opened_file_buffer_refs.empty(); - } void close(); /** @@ -92,8 +78,6 @@ private: std::mutex mutex; std::condition_variable more_data_signal; - std::set opened_file_buffer_refs; // refer to a buffer address - String metadata_class; LocalFileStatus file_status = TO_DOWNLOAD; // for tracking download process IRemoteFileMetadataPtr file_metadata_ptr; diff --git a/src/Storages/Cache/RemoteFileCachePolicy.h b/src/Storages/Cache/RemoteFileCachePolicy.h index ebdb2c4bc48..43d29f85b8d 100644 --- a/src/Storages/Cache/RemoteFileCachePolicy.h +++ b/src/Storages/Cache/RemoteFileCachePolicy.h @@ -3,23 +3,7 @@ namespace DB { struct RemoteFileCacheWeightFunction { - size_t operator()(const RemoteCacheController & cache) const - { - return cache.getFileSize(); - } -}; - -struct RemoteFileCacheEvictPolicy -{ - bool canRelease(std::shared_ptr cache) const - { - return !cache || cache->closable(); - } - void release(std::shared_ptr cache) - { - if (cache) - cache->close(); - } + size_t operator()(const RemoteCacheController & cache) const { return cache.getFileSize(); } }; } diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 4612e006258..f0d479fc6dd 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -111,12 +111,7 @@ void HiveMetastoreClient::clearTableMetadata(const String & db_name, const Strin std::lock_guard lock{mutex}; HiveTableMetadataPtr metadata = table_metadata_cache.get(cache_key); if (metadata) - { - if (!table_metadata_cache.tryRemove(cache_key)) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Try to clear table metadata failed."); - } - } + table_metadata_cache.remove(cache_key); } void HiveMetastoreClient::setClient(std::shared_ptr client_) From 14bce62cc4edd71a1d67d81ee3d257a906884d21 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 4 Jan 2022 15:34:34 +0800 Subject: [PATCH 0717/1260] fix building --- src/Common/LRUResourceCache.h | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Common/LRUResourceCache.h b/src/Common/LRUResourceCache.h index 449ad56ac8b..40c283749d3 100644 --- a/src/Common/LRUResourceCache.h +++ b/src/Common/LRUResourceCache.h @@ -330,11 +330,12 @@ private: // key mustn't be in the cache Cell * set(const Key & insert_key, MappedPtr value) { - auto weight = value ? weight_function(*value) : 0; - auto queue_size = cells.size() + 1; - auto loss_weight = 0; - auto is_overflow = [&] { - return current_weight + weight - loss_weight > max_weight || (max_element_size != 0 && queue_size > max_element_size); + size_t weight = value ? weight_function(*value) : 0; + size_t queue_size = cells.size() + 1; + size_t loss_weight = 0; + auto is_overflow = [&] + { + return current_weight + weight > max_weight + loss_weight || (max_element_size != 0 && queue_size > max_element_size); }; auto key_it = queue.begin(); @@ -355,7 +356,7 @@ private: if (cell.reference_count == 0) { loss_weight += cell.weight; - queue_size -= 1; + queue_size--; to_release_keys.insert(key); } From e548dae8d95b8c37184270b82a30a306772de891 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 3 Jan 2022 19:05:46 +0300 Subject: [PATCH 0718/1260] Pass size of the file to the readers v2: s/size/file_size/g --- src/IO/AsynchronousReadBufferFromFile.cpp | 14 ++++--- src/IO/AsynchronousReadBufferFromFile.h | 33 ++++++++++++----- ...AsynchronousReadBufferFromFileDescriptor.h | 18 ++++++--- src/IO/ReadBufferFromFile.cpp | 13 ++++--- src/IO/ReadBufferFromFile.h | 37 ++++++++++++++----- src/IO/ReadBufferFromFileBase.cpp | 7 +++- src/IO/ReadBufferFromFileBase.h | 8 +++- src/IO/ReadBufferFromFileDescriptor.h | 20 ++++++++-- src/IO/createReadBufferFromFileBase.cpp | 8 ++-- 9 files changed, 111 insertions(+), 47 deletions(-) diff --git a/src/IO/AsynchronousReadBufferFromFile.cpp b/src/IO/AsynchronousReadBufferFromFile.cpp index 9327b80738d..969384cd91c 100644 --- a/src/IO/AsynchronousReadBufferFromFile.cpp +++ b/src/IO/AsynchronousReadBufferFromFile.cpp @@ -30,8 +30,10 @@ AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile( size_t buf_size, int flags, char * existing_memory, - size_t alignment) - : AsynchronousReadBufferFromFileDescriptor(std::move(reader_), priority_, -1, buf_size, existing_memory, alignment), file_name(file_name_) + size_t alignment, + std::optional file_size_) + : AsynchronousReadBufferFromFileDescriptor(std::move(reader_), priority_, -1, buf_size, existing_memory, alignment, file_size_) + , file_name(file_name_) { ProfileEvents::increment(ProfileEvents::FileOpen); @@ -62,10 +64,10 @@ AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile( const std::string & original_file_name, size_t buf_size, char * existing_memory, - size_t alignment) - : - AsynchronousReadBufferFromFileDescriptor(std::move(reader_), priority_, fd_, buf_size, existing_memory, alignment), - file_name(original_file_name.empty() ? "(fd = " + toString(fd_) + ")" : original_file_name) + size_t alignment, + std::optional file_size_) + : AsynchronousReadBufferFromFileDescriptor(std::move(reader_), priority_, fd_, buf_size, existing_memory, alignment, file_size_) + , file_name(original_file_name.empty() ? "(fd = " + toString(fd_) + ")" : original_file_name) { fd_ = -1; } diff --git a/src/IO/AsynchronousReadBufferFromFile.h b/src/IO/AsynchronousReadBufferFromFile.h index d9d5e43e0d4..96834350bab 100644 --- a/src/IO/AsynchronousReadBufferFromFile.h +++ b/src/IO/AsynchronousReadBufferFromFile.h @@ -14,17 +14,25 @@ protected: public: explicit AsynchronousReadBufferFromFile( - AsynchronousReaderPtr reader_, Int32 priority_, - const std::string & file_name_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, int flags = -1, - char * existing_memory = nullptr, size_t alignment = 0); + AsynchronousReaderPtr reader_, + Int32 priority_, + const std::string & file_name_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + int flags = -1, + char * existing_memory = nullptr, + size_t alignment = 0, + std::optional file_size_ = std::nullopt); /// Use pre-opened file descriptor. explicit AsynchronousReadBufferFromFile( - AsynchronousReaderPtr reader_, Int32 priority_, + AsynchronousReaderPtr reader_, + Int32 priority_, int & fd, /// Will be set to -1 if constructor didn't throw and ownership of file descriptor is passed to the object. const std::string & original_file_name = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, - char * existing_memory = nullptr, size_t alignment = 0); + char * existing_memory = nullptr, + size_t alignment = 0, + std::optional file_size_ = std::nullopt); ~AsynchronousReadBufferFromFile() override; @@ -48,11 +56,16 @@ private: public: AsynchronousReadBufferFromFileWithDescriptorsCache( - AsynchronousReaderPtr reader_, Int32 priority_, - const std::string & file_name_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, int flags = -1, - char * existing_memory = nullptr, size_t alignment = 0) - : AsynchronousReadBufferFromFileDescriptor(std::move(reader_), priority_, -1, buf_size, existing_memory, alignment), - file_name(file_name_) + AsynchronousReaderPtr reader_, + Int32 priority_, + const std::string & file_name_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + int flags = -1, + char * existing_memory = nullptr, + size_t alignment = 0, + std::optional file_size_ = std::nullopt) + : AsynchronousReadBufferFromFileDescriptor(std::move(reader_), priority_, -1, buf_size, existing_memory, alignment, file_size_) + , file_name(file_name_) { file = OpenedFileCache::instance().get(file_name, flags); fd = file->getFD(); diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.h b/src/IO/AsynchronousReadBufferFromFileDescriptor.h index 50d8f5819fe..88efdbee6e8 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.h +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.h @@ -35,10 +35,18 @@ protected: public: AsynchronousReadBufferFromFileDescriptor( - AsynchronousReaderPtr reader_, Int32 priority_, - int fd_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0) - : ReadBufferFromFileBase(buf_size, existing_memory, alignment), - reader(std::move(reader_)), priority(priority_), required_alignment(alignment), fd(fd_) + AsynchronousReaderPtr reader_, + Int32 priority_, + int fd_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0, + std::optional file_size_ = std::nullopt) + : ReadBufferFromFileBase(buf_size, existing_memory, alignment, file_size_) + , reader(std::move(reader_)) + , priority(priority_) + , required_alignment(alignment) + , fd(fd_) { prefetch_buffer.alignment = alignment; } @@ -64,7 +72,7 @@ public: void rewind(); private: - std::future readInto(char * data, size_t size); + std::future readInto(char * data, size_t file_size_); }; } diff --git a/src/IO/ReadBufferFromFile.cpp b/src/IO/ReadBufferFromFile.cpp index d0f94441622..4f601301686 100644 --- a/src/IO/ReadBufferFromFile.cpp +++ b/src/IO/ReadBufferFromFile.cpp @@ -28,8 +28,9 @@ ReadBufferFromFile::ReadBufferFromFile( size_t buf_size, int flags, char * existing_memory, - size_t alignment) - : ReadBufferFromFileDescriptor(-1, buf_size, existing_memory, alignment), file_name(file_name_) + size_t alignment, + std::optional file_size_) + : ReadBufferFromFileDescriptor(-1, buf_size, existing_memory, alignment, file_size_), file_name(file_name_) { ProfileEvents::increment(ProfileEvents::FileOpen); @@ -58,10 +59,10 @@ ReadBufferFromFile::ReadBufferFromFile( const std::string & original_file_name, size_t buf_size, char * existing_memory, - size_t alignment) - : - ReadBufferFromFileDescriptor(fd_, buf_size, existing_memory, alignment), - file_name(original_file_name.empty() ? "(fd = " + toString(fd_) + ")" : original_file_name) + size_t alignment, + std::optional file_size_) + : ReadBufferFromFileDescriptor(fd_, buf_size, existing_memory, alignment, file_size_) + , file_name(original_file_name.empty() ? "(fd = " + toString(fd_) + ")" : original_file_name) { fd_ = -1; } diff --git a/src/IO/ReadBufferFromFile.h b/src/IO/ReadBufferFromFile.h index 1a45e4c1829..ff19fa40fdf 100644 --- a/src/IO/ReadBufferFromFile.h +++ b/src/IO/ReadBufferFromFile.h @@ -23,15 +23,22 @@ protected: CurrentMetrics::Increment metric_increment{CurrentMetrics::OpenFileForRead}; public: - explicit ReadBufferFromFile(const std::string & file_name_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, int flags = -1, - char * existing_memory = nullptr, size_t alignment = 0); + explicit ReadBufferFromFile( + const std::string & file_name_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + int flags = -1, + char * existing_memory = nullptr, + size_t alignment = 0, + std::optional file_size_ = std::nullopt); /// Use pre-opened file descriptor. explicit ReadBufferFromFile( int & fd, /// Will be set to -1 if constructor didn't throw and ownership of file descriptor is passed to the object. const std::string & original_file_name = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, - char * existing_memory = nullptr, size_t alignment = 0); + char * existing_memory = nullptr, + size_t alignment = 0, + std::optional file_size_ = std::nullopt); ~ReadBufferFromFile() override; @@ -50,9 +57,14 @@ public: class ReadBufferFromFilePRead : public ReadBufferFromFile { public: - ReadBufferFromFilePRead(const std::string & file_name_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, int flags = -1, - char * existing_memory = nullptr, size_t alignment = 0) - : ReadBufferFromFile(file_name_, buf_size, flags, existing_memory, alignment) + ReadBufferFromFilePRead( + const std::string & file_name_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + int flags = -1, + char * existing_memory = nullptr, + size_t alignment = 0, + std::optional file_size_ = std::nullopt) + : ReadBufferFromFile(file_name_, buf_size, flags, existing_memory, alignment, file_size_) { use_pread = true; } @@ -68,10 +80,15 @@ private: OpenedFileCache::OpenedFilePtr file; public: - ReadBufferFromFilePReadWithDescriptorsCache(const std::string & file_name_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, int flags = -1, - char * existing_memory = nullptr, size_t alignment = 0) - : ReadBufferFromFileDescriptorPRead(-1, buf_size, existing_memory, alignment), - file_name(file_name_) + ReadBufferFromFilePReadWithDescriptorsCache( + const std::string & file_name_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + int flags = -1, + char * existing_memory = nullptr, + size_t alignment = 0, + std::optional file_size_ = std::nullopt) + : ReadBufferFromFileDescriptorPRead(-1, buf_size, existing_memory, alignment, file_size_) + , file_name(file_name_) { file = OpenedFileCache::instance().get(file_name, flags); fd = file->getFD(); diff --git a/src/IO/ReadBufferFromFileBase.cpp b/src/IO/ReadBufferFromFileBase.cpp index b598501a608..4db64755abf 100644 --- a/src/IO/ReadBufferFromFileBase.cpp +++ b/src/IO/ReadBufferFromFileBase.cpp @@ -7,8 +7,13 @@ ReadBufferFromFileBase::ReadBufferFromFileBase() : BufferWithOwnMemory file_size_) : BufferWithOwnMemory(buf_size, existing_memory, alignment) + , file_size(file_size_) { } diff --git a/src/IO/ReadBufferFromFileBase.h b/src/IO/ReadBufferFromFileBase.h index 731fd373e24..a051283b2bb 100644 --- a/src/IO/ReadBufferFromFileBase.h +++ b/src/IO/ReadBufferFromFileBase.h @@ -5,6 +5,7 @@ #include #include +#include #include #include @@ -22,7 +23,11 @@ class ReadBufferFromFileBase : public BufferWithOwnMemory { public: ReadBufferFromFileBase(); - ReadBufferFromFileBase(size_t buf_size, char * existing_memory, size_t alignment); + ReadBufferFromFileBase( + size_t buf_size, + char * existing_memory, + size_t alignment, + std::optional file_size_ = std::nullopt); ~ReadBufferFromFileBase() override; virtual std::string getFileName() const = 0; @@ -44,6 +49,7 @@ public: } protected: + std::optional file_size; ProfileCallback profile_callback; clockid_t clock_type{}; }; diff --git a/src/IO/ReadBufferFromFileDescriptor.h b/src/IO/ReadBufferFromFileDescriptor.h index 8dbe8707bdb..188cdd709b5 100644 --- a/src/IO/ReadBufferFromFileDescriptor.h +++ b/src/IO/ReadBufferFromFileDescriptor.h @@ -27,8 +27,15 @@ protected: std::string getFileName() const override; public: - ReadBufferFromFileDescriptor(int fd_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0) - : ReadBufferFromFileBase(buf_size, existing_memory, alignment), required_alignment(alignment), fd(fd_) + ReadBufferFromFileDescriptor( + int fd_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0, + std::optional file_size_ = std::nullopt) + : ReadBufferFromFileBase(buf_size, existing_memory, alignment, file_size_) + , required_alignment(alignment) + , fd(fd_) { } @@ -63,8 +70,13 @@ private: class ReadBufferFromFileDescriptorPRead : public ReadBufferFromFileDescriptor { public: - ReadBufferFromFileDescriptorPRead(int fd_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0) - : ReadBufferFromFileDescriptor(fd_, buf_size, existing_memory, alignment) + ReadBufferFromFileDescriptorPRead( + int fd_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0, + std::optional file_size_ = std::nullopt) + : ReadBufferFromFileDescriptor(fd_, buf_size, existing_memory, alignment, file_size_) { use_pread = true; } diff --git a/src/IO/createReadBufferFromFileBase.cpp b/src/IO/createReadBufferFromFileBase.cpp index bed97d54ab0..779fdc21ad0 100644 --- a/src/IO/createReadBufferFromFileBase.cpp +++ b/src/IO/createReadBufferFromFileBase.cpp @@ -63,23 +63,23 @@ std::unique_ptr createReadBufferFromFileBase( if (settings.local_fs_method == LocalFSReadMethod::read) { - res = std::make_unique(filename, buffer_size, actual_flags, existing_memory, alignment); + res = std::make_unique(filename, buffer_size, actual_flags, existing_memory, alignment, size); } else if (settings.local_fs_method == LocalFSReadMethod::pread || settings.local_fs_method == LocalFSReadMethod::mmap) { - res = std::make_unique(filename, buffer_size, actual_flags, existing_memory, alignment); + res = std::make_unique(filename, buffer_size, actual_flags, existing_memory, alignment, size); } else if (settings.local_fs_method == LocalFSReadMethod::pread_fake_async) { static AsynchronousReaderPtr reader = std::make_shared(); res = std::make_unique( - reader, settings.priority, filename, buffer_size, actual_flags, existing_memory, alignment); + reader, settings.priority, filename, buffer_size, actual_flags, existing_memory, alignment, size); } else if (settings.local_fs_method == LocalFSReadMethod::pread_threadpool) { static AsynchronousReaderPtr reader = std::make_shared(16, 1000000); res = std::make_unique( - reader, settings.priority, filename, buffer_size, actual_flags, existing_memory, alignment); + reader, settings.priority, filename, buffer_size, actual_flags, existing_memory, alignment, size); } else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown read method"); From 71a99ab27cc5abbe11671f3c8bfafa586a919f22 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 3 Jan 2022 21:36:34 +0300 Subject: [PATCH 0719/1260] Accept real file size in createReadBufferFromFileBase() Right now streams relies on correct file size not the number of bytes that will be read from the stream, to overcome one bug in the linux kernel that may return EIINVAL for pread() with offset pass the EOF. v2: Swap read_hint and file_size (since it is easy to miss something) Before the first argument to readFile()/createReadBufferFromFileBase() was read_hint, not the file_size, and let's preserve the order, since it is easy to miss something This will also fix 02051_read_settings test automatically because now MergeTreeReaderStream will pass estimated_sum_mark_range_bytes to read_hint not file_size, previously it cause on of the following errors: - Attempt to read after EOF w/ O_DIRECT - and LOGICAL_ERROR while adjusting granulas w/o O_DIRECT This will also improve zero-length reads guard (via ReadBufferFromEmptyFile), that had been added in #30190 v3: fix for other storages that wasn't enabled in fast-test v4: ignore ENOENT/ENOTSUP in readFile --- .../AzureBlobStorage/DiskAzureBlobStorage.cpp | 3 ++- .../AzureBlobStorage/DiskAzureBlobStorage.h | 3 ++- src/Disks/DiskCacheWrapper.cpp | 15 +++++++------ src/Disks/DiskCacheWrapper.h | 3 ++- src/Disks/DiskDecorator.cpp | 4 ++-- src/Disks/DiskDecorator.h | 3 ++- src/Disks/DiskEncrypted.cpp | 5 +++-- src/Disks/DiskEncrypted.h | 3 ++- src/Disks/DiskLocal.cpp | 22 +++++++++++++++++-- src/Disks/DiskLocal.h | 3 ++- src/Disks/DiskMemory.cpp | 2 +- src/Disks/DiskMemory.h | 3 ++- src/Disks/DiskRestartProxy.cpp | 4 ++-- src/Disks/DiskRestartProxy.h | 3 ++- src/Disks/DiskWebServer.cpp | 2 +- src/Disks/DiskWebServer.h | 3 ++- src/Disks/HDFS/DiskHDFS.cpp | 2 +- src/Disks/HDFS/DiskHDFS.h | 3 ++- src/Disks/IDisk.h | 3 ++- src/Disks/S3/DiskS3.cpp | 2 +- src/Disks/S3/DiskS3.h | 3 ++- src/Disks/tests/gtest_disk_encrypted.cpp | 2 +- src/Disks/tests/gtest_disk_hdfs.cpp | 10 ++++----- ...AsynchronousReadBufferFromFileDescriptor.h | 2 +- src/IO/createReadBufferFromFileBase.cpp | 20 +++++++++++------ src/IO/createReadBufferFromFileBase.h | 6 +++-- 26 files changed, 87 insertions(+), 47 deletions(-) diff --git a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp index e2ee6ee0153..31e85442c6a 100644 --- a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp +++ b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp @@ -62,7 +62,8 @@ DiskAzureBlobStorage::DiskAzureBlobStorage( std::unique_ptr DiskAzureBlobStorage::readFile( const String & path, const ReadSettings & read_settings, - std::optional /*estimated_size*/) const + std::optional, + std::optional) const { auto settings = current_settings.get(); auto metadata = readMeta(path); diff --git a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h index f90ede1add9..63c3c735812 100644 --- a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h +++ b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h @@ -50,7 +50,8 @@ public: std::unique_ptr readFile( const String & path, const ReadSettings & settings, - std::optional estimated_size) const override; + std::optional read_hint, + std::optional file_size) const override; std::unique_ptr writeFile( const String & path, diff --git a/src/Disks/DiskCacheWrapper.cpp b/src/Disks/DiskCacheWrapper.cpp index b09487c17bc..f741b8242f5 100644 --- a/src/Disks/DiskCacheWrapper.cpp +++ b/src/Disks/DiskCacheWrapper.cpp @@ -86,15 +86,16 @@ std::unique_ptr DiskCacheWrapper::readFile( const String & path, const ReadSettings & settings, - std::optional size) const + std::optional read_hint, + std::optional file_size) const { if (!cache_file_predicate(path)) - return DiskDecorator::readFile(path, settings, size); + return DiskDecorator::readFile(path, settings, read_hint, file_size); LOG_TEST(log, "Read file {} from cache", backQuote(path)); if (cache_disk->exists(path)) - return cache_disk->readFile(path, settings, size); + return cache_disk->readFile(path, settings, read_hint, file_size); auto metadata = acquireDownloadMetadata(path); @@ -128,7 +129,7 @@ DiskCacheWrapper::readFile( auto tmp_path = path + ".tmp"; { - auto src_buffer = DiskDecorator::readFile(path, settings, size); + auto src_buffer = DiskDecorator::readFile(path, settings, read_hint, file_size); auto dst_buffer = cache_disk->writeFile(tmp_path, settings.local_fs_buffer_size, WriteMode::Rewrite); copyData(*src_buffer, *dst_buffer); } @@ -152,9 +153,9 @@ DiskCacheWrapper::readFile( } if (metadata->status == DOWNLOADED) - return cache_disk->readFile(path, settings, size); + return cache_disk->readFile(path, settings, read_hint, file_size); - return DiskDecorator::readFile(path, settings, size); + return DiskDecorator::readFile(path, settings, read_hint, file_size); } std::unique_ptr @@ -174,7 +175,7 @@ DiskCacheWrapper::writeFile(const String & path, size_t buf_size, WriteMode mode [this, path, buf_size, mode]() { /// Copy file from cache to actual disk when cached buffer is finalized. - auto src_buffer = cache_disk->readFile(path, ReadSettings(), /* size= */ {}); + auto src_buffer = cache_disk->readFile(path, ReadSettings(), /* read_hint= */ {}, /* file_size= */ {}); auto dst_buffer = DiskDecorator::writeFile(path, buf_size, mode); copyData(*src_buffer, *dst_buffer); dst_buffer->finalize(); diff --git a/src/Disks/DiskCacheWrapper.h b/src/Disks/DiskCacheWrapper.h index 8b15a8875be..6eb79114a54 100644 --- a/src/Disks/DiskCacheWrapper.h +++ b/src/Disks/DiskCacheWrapper.h @@ -37,7 +37,8 @@ public: std::unique_ptr readFile( const String & path, const ReadSettings & settings, - std::optional size) const override; + std::optional read_hint, + std::optional file_size) const override; std::unique_ptr writeFile(const String & path, size_t buf_size, WriteMode mode) override; diff --git a/src/Disks/DiskDecorator.cpp b/src/Disks/DiskDecorator.cpp index 263c6c9c0ff..d4acb6fab0d 100644 --- a/src/Disks/DiskDecorator.cpp +++ b/src/Disks/DiskDecorator.cpp @@ -115,9 +115,9 @@ void DiskDecorator::listFiles(const String & path, std::vector & file_na std::unique_ptr DiskDecorator::readFile( - const String & path, const ReadSettings & settings, std::optional size) const + const String & path, const ReadSettings & settings, std::optional read_hint, std::optional file_size) const { - return delegate->readFile(path, settings, size); + return delegate->readFile(path, settings, read_hint, file_size); } std::unique_ptr diff --git a/src/Disks/DiskDecorator.h b/src/Disks/DiskDecorator.h index 5b88f4a36fa..9552856f435 100644 --- a/src/Disks/DiskDecorator.h +++ b/src/Disks/DiskDecorator.h @@ -38,7 +38,8 @@ public: std::unique_ptr readFile( const String & path, const ReadSettings & settings, - std::optional size) const override; + std::optional read_hint, + std::optional file_size) const override; std::unique_ptr writeFile( const String & path, diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index de569d82c60..714264b7720 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -252,10 +252,11 @@ void DiskEncrypted::copy(const String & from_path, const std::shared_ptr std::unique_ptr DiskEncrypted::readFile( const String & path, const ReadSettings & settings, - std::optional size) const + std::optional read_hint, + std::optional file_size) const { auto wrapped_path = wrappedPath(path); - auto buffer = delegate->readFile(wrapped_path, settings, size); + auto buffer = delegate->readFile(wrapped_path, settings, read_hint, file_size); if (buffer->eof()) { /// File is empty, that's a normal case, see DiskEncrypted::truncateFile(). diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 5b1bd7c5c6d..d99fe17457d 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -120,7 +120,8 @@ public: std::unique_ptr readFile( const String & path, const ReadSettings & settings, - std::optional size) const override; + std::optional read_hint, + std::optional file_size) const override; std::unique_ptr writeFile( const String & path, diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 0a0764d41b1..3428a9aef54 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -86,6 +86,22 @@ static void loadDiskLocalConfig(const String & name, } } +std::optional fileSizeSafe(const fs::path & path) +{ + std::error_code ec; + + size_t size = fs::file_size(path, ec); + if (!ec) + return size; + + if (ec == std::errc::no_such_file_or_directory) + return std::nullopt; + if (ec == std::errc::operation_not_supported) + return std::nullopt; + + throw fs::filesystem_error("DiskLocal", path, ec); +} + class DiskLocalReservation : public IReservation { public: @@ -269,9 +285,11 @@ void DiskLocal::replaceFile(const String & from_path, const String & to_path) fs::rename(from_file, to_file); } -std::unique_ptr DiskLocal::readFile(const String & path, const ReadSettings & settings, std::optional size) const +std::unique_ptr DiskLocal::readFile(const String & path, const ReadSettings & settings, std::optional read_hint, std::optional file_size) const { - return createReadBufferFromFileBase(fs::path(disk_path) / path, settings, size); + if (!file_size.has_value()) + file_size = fileSizeSafe(fs::path(disk_path) / path); + return createReadBufferFromFileBase(fs::path(disk_path) / path, settings, read_hint, file_size); } std::unique_ptr diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 37855327578..f16497ae432 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -74,7 +74,8 @@ public: std::unique_ptr readFile( const String & path, const ReadSettings & settings, - std::optional size) const override; + std::optional read_hint, + std::optional file_size) const override; std::unique_ptr writeFile( const String & path, diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index 834ed3e0c65..abaea0846a5 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -315,7 +315,7 @@ void DiskMemory::replaceFileImpl(const String & from_path, const String & to_pat files.insert(std::move(node)); } -std::unique_ptr DiskMemory::readFile(const String & path, const ReadSettings &, std::optional) const +std::unique_ptr DiskMemory::readFile(const String & path, const ReadSettings &, std::optional, std::optional) const { std::lock_guard lock(mutex); diff --git a/src/Disks/DiskMemory.h b/src/Disks/DiskMemory.h index d77161d898e..eef7b78502d 100644 --- a/src/Disks/DiskMemory.h +++ b/src/Disks/DiskMemory.h @@ -65,7 +65,8 @@ public: std::unique_ptr readFile( const String & path, const ReadSettings & settings, - std::optional size) const override; + std::optional read_hint, + std::optional file_size) const override; std::unique_ptr writeFile( const String & path, diff --git a/src/Disks/DiskRestartProxy.cpp b/src/Disks/DiskRestartProxy.cpp index a8edd15ba79..9bd59513040 100644 --- a/src/Disks/DiskRestartProxy.cpp +++ b/src/Disks/DiskRestartProxy.cpp @@ -190,10 +190,10 @@ void DiskRestartProxy::listFiles(const String & path, std::vector & file } std::unique_ptr DiskRestartProxy::readFile( - const String & path, const ReadSettings & settings, std::optional size) const + const String & path, const ReadSettings & settings, std::optional read_hint, std::optional file_size) const { ReadLock lock (mutex); - auto impl = DiskDecorator::readFile(path, settings, size); + auto impl = DiskDecorator::readFile(path, settings, read_hint, file_size); return std::make_unique(*this, std::move(impl)); } diff --git a/src/Disks/DiskRestartProxy.h b/src/Disks/DiskRestartProxy.h index e8b5891947a..3644539e941 100644 --- a/src/Disks/DiskRestartProxy.h +++ b/src/Disks/DiskRestartProxy.h @@ -46,7 +46,8 @@ public: std::unique_ptr readFile( const String & path, const ReadSettings & settings, - std::optional size) const override; + std::optional read_hint, + std::optional file_size) const override; std::unique_ptr writeFile(const String & path, size_t buf_size, WriteMode mode) override; void removeFile(const String & path) override; void removeFileIfExists(const String & path) override; diff --git a/src/Disks/DiskWebServer.cpp b/src/Disks/DiskWebServer.cpp index 63e1cc0e6c5..7c94a5b98b1 100644 --- a/src/Disks/DiskWebServer.cpp +++ b/src/Disks/DiskWebServer.cpp @@ -154,7 +154,7 @@ bool DiskWebServer::exists(const String & path) const } -std::unique_ptr DiskWebServer::readFile(const String & path, const ReadSettings & read_settings, std::optional) const +std::unique_ptr DiskWebServer::readFile(const String & path, const ReadSettings & read_settings, std::optional, std::optional) const { LOG_TRACE(log, "Read from path: {}", path); auto iter = files.find(path); diff --git a/src/Disks/DiskWebServer.h b/src/Disks/DiskWebServer.h index 1a193d91adf..bda8c8adaad 100644 --- a/src/Disks/DiskWebServer.h +++ b/src/Disks/DiskWebServer.h @@ -63,7 +63,8 @@ public: std::unique_ptr readFile(const String & path, const ReadSettings & settings, - std::optional size) const override; + std::optional read_hint, + std::optional file_size) const override; /// Disk info diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 41c407c10ee..572c908768b 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -71,7 +71,7 @@ DiskHDFS::DiskHDFS( } -std::unique_ptr DiskHDFS::readFile(const String & path, const ReadSettings & read_settings, std::optional) const +std::unique_ptr DiskHDFS::readFile(const String & path, const ReadSettings & read_settings, std::optional, std::optional) const { auto metadata = readMeta(path); diff --git a/src/Disks/HDFS/DiskHDFS.h b/src/Disks/HDFS/DiskHDFS.h index 47150f1cfd8..de373d8d6ee 100644 --- a/src/Disks/HDFS/DiskHDFS.h +++ b/src/Disks/HDFS/DiskHDFS.h @@ -53,7 +53,8 @@ public: std::unique_ptr readFile( const String & path, const ReadSettings & settings, - std::optional size) const override; + std::optional read_hint, + std::optional file_size) const override; std::unique_ptr writeFile(const String & path, size_t buf_size, WriteMode mode) override; diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 0a63421ae5c..82e997d4b5a 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -161,7 +161,8 @@ public: virtual std::unique_ptr readFile( const String & path, const ReadSettings & settings = ReadSettings{}, - std::optional size = {}) const = 0; + std::optional read_hint = {}, + std::optional file_size = {}) const = 0; /// Open the file for write and return WriteBufferFromFileBase object. virtual std::unique_ptr writeFile( diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 201334cbd12..0bebf91df97 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -214,7 +214,7 @@ void DiskS3::moveFile(const String & from_path, const String & to_path, bool sen metadata_disk->moveFile(from_path, to_path); } -std::unique_ptr DiskS3::readFile(const String & path, const ReadSettings & read_settings, std::optional) const +std::unique_ptr DiskS3::readFile(const String & path, const ReadSettings & read_settings, std::optional, std::optional) const { auto settings = current_settings.get(); auto metadata = readMeta(path); diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 18ed733ff01..c5d0722c6c2 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -76,7 +76,8 @@ public: std::unique_ptr readFile( const String & path, const ReadSettings & settings, - std::optional size) const override; + std::optional read_hint, + std::optional file_size) const override; std::unique_ptr writeFile( const String & path, diff --git a/src/Disks/tests/gtest_disk_encrypted.cpp b/src/Disks/tests/gtest_disk_encrypted.cpp index 85dd8eb78b8..d03128a6b33 100644 --- a/src/Disks/tests/gtest_disk_encrypted.cpp +++ b/src/Disks/tests/gtest_disk_encrypted.cpp @@ -57,7 +57,7 @@ protected: String getFileContents(const String & file_name) { - auto buf = encrypted_disk->readFile(file_name, /* settings= */ {}, /* size= */ {}); + auto buf = encrypted_disk->readFile(file_name, /* settings= */ {}, /* read_hint= */ {}, /* file_size= */ {}); String str; readStringUntilEOF(str, *buf); return str; diff --git a/src/Disks/tests/gtest_disk_hdfs.cpp b/src/Disks/tests/gtest_disk_hdfs.cpp index 2864797aae3..4b5ff182256 100644 --- a/src/Disks/tests/gtest_disk_hdfs.cpp +++ b/src/Disks/tests/gtest_disk_hdfs.cpp @@ -53,7 +53,7 @@ TEST(DiskTestHDFS, WriteReadHDFS) { DB::String result; - auto in = disk.readFile(file_name, {}, 1024); + auto in = disk.readFile(file_name, {}, 1024, 1024); readString(result, *in); EXPECT_EQ("Test write to file", result); } @@ -76,7 +76,7 @@ TEST(DiskTestHDFS, RewriteFileHDFS) { String result; - auto in = disk.readFile(file_name, {}, 1024); + auto in = disk.readFile(file_name, {}, 1024, 1024); readString(result, *in); EXPECT_EQ("Text10", result); readString(result, *in); @@ -104,7 +104,7 @@ TEST(DiskTestHDFS, AppendFileHDFS) { String result, expected; - auto in = disk.readFile(file_name, {}, 1024); + auto in = disk.readFile(file_name, {}, 1024, 1024); readString(result, *in); EXPECT_EQ("Text0123456789", result); @@ -131,7 +131,7 @@ TEST(DiskTestHDFS, SeekHDFS) /// Test SEEK_SET { String buf(4, '0'); - std::unique_ptr in = disk.readFile(file_name, {}, 1024); + std::unique_ptr in = disk.readFile(file_name, {}, 1024, 1024); in->seek(5, SEEK_SET); @@ -141,7 +141,7 @@ TEST(DiskTestHDFS, SeekHDFS) /// Test SEEK_CUR { - std::unique_ptr in = disk.readFile(file_name, {}, 1024); + std::unique_ptr in = disk.readFile(file_name, {}, 1024, 1024); String buf(4, '0'); in->readStrict(buf.data(), 4); diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.h b/src/IO/AsynchronousReadBufferFromFileDescriptor.h index 88efdbee6e8..2a16148812e 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.h +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.h @@ -72,7 +72,7 @@ public: void rewind(); private: - std::future readInto(char * data, size_t file_size_); + std::future readInto(char * data, size_t size); }; } diff --git a/src/IO/createReadBufferFromFileBase.cpp b/src/IO/createReadBufferFromFileBase.cpp index 779fdc21ad0..b83bfdbf3a8 100644 --- a/src/IO/createReadBufferFromFileBase.cpp +++ b/src/IO/createReadBufferFromFileBase.cpp @@ -29,14 +29,20 @@ namespace ErrorCodes std::unique_ptr createReadBufferFromFileBase( const std::string & filename, const ReadSettings & settings, - std::optional size, + std::optional read_hint, + std::optional file_size, int flags, char * existing_memory, size_t alignment) { - if (size.has_value() && !*size) + if (file_size.has_value() && !*file_size) return std::make_unique(); - size_t estimated_size = size.has_value() ? *size : 0; + + size_t estimated_size = 0; + if (read_hint.has_value()) + estimated_size = *read_hint; + else if (file_size.has_value()) + estimated_size = file_size.has_value() ? *file_size : 0; if (!existing_memory && settings.local_fs_method == LocalFSReadMethod::mmap @@ -63,23 +69,23 @@ std::unique_ptr createReadBufferFromFileBase( if (settings.local_fs_method == LocalFSReadMethod::read) { - res = std::make_unique(filename, buffer_size, actual_flags, existing_memory, alignment, size); + res = std::make_unique(filename, buffer_size, actual_flags, existing_memory, alignment, file_size); } else if (settings.local_fs_method == LocalFSReadMethod::pread || settings.local_fs_method == LocalFSReadMethod::mmap) { - res = std::make_unique(filename, buffer_size, actual_flags, existing_memory, alignment, size); + res = std::make_unique(filename, buffer_size, actual_flags, existing_memory, alignment, file_size); } else if (settings.local_fs_method == LocalFSReadMethod::pread_fake_async) { static AsynchronousReaderPtr reader = std::make_shared(); res = std::make_unique( - reader, settings.priority, filename, buffer_size, actual_flags, existing_memory, alignment, size); + reader, settings.priority, filename, buffer_size, actual_flags, existing_memory, alignment, file_size); } else if (settings.local_fs_method == LocalFSReadMethod::pread_threadpool) { static AsynchronousReaderPtr reader = std::make_shared(16, 1000000); res = std::make_unique( - reader, settings.priority, filename, buffer_size, actual_flags, existing_memory, alignment, size); + reader, settings.priority, filename, buffer_size, actual_flags, existing_memory, alignment, file_size); } else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown read method"); diff --git a/src/IO/createReadBufferFromFileBase.h b/src/IO/createReadBufferFromFileBase.h index 86da469b55d..c2e2040587b 100644 --- a/src/IO/createReadBufferFromFileBase.h +++ b/src/IO/createReadBufferFromFileBase.h @@ -11,12 +11,14 @@ namespace DB /** Create an object to read data from a file. * - * @param size - the number of bytes to read + * @param read_hint - the number of bytes to read hint + * @param file_size - size of file */ std::unique_ptr createReadBufferFromFileBase( const std::string & filename, const ReadSettings & settings, - std::optional size = {}, + std::optional read_hint = {}, + std::optional file_size = {}, int flags_ = -1, char * existing_memory = nullptr, size_t alignment = 0); From f0f2a9cc404a9a8045b2abdb077f6b4e67e6a9a3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 3 Jan 2022 20:09:26 +0300 Subject: [PATCH 0720/1260] Add a regression test for pread pass EOF returns EINVAL --- .../0_stateless/02051_read_settings.reference.j2 | 2 ++ .../queries/0_stateless/02051_read_settings.sql.j2 | 14 +++++++++++++- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02051_read_settings.reference.j2 b/tests/queries/0_stateless/02051_read_settings.reference.j2 index 86aa67a9d2d..391cf3adf35 100644 --- a/tests/queries/0_stateless/02051_read_settings.reference.j2 +++ b/tests/queries/0_stateless/02051_read_settings.reference.j2 @@ -1,3 +1,4 @@ +{% for index_granularity_bytes in [0, 10 * 1024 * 1024] -%} {% for read_method in ['read', 'mmap', 'pread_threadpool', 'pread_fake_async'] -%} {% for direct_io in [0, 1] -%} {% for prefetch in [0, 1] -%} @@ -9,3 +10,4 @@ {% endfor -%} {% endfor -%} {% endfor -%} +{% endfor -%} diff --git a/tests/queries/0_stateless/02051_read_settings.sql.j2 b/tests/queries/0_stateless/02051_read_settings.sql.j2 index 9f02274e732..fa19fbd3036 100644 --- a/tests/queries/0_stateless/02051_read_settings.sql.j2 +++ b/tests/queries/0_stateless/02051_read_settings.sql.j2 @@ -4,7 +4,15 @@ drop table if exists data_02051; -create table data_02051 (key Int, value String) engine=MergeTree() order by key +{# check each index_granularity_bytes #} +{% for index_granularity_bytes in [0, 10 * 1024 * 1024] %} +create table data_02051 (key Int, value String) +engine=MergeTree() +order by key +settings + index_granularity_bytes={{ index_granularity_bytes }}, + /* to suppress "Table can't create parts with adaptive granularity, but settings ..." warning */ + min_bytes_for_wide_part=0 as select number, repeat(toString(number), 5) from numbers(1e6); {# check each local_filesystem_read_method #} @@ -29,3 +37,7 @@ select count(ignore(*)) from data_02051 settings {% endfor %} {% endfor %} {% endfor %} + +drop table data_02051; +{# index_granularity_bytes #} +{% endfor %} From 2cdc5b37e3c2ea52501643d4071260998d3f2d91 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 3 Jan 2022 19:21:50 +0300 Subject: [PATCH 0721/1260] Do not try to read pass EOF (to workaround a bug in a kernel) For unaligned offset pread() may return EINVAL even if the offset pass EOF, although it should not, since otherwise there is no abiliity to rely on read() == 0 is EOF (with pread() loop). Here is a reproducer for the problem on 4.9.0-12-amd64: $ head -c27 /dev/urandom > /tmp/pread.issue $ xfs_io xfs_io> open -d /tmp/pread.issue xfs_io> pread 1000 4096 pread: Invalid argument And this is how it should work: xfs_io> pread 29 4096 read 0/4096 bytes at offset 29 Note, here I use interactive mode since we had old xfs_io that does not allow to execute multiple commands at once, and to avoid EMFILE issue Here is some history of a patches that affects this behaviour in the linux kernel: - the issue had been introduced in torvalds/linux@9fe55eea7e4b444bafc42fa0000cc2d1d2847275 v3.14 ("Fix race when checking i_size on direct i/o read") - an attempt to fix it had been made in torvalds/linux@74cedf9b6c603f2278a05bc91b140b32b434d0b5 v4.4 ("direct-io: Fix negative return from dio read beyond eof") - but this wasn't enough, since alignment check was earlier, so eventually fixed in torvalds/linux@41b21af388f94baf7433d4e7845703c7275251de v5.10 ("direct-io: defer alignment check until after the EOF check") Someone may ask why CI does not shows the issue, since: - it had 4.19 kernel when CI was in yandex - now it has 5.4 when CI is in AWS Since both of those kernels does not have the last patch. But, this bug requires the following conditions to met: - index_granularity_bytes=0 - min_merge_bytes_to_use_direct_io=1 Which was not covered by CI yet. --- src/IO/AsynchronousReadBufferFromFileDescriptor.cpp | 9 +++++++++ src/IO/ReadBufferFromFileDescriptor.cpp | 4 ++++ 2 files changed, 13 insertions(+) diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp index a27c9035c61..9c92201b3a1 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp @@ -44,6 +44,15 @@ std::future AsynchronousReadBufferFromFileDescripto request.offset = file_offset_of_buffer_end; request.priority = priority; + /// This is a workaround of a read pass EOF bug in linux kernel with pread() + if (file_size.has_value() && file_offset_of_buffer_end >= *file_size) + { + return std::async(std::launch::deferred, [] + { + return IAsynchronousReader::Result{ .size = 0, .offset = 0 }; + }); + } + return reader->submit(request); } diff --git a/src/IO/ReadBufferFromFileDescriptor.cpp b/src/IO/ReadBufferFromFileDescriptor.cpp index ed8eba62f04..ed6b1a60181 100644 --- a/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/src/IO/ReadBufferFromFileDescriptor.cpp @@ -54,6 +54,10 @@ bool ReadBufferFromFileDescriptor::nextImpl() /// If internal_buffer size is empty, then read() cannot be distinguished from EOF assert(!internal_buffer.empty()); + /// This is a workaround of a read pass EOF bug in linux kernel with pread() + if (file_size.has_value() && file_offset_of_buffer_end >= *file_size) + return false; + size_t bytes_read = 0; while (!bytes_read) { From 717871d9ce5085e653c85b1c8cdc8ecc5183e4ec Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 4 Jan 2022 17:21:01 +0800 Subject: [PATCH 0722/1260] code style 1. remove unused error codes --- src/Storages/Hive/HiveCommon.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index f0d479fc6dd..aa19ff042e2 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -13,7 +13,6 @@ namespace ErrorCodes { extern const int NO_HIVEMETASTORE; extern const int BAD_ARGUMENTS; - extern const int LOGICAL_ERROR; } bool HiveMetastoreClient::shouldUpdateTableMetadata( From 5ed74403812434ba6b8159e108589f11a333b3d1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 4 Jan 2022 13:27:53 +0300 Subject: [PATCH 0723/1260] Fix possible use-after-free for INSERT into MV with concurrent DROP ASan founds [1]: ==553== ERROR: AddressSanitizer: heap-use-after-free on address 0x61e004694080 at pc 0x000029150af2 bp 0x7f70b3f8ef10 sp 0x7f70b3f8ef08 READ of size 8 at 0x61e004694080 thread T477 (QueryPipelineEx) 0 0x29150af1 in DB::MergeTreeDataWriter::writeTempPart() > 1 0x293b8e43 in DB::MergeTreeSink::consume(DB::Chunk) obj-x86_64-linux-gnu/../src/Storages/MergeTree/MergeTreeSink.cpp:27:65 2 0x29dac73b in DB::SinkToStorage::onConsume(DB::Chunk) obj-x86_64-linux-gnu/../src/Processors/Sinks/SinkToStorage.cpp:18:5 3 0x29c72dd2 in DB::ExceptionKeepingTransform::work()::$_1::operator()() const obj-x86_64-linux-gnu/../src/Processors/Transforms/ExceptionKeepingTransform.cpp:151:51 0x61e004694080 is located 2048 bytes inside of 2480-byte region [0x61e004693880,0x61e004694230) freed by thread T199 (BgSchPool) here: ... 4 0x26220f20 in DB::DatabaseCatalog::TableMarkedAsDropped::~TableMarkedAsDropped() obj-x86_64-linux-gnu/../src/Interpreters/DatabaseCatalog.h:248:12 5 0x26220f20 in DB::DatabaseCatalog::dropTableDataTask() obj-x86_64-linux-gnu/../src/Interpreters/DatabaseCatalog.cpp:908:1 [1]: https://s3.amazonaws.com/clickhouse-test-reports/33201/4f04d6af61eabf4899eb8188150dc862aaab80fc/stress_test__address__actions_.html There was a fix in #32572, but it was not complete (yes it reduced the race window a lot, but not completely), since the inner table still can go away after the INSERT chain was built, to fix this obtain the reference earlier. Follow-up for: #32572 (cc @tavplubix) --- src/Interpreters/InterpreterInsertQuery.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index a1f83c81a81..d340308122f 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -263,6 +263,10 @@ BlockIO InterpreterInsertQuery::execute() QueryPipelineBuilder pipeline; StoragePtr table = getTable(query); + StoragePtr inner_table; + if (const auto * mv = dynamic_cast(table.get())) + inner_table = mv->getTargetTable(); + if (query.partition_by && !table->supportsPartitionBy()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "PARTITION BY clause is not supported by storage"); @@ -450,11 +454,8 @@ BlockIO InterpreterInsertQuery::execute() } res.pipeline.addStorageHolder(table); - if (const auto * mv = dynamic_cast(table.get())) - { - if (auto inner_table = mv->tryGetTargetTable()) - res.pipeline.addStorageHolder(inner_table); - } + if (inner_table) + res.pipeline.addStorageHolder(inner_table); return res; } From c84193ac67e9253b35eee9efc8a3f57e576e80a6 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 4 Jan 2022 14:02:46 +0300 Subject: [PATCH 0724/1260] DictionaryStructure fixes --- src/Dictionaries/DictionaryStructure.cpp | 62 ++++++++++++------------ src/Dictionaries/DictionaryStructure.h | 1 - 2 files changed, 32 insertions(+), 31 deletions(-) diff --git a/src/Dictionaries/DictionaryStructure.cpp b/src/Dictionaries/DictionaryStructure.cpp index 21d43031204..6955b3ddfdc 100644 --- a/src/Dictionaries/DictionaryStructure.cpp +++ b/src/Dictionaries/DictionaryStructure.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes namespace { + DictionaryTypedSpecialAttribute makeDictionaryTypedSpecialAttribute( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const std::string & default_type) { @@ -38,7 +39,7 @@ DictionaryTypedSpecialAttribute makeDictionaryTypedSpecialAttribute( return DictionaryTypedSpecialAttribute{std::move(name), std::move(expression), DataTypeFactory::instance().get(type_name)}; } -std::optional maybeGetAttributeUnderlyingType(TypeIndex index) +std::optional tryGetAttributeUnderlyingType(TypeIndex index) { switch (index) /// Special cases which do not map TypeIndex::T -> AttributeUnderlyingType::T { @@ -65,14 +66,16 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration { std::string structure_prefix = config_prefix + ".structure"; - const auto has_id = config.has(structure_prefix + ".id"); - const auto has_key = config.has(structure_prefix + ".key"); + const bool has_id = config.has(structure_prefix + ".id"); + const bool has_key = config.has(structure_prefix + ".key"); if (has_key && has_id) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Only one of 'id' and 'key' should be specified"); if (has_id) + { id.emplace(config, structure_prefix + ".id"); + } else if (has_key) { key.emplace(getAttributes(config, structure_prefix + ".key", /*complex_key_attributes =*/ true)); @@ -80,7 +83,9 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration throw Exception(ErrorCodes::BAD_ARGUMENTS, "Empty 'key' supplied"); } else + { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary structure should specify either 'id' or 'key'"); + } if (id) { @@ -94,7 +99,8 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration parseRangeConfiguration(config, structure_prefix); attributes = getAttributes(config, structure_prefix, /*complex_key_attributes =*/ false); - for (size_t i = 0; i < attributes.size(); ++i) + size_t attributes_size = attributes.size(); + for (size_t i = 0; i < attributes_size; ++i) { const auto & attribute = attributes[i]; const auto & attribute_name = attribute.name; @@ -106,7 +112,6 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration throw Exception(ErrorCodes::TYPE_MISMATCH, "Hierarchical attribute type for dictionary with simple key must be UInt64. Actual {}", attribute.underlying_type); - else if (key) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary with complex key does not support hierarchy"); @@ -121,17 +126,27 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration void DictionaryStructure::validateKeyTypes(const DataTypes & key_types) const { - if (key_types.size() != key->size()) + size_t key_types_size = key_types.size(); + if (key_types_size != getKeysSize()) throw Exception(ErrorCodes::TYPE_MISMATCH, "Key structure does not match, expected {}", getKeyDescription()); - for (size_t i = 0; i < key_types.size(); ++i) + if (id && !isUInt64(key_types[0])) + { + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Key type for simple key does not match, expected {}, found {}", + std::to_string(0), + "UInt64", + key_types[0]->getName()); + } + + for (size_t i = 0; i < key_types_size; ++i) { const auto & expected_type = (*key)[i].type; const auto & actual_type = key_types[i]; if (!areTypesEqual(expected_type, actual_type)) throw Exception(ErrorCodes::TYPE_MISMATCH, - "Key type at position {} does not match, expected {}, found {}", + "Key type for complex key at position {} does not match, expected {}, found {}", std::to_string(i), expected_type->getName(), actual_type->getName()); @@ -204,19 +219,6 @@ std::string DictionaryStructure::getKeyDescription() const return out.str(); } - -bool DictionaryStructure::isKeySizeFixed() const -{ - if (!key) - return true; - - for (const auto & key_i : *key) - if (key_i.underlying_type == AttributeUnderlyingType::String) - return false; - - return true; -} - Strings DictionaryStructure::getKeysNames() const { if (id) @@ -235,7 +237,7 @@ Strings DictionaryStructure::getKeysNames() const static void checkAttributeKeys(const Poco::Util::AbstractConfiguration::Keys & keys) { - static const std::unordered_set valid_keys + static const std::unordered_set valid_keys = {"name", "type", "expression", "null_value", "hierarchical", "injective", "is_object_id"}; for (const auto & key : keys) @@ -256,7 +258,7 @@ std::vector DictionaryStructure::getAttributes( Poco::Util::AbstractConfiguration::Keys config_elems; config.keys(config_prefix, config_elems); - auto has_hierarchy = false; + bool has_hierarchy = false; std::unordered_set attribute_names; std::vector res_attributes; @@ -296,7 +298,7 @@ std::vector DictionaryStructure::getAttributes( auto non_nullable_type = removeNullable(initial_type); - const auto underlying_type_opt = maybeGetAttributeUnderlyingType(non_nullable_type->getTypeId()); + const auto underlying_type_opt = tryGetAttributeUnderlyingType(non_nullable_type->getTypeId()); if (!underlying_type_opt) throw Exception(ErrorCodes::UNKNOWN_TYPE, @@ -336,6 +338,7 @@ std::vector DictionaryStructure::getAttributes( const auto hierarchical = config.getBool(prefix + "hierarchical", false); const auto injective = config.getBool(prefix + "injective", false); const auto is_object_id = config.getBool(prefix + "is_object_id", false); + if (name.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Properties 'name' and 'type' of an attribute cannot be empty"); @@ -388,13 +391,12 @@ void DictionaryStructure::parseRangeConfiguration(const Poco::Util::AbstractConf range_max->type->getName()); } - if (range_min) + if (range_min && !range_min->type->isValueRepresentedByInteger()) { - if (!range_min->type->isValueRepresentedByInteger()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Dictionary structure type of 'range_min' and 'range_max' should be an integer, Date, DateTime, or Enum." - " Actual 'range_min' and 'range_max' type is {}", - range_min->type->getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Dictionary structure type of 'range_min' and 'range_max' should be an integer, Date, DateTime, or Enum." + " Actual 'range_min' and 'range_max' type is {}", + range_min->type->getName()); } if ((range_min && !range_min->expression.empty()) || (range_max && !range_max->expression.empty())) diff --git a/src/Dictionaries/DictionaryStructure.h b/src/Dictionaries/DictionaryStructure.h index 4de00ddd259..817bc8d7824 100644 --- a/src/Dictionaries/DictionaryStructure.h +++ b/src/Dictionaries/DictionaryStructure.h @@ -129,7 +129,6 @@ struct DictionaryStructure final size_t getKeysSize() const; std::string getKeyDescription() const; - bool isKeySizeFixed() const; private: /// range_min and range_max have to be parsed before this function call From ebed6193b4c3670db769c0074abf369685ecc9c0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 4 Jan 2022 14:03:40 +0300 Subject: [PATCH 0725/1260] tests/stress: add core dumps into artifacts gcore is used here because: - default kernel.core_pattern is "|/usr/share/apport/apport %p %s %c %d %P %E" [1] and so to make it work you need to install apport into container and configure it propertly -- to complex [1]: https://s3.amazonaws.com/clickhouse-test-reports/33389/204e459d259570e6bc7fe2903f7e516094a916ca/stress_test__address__actions_/runlog.log - kernel.core_pattern cannot be changed, since proc is mounted in read-only mode in non --privileged containers [2] [2]: https://s3.amazonaws.com/clickhouse-test-reports/33389/dab3afbd94558f9654bf0aa1e06e06e2962f3bb0/stress_test__address__actions_/runlog.log v2: change kernel.core_pattern v3: use gcore, since you are not allowed to change kernel.core_pattern in container (only under --privileged) --- docker/test/stress/run.sh | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 2efb62689ff..4387d16ea7c 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -146,6 +146,7 @@ handle SIGUSR2 nostop noprint pass handle SIG$RTMIN nostop noprint pass info signals continue +gcore backtrace full info locals info registers @@ -263,3 +264,10 @@ done # Write check result into check_status.tsv clickhouse-local --structure "test String, res String" -q "SELECT 'failure', test FROM table WHERE res != 'OK' order by (lower(test) like '%hung%') LIMIT 1" < /test_output/test_results.tsv > /test_output/check_status.tsv [ -s /test_output/check_status.tsv ] || echo -e "success\tNo errors found" > /test_output/check_status.tsv + +# Core dumps (see gcore) +# Default filename is 'core.PROCESS_ID' +for core in core.*; do + pigz $core + mv $core.gz /output/ +done From ef16346af01c65e4ed7bf1fa8966692dd7a5817f Mon Sep 17 00:00:00 2001 From: Adri Fernandez <505694+iladriano@users.noreply.github.com> Date: Tue, 4 Jan 2022 08:20:53 -0800 Subject: [PATCH 0726/1260] Update success.html --- website/templates/index/success.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/templates/index/success.html b/website/templates/index/success.html index e09274c3a6f..7d70f4367b2 100644 --- a/website/templates/index/success.html +++ b/website/templates/index/success.html @@ -62,7 +62,7 @@
-

{{ _('Uber moved it’s logging platform to ClickHouse increasing developer productivity and overall reliability of the platform while seeing 3x data compression, 10x performance increase, and ½ the reduction in hardware cost.') }}

+

{{ _('Uber moved its logging platform to ClickHouse increasing developer productivity and overall reliability of the platform while seeing 3x data compression, 10x performance increase, and ½ the reduction in hardware cost.') }}

{{ _('Read the Case Study') }} From adc577c52f0ed6a4a6d8576d48046e7d6b21a3d8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 4 Jan 2022 20:03:59 +0300 Subject: [PATCH 0727/1260] Remove obsolete and unsupported Docker Builder #33345 --- docker/builder/Dockerfile | 46 --------------------------------------- docker/builder/Makefile | 12 ---------- docker/builder/README.md | 33 ---------------------------- docker/builder/build.sh | 15 ------------- 4 files changed, 106 deletions(-) delete mode 100644 docker/builder/Dockerfile delete mode 100644 docker/builder/Makefile delete mode 100644 docker/builder/README.md delete mode 100755 docker/builder/build.sh diff --git a/docker/builder/Dockerfile b/docker/builder/Dockerfile deleted file mode 100644 index 49c40d576e7..00000000000 --- a/docker/builder/Dockerfile +++ /dev/null @@ -1,46 +0,0 @@ -FROM ubuntu:20.04 - -# ARG for quick switch to a given ubuntu mirror -ARG apt_archive="http://archive.ubuntu.com" -RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list - -ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=13 - -RUN apt-get update \ - && apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \ - --yes --no-install-recommends --verbose-versions \ - && export LLVM_PUBKEY_HASH="bda960a8da687a275a2078d43c111d66b1c6a893a3275271beedf266c1ff4a0cdecb429c7a5cccf9f486ea7aa43fd27f" \ - && wget -nv -O /tmp/llvm-snapshot.gpg.key https://apt.llvm.org/llvm-snapshot.gpg.key \ - && echo "${LLVM_PUBKEY_HASH} /tmp/llvm-snapshot.gpg.key" | sha384sum -c \ - && apt-key add /tmp/llvm-snapshot.gpg.key \ - && export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ - && echo "deb [trusted=yes] http://apt.llvm.org/${CODENAME}/ llvm-toolchain-${CODENAME}-${LLVM_VERSION} main" >> \ - /etc/apt/sources.list - -RUN apt-get update \ - && apt-get install \ - bash \ - ccache \ - cmake \ - curl \ - expect \ - g++ \ - gcc \ - ninja-build \ - perl \ - pkg-config \ - python3 \ - python3-lxml \ - python3-requests \ - python3-termcolor \ - tzdata \ - llvm-${LLVM_VERSION} \ - clang-${LLVM_VERSION} \ - clang-tidy-${LLVM_VERSION} \ - lld-${LLVM_VERSION} \ - lldb-${LLVM_VERSION} \ - --yes --no-install-recommends - -COPY build.sh / - -CMD ["/bin/bash", "/build.sh"] diff --git a/docker/builder/Makefile b/docker/builder/Makefile deleted file mode 100644 index a9a7cddf3f2..00000000000 --- a/docker/builder/Makefile +++ /dev/null @@ -1,12 +0,0 @@ -build: image - mkdir -p $(HOME)/.ccache - docker run --network=host --rm --workdir /server --volume $(realpath ../..):/server --cap-add=SYS_PTRACE --mount=type=bind,source=$(HOME)/.ccache,destination=/ccache -e CCACHE_DIR=/ccache -it yandex/clickhouse-builder - -pull: - docker pull yandex/clickhouse-builder - -image: - docker build --network=host -t yandex/clickhouse-builder . - -image_push: - docker push yandex/clickhouse-builder diff --git a/docker/builder/README.md b/docker/builder/README.md deleted file mode 100644 index cb9fb7d1b77..00000000000 --- a/docker/builder/README.md +++ /dev/null @@ -1,33 +0,0 @@ -Allows to build ClickHouse in Docker. -This is useful if you have an old OS distribution and you don't want to build fresh gcc or clang from sources. - -Usage: - -Prepare image: -``` -make image -``` - -Run build: -``` -make build -``` - -Before run, ensure that your user has access to docker: -To check, that you have access to Docker, run `docker ps`. -If not, you must add this user to `docker` group: `sudo usermod -aG docker $USER` and relogin. -(You must close all your sessions. For example, restart your computer.) - -Build results are available in `build_docker` directory at top level of your working copy. -It builds only binaries, not packages. - -For example, run server: -``` -cd $(git rev-parse --show-toplevel)/src/Server -$(git rev-parse --show-toplevel)/docker/builder/programs/clickhouse server --config-file $(git rev-parse --show-toplevel)/programs/server/config.xml -``` - -Run client: -``` -$(git rev-parse --show-toplevel)/docker/builder/programs/clickhouse client -``` diff --git a/docker/builder/build.sh b/docker/builder/build.sh deleted file mode 100755 index 1025af3f96e..00000000000 --- a/docker/builder/build.sh +++ /dev/null @@ -1,15 +0,0 @@ -#!/usr/bin/env bash -set -e - -#ccache -s # uncomment to display CCache statistics -mkdir -p /server/build_docker -cd /server/build_docker -cmake -G Ninja /server "-DCMAKE_C_COMPILER=$(command -v clang-13)" "-DCMAKE_CXX_COMPILER=$(command -v clang++-13)" - -# Set the number of build jobs to the half of number of virtual CPU cores (rounded up). -# By default, ninja use all virtual CPU cores, that leads to very high memory consumption without much improvement in build time. -# Note that modern x86_64 CPUs use two-way hyper-threading (as of 2018). -# Without this option my laptop with 16 GiB RAM failed to execute build due to full system freeze. -NUM_JOBS=$(( ($(nproc || grep -c ^processor /proc/cpuinfo) + 1) / 2 )) - -ninja -j $NUM_JOBS && env TEST_OPT="--skip long compile $TEST_OPT" ctest -V -j $NUM_JOBS From 184033b7826d191176e665ba96fde64938965ae2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 4 Jan 2022 22:49:41 +0300 Subject: [PATCH 0728/1260] Trying to make it work --- tests/ci/integration_test_check.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 20e33f2f2dc..e87528dd528 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -33,6 +33,7 @@ IMAGES = [ "clickhouse/integration-test", "clickhouse/kerberos-kdc", "clickhouse/integration-helper", + "clickhouse/dotnet-client", ] def get_json_params_dict(check_name, pr_info, docker_images, run_by_hash_total, run_by_hash_num): From 1f82ab2382486561cd7d5854850a2a9f2ab92e16 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 4 Jan 2022 23:40:35 +0300 Subject: [PATCH 0729/1260] Update SECURITY.md --- SECURITY.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/SECURITY.md b/SECURITY.md index 1872d67a529..f002dd53ca9 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -21,9 +21,10 @@ The following versions of ClickHouse server are currently being supported with s | 21.6 | :x: | | 21.7 | :x: | | 21.8 | ✅ | -| 21.9 | ✅ | +| 21.9 | :x: | | 21.10 | ✅ | | 21.11 | ✅ | +| 21.12 | ✅ | ## Reporting a Vulnerability From 0696c1971fbf68b3aeb7dbac8256e1dee4da7603 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 4 Jan 2022 23:52:10 +0300 Subject: [PATCH 0730/1260] Fix clang-tidy --- base/base/wide_integer_impl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/base/wide_integer_impl.h b/base/base/wide_integer_impl.h index 84d3928f993..cffffcc213f 100644 --- a/base/base/wide_integer_impl.h +++ b/base/base/wide_integer_impl.h @@ -827,7 +827,7 @@ public: CompilerUInt128 a = (CompilerUInt128(numerator.items[1]) << 64) + numerator.items[0]; CompilerUInt128 b = (CompilerUInt128(denominator.items[1]) << 64) + denominator.items[0]; - CompilerUInt128 c = a / b; + CompilerUInt128 c = a / b; // NOLINT integer res; res.items[0] = c; From a116495471555ad5707ebad2bfff15c672c1eaa9 Mon Sep 17 00:00:00 2001 From: IlyaTsoi <45252974+IlyaTsoi@users.noreply.github.com> Date: Wed, 5 Jan 2022 01:09:36 +0300 Subject: [PATCH 0731/1260] Update LDAPClient.cpp Create a function escapeForLDAPFilter and use it to escape characters '(' and ')' in a variable that we receive from the LDAP server. This is necessary if these characters are present in the user's DN. Otherwise, the entire LDAP authorization process crashes with the error "Bad serach filter" --- src/Access/LDAPClient.cpp | 26 +++++++++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/src/Access/LDAPClient.cpp b/src/Access/LDAPClient.cpp index 49d01074f6a..c7081bd233f 100644 --- a/src/Access/LDAPClient.cpp +++ b/src/Access/LDAPClient.cpp @@ -95,6 +95,30 @@ namespace return dest; } + + auto escapeForLDAPFilter(const String & src) + { + String dest; + dest.reserve(src.size() * 2); + + for (auto ch : src) + { + switch (ch) + { + case '(': + dest += "\\28"; + break; + case ')': + dest += "\\29"; + break; + default: + dest += ch; + break; + } + } + + return dest; + } auto replacePlaceholders(const String & src, const std::vector> & pairs) { @@ -319,7 +343,7 @@ void LDAPClient::openConnection() if (user_dn_search_results.size() > 1) throw Exception("Failed to detect user DN: more than one entry in the search results", ErrorCodes::LDAP_ERROR); - final_user_dn = *user_dn_search_results.begin(); + final_user_dn = escapeForLDAPFilter(*user_dn_search_results.begin()); } break; From 466078031d1e560b72dbd5bf0adb6a81c5a558ef Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 4 Jan 2022 18:53:01 -0400 Subject: [PATCH 0732/1260] Update date-time-functions.md --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index b85f105758b..7ded7e72d8c 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -57,7 +57,7 @@ Alias: `toTimezone`. **Arguments** - `value` — Time or date and time. [DateTime64](../../sql-reference/data-types/datetime64.md). -- `timezone` — Timezone for the returned value. [String](../../sql-reference/data-types/string.md). +- `timezone` — Timezone for the returned value. [String](../../sql-reference/data-types/string.md). This argument is a constant, because `toTimezone` changes the timezone of a column (timezone is an attribute of `DateTime*` types). **Returned value** From fbf8dd53d81f1c473c1aa344daee0de661814365 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 4 Jan 2022 18:56:14 -0400 Subject: [PATCH 0733/1260] Update date-time-functions.md --- docs/ru/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index fc5533e75b1..b41defdc92d 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -57,7 +57,7 @@ toTimezone(value, timezone) **Аргументы** - `value` — время или дата с временем. [DateTime64](../../sql-reference/data-types/datetime64.md). -- `timezone` — часовой пояс для возвращаемого значения. [String](../../sql-reference/data-types/string.md). +- `timezone` — часовой пояс для возвращаемого значения. [String](../../sql-reference/data-types/string.md). Этот аргумент является константой, потому что `toTimezone` изменяет часовой пояс столбца (часовой пояс является атрибутом типов `DateTime*`). **Возвращаемое значение** From 2bbd2939c6b8719cb3a6c32843c489fdf0dfd062 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 5 Jan 2022 10:22:05 +0800 Subject: [PATCH 0734/1260] try to fix integration test failures --- tests/integration/test_hive_query/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py index bbb1b5f3c9a..081e0ae3703 100644 --- a/tests/integration/test_hive_query/test.py +++ b/tests/integration/test_hive_query/test.py @@ -106,7 +106,7 @@ def test_parquet_groupby_with_cache(started_cluster): assert result == expected_result def test_cache_read_bytes(started_cluster): node = started_cluster.instances['h0_0_0'] - time.sleep(3) + time.sleep(10) result = node.query("select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0") logging.info("Read bytes from cache:{}".format(result)) assert result.strip() != '0' From 6f40d3a342af0b66613cc4996b3b81fe7090850c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 5 Jan 2022 05:30:01 +0300 Subject: [PATCH 0735/1260] Parser simplification (incomplete) --- src/Parsers/ExpressionElementParsers.cpp | 1114 ++++++++++++---------- src/Parsers/ExpressionElementParsers.h | 59 +- 2 files changed, 587 insertions(+), 586 deletions(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 584c2a32afd..7912930dd92 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -23,6 +23,7 @@ #include #include #include +#include #include #include @@ -34,10 +35,10 @@ #include #include -#include "ASTColumnsMatcher.h" #include + namespace DB { @@ -273,6 +274,578 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex return true; } + +ASTPtr createFunctionCast(const ASTPtr & expr_ast, const ASTPtr & type_ast) +{ + /// Convert to canonical representation in functional form: CAST(expr, 'type') + auto type_literal = std::make_shared(queryToString(type_ast)); + + auto expr_list_args = std::make_shared(); + expr_list_args->children.push_back(expr_ast); + expr_list_args->children.push_back(std::move(type_literal)); + + auto func_node = std::make_shared(); + func_node->name = "CAST"; + func_node->arguments = std::move(expr_list_args); + func_node->children.push_back(func_node->arguments); + + return func_node; +} + + +namespace +{ + +class ParserCastAsExpression : public IParserBase +{ +protected: + const char * getName() const override { return "CAST AS expression"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + /// Either CAST(expr AS type) or CAST(expr, 'type') + /// The latter will be parsed normally as a function later. + + ASTPtr expr_node; + ASTPtr type_node; + + if (ParserKeyword("CAST").ignore(pos, expected) + && ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected) + && ParserExpression().parse(pos, expr_node, expected) + && ParserKeyword("AS").ignore(pos, expected) + && ParserDataType().parse(pos, type_node, expected) + && ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) + { + node = createFunctionCast(expr_node, type_node); + return true; + } + + return false; + } +}; + +class ParserSubstringExpression : public IParserBase +{ +protected: + const char * getName() const override { return "SUBSTRING expression"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + /// Either SUBSTRING(expr FROM start) or SUBSTRING(expr FROM start FOR length) or SUBSTRING(expr, start, length) + /// The latter will be parsed normally as a function later. + + ASTPtr expr_node; + ASTPtr start_node; + ASTPtr length_node; + + if (!ParserKeyword("SUBSTRING").ignore(pos, expected)) + return false; + + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + if (!ParserExpression().parse(pos, expr_node, expected)) + return false; + + if (pos->type != TokenType::Comma) + { + if (!ParserKeyword("FROM").ignore(pos, expected)) + return false; + } + else + { + ++pos; + } + + if (!ParserExpression().parse(pos, start_node, expected)) + return false; + + if (pos->type == TokenType::ClosingRoundBracket) + { + ++pos; + } + else + { + if (pos->type != TokenType::Comma) + { + if (!ParserKeyword("FOR").ignore(pos, expected)) + return false; + } + else + { + ++pos; + } + + if (!ParserExpression().parse(pos, length_node, expected)) + return false; + + ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected); + } + + /// Convert to canonical representation in functional form: SUBSTRING(expr, start, length) + + auto expr_list_args = std::make_shared(); + expr_list_args->children = {expr_node, start_node}; + + if (length_node) + expr_list_args->children.push_back(length_node); + + auto func_node = std::make_shared(); + func_node->name = "substring"; + func_node->arguments = std::move(expr_list_args); + func_node->children.push_back(func_node->arguments); + + node = std::move(func_node); + return true; + } +}; + +class ParserTrimExpression : public IParserBase +{ +protected: + const char * getName() const override { return "TRIM expression"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + /// Handles all possible TRIM/LTRIM/RTRIM call variants + + std::string func_name; + bool trim_left = false; + bool trim_right = false; + bool char_override = false; + ASTPtr expr_node; + ASTPtr pattern_node; + ASTPtr to_remove; + + if (ParserKeyword("LTRIM").ignore(pos, expected)) + { + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + trim_left = true; + } + else if (ParserKeyword("RTRIM").ignore(pos, expected)) + { + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + trim_right = true; + } + else if (ParserKeyword("TRIM").ignore(pos, expected)) + { + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + if (ParserKeyword("BOTH").ignore(pos, expected)) + { + trim_left = true; + trim_right = true; + char_override = true; + } + else if (ParserKeyword("LEADING").ignore(pos, expected)) + { + trim_left = true; + char_override = true; + } + else if (ParserKeyword("TRAILING").ignore(pos, expected)) + { + trim_right = true; + char_override = true; + } + else + { + trim_left = true; + trim_right = true; + } + + if (char_override) + { + if (!ParserExpression().parse(pos, to_remove, expected)) + return false; + if (!ParserKeyword("FROM").ignore(pos, expected)) + return false; + + auto quote_meta_func_node = std::make_shared(); + auto quote_meta_list_args = std::make_shared(); + quote_meta_list_args->children = {to_remove}; + + quote_meta_func_node->name = "regexpQuoteMeta"; + quote_meta_func_node->arguments = std::move(quote_meta_list_args); + quote_meta_func_node->children.push_back(quote_meta_func_node->arguments); + + to_remove = std::move(quote_meta_func_node); + } + } + + if (!(trim_left || trim_right)) + return false; + + if (!ParserExpression().parse(pos, expr_node, expected)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + /// Convert to regexp replace function call + + if (char_override) + { + auto pattern_func_node = std::make_shared(); + auto pattern_list_args = std::make_shared(); + if (trim_left && trim_right) + { + pattern_list_args->children = { + std::make_shared("^["), + to_remove, + std::make_shared("]*|["), + to_remove, + std::make_shared("]*$") + }; + func_name = "replaceRegexpAll"; + } + else + { + if (trim_left) + { + pattern_list_args->children = { + std::make_shared("^["), + to_remove, + std::make_shared("]*") + }; + } + else + { + /// trim_right == false not possible + pattern_list_args->children = { + std::make_shared("["), + to_remove, + std::make_shared("]*$") + }; + } + func_name = "replaceRegexpOne"; + } + + pattern_func_node->name = "concat"; + pattern_func_node->arguments = std::move(pattern_list_args); + pattern_func_node->children.push_back(pattern_func_node->arguments); + + pattern_node = std::move(pattern_func_node); + } + else + { + if (trim_left && trim_right) + { + func_name = "trimBoth"; + } + else + { + if (trim_left) + { + func_name = "trimLeft"; + } + else + { + /// trim_right == false not possible + func_name = "trimRight"; + } + } + } + + auto expr_list_args = std::make_shared(); + if (char_override) + expr_list_args->children = {expr_node, pattern_node, std::make_shared("")}; + else + expr_list_args->children = {expr_node}; + + auto func_node = std::make_shared(); + func_node->name = func_name; + func_node->arguments = std::move(expr_list_args); + func_node->children.push_back(func_node->arguments); + + node = std::move(func_node); + return true; + } +}; + +class ParserLeftExpression : public IParserBase +{ +protected: + const char * getName() const override { return "LEFT expression"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + /// Rewrites left(expr, length) to SUBSTRING(expr, 1, length) + + ASTPtr expr_node; + ASTPtr start_node; + ASTPtr length_node; + + if (!ParserKeyword("LEFT").ignore(pos, expected)) + return false; + + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + if (!ParserExpression().parse(pos, expr_node, expected)) + return false; + + ParserToken(TokenType::Comma).ignore(pos, expected); + + if (!ParserExpression().parse(pos, length_node, expected)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + auto expr_list_args = std::make_shared(); + start_node = std::make_shared(1); + expr_list_args->children = {expr_node, start_node, length_node}; + + auto func_node = std::make_shared(); + func_node->name = "substring"; + func_node->arguments = std::move(expr_list_args); + func_node->children.push_back(func_node->arguments); + + node = std::move(func_node); + return true; + } +}; + +class ParserRightExpression : public IParserBase +{ +protected: + const char * getName() const override { return "RIGHT expression"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + /// Rewrites RIGHT(expr, length) to substring(expr, -length) + + ASTPtr expr_node; + ASTPtr length_node; + + if (!ParserKeyword("RIGHT").ignore(pos, expected)) + return false; + + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + if (!ParserExpression().parse(pos, expr_node, expected)) + return false; + + ParserToken(TokenType::Comma).ignore(pos, expected); + + if (!ParserExpression().parse(pos, length_node, expected)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + auto start_expr_list_args = std::make_shared(); + start_expr_list_args->children = {length_node}; + + auto start_node = std::make_shared(); + start_node->name = "negate"; + start_node->arguments = std::move(start_expr_list_args); + start_node->children.push_back(start_node->arguments); + + auto expr_list_args = std::make_shared(); + expr_list_args->children = {expr_node, start_node}; + + auto func_node = std::make_shared(); + func_node->name = "substring"; + func_node->arguments = std::move(expr_list_args); + func_node->children.push_back(func_node->arguments); + + node = std::move(func_node); + return true; + } +}; + +class ParserExtractExpression : public IParserBase +{ +protected: + const char * getName() const override { return "EXTRACT expression"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + if (!ParserKeyword("EXTRACT").ignore(pos, expected)) + return false; + + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + ASTPtr expr; + + IntervalKind interval_kind; + if (!parseIntervalKind(pos, expected, interval_kind)) + return false; + + ParserKeyword s_from("FROM"); + if (!s_from.ignore(pos, expected)) + return false; + + ParserExpression elem_parser; + if (!elem_parser.parse(pos, expr, expected)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + auto function = std::make_shared(); + auto exp_list = std::make_shared(); + function->name = interval_kind.toNameOfFunctionExtractTimePart(); + function->arguments = exp_list; + function->children.push_back(exp_list); + exp_list->children.push_back(expr); + node = function; + + return true; + } +}; + +class ParserDateAddExpression : public IParserBase +{ +protected: + const char * getName() const override { return "DATE_ADD expression"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + const char * function_name = nullptr; + ASTPtr timestamp_node; + ASTPtr offset_node; + + if (ParserKeyword("DATEADD").ignore(pos, expected) || ParserKeyword("DATE_ADD").ignore(pos, expected) + || ParserKeyword("TIMESTAMPADD").ignore(pos, expected) || ParserKeyword("TIMESTAMP_ADD").ignore(pos, expected)) + function_name = "plus"; + else if (ParserKeyword("DATESUB").ignore(pos, expected) || ParserKeyword("DATE_SUB").ignore(pos, expected) + || ParserKeyword("TIMESTAMPSUB").ignore(pos, expected) || ParserKeyword("TIMESTAMP_SUB").ignore(pos, expected)) + function_name = "minus"; + else + return false; + + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + IntervalKind interval_kind; + ASTPtr interval_func_node; + if (parseIntervalKind(pos, expected, interval_kind)) + { + /// function(unit, offset, timestamp) + if (pos->type != TokenType::Comma) + return false; + ++pos; + + if (!ParserExpression().parse(pos, offset_node, expected)) + return false; + + if (pos->type != TokenType::Comma) + return false; + ++pos; + + if (!ParserExpression().parse(pos, timestamp_node, expected)) + return false; + auto interval_expr_list_args = std::make_shared(); + interval_expr_list_args->children = {offset_node}; + + interval_func_node = std::make_shared(); + interval_func_node->as().name = interval_kind.toNameOfFunctionToIntervalDataType(); + interval_func_node->as().arguments = std::move(interval_expr_list_args); + interval_func_node->as().children.push_back(interval_func_node->as().arguments); + } + else + { + /// function(timestamp, INTERVAL offset unit) + if (!ParserExpression().parse(pos, timestamp_node, expected)) + return false; + + if (pos->type != TokenType::Comma) + return false; + ++pos; + + if (!ParserIntervalOperatorExpression{}.parse(pos, interval_func_node, expected)) + return false; + } + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + auto expr_list_args = std::make_shared(); + expr_list_args->children = {timestamp_node, interval_func_node}; + + auto func_node = std::make_shared(); + func_node->name = function_name; + func_node->arguments = std::move(expr_list_args); + func_node->children.push_back(func_node->arguments); + + node = std::move(func_node); + + return true; + } +}; + +class ParserDateDiffExpression : public IParserBase +{ +protected: + const char * getName() const override { return "DATE_DIFF expression"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + ASTPtr left_node; + ASTPtr right_node; + + if (!(ParserKeyword("DATEDIFF").ignore(pos, expected) || ParserKeyword("DATE_DIFF").ignore(pos, expected) + || ParserKeyword("TIMESTAMPDIFF").ignore(pos, expected) || ParserKeyword("TIMESTAMP_DIFF").ignore(pos, expected))) + return false; + + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + IntervalKind interval_kind; + if (!parseIntervalKind(pos, expected, interval_kind)) + return false; + + if (pos->type != TokenType::Comma) + return false; + ++pos; + + if (!ParserExpression().parse(pos, left_node, expected)) + return false; + + if (pos->type != TokenType::Comma) + return false; + ++pos; + + if (!ParserExpression().parse(pos, right_node, expected)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + auto expr_list_args = std::make_shared(); + expr_list_args->children = {std::make_shared(interval_kind.toDateDiffUnit()), left_node, right_node}; + + auto func_node = std::make_shared(); + func_node->name = "dateDiff"; + func_node->arguments = std::move(expr_list_args); + func_node->children.push_back(func_node->arguments); + + node = std::move(func_node); + + return true; + } +}; + +} + + bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserIdentifier id_parser; @@ -304,6 +877,16 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; ++pos; + /// Special cases for expressions that look like functions but contain some syntax sugar: + /// CAST, EXTRACT, + /// DATE_ADD, DATEADD, TIMESTAMPADD, DATE_SUB, DATESUB, TIMESTAMPSUB, + /// DATE_DIFF, DATEDIFF, TIMESTAMPDIFF, TIMESTAMP_DIFF, + /// SUBSTRING, TRIM, LEFT, RIGHT, POSITION + + /// CAST(x AS type) + /// EXTRACT(interval FROM x) + /// + auto pos_after_bracket = pos; auto old_expected = expected; @@ -877,22 +1460,6 @@ bool ParserCodec::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return true; } -ASTPtr createFunctionCast(const ASTPtr & expr_ast, const ASTPtr & type_ast) -{ - /// Convert to canonical representation in functional form: CAST(expr, 'type') - auto type_literal = std::make_shared(queryToString(type_ast)); - - auto expr_list_args = std::make_shared(); - expr_list_args->children.push_back(expr_ast); - expr_list_args->children.push_back(std::move(type_literal)); - - auto func_node = std::make_shared(); - func_node->name = "CAST"; - func_node->arguments = std::move(expr_list_args); - func_node->children.push_back(func_node->arguments); - - return func_node; -} template static bool isOneOf(TokenType token) @@ -1005,509 +1572,6 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected return false; } -bool ParserCastAsExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - /// Either CAST(expr AS type) or CAST(expr, 'type') - /// The latter will be parsed normally as a function later. - - ASTPtr expr_node; - ASTPtr type_node; - - if (ParserKeyword("CAST").ignore(pos, expected) - && ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected) - && ParserExpression().parse(pos, expr_node, expected) - && ParserKeyword("AS").ignore(pos, expected) - && ParserDataType().parse(pos, type_node, expected) - && ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) - { - node = createFunctionCast(expr_node, type_node); - return true; - } - - return false; -} - -bool ParserSubstringExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - /// Either SUBSTRING(expr FROM start) or SUBSTRING(expr FROM start FOR length) or SUBSTRING(expr, start, length) - /// The latter will be parsed normally as a function later. - - ASTPtr expr_node; - ASTPtr start_node; - ASTPtr length_node; - - if (!ParserKeyword("SUBSTRING").ignore(pos, expected)) - return false; - - if (pos->type != TokenType::OpeningRoundBracket) - return false; - ++pos; - - if (!ParserExpression().parse(pos, expr_node, expected)) - return false; - - if (pos->type != TokenType::Comma) - { - if (!ParserKeyword("FROM").ignore(pos, expected)) - return false; - } - else - { - ++pos; - } - - if (!ParserExpression().parse(pos, start_node, expected)) - return false; - - if (pos->type == TokenType::ClosingRoundBracket) - { - ++pos; - } - else - { - if (pos->type != TokenType::Comma) - { - if (!ParserKeyword("FOR").ignore(pos, expected)) - return false; - } - else - { - ++pos; - } - - if (!ParserExpression().parse(pos, length_node, expected)) - return false; - - ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected); - } - - /// Convert to canonical representation in functional form: SUBSTRING(expr, start, length) - - auto expr_list_args = std::make_shared(); - expr_list_args->children = {expr_node, start_node}; - - if (length_node) - expr_list_args->children.push_back(length_node); - - auto func_node = std::make_shared(); - func_node->name = "substring"; - func_node->arguments = std::move(expr_list_args); - func_node->children.push_back(func_node->arguments); - - node = std::move(func_node); - return true; -} - -bool ParserTrimExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - /// Handles all possible TRIM/LTRIM/RTRIM call variants - - std::string func_name; - bool trim_left = false; - bool trim_right = false; - bool char_override = false; - ASTPtr expr_node; - ASTPtr pattern_node; - ASTPtr to_remove; - - if (ParserKeyword("LTRIM").ignore(pos, expected)) - { - if (pos->type != TokenType::OpeningRoundBracket) - return false; - ++pos; - trim_left = true; - } - else if (ParserKeyword("RTRIM").ignore(pos, expected)) - { - if (pos->type != TokenType::OpeningRoundBracket) - return false; - ++pos; - trim_right = true; - } - else if (ParserKeyword("TRIM").ignore(pos, expected)) - { - if (pos->type != TokenType::OpeningRoundBracket) - return false; - ++pos; - - if (ParserKeyword("BOTH").ignore(pos, expected)) - { - trim_left = true; - trim_right = true; - char_override = true; - } - else if (ParserKeyword("LEADING").ignore(pos, expected)) - { - trim_left = true; - char_override = true; - } - else if (ParserKeyword("TRAILING").ignore(pos, expected)) - { - trim_right = true; - char_override = true; - } - else - { - trim_left = true; - trim_right = true; - } - - if (char_override) - { - if (!ParserExpression().parse(pos, to_remove, expected)) - return false; - if (!ParserKeyword("FROM").ignore(pos, expected)) - return false; - - auto quote_meta_func_node = std::make_shared(); - auto quote_meta_list_args = std::make_shared(); - quote_meta_list_args->children = {to_remove}; - - quote_meta_func_node->name = "regexpQuoteMeta"; - quote_meta_func_node->arguments = std::move(quote_meta_list_args); - quote_meta_func_node->children.push_back(quote_meta_func_node->arguments); - - to_remove = std::move(quote_meta_func_node); - } - } - - if (!(trim_left || trim_right)) - return false; - - if (!ParserExpression().parse(pos, expr_node, expected)) - return false; - - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; - - /// Convert to regexp replace function call - - if (char_override) - { - auto pattern_func_node = std::make_shared(); - auto pattern_list_args = std::make_shared(); - if (trim_left && trim_right) - { - pattern_list_args->children = { - std::make_shared("^["), - to_remove, - std::make_shared("]*|["), - to_remove, - std::make_shared("]*$") - }; - func_name = "replaceRegexpAll"; - } - else - { - if (trim_left) - { - pattern_list_args->children = { - std::make_shared("^["), - to_remove, - std::make_shared("]*") - }; - } - else - { - /// trim_right == false not possible - pattern_list_args->children = { - std::make_shared("["), - to_remove, - std::make_shared("]*$") - }; - } - func_name = "replaceRegexpOne"; - } - - pattern_func_node->name = "concat"; - pattern_func_node->arguments = std::move(pattern_list_args); - pattern_func_node->children.push_back(pattern_func_node->arguments); - - pattern_node = std::move(pattern_func_node); - } - else - { - if (trim_left && trim_right) - { - func_name = "trimBoth"; - } - else - { - if (trim_left) - { - func_name = "trimLeft"; - } - else - { - /// trim_right == false not possible - func_name = "trimRight"; - } - } - } - - auto expr_list_args = std::make_shared(); - if (char_override) - expr_list_args->children = {expr_node, pattern_node, std::make_shared("")}; - else - expr_list_args->children = {expr_node}; - - auto func_node = std::make_shared(); - func_node->name = func_name; - func_node->arguments = std::move(expr_list_args); - func_node->children.push_back(func_node->arguments); - - node = std::move(func_node); - return true; -} - -bool ParserLeftExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - /// Rewrites left(expr, length) to SUBSTRING(expr, 1, length) - - ASTPtr expr_node; - ASTPtr start_node; - ASTPtr length_node; - - if (!ParserKeyword("LEFT").ignore(pos, expected)) - return false; - - if (pos->type != TokenType::OpeningRoundBracket) - return false; - ++pos; - - if (!ParserExpression().parse(pos, expr_node, expected)) - return false; - - ParserToken(TokenType::Comma).ignore(pos, expected); - - if (!ParserExpression().parse(pos, length_node, expected)) - return false; - - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; - - auto expr_list_args = std::make_shared(); - start_node = std::make_shared(1); - expr_list_args->children = {expr_node, start_node, length_node}; - - auto func_node = std::make_shared(); - func_node->name = "substring"; - func_node->arguments = std::move(expr_list_args); - func_node->children.push_back(func_node->arguments); - - node = std::move(func_node); - return true; -} - -bool ParserRightExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - /// Rewrites RIGHT(expr, length) to substring(expr, -length) - - ASTPtr expr_node; - ASTPtr length_node; - - if (!ParserKeyword("RIGHT").ignore(pos, expected)) - return false; - - if (pos->type != TokenType::OpeningRoundBracket) - return false; - ++pos; - - if (!ParserExpression().parse(pos, expr_node, expected)) - return false; - - ParserToken(TokenType::Comma).ignore(pos, expected); - - if (!ParserExpression().parse(pos, length_node, expected)) - return false; - - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; - - auto start_expr_list_args = std::make_shared(); - start_expr_list_args->children = {length_node}; - - auto start_node = std::make_shared(); - start_node->name = "negate"; - start_node->arguments = std::move(start_expr_list_args); - start_node->children.push_back(start_node->arguments); - - auto expr_list_args = std::make_shared(); - expr_list_args->children = {expr_node, start_node}; - - auto func_node = std::make_shared(); - func_node->name = "substring"; - func_node->arguments = std::move(expr_list_args); - func_node->children.push_back(func_node->arguments); - - node = std::move(func_node); - return true; -} - -bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - if (!ParserKeyword("EXTRACT").ignore(pos, expected)) - return false; - - if (pos->type != TokenType::OpeningRoundBracket) - return false; - ++pos; - - ASTPtr expr; - - IntervalKind interval_kind; - if (!parseIntervalKind(pos, expected, interval_kind)) - return false; - - ParserKeyword s_from("FROM"); - if (!s_from.ignore(pos, expected)) - return false; - - ParserExpression elem_parser; - if (!elem_parser.parse(pos, expr, expected)) - return false; - - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; - - auto function = std::make_shared(); - auto exp_list = std::make_shared(); - function->name = interval_kind.toNameOfFunctionExtractTimePart(); - function->arguments = exp_list; - function->children.push_back(exp_list); - exp_list->children.push_back(expr); - node = function; - - return true; -} - -bool ParserDateAddExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - const char * function_name = nullptr; - ASTPtr timestamp_node; - ASTPtr offset_node; - - if (ParserKeyword("DATEADD").ignore(pos, expected) || ParserKeyword("DATE_ADD").ignore(pos, expected) - || ParserKeyword("TIMESTAMPADD").ignore(pos, expected) || ParserKeyword("TIMESTAMP_ADD").ignore(pos, expected)) - function_name = "plus"; - else if (ParserKeyword("DATESUB").ignore(pos, expected) || ParserKeyword("DATE_SUB").ignore(pos, expected) - || ParserKeyword("TIMESTAMPSUB").ignore(pos, expected) || ParserKeyword("TIMESTAMP_SUB").ignore(pos, expected)) - function_name = "minus"; - else - return false; - - if (pos->type != TokenType::OpeningRoundBracket) - return false; - ++pos; - - IntervalKind interval_kind; - ASTPtr interval_func_node; - if (parseIntervalKind(pos, expected, interval_kind)) - { - /// function(unit, offset, timestamp) - if (pos->type != TokenType::Comma) - return false; - ++pos; - - if (!ParserExpression().parse(pos, offset_node, expected)) - return false; - - if (pos->type != TokenType::Comma) - return false; - ++pos; - - if (!ParserExpression().parse(pos, timestamp_node, expected)) - return false; - auto interval_expr_list_args = std::make_shared(); - interval_expr_list_args->children = {offset_node}; - - interval_func_node = std::make_shared(); - interval_func_node->as().name = interval_kind.toNameOfFunctionToIntervalDataType(); - interval_func_node->as().arguments = std::move(interval_expr_list_args); - interval_func_node->as().children.push_back(interval_func_node->as().arguments); - } - else - { - /// function(timestamp, INTERVAL offset unit) - if (!ParserExpression().parse(pos, timestamp_node, expected)) - return false; - - if (pos->type != TokenType::Comma) - return false; - ++pos; - - if (!ParserIntervalOperatorExpression{}.parse(pos, interval_func_node, expected)) - return false; - } - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; - - auto expr_list_args = std::make_shared(); - expr_list_args->children = {timestamp_node, interval_func_node}; - - auto func_node = std::make_shared(); - func_node->name = function_name; - func_node->arguments = std::move(expr_list_args); - func_node->children.push_back(func_node->arguments); - - node = std::move(func_node); - - return true; -} - -bool ParserDateDiffExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - ASTPtr left_node; - ASTPtr right_node; - - if (!(ParserKeyword("DATEDIFF").ignore(pos, expected) || ParserKeyword("DATE_DIFF").ignore(pos, expected) - || ParserKeyword("TIMESTAMPDIFF").ignore(pos, expected) || ParserKeyword("TIMESTAMP_DIFF").ignore(pos, expected))) - return false; - - if (pos->type != TokenType::OpeningRoundBracket) - return false; - ++pos; - - IntervalKind interval_kind; - if (!parseIntervalKind(pos, expected, interval_kind)) - return false; - - if (pos->type != TokenType::Comma) - return false; - ++pos; - - if (!ParserExpression().parse(pos, left_node, expected)) - return false; - - if (pos->type != TokenType::Comma) - return false; - ++pos; - - if (!ParserExpression().parse(pos, right_node, expected)) - return false; - - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; - - auto expr_list_args = std::make_shared(); - expr_list_args->children = {std::make_shared(interval_kind.toDateDiffUnit()), left_node, right_node}; - - auto func_node = std::make_shared(); - func_node->name = "dateDiff"; - func_node->arguments = std::move(expr_list_args); - func_node->children.push_back(func_node->arguments); - - node = std::move(func_node); - - return true; -} - bool ParserNull::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { @@ -2266,14 +2330,6 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp || ParserArrayOfLiterals().parse(pos, node, expected) || ParserArray().parse(pos, node, expected) || ParserLiteral().parse(pos, node, expected) - || ParserCastAsExpression().parse(pos, node, expected) - || ParserExtractExpression().parse(pos, node, expected) - || ParserDateAddExpression().parse(pos, node, expected) - || ParserDateDiffExpression().parse(pos, node, expected) - || ParserSubstringExpression().parse(pos, node, expected) - || ParserTrimExpression().parse(pos, node, expected) - || ParserLeftExpression().parse(pos, node, expected) - || ParserRightExpression().parse(pos, node, expected) || ParserCase().parse(pos, node, expected) || ParserColumnsMatcher().parse(pos, node, expected) /// before ParserFunction because it can be also parsed as a function. || ParserFunction().parse(pos, node, expected) diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index 5dff2e026be..2d1e24d60a1 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -228,63 +228,6 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; -ASTPtr createFunctionCast(const ASTPtr & expr_ast, const ASTPtr & type_ast); -class ParserCastAsExpression : public IParserBase -{ -protected: - const char * getName() const override { return "CAST AS expression"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - -class ParserSubstringExpression : public IParserBase -{ -protected: - const char * getName() const override { return "SUBSTRING expression"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - -class ParserTrimExpression : public IParserBase -{ -protected: - const char * getName() const override { return "TRIM expression"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - -class ParserLeftExpression : public IParserBase -{ -protected: - const char * getName() const override { return "LEFT expression"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - -class ParserRightExpression : public IParserBase -{ -protected: - const char * getName() const override { return "RIGHT expression"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - -class ParserExtractExpression : public IParserBase -{ -protected: - const char * getName() const override { return "EXTRACT expression"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - -class ParserDateAddExpression : public IParserBase -{ -protected: - const char * getName() const override { return "DATE_ADD expression"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - -class ParserDateDiffExpression : public IParserBase -{ -protected: - const char * getName() const override { return "DATE_DIFF expression"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - /** NULL literal. */ class ParserNull : public IParserBase @@ -535,4 +478,6 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +ASTPtr createFunctionCast(const ASTPtr & expr_ast, const ASTPtr & type_ast); + } From bf59114cee604469968bb69bcfbe599e5681e497 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 5 Jan 2022 05:32:40 +0300 Subject: [PATCH 0736/1260] Make "left" and "right" real functions --- src/Functions/GatherUtils/Algorithms.h | 69 +++++++-- src/Functions/GatherUtils/GatherUtils.h | 3 + .../sliceFromLeftDynamicLength.cpp | 60 ++++++++ .../sliceFromRightDynamicLength.cpp | 60 ++++++++ src/Functions/LeftRight.h | 143 ++++++++++++++++++ src/Functions/left.cpp | 13 ++ src/Functions/registerFunctionsString.cpp | 4 + src/Functions/right.cpp | 13 ++ src/Parsers/ExpressionElementParsers.cpp | 88 ----------- src/Parsers/ExpressionElementParsers.h | 14 -- 10 files changed, 352 insertions(+), 115 deletions(-) create mode 100644 src/Functions/GatherUtils/sliceFromLeftDynamicLength.cpp create mode 100644 src/Functions/GatherUtils/sliceFromRightDynamicLength.cpp create mode 100644 src/Functions/LeftRight.h create mode 100644 src/Functions/left.cpp create mode 100644 src/Functions/right.cpp diff --git a/src/Functions/GatherUtils/Algorithms.h b/src/Functions/GatherUtils/Algorithms.h index fc54eaf88ab..0b826792e41 100644 --- a/src/Functions/GatherUtils/Algorithms.h +++ b/src/Functions/GatherUtils/Algorithms.h @@ -347,18 +347,31 @@ void NO_INLINE sliceDynamicOffsetUnbounded(Source && src, Sink && sink, const IC } } -template -void NO_INLINE sliceDynamicOffsetBounded(Source && src, Sink && sink, const IColumn & offset_column, const IColumn & length_column) -{ - const bool is_offset_null = offset_column.onlyNull(); - const auto * offset_nullable = typeid_cast(&offset_column); - const ColumnUInt8::Container * offset_null_map = offset_nullable ? &offset_nullable->getNullMapData() : nullptr; - const IColumn * offset_nested_column = offset_nullable ? &offset_nullable->getNestedColumn() : &offset_column; - const bool is_length_null = length_column.onlyNull(); - const auto * length_nullable = typeid_cast(&length_column); - const ColumnUInt8::Container * length_null_map = length_nullable ? &length_nullable->getNullMapData() : nullptr; - const IColumn * length_nested_column = length_nullable ? &length_nullable->getNestedColumn() : &length_column; +template +static void sliceDynamicOffsetBoundedImpl(Source && src, Sink && sink, const IColumn * offset_column, const IColumn * length_column) +{ + const bool is_offset_null = !offset_column || offset_column->onlyNull(); + const ColumnUInt8::Container * offset_null_map = nullptr; + const IColumn * offset_nested_column = nullptr; + + if (!is_offset_null) + { + const auto * offset_nullable = typeid_cast(offset_column); + offset_null_map = offset_nullable ? &offset_nullable->getNullMapData() : nullptr; + offset_nested_column = offset_nullable ? &offset_nullable->getNestedColumn() : offset_column; + } + + const bool is_length_null = !length_column || length_column->onlyNull(); + const ColumnUInt8::Container * length_null_map = nullptr; + const IColumn * length_nested_column = nullptr; + + if (!is_length_null) + { + const auto * length_nullable = typeid_cast(length_column); + length_null_map = length_nullable ? &length_nullable->getNullMapData() : nullptr; + length_nested_column = length_nullable ? &length_nullable->getNestedColumn() : length_column; + } while (!src.isEnd()) { @@ -376,9 +389,19 @@ void NO_INLINE sliceDynamicOffsetBounded(Source && src, Sink && sink, const ICol typename std::decay_t::Slice slice; if (offset > 0) - slice = src.getSliceFromLeft(offset - 1, size); + { + if constexpr (inverse_offset) + slice = src.getSliceFromRight(offset - 1, size); + else + slice = src.getSliceFromLeft(offset - 1, size); + } else - slice = src.getSliceFromRight(-UInt64(offset), size); + { + if constexpr (inverse_offset) + slice = src.getSliceFromLeft(-UInt64(offset), size); + else + slice = src.getSliceFromRight(-UInt64(offset), size); + } writeSlice(slice, sink); } @@ -389,6 +412,26 @@ void NO_INLINE sliceDynamicOffsetBounded(Source && src, Sink && sink, const ICol } +template +void NO_INLINE sliceDynamicOffsetBounded(Source && src, Sink && sink, const IColumn & offset_column, const IColumn & length_column) +{ + sliceDynamicOffsetBoundedImpl(std::forward(src), std::forward(sink), &offset_column, &length_column); +} + +/// Similar to above, but with no offset. +template +void NO_INLINE sliceFromLeftDynamicLength(Source && src, Sink && sink, const IColumn & length_column) +{ + sliceDynamicOffsetBoundedImpl(std::forward(src), std::forward(sink), nullptr, &length_column); +} + +template +void NO_INLINE sliceFromRightDynamicLength(Source && src, Sink && sink, const IColumn & length_column) +{ + sliceDynamicOffsetBoundedImpl(std::forward(src), std::forward(sink), nullptr, &length_column); +} + + template void NO_INLINE conditional(SourceA && src_a, SourceB && src_b, Sink && sink, const PaddedPODArray & condition) { diff --git a/src/Functions/GatherUtils/GatherUtils.h b/src/Functions/GatherUtils/GatherUtils.h index c2513214a79..b3b50a812ea 100644 --- a/src/Functions/GatherUtils/GatherUtils.h +++ b/src/Functions/GatherUtils/GatherUtils.h @@ -52,6 +52,9 @@ ColumnArray::MutablePtr sliceFromRightConstantOffsetBounded(IArraySource & src, ColumnArray::MutablePtr sliceDynamicOffsetUnbounded(IArraySource & src, const IColumn & offset_column); ColumnArray::MutablePtr sliceDynamicOffsetBounded(IArraySource & src, const IColumn & offset_column, const IColumn & length_column); +ColumnArray::MutablePtr sliceFromLeftDynamicLength(IArraySource & src, const IColumn & length_column); +ColumnArray::MutablePtr sliceFromRightDynamicLength(IArraySource & src, const IColumn & length_column); + void sliceHasAny(IArraySource & first, IArraySource & second, ColumnUInt8 & result); void sliceHasAll(IArraySource & first, IArraySource & second, ColumnUInt8 & result); void sliceHasSubstr(IArraySource & first, IArraySource & second, ColumnUInt8 & result); diff --git a/src/Functions/GatherUtils/sliceFromLeftDynamicLength.cpp b/src/Functions/GatherUtils/sliceFromLeftDynamicLength.cpp new file mode 100644 index 00000000000..b704f7ada7d --- /dev/null +++ b/src/Functions/GatherUtils/sliceFromLeftDynamicLength.cpp @@ -0,0 +1,60 @@ +#ifndef __clang_analyzer__ // It's too hard to analyze. + +#include "GatherUtils.h" +#include "Selectors.h" +#include "Algorithms.h" + +namespace DB::GatherUtils +{ + +namespace +{ + +struct Selector : public ArraySourceSelector +{ + template + static void selectSource(bool is_const, bool is_nullable, Source && source, + const IColumn & length_column, ColumnArray::MutablePtr & result) + { + using SourceType = typename std::decay::type; + using Sink = typename SourceType::SinkType; + + if (is_nullable) + { + using NullableSource = NullableArraySource; + using NullableSink = typename NullableSource::SinkType; + + auto & nullable_source = static_cast(source); + + result = ColumnArray::create(nullable_source.createValuesColumn()); + NullableSink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + + if (is_const) + sliceFromLeftDynamicLength(static_cast &>(source), sink, length_column); + else + sliceFromLeftDynamicLength(static_cast(source), sink, length_column); + } + else + { + result = ColumnArray::create(source.createValuesColumn()); + Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + + if (is_const) + sliceFromLeftDynamicLength(static_cast &>(source), sink, length_column); + else + sliceFromLeftDynamicLength(source, sink, length_column); + } + } +}; + +} + +ColumnArray::MutablePtr sliceFromLeftDynamicLength(IArraySource & src, const IColumn & length_column) +{ + ColumnArray::MutablePtr res; + Selector::select(src, length_column, res); + return res; +} +} + +#endif diff --git a/src/Functions/GatherUtils/sliceFromRightDynamicLength.cpp b/src/Functions/GatherUtils/sliceFromRightDynamicLength.cpp new file mode 100644 index 00000000000..1db86b4fda9 --- /dev/null +++ b/src/Functions/GatherUtils/sliceFromRightDynamicLength.cpp @@ -0,0 +1,60 @@ +#ifndef __clang_analyzer__ // It's too hard to analyze. + +#include "GatherUtils.h" +#include "Selectors.h" +#include "Algorithms.h" + +namespace DB::GatherUtils +{ + +namespace +{ + +struct Selector : public ArraySourceSelector +{ + template + static void selectSource(bool is_const, bool is_nullable, Source && source, + const IColumn & length_column, ColumnArray::MutablePtr & result) + { + using SourceType = typename std::decay::type; + using Sink = typename SourceType::SinkType; + + if (is_nullable) + { + using NullableSource = NullableArraySource; + using NullableSink = typename NullableSource::SinkType; + + auto & nullable_source = static_cast(source); + + result = ColumnArray::create(nullable_source.createValuesColumn()); + NullableSink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + + if (is_const) + sliceFromRightDynamicLength(static_cast &>(source), sink, length_column); + else + sliceFromRightDynamicLength(static_cast(source), sink, length_column); + } + else + { + result = ColumnArray::create(source.createValuesColumn()); + Sink sink(result->getData(), result->getOffsets(), source.getColumnSize()); + + if (is_const) + sliceFromRightDynamicLength(static_cast &>(source), sink, length_column); + else + sliceFromRightDynamicLength(source, sink, length_column); + } + } +}; + +} + +ColumnArray::MutablePtr sliceFromRightDynamicLength(IArraySource & src, const IColumn & length_column) +{ + ColumnArray::MutablePtr res; + Selector::select(src, length_column, res); + return res; +} +} + +#endif diff --git a/src/Functions/LeftRight.h b/src/Functions/LeftRight.h new file mode 100644 index 00000000000..079a9f2f2de --- /dev/null +++ b/src/Functions/LeftRight.h @@ -0,0 +1,143 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +using namespace GatherUtils; + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +enum class SubstringDirection +{ + Left, + Right +}; + +template +class FunctionLeftRight : public IFunction +{ +public: + static constexpr auto name = direction == SubstringDirection::Left + ? (is_utf8 ? "leftUTF8" : "left") + : (is_utf8 ? "rightUTF8" : "right"); + + static FunctionPtr create(ContextPtr) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + bool isVariadic() const override { return false; } + size_t getNumberOfArguments() const override { return 2; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if ((is_utf8 && !isString(arguments[0])) || !isStringOrFixedString(arguments[0])) + throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isNativeNumber(arguments[1])) + throw Exception("Illegal type " + arguments[1]->getName() + + " of second argument of function " + + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + template + ColumnPtr executeForSource(const ColumnPtr & column_length, + const ColumnConst * column_length_const, + Int64 length_value, Source && source, + size_t input_rows_count) const + { + auto col_res = ColumnString::create(); + + if constexpr (direction == SubstringDirection::Left) + { + if (column_length_const) + sliceFromLeftConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), 0, length_value); + else + sliceFromLeftDynamicLength(source, StringSink(*col_res, input_rows_count), *column_length); + } + else + { + if (column_length_const) + sliceFromRightConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), 0, length_value); + else + sliceFromRightDynamicLength(source, StringSink(*col_res, input_rows_count), *column_length); + } + + return col_res; + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + ColumnPtr column_string = arguments[0].column; + ColumnPtr column_length = arguments[1].column; + + const ColumnConst * column_length_const = checkAndGetColumn(column_length.get()); + + Int64 length_value = 0; + + if (column_length_const) + length_value = column_length_const->getInt(0); + + if constexpr (is_utf8) + { + if (const ColumnString * col = checkAndGetColumn(column_string.get())) + return executeForSource(column_length, column_length_const, + length_value, UTF8StringSource(*col), input_rows_count); + else if (const ColumnConst * col_const = checkAndGetColumnConst(column_string.get())) + return executeForSource(column_length, column_length_const, + length_value, ConstSource(*col_const), input_rows_count); + else + throw Exception( + "Illegal column " + arguments[0].column->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } + else + { + if (const ColumnString * col = checkAndGetColumn(column_string.get())) + return executeForSource(column_length, column_length_const, + length_value, StringSource(*col), input_rows_count); + else if (const ColumnFixedString * col_fixed = checkAndGetColumn(column_string.get())) + return executeForSource(column_length, column_length_const, + length_value, FixedStringSource(*col_fixed), input_rows_count); + else if (const ColumnConst * col_const = checkAndGetColumnConst(column_string.get())) + return executeForSource(column_length, column_length_const, + length_value, ConstSource(*col_const), input_rows_count); + else if (const ColumnConst * col_const_fixed = checkAndGetColumnConst(column_string.get())) + return executeForSource(column_length, column_length_const, + length_value, ConstSource(*col_const_fixed), input_rows_count); + else + throw Exception( + "Illegal column " + arguments[0].column->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } + } +}; + +} diff --git a/src/Functions/left.cpp b/src/Functions/left.cpp new file mode 100644 index 00000000000..aa7a2cdd5a8 --- /dev/null +++ b/src/Functions/left.cpp @@ -0,0 +1,13 @@ +#include +#include + +namespace DB +{ + +void registerFunctionLeft(FunctionFactory & factory) +{ + factory.registerFunction>(FunctionFactory::CaseInsensitive); + factory.registerFunction>(FunctionFactory::CaseSensitive); +} + +} diff --git a/src/Functions/registerFunctionsString.cpp b/src/Functions/registerFunctionsString.cpp index 79002f0a97d..7d1673aff7c 100644 --- a/src/Functions/registerFunctionsString.cpp +++ b/src/Functions/registerFunctionsString.cpp @@ -23,6 +23,8 @@ void registerFunctionsConcat(FunctionFactory &); void registerFunctionFormat(FunctionFactory &); void registerFunctionFormatRow(FunctionFactory &); void registerFunctionSubstring(FunctionFactory &); +void registerFunctionLeft(FunctionFactory &); +void registerFunctionRight(FunctionFactory &); void registerFunctionCRC(FunctionFactory &); void registerFunctionAppendTrailingCharIfAbsent(FunctionFactory &); void registerFunctionStartsWith(FunctionFactory &); @@ -74,6 +76,8 @@ void registerFunctionsString(FunctionFactory & factory) registerFunctionFormat(factory); registerFunctionFormatRow(factory); registerFunctionSubstring(factory); + registerFunctionLeft(factory); + registerFunctionRight(factory); registerFunctionAppendTrailingCharIfAbsent(factory); registerFunctionStartsWith(factory); registerFunctionEndsWith(factory); diff --git a/src/Functions/right.cpp b/src/Functions/right.cpp new file mode 100644 index 00000000000..ca0df8b2d98 --- /dev/null +++ b/src/Functions/right.cpp @@ -0,0 +1,13 @@ +#include +#include + +namespace DB +{ + +void registerFunctionRight(FunctionFactory & factory) +{ + factory.registerFunction>(FunctionFactory::CaseInsensitive); + factory.registerFunction>(FunctionFactory::CaseSensitive); +} + +} diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 584c2a32afd..9975e1ee8b9 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1261,92 +1261,6 @@ bool ParserTrimExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expect return true; } -bool ParserLeftExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - /// Rewrites left(expr, length) to SUBSTRING(expr, 1, length) - - ASTPtr expr_node; - ASTPtr start_node; - ASTPtr length_node; - - if (!ParserKeyword("LEFT").ignore(pos, expected)) - return false; - - if (pos->type != TokenType::OpeningRoundBracket) - return false; - ++pos; - - if (!ParserExpression().parse(pos, expr_node, expected)) - return false; - - ParserToken(TokenType::Comma).ignore(pos, expected); - - if (!ParserExpression().parse(pos, length_node, expected)) - return false; - - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; - - auto expr_list_args = std::make_shared(); - start_node = std::make_shared(1); - expr_list_args->children = {expr_node, start_node, length_node}; - - auto func_node = std::make_shared(); - func_node->name = "substring"; - func_node->arguments = std::move(expr_list_args); - func_node->children.push_back(func_node->arguments); - - node = std::move(func_node); - return true; -} - -bool ParserRightExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - /// Rewrites RIGHT(expr, length) to substring(expr, -length) - - ASTPtr expr_node; - ASTPtr length_node; - - if (!ParserKeyword("RIGHT").ignore(pos, expected)) - return false; - - if (pos->type != TokenType::OpeningRoundBracket) - return false; - ++pos; - - if (!ParserExpression().parse(pos, expr_node, expected)) - return false; - - ParserToken(TokenType::Comma).ignore(pos, expected); - - if (!ParserExpression().parse(pos, length_node, expected)) - return false; - - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; - - auto start_expr_list_args = std::make_shared(); - start_expr_list_args->children = {length_node}; - - auto start_node = std::make_shared(); - start_node->name = "negate"; - start_node->arguments = std::move(start_expr_list_args); - start_node->children.push_back(start_node->arguments); - - auto expr_list_args = std::make_shared(); - expr_list_args->children = {expr_node, start_node}; - - auto func_node = std::make_shared(); - func_node->name = "substring"; - func_node->arguments = std::move(expr_list_args); - func_node->children.push_back(func_node->arguments); - - node = std::move(func_node); - return true; -} - bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (!ParserKeyword("EXTRACT").ignore(pos, expected)) @@ -2272,8 +2186,6 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp || ParserDateDiffExpression().parse(pos, node, expected) || ParserSubstringExpression().parse(pos, node, expected) || ParserTrimExpression().parse(pos, node, expected) - || ParserLeftExpression().parse(pos, node, expected) - || ParserRightExpression().parse(pos, node, expected) || ParserCase().parse(pos, node, expected) || ParserColumnsMatcher().parse(pos, node, expected) /// before ParserFunction because it can be also parsed as a function. || ParserFunction().parse(pos, node, expected) diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index 5dff2e026be..366623885bd 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -250,20 +250,6 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; -class ParserLeftExpression : public IParserBase -{ -protected: - const char * getName() const override { return "LEFT expression"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - -class ParserRightExpression : public IParserBase -{ -protected: - const char * getName() const override { return "RIGHT expression"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - class ParserExtractExpression : public IParserBase { protected: From 63c07ff92574691bbe5bb9d1fd3662512d2ed409 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 5 Jan 2022 10:59:21 +0800 Subject: [PATCH 0737/1260] fix stateless test tests/queries/0_stateless/02147_arrow_duplicate_columns.sh --- tests/queries/0_stateless/02147_arrow_duplicate_columns.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02147_arrow_duplicate_columns.sh b/tests/queries/0_stateless/02147_arrow_duplicate_columns.sh index 11c1522d10b..938b45fee98 100755 --- a/tests/queries/0_stateless/02147_arrow_duplicate_columns.sh +++ b/tests/queries/0_stateless/02147_arrow_duplicate_columns.sh @@ -26,6 +26,6 @@ GZDATA="H4sIAHTzuWEAA9VTuw3CMBB9+RCsyIULhFIwAC0SJQWZACkNi1CAxCCMwCCMQMEIKdkgPJ8P ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS t1" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE t1 ( x Int64, y Int64, z Int64 ) ENGINE = Memory" -echo ${GZDATA} | base64 --decode | gunzip | ${CLICKHOUSE_CLIENT} -q "INSERT INTO t1 FORMAT Arrow" 2>&1 | grep -qF "DUPLICATE_COLUMN" && echo 'OK' || echo 'FAIL' ||: +echo ${GZDATA} | base64 --decode | gunzip | ${CLICKHOUSE_CLIENT} -q "INSERT INTO t1 FORMAT Arrow settings input_format_arrow_allow_missing_columns = true" 2>&1 | grep -qF "DUPLICATE_COLUMN" && echo 'OK' || echo 'FAIL' ||: ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t1" From e1a713addb20c7ed5a3a1c8bd8bf340a10b1558c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 5 Jan 2022 11:12:45 +0800 Subject: [PATCH 0738/1260] fix stateless test tests/queries/0_stateless/01039_test_setting_parse.sql --- tests/queries/0_stateless/01039_test_setting_parse.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01039_test_setting_parse.sql b/tests/queries/0_stateless/01039_test_setting_parse.sql index 6a4eadf6a40..fd8580d26a5 100644 --- a/tests/queries/0_stateless/01039_test_setting_parse.sql +++ b/tests/queries/0_stateless/01039_test_setting_parse.sql @@ -4,11 +4,11 @@ SET max_memory_usage = '3Gi'; SELECT value FROM system.settings WHERE name = 'max_memory_usage'; SET max_memory_usage = '1567k'; SELECT value FROM system.settings WHERE name = 'max_memory_usage'; -SET max_memory_usage = '123ki'; +SET max_memory_usage = '1234ki'; SELECT value FROM system.settings WHERE name = 'max_memory_usage'; SET max_memory_usage = '1567K'; SELECT value FROM system.settings WHERE name = 'max_memory_usage'; -SET max_memory_usage = '123Ki'; +SET max_memory_usage = '1234Ki'; SELECT value FROM system.settings WHERE name = 'max_memory_usage'; SET max_memory_usage = '12M'; SELECT value FROM system.settings WHERE name = 'max_memory_usage'; From ff779b64099a567aa328972cac64f62c7e708263 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 5 Jan 2022 06:33:28 +0300 Subject: [PATCH 0739/1260] Fix error in #33124 --- .../System/StorageSystemBuildOptions.generated.cpp.in | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index 8a19d7649aa..91b521ef235 100644 --- a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -51,9 +51,9 @@ const char * auto_config_build[] "USE_FILELOG", "@USE_FILELOG@", "USE_BZIP2", "@USE_BZIP2@", "GIT_HASH", "@GIT_HASH@", - "GIT_BRANCH", "@GIT_BRANCH@", + "GIT_BRANCH", R"IRjaNsZIL9Yh7FQ4(@GIT_BRANCH@)IRjaNsZIL9Yh7FQ4", "GIT_DATE", "@GIT_DATE@", - "GIT_COMMIT_SUBJECT", "@GIT_COMMIT_SUBJECT@", + "GIT_COMMIT_SUBJECT", R"Gi17KJMlbGCjErEN(@GIT_COMMIT_SUBJECT@)Gi17KJMlbGCjErEN", nullptr, nullptr }; From af601f98d3287088ee076b05e031650c3439c895 Mon Sep 17 00:00:00 2001 From: freedomDR <1640145602@qq.com> Date: Wed, 5 Jan 2022 03:37:54 +0000 Subject: [PATCH 0740/1260] doc fix bloomfilter --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 4b7473f76ad..6769f48a466 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -339,7 +339,7 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234 For `Map` data type client can specify if index should be created for keys or values using [mapKeys](../../../sql-reference/functions/tuple-map-functions.md#mapkeys) or [mapValues](../../../sql-reference/functions/tuple-map-functions.md#mapvalues) function. - The following functions can use the filter: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions.md), [notIn](../../../sql-reference/functions/in-functions.md), [has](../../../sql-reference/functions/array-functions.md#hasarr-elem). + The following functions can use the filter: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions.md), [notIn](../../../sql-reference/functions/in-functions.md), [has](../../../sql-reference/functions/array-functions.md#hasarr-elem), [hasAny](../../../sql-reference/functions/array-functions.md#hasany), [hasAll](../../../sql-reference/functions/array-functions.md#hasall). Example of index creation for `Map` data type From c4713ee34b912c598ae23f75bcdf4ea57d45ceed Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 5 Jan 2022 06:46:31 +0300 Subject: [PATCH 0741/1260] Normalize left, right, leftUTF8, rightUTF8 functions --- src/Functions/GatherUtils/Algorithms.h | 8 +- src/Functions/GatherUtils/Sources.h | 5 + src/Functions/LeftRight.h | 2 +- .../0_stateless/02159_left_right.reference | 230 ++++++++++++++++++ .../queries/0_stateless/02159_left_right.sql | 71 ++++++ 5 files changed, 311 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02159_left_right.reference create mode 100644 tests/queries/0_stateless/02159_left_right.sql diff --git a/src/Functions/GatherUtils/Algorithms.h b/src/Functions/GatherUtils/Algorithms.h index 0b826792e41..97811722d89 100644 --- a/src/Functions/GatherUtils/Algorithms.h +++ b/src/Functions/GatherUtils/Algorithms.h @@ -348,7 +348,7 @@ void NO_INLINE sliceDynamicOffsetUnbounded(Source && src, Sink && sink, const IC } -template +template static void sliceDynamicOffsetBoundedImpl(Source && src, Sink && sink, const IColumn * offset_column, const IColumn * length_column) { const bool is_offset_null = !offset_column || offset_column->onlyNull(); @@ -390,14 +390,14 @@ static void sliceDynamicOffsetBoundedImpl(Source && src, Sink && sink, const ICo if (offset > 0) { - if constexpr (inverse_offset) - slice = src.getSliceFromRight(offset - 1, size); + if constexpr (inverse) + slice = src.getSliceFromRight(size + offset - 1, size); else slice = src.getSliceFromLeft(offset - 1, size); } else { - if constexpr (inverse_offset) + if constexpr (inverse) slice = src.getSliceFromLeft(-UInt64(offset), size); else slice = src.getSliceFromRight(-UInt64(offset), size); diff --git a/src/Functions/GatherUtils/Sources.h b/src/Functions/GatherUtils/Sources.h index c8014d3e855..7d1241be7d1 100644 --- a/src/Functions/GatherUtils/Sources.h +++ b/src/Functions/GatherUtils/Sources.h @@ -358,6 +358,11 @@ struct UTF8StringSource : public StringSource return pos; } + size_t getElementSize() const + { + return UTF8::countCodePoints(&elements[prev_offset], StringSource::getElementSize()); + } + Slice getSliceFromLeft(size_t offset) const { const auto * begin = &elements[prev_offset]; diff --git a/src/Functions/LeftRight.h b/src/Functions/LeftRight.h index 079a9f2f2de..8aeaed26803 100644 --- a/src/Functions/LeftRight.h +++ b/src/Functions/LeftRight.h @@ -85,7 +85,7 @@ public: else { if (column_length_const) - sliceFromRightConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), 0, length_value); + sliceFromRightConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), length_value); else sliceFromRightDynamicLength(source, StringSink(*col_res, input_rows_count), *column_length); } diff --git a/tests/queries/0_stateless/02159_left_right.reference b/tests/queries/0_stateless/02159_left_right.reference new file mode 100644 index 00000000000..8d7ba1686a7 --- /dev/null +++ b/tests/queries/0_stateless/02159_left_right.reference @@ -0,0 +1,230 @@ +-- { echo } + +SELECT left('Hello', 3); +Hel +SELECT left('Hello', -3); +He +SELECT left('Hello', 5); +Hello +SELECT left('Hello', -5); + +SELECT left('Hello', 6); +Hello +SELECT left('Hello', -6); + +SELECT left('Hello', 0); + +SELECT left('Hello', NULL); +\N +SELECT left(materialize('Привет'), 4); +Пр +SELECT LEFT('Привет', -4); +Прив +SELECT left(toNullable('Привет'), 12); +Привет +SELECT lEFT('Привет', -12); + +SELECT left(materialize(toNullable('Привет')), 13); +Привет +SELECT left('Привет', -13); + +SELECT Left('Привет', 0); + +SELECT left('Привет', NULL); +\N +SELECT leftUTF8('Привет', 4); +Прив +SELECT leftUTF8('Привет', -4); +Пр +SELECT leftUTF8('Привет', 12); +Привет +SELECT leftUTF8('Привет', -12); + +SELECT leftUTF8('Привет', 13); +Привет +SELECT leftUTF8('Привет', -13); + +SELECT leftUTF8('Привет', 0); + +SELECT leftUTF8('Привет', NULL); +\N +SELECT left('Hello', number) FROM numbers(10); + +H +He +Hel +Hell +Hello +Hello +Hello +Hello +Hello +SELECT leftUTF8('Привет', number) FROM numbers(10); + +П +Пр +При +Прив +Приве +Привет +Привет +Привет +Привет +SELECT left('Hello', -number) FROM numbers(10); + +Hell +Hel +He +H + + + + + +SELECT leftUTF8('Привет', -number) FROM numbers(10); + +Приве +Прив +При +Пр +П + + + + +SELECT leftUTF8('Привет', number % 3 = 0 ? NULL : (number % 2 ? toInt64(number) : -number)) FROM numbers(10); +\N +П +Прив +\N +Пр +Приве +\N +Привет + +\N +SELECT leftUTF8(number < 5 ? 'Hello' : 'Привет', number % 3 = 0 ? NULL : (number % 2 ? toInt64(number) : -number)) FROM numbers(10); +\N +H +Hel +\N +H +Приве +\N +Привет + +\N +SELECT right('Hello', 3); +llo +SELECT right('Hello', -3); +lo +SELECT right('Hello', 5); +Hello +SELECT right('Hello', -5); + +SELECT right('Hello', 6); +Hello +SELECT right('Hello', -6); + +SELECT right('Hello', 0); + +SELECT right('Hello', NULL); +\N +SELECT RIGHT(materialize('Привет'), 4); +ет +SELECT right('Привет', -4); +ивет +SELECT Right(toNullable('Привет'), 12); +Привет +SELECT right('Привет', -12); + +SELECT rIGHT(materialize(toNullable('Привет')), 13); +Привет +SELECT right('Привет', -13); + +SELECT rIgHt('Привет', 0); + +SELECT RiGhT('Привет', NULL); +\N +SELECT rightUTF8('Привет', 4); +ивет +SELECT rightUTF8('Привет', -4); +ет +SELECT rightUTF8('Привет', 12); +Привет +SELECT rightUTF8('Привет', -12); + +SELECT rightUTF8('Привет', 13); +Привет +SELECT rightUTF8('Привет', -13); + +SELECT rightUTF8('Привет', 0); + +SELECT rightUTF8('Привет', NULL); +\N +SELECT right('Hello', number) FROM numbers(10); + +o +lo +llo +ello +Hello +Hello +Hello +Hello +Hello +SELECT rightUTF8('Привет', number) FROM numbers(10); + +т +ет +вет +ивет +ривет +Привет +Привет +Привет +Привет +SELECT right('Hello', -number) FROM numbers(10); + +ello +llo +lo +o + + + + + +SELECT rightUTF8('Привет', -number) FROM numbers(10); + +ривет +ивет +вет +ет +т + + + + +SELECT rightUTF8('Привет', number % 3 = 0 ? NULL : (number % 2 ? toInt64(number) : -number)) FROM numbers(10); +\N +т +ивет +\N +ет +ривет +\N +Привет + +\N +SELECT rightUTF8(number < 5 ? 'Hello' : 'Привет', number % 3 = 0 ? NULL : (number % 2 ? toInt64(number) : -number)) FROM numbers(10); +\N +o +llo +\N +o +ривет +\N +Привет + +\N diff --git a/tests/queries/0_stateless/02159_left_right.sql b/tests/queries/0_stateless/02159_left_right.sql new file mode 100644 index 00000000000..a45ca3db961 --- /dev/null +++ b/tests/queries/0_stateless/02159_left_right.sql @@ -0,0 +1,71 @@ +-- { echo } + +SELECT left('Hello', 3); +SELECT left('Hello', -3); +SELECT left('Hello', 5); +SELECT left('Hello', -5); +SELECT left('Hello', 6); +SELECT left('Hello', -6); +SELECT left('Hello', 0); +SELECT left('Hello', NULL); + +SELECT left(materialize('Привет'), 4); +SELECT LEFT('Привет', -4); +SELECT left(toNullable('Привет'), 12); +SELECT lEFT('Привет', -12); +SELECT left(materialize(toNullable('Привет')), 13); +SELECT left('Привет', -13); +SELECT Left('Привет', 0); +SELECT left('Привет', NULL); + +SELECT leftUTF8('Привет', 4); +SELECT leftUTF8('Привет', -4); +SELECT leftUTF8('Привет', 12); +SELECT leftUTF8('Привет', -12); +SELECT leftUTF8('Привет', 13); +SELECT leftUTF8('Привет', -13); +SELECT leftUTF8('Привет', 0); +SELECT leftUTF8('Привет', NULL); + +SELECT left('Hello', number) FROM numbers(10); +SELECT leftUTF8('Привет', number) FROM numbers(10); +SELECT left('Hello', -number) FROM numbers(10); +SELECT leftUTF8('Привет', -number) FROM numbers(10); + +SELECT leftUTF8('Привет', number % 3 = 0 ? NULL : (number % 2 ? toInt64(number) : -number)) FROM numbers(10); +SELECT leftUTF8(number < 5 ? 'Hello' : 'Привет', number % 3 = 0 ? NULL : (number % 2 ? toInt64(number) : -number)) FROM numbers(10); + +SELECT right('Hello', 3); +SELECT right('Hello', -3); +SELECT right('Hello', 5); +SELECT right('Hello', -5); +SELECT right('Hello', 6); +SELECT right('Hello', -6); +SELECT right('Hello', 0); +SELECT right('Hello', NULL); + +SELECT RIGHT(materialize('Привет'), 4); +SELECT right('Привет', -4); +SELECT Right(toNullable('Привет'), 12); +SELECT right('Привет', -12); +SELECT rIGHT(materialize(toNullable('Привет')), 13); +SELECT right('Привет', -13); +SELECT rIgHt('Привет', 0); +SELECT RiGhT('Привет', NULL); + +SELECT rightUTF8('Привет', 4); +SELECT rightUTF8('Привет', -4); +SELECT rightUTF8('Привет', 12); +SELECT rightUTF8('Привет', -12); +SELECT rightUTF8('Привет', 13); +SELECT rightUTF8('Привет', -13); +SELECT rightUTF8('Привет', 0); +SELECT rightUTF8('Привет', NULL); + +SELECT right('Hello', number) FROM numbers(10); +SELECT rightUTF8('Привет', number) FROM numbers(10); +SELECT right('Hello', -number) FROM numbers(10); +SELECT rightUTF8('Привет', -number) FROM numbers(10); + +SELECT rightUTF8('Привет', number % 3 = 0 ? NULL : (number % 2 ? toInt64(number) : -number)) FROM numbers(10); +SELECT rightUTF8(number < 5 ? 'Hello' : 'Привет', number % 3 = 0 ? NULL : (number % 2 ? toInt64(number) : -number)) FROM numbers(10); From eb80dfebf2b75e1cef090ef64ac6233db234e27a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 5 Jan 2022 07:12:20 +0300 Subject: [PATCH 0742/1260] Fix style --- src/Functions/GatherUtils/Algorithms.h | 1 + src/Functions/GatherUtils/GatherUtils.h | 6 +++--- src/Functions/LeftRight.h | 2 ++ 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Functions/GatherUtils/Algorithms.h b/src/Functions/GatherUtils/Algorithms.h index 97811722d89..a44562ea3b4 100644 --- a/src/Functions/GatherUtils/Algorithms.h +++ b/src/Functions/GatherUtils/Algorithms.h @@ -636,6 +636,7 @@ bool insliceEqualElements(const NumericArraySlice & first [[maybe_unused]], else return accurate::equalsOp(first.data[first_ind], first.data[second_ind]); } + inline ALWAYS_INLINE bool insliceEqualElements(const GenericArraySlice & first, size_t first_ind, size_t second_ind) { return first.elements->compareAt(first_ind + first.begin, second_ind + first.begin, *first.elements, -1) == 0; diff --git a/src/Functions/GatherUtils/GatherUtils.h b/src/Functions/GatherUtils/GatherUtils.h index b3b50a812ea..8a623caa297 100644 --- a/src/Functions/GatherUtils/GatherUtils.h +++ b/src/Functions/GatherUtils/GatherUtils.h @@ -32,9 +32,9 @@ namespace DB::GatherUtils enum class ArraySearchType { - Any, // Corresponds to the hasAny array function - All, // Corresponds to the hasAll array function - Substr // Corresponds to the hasSubstr array function + Any, // Corresponds to the hasAny array function + All, // Corresponds to the hasAll array function + Substr // Corresponds to the hasSubstr array function }; std::unique_ptr createArraySource(const ColumnArray & col, bool is_const, size_t total_rows); diff --git a/src/Functions/LeftRight.h b/src/Functions/LeftRight.h index 8aeaed26803..054e76b7792 100644 --- a/src/Functions/LeftRight.h +++ b/src/Functions/LeftRight.h @@ -1,3 +1,5 @@ +#pragma once + #include #include #include From 9140caf85a2ce5e8c92f87ce210ab61b6e4dc758 Mon Sep 17 00:00:00 2001 From: "Chun-Sheng, Li" Date: Wed, 5 Jan 2022 14:09:00 +0800 Subject: [PATCH 0743/1260] Fixing typo and add MariaDB command-line tool --- docs/en/interfaces/mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/mysql.md b/docs/en/interfaces/mysql.md index 38bcc2b68f8..9932e6b6cb3 100644 --- a/docs/en/interfaces/mysql.md +++ b/docs/en/interfaces/mysql.md @@ -36,7 +36,7 @@ mysql> ``` For compatibility with all MySQL clients, it is recommended to specify user password with [double SHA1](../operations/settings/settings-users.md#password_double_sha1_hex) in configuration file. -If user password is specified using [SHA256](../operations/settings/settings-users.md#password_sha256_hex), some clients won’t be able to authenticate (mysqljs and old versions of command-line tool mysql). +If user password is specified using [SHA256](../operations/settings/settings-users.md#password_sha256_hex), some clients won’t be able to authenticate (mysqljs and old versions of command-line tool MySQL and MariaDB). Restrictions: From 624da11ee299ebcb9014332cb060a6513f36d36f Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 5 Jan 2022 14:29:34 +0800 Subject: [PATCH 0744/1260] fix a bug : cache files are released in wrong way --- src/Common/LRUResourceCache.h | 15 +++++++++++++-- src/Storages/Cache/ExternalDataSourceCache.h | 2 +- src/Storages/Cache/RemoteCacheController.cpp | 1 - src/Storages/Cache/RemoteCacheController.h | 2 +- src/Storages/Cache/RemoteFileCachePolicy.h | 8 ++++++++ 5 files changed, 23 insertions(+), 5 deletions(-) diff --git a/src/Common/LRUResourceCache.h b/src/Common/LRUResourceCache.h index 40c283749d3..8f41324dd4c 100644 --- a/src/Common/LRUResourceCache.h +++ b/src/Common/LRUResourceCache.h @@ -12,11 +12,17 @@ namespace DB { template -struct TrivailLRUResourceCacheWeightFunction +struct TrivialLRUResourceCacheWeightFunction { size_t operator()(const T &) const { return 1; } }; +template +struct TrivialLRUResourceCacheReleaseFunction +{ + void operator()(std::shared_ptr) {} +}; + /** * Similar to implementation in LRUCache.h, but with the difference that keys can * only be evicted when they are releasable. Release state is controlled by this implementation. @@ -26,7 +32,8 @@ struct TrivailLRUResourceCacheWeightFunction */ template , + typename WeightFunction = TrivialLRUResourceCacheWeightFunction, + typename ReleaseFunction = TrivialLRUResourceCacheReleaseFunction, typename HashFunction = std::hash> class LRUResourceCache { @@ -86,6 +93,7 @@ public: { queue.erase(cell.queue_iterator); current_weight -= cell.weight; + release_function(cell.value); cells.erase(it); } else @@ -198,6 +206,7 @@ private: friend struct InsertTokenHolder; InsertTokenById insert_tokens; WeightFunction weight_function; + ReleaseFunction release_function; std::atomic hits{0}; std::atomic misses{0}; std::atomic evict_count{0}; @@ -305,6 +314,7 @@ private: { queue.erase(cell.queue_iterator); current_weight -= cell.weight; + release_function(cell.value); cells.erase(it); } } @@ -376,6 +386,7 @@ private: { auto & cell = cells[key]; queue.erase(cell.queue_iterator); + release_function(cell.value); cells.erase(key); ++evict_count; } diff --git a/src/Storages/Cache/ExternalDataSourceCache.h b/src/Storages/Cache/ExternalDataSourceCache.h index 9e69c9f64fd..3e2bbb05104 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.h +++ b/src/Storages/Cache/ExternalDataSourceCache.h @@ -28,7 +28,7 @@ namespace DB { -using RemoteFileCacheType = LRUResourceCache; +using RemoteFileCacheType = LRUResourceCache; class LocalFileHolder { diff --git a/src/Storages/Cache/RemoteCacheController.cpp b/src/Storages/Cache/RemoteCacheController.cpp index 03c5ef17fe8..b0bb31c09e7 100644 --- a/src/Storages/Cache/RemoteCacheController.cpp +++ b/src/Storages/Cache/RemoteCacheController.cpp @@ -198,7 +198,6 @@ RemoteCacheController::~RemoteCacheController() { if (download_task_holder) download_task_holder->deactivate(); - close(); } void RemoteCacheController::close() diff --git a/src/Storages/Cache/RemoteCacheController.h b/src/Storages/Cache/RemoteCacheController.h index 4399a9e2b90..6047dbd5eb4 100644 --- a/src/Storages/Cache/RemoteCacheController.h +++ b/src/Storages/Cache/RemoteCacheController.h @@ -83,7 +83,7 @@ private: IRemoteFileMetadataPtr file_metadata_ptr; std::filesystem::path local_path; - bool valid; + bool valid = true; size_t local_cache_bytes_read_before_flush; size_t current_offset; diff --git a/src/Storages/Cache/RemoteFileCachePolicy.h b/src/Storages/Cache/RemoteFileCachePolicy.h index 43d29f85b8d..b4fad5f80f6 100644 --- a/src/Storages/Cache/RemoteFileCachePolicy.h +++ b/src/Storages/Cache/RemoteFileCachePolicy.h @@ -6,4 +6,12 @@ struct RemoteFileCacheWeightFunction size_t operator()(const RemoteCacheController & cache) const { return cache.getFileSize(); } }; +struct RemoteFileCacheReleaseFunction +{ + void operator() (std::shared_ptr controller) + { + if (controller) + controller->close(); + } +}; } From f78564491c336253be5b33e2719ae0d9bceb456e Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 5 Jan 2022 14:40:02 +0800 Subject: [PATCH 0745/1260] code style --- src/Common/LRUResourceCache.h | 15 ++++++++------- src/Storages/Cache/RemoteFileCachePolicy.h | 2 +- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/src/Common/LRUResourceCache.h b/src/Common/LRUResourceCache.h index 8f41324dd4c..9f46139972c 100644 --- a/src/Common/LRUResourceCache.h +++ b/src/Common/LRUResourceCache.h @@ -20,7 +20,7 @@ struct TrivialLRUResourceCacheWeightFunction template struct TrivialLRUResourceCacheReleaseFunction { - void operator()(std::shared_ptr) {} + void operator()(std::shared_ptr) { } }; /** @@ -30,7 +30,8 @@ struct TrivialLRUResourceCacheReleaseFunction * * Warning (!): This implementation is in development, not to be used. */ -template , typename ReleaseFunction = TrivialLRUResourceCacheReleaseFunction, @@ -45,8 +46,7 @@ public: class MappedHolder { public: - MappedHolder(LRUResourceCache * cache_, const Key & key_, MappedPtr value_) - : cache(cache_), key(key_), val(value_) {} + MappedHolder(LRUResourceCache * cache_, const Key & key_, MappedPtr value_) : cache(cache_), key(key_), val(value_) { } ~MappedHolder() { cache->release(key); } @@ -61,7 +61,9 @@ public: using MappedHolderPtr = std::unique_ptr; explicit LRUResourceCache(size_t max_weight_, size_t max_element_size_ = 0) - : max_weight(max_weight_), max_element_size(max_element_size_) {} + : max_weight(max_weight_), max_element_size(max_element_size_) + { + } MappedHolderPtr get(const Key & key) { @@ -343,8 +345,7 @@ private: size_t weight = value ? weight_function(*value) : 0; size_t queue_size = cells.size() + 1; size_t loss_weight = 0; - auto is_overflow = [&] - { + auto is_overflow = [&] { return current_weight + weight > max_weight + loss_weight || (max_element_size != 0 && queue_size > max_element_size); }; diff --git a/src/Storages/Cache/RemoteFileCachePolicy.h b/src/Storages/Cache/RemoteFileCachePolicy.h index b4fad5f80f6..7d742d6ea14 100644 --- a/src/Storages/Cache/RemoteFileCachePolicy.h +++ b/src/Storages/Cache/RemoteFileCachePolicy.h @@ -8,7 +8,7 @@ struct RemoteFileCacheWeightFunction struct RemoteFileCacheReleaseFunction { - void operator() (std::shared_ptr controller) + void operator()(std::shared_ptr controller) { if (controller) controller->close(); From 50e6f729d1e01fc3d3a1e3e31cbbafe5e84d81e7 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Thu, 30 Dec 2021 11:01:47 +0000 Subject: [PATCH 0746/1260] Try ping connection on unexpected errors Workaround for https://www.postgresql.org/message-id/CAC5UznEsaG75-Q89z4Ypz1q48UT7O%2B1U7drUPt6Xs%2Bma9_hEGQ%40mail.gmail.com --- programs/odbc-bridge/ODBCConnectionFactory.h | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/programs/odbc-bridge/ODBCConnectionFactory.h b/programs/odbc-bridge/ODBCConnectionFactory.h index 13396206a46..38f78916107 100644 --- a/programs/odbc-bridge/ODBCConnectionFactory.h +++ b/programs/odbc-bridge/ODBCConnectionFactory.h @@ -91,6 +91,25 @@ T execute(nanodbc::ConnectionHolderPtr connection_holder, std::functionupdateConnection(); return query_func(connection_holder->get()); } + + /// psqlodbc driver error handling is incomplete and under some scenarious + /// it doesn't propagate correct errors to the caller. + /// As a quick workaround we run a quick "ping" query over the connection + /// on generic errors. + /// If "ping" fails, recycle the connection and try the query once more. + if (e.state().starts_with("HY00")) + { + try + { + just_execute(connection_holder->get(), "SELECT 1"); + } + catch (...) + { + connection_holder->updateConnection(); + return query_func(connection_holder->get()); + } + } + throw; } } From 699669b81d6bdb335979787afa9fb9769f7da3fa Mon Sep 17 00:00:00 2001 From: Andrii Buriachevskyi Date: Wed, 5 Jan 2022 19:17:25 +0100 Subject: [PATCH 0747/1260] Add tests for INSERT DEFAULT with format Values --- ...02155_default_keyword_format_values.reference | 8 ++++++++ .../02155_default_keyword_format_values.sh | 16 ++++++++++++++++ 2 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/02155_default_keyword_format_values.reference create mode 100755 tests/queries/0_stateless/02155_default_keyword_format_values.sh diff --git a/tests/queries/0_stateless/02155_default_keyword_format_values.reference b/tests/queries/0_stateless/02155_default_keyword_format_values.reference new file mode 100644 index 00000000000..022aeaff0a9 --- /dev/null +++ b/tests/queries/0_stateless/02155_default_keyword_format_values.reference @@ -0,0 +1,8 @@ +0 1 33 +0 42 33 +1 42 2 +2 42 33 +3 42 3 +4 42 3 +5 42 33 +6 6 33 diff --git a/tests/queries/0_stateless/02155_default_keyword_format_values.sh b/tests/queries/0_stateless/02155_default_keyword_format_values.sh new file mode 100755 index 00000000000..53f88747fdc --- /dev/null +++ b/tests/queries/0_stateless/02155_default_keyword_format_values.sh @@ -0,0 +1,16 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "CREATE TABLE IF NOT EXISTS default_table (x UInt32, y UInt32 DEFAULT 42, z UInt32 DEFAULT 33) ENGINE = Memory;" + +echo "(DEFAULT, 1, DEFAULT), (1, DEFAULT, 2)" | $CLICKHOUSE_CLIENT --input_format_values_interpret_expressions=0 -q "INSERT INTO default_table FORMAT Values" +echo "(2, DEFAULT), (3, 3)" | $CLICKHOUSE_CLIENT --input_format_values_interpret_expressions=0 -q "INSERT INTO default_table(x, z) FORMAT Values" + +echo "(DEFAULT, DEFAULT, DEFAULT), (4, DEFAULT, 3)" | $CLICKHOUSE_CLIENT --input_format_values_interpret_expressions=1 -q "INSERT INTO default_table FORMAT Values" +echo "(5, DEFAULT), (6, 6)" | $CLICKHOUSE_CLIENT --input_format_values_interpret_expressions=1 -q "INSERT INTO default_table(x, y) FORMAT Values" + +$CLICKHOUSE_CLIENT --query="SELECT * FROM default_table ORDER BY x, y"; +$CLICKHOUSE_CLIENT --query="DROP TABLE default_table" From 6157f73fb52b0ede1e76352f3d9135af732e29ce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 5 Jan 2022 22:15:22 +0300 Subject: [PATCH 0748/1260] Fix UBSan --- src/Functions/GatherUtils/Algorithms.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/GatherUtils/Algorithms.h b/src/Functions/GatherUtils/Algorithms.h index a44562ea3b4..046e2dcf70f 100644 --- a/src/Functions/GatherUtils/Algorithms.h +++ b/src/Functions/GatherUtils/Algorithms.h @@ -391,9 +391,9 @@ static void sliceDynamicOffsetBoundedImpl(Source && src, Sink && sink, const ICo if (offset > 0) { if constexpr (inverse) - slice = src.getSliceFromRight(size + offset - 1, size); + slice = src.getSliceFromRight(UInt64(size) + UInt64(offset) - 1, size); else - slice = src.getSliceFromLeft(offset - 1, size); + slice = src.getSliceFromLeft(UInt64(offset) - 1, size); } else { From 0139820ac4be32071c8e8a3607816f417643ab66 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 5 Jan 2022 22:33:02 +0300 Subject: [PATCH 0749/1260] Improve install on FreeBSD #33384 --- programs/install/Install.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 706e273e2b4..d6a88b88bfa 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -233,6 +233,9 @@ int mainEntryClickHouseInstall(int argc, char ** argv) path.pop_back(); fs::path binary_self_path(path); +#elif defined(OS_FREEBSD) + /// https://stackoverflow.com/questions/1023306/finding-current-executables-path-without-proc-self-exe + fs::path binary_self_path = "/proc/curproc/file"; #else fs::path binary_self_path = "/proc/self/exe"; #endif From 269aa9404b32f2054c0aceab5fac992506f525ce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 5 Jan 2022 22:59:22 +0300 Subject: [PATCH 0750/1260] Maybe better --- programs/install/Install.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index d6a88b88bfa..c5fc9140522 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -235,7 +235,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) fs::path binary_self_path(path); #elif defined(OS_FREEBSD) /// https://stackoverflow.com/questions/1023306/finding-current-executables-path-without-proc-self-exe - fs::path binary_self_path = "/proc/curproc/file"; + fs::path binary_self_path = argc >= 1 ? argv[0] : "/proc/curproc/file"; #else fs::path binary_self_path = "/proc/self/exe"; #endif From dba9bf0f299795c6971c5f31275b30551d67eb61 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jan 2022 02:30:41 +0300 Subject: [PATCH 0751/1260] Support for creating users --- programs/install/Install.cpp | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index c5fc9140522..22d4c21018b 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -153,10 +153,12 @@ static void createGroup(const String & group_name) if (!group_name.empty()) { #if defined(OS_DARWIN) - // TODO: implement. - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unable to create a group in macOS"); +#elif defined(OS_FREEBSD) + std::string command = fmt::format("pw groupadd {}", group_name); + fmt::print(" {}\n", command); + executeScript(command); #else std::string command = fmt::format("groupadd -r {}", group_name); fmt::print(" {}\n", command); @@ -170,10 +172,14 @@ static void createUser(const String & user_name, [[maybe_unused]] const String & if (!user_name.empty()) { #if defined(OS_DARWIN) - // TODO: implement. - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unable to create a user in macOS"); +#elif defined(OS_FREEBSD) + std::string command = group_name.empty() + ? fmt::format("pw useradd -s /bin/false -d /nonexistent -n {}", user_name) + : fmt::format("pw useradd -s /bin/false -d /nonexistent -g {} -n {}", group_name, user_name); + fmt::print(" {}\n", command); + executeScript(command); #else std::string command = group_name.empty() ? fmt::format("useradd -r --shell /bin/false --home-dir /nonexistent --user-group {}", user_name) From 5e6f1f1f5fc3c4ec6144b6e07063f30647064041 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jan 2022 04:08:56 +0300 Subject: [PATCH 0752/1260] Add support for start --- programs/install/Install.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 22d4c21018b..9d58b3cee84 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -919,7 +919,13 @@ namespace command = fmt::format("sudo -u '{}' {}", user, command); } else + { +#if defined(OS_FREEBSD) + command = fmt::format("su -m '{}' -c '{}', user, command); +#else command = fmt::format("su -s /bin/sh '{}' -c '{}'", user, command); +#endif + } } fmt::print("Will run {}\n", command); From 75116c69c60697772b9dad95df3ab3e5278a172b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jan 2022 04:26:02 +0300 Subject: [PATCH 0753/1260] Better messages --- programs/install/Install.cpp | 57 ++++++++++++++++++------------------ 1 file changed, 29 insertions(+), 28 deletions(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 9d58b3cee84..f2aeeed98ad 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -191,6 +191,20 @@ static void createUser(const String & user_name, [[maybe_unused]] const String & } +static std::string formatWithSudo(std::string command, bool needed = true) +{ + if (!needed) + return command; + +#if defined(OS_FREEBSD) + /// FreeBSD does not have 'sudo' installed. + return fmt::format("su -m root -c '{}'", command); +#else + return fmt::format("sudo {}", command); +#endif +} + + int mainEntryClickHouseInstall(int argc, char ** argv) { try @@ -213,10 +227,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) if (options.count("help")) { - std::cout << "Usage: " - << (getuid() == 0 ? "" : "sudo ") - << argv[0] - << " install [options]\n"; + std::cout << "Usage: " << formatWithSudo(std::string(argv[0]) + " install [options]", getuid() != 0) << '\n'; std::cout << desc << '\n'; return 1; } @@ -323,7 +334,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) catch (const Exception & e) { if (e.code() == ErrorCodes::CANNOT_OPEN_FILE && geteuid() != 0) - std::cerr << "Install must be run as root: sudo ./clickhouse install\n"; + std::cerr << "Install must be run as root: " << formatWithSudo("./clickhouse install") << '\n'; throw; } @@ -833,9 +844,10 @@ int mainEntryClickHouseInstall(int argc, char ** argv) fmt::print( "\nClickHouse has been successfully installed.\n" "\nRestart clickhouse-server with:\n" - " sudo clickhouse restart\n" + " {}\n" "\nStart clickhouse-client with:\n" " clickhouse-client{}\n\n", + formatWithSudo("clickhouse restart"), maybe_password); } else @@ -843,9 +855,10 @@ int mainEntryClickHouseInstall(int argc, char ** argv) fmt::print( "\nClickHouse has been successfully installed.\n" "\nStart clickhouse-server with:\n" - " sudo clickhouse start\n" + " {}\n" "\nStart clickhouse-client with:\n" " clickhouse-client{}\n\n", + formatWithSudo("clickhouse start"), maybe_password); } } @@ -854,7 +867,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) std::cerr << getCurrentExceptionMessage(false) << '\n'; if (getuid() != 0) - std::cerr << "\nRun with sudo.\n"; + std::cerr << "\nRun with " << formatWithSudo("...") << "\n"; return getCurrentExceptionCode(); } @@ -910,6 +923,9 @@ namespace if (!user.empty()) { +#if defined(OS_FREEBSD) + command = fmt::format("su -m '{}' -c '{}', user, command); +#else bool may_need_sudo = geteuid() != 0; if (may_need_sudo) { @@ -920,12 +936,9 @@ namespace } else { -#if defined(OS_FREEBSD) - command = fmt::format("su -m '{}' -c '{}', user, command); -#else command = fmt::format("su -s /bin/sh '{}' -c '{}'", user, command); -#endif } +#endif } fmt::print("Will run {}\n", command); @@ -1129,10 +1142,7 @@ int mainEntryClickHouseStart(int argc, char ** argv) if (options.count("help")) { - std::cout << "Usage: " - << (getuid() == 0 ? "" : "sudo ") - << argv[0] - << " start\n"; + std::cout << "Usage: " << formatWithSudo(std::string(argv[0]) + " start", getuid() != 0) << '\n'; return 1; } @@ -1170,10 +1180,7 @@ int mainEntryClickHouseStop(int argc, char ** argv) if (options.count("help")) { - std::cout << "Usage: " - << (getuid() == 0 ? "" : "sudo ") - << argv[0] - << " stop\n"; + std::cout << "Usage: " << formatWithSudo(std::string(argv[0]) + " stop", getuid() != 0) << '\n'; return 1; } @@ -1206,10 +1213,7 @@ int mainEntryClickHouseStatus(int argc, char ** argv) if (options.count("help")) { - std::cout << "Usage: " - << (getuid() == 0 ? "" : "sudo ") - << argv[0] - << " status\n"; + std::cout << "Usage: " << formatWithSudo(std::string(argv[0]) + " status", getuid() != 0) << '\n'; return 1; } @@ -1248,10 +1252,7 @@ int mainEntryClickHouseRestart(int argc, char ** argv) if (options.count("help")) { - std::cout << "Usage: " - << (getuid() == 0 ? "" : "sudo ") - << argv[0] - << " restart\n"; + std::cout << "Usage: " << formatWithSudo(std::string(argv[0]) + " restart", getuid() != 0) << '\n'; return 1; } From cc3fdb8318491452548dc127354c99a56f08bc41 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 6 Jan 2022 09:27:33 +0800 Subject: [PATCH 0754/1260] update 01039_test_setting_parse.reference --- tests/queries/0_stateless/01039_test_setting_parse.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01039_test_setting_parse.reference b/tests/queries/0_stateless/01039_test_setting_parse.reference index 49233946390..199b64e7f4d 100644 --- a/tests/queries/0_stateless/01039_test_setting_parse.reference +++ b/tests/queries/0_stateless/01039_test_setting_parse.reference @@ -1,9 +1,9 @@ 1000000000 3221225472 1567000 -125952 +1263616 1567000 -125952 +1263616 12000000 32505856 1000000000000 From be2cbbcccd168c9a8b2f9d2c2c07dbc9055579c3 Mon Sep 17 00:00:00 2001 From: "Chun-Sheng, Li" Date: Thu, 6 Jan 2022 09:59:03 +0800 Subject: [PATCH 0755/1260] Add postgresql_port config parameter description --- .../server-configuration-parameters/settings.md | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 3c643a447a3..78f6c71c65f 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1288,6 +1288,20 @@ Example 9004 ``` +## postgresql_port {#server_configuration_parameters-postgresql_port} + +Port for communicating with clients over PostgreSQL protocol. + +**Possible values** + +Positive integer. + +Example + +``` xml +9005 +``` + ## tmp_path {#tmp-path} Path to temporary data for processing large queries. From 1153228f3b0797b113d463ba97ac6317eef34556 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jan 2022 05:16:09 +0300 Subject: [PATCH 0756/1260] Maybe improve robustness of the "query profiler" test --- tests/queries/0_stateless/00974_query_profiler.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00974_query_profiler.sql b/tests/queries/0_stateless/00974_query_profiler.sql index 45ba6504a79..24e4241b813 100644 --- a/tests/queries/0_stateless/00974_query_profiler.sql +++ b/tests/queries/0_stateless/00974_query_profiler.sql @@ -15,7 +15,7 @@ SELECT count() > 0 FROM system.trace_log t WHERE query_id = (SELECT query_id FRO SET query_profiler_real_time_period_ns = 0; SET query_profiler_cpu_time_period_ns = 1000000; SET log_queries = 1; -SELECT count(), ignore('test cpu time query profiler') FROM numbers(1000000000); +SELECT count(), ignore('test cpu time query profiler') FROM numbers_mt(10000000000); SET log_queries = 0; SYSTEM FLUSH LOGS; From f41f7b6b92fb07cb5afe196384eb63ef945ae958 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jan 2022 05:21:03 +0300 Subject: [PATCH 0757/1260] Fix error --- programs/install/Install.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index f2aeeed98ad..dd93e0b49ab 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -924,7 +924,7 @@ namespace if (!user.empty()) { #if defined(OS_FREEBSD) - command = fmt::format("su -m '{}' -c '{}', user, command); + command = fmt::format("su -m '{}' -c '{}'", user, command); #else bool may_need_sudo = geteuid() != 0; if (may_need_sudo) From ede65e3b6ef8bc46a6c4d34566d83fee0bc7eebb Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 6 Jan 2022 16:12:59 +0800 Subject: [PATCH 0758/1260] add some building options --- ...StorageSystemBuildOptions.generated.cpp.in | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index 91b521ef235..9435bdcc65b 100644 --- a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -50,6 +50,27 @@ const char * auto_config_build[] "USE_KRB5", "@USE_KRB5@", "USE_FILELOG", "@USE_FILELOG@", "USE_BZIP2", "@USE_BZIP2@", + "USE_AMQPCPP", "@USE_AMQPCPP@", + "USE_ROCKSDB", "@USE_ROCKSDB@", + "USE_NURAFT", "@USE_NURAFT@", + "USE_NLP", "@USE_NLP@", + "USE_SQLITE", "@USE_SQLITE@", + "USE_INTERNAL_LLVM_LIBRARY", "@USE_INTERNAL_LLVM_LIBRARY@", + "USE_OPENCL", "@USE_OPENCL@", + "USE_LIBPQXX", "@USE_LIBPQXX@", + "USE_AZURE_BLOB_STORAGE", "@USE_AZURE_BLOB_STORAGE@", + "USE_INTERNAL_SSL_LIBRARY", "@USE_INTERNAL_SSL_LIBRARY@", + "USE_AWS_S3", "@USE_AWS_S3@", + "USE_CASSANDRA", "@USE_CASSANDRA@", + "USE_YAML_CPP", "@USE_YAML_CPP@", + "USE_INTERNAL_HDFS3_LIBRARY", "@USE_INTERNAL_HDFS3_LIBRARY@", + "CLICKHOUSE_SPLIT_BINARY", "@CLICKHOUSE_SPLIT_BINARY@", + "USE_SENTRY", "@USE_SENTRY@", + "USE_DATASKETCHES", "@USE_DATASKETCHES@", + "USE_AVRO", "@USE_AVRO@", + "USE_ARROW", "@USE_ARROW@", + "USE_ORC", "@USE_ORC@", + "USE_MSGPACK", "@USE_MSGPACK@", "GIT_HASH", "@GIT_HASH@", "GIT_BRANCH", R"IRjaNsZIL9Yh7FQ4(@GIT_BRANCH@)IRjaNsZIL9Yh7FQ4", "GIT_DATE", "@GIT_DATE@", From a52e6013b5bc61ee4fe6a16e3ceba7929e6acf67 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 6 Jan 2022 13:48:16 +0100 Subject: [PATCH 0759/1260] Fix wrong formatting for kerberos tag --- tests/integration/runner | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/runner b/tests/integration/runner index 89001918730..3687ca4068c 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -239,7 +239,7 @@ if __name__ == "__main__": elif image == "clickhouse/integration-test": env_tags += "-e {}={} ".format("DOCKER_BASE_TAG", tag) elif image == "clickhouse/kerberos-kdc": - env_tags += "-e {}={}".format("DOCKER_KERBEROS_KDC_TAG", tag) + env_tags += "-e {}={} ".format("DOCKER_KERBEROS_KDC_TAG", tag) else: logging.info("Unknown image %s" % (image)) From 90a61b84e532b61e31b0c0fa3110797497347b2f Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 6 Jan 2022 12:48:38 +0000 Subject: [PATCH 0760/1260] fix misleading log message --- src/Storages/WindowView/StorageWindowView.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 25ecc0e16ef..73909b152e5 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -877,12 +877,12 @@ void StorageWindowView::threadFuncFireEvent() std::unique_lock lock(fire_signal_mutex); while (!shutdown_called) { - LOG_TRACE(log, "Fire events: {}", fire_signal.size()); - bool signaled = std::cv_status::no_timeout == fire_signal_condition.wait_for(lock, std::chrono::seconds(5)); if (!signaled) continue; + LOG_TRACE(log, "Fire events: {}", fire_signal.size()); + while (!fire_signal.empty()) { fire(fire_signal.front()); From f919bb8664fc03e333eea26e56abd237d163931e Mon Sep 17 00:00:00 2001 From: usurai Date: Fri, 7 Jan 2022 00:11:54 +0800 Subject: [PATCH 0761/1260] Move dataName.cpp to dateName.h --- src/Functions/{dateName.cpp => dateName.h} | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) rename src/Functions/{dateName.cpp => dateName.h} (98%) diff --git a/src/Functions/dateName.cpp b/src/Functions/dateName.h similarity index 98% rename from src/Functions/dateName.cpp rename to src/Functions/dateName.h index c89a7f80dfd..533385f632a 100644 --- a/src/Functions/dateName.cpp +++ b/src/Functions/dateName.h @@ -1,3 +1,4 @@ +#pragma once #include #include @@ -65,7 +66,7 @@ public: bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + virtual DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.size() != 2 && arguments.size() != 3) throw Exception( @@ -100,7 +101,7 @@ public: return std::make_shared(); } - ColumnPtr executeImpl( + virtual ColumnPtr executeImpl( const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, [[maybe_unused]] size_t input_rows_count) const override @@ -112,7 +113,7 @@ public: || (res = executeType(arguments, result_type)))) throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of function {], must be Date or DateTime.", + "Illegal column {} of function {}, must be Date or DateTime.", arguments[1].column->getName(), getName()); @@ -338,7 +339,6 @@ private: else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid date part {} for function {}", date_part, getName()); } - }; } From c0892f6e6cbb0240fb04edd32b1ba3dd6bbaeca9 Mon Sep 17 00:00:00 2001 From: usurai Date: Fri, 7 Jan 2022 00:32:38 +0800 Subject: [PATCH 0762/1260] Add monthName --- src/Functions/monthName.cpp | 83 +++++++++++++++++++++ src/Functions/registerFunctionsDateTime.cpp | 2 + 2 files changed, 85 insertions(+) create mode 100644 src/Functions/monthName.cpp diff --git a/src/Functions/monthName.cpp b/src/Functions/monthName.cpp new file mode 100644 index 00000000000..43f5ba702f7 --- /dev/null +++ b/src/Functions/monthName.cpp @@ -0,0 +1,83 @@ +#include + +namespace DB +{ +class FunctionMonthNameImpl : public FunctionDateNameImpl +{ +public: + static constexpr auto name = "monthName"; + + static constexpr auto month_str = "month"; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; } + + bool isVariadic() const override { return false; } + size_t getNumberOfArguments() const override { return 1; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() != 1) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}", + getName(), + toString(arguments.size())); + + WhichDataType first_argument_type(arguments[0].type); + + if (!(first_argument_type.isDate() || first_argument_type.isDateTime() || first_argument_type.isDateTime64())) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of 2 argument of function {}. Must be a date or a date with time", + arguments[1].type->getName(), + getName()); + + return std::make_shared(); + } + + template + ColumnPtr transformAndExecuteType(const ColumnsWithTypeAndName &arguments, const DataTypePtr & result_type) const + { + auto * times = checkAndGetColumn(arguments[0].column.get()); + if (!times) + return nullptr; + + auto month_column = DataTypeString().createColumnConst(times->getData().size(), month_str); + + ColumnsWithTypeAndName temporary_columns + { + ColumnWithTypeAndName(month_column, std::make_shared(), ""), + arguments[0] + }; + + return this->executeType(temporary_columns, result_type); + } + + ColumnPtr executeImpl( + const ColumnsWithTypeAndName & arguments, + const DataTypePtr & result_type, + [[maybe_unused]] size_t input_rows_count) const override + { + ColumnPtr res; + + if (!((res = transformAndExecuteType(arguments, result_type)) + || (res = transformAndExecuteType(arguments, result_type)) + || (res = transformAndExecuteType(arguments, result_type)))) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of function {], must be Date or DateTime.", + arguments[1].column->getName(), + getName()); + + return res; + } +}; + +void registerFunctionMonthName(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/registerFunctionsDateTime.cpp b/src/Functions/registerFunctionsDateTime.cpp index abbc52c8360..5211a62ff1e 100644 --- a/src/Functions/registerFunctionsDateTime.cpp +++ b/src/Functions/registerFunctionsDateTime.cpp @@ -65,6 +65,7 @@ void registerFunctionSubtractQuarters(FunctionFactory &); void registerFunctionSubtractYears(FunctionFactory &); void registerFunctionDateDiff(FunctionFactory &); void registerFunctionDateName(FunctionFactory &); +void registerFunctionMonthName(FunctionFactory &); void registerFunctionToTimeZone(FunctionFactory &); void registerFunctionFormatDateTime(FunctionFactory &); void registerFunctionFromModifiedJulianDay(FunctionFactory &); @@ -136,6 +137,7 @@ void registerFunctionsDateTime(FunctionFactory & factory) registerFunctionSubtractYears(factory); registerFunctionDateDiff(factory); registerFunctionDateName(factory); + registerFunctionMonthName(factory); registerFunctionToTimeZone(factory); registerFunctionFormatDateTime(factory); registerFunctionFromModifiedJulianDay(factory); From b9e69296daa3392a0bfdd5c1ade0fcf9e56f3a76 Mon Sep 17 00:00:00 2001 From: usurai Date: Fri, 7 Jan 2022 00:33:01 +0800 Subject: [PATCH 0763/1260] Add test for monthName --- .../0_stateless/02160_monthname.reference | 12 ++++ tests/queries/0_stateless/02160_monthname.sql | 72 +++++++++++++++++++ 2 files changed, 84 insertions(+) create mode 100644 tests/queries/0_stateless/02160_monthname.reference create mode 100644 tests/queries/0_stateless/02160_monthname.sql diff --git a/tests/queries/0_stateless/02160_monthname.reference b/tests/queries/0_stateless/02160_monthname.reference new file mode 100644 index 00000000000..a3386cb33c7 --- /dev/null +++ b/tests/queries/0_stateless/02160_monthname.reference @@ -0,0 +1,12 @@ +January January January +February February February +March March March +April April April +May May May +June June June +July July July +August August August +September September September +October October October +November November November +December December December diff --git a/tests/queries/0_stateless/02160_monthname.sql b/tests/queries/0_stateless/02160_monthname.sql new file mode 100644 index 00000000000..57cdab6f7ac --- /dev/null +++ b/tests/queries/0_stateless/02160_monthname.sql @@ -0,0 +1,72 @@ +WITH + toDate('2021-01-14') AS date_value, + toDateTime('2021-01-14 11:22:33') AS date_time_value, + toDateTime64('2021-01-14 11:22:33', 3) AS date_time_64_value +SELECT monthName(date_value), monthName(date_time_value), monthName(date_time_64_value); + +WITH + toDate('2021-02-14') AS date_value, + toDateTime('2021-02-14 11:22:33') AS date_time_value, + toDateTime64('2021-02-14 11:22:33', 3) AS date_time_64_value +SELECT monthName(date_value), monthName(date_time_value), monthName(date_time_64_value); + +WITH + toDate('2021-03-14') AS date_value, + toDateTime('2021-03-14 11:22:33') AS date_time_value, + toDateTime64('2021-03-14 11:22:33', 3) AS date_time_64_value +SELECT monthName(date_value), monthName(date_time_value), monthName(date_time_64_value); + +WITH + toDate('2021-04-14') AS date_value, + toDateTime('2021-04-14 11:22:33') AS date_time_value, + toDateTime64('2021-04-14 11:22:33', 3) AS date_time_64_value +SELECT monthName(date_value), monthName(date_time_value), monthName(date_time_64_value); + +WITH + toDate('2021-05-14') AS date_value, + toDateTime('2021-05-14 11:22:33') AS date_time_value, + toDateTime64('2021-05-14 11:22:33', 3) AS date_time_64_value +SELECT monthName(date_value), monthName(date_time_value), monthName(date_time_64_value); + +WITH + toDate('2021-06-14') AS date_value, + toDateTime('2021-06-14 11:22:33') AS date_time_value, + toDateTime64('2021-06-14 11:22:33', 3) AS date_time_64_value +SELECT monthName(date_value), monthName(date_time_value), monthName(date_time_64_value); + +WITH + toDate('2021-07-14') AS date_value, + toDateTime('2021-07-14 11:22:33') AS date_time_value, + toDateTime64('2021-07-14 11:22:33', 3) AS date_time_64_value +SELECT monthName(date_value), monthName(date_time_value), monthName(date_time_64_value); + +WITH + toDate('2021-08-14') AS date_value, + toDateTime('2021-08-14 11:22:33') AS date_time_value, + toDateTime64('2021-08-14 11:22:33', 3) AS date_time_64_value +SELECT monthName(date_value), monthName(date_time_value), monthName(date_time_64_value); + +WITH + toDate('2021-09-14') AS date_value, + toDateTime('2021-09-14 11:22:33') AS date_time_value, + toDateTime64('2021-09-14 11:22:33', 3) AS date_time_64_value +SELECT monthName(date_value), monthName(date_time_value), monthName(date_time_64_value); + +WITH + toDate('2021-10-14') AS date_value, + toDateTime('2021-10-14 11:22:33') AS date_time_value, + toDateTime64('2021-10-14 11:22:33', 3) AS date_time_64_value +SELECT monthName(date_value), monthName(date_time_value), monthName(date_time_64_value); + +WITH + toDate('2021-11-14') AS date_value, + toDateTime('2021-11-14 11:22:33') AS date_time_value, + toDateTime64('2021-11-14 11:22:33', 3) AS date_time_64_value +SELECT monthName(date_value), monthName(date_time_value), monthName(date_time_64_value); + +WITH + toDate('2021-12-14') AS date_value, + toDateTime('2021-12-14 11:22:33') AS date_time_value, + toDateTime64('2021-12-14 11:22:33', 3) AS date_time_64_value +SELECT monthName(date_value), monthName(date_time_value), monthName(date_time_64_value); + From 984c98cfcdae950dfe53d7567c145a2781d3899b Mon Sep 17 00:00:00 2001 From: usurai Date: Fri, 7 Jan 2022 01:03:17 +0800 Subject: [PATCH 0764/1260] empty commit From d853b0cff9b2a03510cc65a1c3d5f6611bee93c4 Mon Sep 17 00:00:00 2001 From: DimaAmega Date: Thu, 6 Jan 2022 22:06:56 +0300 Subject: [PATCH 0765/1260] fix load function: alert if request failed --- utils/tests-visualizer/index.html | 29 ++++++++++++++--------------- 1 file changed, 14 insertions(+), 15 deletions(-) diff --git a/utils/tests-visualizer/index.html b/utils/tests-visualizer/index.html index a15b09ea58e..8a8687c7a1f 100644 --- a/utils/tests-visualizer/index.html +++ b/utils/tests-visualizer/index.html @@ -40,17 +40,16 @@ let query = ` FORMAT TSV`; function load(query, callback) { - const xhr = new XMLHttpRequest; - xhr.open('POST', "https://play-ci.clickhouse.com/?user=play&add_http_cors_header=1", true); - - xhr.onreadystatechange = function() - { - if (this.readyState === XMLHttpRequest.DONE && this.status == 200) { - callback(this.response); - } - } - - xhr.send(query); + fetch( + "https://play-ci.clickhouse.com?user=play&add_http_cors_header=1", + { method: "POST", body: query } + ) + .then((response) => { + if (!response.ok) throw new Error(`Data download failed\nHTTP status ${response.status}`); + return response.json(); + }) + .then((json) => callback(json)) + .catch(alert); } load(query, renderResponse); @@ -58,8 +57,8 @@ load(query, renderResponse); let data; let canvas = document.getElementById('canvas'); -function renderResponse(response) { - data = JSON.parse(response); +function renderResponse(response_json) { + data = response_json; const last_pixel = data[data.length - 1]; canvas.width = last_pixel[0] + 1; @@ -100,9 +99,9 @@ let test_names; load(test_names_query, saveTestNames); -function saveTestNames(response) +function saveTestNames(response_json) { - test_names = JSON.parse(response).data; + test_names = response_json.data; } canvas.addEventListener('mousemove', event => { From 2cdd8d321fcb00ea1d6b67afb7d9b69b371d2f65 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jan 2022 22:07:12 +0300 Subject: [PATCH 0766/1260] Fix error --- .../dotnet.reference | Bin 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/integration/{test_mysql_protocol => test_dotnet_client}/dotnet.reference (100%) diff --git a/tests/integration/test_mysql_protocol/dotnet.reference b/tests/integration/test_dotnet_client/dotnet.reference similarity index 100% rename from tests/integration/test_mysql_protocol/dotnet.reference rename to tests/integration/test_dotnet_client/dotnet.reference From c1a44ca25a79fdba4e323ebbbc176e31aea4edfd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jan 2022 22:56:32 +0300 Subject: [PATCH 0767/1260] Fix quick start for FreeBSD --- docs/_includes/install/freebsd.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/_includes/install/freebsd.sh b/docs/_includes/install/freebsd.sh index 50e3bc02cb7..2a715a1795f 100644 --- a/docs/_includes/install/freebsd.sh +++ b/docs/_includes/install/freebsd.sh @@ -1,3 +1,3 @@ -wget 'https://builds.clickhouse.com/master/freebsd/clickhouse' +fetch 'https://builds.clickhouse.com/master/freebsd/clickhouse' chmod a+x ./clickhouse -sudo ./clickhouse install +su -m root -c './clickhouse install' From 2495b7c913b7cdfacb2a7999f693f96428dcba0c Mon Sep 17 00:00:00 2001 From: DimaAmega Date: Fri, 7 Jan 2022 00:35:02 +0300 Subject: [PATCH 0768/1260] better index.html --- utils/tests-visualizer/index.html | 216 ++++++++++++++++++++---------- 1 file changed, 144 insertions(+), 72 deletions(-) diff --git a/utils/tests-visualizer/index.html b/utils/tests-visualizer/index.html index 8a8687c7a1f..c5b6ee4b3d8 100644 --- a/utils/tests-visualizer/index.html +++ b/utils/tests-visualizer/index.html @@ -1,16 +1,82 @@ - + + - -

Loading (10 seconds, 20 MB)...

- - + +
+ +

Loading (~10 seconds, try load ~20 MB)

+
+

+
 
 

From 12c3130a6da913e2f8d20ab96d4577c4aeddd6f6 Mon Sep 17 00:00:00 2001
From: Alexey Milovidov 
Date: Fri, 7 Jan 2022 02:03:44 +0300
Subject: [PATCH 0769/1260] Update reference

---
 tests/queries/0_stateless/00974_query_profiler.reference | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/tests/queries/0_stateless/00974_query_profiler.reference b/tests/queries/0_stateless/00974_query_profiler.reference
index e37cf5f7642..708c4988416 100644
--- a/tests/queries/0_stateless/00974_query_profiler.reference
+++ b/tests/queries/0_stateless/00974_query_profiler.reference
@@ -1,4 +1,4 @@
 0	0
 1
-1000000000	0
+10000000000	0
 1

From e849ea4294cb37481f075c698e02d2401cb1065f Mon Sep 17 00:00:00 2001
From: usurai 
Date: Fri, 7 Jan 2022 09:45:27 +0800
Subject: [PATCH 0770/1260] Revert "Add monthName"

This reverts commit c0892f6e6cbb0240fb04edd32b1ba3dd6bbaeca9.
---
 src/Functions/monthName.cpp                 | 83 ---------------------
 src/Functions/registerFunctionsDateTime.cpp |  2 -
 2 files changed, 85 deletions(-)
 delete mode 100644 src/Functions/monthName.cpp

diff --git a/src/Functions/monthName.cpp b/src/Functions/monthName.cpp
deleted file mode 100644
index 43f5ba702f7..00000000000
--- a/src/Functions/monthName.cpp
+++ /dev/null
@@ -1,83 +0,0 @@
-#include 
-
-namespace DB
-{
-class FunctionMonthNameImpl : public FunctionDateNameImpl
-{
-public:
-    static constexpr auto name = "monthName";
-
-    static constexpr auto month_str = "month";
-
-    static FunctionPtr create(ContextPtr) { return std::make_shared(); }
-
-    ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; }
-
-    bool isVariadic() const override { return false; }
-    size_t getNumberOfArguments() const override { return 1; }
-
-    DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
-    {
-        if (arguments.size() != 1)
-            throw Exception(
-                ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
-                "Number of arguments for function {} doesn't match: passed {}",
-                getName(),
-                toString(arguments.size()));
-
-        WhichDataType first_argument_type(arguments[0].type);
-
-        if (!(first_argument_type.isDate() || first_argument_type.isDateTime() || first_argument_type.isDateTime64()))
-            throw Exception(
-                ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
-                "Illegal type {} of 2 argument of function {}. Must be a date or a date with time",
-                arguments[1].type->getName(),
-                getName());
-
-        return std::make_shared();
-    }
-    
-    template 
-    ColumnPtr transformAndExecuteType(const ColumnsWithTypeAndName &arguments, const DataTypePtr & result_type) const
-    {
-        auto * times = checkAndGetColumn(arguments[0].column.get());
-        if (!times)
-            return nullptr;
-
-        auto month_column = DataTypeString().createColumnConst(times->getData().size(), month_str);
-
-        ColumnsWithTypeAndName temporary_columns
-        {
-            ColumnWithTypeAndName(month_column, std::make_shared(), ""),
-            arguments[0]
-        };
-
-        return this->executeType(temporary_columns, result_type);
-    }
-
-    ColumnPtr executeImpl(
-            const ColumnsWithTypeAndName & arguments,
-            const DataTypePtr & result_type,
-            [[maybe_unused]] size_t input_rows_count) const override
-    {
-        ColumnPtr res;
-
-        if (!((res = transformAndExecuteType(arguments, result_type))
-            || (res = transformAndExecuteType(arguments, result_type))
-            || (res = transformAndExecuteType(arguments, result_type))))
-            throw Exception(
-                ErrorCodes::ILLEGAL_COLUMN,
-                "Illegal column {} of function {], must be Date or DateTime.",
-                arguments[1].column->getName(),
-                getName());
-
-        return res;
-    }
-};
-
-void registerFunctionMonthName(FunctionFactory & factory)
-{
-    factory.registerFunction(FunctionFactory::CaseInsensitive);
-}
-
-}
diff --git a/src/Functions/registerFunctionsDateTime.cpp b/src/Functions/registerFunctionsDateTime.cpp
index 5211a62ff1e..abbc52c8360 100644
--- a/src/Functions/registerFunctionsDateTime.cpp
+++ b/src/Functions/registerFunctionsDateTime.cpp
@@ -65,7 +65,6 @@ void registerFunctionSubtractQuarters(FunctionFactory &);
 void registerFunctionSubtractYears(FunctionFactory &);
 void registerFunctionDateDiff(FunctionFactory &);
 void registerFunctionDateName(FunctionFactory &);
-void registerFunctionMonthName(FunctionFactory &);
 void registerFunctionToTimeZone(FunctionFactory &);
 void registerFunctionFormatDateTime(FunctionFactory &);
 void registerFunctionFromModifiedJulianDay(FunctionFactory &);
@@ -137,7 +136,6 @@ void registerFunctionsDateTime(FunctionFactory & factory)
     registerFunctionSubtractYears(factory);
     registerFunctionDateDiff(factory);
     registerFunctionDateName(factory);
-    registerFunctionMonthName(factory);
     registerFunctionToTimeZone(factory);
     registerFunctionFormatDateTime(factory);
     registerFunctionFromModifiedJulianDay(factory);

From aaa45beba1e7653e96f8807af4a5104cd7d613db Mon Sep 17 00:00:00 2001
From: usurai 
Date: Fri, 7 Jan 2022 09:45:34 +0800
Subject: [PATCH 0771/1260] Revert "Move dataName.cpp to dateName.h"

This reverts commit f919bb8664fc03e333eea26e56abd237d163931e.
---
 src/Functions/{dateName.h => dateName.cpp} | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
 rename src/Functions/{dateName.h => dateName.cpp} (98%)

diff --git a/src/Functions/dateName.h b/src/Functions/dateName.cpp
similarity index 98%
rename from src/Functions/dateName.h
rename to src/Functions/dateName.cpp
index 533385f632a..c89a7f80dfd 100644
--- a/src/Functions/dateName.h
+++ b/src/Functions/dateName.cpp
@@ -1,4 +1,3 @@
-#pragma once
 #include 
 
 #include 
@@ -66,7 +65,7 @@ public:
     bool isVariadic() const override { return true; }
     size_t getNumberOfArguments() const override { return 0; }
 
-    virtual DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
+    DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
     {
         if (arguments.size() != 2 && arguments.size() != 3)
             throw Exception(
@@ -101,7 +100,7 @@ public:
         return std::make_shared();
     }
 
-    virtual ColumnPtr executeImpl(
+    ColumnPtr executeImpl(
         const ColumnsWithTypeAndName & arguments,
         const DataTypePtr & result_type,
         [[maybe_unused]] size_t input_rows_count) const override
@@ -113,7 +112,7 @@ public:
             || (res = executeType(arguments, result_type))))
             throw Exception(
                 ErrorCodes::ILLEGAL_COLUMN,
-                "Illegal column {} of function {}, must be Date or DateTime.",
+                "Illegal column {} of function {], must be Date or DateTime.",
                 arguments[1].column->getName(),
                 getName());
 
@@ -339,6 +338,7 @@ private:
         else
             throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid date part {} for function {}", date_part, getName());
     }
+
 };
 
 }

From b7320ee5b15ec4f14d0d96b5fd5cca94db2f1e66 Mon Sep 17 00:00:00 2001
From: Alexey Milovidov 
Date: Fri, 7 Jan 2022 05:16:31 +0300
Subject: [PATCH 0772/1260] Remove some garbage

---
 .../test_dotnet_client/dotnet.reference         | Bin 546 -> 452 bytes
 .../golang.reference                            |   0
 2 files changed, 0 insertions(+), 0 deletions(-)
 rename tests/integration/{test_dotnet_client => test_mysql_protocol}/golang.reference (100%)

diff --git a/tests/integration/test_dotnet_client/dotnet.reference b/tests/integration/test_dotnet_client/dotnet.reference
index 5a04e3f56cc02e359db84266215fdb422a810182..a3d6e1d5ba83740f4c34457b471eebb35961a826 100644
GIT binary patch
literal 452
zcmZ8dO>cuR4DH#!BF7>%V8;oB+&1kv?b6<=N(&#IkQkMKZsOlBK)Y3Qp5ymyONng=
zUYqVZh{L7-@@+K$NDCGur4S%lL4Zs(1hB%z=JbuF`kwk_hk6+IW8ZWi@U(xKeY1hJ
z3sKJG-*vA+N@pxJYH7-XB~hKp9J50D6fyM`SA|EQSj8M`P{za$4t_cNX&CFz_<0)Z
z;|XqR{mq|#ySA+_+WMW25B;amK#Cz)d-*PXUp1X?n{k3I?S$V68wI9#C$?tMSrh?L
zLMov=&jq4e0B93sL(W=(nN!L!19F0tDAX*-IMxpJ{8+3_SoIpiV7$mRQW#Uu#W{p^
VldB5rtnuue^{RH9q02ZFe*he(Xe9su

literal 546
zcmah{!D_=W4DH$AFX$!57V1Q@;>0nGv9aUWrM;EHoF>}>Sqn|N(Z8RZCQx>%BiS!c
zFScZ5zJ0{6`NRDFVo-|L=5X$!&GGc(+p3d73T35WQ4aD}!A;%u>;BXnKBahCzYNoCCdD-lpaS?eO))+TKVY9is?5G5g;?jRfV@;b1IT_Sy>Fe!Iit+WD*bVV^Zd+fr^*e5FzSso?Y6^D~
zp7!{=_ou2k__paUQY>&4`Bh|>0cu)B3o{V;k{QGVb%I4vL;!9uuHGivGHGoDxzpMa
zZw{0ICK^(T&^SsRa7nyhoSM*#5pc0|B}RZ;JPTPctlSxNRcW0yo}9B@*N$>
Date: Fri, 7 Jan 2022 12:22:57 +0800
Subject: [PATCH 0773/1260] update test_hive_query

---
 tests/integration/test_hive_query/test.py | 10 +++++++++-
 1 file changed, 9 insertions(+), 1 deletion(-)

diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py
index 081e0ae3703..8152b158b65 100644
--- a/tests/integration/test_hive_query/test.py
+++ b/tests/integration/test_hive_query/test.py
@@ -106,7 +106,15 @@ def test_parquet_groupby_with_cache(started_cluster):
     assert result == expected_result
 def test_cache_read_bytes(started_cluster):
     node = started_cluster.instances['h0_0_0']
-    time.sleep(10)
+    result = node.query("""
+    SELECT day, count(*) FROM default.demo_parquet group by day order by day
+            """)
+    expected_result = """2021-11-01	1
+2021-11-05	2
+2021-11-11	1
+2021-11-16	2
+"""
+    assert result == expected_result
     result = node.query("select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes)  from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0")
     logging.info("Read bytes from cache:{}".format(result))
     assert result.strip() != '0'

From 76848e7a2a2e3a7f557f107e4e9f39bdff8906a9 Mon Sep 17 00:00:00 2001
From: usurai 
Date: Fri, 7 Jan 2022 13:06:47 +0800
Subject: [PATCH 0774/1260] Use FunctionFactory to invoke dateName

---
 src/Functions/monthName.cpp                 | 77 +++++++++++++++++++++
 src/Functions/registerFunctionsDateTime.cpp |  2 +
 2 files changed, 79 insertions(+)
 create mode 100644 src/Functions/monthName.cpp

diff --git a/src/Functions/monthName.cpp b/src/Functions/monthName.cpp
new file mode 100644
index 00000000000..2d244fc71fc
--- /dev/null
+++ b/src/Functions/monthName.cpp
@@ -0,0 +1,77 @@
+#include 
+#include 
+#include 
+#include 
+
+namespace DB
+{
+namespace ErrorCodes
+{
+    extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
+    extern const int ILLEGAL_TYPE_OF_ARGUMENT;
+}
+
+class FunctionMonthNameImpl : public IFunction
+{
+public:
+    static constexpr auto name = "monthName";
+
+    static constexpr auto month_str = "month";
+
+    static FunctionPtr create(ContextPtr context) { return std::make_shared(context); }
+
+    explicit FunctionMonthNameImpl(ContextPtr context_) : context(context_) {}
+
+    String getName() const override { return name; }
+
+    bool useDefaultImplementationForConstants() const override { return true; }
+
+    bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
+
+    size_t getNumberOfArguments() const override { return 1; }
+
+    DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
+    {
+        if (arguments.size() != 1)
+            throw Exception(
+                ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
+                "Number of arguments for function {} doesn't match: passed {}, should be 1",
+                getName(),
+                toString(arguments.size()));
+
+        WhichDataType argument_type(arguments[0].type);
+        if (!argument_type.isDate() && !argument_type.isDateTime() && !argument_type.isDateTime64())
+            throw Exception(
+                ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
+                "Illegal type of argument of function {}, should be Date, DateTime or DateTime64",
+                getName());
+
+        return std::make_shared();
+    }
+
+    ColumnPtr executeImpl(
+        const ColumnsWithTypeAndName & arguments,
+        const DataTypePtr & result_type,
+        [[maybe_unused]] size_t input_rows_count) const override
+    {
+        auto month_column = DataTypeString().createColumnConst(arguments[0].column->size(), month_str);
+        ColumnsWithTypeAndName temporary_columns
+        {
+            ColumnWithTypeAndName(month_column, std::make_shared(), ""),
+            arguments[0]
+        };
+
+        auto dateName = FunctionFactory::instance().get("dateName", context)->build(temporary_columns);
+        return dateName->execute(temporary_columns, result_type, input_rows_count);
+    }
+
+private:
+    ContextPtr context;
+};
+
+void registerFunctionMonthName(FunctionFactory & factory)
+{
+    factory.registerFunction(FunctionFactory::CaseInsensitive);
+}
+
+}
diff --git a/src/Functions/registerFunctionsDateTime.cpp b/src/Functions/registerFunctionsDateTime.cpp
index abbc52c8360..5211a62ff1e 100644
--- a/src/Functions/registerFunctionsDateTime.cpp
+++ b/src/Functions/registerFunctionsDateTime.cpp
@@ -65,6 +65,7 @@ void registerFunctionSubtractQuarters(FunctionFactory &);
 void registerFunctionSubtractYears(FunctionFactory &);
 void registerFunctionDateDiff(FunctionFactory &);
 void registerFunctionDateName(FunctionFactory &);
+void registerFunctionMonthName(FunctionFactory &);
 void registerFunctionToTimeZone(FunctionFactory &);
 void registerFunctionFormatDateTime(FunctionFactory &);
 void registerFunctionFromModifiedJulianDay(FunctionFactory &);
@@ -136,6 +137,7 @@ void registerFunctionsDateTime(FunctionFactory & factory)
     registerFunctionSubtractYears(factory);
     registerFunctionDateDiff(factory);
     registerFunctionDateName(factory);
+    registerFunctionMonthName(factory);
     registerFunctionToTimeZone(factory);
     registerFunctionFormatDateTime(factory);
     registerFunctionFromModifiedJulianDay(factory);

From 96aa6060c1d2982fd4355fff5c9bf364ea28cca4 Mon Sep 17 00:00:00 2001
From: Alexey Milovidov 
Date: Fri, 7 Jan 2022 08:14:09 +0300
Subject: [PATCH 0775/1260] Improve parser

---
 src/Parsers/ExpressionElementParsers.cpp | 201 +++++++++--------------
 1 file changed, 80 insertions(+), 121 deletions(-)

diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp
index 6ebfb2caf4c..e8b65118ca9 100644
--- a/src/Parsers/ExpressionElementParsers.cpp
+++ b/src/Parsers/ExpressionElementParsers.cpp
@@ -303,18 +303,14 @@ protected:
 
     bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
     {
-        /// Either CAST(expr AS type) or CAST(expr, 'type')
-        /// The latter will be parsed normally as a function later.
+        /// expr AS type
 
         ASTPtr expr_node;
         ASTPtr type_node;
 
-        if (ParserKeyword("CAST").ignore(pos, expected)
-            && ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected)
-            && ParserExpression().parse(pos, expr_node, expected)
+        if (ParserExpression().parse(pos, expr_node, expected)
             && ParserKeyword("AS").ignore(pos, expected)
-            && ParserDataType().parse(pos, type_node, expected)
-            && ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected))
+            && ParserDataType().parse(pos, type_node, expected))
         {
             node = createFunctionCast(expr_node, type_node);
             return true;
@@ -338,13 +334,6 @@ protected:
         ASTPtr start_node;
         ASTPtr length_node;
 
-        if (!ParserKeyword("SUBSTRING").ignore(pos, expected))
-            return false;
-
-        if (pos->type != TokenType::OpeningRoundBracket)
-            return false;
-        ++pos;
-
         if (!ParserExpression().parse(pos, expr_node, expected))
             return false;
 
@@ -361,11 +350,7 @@ protected:
         if (!ParserExpression().parse(pos, start_node, expected))
             return false;
 
-        if (pos->type == TokenType::ClosingRoundBracket)
-        {
-            ++pos;
-        }
-        else
+        if (pos->type != TokenType::ClosingRoundBracket)
         {
             if (pos->type != TokenType::Comma)
             {
@@ -379,8 +364,6 @@ protected:
 
             if (!ParserExpression().parse(pos, length_node, expected))
                 return false;
-
-            ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected);
         }
 
         /// Convert to canonical representation in functional form: SUBSTRING(expr, start, length)
@@ -403,40 +386,29 @@ protected:
 
 class ParserTrimExpression : public IParserBase
 {
-protected:
+public:
+    ParserTrimExpression(bool trim_left_, bool trim_right_)
+        : trim_left(trim_left_), trim_right(trim_right_)
+    {
+    }
+private:
+    bool trim_left = false;
+    bool trim_right = false;
+
     const char * getName() const override { return "TRIM expression"; }
+
     bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
     {
         /// Handles all possible TRIM/LTRIM/RTRIM call variants
 
         std::string func_name;
-        bool trim_left = false;
-        bool trim_right = false;
         bool char_override = false;
         ASTPtr expr_node;
         ASTPtr pattern_node;
         ASTPtr to_remove;
 
-        if (ParserKeyword("LTRIM").ignore(pos, expected))
+        if (!trim_left && !trim_right)
         {
-            if (pos->type != TokenType::OpeningRoundBracket)
-                return false;
-            ++pos;
-            trim_left = true;
-        }
-        else if (ParserKeyword("RTRIM").ignore(pos, expected))
-        {
-            if (pos->type != TokenType::OpeningRoundBracket)
-                return false;
-            ++pos;
-            trim_right = true;
-        }
-        else if (ParserKeyword("TRIM").ignore(pos, expected))
-        {
-            if (pos->type != TokenType::OpeningRoundBracket)
-                return false;
-            ++pos;
-
             if (ParserKeyword("BOTH").ignore(pos, expected))
             {
                 trim_left = true;
@@ -478,16 +450,9 @@ protected:
             }
         }
 
-        if (!(trim_left || trim_right))
-            return false;
-
         if (!ParserExpression().parse(pos, expr_node, expected))
             return false;
 
-        if (pos->type != TokenType::ClosingRoundBracket)
-            return false;
-        ++pos;
-
         /// Convert to regexp replace function call
 
         if (char_override)
@@ -575,13 +540,6 @@ protected:
     const char * getName() const override { return "EXTRACT expression"; }
     bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
     {
-        if (!ParserKeyword("EXTRACT").ignore(pos, expected))
-            return false;
-
-        if (pos->type != TokenType::OpeningRoundBracket)
-            return false;
-        ++pos;
-
         ASTPtr expr;
 
         IntervalKind interval_kind;
@@ -596,10 +554,6 @@ protected:
         if (!elem_parser.parse(pos, expr, expected))
             return false;
 
-        if (pos->type != TokenType::ClosingRoundBracket)
-            return false;
-        ++pos;
-
         auto function = std::make_shared();
         auto exp_list = std::make_shared();
         function->name = interval_kind.toNameOfFunctionExtractTimePart();
@@ -612,30 +566,56 @@ protected:
     }
 };
 
-class ParserDateAddExpression : public IParserBase
+class ParserPositionExpression : public IParserBase
 {
 protected:
+    const char * getName() const override { return "POSITION expression"; }
+    bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
+    {
+        ParserExpression elem_parser;
+
+        ASTPtr in_expr;
+        if (!ParserExpression().parse(pos, in_expr, expected))
+            return false;
+
+        auto * in_func = in_expr->as();
+        if (!in_func || in_func->name != "in")
+            return false;
+
+        auto & arg_list = in_func->arguments->as();
+        if (arg_list.children.size() != 2)
+            return false;
+
+        arg_list.children = {arg_list.children[1], arg_list.children[0]};
+
+        auto function = std::make_shared();
+        auto exp_list = std::make_shared();
+        function->name = "position";
+        function->arguments = in_func->arguments;
+        function->children.push_back(function->arguments);
+        node = function;
+
+        return true;
+    }
+};
+
+class ParserDateAddExpression : public IParserBase
+{
+public:
+    ParserDateAddExpression(const char * function_name_)
+        : function_name(function_name_)
+    {
+    }
+private:
+    const char * function_name;
+
     const char * getName() const override { return "DATE_ADD expression"; }
 
     bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
     {
-        const char * function_name = nullptr;
         ASTPtr timestamp_node;
         ASTPtr offset_node;
 
-        if (ParserKeyword("DATEADD").ignore(pos, expected) || ParserKeyword("DATE_ADD").ignore(pos, expected)
-            || ParserKeyword("TIMESTAMPADD").ignore(pos, expected) || ParserKeyword("TIMESTAMP_ADD").ignore(pos, expected))
-            function_name = "plus";
-        else if (ParserKeyword("DATESUB").ignore(pos, expected) || ParserKeyword("DATE_SUB").ignore(pos, expected)
-            || ParserKeyword("TIMESTAMPSUB").ignore(pos, expected) || ParserKeyword("TIMESTAMP_SUB").ignore(pos, expected))
-            function_name = "minus";
-        else
-            return false;
-
-        if (pos->type != TokenType::OpeningRoundBracket)
-            return false;
-        ++pos;
-
         IntervalKind interval_kind;
         ASTPtr interval_func_node;
         if (parseIntervalKind(pos, expected, interval_kind))
@@ -675,9 +655,6 @@ protected:
             if (!ParserIntervalOperatorExpression{}.parse(pos, interval_func_node, expected))
                 return false;
         }
-        if (pos->type != TokenType::ClosingRoundBracket)
-            return false;
-        ++pos;
 
         auto expr_list_args = std::make_shared();
         expr_list_args->children = {timestamp_node, interval_func_node};
@@ -703,14 +680,6 @@ protected:
         ASTPtr left_node;
         ASTPtr right_node;
 
-        if (!(ParserKeyword("DATEDIFF").ignore(pos, expected) || ParserKeyword("DATE_DIFF").ignore(pos, expected)
-            || ParserKeyword("TIMESTAMPDIFF").ignore(pos, expected) || ParserKeyword("TIMESTAMP_DIFF").ignore(pos, expected)))
-            return false;
-
-        if (pos->type != TokenType::OpeningRoundBracket)
-            return false;
-        ++pos;
-
         IntervalKind interval_kind;
         if (!parseIntervalKind(pos, expected, interval_kind))
             return false;
@@ -729,10 +698,6 @@ protected:
         if (!ParserExpression().parse(pos, right_node, expected))
             return false;
 
-        if (pos->type != TokenType::ClosingRoundBracket)
-            return false;
-        ++pos;
-
         auto expr_list_args = std::make_shared();
         expr_list_args->children = {std::make_shared(interval_kind.toDateDiffUnit()), left_node, right_node};
 
@@ -787,9 +752,29 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
     /// DATE_DIFF, DATEDIFF, TIMESTAMPDIFF, TIMESTAMP_DIFF,
     /// SUBSTRING, TRIM, LEFT, RIGHT, POSITION
 
-    /// CAST(x AS type)
-    /// EXTRACT(interval FROM x)
-    ///
+    String function_name = getIdentifierName(identifier);
+    String function_name_lowercase = Poco::toLower(function_name);
+
+    if (((function_name_lowercase == "cast" && ParserCastAsExpression().parse(pos, node, expected))
+        || (function_name_lowercase == "extract" && ParserExtractExpression().parse(pos, node, expected))
+        || (function_name_lowercase == "substring" && ParserSubstringExpression().parse(pos, node, expected))
+        || (function_name_lowercase == "position" && ParserPositionExpression().parse(pos, node, expected))
+        || (function_name_lowercase == "trim" && ParserTrimExpression(false, false).parse(pos, node, expected))
+        || (function_name_lowercase == "ltrim" && ParserTrimExpression(true, false).parse(pos, node, expected))
+        || (function_name_lowercase == "rtrim" && ParserTrimExpression(false, true).parse(pos, node, expected))
+        || ((function_name_lowercase == "dateadd" || function_name_lowercase == "date_add"
+            || function_name_lowercase == "timestampadd" || function_name_lowercase == "timestamp_add")
+            && ParserDateAddExpression("plus").parse(pos, node, expected))
+        || ((function_name_lowercase == "datesub" || function_name_lowercase == "date_sub"
+            || function_name_lowercase == "timestampsub" || function_name_lowercase == "timestamp_sub")
+            && ParserDateAddExpression("minus").parse(pos, node, expected))
+        || ((function_name_lowercase == "datediff" || function_name_lowercase == "date_diff"
+            || function_name_lowercase == "timestampdiff" || function_name_lowercase == "timestamp_diff")
+            && ParserDateDiffExpression().parse(pos, node, expected)))
+        && ParserToken(TokenType::ClosingRoundBracket).ignore(pos))
+    {
+        return true;
+    }
 
     auto pos_after_bracket = pos;
     auto old_expected = expected;
@@ -832,7 +817,7 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
       * If you do not report that the first option is an error, then the argument will be interpreted as 2014 - 01 - 01 - some number,
       *  and the query silently returns an unexpected result.
       */
-    if (getIdentifierName(identifier) == "toDate"
+    if (function_name == "toDate"
         && contents_end - contents_begin == strlen("2014-01-01")
         && contents_begin[0] >= '2' && contents_begin[0] <= '3'
         && contents_begin[1] >= '0' && contents_begin[1] <= '9'
@@ -849,26 +834,6 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
         throw Exception("Argument of function toDate is unquoted: toDate(" + contents_str + "), must be: toDate('" + contents_str + "')"
             , ErrorCodes::SYNTAX_ERROR);
     }
-    else if (Poco::toLower(getIdentifierName(identifier)) == "position")
-    {
-        /// POSITION(needle IN haystack) is equivalent to function position(haystack, needle)
-        if (const auto * list = expr_list_args->as())
-        {
-            if (list->children.size() == 1)
-            {
-                if (const auto * in_func = list->children[0]->as())
-                {
-                    if (in_func->name == "in")
-                    {
-                        // switch the two arguments
-                        const auto & arg_list = in_func->arguments->as();
-                        if (arg_list.children.size() == 2)
-                            expr_list_args->children = {arg_list.children[1], arg_list.children[0]};
-                    }
-                }
-            }
-        }
-    }
 
     /// The parametric aggregate function has two lists (parameters and arguments) in parentheses. Example: quantile(0.9)(x).
     if (allow_function_parameters && pos->type == TokenType::OpeningRoundBracket)
@@ -2234,12 +2199,6 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
         || ParserArrayOfLiterals().parse(pos, node, expected)
         || ParserArray().parse(pos, node, expected)
         || ParserLiteral().parse(pos, node, expected)
-        || ParserCastAsExpression().parse(pos, node, expected)
-        || ParserExtractExpression().parse(pos, node, expected)
-        || ParserDateAddExpression().parse(pos, node, expected)
-        || ParserDateDiffExpression().parse(pos, node, expected)
-        || ParserSubstringExpression().parse(pos, node, expected)
-        || ParserTrimExpression().parse(pos, node, expected)
         || ParserCase().parse(pos, node, expected)
         || ParserColumnsMatcher().parse(pos, node, expected) /// before ParserFunction because it can be also parsed as a function.
         || ParserFunction().parse(pos, node, expected)

From 4e402baa8443964fda8c746d107721e927e28646 Mon Sep 17 00:00:00 2001
From: tekeri <47343275+tekeri@users.noreply.github.com>
Date: Fri, 7 Jan 2022 14:19:48 +0900
Subject: [PATCH 0776/1260] Fix typo: command_write_timeout

---
 docs/en/sql-reference/functions/index.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/en/sql-reference/functions/index.md b/docs/en/sql-reference/functions/index.md
index 23b45c4fffc..ddc113d31f9 100644
--- a/docs/en/sql-reference/functions/index.md
+++ b/docs/en/sql-reference/functions/index.md
@@ -81,7 +81,7 @@ A function configuration contains the following settings:
 -   `max_command_execution_time` - maximum execution time in seconds for processing block of data. This setting is valid for `executable_pool` commands only. Optional. Default value is `10`.
 -   `command_termination_timeout` - time in seconds during which a command should finish after its pipe is closed. After that time `SIGTERM` is sent to the process executing the command. Optional. Default value is `10`.
 -   `command_read_timeout` - timeout for reading data from command stdout in milliseconds. Default value 10000. Optional parameter.
--   `command_read_timeout` - timeout for writing data to command stdin in milliseconds. Default value 10000. Optional parameter.
+-   `command_write_timeout` - timeout for writing data to command stdin in milliseconds. Default value 10000. Optional parameter.
 -   `pool_size` - the size of a command pool. Optional. Default value is `16`.
 -   `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`.
 -   `execute_direct` - If `execute_direct` = `1`, then `command` will be searched inside user_scripts folder. Additional script arguments can be specified using whitespace separator. Example: `script_name arg1 arg2`. If `execute_direct` = `0`, `command` is passed as argument for `bin/sh -c`. Default value is `1`. Optional parameter.

From 9fd8b99b7c8fe5f1e1e29c754ab4f2626381210d Mon Sep 17 00:00:00 2001
From: usurai 
Date: Fri, 7 Jan 2022 13:23:22 +0800
Subject: [PATCH 0777/1260] Remove maybe_unused

---
 src/Functions/monthName.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/Functions/monthName.cpp b/src/Functions/monthName.cpp
index 2d244fc71fc..bcea4ab3457 100644
--- a/src/Functions/monthName.cpp
+++ b/src/Functions/monthName.cpp
@@ -52,7 +52,7 @@ public:
     ColumnPtr executeImpl(
         const ColumnsWithTypeAndName & arguments,
         const DataTypePtr & result_type,
-        [[maybe_unused]] size_t input_rows_count) const override
+        size_t input_rows_count) const override
     {
         auto month_column = DataTypeString().createColumnConst(arguments[0].column->size(), month_str);
         ColumnsWithTypeAndName temporary_columns

From dfb8f9d4fa76a8fa88f172948c7571c1d48527af Mon Sep 17 00:00:00 2001
From: Alexey Milovidov 
Date: Fri, 7 Jan 2022 08:34:33 +0300
Subject: [PATCH 0778/1260] Simplification

---
 src/Parsers/ExpressionElementParsers.cpp | 107 +++++++----------------
 src/Parsers/ExpressionElementParsers.h   |  11 ---
 2 files changed, 33 insertions(+), 85 deletions(-)

diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp
index e8b65118ca9..0b72c2132b7 100644
--- a/src/Parsers/ExpressionElementParsers.cpp
+++ b/src/Parsers/ExpressionElementParsers.cpp
@@ -367,19 +367,11 @@ protected:
         }
 
         /// Convert to canonical representation in functional form: SUBSTRING(expr, start, length)
-
-        auto expr_list_args = std::make_shared();
-        expr_list_args->children = {expr_node, start_node};
-
         if (length_node)
-            expr_list_args->children.push_back(length_node);
+            node = makeASTFunction("substring", expr_node, start_node, length_node);
+        else
+            node = makeASTFunction("substring", expr_node, start_node);
 
-        auto func_node = std::make_shared();
-        func_node->name = "substring";
-        func_node->arguments = std::move(expr_list_args);
-        func_node->children.push_back(func_node->arguments);
-
-        node = std::move(func_node);
         return true;
     }
 };
@@ -518,18 +510,10 @@ private:
             }
         }
 
-        auto expr_list_args = std::make_shared();
         if (char_override)
-            expr_list_args->children = {expr_node, pattern_node, std::make_shared("")};
+            node = makeASTFunction(func_name, expr_node, pattern_node, std::make_shared(""));
         else
-            expr_list_args->children = {expr_node};
-
-        auto func_node = std::make_shared();
-        func_node->name = func_name;
-        func_node->arguments = std::move(expr_list_args);
-        func_node->children.push_back(func_node->arguments);
-
-        node = std::move(func_node);
+            node = makeASTFunction(func_name, expr_node);
         return true;
     }
 };
@@ -554,14 +538,7 @@ protected:
         if (!elem_parser.parse(pos, expr, expected))
             return false;
 
-        auto function = std::make_shared();
-        auto exp_list = std::make_shared();
-        function->name = interval_kind.toNameOfFunctionExtractTimePart();
-        function->arguments = exp_list;
-        function->children.push_back(exp_list);
-        exp_list->children.push_back(expr);
-        node = function;
-
+        node = makeASTFunction(interval_kind.toNameOfFunctionExtractTimePart(), expr);
         return true;
     }
 };
@@ -586,15 +563,7 @@ protected:
         if (arg_list.children.size() != 2)
             return false;
 
-        arg_list.children = {arg_list.children[1], arg_list.children[0]};
-
-        auto function = std::make_shared();
-        auto exp_list = std::make_shared();
-        function->name = "position";
-        function->arguments = in_func->arguments;
-        function->children.push_back(function->arguments);
-        node = function;
-
+        node = makeASTFunction("position", arg_list.children[1], arg_list.children[0]);
         return true;
     }
 };
@@ -656,16 +625,7 @@ private:
                 return false;
         }
 
-        auto expr_list_args = std::make_shared();
-        expr_list_args->children = {timestamp_node, interval_func_node};
-
-        auto func_node = std::make_shared();
-        func_node->name = function_name;
-        func_node->arguments = std::move(expr_list_args);
-        func_node->children.push_back(func_node->arguments);
-
-        node = std::move(func_node);
-
+        node = makeASTFunction(function_name, timestamp_node, interval_func_node);
         return true;
     }
 };
@@ -698,16 +658,23 @@ protected:
         if (!ParserExpression().parse(pos, right_node, expected))
             return false;
 
-        auto expr_list_args = std::make_shared();
-        expr_list_args->children = {std::make_shared(interval_kind.toDateDiffUnit()), left_node, right_node};
+        node = makeASTFunction("dateDiff", std::make_shared(interval_kind.toDateDiffUnit()), left_node, right_node);
+        return true;
+    }
+};
 
-        auto func_node = std::make_shared();
-        func_node->name = "dateDiff";
-        func_node->arguments = std::move(expr_list_args);
-        func_node->children.push_back(func_node->arguments);
-
-        node = std::move(func_node);
+class ParserExistsExpression : public IParserBase
+{
+protected:
+    const char * getName() const override { return "EXISTS subquery"; }
+    bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
+    {
+        if (!ParserSelectWithUnionQuery().parse(pos, node, expected))
+            return false;
 
+        auto subquery = std::make_shared();
+        subquery->children.push_back(node);
+        node = makeASTFunction("exists", subquery);
         return true;
     }
 };
@@ -718,12 +685,6 @@ protected:
 bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
 {
     ParserIdentifier id_parser;
-    ParserKeyword distinct("DISTINCT");
-    ParserKeyword all("ALL");
-    ParserExpressionList contents(false, is_table_function);
-    ParserSelectWithUnionQuery select;
-    ParserKeyword filter("FILTER");
-    ParserKeyword over("OVER");
 
     bool has_all = false;
     bool has_distinct = false;
@@ -747,7 +708,7 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
     ++pos;
 
     /// Special cases for expressions that look like functions but contain some syntax sugar:
-    /// CAST, EXTRACT,
+    /// CAST, EXTRACT, POSITION, EXISTS
     /// DATE_ADD, DATEADD, TIMESTAMPADD, DATE_SUB, DATESUB, TIMESTAMPSUB,
     /// DATE_DIFF, DATEDIFF, TIMESTAMPDIFF, TIMESTAMP_DIFF,
     /// SUBSTRING, TRIM, LEFT, RIGHT, POSITION
@@ -759,6 +720,7 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
         || (function_name_lowercase == "extract" && ParserExtractExpression().parse(pos, node, expected))
         || (function_name_lowercase == "substring" && ParserSubstringExpression().parse(pos, node, expected))
         || (function_name_lowercase == "position" && ParserPositionExpression().parse(pos, node, expected))
+        || (function_name_lowercase == "exists" && ParserExistsExpression().parse(pos, node, expected))
         || (function_name_lowercase == "trim" && ParserTrimExpression(false, false).parse(pos, node, expected))
         || (function_name_lowercase == "ltrim" && ParserTrimExpression(true, false).parse(pos, node, expected))
         || (function_name_lowercase == "rtrim" && ParserTrimExpression(false, true).parse(pos, node, expected))
@@ -779,6 +741,9 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
     auto pos_after_bracket = pos;
     auto old_expected = expected;
 
+    ParserKeyword all("ALL");
+    ParserKeyword distinct("DISTINCT");
+
     if (all.ignore(pos, expected))
         has_all = true;
 
@@ -803,6 +768,8 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
         }
     }
 
+    ParserExpressionList contents(false, is_table_function);
+
     const char * contents_begin = pos->begin;
     if (!contents.parse(pos, expr_list_args, expected))
         return false;
@@ -897,6 +864,9 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
         function_node->children.push_back(function_node->parameters);
     }
 
+    ParserKeyword filter("FILTER");
+    ParserKeyword over("OVER");
+
     if (filter.ignore(pos, expected))
     {
         // We are slightly breaking the parser interface by parsing the window
@@ -2179,16 +2149,6 @@ bool ParserMySQLGlobalVariable::parseImpl(Pos & pos, ASTPtr & node, Expected & e
     return true;
 }
 
-bool ParserExistsExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
-{
-    if (ParserKeyword("EXISTS").ignore(pos, expected) && ParserSubquery().parse(pos, node, expected))
-    {
-        node = makeASTFunction("exists", node);
-        return true;
-    }
-    return false;
-}
-
 
 bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
 {
@@ -2204,7 +2164,6 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
         || ParserFunction().parse(pos, node, expected)
         || ParserQualifiedAsterisk().parse(pos, node, expected)
         || ParserAsterisk().parse(pos, node, expected)
-        || ParserExistsExpression().parse(pos, node, expected)
         || ParserCompoundIdentifier(false, true).parse(pos, node, expected)
         || ParserSubstitution().parse(pos, node, expected)
         || ParserMySQLGlobalVariable().parse(pos, node, expected);
diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h
index 2d1e24d60a1..c86721dca18 100644
--- a/src/Parsers/ExpressionElementParsers.h
+++ b/src/Parsers/ExpressionElementParsers.h
@@ -276,17 +276,6 @@ protected:
 };
 
 
-/**
-  * Parse query with EXISTS expression.
-  */
-class ParserExistsExpression : public IParserBase
-{
-protected:
-    const char * getName() const override { return "exists expression"; }
-    bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
-};
-
-
 /** An array or tuple of literals.
   * Arrays can also be parsed as an application of [] operator and tuples as an application of 'tuple' function.
   * But parsing the whole array/tuple as a whole constant seriously speeds up the analysis of expressions in the case of very large collection.

From 3787bf72c25338e2e9f21ca165f83daf297210eb Mon Sep 17 00:00:00 2001
From: Alexey Milovidov 
Date: Fri, 7 Jan 2022 09:15:10 +0300
Subject: [PATCH 0779/1260] Fix potential exponential growth of untuple
 function

---
 src/Interpreters/ActionsVisitor.cpp | 4 ++++
 1 file changed, 4 insertions(+)

diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp
index d5eb701e2aa..bc937755618 100644
--- a/src/Interpreters/ActionsVisitor.cpp
+++ b/src/Interpreters/ActionsVisitor.cpp
@@ -697,6 +697,10 @@ ASTs ActionsMatcher::doUntuple(const ASTFunction * function, ActionsMatcher::Dat
     for (const auto & name [[maybe_unused]] : tuple_type->getElementNames())
     {
         auto tuple_ast = function->arguments->children[0];
+
+        /// This transformation can lead to exponential growth of AST size, let's check it.
+        tuple_ast->checkSize(data.getContext()->getSettingsRef().max_ast_elements);
+
         if (tid != 0)
             tuple_ast = tuple_ast->clone();
 

From cc329c3bf904ce2810e8615d4877828ff07c5bd1 Mon Sep 17 00:00:00 2001
From: Alexey Milovidov 
Date: Fri, 7 Jan 2022 09:18:13 +0300
Subject: [PATCH 0780/1260] Add a test

---
 .../02160_untuple_exponential_growth.reference        |  2 ++
 .../0_stateless/02160_untuple_exponential_growth.sh   | 11 +++++++++++
 2 files changed, 13 insertions(+)
 create mode 100644 tests/queries/0_stateless/02160_untuple_exponential_growth.reference
 create mode 100755 tests/queries/0_stateless/02160_untuple_exponential_growth.sh

diff --git a/tests/queries/0_stateless/02160_untuple_exponential_growth.reference b/tests/queries/0_stateless/02160_untuple_exponential_growth.reference
new file mode 100644
index 00000000000..6ed281c757a
--- /dev/null
+++ b/tests/queries/0_stateless/02160_untuple_exponential_growth.reference
@@ -0,0 +1,2 @@
+1
+1
diff --git a/tests/queries/0_stateless/02160_untuple_exponential_growth.sh b/tests/queries/0_stateless/02160_untuple_exponential_growth.sh
new file mode 100755
index 00000000000..9ec6594af69
--- /dev/null
+++ b/tests/queries/0_stateless/02160_untuple_exponential_growth.sh
@@ -0,0 +1,11 @@
+#!/usr/bin/env bash
+
+CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
+# shellcheck source=../shell_config.sh
+. "$CURDIR"/../shell_config.sh
+
+# Should finish in reasonable time (milliseconds).
+# In previous versions this query led to exponential complexity of query analysis.
+
+${CLICKHOUSE_LOCAL} --query "SELECT untuple(tuple(untuple((1, untuple((untuple(tuple(untuple(tuple(untuple((untuple((1, 1, 1, 1)), 1, 1, 1)))))), 1, 1))))))" 2>&1 | grep -cF 'TOO_BIG_AST'
+${CLICKHOUSE_LOCAL} --query "SELECT untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple((1, 1, 1, 1, 1))))))))))))))))))))))))))" 2>&1 | grep -cF 'TOO_BIG_AST'

From 99e6812288ab090988cde3d5b512f2859831f12e Mon Sep 17 00:00:00 2001
From: JackyWoo 
Date: Fri, 7 Jan 2022 15:53:20 +0800
Subject: [PATCH 0781/1260] add tests

---
 .../test_keeper_four_word_command/test.py     |   7 +-
 tests/integration/test_keeper_session/test.py | 107 ++++++++++++++++--
 2 files changed, 102 insertions(+), 12 deletions(-)

diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py
index 7a464a67c81..1887b18655a 100644
--- a/tests/integration/test_keeper_four_word_command/test.py
+++ b/tests/integration/test_keeper_four_word_command/test.py
@@ -475,12 +475,17 @@ def test_cmd_crst(started_cluster):
         print(data)
 
         data = send_4lw_cmd(cmd='cons')
+        print("cons output(after crst) -------------------------------------")
+        print(data)
 
         # 2 connections, 1 for 'cons' command, 1 for zk
         cons = [n for n in data.split('\n') if len(n) > 0]
         assert len(cons) == 2
 
-        conn_stat = re.match(r'(.*?)[:].*[(](.*?)[)].*', cons[0].strip(), re.S).group(2)
+        # connection for zk
+        zk_conn = [n for n in cons if not n.__contains__('sid=0xffffffffffffffff')][0]
+
+        conn_stat = re.match(r'(.*?)[:].*[(](.*?)[)].*', zk_conn.strip(), re.S).group(2)
         assert conn_stat is not None
 
         result = {}
diff --git a/tests/integration/test_keeper_session/test.py b/tests/integration/test_keeper_session/test.py
index 2e552492762..4faccf3e858 100644
--- a/tests/integration/test_keeper_session/test.py
+++ b/tests/integration/test_keeper_session/test.py
@@ -1,11 +1,25 @@
 import pytest
 from helpers.cluster import ClickHouseCluster
 import time
+import socket
+import struct
+
+from kazoo.client import KazooClient
+# from kazoo.protocol.serialization import Connect, read_buffer, write_buffer
 
 cluster = ClickHouseCluster(__file__)
 node1 = cluster.add_instance('node1', main_configs=['configs/keeper_config.xml'], stay_alive=True)
 
-from kazoo.client import KazooClient, KazooState
+bool_struct = struct.Struct('B')
+int_struct = struct.Struct('!i')
+int_int_struct = struct.Struct('!ii')
+int_int_long_struct = struct.Struct('!iiq')
+
+int_long_int_long_struct = struct.Struct('!iqiq')
+long_struct = struct.Struct('!q')
+multiheader_struct = struct.Struct('!iBi')
+reply_header_struct = struct.Struct('!iqi')
+stat_struct = struct.Struct('!qqqqiiiqiiq')
 
 
 @pytest.fixture(scope="module")
@@ -54,18 +68,89 @@ def get_fake_zk(nodename, timeout=30.0):
     return _fake_zk_instance
 
 
-def test_session_timeout(started_cluster):
-    zk = None
+def get_keeper_socket(node_name):
+    hosts = cluster.get_instance_ip(node_name)
+    client = socket.socket()
+    client.settimeout(10)
+    client.connect((hosts, 9181))
+    return client
+
+
+def close_keeper_socket(cli):
+    if cli is not None:
+        cli.close()
+
+
+def write_buffer(bytes):
+    if bytes is None:
+        return int_struct.pack(-1)
+    else:
+        return int_struct.pack(len(bytes)) + bytes
+
+
+def read_buffer(bytes, offset):
+    length = int_struct.unpack_from(bytes, offset)[0]
+    offset += int_struct.size
+    if length < 0:
+        return None, offset
+    else:
+        index = offset
+        offset += length
+        return bytes[index:index + length], offset
+
+
+def handshake(node_name=node1.name, session_timeout=1000, session_id=0):
+    client = None
     try:
-        wait_nodes()
+        client = get_keeper_socket(node_name)
+        protocol_version = 0
+        last_zxid_seen = 0
+        session_passwd = b'\x00' * 16
+        read_only = 0
 
-        zk1 = get_fake_zk(node1.name, timeout=1.0)
-        assert zk1._session_timeout == 5000
+        # Handshake serialize and deserialize code is from 'kazoo.protocol.serialization'.
 
-        zk1 = get_fake_zk(node1.name, timeout=8.0)
-        assert zk1._session_timeout == 8000
+        # serialize handshake
+        req = bytearray()
+        req.extend(int_long_int_long_struct.pack(protocol_version, last_zxid_seen, session_timeout, session_id))
+        req.extend(write_buffer(session_passwd))
+        req.extend([1 if read_only else 0])
+        # add header
+        req = int_struct.pack(45) + req
+        print("handshake request - len:", req.hex(), len(req))
 
-        zk1 = get_fake_zk(node1.name, timeout=20.0)
-        assert zk1._session_timeout == 10000
+        # send request
+        client.send(req)
+        data = client.recv(1_000)
+
+        # deserialize response
+        print("handshake response - len:", data.hex(), len(data))
+        # ignore header
+        offset = 4
+        proto_version, negotiated_timeout, session_id = int_int_long_struct.unpack_from(data, offset)
+        offset += int_int_long_struct.size
+        password, offset = read_buffer(data, offset)
+        try:
+            read_only = bool_struct.unpack_from(data, offset)[0] == 1
+            offset += bool_struct.size
+        except struct.error:
+            read_only = False
+
+        print("negotiated_timeout - session_id", negotiated_timeout, session_id)
+        return negotiated_timeout, session_id
     finally:
-        destroy_zk_client(zk)
+        if client is not None:
+            client.close()
+
+
+def test_session_timeout(started_cluster):
+    wait_nodes()
+
+    negotiated_timeout, _ = handshake(node1.name, session_timeout=1000, session_id=0)
+    assert negotiated_timeout == 5000
+
+    negotiated_timeout, _ = handshake(node1.name, session_timeout=8000, session_id=0)
+    assert negotiated_timeout == 8000
+
+    negotiated_timeout, _ = handshake(node1.name, session_timeout=20000, session_id=0)
+    assert negotiated_timeout == 10000

From 30ef16ab58709ef9522b3d7eb62d74513cc6b4ca Mon Sep 17 00:00:00 2001
From: IlyaTsoi <45252974+IlyaTsoi@users.noreply.github.com>
Date: Fri, 7 Jan 2022 12:02:48 +0300
Subject: [PATCH 0782/1260] Update LDAPClient.cpp

refactoring of escapeForLDAPFilter function
---
 src/Access/LDAPClient.cpp | 10 +++-------
 1 file changed, 3 insertions(+), 7 deletions(-)

diff --git a/src/Access/LDAPClient.cpp b/src/Access/LDAPClient.cpp
index c7081bd233f..6fc4ffb52cd 100644
--- a/src/Access/LDAPClient.cpp
+++ b/src/Access/LDAPClient.cpp
@@ -106,15 +106,11 @@ namespace
             switch (ch)
             {
                 case '(':
-                    dest += "\\28";
-                    break;
                 case ')':
-                    dest += "\\29";
-                    break;
-                default:
-                    dest += ch;
-                    break;
+                    dest += '\\';
+                    break;         
             }
+            dest += ch;
         }
 
         return dest;

From 6a4d5a985b0f3d900f681e9a2259a242b3172f6a Mon Sep 17 00:00:00 2001
From: alesapin 
Date: Fri, 7 Jan 2022 12:05:38 +0300
Subject: [PATCH 0783/1260] Rerun docs release check as well

---
 tests/ci/docs_release.py                      | 9 +++++++--
 tests/ci/workflow_approve_rerun_lambda/app.py | 1 +
 2 files changed, 8 insertions(+), 2 deletions(-)

diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py
index 90588848f12..825bca0b68b 100644
--- a/tests/ci/docs_release.py
+++ b/tests/ci/docs_release.py
@@ -2,6 +2,7 @@
 import logging
 import subprocess
 import os
+import sys
 
 from github import Github
 
@@ -13,6 +14,7 @@ from ssh import SSHKey
 from upload_result_helper import upload_results
 from docker_pull_helper import get_image_with_version
 from commit_status_helper import get_commit
+from rerun_helper import RerunHelper
 
 NAME = "Docs Release (actions)"
 
@@ -22,9 +24,12 @@ if __name__ == "__main__":
     temp_path = TEMP_PATH
     repo_path = REPO_COPY
 
-    pr_info = PRInfo(need_changed_files=True)
-
     gh = Github(get_best_robot_token())
+    pr_info = PRInfo(need_changed_files=True)
+    rerun_helper = RerunHelper(gh, pr_info, NAME)
+    if rerun_helper.is_already_finished_by_status():
+        logging.info("Check is already finished according to github status, exiting")
+        sys.exit(0)
 
     if not os.path.exists(temp_path):
         os.makedirs(temp_path)
diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py
index f2502f605af..d4d642e0ba3 100644
--- a/tests/ci/workflow_approve_rerun_lambda/app.py
+++ b/tests/ci/workflow_approve_rerun_lambda/app.py
@@ -41,6 +41,7 @@ TRUSTED_ORG_IDS = {
 
 NEED_RERUN_WORKFLOWS = {
     13241696, # PR
+    14738810, # DocsRelease
     15834118, # Docs
     15522500, # MasterCI
     15516108, # ReleaseCI

From a6c1bd09355fbe21904a3cf73b901116781fc53c Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?=E6=9E=A2=E6=9C=A8?= 
Date: Fri, 7 Jan 2022 17:22:20 +0800
Subject: [PATCH 0784/1260] Add config to enable ipv4 or ipv6.

---
 contrib/NuRaft                            | 2 +-
 src/Coordination/CoordinationSettings.cpp | 6 ++++++
 src/Coordination/CoordinationSettings.h   | 1 +
 src/Coordination/KeeperDispatcher.cpp     | 2 +-
 src/Coordination/KeeperServer.cpp         | 7 ++++---
 src/Coordination/KeeperServer.h           | 3 ++-
 6 files changed, 15 insertions(+), 6 deletions(-)

diff --git a/contrib/NuRaft b/contrib/NuRaft
index ff100a87131..c2043aa250e 160000
--- a/contrib/NuRaft
+++ b/contrib/NuRaft
@@ -1 +1 @@
-Subproject commit ff100a8713146e1ca4b4158dd6cc4eef9af47fc3
+Subproject commit c2043aa250e53ad5cf75e596e319d587af4dcb3c
diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp
index bb160c98402..22a6bd1d941 100644
--- a/src/Coordination/CoordinationSettings.cpp
+++ b/src/Coordination/CoordinationSettings.cpp
@@ -41,6 +41,7 @@ const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "con
 
 KeeperConfigurationAndSettings::KeeperConfigurationAndSettings()
     : server_id(NOT_EXIST)
+    , enable_ipv6(true)
     , tcp_port(NOT_EXIST)
     , tcp_port_secure(NOT_EXIST)
     , standalone_keeper(false)
@@ -67,6 +68,9 @@ void KeeperConfigurationAndSettings::dump(WriteBufferFromOwnString & buf) const
     writeText("server_id=", buf);
     write_int(server_id);
 
+    writeText("enable_ipv6=", buf);
+    write_bool(enable_ipv6);
+
     if (tcp_port != NOT_EXIST)
     {
         writeText("tcp_port=", buf);
@@ -156,6 +160,8 @@ KeeperConfigurationAndSettings::loadFromConfig(const Poco::Util::AbstractConfigu
     ret->server_id = config.getInt("keeper_server.server_id");
     ret->standalone_keeper = standalone_keeper_;
 
+    ret->enable_ipv6 = config.getBool("keeper_server.enable_ipv6", true);
+
     if (config.has("keeper_server.tcp_port"))
     {
         ret->tcp_port = config.getInt("keeper_server.tcp_port");
diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h
index 9cf566f95df..5546551cf3a 100644
--- a/src/Coordination/CoordinationSettings.h
+++ b/src/Coordination/CoordinationSettings.h
@@ -63,6 +63,7 @@ struct KeeperConfigurationAndSettings
     KeeperConfigurationAndSettings();
     int server_id;
 
+    bool enable_ipv6;
     int tcp_port;
     int tcp_port_secure;
 
diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp
index 438e337b64f..8423f10f3a6 100644
--- a/src/Coordination/KeeperDispatcher.cpp
+++ b/src/Coordination/KeeperDispatcher.cpp
@@ -276,7 +276,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf
     try
     {
         LOG_DEBUG(log, "Waiting server to initialize");
-        server->startup();
+        server->startup(configuration_and_settings->enable_ipv6);
         LOG_DEBUG(log, "Server initialized, waiting for quorum");
 
         if (!start_async)
diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp
index 82ea100bccb..25d57e64e0a 100644
--- a/src/Coordination/KeeperServer.cpp
+++ b/src/Coordination/KeeperServer.cpp
@@ -107,7 +107,7 @@ KeeperServer::KeeperServer(
         LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower.");
 }
 
-void KeeperServer::startup()
+void KeeperServer::startup(bool enable_ipv6)
 {
     state_machine->init();
 
@@ -171,13 +171,14 @@ void KeeperServer::startup()
 #endif
     }
 
-    launchRaftServer(params, asio_opts);
+    launchRaftServer(enable_ipv6, params, asio_opts);
 
     if (!raft_instance)
         throw Exception(ErrorCodes::RAFT_ERROR, "Cannot allocate RAFT instance");
 }
 
 void KeeperServer::launchRaftServer(
+    bool enable_ipv6,
     const nuraft::raft_params & params,
     const nuraft::asio_service::options & asio_opts)
 {
@@ -192,7 +193,7 @@ void KeeperServer::launchRaftServer(
 
     nuraft::ptr logger = nuraft::cs_new("RaftInstance", coordination_settings->raft_logs_level);
     asio_service = nuraft::cs_new(asio_opts, logger);
-    asio_listener = asio_service->create_rpc_listener(state_manager->getPort(), logger);
+    asio_listener = asio_service->create_rpc_listener(state_manager->getPort(), logger, enable_ipv6);
 
     if (!asio_listener)
         return;
diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h
index 376fe111f15..1fb02bb0987 100644
--- a/src/Coordination/KeeperServer.h
+++ b/src/Coordination/KeeperServer.h
@@ -44,6 +44,7 @@ private:
     /// Almost copy-paste from nuraft::launcher, but with separated server init and start
     /// Allows to avoid race conditions.
     void launchRaftServer(
+        bool enable_ipv6,
         const nuraft::raft_params & params,
         const nuraft::asio_service::options & asio_opts);
 
@@ -57,7 +58,7 @@ public:
         SnapshotsQueue & snapshots_queue_);
 
     /// Load state machine from the latest snapshot and load log storage. Start NuRaft with required settings.
-    void startup();
+    void startup(bool enable_ipv6 = true);
 
     /// Put local read request and execute in state machine directly and response into
     /// responses queue

From 4bfe85b2b0f43f9b44cb0345d1c61ff7ebe76cf8 Mon Sep 17 00:00:00 2001
From: DimaAmega 
Date: Fri, 7 Jan 2022 12:26:17 +0300
Subject: [PATCH 0785/1260] better index.html

---
 utils/tests-visualizer/index.html | 48 ++++++++++++++++++++-----------
 1 file changed, 31 insertions(+), 17 deletions(-)

diff --git a/utils/tests-visualizer/index.html b/utils/tests-visualizer/index.html
index c5b6ee4b3d8..52da6e388f6 100644
--- a/utils/tests-visualizer/index.html
+++ b/utils/tests-visualizer/index.html
@@ -8,6 +8,10 @@
             font-family: sans-serif;
         }
 
+        .hide {
+            display: none;
+        }
+
         #loading {
             margin-top: 1em;
         }
@@ -21,7 +25,7 @@
             animation: spin 2s ease-in-out infinite;
         }
 
-        #loading h1 {
+        h1 {
             text-align: center;
         }
 
@@ -33,10 +37,9 @@
             float: right;
             font-size: 16pt;
             padding: 0.5rem;
-            border: 1px solid #111;
+            border: 1px solid white;
             border-radius: 10px;
             box-shadow: white 5px 5px 28px -10px;
-            display: none;
         }
 
         canvas {
@@ -58,11 +61,12 @@
     
 
 
+

+

Data not load

Loading (~10 seconds, try load ~20 MB)

-

 
 
 
 

From ec75d2340454efb4b9ff417ad4d9b353a460f735 Mon Sep 17 00:00:00 2001
From: zhangxiao871 <821008736@qq.com>
Date: Fri, 7 Jan 2022 17:45:30 +0800
Subject: [PATCH 0786/1260] add test

---
 tests/integration/test_keeper_auth/test.py | 24 ++++++++++++++++++++++
 1 file changed, 24 insertions(+)

diff --git a/tests/integration/test_keeper_auth/test.py b/tests/integration/test_keeper_auth/test.py
index 2df08cc94b7..7dc72274465 100644
--- a/tests/integration/test_keeper_auth/test.py
+++ b/tests/integration/test_keeper_auth/test.py
@@ -36,6 +36,30 @@ def get_genuine_zk():
         get_fake_zk
     ]
 )
+
+def test_remove_acl(started_cluster, get_zk):
+    auth_connection = get_zk()
+
+    auth_connection.add_auth('digest', 'user1:password1')
+
+    # Consistent with zookeeper, accept generated digest
+    auth_connection.create("/test_remove_acl1", b"dataX", acl=[make_acl("digest", "user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=", read=True, write=False, create=False, delete=False, admin=False)])
+    auth_connection.create("/test_remove_acl2", b"dataX", acl=[make_acl("digest", "user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=", read=True, write=True, create=False, delete=False, admin=False)])
+    auth_connection.create("/test_remove_acl3", b"dataX", acl=[make_acl("digest", "user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=", all=True)])
+
+    auth_connection.delete("/test_remove_acl2")
+
+    auth_connection.create("/test_remove_acl4", b"dataX", acl=[make_acl("digest", "user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=", read=True, write=True, create=True, delete=False, admin=False)])
+
+    acls, stat = auth_connection.get_acls("/test_remove_acl3")
+
+    assert stat.aversion == 0
+    assert len(acls) == 1
+    for acl in acls:
+        assert acl.acl_list == ['ALL']
+        assert acl.perms == 31
+
+
 def test_digest_auth_basic(started_cluster, get_zk):
     auth_connection = get_zk()
 

From 3c7ad9a8e9e4b77b3463cc8e790400e09c0794b6 Mon Sep 17 00:00:00 2001
From: zhangxiao871 <821008736@qq.com>
Date: Fri, 7 Jan 2022 18:09:23 +0800
Subject: [PATCH 0787/1260] fix test

---
 tests/integration/test_keeper_auth/test.py | 8 ++++++++
 1 file changed, 8 insertions(+)

diff --git a/tests/integration/test_keeper_auth/test.py b/tests/integration/test_keeper_auth/test.py
index 7dc72274465..6be78f95483 100644
--- a/tests/integration/test_keeper_auth/test.py
+++ b/tests/integration/test_keeper_auth/test.py
@@ -60,6 +60,14 @@ def test_remove_acl(started_cluster, get_zk):
         assert acl.perms == 31
 
 
+@pytest.mark.parametrize(
+    ('get_zk'),
+    [
+        get_genuine_zk,
+        get_fake_zk
+    ]
+)
+
 def test_digest_auth_basic(started_cluster, get_zk):
     auth_connection = get_zk()
 

From 783b683cce039196f8b58abe5850360df32d97c6 Mon Sep 17 00:00:00 2001
From: DimaAmega 
Date: Fri, 7 Jan 2022 14:59:07 +0300
Subject: [PATCH 0788/1260] better index.html

---
 utils/tests-visualizer/index.html | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)

diff --git a/utils/tests-visualizer/index.html b/utils/tests-visualizer/index.html
index 52da6e388f6..e4293aaf931 100644
--- a/utils/tests-visualizer/index.html
+++ b/utils/tests-visualizer/index.html
@@ -65,7 +65,7 @@
 

Data not load

-

Loading (~10 seconds, try load ~20 MB)

+

Loading (~10 seconds, ~20 MB)